From 969940b4c9ba13219531796fbaa3573a8d9ffdbc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 19 Aug 2020 16:34:38 +0300 Subject: [PATCH 01/42] write table tructure for table function remote(...) --- src/Databases/DatabaseOnDisk.cpp | 8 ++- src/Interpreters/InterpreterCreateQuery.cpp | 13 ++++- src/Parsers/ASTCreateQuery.cpp | 12 ++++- src/Parsers/ParserCreateQuery.cpp | 7 ++- src/Storages/StorageDistributed.cpp | 36 +++---------- src/Storages/StorageDistributed.h | 21 ++------ src/Storages/StorageProxy.h | 60 +++++++++++++++++++++ src/TableFunctions/ITableFunction.cpp | 3 +- src/TableFunctions/ITableFunction.h | 8 ++- src/TableFunctions/TableFunctionRemote.cpp | 58 ++++++++++++++------ src/TableFunctions/TableFunctionRemote.h | 10 ++++ 11 files changed, 166 insertions(+), 70 deletions(-) create mode 100644 src/Storages/StorageProxy.h diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index b69454e3b83..fdfc185f920 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -54,9 +54,13 @@ std::pair createTableFromAST( if (ast_create_query.as_table_function) { - const auto & table_function = ast_create_query.as_table_function->as(); + const auto & ast_table_function = ast_create_query.as_table_function->as(); const auto & factory = TableFunctionFactory::instance(); - StoragePtr storage = factory.get(table_function.name, context)->execute(ast_create_query.as_table_function, context, ast_create_query.table); + auto table_function = factory.get(ast_table_function.name, context); + ColumnsDescription columns; + if (ast_create_query.columns_list && ast_create_query.columns_list->columns) + columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context, false); + StoragePtr storage = table_function->execute(ast_create_query.as_table_function, context, ast_create_query.table, std::move(columns)); storage->renameInMemory(ast_create_query); return {ast_create_query.table, storage}; } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ef81eb41672..dd5fb4d063a 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -456,6 +456,9 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(AS if (create.columns_list) { + if (create.as_table_function && (create.columns_list->indices || create.columns_list->constraints)) + throw Exception("Indexes and constraints are not supported for table functions", ErrorCodes::INCORRECT_QUERY); + if (create.columns_list->columns) { bool sanity_check_compression_codecs = !create.attach && !context.getSettingsRef().allow_suspicious_codecs; @@ -492,7 +495,13 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(AS properties.columns = ColumnsDescription(as_select_sample.getNamesAndTypesList()); } else if (create.as_table_function) - return {}; + { + /// Table function without columns list. + auto table_function = TableFunctionFactory::instance().get(create.as_table_function->as().name, context); + properties.columns = table_function->getActualTableStructure(create.as_table_function, context); + if (properties.columns.empty()) //FIXME + return {}; + } else throw Exception("Incorrect CREATE query: required list of column descriptions or AS section or SELECT.", ErrorCodes::INCORRECT_QUERY); @@ -761,7 +770,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, { const auto & table_function = create.as_table_function->as(); const auto & factory = TableFunctionFactory::instance(); - res = factory.get(table_function.name, context)->execute(create.as_table_function, context, create.table); + res = factory.get(table_function.name, context)->execute(create.as_table_function, context, create.table, properties.columns); res->renameInMemory({create.database, create.table, create.uuid}); } else diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index 73903e28f84..8c5681c3a9c 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -282,13 +282,23 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat if (as_table_function) { + if (columns_list) + { + frame.expression_list_always_start_on_new_line = true; + settings.ostr << (settings.one_line ? " (" : "\n("); + FormatStateStacked frame_nested = frame; + columns_list->formatImpl(settings, state, frame_nested); + settings.ostr << (settings.one_line ? ")" : "\n)"); + frame.expression_list_always_start_on_new_line = false; + } + settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : ""); as_table_function->formatImpl(settings, state, frame); } frame.expression_list_always_start_on_new_line = true; - if (columns_list) + if (columns_list && !as_table_function) { settings.ostr << (settings.one_line ? " (" : "\n("); FormatStateStacked frame_nested = frame; diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 55208ca4133..fb0fd20b4ed 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -416,7 +416,12 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe return false; if (!storage_p.parse(pos, storage, expected) && !is_temporary) - return false; + { + if (!s_as.ignore(pos, expected)) + return false; + if (!table_function_p.parse(pos, as_table_function, expected)) + return false; + } } else { diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index f536c6ee763..fe756e613ac 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -292,12 +292,14 @@ StorageDistributed::StorageDistributed( const ASTPtr & sharding_key_, const String & storage_policy_name_, const String & relative_data_path_, - bool attach_) + bool attach_, + ClusterPtr owned_cluster_) : IStorage(id_) , remote_database(remote_database_) , remote_table(remote_table_) , global_context(std::make_unique(context_)) , log(&Poco::Logger::get("StorageDistributed (" + id_.table_name + ")")) + , owned_cluster(std::move(owned_cluster_)) , cluster_name(global_context->getMacros()->expand(cluster_name_)) , has_sharding_key(sharding_key_) , relative_data_path(relative_data_path_) @@ -341,40 +343,14 @@ StorageDistributed::StorageDistributed( const ASTPtr & sharding_key_, const String & storage_policy_name_, const String & relative_data_path_, - bool attach) - : StorageDistributed(id_, columns_, constraints_, String{}, String{}, cluster_name_, context_, sharding_key_, storage_policy_name_, relative_data_path_, attach) + bool attach, + ClusterPtr owned_cluster_) + : StorageDistributed(id_, columns_, constraints_, String{}, String{}, cluster_name_, context_, sharding_key_, storage_policy_name_, relative_data_path_, attach, std::move(owned_cluster_)) { remote_table_function_ptr = std::move(remote_table_function_ptr_); } -StoragePtr StorageDistributed::createWithOwnCluster( - const StorageID & table_id_, - const ColumnsDescription & columns_, - const String & remote_database_, /// database on remote servers. - const String & remote_table_, /// The name of the table on the remote servers. - ClusterPtr owned_cluster_, - const Context & context_) -{ - auto res = create(table_id_, columns_, ConstraintsDescription{}, remote_database_, remote_table_, String{}, context_, ASTPtr(), String(), String(), false); - res->owned_cluster = std::move(owned_cluster_); - return res; -} - - -StoragePtr StorageDistributed::createWithOwnCluster( - const StorageID & table_id_, - const ColumnsDescription & columns_, - ASTPtr & remote_table_function_ptr_, - ClusterPtr & owned_cluster_, - const Context & context_) -{ - auto res = create(table_id_, columns_, ConstraintsDescription{}, remote_table_function_ptr_, String{}, context_, ASTPtr(), String(), String(), false); - res->owned_cluster = owned_cluster_; - return res; -} - - bool StorageDistributed::canForceGroupByNoMerge(const Context &context, QueryProcessingStage::Enum to_stage, const ASTPtr & query_ptr) const { const auto & settings = context.getSettingsRef(); diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index b7663676d52..24c07d7e7b7 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -42,21 +42,6 @@ class StorageDistributed final : public ext::shared_ptr_helper +#include + +namespace DB +{ + +template +class StorageTableFunction : public StorageT +{ +public: + using GetStructureFunc = std::function; + + template + StorageTableFunction(GetStructureFunc get_structure_, StorageArgs && ... args) + : StorageT(std::forward(args)...), get_structure(std::move(get_structure_)) + { + } + + Pipe read( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) + { + assertBlocksHaveEqualStructure(); + return StorageT::read(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); + } + + BlockOutputStreamPtr write( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + const Context & context) + { + assertBlocksHaveEqualStructure(); + return StorageT::write(query, metadata_snapshot, context); + } + +private: + void assertSourceStructure() + { + if (!get_structure) + return; + + StorageInMemoryMetadata source_metadata; + source_metadata.setColumns(get_structure()); + actual_source_structure = source_metadata.getSampleBlock(); + assertBlocksHaveEqualStructure(getInMemoryMetadataPtr()->getSampleBlock(), actual_source_structure); + + get_structure = {}; + } + + GetStructureFunc get_structure; + Block actual_source_structure; +}; + +} diff --git a/src/TableFunctions/ITableFunction.cpp b/src/TableFunctions/ITableFunction.cpp index 6a784c062da..678f1921385 100644 --- a/src/TableFunctions/ITableFunction.cpp +++ b/src/TableFunctions/ITableFunction.cpp @@ -13,10 +13,11 @@ namespace ProfileEvents namespace DB { -StoragePtr ITableFunction::execute(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +StoragePtr ITableFunction::execute(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns_) const { ProfileEvents::increment(ProfileEvents::TableFunctionExecute); context.checkAccess(AccessType::CREATE_TEMPORARY_TABLE | StorageFactory::instance().getSourceAccessType(getStorageTypeName())); + cached_columns = std::move(cached_columns_); return executeImpl(ast_function, context, table_name); } diff --git a/src/TableFunctions/ITableFunction.h b/src/TableFunctions/ITableFunction.h index 0bbd7e787a5..09f4d3ca80d 100644 --- a/src/TableFunctions/ITableFunction.h +++ b/src/TableFunctions/ITableFunction.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -31,14 +32,19 @@ public: /// Get the main function name. virtual std::string getName() const = 0; + virtual ColumnsDescription getActualTableStructure(const ASTPtr & /*ast_function*/, const Context & /*context*/) { return {}; } + /// Create storage according to the query. - StoragePtr execute(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const; + StoragePtr execute(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns_ = {}) const; virtual ~ITableFunction() {} private: virtual StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const = 0; virtual const char * getStorageTypeName() const = 0; + +protected: + mutable ColumnsDescription cached_columns; }; using TableFunctionPtr = std::shared_ptr; diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 7fa3868d142..3dd258f979c 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -27,8 +27,12 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const + +void TableFunctionRemote::prepareClusterInfo(const ASTPtr & ast_function, const Context & context) const { + if (cluster) + return; + ASTs & args_func = ast_function->children; if (args_func.size() != 1) @@ -44,7 +48,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C String cluster_description; String remote_database; String remote_table; - ASTPtr remote_table_function_ptr; + //ASTPtr remote_table_function_ptr; String username; String password; @@ -136,7 +140,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C for (auto ast : args) setIdentifierSpecial(ast); - ClusterPtr cluster; + //ClusterPtr cluster; if (!cluster_name.empty()) { /// Use an existing cluster from the main config @@ -189,30 +193,54 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C if (!remote_table_function_ptr && remote_table.empty()) throw Exception("The name of remote table cannot be empty", ErrorCodes::BAD_ARGUMENTS); - auto remote_table_id = StorageID::createEmpty(); + //auto remote_table_id = StorageID::createEmpty(); remote_table_id.database_name = remote_database; remote_table_id.table_name = remote_table; - auto structure_remote_table = getStructureOfRemoteTable(*cluster, remote_table_id, context, remote_table_function_ptr); +} + +StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +{ + prepareClusterInfo(ast_function, context); + if (cached_columns.empty()) + cached_columns = getStructureOfRemoteTable(*cluster, remote_table_id, context, remote_table_function_ptr); + //auto structure_remote_table = getStructureOfRemoteTable(*cluster, remote_table_id, context, remote_table_function_ptr); StoragePtr res = remote_table_function_ptr - ? StorageDistributed::createWithOwnCluster( + ? StorageDistributed::create( StorageID(getDatabaseName(), table_name), - structure_remote_table, + cached_columns, + ConstraintsDescription{}, remote_table_function_ptr, - cluster, - context) - : StorageDistributed::createWithOwnCluster( + String{}, + context, + ASTPtr{}, + String{}, + String{}, + false, + cluster) + : StorageDistributed::create( StorageID(getDatabaseName(), table_name), - structure_remote_table, - remote_database, - remote_table, - cluster, - context); + cached_columns, + ConstraintsDescription{}, + remote_table_id.database_name, + remote_table_id.table_name, + String{}, + context, + ASTPtr{}, + String{}, + String{}, + false, + cluster); res->startup(); return res; } +ColumnsDescription TableFunctionRemote::getActualTableStructure(const ASTPtr & ast_function, const Context & context) +{ + prepareClusterInfo(ast_function, context); + return getStructureOfRemoteTable(*cluster, remote_table_id, context, remote_table_function_ptr); +} TableFunctionRemote::TableFunctionRemote(const std::string & name_, bool secure_) : name{name_}, secure{secure_} diff --git a/src/TableFunctions/TableFunctionRemote.h b/src/TableFunctions/TableFunctionRemote.h index 2dd58a8a6a7..27885280920 100644 --- a/src/TableFunctions/TableFunctionRemote.h +++ b/src/TableFunctions/TableFunctionRemote.h @@ -1,6 +1,8 @@ #pragma once #include +#include +#include namespace DB @@ -20,14 +22,22 @@ public: std::string getName() const override { return name; } + ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) override; + private: StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; const char * getStorageTypeName() const override { return "Distributed"; } + void prepareClusterInfo(const ASTPtr & ast_function, const Context & context) const; + std::string name; bool is_cluster_function; std::string help_message; bool secure; + + mutable ClusterPtr cluster; + mutable StorageID remote_table_id = StorageID::createEmpty(); + mutable ASTPtr remote_table_function_ptr; }; } From b521ca9b181be25d5b081b9a2e76f0e6457452a7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 26 Aug 2020 23:56:30 +0300 Subject: [PATCH 02/42] save structure for all table functions --- src/Storages/StorageFile.cpp | 6 +- src/Storages/StorageFile.h | 2 +- src/Storages/StorageGenerateRandom.h | 2 +- src/Storages/StorageHDFS.h | 2 +- src/Storages/StorageInput.h | 2 +- src/Storages/StorageMerge.h | 2 +- src/Storages/StorageMySQL.h | 2 +- ...{StorageProxy.h => StorageTableFunction.h} | 18 ++++- src/Storages/StorageURL.h | 2 +- src/Storages/StorageValues.h | 2 +- src/Storages/StorageXDBC.h | 3 +- src/Storages/System/StorageSystemNumbers.h | 2 +- src/Storages/System/StorageSystemZeros.h | 2 +- src/TableFunctions/ITableFunction.h | 4 +- src/TableFunctions/ITableFunctionFileLike.cpp | 43 ++++++++--- src/TableFunctions/ITableFunctionFileLike.h | 14 +++- src/TableFunctions/ITableFunctionXDBC.cpp | 37 ++++++--- src/TableFunctions/ITableFunctionXDBC.h | 9 +++ src/TableFunctions/TableFunctionFile.cpp | 5 +- src/TableFunctions/TableFunctionFile.h | 3 +- .../TableFunctionGenerateRandom.cpp | 33 ++++++-- .../TableFunctionGenerateRandom.h | 9 +++ src/TableFunctions/TableFunctionHDFS.cpp | 6 +- src/TableFunctions/TableFunctionHDFS.h | 3 +- src/TableFunctions/TableFunctionInput.cpp | 31 +++++++- src/TableFunctions/TableFunctionInput.h | 5 ++ src/TableFunctions/TableFunctionMerge.cpp | 30 ++++++-- src/TableFunctions/TableFunctionMerge.h | 6 ++ src/TableFunctions/TableFunctionMySQL.cpp | 53 ++++++++----- src/TableFunctions/TableFunctionMySQL.h | 20 +++++ src/TableFunctions/TableFunctionNumbers.cpp | 47 +++++++++--- src/TableFunctions/TableFunctionNumbers.h | 6 ++ src/TableFunctions/TableFunctionRemote.cpp | 2 +- src/TableFunctions/TableFunctionRemote.h | 2 +- src/TableFunctions/TableFunctionS3.cpp | 75 ++++++++++--------- src/TableFunctions/TableFunctionS3.h | 20 ++--- src/TableFunctions/TableFunctionURL.cpp | 7 +- src/TableFunctions/TableFunctionURL.h | 3 +- src/TableFunctions/TableFunctionValues.cpp | 33 ++++++-- src/TableFunctions/TableFunctionValues.h | 5 ++ src/TableFunctions/TableFunctionZeros.cpp | 41 +++++++--- src/TableFunctions/TableFunctionZeros.h | 5 ++ 42 files changed, 447 insertions(+), 157 deletions(-) rename src/Storages/{StorageProxy.h => StorageTableFunction.h} (73%) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 558216a6216..6ead7a914ff 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -161,7 +161,11 @@ StorageFile::StorageFile(const std::string & table_path_, const std::string & us if (args.format_name == "Distributed") { - if (!paths.empty()) + if (paths.empty()) + { + throw Exception("Cannot get table structure from file, because no files match specified name", ErrorCodes::INCORRECT_FILE_NAME); + } + else { auto & first_path = paths[0]; Block header = StorageDistributedDirectoryMonitor::createStreamFromFile(first_path)->getHeader(); diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index ea70dcd5311..872a0fc163f 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -18,7 +18,7 @@ namespace DB class StorageFileBlockInputStream; class StorageFileBlockOutputStream; -class StorageFile final : public ext::shared_ptr_helper, public IStorage +class StorageFile : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; public: diff --git a/src/Storages/StorageGenerateRandom.h b/src/Storages/StorageGenerateRandom.h index e0f037f9a08..09208e9c5cd 100644 --- a/src/Storages/StorageGenerateRandom.h +++ b/src/Storages/StorageGenerateRandom.h @@ -9,7 +9,7 @@ namespace DB { /* Generates random data for given schema. */ -class StorageGenerateRandom final : public ext::shared_ptr_helper, public IStorage +class StorageGenerateRandom : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; public: diff --git a/src/Storages/StorageHDFS.h b/src/Storages/StorageHDFS.h index fdeaf4ae1b3..ecb77ef83af 100644 --- a/src/Storages/StorageHDFS.h +++ b/src/Storages/StorageHDFS.h @@ -13,7 +13,7 @@ namespace DB * This class represents table engine for external hdfs files. * Read method is supported for now. */ -class StorageHDFS final : public ext::shared_ptr_helper, public IStorage +class StorageHDFS : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; public: diff --git a/src/Storages/StorageInput.h b/src/Storages/StorageInput.h index c19b19e4703..40b0fd39139 100644 --- a/src/Storages/StorageInput.h +++ b/src/Storages/StorageInput.h @@ -8,7 +8,7 @@ namespace DB /** Internal temporary storage for table function input(...) */ -class StorageInput final : public ext::shared_ptr_helper, public IStorage +class StorageInput : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; public: diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 5e2d56d18c0..ad898ba2ffd 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -13,7 +13,7 @@ namespace DB /** A table that represents the union of an arbitrary number of other tables. * All tables must have the same structure. */ -class StorageMerge final : public ext::shared_ptr_helper, public IStorage +class StorageMerge : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; public: diff --git a/src/Storages/StorageMySQL.h b/src/Storages/StorageMySQL.h index a7f98c4379b..fedd8467210 100644 --- a/src/Storages/StorageMySQL.h +++ b/src/Storages/StorageMySQL.h @@ -20,7 +20,7 @@ namespace DB * Use ENGINE = mysql(host_port, database_name, table_name, user_name, password) * Read only. */ -class StorageMySQL final : public ext::shared_ptr_helper, public IStorage +class StorageMySQL : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; public: diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageTableFunction.h similarity index 73% rename from src/Storages/StorageProxy.h rename to src/Storages/StorageTableFunction.h index 510368b1d15..5dbbae9d46e 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageTableFunction.h @@ -1,15 +1,22 @@ #pragma once #include #include +#include namespace DB { +namespace ErrorCodes +{ + extern const int INCOMPATIBLE_COLUMNS; +} + +using GetStructureFunc = std::function; + template class StorageTableFunction : public StorageT { public: - using GetStructureFunc = std::function; template StorageTableFunction(GetStructureFunc get_structure_, StorageArgs && ... args) @@ -17,6 +24,8 @@ public: { } + String getName() const { return "TableFunction" + StorageT::getName(); } + Pipe read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -26,7 +35,7 @@ public: size_t max_block_size, unsigned num_streams) { - assertBlocksHaveEqualStructure(); + assertSourceStructure(); return StorageT::read(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); } @@ -35,7 +44,7 @@ public: const StorageMetadataPtr & metadata_snapshot, const Context & context) { - assertBlocksHaveEqualStructure(); + assertSourceStructure(); return StorageT::write(query, metadata_snapshot, context); } @@ -48,7 +57,8 @@ private: StorageInMemoryMetadata source_metadata; source_metadata.setColumns(get_structure()); actual_source_structure = source_metadata.getSampleBlock(); - assertBlocksHaveEqualStructure(getInMemoryMetadataPtr()->getSampleBlock(), actual_source_structure); + if (!blocksHaveEqualStructure(StorageT::getInMemoryMetadataPtr()->getSampleBlock(), actual_source_structure)) + throw Exception("Source storage and table function have different structure", ErrorCodes::INCOMPATIBLE_COLUMNS); get_structure = {}; } diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 7983ad71520..69fe19650eb 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -93,7 +93,7 @@ private: BlockOutputStreamPtr writer; }; -class StorageURL final : public ext::shared_ptr_helper, public IStorageURLBase +class StorageURL : public ext::shared_ptr_helper, public IStorageURLBase { friend struct ext::shared_ptr_helper; public: diff --git a/src/Storages/StorageValues.h b/src/Storages/StorageValues.h index 8a1a06eeb54..b69af140567 100644 --- a/src/Storages/StorageValues.h +++ b/src/Storages/StorageValues.h @@ -9,7 +9,7 @@ namespace DB /* One block storage used for values table function * It's structure is similar to IStorageSystemOneBlock */ -class StorageValues final : public ext::shared_ptr_helper, public IStorage +class StorageValues : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; public: diff --git a/src/Storages/StorageXDBC.h b/src/Storages/StorageXDBC.h index 508edf22684..7f93cbcd320 100644 --- a/src/Storages/StorageXDBC.h +++ b/src/Storages/StorageXDBC.h @@ -34,6 +34,7 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; + std::string getName() const override; private: BridgeHelperPtr bridge_helper; @@ -61,8 +62,6 @@ private: size_t max_block_size) const override; Block getHeaderBlock(const Names & column_names, const StorageMetadataPtr & metadata_snapshot) const override; - - std::string getName() const override; }; } diff --git a/src/Storages/System/StorageSystemNumbers.h b/src/Storages/System/StorageSystemNumbers.h index f907f3d5f93..cdffd6c8434 100644 --- a/src/Storages/System/StorageSystemNumbers.h +++ b/src/Storages/System/StorageSystemNumbers.h @@ -23,7 +23,7 @@ class Context; * In multithreaded case, if even_distributed is False, implementation with atomic is used, * and result is always in [0 ... limit - 1] range. */ -class StorageSystemNumbers final : public ext::shared_ptr_helper, public IStorage +class StorageSystemNumbers : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; public: diff --git a/src/Storages/System/StorageSystemZeros.h b/src/Storages/System/StorageSystemZeros.h index 41de3ce6246..26379df5956 100644 --- a/src/Storages/System/StorageSystemZeros.h +++ b/src/Storages/System/StorageSystemZeros.h @@ -14,7 +14,7 @@ namespace DB * You could also specify a limit (how many zeros to give). * If multithreaded is specified, zeros will be generated in several streams. */ -class StorageSystemZeros final : public ext::shared_ptr_helper, public IStorage +class StorageSystemZeros : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; public: diff --git a/src/TableFunctions/ITableFunction.h b/src/TableFunctions/ITableFunction.h index 09f4d3ca80d..2db943225f9 100644 --- a/src/TableFunctions/ITableFunction.h +++ b/src/TableFunctions/ITableFunction.h @@ -24,7 +24,7 @@ class Context; * - go to `example01-01-1`, in `merge` database, `hits` table. */ -class ITableFunction +class ITableFunction : public std::enable_shared_from_this { public: static inline std::string getDatabaseName() { return "_table_function"; } @@ -32,7 +32,7 @@ public: /// Get the main function name. virtual std::string getName() const = 0; - virtual ColumnsDescription getActualTableStructure(const ASTPtr & /*ast_function*/, const Context & /*context*/) { return {}; } + virtual ColumnsDescription getActualTableStructure(const ASTPtr & /*ast_function*/, const Context & /*context*/) const { return {}; } /// Create storage according to the query. StoragePtr execute(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns_ = {}) const; diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index 46a64cef785..cf52c6e7163 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -13,7 +13,6 @@ #include #include - namespace DB { @@ -23,8 +22,11 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, const Context & context) const { + if (!filename.empty()) + return; + /// Parse args ASTs & args_func = ast_function->children; @@ -39,8 +41,8 @@ StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & ast_function, cons for (auto & arg : args) arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); - std::string filename = args[0]->as().value.safeGet(); - std::string format = args[1]->as().value.safeGet(); + filename = args[0]->as().value.safeGet(); + format = args[1]->as().value.safeGet(); if (args.size() == 2 && getName() == "file") { @@ -51,24 +53,41 @@ StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & ast_function, cons throw Exception("Table function '" + getName() + "' requires 3 or 4 arguments: filename, format, structure and compression method (default auto).", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - ColumnsDescription columns; - std::string compression_method = "auto"; - if (args.size() > 2) - { - auto structure = args[2]->as().value.safeGet(); - columns = parseColumnsListFromString(structure, context); - } + structure = args[2]->as().value.safeGet(); if (args.size() == 4) compression_method = args[3]->as().value.safeGet(); +} + +StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +{ + parseArguments(ast_function, context); + if (cached_columns.empty()) + cached_columns = getActualTableStructure(ast_function, context); + + auto get_structure = [=, tf = shared_from_this()]() + { + return tf->getActualTableStructure(ast_function, context); + }; /// Create table - StoragePtr storage = getStorage(filename, format, columns, const_cast(context), table_name, compression_method); + StoragePtr storage = getStorage(filename, format, cached_columns, const_cast(context), table_name, compression_method, std::move(get_structure)); storage->startup(); return storage; } +ColumnsDescription ITableFunctionFileLike::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const +{ + parseArguments(ast_function, context); + if (structure.empty()) + { + assert(getName() == "file" && format == "Distributed"); + return {}; /// TODO get matching path, read structure + } + return parseColumnsListFromString(structure, context); +} + } diff --git a/src/TableFunctions/ITableFunctionFileLike.h b/src/TableFunctions/ITableFunctionFileLike.h index 101f4104cb8..56e5521ceb6 100644 --- a/src/TableFunctions/ITableFunctionFileLike.h +++ b/src/TableFunctions/ITableFunctionFileLike.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { @@ -14,7 +15,18 @@ class ITableFunctionFileLike : public ITableFunction { private: StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; + virtual StoragePtr getStorage( - const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const = 0; + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, + const std::string & table_name, const String & compression_method, GetStructureFunc get_structure) const = 0; + + ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; + + void parseArguments(const ASTPtr & ast_function, const Context & context) const; + + mutable String filename; + mutable String format; + mutable String structure; + mutable String compression_method = "auto"; }; } diff --git a/src/TableFunctions/ITableFunctionXDBC.cpp b/src/TableFunctions/ITableFunctionXDBC.cpp index e3b36e97525..ea1f8118ce1 100644 --- a/src/TableFunctions/ITableFunctionXDBC.cpp +++ b/src/TableFunctions/ITableFunctionXDBC.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -28,8 +29,11 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +void ITableFunctionXDBC::parseArguments(const ASTPtr & ast_function, const Context & context) const { + if (helper) + return; + const auto & args_func = ast_function->as(); if (!args_func.arguments) @@ -44,10 +48,6 @@ StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & ast_function, const Co for (auto & arg : args) arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); - std::string connection_string; - std::string schema_name; - std::string remote_table_name; - if (args.size() == 3) { connection_string = args[0]->as().value.safeGet(); @@ -60,11 +60,16 @@ StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & ast_function, const Co remote_table_name = args[1]->as().value.safeGet(); } - /* Infer external table structure */ /// Have to const_cast, because bridges store their commands inside context - BridgeHelperPtr helper = createBridgeHelper(const_cast(context), context.getSettingsRef().http_receive_timeout.value, connection_string); + helper = createBridgeHelper(const_cast(context), context.getSettingsRef().http_receive_timeout.value, connection_string); helper->startBridgeSync(); +} +ColumnsDescription ITableFunctionXDBC::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const +{ + parseArguments(ast_function, context); + + /* Infer external table structure */ Poco::URI columns_info_uri = helper->getColumnsInfoURI(); columns_info_uri.addQueryParameter("connection_string", connection_string); if (!schema_name.empty()) @@ -73,7 +78,7 @@ StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & ast_function, const Co const auto use_nulls = context.getSettingsRef().external_table_functions_use_nulls; columns_info_uri.addQueryParameter("external_table_functions_use_nulls", - Poco::NumberFormatter::format(use_nulls)); + Poco::NumberFormatter::format(use_nulls)); ReadWriteBufferFromHTTP buf(columns_info_uri, Poco::Net::HTTPRequest::HTTP_POST, {}, ConnectionTimeouts::getHTTPTimeouts(context)); @@ -81,7 +86,21 @@ StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & ast_function, const Co readStringBinary(columns_info, buf); NamesAndTypesList columns = NamesAndTypesList::parse(columns_info); - auto result = std::make_shared(StorageID(getDatabaseName(), table_name), schema_name, remote_table_name, ColumnsDescription{columns}, context, helper); + return ColumnsDescription{columns}; +} + +StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +{ + parseArguments(ast_function, context); + if (cached_columns.empty()) + cached_columns = getActualTableStructure(ast_function, context); + + auto get_structure = [=, tf = shared_from_this()]() + { + return tf->getActualTableStructure(ast_function, context); + }; + + auto result = std::make_shared>(get_structure, StorageID(getDatabaseName(), table_name), schema_name, remote_table_name, cached_columns, context, helper); if (!result) throw Exception("Failed to instantiate storage from table function " + getName(), ErrorCodes::UNKNOWN_EXCEPTION); diff --git a/src/TableFunctions/ITableFunctionXDBC.h b/src/TableFunctions/ITableFunctionXDBC.h index 547fda3f734..f2b3e70a3a9 100644 --- a/src/TableFunctions/ITableFunctionXDBC.h +++ b/src/TableFunctions/ITableFunctionXDBC.h @@ -24,6 +24,15 @@ private: virtual BridgeHelperPtr createBridgeHelper(Context & context, const Poco::Timespan & http_timeout_, const std::string & connection_string_) const = 0; + + ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; + + void parseArguments(const ASTPtr & ast_function, const Context & context) const; + + mutable String connection_string; + mutable String schema_name; + mutable String remote_table_name; + mutable BridgeHelperPtr helper; }; class TableFunctionJDBC : public ITableFunctionXDBC diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index 4f67f4cfd10..77161d5f02e 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -9,11 +9,12 @@ namespace DB { StoragePtr TableFunctionFile::getStorage( - const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const std::string & compression_method) const + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, + const std::string & table_name, const std::string & compression_method, GetStructureFunc get_structure) const { StorageFile::CommonArguments args{StorageID(getDatabaseName(), table_name), format, compression_method, columns, ConstraintsDescription{}, global_context}; - return StorageFile::create(source, global_context.getUserFilesPath(), args); + return std::make_shared>(std::move(get_structure), source, global_context.getUserFilesPath(), args); } void registerTableFunctionFile(TableFunctionFactory & factory) diff --git a/src/TableFunctions/TableFunctionFile.h b/src/TableFunctions/TableFunctionFile.h index e0d8c20ac61..8a8d8d887a8 100644 --- a/src/TableFunctions/TableFunctionFile.h +++ b/src/TableFunctions/TableFunctionFile.h @@ -23,6 +23,7 @@ public: private: StoragePtr getStorage( - const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const std::string & compression_method) const override; + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, + const std::string & table_name, const std::string & compression_method, GetStructureFunc get_structure) const override; const char * getStorageTypeName() const override { return "File"; } };} diff --git a/src/TableFunctions/TableFunctionGenerateRandom.cpp b/src/TableFunctions/TableFunctionGenerateRandom.cpp index 548db38515c..086a76e0ae9 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.cpp +++ b/src/TableFunctions/TableFunctionGenerateRandom.cpp @@ -13,6 +13,9 @@ #include #include +#include +#include + #include "registerTableFunctions.h" @@ -26,8 +29,11 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -StoragePtr TableFunctionGenerateRandom::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, const Context & /*context*/) const { + if (!structure.empty()) + return; + ASTs & args_func = ast_function->children; if (args_func.size() != 1) @@ -58,11 +64,7 @@ StoragePtr TableFunctionGenerateRandom::executeImpl(const ASTPtr & ast_function, } /// Parsing first argument as table structure and creating a sample block - std::string structure = args[0]->as().value.safeGet(); - - UInt64 max_string_length = 10; - UInt64 max_array_length = 10; - std::optional random_seed; + structure = args[0]->as().value.safeGet(); if (args.size() >= 2) { @@ -76,11 +78,26 @@ StoragePtr TableFunctionGenerateRandom::executeImpl(const ASTPtr & ast_function, if (args.size() == 4) max_array_length = args[3]->as().value.safeGet(); +} +ColumnsDescription TableFunctionGenerateRandom::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const +{ + parseArguments(ast_function, context); + return parseColumnsListFromString(structure, context); +} - ColumnsDescription columns = parseColumnsListFromString(structure, context); +StoragePtr TableFunctionGenerateRandom::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +{ + parseArguments(ast_function, context); + if (cached_columns.empty()) + cached_columns = getActualTableStructure(ast_function, context); - auto res = StorageGenerateRandom::create(StorageID(getDatabaseName(), table_name), columns, max_array_length, max_string_length, random_seed); + auto get_structure = [=, tf = shared_from_this()]() + { + return tf->getActualTableStructure(ast_function, context); + }; + + auto res = std::make_shared>(get_structure, StorageID(getDatabaseName(), table_name), cached_columns, max_array_length, max_string_length, random_seed); res->startup(); return res; } diff --git a/src/TableFunctions/TableFunctionGenerateRandom.h b/src/TableFunctions/TableFunctionGenerateRandom.h index b0919608737..45ad93ffee7 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.h +++ b/src/TableFunctions/TableFunctionGenerateRandom.h @@ -16,6 +16,15 @@ public: private: StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; const char * getStorageTypeName() const override { return "GenerateRandom"; } + + ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; + void parseArguments(const ASTPtr & ast_function, const Context & context) const; + + mutable String structure; + mutable UInt64 max_string_length = 10; + mutable UInt64 max_array_length = 10; + mutable std::optional random_seed; + }; diff --git a/src/TableFunctions/TableFunctionHDFS.cpp b/src/TableFunctions/TableFunctionHDFS.cpp index 3bd6cd3ed76..52397caac67 100644 --- a/src/TableFunctions/TableFunctionHDFS.cpp +++ b/src/TableFunctions/TableFunctionHDFS.cpp @@ -10,9 +10,11 @@ namespace DB { StoragePtr TableFunctionHDFS::getStorage( - const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, + const std::string & table_name, const String & compression_method, GetStructureFunc get_structure) const { - return StorageHDFS::create(source, + return std::make_shared>(std::move(get_structure), + source, StorageID(getDatabaseName(), table_name), format, columns, diff --git a/src/TableFunctions/TableFunctionHDFS.h b/src/TableFunctions/TableFunctionHDFS.h index 5e8503b318e..91cbfd7c578 100644 --- a/src/TableFunctions/TableFunctionHDFS.h +++ b/src/TableFunctions/TableFunctionHDFS.h @@ -26,7 +26,8 @@ public: private: StoragePtr getStorage( - const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const override; + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, + const std::string & table_name, const String & compression_method, GetStructureFunc get_structure) const override; const char * getStorageTypeName() const override { return "HDFS"; } }; diff --git a/src/TableFunctions/TableFunctionInput.cpp b/src/TableFunctions/TableFunctionInput.cpp index 5b4a3d97ee4..5b36bb77b2c 100644 --- a/src/TableFunctions/TableFunctionInput.cpp +++ b/src/TableFunctions/TableFunctionInput.cpp @@ -7,8 +7,10 @@ #include #include #include +#include #include #include +#include #include #include "registerTableFunctions.h" @@ -22,8 +24,11 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -StoragePtr TableFunctionInput::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +void TableFunctionInput::parseArguments(const ASTPtr & ast_function, const Context & context) const { + if (!structure.empty()) + return; + const auto * function = ast_function->as(); if (!function->arguments) @@ -35,9 +40,27 @@ StoragePtr TableFunctionInput::executeImpl(const ASTPtr & ast_function, const Co throw Exception("Table function '" + getName() + "' requires exactly 1 argument: structure", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - String structure = evaluateConstantExpressionOrIdentifierAsLiteral(args[0], context)->as().value.safeGet(); - auto columns = parseColumnsListFromString(structure, context); - StoragePtr storage = StorageInput::create(StorageID(getDatabaseName(), table_name), columns); + structure = evaluateConstantExpressionOrIdentifierAsLiteral(args[0], context)->as().value.safeGet(); +} + +ColumnsDescription TableFunctionInput::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const +{ + parseArguments(ast_function, context); + return parseColumnsListFromString(structure, context); +} + +StoragePtr TableFunctionInput::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +{ + parseArguments(ast_function, context); + if (cached_columns.empty()) + cached_columns = getActualTableStructure(ast_function, context); + + auto get_structure = [=, tf = shared_from_this()]() + { + return tf->getActualTableStructure(ast_function, context); + }; + + StoragePtr storage = std::make_shared>(get_structure, StorageID(getDatabaseName(), table_name), cached_columns); storage->startup(); diff --git a/src/TableFunctions/TableFunctionInput.h b/src/TableFunctions/TableFunctionInput.h index a2791533c5d..50744d3ca14 100644 --- a/src/TableFunctions/TableFunctionInput.h +++ b/src/TableFunctions/TableFunctionInput.h @@ -19,6 +19,11 @@ public: private: StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; const char * getStorageTypeName() const override { return "Input"; } + + ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; + void parseArguments(const ASTPtr & ast_function, const Context & context) const; + + mutable String structure; }; } diff --git a/src/TableFunctions/TableFunctionMerge.cpp b/src/TableFunctions/TableFunctionMerge.cpp index 7c0c1fb233f..ffc8076b868 100644 --- a/src/TableFunctions/TableFunctionMerge.cpp +++ b/src/TableFunctions/TableFunctionMerge.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -45,8 +46,7 @@ static NamesAndTypesList chooseColumns(const String & source_database, const Str return any_table->getInMemoryMetadataPtr()->getColumns().getAllPhysical(); } - -StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, const Context & context) const { ASTs & args_func = ast_function->children; @@ -65,12 +65,30 @@ StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & ast_function, const Co args[0] = evaluateConstantExpressionForDatabaseName(args[0], context); args[1] = evaluateConstantExpressionAsLiteral(args[1], context); - String source_database = args[0]->as().value.safeGet(); - String table_name_regexp = args[1]->as().value.safeGet(); + source_database = args[0]->as().value.safeGet(); + table_name_regexp = args[1]->as().value.safeGet(); +} - auto res = StorageMerge::create( +ColumnsDescription TableFunctionMerge::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const +{ + parseArguments(ast_function, context); + return ColumnsDescription{chooseColumns(source_database, table_name_regexp, context)}; +} + +StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +{ + parseArguments(ast_function, context); + if (cached_columns.empty()) + cached_columns = getActualTableStructure(ast_function, context); + + auto get_structure = [=, tf = shared_from_this()]() + { + return tf->getActualTableStructure(ast_function, context); + }; + + auto res = std::make_shared>(std::move(get_structure), StorageID(getDatabaseName(), table_name), - ColumnsDescription{chooseColumns(source_database, table_name_regexp, context)}, + cached_columns, source_database, table_name_regexp, context); diff --git a/src/TableFunctions/TableFunctionMerge.h b/src/TableFunctions/TableFunctionMerge.h index b11a9551d34..0e6878983e3 100644 --- a/src/TableFunctions/TableFunctionMerge.h +++ b/src/TableFunctions/TableFunctionMerge.h @@ -18,6 +18,12 @@ public: private: StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; const char * getStorageTypeName() const override { return "Merge"; } + + ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; + void parseArguments(const ASTPtr & ast_function, const Context & context) const; + + mutable String source_database; + mutable String table_name_regexp; }; diff --git a/src/TableFunctions/TableFunctionMySQL.cpp b/src/TableFunctions/TableFunctionMySQL.cpp index 0afc74f163e..66496778460 100644 --- a/src/TableFunctions/TableFunctionMySQL.cpp +++ b/src/TableFunctions/TableFunctionMySQL.cpp @@ -13,6 +13,7 @@ # include # include # include +# include # include # include # include @@ -21,8 +22,6 @@ # include # include "registerTableFunctions.h" -# include - namespace DB { @@ -35,8 +34,7 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; } - -StoragePtr TableFunctionMySQL::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, const Context & context) const { const auto & args_func = ast_function->as(); @@ -52,14 +50,12 @@ StoragePtr TableFunctionMySQL::executeImpl(const ASTPtr & ast_function, const Co for (auto & arg : args) arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); - std::string host_port = args[0]->as().value.safeGet(); - std::string remote_database_name = args[1]->as().value.safeGet(); - std::string remote_table_name = args[2]->as().value.safeGet(); - std::string user_name = args[3]->as().value.safeGet(); - std::string password = args[4]->as().value.safeGet(); + String host_port = args[0]->as().value.safeGet(); + remote_database_name = args[1]->as().value.safeGet(); + remote_table_name = args[2]->as().value.safeGet(); + user_name = args[3]->as().value.safeGet(); + password = args[4]->as().value.safeGet(); - bool replace_query = false; - std::string on_duplicate_clause; if (args.size() >= 6) replace_query = args[5]->as().value.safeGet() > 0; if (args.size() == 7) @@ -71,9 +67,14 @@ StoragePtr TableFunctionMySQL::executeImpl(const ASTPtr & ast_function, const Co ErrorCodes::BAD_ARGUMENTS); /// 3306 is the default MySQL port number - auto parsed_host_port = parseAddress(host_port, 3306); + parsed_host_port = parseAddress(host_port, 3306); +} - mysqlxx::Pool pool(remote_database_name, parsed_host_port.first, user_name, password, parsed_host_port.second); +ColumnsDescription TableFunctionMySQL::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const +{ + parseArguments(ast_function, context); + if (!pool) + pool.emplace(remote_database_name, parsed_host_port.first, user_name, password, parsed_host_port.second); /// Determine table definition by running a query to INFORMATION_SCHEMA. @@ -99,7 +100,7 @@ StoragePtr TableFunctionMySQL::executeImpl(const ASTPtr & ast_function, const Co << " ORDER BY ORDINAL_POSITION"; NamesAndTypesList columns; - MySQLBlockInputStream result(pool.get(), query.str(), sample_block, DEFAULT_BLOCK_SIZE); + MySQLBlockInputStream result(pool->get(), query.str(), sample_block, DEFAULT_BLOCK_SIZE); while (Block block = result.read()) { size_t rows = block.rows(); @@ -117,17 +118,35 @@ StoragePtr TableFunctionMySQL::executeImpl(const ASTPtr & ast_function, const Co if (columns.empty()) throw Exception("MySQL table " + backQuoteIfNeed(remote_database_name) + "." + backQuoteIfNeed(remote_table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); - auto res = StorageMySQL::create( + return ColumnsDescription{columns}; +} + +StoragePtr TableFunctionMySQL::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +{ + parseArguments(ast_function, context); + if (cached_columns.empty()) + cached_columns = getActualTableStructure(ast_function, context); + if (!pool) + pool.emplace(remote_database_name, parsed_host_port.first, user_name, password, parsed_host_port.second); + + auto get_structure = [=, tf = shared_from_this()]() + { + return tf->getActualTableStructure(ast_function, context); + }; + + auto res = std::make_shared>(std::move(get_structure), StorageID(getDatabaseName(), table_name), - std::move(pool), + std::move(*pool), remote_database_name, remote_table_name, replace_query, on_duplicate_clause, - ColumnsDescription{columns}, + cached_columns, ConstraintsDescription{}, context); + pool.reset(); + res->startup(); return res; } diff --git a/src/TableFunctions/TableFunctionMySQL.h b/src/TableFunctions/TableFunctionMySQL.h index 850affc5887..e8f5851b502 100644 --- a/src/TableFunctions/TableFunctionMySQL.h +++ b/src/TableFunctions/TableFunctionMySQL.h @@ -1,6 +1,11 @@ #pragma once +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif +#if USE_MYSQL #include +#include namespace DB @@ -21,6 +26,21 @@ public: private: StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; const char * getStorageTypeName() const override { return "MySQL"; } + + ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; + void parseArguments(const ASTPtr & ast_function, const Context & context) const; + + mutable std::pair parsed_host_port; + mutable String remote_database_name; + mutable String remote_table_name; + mutable String user_name; + mutable String password; + mutable bool replace_query = false; + mutable String on_duplicate_clause; + + mutable std::optional pool; }; } + +#endif diff --git a/src/TableFunctions/TableFunctionNumbers.cpp b/src/TableFunctions/TableFunctionNumbers.cpp index c8c0fe96092..256ec47b21d 100644 --- a/src/TableFunctions/TableFunctionNumbers.cpp +++ b/src/TableFunctions/TableFunctionNumbers.cpp @@ -5,8 +5,10 @@ #include #include #include +#include #include #include +#include #include "registerTableFunctions.h" @@ -19,8 +21,10 @@ namespace ErrorCodes } template -StoragePtr TableFunctionNumbers::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +void TableFunctionNumbers::parseArguments(const ASTPtr & ast_function, const Context & context) const { + + if (const auto * function = ast_function->as()) { auto arguments = function->arguments->children; @@ -28,15 +32,40 @@ StoragePtr TableFunctionNumbers::executeImpl(const ASTPtr & ast_f if (arguments.size() != 1 && arguments.size() != 2) throw Exception("Table function '" + getName() + "' requires 'length' or 'offset, length'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - UInt64 offset = arguments.size() == 2 ? evaluateArgument(context, arguments[0]) : 0; - UInt64 length = arguments.size() == 2 ? evaluateArgument(context, arguments[1]) : evaluateArgument(context, arguments[0]); - - auto res = StorageSystemNumbers::create(StorageID(getDatabaseName(), table_name), multithreaded, length, offset, false); - res->startup(); - return res; + if (arguments.size() == 1) + length = evaluateArgument(context, arguments[0]); + else + { + offset = evaluateArgument(context, arguments[0]); + length = evaluateArgument(context, arguments[1]); + } } - throw Exception("Table function '" + getName() + "' requires 'limit' or 'offset, limit'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + else + throw Exception("Table function '" + getName() + "' requires 'limit' or 'offset, limit'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); +} + +template +ColumnsDescription TableFunctionNumbers::getActualTableStructure(const ASTPtr & /*ast_function*/, const Context & /*context*/) const +{ + return ColumnsDescription({{"number", std::make_shared()}}); +} + +template +StoragePtr TableFunctionNumbers::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +{ + parseArguments(ast_function, context); + + if (cached_columns.empty()) + cached_columns = getActualTableStructure(ast_function, context); + + auto get_structure = [=, tf = shared_from_this()]() + { + return tf->getActualTableStructure(ast_function, context); + }; + + auto res = std::make_shared>(get_structure, StorageID(getDatabaseName(), table_name), multithreaded, length, offset, false); + res->startup(); + return res; } void registerTableFunctionNumbers(TableFunctionFactory & factory) diff --git a/src/TableFunctions/TableFunctionNumbers.h b/src/TableFunctions/TableFunctionNumbers.h index c3efbc426ef..5d12eefd21c 100644 --- a/src/TableFunctions/TableFunctionNumbers.h +++ b/src/TableFunctions/TableFunctionNumbers.h @@ -22,6 +22,12 @@ private: const char * getStorageTypeName() const override { return "SystemNumbers"; } UInt64 evaluateArgument(const Context & context, ASTPtr & argument) const; + + ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; + void parseArguments(const ASTPtr & ast_function, const Context & context) const; + + mutable UInt64 offset = 0; + mutable UInt64 length; }; diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 3dd258f979c..cc5485189d0 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -236,7 +236,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C return res; } -ColumnsDescription TableFunctionRemote::getActualTableStructure(const ASTPtr & ast_function, const Context & context) +ColumnsDescription TableFunctionRemote::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const { prepareClusterInfo(ast_function, context); return getStructureOfRemoteTable(*cluster, remote_table_id, context, remote_table_function_ptr); diff --git a/src/TableFunctions/TableFunctionRemote.h b/src/TableFunctions/TableFunctionRemote.h index 27885280920..35d1ca3b9ff 100644 --- a/src/TableFunctions/TableFunctionRemote.h +++ b/src/TableFunctions/TableFunctionRemote.h @@ -22,7 +22,7 @@ public: std::string getName() const override { return name; } - ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) override; + ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; private: StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 1a0bff12ba2..4c2779f72d0 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include #include @@ -21,8 +22,10 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -StoragePtr TableFunctionS3::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +void TableFunctionS3::parseArguments(const ASTPtr & ast_function, const Context & context) const { + + /// Parse args ASTs & args_func = ast_function->children; @@ -38,11 +41,7 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & ast_function, const Conte for (auto & arg : args) arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); - String filename = args[0]->as().value.safeGet(); - String format; - String structure; - String access_key_id; - String secret_access_key; + filename = args[0]->as().value.safeGet(); if (args.size() < 5) { @@ -57,47 +56,49 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & ast_function, const Conte structure = args[4]->as().value.safeGet(); } - String compression_method; if (args.size() == 4 || args.size() == 6) compression_method = args.back()->as().value.safeGet(); - else - compression_method = "auto"; +} - ColumnsDescription columns = parseColumnsListFromString(structure, context); +ColumnsDescription TableFunctionS3::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const +{ + parseArguments(ast_function, context); + return parseColumnsListFromString(structure, context); +} - /// Create table - StoragePtr storage = getStorage(filename, access_key_id, secret_access_key, format, columns, const_cast(context), table_name, compression_method); +StoragePtr TableFunctionS3::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +{ + parseArguments(ast_function, context); + + if (cached_columns.empty()) + cached_columns = getActualTableStructure(ast_function, context); + + auto get_structure = [=, tf = shared_from_this()]() + { + return tf->getActualTableStructure(ast_function, context); + }; + + Poco::URI uri (filename); + S3::URI s3_uri (uri); + UInt64 min_upload_part_size = context.getSettingsRef().s3_min_upload_part_size; + + StoragePtr storage = std::make_shared>(std::move(get_structure), + s3_uri, + access_key_id, + secret_access_key, + StorageID(getDatabaseName(), table_name), + format, + min_upload_part_size, + cached_columns, + ConstraintsDescription{}, + const_cast(context), + compression_method); storage->startup(); return storage; } -StoragePtr TableFunctionS3::getStorage( - const String & source, - const String & access_key_id, - const String & secret_access_key, - const String & format, - const ColumnsDescription & columns, - Context & global_context, - const std::string & table_name, - const String & compression_method) -{ - Poco::URI uri (source); - S3::URI s3_uri (uri); - UInt64 min_upload_part_size = global_context.getSettingsRef().s3_min_upload_part_size; - return StorageS3::create( - s3_uri, - access_key_id, - secret_access_key, - StorageID(getDatabaseName(), table_name), - format, - min_upload_part_size, - columns, - ConstraintsDescription{}, - global_context, - compression_method); -} void registerTableFunctionS3(TableFunctionFactory & factory) { diff --git a/src/TableFunctions/TableFunctionS3.h b/src/TableFunctions/TableFunctionS3.h index b66da57d4fa..23dec9a689c 100644 --- a/src/TableFunctions/TableFunctionS3.h +++ b/src/TableFunctions/TableFunctionS3.h @@ -29,17 +29,17 @@ protected: const Context & context, const std::string & table_name) const override; - static StoragePtr getStorage( - const String & source, - const String & access_key_id, - const String & secret_access_key, - const String & format, - const ColumnsDescription & columns, - Context & global_context, - const std::string & table_name, - const String & compression_method); - const char * getStorageTypeName() const override { return "S3"; } + + ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; + void parseArguments(const ASTPtr & ast_function, const Context & context) const; + + mutable String filename; + mutable String format; + mutable String structure; + mutable String access_key_id; + mutable String secret_access_key; + mutable String compression_method = "auto"; }; class TableFunctionCOS : public TableFunctionS3 diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index 59978ae08b1..74cab6e9f00 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -10,10 +10,13 @@ namespace DB { StoragePtr TableFunctionURL::getStorage( - const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, + const std::string & table_name, const String & compression_method, GetStructureFunc get_structure) const { Poco::URI uri(source); - return StorageURL::create(uri, StorageID(getDatabaseName(), table_name), format, columns, ConstraintsDescription{}, global_context, compression_method); + return std::make_shared>(std::move(get_structure), uri, + StorageID(getDatabaseName(), table_name), format, columns, ConstraintsDescription{}, + global_context, compression_method); } void registerTableFunctionURL(TableFunctionFactory & factory) diff --git a/src/TableFunctions/TableFunctionURL.h b/src/TableFunctions/TableFunctionURL.h index 366d573bcf4..d69b7ff33a1 100644 --- a/src/TableFunctions/TableFunctionURL.h +++ b/src/TableFunctions/TableFunctionURL.h @@ -21,7 +21,8 @@ public: private: StoragePtr getStorage( - const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const override; + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, + const std::string & table_name, const String & compression_method, GetStructureFunc get_structure) const override; const char * getStorageTypeName() const override { return "URL"; } }; diff --git a/src/TableFunctions/TableFunctionValues.cpp b/src/TableFunctions/TableFunctionValues.cpp index b4b243416f2..ca1d8b51b22 100644 --- a/src/TableFunctions/TableFunctionValues.cpp +++ b/src/TableFunctions/TableFunctionValues.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include @@ -62,8 +63,10 @@ static void parseAndInsertValues(MutableColumns & res_columns, const ASTs & args } } -StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +void TableFunctionValues::parseArguments(const ASTPtr & ast_function, const Context & /*context*/) const { + + ASTs & args_func = ast_function->children; if (args_func.size() != 1) @@ -83,22 +86,42 @@ StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, const C "Got '{}' instead", getName(), args[0]->formatForErrorMessage()), ErrorCodes::BAD_ARGUMENTS); } - std::string structure = args[0]->as().value.safeGet(); - ColumnsDescription columns = parseColumnsListFromString(structure, context); + structure = args[0]->as().value.safeGet(); +} + +ColumnsDescription TableFunctionValues::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const +{ + parseArguments(ast_function, context); + return parseColumnsListFromString(structure, context); +} + +StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +{ + parseArguments(ast_function, context); + + if (cached_columns.empty()) + cached_columns = getActualTableStructure(ast_function, context); + + auto get_structure = [=, tf = shared_from_this()]() + { + return tf->getActualTableStructure(ast_function, context); + }; Block sample_block; - for (const auto & name_type : columns.getOrdinary()) + for (const auto & name_type : cached_columns.getOrdinary()) sample_block.insert({ name_type.type->createColumn(), name_type.type, name_type.name }); MutableColumns res_columns = sample_block.cloneEmptyColumns(); + ASTs & args = ast_function->children.at(0)->children; + /// Parsing other arguments as values and inserting them into columns parseAndInsertValues(res_columns, args, sample_block, context); Block res_block = sample_block.cloneWithColumns(std::move(res_columns)); - auto res = StorageValues::create(StorageID(getDatabaseName(), table_name), columns, res_block); + auto res = std::make_shared>(get_structure, StorageID(getDatabaseName(), table_name), cached_columns, res_block); res->startup(); return res; } diff --git a/src/TableFunctions/TableFunctionValues.h b/src/TableFunctions/TableFunctionValues.h index 3cc3687dab5..9ab2dfb047e 100644 --- a/src/TableFunctions/TableFunctionValues.h +++ b/src/TableFunctions/TableFunctionValues.h @@ -15,6 +15,11 @@ public: private: StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; const char * getStorageTypeName() const override { return "Values"; } + + ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; + void parseArguments(const ASTPtr & ast_function, const Context & context) const; + + mutable String structure; }; diff --git a/src/TableFunctions/TableFunctionZeros.cpp b/src/TableFunctions/TableFunctionZeros.cpp index 13436f04e1c..56413431c37 100644 --- a/src/TableFunctions/TableFunctionZeros.cpp +++ b/src/TableFunctions/TableFunctionZeros.cpp @@ -3,8 +3,9 @@ #include #include #include -#include #include +#include +#include #include #include #include "registerTableFunctions.h" @@ -19,8 +20,10 @@ extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } template -StoragePtr TableFunctionZeros::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +void TableFunctionZeros::parseArguments(const ASTPtr & ast_function, const Context & context) const { + + if (const auto * function = ast_function->as()) { auto arguments = function->arguments->children; @@ -28,14 +31,34 @@ StoragePtr TableFunctionZeros::executeImpl(const ASTPtr & ast_fun if (arguments.size() != 1) throw Exception("Table function '" + getName() + "' requires 'length'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - UInt64 length = evaluateArgument(context, arguments[0]); - - auto res = StorageSystemZeros::create(StorageID(getDatabaseName(), table_name), multithreaded, length); - res->startup(); - return res; + length = evaluateArgument(context, arguments[0]); } - throw Exception("Table function '" + getName() + "' requires 'limit'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + else + throw Exception("Table function '" + getName() + "' requires 'limit'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); +} + +template +ColumnsDescription TableFunctionZeros::getActualTableStructure(const ASTPtr & /*ast_function*/, const Context & /*context*/) const +{ + return ColumnsDescription({{"zero", std::make_shared()}}); +} + +template +StoragePtr TableFunctionZeros::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +{ + parseArguments(ast_function, context); + + if (cached_columns.empty()) + cached_columns = getActualTableStructure(ast_function, context); + + auto get_structure = [=, tf = shared_from_this()]() + { + return tf->getActualTableStructure(ast_function, context); + }; + + auto res = std::make_shared>(std::move(get_structure), StorageID(getDatabaseName(), table_name), multithreaded, length); + res->startup(); + return res; } void registerTableFunctionZeros(TableFunctionFactory & factory) diff --git a/src/TableFunctions/TableFunctionZeros.h b/src/TableFunctions/TableFunctionZeros.h index 71570c23a89..0d00514c6dc 100644 --- a/src/TableFunctions/TableFunctionZeros.h +++ b/src/TableFunctions/TableFunctionZeros.h @@ -22,6 +22,11 @@ private: const char * getStorageTypeName() const override { return "SystemZeros"; } UInt64 evaluateArgument(const Context & context, ASTPtr & argument) const; + + ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; + void parseArguments(const ASTPtr & ast_function, const Context & context) const; + + mutable UInt64 length; }; From f0a5f19dae5598e5dd6f019543816ef4e37b016b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 31 Aug 2020 20:45:21 +0300 Subject: [PATCH 03/42] try use another wrapper for table functions --- src/Storages/StorageProxy.h | 158 ++++++++++++++++++ src/Storages/StorageTableFunction.h | 109 ++++++++---- src/TableFunctions/ITableFunction.cpp | 13 +- src/TableFunctions/ITableFunction.h | 4 +- src/TableFunctions/ITableFunctionFileLike.cpp | 14 +- src/TableFunctions/ITableFunctionFileLike.h | 2 +- src/TableFunctions/ITableFunctionXDBC.cpp | 15 +- src/TableFunctions/TableFunctionFile.cpp | 4 +- src/TableFunctions/TableFunctionFile.h | 2 +- .../TableFunctionGenerateRandom.cpp | 11 +- src/TableFunctions/TableFunctionHDFS.cpp | 4 +- src/TableFunctions/TableFunctionHDFS.h | 2 +- src/TableFunctions/TableFunctionInput.cpp | 12 +- src/TableFunctions/TableFunctionMerge.cpp | 12 +- src/TableFunctions/TableFunctionMySQL.cpp | 11 +- src/TableFunctions/TableFunctionNumbers.cpp | 47 ++---- src/TableFunctions/TableFunctionNumbers.h | 4 - src/TableFunctions/TableFunctionRemote.cpp | 7 +- src/TableFunctions/TableFunctionS3.cpp | 12 +- src/TableFunctions/TableFunctionURL.cpp | 5 +- src/TableFunctions/TableFunctionURL.h | 2 +- src/TableFunctions/TableFunctionValues.cpp | 12 +- src/TableFunctions/TableFunctionZeros.cpp | 41 ++--- src/TableFunctions/TableFunctionZeros.h | 3 - 24 files changed, 308 insertions(+), 198 deletions(-) create mode 100644 src/Storages/StorageProxy.h diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h new file mode 100644 index 00000000000..81edcf27c64 --- /dev/null +++ b/src/Storages/StorageProxy.h @@ -0,0 +1,158 @@ +#pragma once + +#include + + +namespace DB +{ + + +class StorageProxy : public IStorage +{ +public: + + StorageProxy(const StorageID & table_id_) : IStorage(table_id_) {} + + virtual StoragePtr getNested() const = 0; + + String getName() const override { return "StorageProxy"; } + + bool isRemote() const override { return getNested()->isRemote(); } + bool isView() const override { return getNested()->isView(); } + bool supportsSampling() const override { return getNested()->supportsSampling(); } + bool supportsFinal() const override { return getNested()->supportsFinal(); } + bool supportsPrewhere() const override { return getNested()->supportsPrewhere(); } + bool supportsReplication() const override { return getNested()->supportsReplication(); } + bool supportsParallelInsert() const override { return getNested()->supportsParallelInsert(); } + bool supportsDeduplication() const override { return getNested()->supportsDeduplication(); } + bool supportsSettings() const override { return getNested()->supportsSettings(); } + bool noPushingToViews() const override { return getNested()->noPushingToViews(); } + bool hasEvenlyDistributedRead() const override { return getNested()->hasEvenlyDistributedRead(); } + + ColumnSizeByName getColumnSizes() const override { return getNested()->getColumnSizes(); } + NamesAndTypesList getVirtuals() const override { return getNested()->getVirtuals(); } + QueryProcessingStage::Enum getQueryProcessingStage(const Context & context, QueryProcessingStage::Enum to_stage, const ASTPtr & ast) const override + { + return getNested()->getQueryProcessingStage(context, to_stage, ast); + } + + BlockInputStreams watch( + const Names & column_names, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + size_t max_block_size, + unsigned num_streams) override + { + return getNested()->watch(column_names, query_info, context, processed_stage, max_block_size, num_streams); + } + + Pipe read( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override + { + return getNested()->read(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); + } + + BlockOutputStreamPtr write( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + const Context & context) override + { + return getNested()->write(query, metadata_snapshot, context); + } + + void drop() override { getNested()->drop(); } + + void truncate( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + const Context & context, + TableExclusiveLockHolder & lock) override + { + getNested()->truncate(query, metadata_snapshot, context, lock); + } + + void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override + { + getNested()->rename(new_path_to_table_data, new_table_id); + IStorage::renameInMemory(new_table_id); + } + + void renameInMemory(const StorageID & new_table_id) override + { + getNested()->renameInMemory(new_table_id); + IStorage::renameInMemory(new_table_id); + } + + void alter(const AlterCommands & params, const Context & context, TableLockHolder & alter_lock_holder) override + { + getNested()->alter(params, context, alter_lock_holder); + IStorage::setInMemoryMetadata(getNested()->getInMemoryMetadata()); + } + + void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override + { + getNested()->checkAlterIsPossible(commands, settings); + } + + Pipe alterPartition( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + const PartitionCommands & commands, + const Context & context) override + { + return getNested()->alterPartition(query, metadata_snapshot, commands, context); + } + + void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const override + { + getNested()->checkAlterPartitionIsPossible(commands, metadata_snapshot, settings); + } + + bool optimize( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + const ASTPtr & partition, + bool final, + bool deduplicate, + const Context & context) override + { + return getNested()->optimize(query, metadata_snapshot, partition, final, deduplicate, context); + } + + void mutate(const MutationCommands & commands, const Context & context) override { getNested()->mutate(commands, context); } + + CancellationCode killMutation(const String & mutation_id) override { return getNested()->killMutation(mutation_id); } + + void startup() override { getNested()->startup(); } + void shutdown() override { getNested()->shutdown(); } + + ActionLock getActionLock(StorageActionBlockType action_type) override { return getNested()->getActionLock(action_type); } + + bool supportsIndexForIn() const override { return getNested()->supportsIndexForIn(); } + bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context, const StorageMetadataPtr & metadata_snapshot) const override + { + return getNested()->mayBenefitFromIndexForIn(left_in_operand, query_context, metadata_snapshot); + } + + CheckResults checkData(const ASTPtr & query , const Context & context) override { return getNested()->checkData(query, context); } + void checkTableCanBeDropped() const override { getNested()->checkTableCanBeDropped(); } + void checkPartitionCanBeDropped(const ASTPtr & partition) override { getNested()->checkPartitionCanBeDropped(partition); } + Strings getDataPaths() const override { return getNested()->getDataPaths(); } + StoragePolicyPtr getStoragePolicy() const override { return getNested()->getStoragePolicy(); } + std::optional totalRows() const override { return getNested()->totalRows(); } + std::optional totalBytes() const override { return getNested()->totalBytes(); } + std::optional lifetimeRows() const override { return getNested()->lifetimeRows(); } + std::optional lifetimeBytes() const override { return getNested()->lifetimeBytes(); } + +}; + + +} + diff --git a/src/Storages/StorageTableFunction.h b/src/Storages/StorageTableFunction.h index 5dbbae9d46e..319a2d37dca 100644 --- a/src/Storages/StorageTableFunction.h +++ b/src/Storages/StorageTableFunction.h @@ -2,6 +2,9 @@ #include #include #include +#include +#include +#include namespace DB { @@ -11,20 +14,53 @@ namespace ErrorCodes extern const int INCOMPATIBLE_COLUMNS; } -using GetStructureFunc = std::function; +using GetNestedStorageFunc = std::function; -template -class StorageTableFunction : public StorageT +class StorageTableFunctionProxy final : public StorageProxy { public: - - template - StorageTableFunction(GetStructureFunc get_structure_, StorageArgs && ... args) - : StorageT(std::forward(args)...), get_structure(std::move(get_structure_)) + StorageTableFunctionProxy(const StorageID & table_id_, GetNestedStorageFunc get_nested_, ColumnsDescription cached_columns) + : StorageProxy(table_id_), get_nested(std::move(get_nested_)) { + StorageInMemoryMetadata cached_metadata; + cached_metadata.setColumns(std::move(cached_columns)); + setInMemoryMetadata(cached_metadata); } - String getName() const { return "TableFunction" + StorageT::getName(); } + StoragePtr getNested() const override + { + std::lock_guard lock{nested_mutex}; + if (nested) + return nested; + + auto nested_storage = get_nested(); + nested_storage->startup(); + nested = nested_storage; + get_nested = {}; + return nested; + } + + StoragePtr maybeGetNested() const + { + std::lock_guard lock{nested_mutex}; + return nested; + } + + String getName() const override + { + std::lock_guard lock{nested_mutex}; + if (nested) + return nested->getName(); + return StorageProxy::getName(); + } + + void startup() override { } + void shutdown() override + { + auto storage = maybeGetNested(); + if (storage) + storage->shutdown(); + } Pipe read( const Names & column_names, @@ -33,38 +69,51 @@ public: const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) + unsigned num_streams) override { - assertSourceStructure(); - return StorageT::read(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); + auto storage = getNested(); + auto nested_metadata = storage->getInMemoryMetadataPtr(); + auto pipe = storage->read(column_names, nested_metadata, query_info, context, processed_stage, max_block_size, num_streams); + if (!pipe.empty()) + { + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, + metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()), + ConvertingTransform::MatchColumnsMode::Name); + }); + } + return pipe; } BlockOutputStreamPtr write( const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, - const Context & context) + const Context & context) override { - assertSourceStructure(); - return StorageT::write(query, metadata_snapshot, context); + auto storage = getNested(); + auto cached_structure = metadata_snapshot->getSampleBlock(); + auto actual_structure = storage->getInMemoryMetadataPtr()->getSampleBlock(); + if (!blocksHaveEqualStructure(actual_structure, cached_structure)) + { + throw Exception("Source storage and table function have different structure", ErrorCodes::INCOMPATIBLE_COLUMNS); + } + return storage->write(query, metadata_snapshot, context); + } + + void renameInMemory(const StorageID & new_table_id) override + { + if (maybeGetNested()) + StorageProxy::renameInMemory(new_table_id); + else + IStorage::renameInMemory(new_table_id); } private: - void assertSourceStructure() - { - if (!get_structure) - return; - - StorageInMemoryMetadata source_metadata; - source_metadata.setColumns(get_structure()); - actual_source_structure = source_metadata.getSampleBlock(); - if (!blocksHaveEqualStructure(StorageT::getInMemoryMetadataPtr()->getSampleBlock(), actual_source_structure)) - throw Exception("Source storage and table function have different structure", ErrorCodes::INCOMPATIBLE_COLUMNS); - - get_structure = {}; - } - - GetStructureFunc get_structure; - Block actual_source_structure; + mutable std::mutex nested_mutex; + mutable GetNestedStorageFunc get_nested; + mutable StoragePtr nested; }; } diff --git a/src/TableFunctions/ITableFunction.cpp b/src/TableFunctions/ITableFunction.cpp index 678f1921385..d5a78bda08a 100644 --- a/src/TableFunctions/ITableFunction.cpp +++ b/src/TableFunctions/ITableFunction.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -18,7 +19,17 @@ StoragePtr ITableFunction::execute(const ASTPtr & ast_function, const Context & ProfileEvents::increment(ProfileEvents::TableFunctionExecute); context.checkAccess(AccessType::CREATE_TEMPORARY_TABLE | StorageFactory::instance().getSourceAccessType(getStorageTypeName())); cached_columns = std::move(cached_columns_); - return executeImpl(ast_function, context, table_name); + + bool no_conversion_required = hasStaticStructure() && cached_columns == getActualTableStructure(ast_function, context); + if (cached_columns.empty() || no_conversion_required) + return executeImpl(ast_function, context, table_name); + + auto get_storage = [=, tf = shared_from_this()]() -> StoragePtr + { + return tf->executeImpl(ast_function, context, table_name); + }; + + return std::make_shared(StorageID(getDatabaseName(), table_name), std::move(get_storage), cached_columns); } } diff --git a/src/TableFunctions/ITableFunction.h b/src/TableFunctions/ITableFunction.h index 2db943225f9..be1593fcaff 100644 --- a/src/TableFunctions/ITableFunction.h +++ b/src/TableFunctions/ITableFunction.h @@ -32,7 +32,9 @@ public: /// Get the main function name. virtual std::string getName() const = 0; - virtual ColumnsDescription getActualTableStructure(const ASTPtr & /*ast_function*/, const Context & /*context*/) const { return {}; } + virtual bool hasStaticStructure() const { return false; } + + virtual ColumnsDescription getActualTableStructure(const ASTPtr & /*ast_function*/, const Context & /*context*/) const = 0; /// Create storage according to the query. StoragePtr execute(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns_ = {}) const; diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index cf52c6e7163..22c537bca2f 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -63,19 +63,9 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, const C StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { parseArguments(ast_function, context); - if (cached_columns.empty()) - cached_columns = getActualTableStructure(ast_function, context); - - auto get_structure = [=, tf = shared_from_this()]() - { - return tf->getActualTableStructure(ast_function, context); - }; - - /// Create table - StoragePtr storage = getStorage(filename, format, cached_columns, const_cast(context), table_name, compression_method, std::move(get_structure)); - + auto columns = getActualTableStructure(ast_function, context); + StoragePtr storage = getStorage(filename, format, columns, const_cast(context), table_name, compression_method); storage->startup(); - return storage; } diff --git a/src/TableFunctions/ITableFunctionFileLike.h b/src/TableFunctions/ITableFunctionFileLike.h index 56e5521ceb6..47a41c8d1bb 100644 --- a/src/TableFunctions/ITableFunctionFileLike.h +++ b/src/TableFunctions/ITableFunctionFileLike.h @@ -18,7 +18,7 @@ private: virtual StoragePtr getStorage( const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, - const std::string & table_name, const String & compression_method, GetStructureFunc get_structure) const = 0; + const std::string & table_name, const String & compression_method) const = 0; ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; diff --git a/src/TableFunctions/ITableFunctionXDBC.cpp b/src/TableFunctions/ITableFunctionXDBC.cpp index ea1f8118ce1..6f1c1f5b2db 100644 --- a/src/TableFunctions/ITableFunctionXDBC.cpp +++ b/src/TableFunctions/ITableFunctionXDBC.cpp @@ -92,19 +92,8 @@ ColumnsDescription ITableFunctionXDBC::getActualTableStructure(const ASTPtr & as StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { parseArguments(ast_function, context); - if (cached_columns.empty()) - cached_columns = getActualTableStructure(ast_function, context); - - auto get_structure = [=, tf = shared_from_this()]() - { - return tf->getActualTableStructure(ast_function, context); - }; - - auto result = std::make_shared>(get_structure, StorageID(getDatabaseName(), table_name), schema_name, remote_table_name, cached_columns, context, helper); - - if (!result) - throw Exception("Failed to instantiate storage from table function " + getName(), ErrorCodes::UNKNOWN_EXCEPTION); - + auto columns = getActualTableStructure(ast_function, context); + auto result = std::make_shared(StorageID(getDatabaseName(), table_name), schema_name, remote_table_name, columns, context, helper); result->startup(); return result; } diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index 77161d5f02e..2814c2a940b 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -10,11 +10,11 @@ namespace DB { StoragePtr TableFunctionFile::getStorage( const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, - const std::string & table_name, const std::string & compression_method, GetStructureFunc get_structure) const + const std::string & table_name, const std::string & compression_method) const { StorageFile::CommonArguments args{StorageID(getDatabaseName(), table_name), format, compression_method, columns, ConstraintsDescription{}, global_context}; - return std::make_shared>(std::move(get_structure), source, global_context.getUserFilesPath(), args); + return StorageFile::create(source, global_context.getUserFilesPath(), args); } void registerTableFunctionFile(TableFunctionFactory & factory) diff --git a/src/TableFunctions/TableFunctionFile.h b/src/TableFunctions/TableFunctionFile.h index 8a8d8d887a8..589b9028d96 100644 --- a/src/TableFunctions/TableFunctionFile.h +++ b/src/TableFunctions/TableFunctionFile.h @@ -24,6 +24,6 @@ public: private: StoragePtr getStorage( const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, - const std::string & table_name, const std::string & compression_method, GetStructureFunc get_structure) const override; + const std::string & table_name, const std::string & compression_method) const override; const char * getStorageTypeName() const override { return "File"; } };} diff --git a/src/TableFunctions/TableFunctionGenerateRandom.cpp b/src/TableFunctions/TableFunctionGenerateRandom.cpp index 086a76e0ae9..eb741ad0ba1 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.cpp +++ b/src/TableFunctions/TableFunctionGenerateRandom.cpp @@ -89,15 +89,8 @@ ColumnsDescription TableFunctionGenerateRandom::getActualTableStructure(const AS StoragePtr TableFunctionGenerateRandom::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { parseArguments(ast_function, context); - if (cached_columns.empty()) - cached_columns = getActualTableStructure(ast_function, context); - - auto get_structure = [=, tf = shared_from_this()]() - { - return tf->getActualTableStructure(ast_function, context); - }; - - auto res = std::make_shared>(get_structure, StorageID(getDatabaseName(), table_name), cached_columns, max_array_length, max_string_length, random_seed); + auto columns = getActualTableStructure(ast_function, context); + auto res = StorageGenerateRandom::create(StorageID(getDatabaseName(), table_name), columns, max_array_length, max_string_length, random_seed); res->startup(); return res; } diff --git a/src/TableFunctions/TableFunctionHDFS.cpp b/src/TableFunctions/TableFunctionHDFS.cpp index 52397caac67..da4c89b1fd1 100644 --- a/src/TableFunctions/TableFunctionHDFS.cpp +++ b/src/TableFunctions/TableFunctionHDFS.cpp @@ -11,9 +11,9 @@ namespace DB { StoragePtr TableFunctionHDFS::getStorage( const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, - const std::string & table_name, const String & compression_method, GetStructureFunc get_structure) const + const std::string & table_name, const String & compression_method) const { - return std::make_shared>(std::move(get_structure), + return StorageHDFS::create( source, StorageID(getDatabaseName(), table_name), format, diff --git a/src/TableFunctions/TableFunctionHDFS.h b/src/TableFunctions/TableFunctionHDFS.h index 91cbfd7c578..7e631d1b787 100644 --- a/src/TableFunctions/TableFunctionHDFS.h +++ b/src/TableFunctions/TableFunctionHDFS.h @@ -27,7 +27,7 @@ public: private: StoragePtr getStorage( const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, - const std::string & table_name, const String & compression_method, GetStructureFunc get_structure) const override; + const std::string & table_name, const String & compression_method) const override; const char * getStorageTypeName() const override { return "HDFS"; } }; diff --git a/src/TableFunctions/TableFunctionInput.cpp b/src/TableFunctions/TableFunctionInput.cpp index 5b36bb77b2c..4d0f8df55e6 100644 --- a/src/TableFunctions/TableFunctionInput.cpp +++ b/src/TableFunctions/TableFunctionInput.cpp @@ -52,18 +52,8 @@ ColumnsDescription TableFunctionInput::getActualTableStructure(const ASTPtr & as StoragePtr TableFunctionInput::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { parseArguments(ast_function, context); - if (cached_columns.empty()) - cached_columns = getActualTableStructure(ast_function, context); - - auto get_structure = [=, tf = shared_from_this()]() - { - return tf->getActualTableStructure(ast_function, context); - }; - - StoragePtr storage = std::make_shared>(get_structure, StorageID(getDatabaseName(), table_name), cached_columns); - + auto storage = StorageInput::create(StorageID(getDatabaseName(), table_name), getActualTableStructure(ast_function, context)); storage->startup(); - return storage; } diff --git a/src/TableFunctions/TableFunctionMerge.cpp b/src/TableFunctions/TableFunctionMerge.cpp index ffc8076b868..360ec583740 100644 --- a/src/TableFunctions/TableFunctionMerge.cpp +++ b/src/TableFunctions/TableFunctionMerge.cpp @@ -78,20 +78,14 @@ ColumnsDescription TableFunctionMerge::getActualTableStructure(const ASTPtr & as StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { parseArguments(ast_function, context); - if (cached_columns.empty()) - cached_columns = getActualTableStructure(ast_function, context); - auto get_structure = [=, tf = shared_from_this()]() - { - return tf->getActualTableStructure(ast_function, context); - }; - - auto res = std::make_shared>(std::move(get_structure), + auto res = StorageMerge::create( StorageID(getDatabaseName(), table_name), - cached_columns, + getActualTableStructure(ast_function, context), source_database, table_name_regexp, context); + res->startup(); return res; } diff --git a/src/TableFunctions/TableFunctionMySQL.cpp b/src/TableFunctions/TableFunctionMySQL.cpp index 66496778460..bff2842c8c1 100644 --- a/src/TableFunctions/TableFunctionMySQL.cpp +++ b/src/TableFunctions/TableFunctionMySQL.cpp @@ -124,24 +124,19 @@ ColumnsDescription TableFunctionMySQL::getActualTableStructure(const ASTPtr & as StoragePtr TableFunctionMySQL::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { parseArguments(ast_function, context); - if (cached_columns.empty()) - cached_columns = getActualTableStructure(ast_function, context); if (!pool) pool.emplace(remote_database_name, parsed_host_port.first, user_name, password, parsed_host_port.second); - auto get_structure = [=, tf = shared_from_this()]() - { - return tf->getActualTableStructure(ast_function, context); - }; + auto columns = getActualTableStructure(ast_function, context); - auto res = std::make_shared>(std::move(get_structure), + auto res = StorageMySQL::create( StorageID(getDatabaseName(), table_name), std::move(*pool), remote_database_name, remote_table_name, replace_query, on_duplicate_clause, - cached_columns, + columns, ConstraintsDescription{}, context); diff --git a/src/TableFunctions/TableFunctionNumbers.cpp b/src/TableFunctions/TableFunctionNumbers.cpp index 256ec47b21d..03756b37b07 100644 --- a/src/TableFunctions/TableFunctionNumbers.cpp +++ b/src/TableFunctions/TableFunctionNumbers.cpp @@ -20,29 +20,6 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -template -void TableFunctionNumbers::parseArguments(const ASTPtr & ast_function, const Context & context) const -{ - - - if (const auto * function = ast_function->as()) - { - auto arguments = function->arguments->children; - - if (arguments.size() != 1 && arguments.size() != 2) - throw Exception("Table function '" + getName() + "' requires 'length' or 'offset, length'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - if (arguments.size() == 1) - length = evaluateArgument(context, arguments[0]); - else - { - offset = evaluateArgument(context, arguments[0]); - length = evaluateArgument(context, arguments[1]); - } - } - else - throw Exception("Table function '" + getName() + "' requires 'limit' or 'offset, limit'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); -} template ColumnsDescription TableFunctionNumbers::getActualTableStructure(const ASTPtr & /*ast_function*/, const Context & /*context*/) const @@ -53,19 +30,21 @@ ColumnsDescription TableFunctionNumbers::getActualTableStructure( template StoragePtr TableFunctionNumbers::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { - parseArguments(ast_function, context); - - if (cached_columns.empty()) - cached_columns = getActualTableStructure(ast_function, context); - - auto get_structure = [=, tf = shared_from_this()]() + if (const auto * function = ast_function->as()) { - return tf->getActualTableStructure(ast_function, context); - }; + auto arguments = function->arguments->children; - auto res = std::make_shared>(get_structure, StorageID(getDatabaseName(), table_name), multithreaded, length, offset, false); - res->startup(); - return res; + if (arguments.size() != 1 && arguments.size() != 2) + throw Exception("Table function '" + getName() + "' requires 'length' or 'offset, length'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + UInt64 offset = arguments.size() == 2 ? evaluateArgument(context, arguments[0]) : 0; + UInt64 length = arguments.size() == 2 ? evaluateArgument(context, arguments[1]) : evaluateArgument(context, arguments[0]); + + auto res = StorageSystemNumbers::create(StorageID(getDatabaseName(), table_name), multithreaded, length, offset, false); + res->startup(); + return res; + } + throw Exception("Table function '" + getName() + "' requires 'limit' or 'offset, limit'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } void registerTableFunctionNumbers(TableFunctionFactory & factory) diff --git a/src/TableFunctions/TableFunctionNumbers.h b/src/TableFunctions/TableFunctionNumbers.h index 5d12eefd21c..a3c4f6f13cb 100644 --- a/src/TableFunctions/TableFunctionNumbers.h +++ b/src/TableFunctions/TableFunctionNumbers.h @@ -24,10 +24,6 @@ private: UInt64 evaluateArgument(const Context & context, ASTPtr & argument) const; ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; - void parseArguments(const ASTPtr & ast_function, const Context & context) const; - - mutable UInt64 offset = 0; - mutable UInt64 length; }; diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index cc5485189d0..3d00d0ad897 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -201,14 +201,11 @@ void TableFunctionRemote::prepareClusterInfo(const ASTPtr & ast_function, const StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { prepareClusterInfo(ast_function, context); - if (cached_columns.empty()) - cached_columns = getStructureOfRemoteTable(*cluster, remote_table_id, context, remote_table_function_ptr); - //auto structure_remote_table = getStructureOfRemoteTable(*cluster, remote_table_id, context, remote_table_function_ptr); StoragePtr res = remote_table_function_ptr ? StorageDistributed::create( StorageID(getDatabaseName(), table_name), - cached_columns, + getActualTableStructure(ast_function, context), ConstraintsDescription{}, remote_table_function_ptr, String{}, @@ -220,7 +217,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C cluster) : StorageDistributed::create( StorageID(getDatabaseName(), table_name), - cached_columns, + getActualTableStructure(ast_function, context), ConstraintsDescription{}, remote_table_id.database_name, remote_table_id.table_name, diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 4c2779f72d0..b4fe1baffce 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -70,26 +70,18 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & ast_function, const Conte { parseArguments(ast_function, context); - if (cached_columns.empty()) - cached_columns = getActualTableStructure(ast_function, context); - - auto get_structure = [=, tf = shared_from_this()]() - { - return tf->getActualTableStructure(ast_function, context); - }; - Poco::URI uri (filename); S3::URI s3_uri (uri); UInt64 min_upload_part_size = context.getSettingsRef().s3_min_upload_part_size; - StoragePtr storage = std::make_shared>(std::move(get_structure), + StoragePtr storage = StorageS3::create( s3_uri, access_key_id, secret_access_key, StorageID(getDatabaseName(), table_name), format, min_upload_part_size, - cached_columns, + getActualTableStructure(ast_function, context), ConstraintsDescription{}, const_cast(context), compression_method); diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index 74cab6e9f00..69395c9140e 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -11,11 +11,10 @@ namespace DB { StoragePtr TableFunctionURL::getStorage( const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, - const std::string & table_name, const String & compression_method, GetStructureFunc get_structure) const + const std::string & table_name, const String & compression_method) const { Poco::URI uri(source); - return std::make_shared>(std::move(get_structure), uri, - StorageID(getDatabaseName(), table_name), format, columns, ConstraintsDescription{}, + return StorageURL::create( uri, StorageID(getDatabaseName(), table_name), format, columns, ConstraintsDescription{}, global_context, compression_method); } diff --git a/src/TableFunctions/TableFunctionURL.h b/src/TableFunctions/TableFunctionURL.h index d69b7ff33a1..4760e0c70f0 100644 --- a/src/TableFunctions/TableFunctionURL.h +++ b/src/TableFunctions/TableFunctionURL.h @@ -22,7 +22,7 @@ public: private: StoragePtr getStorage( const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, - const std::string & table_name, const String & compression_method, GetStructureFunc get_structure) const override; + const std::string & table_name, const String & compression_method) const override; const char * getStorageTypeName() const override { return "URL"; } }; diff --git a/src/TableFunctions/TableFunctionValues.cpp b/src/TableFunctions/TableFunctionValues.cpp index ca1d8b51b22..e51e28abff3 100644 --- a/src/TableFunctions/TableFunctionValues.cpp +++ b/src/TableFunctions/TableFunctionValues.cpp @@ -100,16 +100,10 @@ StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, const C { parseArguments(ast_function, context); - if (cached_columns.empty()) - cached_columns = getActualTableStructure(ast_function, context); - - auto get_structure = [=, tf = shared_from_this()]() - { - return tf->getActualTableStructure(ast_function, context); - }; + auto columns = getActualTableStructure(ast_function, context); Block sample_block; - for (const auto & name_type : cached_columns.getOrdinary()) + for (const auto & name_type : columns.getOrdinary()) sample_block.insert({ name_type.type->createColumn(), name_type.type, name_type.name }); MutableColumns res_columns = sample_block.cloneEmptyColumns(); @@ -121,7 +115,7 @@ StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, const C Block res_block = sample_block.cloneWithColumns(std::move(res_columns)); - auto res = std::make_shared>(get_structure, StorageID(getDatabaseName(), table_name), cached_columns, res_block); + auto res = StorageValues::create(StorageID(getDatabaseName(), table_name), columns, res_block); res->startup(); return res; } diff --git a/src/TableFunctions/TableFunctionZeros.cpp b/src/TableFunctions/TableFunctionZeros.cpp index 56413431c37..b633260861c 100644 --- a/src/TableFunctions/TableFunctionZeros.cpp +++ b/src/TableFunctions/TableFunctionZeros.cpp @@ -19,23 +19,6 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -template -void TableFunctionZeros::parseArguments(const ASTPtr & ast_function, const Context & context) const -{ - - - if (const auto * function = ast_function->as()) - { - auto arguments = function->arguments->children; - - if (arguments.size() != 1) - throw Exception("Table function '" + getName() + "' requires 'length'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - length = evaluateArgument(context, arguments[0]); - } - else - throw Exception("Table function '" + getName() + "' requires 'limit'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); -} template ColumnsDescription TableFunctionZeros::getActualTableStructure(const ASTPtr & /*ast_function*/, const Context & /*context*/) const @@ -46,19 +29,21 @@ ColumnsDescription TableFunctionZeros::getActualTableStructure(co template StoragePtr TableFunctionZeros::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { - parseArguments(ast_function, context); - - if (cached_columns.empty()) - cached_columns = getActualTableStructure(ast_function, context); - - auto get_structure = [=, tf = shared_from_this()]() + if (const auto * function = ast_function->as()) { - return tf->getActualTableStructure(ast_function, context); - }; + auto arguments = function->arguments->children; - auto res = std::make_shared>(std::move(get_structure), StorageID(getDatabaseName(), table_name), multithreaded, length); - res->startup(); - return res; + if (arguments.size() != 1) + throw Exception("Table function '" + getName() + "' requires 'length'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + + UInt64 length = evaluateArgument(context, arguments[0]); + + auto res = StorageSystemZeros::create(StorageID(getDatabaseName(), table_name), multithreaded, length); + res->startup(); + return res; + } + throw Exception("Table function '" + getName() + "' requires 'limit'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } void registerTableFunctionZeros(TableFunctionFactory & factory) diff --git a/src/TableFunctions/TableFunctionZeros.h b/src/TableFunctions/TableFunctionZeros.h index 0d00514c6dc..aba161af4d0 100644 --- a/src/TableFunctions/TableFunctionZeros.h +++ b/src/TableFunctions/TableFunctionZeros.h @@ -24,9 +24,6 @@ private: UInt64 evaluateArgument(const Context & context, ASTPtr & argument) const; ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; - void parseArguments(const ASTPtr & ast_function, const Context & context) const; - - mutable UInt64 length; }; From b6093d9a86aaf89362194be997690288d19afc8a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 1 Sep 2020 13:37:42 +0300 Subject: [PATCH 04/42] fix query header --- src/Storages/StorageMerge.cpp | 7 ++++--- src/Storages/StorageMerge.h | 15 ++++++++------- src/Storages/StorageTableFunction.h | 15 ++++++++++++++- src/TableFunctions/TableFunctionFile.cpp | 6 +++--- src/TableFunctions/TableFunctionFile.h | 4 ++-- src/TableFunctions/TableFunctionHDFS.cpp | 8 ++++---- src/TableFunctions/TableFunctionHDFS.h | 4 ++-- src/TableFunctions/TableFunctionURL.cpp | 8 ++++---- src/TableFunctions/TableFunctionURL.h | 4 ++-- 9 files changed, 43 insertions(+), 28 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 2d96a59392b..9ca93288197 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -178,7 +178,7 @@ Pipe StorageMerge::read( modified_context->setSetting("optimize_move_to_prewhere", false); /// What will be result structure depending on query processed stage in source tables? - Block header = getQueryHeader(column_names, metadata_snapshot, query_info, context, processed_stage); + Block header = getQueryHeader(*this, column_names, metadata_snapshot, query_info, context, processed_stage); /** First we make list of selected tables to find out its size. * This is necessary to correctly pass the recommended number of threads to each table. @@ -431,6 +431,7 @@ void StorageMerge::alter( } Block StorageMerge::getQueryHeader( + const IStorage & storage, const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -441,7 +442,7 @@ Block StorageMerge::getQueryHeader( { case QueryProcessingStage::FetchColumns: { - Block header = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); + Block header = metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID()); if (query_info.prewhere_info) { query_info.prewhere_info->prewhere_actions->execute(header); @@ -457,7 +458,7 @@ Block StorageMerge::getQueryHeader( removeJoin(*query->as()); auto stream = std::make_shared( - metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID())); + metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID())); return InterpreterSelectQuery(query, context, stream, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); } } diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index ad898ba2ffd..992fe309c02 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -47,6 +47,14 @@ public: bool mayBenefitFromIndexForIn( const ASTPtr & left_in_operand, const Context & query_context, const StorageMetadataPtr & metadata_snapshot) const override; + static Block getQueryHeader( + const IStorage & storage, + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage); + private: String source_database; OptimizedRegularExpression table_name_regexp; @@ -75,13 +83,6 @@ protected: const String & table_name_regexp_, const Context & context_); - Block getQueryHeader( - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum processed_stage); - Pipe createSources( const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageTableFunction.h b/src/Storages/StorageTableFunction.h index 319a2d37dca..8aa297a1e7a 100644 --- a/src/Storages/StorageTableFunction.h +++ b/src/Storages/StorageTableFunction.h @@ -6,6 +6,9 @@ #include #include +//#include +#include + namespace DB { @@ -25,14 +28,17 @@ public: StorageInMemoryMetadata cached_metadata; cached_metadata.setColumns(std::move(cached_columns)); setInMemoryMetadata(cached_metadata); + //log = &Poco::Logger::get("TABLE_FUNCTION_PROXY"); } StoragePtr getNested() const override { + //LOG_WARNING(log, "getNested()"); std::lock_guard lock{nested_mutex}; if (nested) return nested; + //LOG_WARNING(log, "getNested() creating"); auto nested_storage = get_nested(); nested_storage->startup(); nested = nested_storage; @@ -71,6 +77,10 @@ public: size_t max_block_size, unsigned num_streams) override { + String cnames; + for (const auto & c : column_names) + cnames += c + " "; + //LOG_WARNING(log, "read() {} cols: {}", QueryProcessingStage::toString(processed_stage), cnames); auto storage = getNested(); auto nested_metadata = storage->getInMemoryMetadataPtr(); auto pipe = storage->read(column_names, nested_metadata, query_info, context, processed_stage, max_block_size, num_streams); @@ -78,9 +88,11 @@ public: { pipe.addSimpleTransform([&](const Block & header) { + auto to = StorageMerge::getQueryHeader(*this, column_names, metadata_snapshot, query_info, context, processed_stage); + //LOG_WARNING(log, "try convert \n{}\n to \n{}\n", header.dumpStructure(), to.dumpStructure()); return std::make_shared( header, - metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()), + to, ConvertingTransform::MatchColumnsMode::Name); }); } @@ -114,6 +126,7 @@ private: mutable std::mutex nested_mutex; mutable GetNestedStorageFunc get_nested; mutable StoragePtr nested; + //mutable Poco::Logger * log; }; } diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index 2814c2a940b..39de6dce92c 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -9,10 +9,10 @@ namespace DB { StoragePtr TableFunctionFile::getStorage( - const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, - const std::string & table_name, const std::string & compression_method) const + const String & source, const String & format_, const ColumnsDescription & columns, Context & global_context, + const std::string & table_name, const std::string & compression_method_) const { - StorageFile::CommonArguments args{StorageID(getDatabaseName(), table_name), format, compression_method, columns, ConstraintsDescription{}, global_context}; + StorageFile::CommonArguments args{StorageID(getDatabaseName(), table_name), format_, compression_method_, columns, ConstraintsDescription{}, global_context}; return StorageFile::create(source, global_context.getUserFilesPath(), args); } diff --git a/src/TableFunctions/TableFunctionFile.h b/src/TableFunctions/TableFunctionFile.h index 589b9028d96..91e2225a6d0 100644 --- a/src/TableFunctions/TableFunctionFile.h +++ b/src/TableFunctions/TableFunctionFile.h @@ -23,7 +23,7 @@ public: private: StoragePtr getStorage( - const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, - const std::string & table_name, const std::string & compression_method) const override; + const String & source, const String & format_, const ColumnsDescription & columns, Context & global_context, + const std::string & table_name, const std::string & compression_method_) const override; const char * getStorageTypeName() const override { return "File"; } };} diff --git a/src/TableFunctions/TableFunctionHDFS.cpp b/src/TableFunctions/TableFunctionHDFS.cpp index da4c89b1fd1..e2f227ef7b5 100644 --- a/src/TableFunctions/TableFunctionHDFS.cpp +++ b/src/TableFunctions/TableFunctionHDFS.cpp @@ -10,17 +10,17 @@ namespace DB { StoragePtr TableFunctionHDFS::getStorage( - const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, - const std::string & table_name, const String & compression_method) const + const String & source, const String & format_, const ColumnsDescription & columns, Context & global_context, + const std::string & table_name, const String & compression_method_) const { return StorageHDFS::create( source, StorageID(getDatabaseName(), table_name), - format, + format_, columns, ConstraintsDescription{}, global_context, - compression_method); + compression_method_); } diff --git a/src/TableFunctions/TableFunctionHDFS.h b/src/TableFunctions/TableFunctionHDFS.h index 7e631d1b787..47e040f7593 100644 --- a/src/TableFunctions/TableFunctionHDFS.h +++ b/src/TableFunctions/TableFunctionHDFS.h @@ -26,8 +26,8 @@ public: private: StoragePtr getStorage( - const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, - const std::string & table_name, const String & compression_method) const override; + const String & source, const String & format_, const ColumnsDescription & columns, Context & global_context, + const std::string & table_name, const String & compression_method_) const override; const char * getStorageTypeName() const override { return "HDFS"; } }; diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index 69395c9140e..e867fc836f7 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -10,12 +10,12 @@ namespace DB { StoragePtr TableFunctionURL::getStorage( - const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, - const std::string & table_name, const String & compression_method) const + const String & source, const String & format_, const ColumnsDescription & columns, Context & global_context, + const std::string & table_name, const String & compression_method_) const { Poco::URI uri(source); - return StorageURL::create( uri, StorageID(getDatabaseName(), table_name), format, columns, ConstraintsDescription{}, - global_context, compression_method); + return StorageURL::create( uri, StorageID(getDatabaseName(), table_name), format_, columns, ConstraintsDescription{}, + global_context, compression_method_); } void registerTableFunctionURL(TableFunctionFactory & factory) diff --git a/src/TableFunctions/TableFunctionURL.h b/src/TableFunctions/TableFunctionURL.h index 4760e0c70f0..5eb027e2b8a 100644 --- a/src/TableFunctions/TableFunctionURL.h +++ b/src/TableFunctions/TableFunctionURL.h @@ -21,8 +21,8 @@ public: private: StoragePtr getStorage( - const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, - const std::string & table_name, const String & compression_method) const override; + const String & source, const String & format_, const ColumnsDescription & columns, Context & global_context, + const std::string & table_name, const String & compression_method_) const override; const char * getStorageTypeName() const override { return "URL"; } }; From 3a9779adb437e749433e99ac484a5033fd276163 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 1 Sep 2020 17:41:49 +0300 Subject: [PATCH 05/42] slightly better code --- src/Databases/DatabaseOnDisk.cpp | 3 +- .../ClusterProxy/SelectStreamFactory.cpp | 3 +- src/Interpreters/Context.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 9 ++- src/Interpreters/InterpreterDescribeQuery.cpp | 23 +++---- src/Interpreters/InterpreterInsertQuery.cpp | 3 +- .../getHeaderForProcessingStage.cpp | 66 +++++++++++++++++++ .../getHeaderForProcessingStage.h | 27 ++++++++ src/Storages/StorageFile.h | 2 +- src/Storages/StorageGenerateRandom.h | 2 +- src/Storages/StorageHDFS.h | 2 +- src/Storages/StorageInput.h | 2 +- src/Storages/StorageMerge.cpp | 59 +---------------- src/Storages/StorageMerge.h | 10 +-- src/Storages/StorageMySQL.h | 2 +- src/Storages/StorageTableFunction.h | 28 +++----- src/Storages/StorageURL.h | 2 +- src/Storages/StorageValues.h | 2 +- src/Storages/System/StorageSystemNumbers.h | 2 +- src/Storages/System/StorageSystemZeros.h | 2 +- src/Storages/getStructureOfRemoteTable.cpp | 6 +- src/TableFunctions/ITableFunction.cpp | 7 +- src/TableFunctions/ITableFunction.h | 7 +- src/TableFunctions/ITableFunctionFileLike.cpp | 13 ++-- src/TableFunctions/ITableFunctionFileLike.h | 12 ++-- src/TableFunctions/ITableFunctionXDBC.cpp | 16 ++--- src/TableFunctions/ITableFunctionXDBC.h | 12 ++-- src/TableFunctions/TableFunctionFactory.cpp | 13 ++-- src/TableFunctions/TableFunctionFactory.h | 2 +- .../TableFunctionGenerateRandom.cpp | 13 ++-- .../TableFunctionGenerateRandom.h | 12 ++-- src/TableFunctions/TableFunctionInput.cpp | 13 ++-- src/TableFunctions/TableFunctionInput.h | 6 +- src/TableFunctions/TableFunctionMerge.cpp | 11 ++-- src/TableFunctions/TableFunctionMerge.h | 8 +-- src/TableFunctions/TableFunctionMySQL.cpp | 12 ++-- src/TableFunctions/TableFunctionMySQL.h | 18 ++--- src/TableFunctions/TableFunctionNumbers.cpp | 2 +- src/TableFunctions/TableFunctionNumbers.h | 2 +- src/TableFunctions/TableFunctionRemote.cpp | 18 ++--- src/TableFunctions/TableFunctionRemote.h | 10 +-- src/TableFunctions/TableFunctionS3.cpp | 13 ++-- src/TableFunctions/TableFunctionS3.h | 16 ++--- src/TableFunctions/TableFunctionURL.cpp | 2 +- src/TableFunctions/TableFunctionValues.cpp | 9 +-- src/TableFunctions/TableFunctionValues.h | 6 +- src/TableFunctions/TableFunctionZeros.cpp | 2 +- src/TableFunctions/TableFunctionZeros.h | 2 +- 48 files changed, 243 insertions(+), 271 deletions(-) create mode 100644 src/Interpreters/getHeaderForProcessingStage.cpp create mode 100644 src/Interpreters/getHeaderForProcessingStage.h diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index fdfc185f920..a563f6262fc 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -54,9 +54,8 @@ std::pair createTableFromAST( if (ast_create_query.as_table_function) { - const auto & ast_table_function = ast_create_query.as_table_function->as(); const auto & factory = TableFunctionFactory::instance(); - auto table_function = factory.get(ast_table_function.name, context); + auto table_function = factory.get(ast_create_query.as_table_function, context); ColumnsDescription columns; if (ast_create_query.columns_list && ast_create_query.columns_list->columns) columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context, false); diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 986de85d712..9754fa7ad73 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -154,8 +154,7 @@ void SelectStreamFactory::createForShard( if (table_func_ptr) { - const auto * table_function = table_func_ptr->as(); - TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_function->name, context); + TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_func_ptr, context); main_table_storage = table_function_ptr->execute(table_func_ptr, context, table_function_ptr->getName()); } else diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 70cf41a679c..2cd65a62d25 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -916,7 +916,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression) if (!res) { - TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression->as()->name, *this); + TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression, *this); /// Run it and remember the result res = table_function_ptr->execute(table_expression, *this, table_function_ptr->getName()); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index dd5fb4d063a..c6ad6c5bb2b 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -497,9 +497,9 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(AS else if (create.as_table_function) { /// Table function without columns list. - auto table_function = TableFunctionFactory::instance().get(create.as_table_function->as().name, context); - properties.columns = table_function->getActualTableStructure(create.as_table_function, context); - if (properties.columns.empty()) //FIXME + auto table_function = TableFunctionFactory::instance().get(create.as_table_function, context); + properties.columns = table_function->getActualTableStructure(context); + if (properties.columns.empty()) //FIXME TableFunctionFile may return empty structure for Distributed format return {}; } else @@ -768,9 +768,8 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, /// NOTE: CREATE query may be rewritten by Storage creator or table function if (create.as_table_function) { - const auto & table_function = create.as_table_function->as(); const auto & factory = TableFunctionFactory::instance(); - res = factory.get(table_function.name, context)->execute(create.as_table_function, context, create.table, properties.columns); + res = factory.get(create.as_table_function, context)->execute(create.as_table_function, context, create.table, properties.columns); res->renameInMemory({create.database, create.table, create.uuid}); } else diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 94fa748ea15..b540290a653 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -72,23 +72,16 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() table_expression.subquery->children.at(0), context).getNamesAndTypesList(); columns = ColumnsDescription(std::move(names_and_types)); } + else if (table_expression.table_function) + { + TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression.table_function, context); + columns = table_function_ptr->getActualTableStructure(context); + } else { - StoragePtr table; - if (table_expression.table_function) - { - const auto & table_function = table_expression.table_function->as(); - TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_function.name, context); - /// Run the table function and remember the result - table = table_function_ptr->execute(table_expression.table_function, context, table_function_ptr->getName()); - } - else - { - auto table_id = context.resolveStorageID(table_expression.database_and_table_name); - context.checkAccess(AccessType::SHOW_COLUMNS, table_id); - table = DatabaseCatalog::instance().getTable(table_id, context); - } - + auto table_id = context.resolveStorageID(table_expression.database_and_table_name); + context.checkAccess(AccessType::SHOW_COLUMNS, table_id); + auto table = DatabaseCatalog::instance().getTable(table_id, context); auto table_lock = table->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout); auto metadata_snapshot = table->getInMemoryMetadataPtr(); columns = metadata_snapshot->getColumns(); diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 1557f065b32..b1ee7cfe0b1 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -59,9 +59,8 @@ StoragePtr InterpreterInsertQuery::getTable(ASTInsertQuery & query) { if (query.table_function) { - const auto * table_function = query.table_function->as(); const auto & factory = TableFunctionFactory::instance(); - TableFunctionPtr table_function_ptr = factory.get(table_function->name, context); + TableFunctionPtr table_function_ptr = factory.get(query.table_function, context); return table_function_ptr->execute(query.table_function, context, table_function_ptr->getName()); } diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp new file mode 100644 index 00000000000..d034f900842 --- /dev/null +++ b/src/Interpreters/getHeaderForProcessingStage.cpp @@ -0,0 +1,66 @@ +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +/// Rewrite original query removing joined tables from it +void removeJoin(const ASTSelectQuery & select) +{ + const auto & tables = select.tables(); + if (!tables || tables->children.size() < 2) + return; + + const auto & joined_table = tables->children[1]->as(); + if (!joined_table.table_join) + return; + + /// The most simple temporary solution: leave only the first table in query. + /// TODO: we also need to remove joined columns and related functions (taking in account aliases if any). + tables->children.resize(1); +} + +Block getHeaderForProcessingStage( + const IStorage & storage, + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage) +{ + switch (processed_stage) + { + case QueryProcessingStage::FetchColumns: + { + Block header = metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID()); + if (query_info.prewhere_info) + { + query_info.prewhere_info->prewhere_actions->execute(header); + if (query_info.prewhere_info->remove_prewhere_column) + header.erase(query_info.prewhere_info->prewhere_column_name); + } + return header; + } + case QueryProcessingStage::WithMergeableState: + case QueryProcessingStage::Complete: + { + auto query = query_info.query->clone(); + removeJoin(*query->as()); + + auto stream = std::make_shared( + metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID())); + return InterpreterSelectQuery(query, context, stream, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + } + } + throw Exception("Logical Error: unknown processed stage.", ErrorCodes::LOGICAL_ERROR); +} + +} + diff --git a/src/Interpreters/getHeaderForProcessingStage.h b/src/Interpreters/getHeaderForProcessingStage.h new file mode 100644 index 00000000000..a4ebcc1db7d --- /dev/null +++ b/src/Interpreters/getHeaderForProcessingStage.h @@ -0,0 +1,27 @@ +#pragma once +#include +#include +#include + + +namespace DB +{ + +class IStorage; +struct StorageInMemoryMetadata; +using StorageMetadataPtr = std::shared_ptr; +struct SelectQueryInfo; +class Context; +class ASTSelectQuery; + +void removeJoin(const ASTSelectQuery & select); + +Block getHeaderForProcessingStage( + const IStorage & storage, + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage); + +} diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 872a0fc163f..ea70dcd5311 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -18,7 +18,7 @@ namespace DB class StorageFileBlockInputStream; class StorageFileBlockOutputStream; -class StorageFile : public ext::shared_ptr_helper, public IStorage +class StorageFile final : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; public: diff --git a/src/Storages/StorageGenerateRandom.h b/src/Storages/StorageGenerateRandom.h index 09208e9c5cd..e0f037f9a08 100644 --- a/src/Storages/StorageGenerateRandom.h +++ b/src/Storages/StorageGenerateRandom.h @@ -9,7 +9,7 @@ namespace DB { /* Generates random data for given schema. */ -class StorageGenerateRandom : public ext::shared_ptr_helper, public IStorage +class StorageGenerateRandom final : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; public: diff --git a/src/Storages/StorageHDFS.h b/src/Storages/StorageHDFS.h index ecb77ef83af..fdeaf4ae1b3 100644 --- a/src/Storages/StorageHDFS.h +++ b/src/Storages/StorageHDFS.h @@ -13,7 +13,7 @@ namespace DB * This class represents table engine for external hdfs files. * Read method is supported for now. */ -class StorageHDFS : public ext::shared_ptr_helper, public IStorage +class StorageHDFS final : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; public: diff --git a/src/Storages/StorageInput.h b/src/Storages/StorageInput.h index 40b0fd39139..c19b19e4703 100644 --- a/src/Storages/StorageInput.h +++ b/src/Storages/StorageInput.h @@ -8,7 +8,7 @@ namespace DB /** Internal temporary storage for table function input(...) */ -class StorageInput : public ext::shared_ptr_helper, public IStorage +class StorageInput final : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; public: diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 9ca93288197..9caec05e2f4 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -37,27 +38,6 @@ namespace ErrorCodes extern const int SAMPLING_NOT_SUPPORTED; } -namespace -{ - -/// Rewrite original query removing joined tables from it -void removeJoin(const ASTSelectQuery & select) -{ - const auto & tables = select.tables(); - if (!tables || tables->children.size() < 2) - return; - - const auto & joined_table = tables->children[1]->as(); - if (!joined_table.table_join) - return; - - /// The most simple temporary solution: leave only the first table in query. - /// TODO: we also need to remove joined columns and related functions (taking in account aliases if any). - tables->children.resize(1); -} - -} - StorageMerge::StorageMerge( const StorageID & table_id_, @@ -178,7 +158,7 @@ Pipe StorageMerge::read( modified_context->setSetting("optimize_move_to_prewhere", false); /// What will be result structure depending on query processed stage in source tables? - Block header = getQueryHeader(*this, column_names, metadata_snapshot, query_info, context, processed_stage); + Block header = getHeaderForProcessingStage(*this, column_names, metadata_snapshot, query_info, context, processed_stage); /** First we make list of selected tables to find out its size. * This is necessary to correctly pass the recommended number of threads to each table. @@ -430,41 +410,6 @@ void StorageMerge::alter( setInMemoryMetadata(storage_metadata); } -Block StorageMerge::getQueryHeader( - const IStorage & storage, - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum processed_stage) -{ - switch (processed_stage) - { - case QueryProcessingStage::FetchColumns: - { - Block header = metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID()); - if (query_info.prewhere_info) - { - query_info.prewhere_info->prewhere_actions->execute(header); - if (query_info.prewhere_info->remove_prewhere_column) - header.erase(query_info.prewhere_info->prewhere_column_name); - } - return header; - } - case QueryProcessingStage::WithMergeableState: - case QueryProcessingStage::Complete: - { - auto query = query_info.query->clone(); - removeJoin(*query->as()); - - auto stream = std::make_shared( - metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID())); - return InterpreterSelectQuery(query, context, stream, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); - } - } - throw Exception("Logical Error: unknown processed stage.", ErrorCodes::LOGICAL_ERROR); -} - void StorageMerge::convertingSourceStream( const Block & header, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 992fe309c02..63d803cedf1 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -13,7 +13,7 @@ namespace DB /** A table that represents the union of an arbitrary number of other tables. * All tables must have the same structure. */ -class StorageMerge : public ext::shared_ptr_helper, public IStorage +class StorageMerge final : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; public: @@ -47,14 +47,6 @@ public: bool mayBenefitFromIndexForIn( const ASTPtr & left_in_operand, const Context & query_context, const StorageMetadataPtr & metadata_snapshot) const override; - static Block getQueryHeader( - const IStorage & storage, - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum processed_stage); - private: String source_database; OptimizedRegularExpression table_name_regexp; diff --git a/src/Storages/StorageMySQL.h b/src/Storages/StorageMySQL.h index fedd8467210..a7f98c4379b 100644 --- a/src/Storages/StorageMySQL.h +++ b/src/Storages/StorageMySQL.h @@ -20,7 +20,7 @@ namespace DB * Use ENGINE = mysql(host_port, database_name, table_name, user_name, password) * Read only. */ -class StorageMySQL : public ext::shared_ptr_helper, public IStorage +class StorageMySQL final : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; public: diff --git a/src/Storages/StorageTableFunction.h b/src/Storages/StorageTableFunction.h index 8aa297a1e7a..530e614c856 100644 --- a/src/Storages/StorageTableFunction.h +++ b/src/Storages/StorageTableFunction.h @@ -5,9 +5,8 @@ #include #include #include +#include -//#include -#include namespace DB { @@ -28,17 +27,14 @@ public: StorageInMemoryMetadata cached_metadata; cached_metadata.setColumns(std::move(cached_columns)); setInMemoryMetadata(cached_metadata); - //log = &Poco::Logger::get("TABLE_FUNCTION_PROXY"); } StoragePtr getNested() const override { - //LOG_WARNING(log, "getNested()"); std::lock_guard lock{nested_mutex}; if (nested) return nested; - //LOG_WARNING(log, "getNested() creating"); auto nested_storage = get_nested(); nested_storage->startup(); nested = nested_storage; @@ -46,12 +42,6 @@ public: return nested; } - StoragePtr maybeGetNested() const - { - std::lock_guard lock{nested_mutex}; - return nested; - } - String getName() const override { std::lock_guard lock{nested_mutex}; @@ -63,9 +53,9 @@ public: void startup() override { } void shutdown() override { - auto storage = maybeGetNested(); - if (storage) - storage->shutdown(); + std::lock_guard lock{nested_mutex}; + if (nested) + nested->shutdown(); } Pipe read( @@ -80,19 +70,17 @@ public: String cnames; for (const auto & c : column_names) cnames += c + " "; - //LOG_WARNING(log, "read() {} cols: {}", QueryProcessingStage::toString(processed_stage), cnames); auto storage = getNested(); auto nested_metadata = storage->getInMemoryMetadataPtr(); auto pipe = storage->read(column_names, nested_metadata, query_info, context, processed_stage, max_block_size, num_streams); if (!pipe.empty()) { + auto to_header = getHeaderForProcessingStage(*this, column_names, metadata_snapshot, query_info, context, processed_stage); pipe.addSimpleTransform([&](const Block & header) { - auto to = StorageMerge::getQueryHeader(*this, column_names, metadata_snapshot, query_info, context, processed_stage); - //LOG_WARNING(log, "try convert \n{}\n to \n{}\n", header.dumpStructure(), to.dumpStructure()); return std::make_shared( header, - to, + to_header, ConvertingTransform::MatchColumnsMode::Name); }); } @@ -116,7 +104,8 @@ public: void renameInMemory(const StorageID & new_table_id) override { - if (maybeGetNested()) + std::lock_guard lock{nested_mutex}; + if (nested) StorageProxy::renameInMemory(new_table_id); else IStorage::renameInMemory(new_table_id); @@ -126,7 +115,6 @@ private: mutable std::mutex nested_mutex; mutable GetNestedStorageFunc get_nested; mutable StoragePtr nested; - //mutable Poco::Logger * log; }; } diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 69fe19650eb..7983ad71520 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -93,7 +93,7 @@ private: BlockOutputStreamPtr writer; }; -class StorageURL : public ext::shared_ptr_helper, public IStorageURLBase +class StorageURL final : public ext::shared_ptr_helper, public IStorageURLBase { friend struct ext::shared_ptr_helper; public: diff --git a/src/Storages/StorageValues.h b/src/Storages/StorageValues.h index b69af140567..8a1a06eeb54 100644 --- a/src/Storages/StorageValues.h +++ b/src/Storages/StorageValues.h @@ -9,7 +9,7 @@ namespace DB /* One block storage used for values table function * It's structure is similar to IStorageSystemOneBlock */ -class StorageValues : public ext::shared_ptr_helper, public IStorage +class StorageValues final : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; public: diff --git a/src/Storages/System/StorageSystemNumbers.h b/src/Storages/System/StorageSystemNumbers.h index cdffd6c8434..f907f3d5f93 100644 --- a/src/Storages/System/StorageSystemNumbers.h +++ b/src/Storages/System/StorageSystemNumbers.h @@ -23,7 +23,7 @@ class Context; * In multithreaded case, if even_distributed is False, implementation with atomic is used, * and result is always in [0 ... limit - 1] range. */ -class StorageSystemNumbers : public ext::shared_ptr_helper, public IStorage +class StorageSystemNumbers final : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; public: diff --git a/src/Storages/System/StorageSystemZeros.h b/src/Storages/System/StorageSystemZeros.h index 26379df5956..41de3ce6246 100644 --- a/src/Storages/System/StorageSystemZeros.h +++ b/src/Storages/System/StorageSystemZeros.h @@ -14,7 +14,7 @@ namespace DB * You could also specify a limit (how many zeros to give). * If multithreaded is specified, zeros will be generated in several streams. */ -class StorageSystemZeros : public ext::shared_ptr_helper, public IStorage +class StorageSystemZeros final : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; public: diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index 7dfee90588a..ea1be8223b2 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -73,10 +73,8 @@ ColumnsDescription getStructureOfRemoteTableInShard( { if (shard_info.isLocal()) { - const auto * table_function = table_func_ptr->as(); - TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_function->name, context); - auto storage_ptr = table_function_ptr->execute(table_func_ptr, context, table_function_ptr->getName()); - return storage_ptr->getInMemoryMetadataPtr()->getColumns(); + TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_func_ptr, context); + return table_function_ptr->getActualTableStructure(context); } auto table_func_name = queryToString(table_func_ptr); diff --git a/src/TableFunctions/ITableFunction.cpp b/src/TableFunctions/ITableFunction.cpp index d5a78bda08a..13282275322 100644 --- a/src/TableFunctions/ITableFunction.cpp +++ b/src/TableFunctions/ITableFunction.cpp @@ -14,13 +14,12 @@ namespace ProfileEvents namespace DB { -StoragePtr ITableFunction::execute(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns_) const +StoragePtr ITableFunction::execute(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const { ProfileEvents::increment(ProfileEvents::TableFunctionExecute); context.checkAccess(AccessType::CREATE_TEMPORARY_TABLE | StorageFactory::instance().getSourceAccessType(getStorageTypeName())); - cached_columns = std::move(cached_columns_); - bool no_conversion_required = hasStaticStructure() && cached_columns == getActualTableStructure(ast_function, context); + bool no_conversion_required = hasStaticStructure() && cached_columns == getActualTableStructure(context); if (cached_columns.empty() || no_conversion_required) return executeImpl(ast_function, context, table_name); @@ -29,7 +28,7 @@ StoragePtr ITableFunction::execute(const ASTPtr & ast_function, const Context & return tf->executeImpl(ast_function, context, table_name); }; - return std::make_shared(StorageID(getDatabaseName(), table_name), std::move(get_storage), cached_columns); + return std::make_shared(StorageID(getDatabaseName(), table_name), std::move(get_storage), std::move(cached_columns)); } } diff --git a/src/TableFunctions/ITableFunction.h b/src/TableFunctions/ITableFunction.h index be1593fcaff..b41213a5e09 100644 --- a/src/TableFunctions/ITableFunction.h +++ b/src/TableFunctions/ITableFunction.h @@ -34,7 +34,9 @@ public: virtual bool hasStaticStructure() const { return false; } - virtual ColumnsDescription getActualTableStructure(const ASTPtr & /*ast_function*/, const Context & /*context*/) const = 0; + virtual void parseArguments(const ASTPtr & /*ast_function*/, const Context & /*context*/) {} + + virtual ColumnsDescription getActualTableStructure(const Context & /*context*/) const = 0; /// Create storage according to the query. StoragePtr execute(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns_ = {}) const; @@ -44,9 +46,6 @@ public: private: virtual StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const = 0; virtual const char * getStorageTypeName() const = 0; - -protected: - mutable ColumnsDescription cached_columns; }; using TableFunctionPtr = std::shared_ptr; diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index 22c537bca2f..b362a646f9e 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -22,11 +22,8 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, const Context & context) const +void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, const Context & context) { - if (!filename.empty()) - return; - /// Parse args ASTs & args_func = ast_function->children; @@ -60,18 +57,16 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, const C compression_method = args[3]->as().value.safeGet(); } -StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const { - parseArguments(ast_function, context); - auto columns = getActualTableStructure(ast_function, context); + auto columns = getActualTableStructure(context); StoragePtr storage = getStorage(filename, format, columns, const_cast(context), table_name, compression_method); storage->startup(); return storage; } -ColumnsDescription ITableFunctionFileLike::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const +ColumnsDescription ITableFunctionFileLike::getActualTableStructure(const Context & context) const { - parseArguments(ast_function, context); if (structure.empty()) { assert(getName() == "file" && format == "Distributed"); diff --git a/src/TableFunctions/ITableFunctionFileLike.h b/src/TableFunctions/ITableFunctionFileLike.h index 47a41c8d1bb..40111eebc5c 100644 --- a/src/TableFunctions/ITableFunctionFileLike.h +++ b/src/TableFunctions/ITableFunctionFileLike.h @@ -20,13 +20,13 @@ private: const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const = 0; - ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; + ColumnsDescription getActualTableStructure(const Context & context) const override; - void parseArguments(const ASTPtr & ast_function, const Context & context) const; + void parseArguments(const ASTPtr & ast_function, const Context & context) override; - mutable String filename; - mutable String format; - mutable String structure; - mutable String compression_method = "auto"; + String filename; + String format; + String structure; + String compression_method = "auto"; }; } diff --git a/src/TableFunctions/ITableFunctionXDBC.cpp b/src/TableFunctions/ITableFunctionXDBC.cpp index 6f1c1f5b2db..6ba62b8ebf1 100644 --- a/src/TableFunctions/ITableFunctionXDBC.cpp +++ b/src/TableFunctions/ITableFunctionXDBC.cpp @@ -25,15 +25,11 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int UNKNOWN_EXCEPTION; extern const int LOGICAL_ERROR; } -void ITableFunctionXDBC::parseArguments(const ASTPtr & ast_function, const Context & context) const +void ITableFunctionXDBC::parseArguments(const ASTPtr & ast_function, const Context & context) { - if (helper) - return; - const auto & args_func = ast_function->as(); if (!args_func.arguments) @@ -65,9 +61,9 @@ void ITableFunctionXDBC::parseArguments(const ASTPtr & ast_function, const Conte helper->startBridgeSync(); } -ColumnsDescription ITableFunctionXDBC::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const +ColumnsDescription ITableFunctionXDBC::getActualTableStructure(const Context & context) const { - parseArguments(ast_function, context); + assert(helper); /* Infer external table structure */ Poco::URI columns_info_uri = helper->getColumnsInfoURI(); @@ -89,10 +85,10 @@ ColumnsDescription ITableFunctionXDBC::getActualTableStructure(const ASTPtr & as return ColumnsDescription{columns}; } -StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const { - parseArguments(ast_function, context); - auto columns = getActualTableStructure(ast_function, context); + assert(helper); + auto columns = getActualTableStructure(context); auto result = std::make_shared(StorageID(getDatabaseName(), table_name), schema_name, remote_table_name, columns, context, helper); result->startup(); return result; diff --git a/src/TableFunctions/ITableFunctionXDBC.h b/src/TableFunctions/ITableFunctionXDBC.h index f2b3e70a3a9..218f361c025 100644 --- a/src/TableFunctions/ITableFunctionXDBC.h +++ b/src/TableFunctions/ITableFunctionXDBC.h @@ -25,14 +25,14 @@ private: const Poco::Timespan & http_timeout_, const std::string & connection_string_) const = 0; - ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; + ColumnsDescription getActualTableStructure(const Context & context) const override; - void parseArguments(const ASTPtr & ast_function, const Context & context) const; + void parseArguments(const ASTPtr & ast_function, const Context & context) override; - mutable String connection_string; - mutable String schema_name; - mutable String remote_table_name; - mutable BridgeHelperPtr helper; + String connection_string; + String schema_name; + String remote_table_name; + BridgeHelperPtr helper; }; class TableFunctionJDBC : public ITableFunctionXDBC diff --git a/src/TableFunctions/TableFunctionFactory.cpp b/src/TableFunctions/TableFunctionFactory.cpp index bc139edfb73..0399d83fe4d 100644 --- a/src/TableFunctions/TableFunctionFactory.cpp +++ b/src/TableFunctions/TableFunctionFactory.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -28,19 +29,21 @@ void TableFunctionFactory::registerFunction(const std::string & name, Value crea } TableFunctionPtr TableFunctionFactory::get( - const std::string & name, + const ASTPtr & ast_function, const Context & context) const { - auto res = tryGet(name, context); + const auto * table_function = ast_function->as(); + auto res = tryGet(table_function->name, context); if (!res) { - auto hints = getHints(name); + auto hints = getHints(table_function->name); if (!hints.empty()) - throw Exception("Unknown table function " + name + ". Maybe you meant: " + toString(hints), ErrorCodes::UNKNOWN_FUNCTION); + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}. Maybe you meant: {}", table_function->name , toString(hints)); else - throw Exception("Unknown table function " + name, ErrorCodes::UNKNOWN_FUNCTION); + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}", table_function->name); } + res->parseArguments(ast_function, context); return res; } diff --git a/src/TableFunctions/TableFunctionFactory.h b/src/TableFunctions/TableFunctionFactory.h index 6d0302a64ff..820b5eb1c7b 100644 --- a/src/TableFunctions/TableFunctionFactory.h +++ b/src/TableFunctions/TableFunctionFactory.h @@ -41,7 +41,7 @@ public: } /// Throws an exception if not found. - TableFunctionPtr get(const std::string & name, const Context & context) const; + TableFunctionPtr get(const ASTPtr & ast_function, const Context & context) const; /// Returns nullptr if not found. TableFunctionPtr tryGet(const std::string & name, const Context & context) const; diff --git a/src/TableFunctions/TableFunctionGenerateRandom.cpp b/src/TableFunctions/TableFunctionGenerateRandom.cpp index eb741ad0ba1..c88cb3229b5 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.cpp +++ b/src/TableFunctions/TableFunctionGenerateRandom.cpp @@ -29,11 +29,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, const Context & /*context*/) const +void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, const Context & /*context*/) { - if (!structure.empty()) - return; - ASTs & args_func = ast_function->children; if (args_func.size() != 1) @@ -80,16 +77,14 @@ void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, co max_array_length = args[3]->as().value.safeGet(); } -ColumnsDescription TableFunctionGenerateRandom::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const +ColumnsDescription TableFunctionGenerateRandom::getActualTableStructure(const Context & context) const { - parseArguments(ast_function, context); return parseColumnsListFromString(structure, context); } -StoragePtr TableFunctionGenerateRandom::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +StoragePtr TableFunctionGenerateRandom::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const { - parseArguments(ast_function, context); - auto columns = getActualTableStructure(ast_function, context); + auto columns = getActualTableStructure(context); auto res = StorageGenerateRandom::create(StorageID(getDatabaseName(), table_name), columns, max_array_length, max_string_length, random_seed); res->startup(); return res; diff --git a/src/TableFunctions/TableFunctionGenerateRandom.h b/src/TableFunctions/TableFunctionGenerateRandom.h index 45ad93ffee7..8c13f853a0d 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.h +++ b/src/TableFunctions/TableFunctionGenerateRandom.h @@ -17,13 +17,13 @@ private: StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; const char * getStorageTypeName() const override { return "GenerateRandom"; } - ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; - void parseArguments(const ASTPtr & ast_function, const Context & context) const; + ColumnsDescription getActualTableStructure(const Context & context) const override; + void parseArguments(const ASTPtr & ast_function, const Context & context) override; - mutable String structure; - mutable UInt64 max_string_length = 10; - mutable UInt64 max_array_length = 10; - mutable std::optional random_seed; + String structure; + UInt64 max_string_length = 10; + UInt64 max_array_length = 10; + std::optional random_seed; }; diff --git a/src/TableFunctions/TableFunctionInput.cpp b/src/TableFunctions/TableFunctionInput.cpp index 4d0f8df55e6..6563d8f0b3a 100644 --- a/src/TableFunctions/TableFunctionInput.cpp +++ b/src/TableFunctions/TableFunctionInput.cpp @@ -24,11 +24,8 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -void TableFunctionInput::parseArguments(const ASTPtr & ast_function, const Context & context) const +void TableFunctionInput::parseArguments(const ASTPtr & ast_function, const Context & context) { - if (!structure.empty()) - return; - const auto * function = ast_function->as(); if (!function->arguments) @@ -43,16 +40,14 @@ void TableFunctionInput::parseArguments(const ASTPtr & ast_function, const Conte structure = evaluateConstantExpressionOrIdentifierAsLiteral(args[0], context)->as().value.safeGet(); } -ColumnsDescription TableFunctionInput::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const +ColumnsDescription TableFunctionInput::getActualTableStructure(const Context & context) const { - parseArguments(ast_function, context); return parseColumnsListFromString(structure, context); } -StoragePtr TableFunctionInput::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +StoragePtr TableFunctionInput::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const { - parseArguments(ast_function, context); - auto storage = StorageInput::create(StorageID(getDatabaseName(), table_name), getActualTableStructure(ast_function, context)); + auto storage = StorageInput::create(StorageID(getDatabaseName(), table_name), getActualTableStructure(context)); storage->startup(); return storage; } diff --git a/src/TableFunctions/TableFunctionInput.h b/src/TableFunctions/TableFunctionInput.h index 50744d3ca14..9190fe600f6 100644 --- a/src/TableFunctions/TableFunctionInput.h +++ b/src/TableFunctions/TableFunctionInput.h @@ -20,10 +20,10 @@ private: StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; const char * getStorageTypeName() const override { return "Input"; } - ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; - void parseArguments(const ASTPtr & ast_function, const Context & context) const; + ColumnsDescription getActualTableStructure(const Context & context) const override; + void parseArguments(const ASTPtr & ast_function, const Context & context) override; - mutable String structure; + String structure; }; } diff --git a/src/TableFunctions/TableFunctionMerge.cpp b/src/TableFunctions/TableFunctionMerge.cpp index 360ec583740..6358a0bbbef 100644 --- a/src/TableFunctions/TableFunctionMerge.cpp +++ b/src/TableFunctions/TableFunctionMerge.cpp @@ -46,7 +46,7 @@ static NamesAndTypesList chooseColumns(const String & source_database, const Str return any_table->getInMemoryMetadataPtr()->getColumns().getAllPhysical(); } -void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, const Context & context) const +void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, const Context & context) { ASTs & args_func = ast_function->children; @@ -69,19 +69,16 @@ void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, const Conte table_name_regexp = args[1]->as().value.safeGet(); } -ColumnsDescription TableFunctionMerge::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const +ColumnsDescription TableFunctionMerge::getActualTableStructure(const Context & context) const { - parseArguments(ast_function, context); return ColumnsDescription{chooseColumns(source_database, table_name_regexp, context)}; } -StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const { - parseArguments(ast_function, context); - auto res = StorageMerge::create( StorageID(getDatabaseName(), table_name), - getActualTableStructure(ast_function, context), + getActualTableStructure(context), source_database, table_name_regexp, context); diff --git a/src/TableFunctions/TableFunctionMerge.h b/src/TableFunctions/TableFunctionMerge.h index 0e6878983e3..2d038de32fb 100644 --- a/src/TableFunctions/TableFunctionMerge.h +++ b/src/TableFunctions/TableFunctionMerge.h @@ -19,11 +19,11 @@ private: StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; const char * getStorageTypeName() const override { return "Merge"; } - ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; - void parseArguments(const ASTPtr & ast_function, const Context & context) const; + ColumnsDescription getActualTableStructure(const Context & context) const override; + void parseArguments(const ASTPtr & ast_function, const Context & context) override; - mutable String source_database; - mutable String table_name_regexp; + String source_database; + String table_name_regexp; }; diff --git a/src/TableFunctions/TableFunctionMySQL.cpp b/src/TableFunctions/TableFunctionMySQL.cpp index bff2842c8c1..c957bb8c05f 100644 --- a/src/TableFunctions/TableFunctionMySQL.cpp +++ b/src/TableFunctions/TableFunctionMySQL.cpp @@ -34,7 +34,7 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; } -void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, const Context & context) const +void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, const Context & context) { const auto & args_func = ast_function->as(); @@ -70,9 +70,9 @@ void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, const Conte parsed_host_port = parseAddress(host_port, 3306); } -ColumnsDescription TableFunctionMySQL::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const +ColumnsDescription TableFunctionMySQL::getActualTableStructure(const Context & context) const { - parseArguments(ast_function, context); + assert(!parsed_host_port.first.empty()); if (!pool) pool.emplace(remote_database_name, parsed_host_port.first, user_name, password, parsed_host_port.second); @@ -121,13 +121,13 @@ ColumnsDescription TableFunctionMySQL::getActualTableStructure(const ASTPtr & as return ColumnsDescription{columns}; } -StoragePtr TableFunctionMySQL::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +StoragePtr TableFunctionMySQL::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const { - parseArguments(ast_function, context); + assert(!parsed_host_port.first.empty()); if (!pool) pool.emplace(remote_database_name, parsed_host_port.first, user_name, password, parsed_host_port.second); - auto columns = getActualTableStructure(ast_function, context); + auto columns = getActualTableStructure(context); auto res = StorageMySQL::create( StorageID(getDatabaseName(), table_name), diff --git a/src/TableFunctions/TableFunctionMySQL.h b/src/TableFunctions/TableFunctionMySQL.h index e8f5851b502..f3c15a5fdd9 100644 --- a/src/TableFunctions/TableFunctionMySQL.h +++ b/src/TableFunctions/TableFunctionMySQL.h @@ -27,16 +27,16 @@ private: StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; const char * getStorageTypeName() const override { return "MySQL"; } - ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; - void parseArguments(const ASTPtr & ast_function, const Context & context) const; + ColumnsDescription getActualTableStructure(const Context & context) const override; + void parseArguments(const ASTPtr & ast_function, const Context & context) override; - mutable std::pair parsed_host_port; - mutable String remote_database_name; - mutable String remote_table_name; - mutable String user_name; - mutable String password; - mutable bool replace_query = false; - mutable String on_duplicate_clause; + std::pair parsed_host_port; + String remote_database_name; + String remote_table_name; + String user_name; + String password; + bool replace_query = false; + String on_duplicate_clause; mutable std::optional pool; }; diff --git a/src/TableFunctions/TableFunctionNumbers.cpp b/src/TableFunctions/TableFunctionNumbers.cpp index 03756b37b07..84edd3935b3 100644 --- a/src/TableFunctions/TableFunctionNumbers.cpp +++ b/src/TableFunctions/TableFunctionNumbers.cpp @@ -22,7 +22,7 @@ namespace ErrorCodes template -ColumnsDescription TableFunctionNumbers::getActualTableStructure(const ASTPtr & /*ast_function*/, const Context & /*context*/) const +ColumnsDescription TableFunctionNumbers::getActualTableStructure(const Context & /*context*/) const { return ColumnsDescription({{"number", std::make_shared()}}); } diff --git a/src/TableFunctions/TableFunctionNumbers.h b/src/TableFunctions/TableFunctionNumbers.h index a3c4f6f13cb..5cbde6cf0a3 100644 --- a/src/TableFunctions/TableFunctionNumbers.h +++ b/src/TableFunctions/TableFunctionNumbers.h @@ -23,7 +23,7 @@ private: UInt64 evaluateArgument(const Context & context, ASTPtr & argument) const; - ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; + ColumnsDescription getActualTableStructure(const Context & context) const override; }; diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 3d00d0ad897..594b47a3666 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -28,11 +28,8 @@ namespace ErrorCodes } -void TableFunctionRemote::prepareClusterInfo(const ASTPtr & ast_function, const Context & context) const +void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, const Context & context) { - if (cluster) - return; - ASTs & args_func = ast_function->children; if (args_func.size() != 1) @@ -198,14 +195,13 @@ void TableFunctionRemote::prepareClusterInfo(const ASTPtr & ast_function, const remote_table_id.table_name = remote_table; } -StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const { - prepareClusterInfo(ast_function, context); - + assert(cluster); StoragePtr res = remote_table_function_ptr ? StorageDistributed::create( StorageID(getDatabaseName(), table_name), - getActualTableStructure(ast_function, context), + getActualTableStructure(context), ConstraintsDescription{}, remote_table_function_ptr, String{}, @@ -217,7 +213,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C cluster) : StorageDistributed::create( StorageID(getDatabaseName(), table_name), - getActualTableStructure(ast_function, context), + getActualTableStructure(context), ConstraintsDescription{}, remote_table_id.database_name, remote_table_id.table_name, @@ -233,9 +229,9 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C return res; } -ColumnsDescription TableFunctionRemote::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const +ColumnsDescription TableFunctionRemote::getActualTableStructure(const Context & context) const { - prepareClusterInfo(ast_function, context); + assert(cluster); return getStructureOfRemoteTable(*cluster, remote_table_id, context, remote_table_function_ptr); } diff --git a/src/TableFunctions/TableFunctionRemote.h b/src/TableFunctions/TableFunctionRemote.h index 35d1ca3b9ff..ff214bb7332 100644 --- a/src/TableFunctions/TableFunctionRemote.h +++ b/src/TableFunctions/TableFunctionRemote.h @@ -22,22 +22,22 @@ public: std::string getName() const override { return name; } - ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; + ColumnsDescription getActualTableStructure(const Context & context) const override; private: StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; const char * getStorageTypeName() const override { return "Distributed"; } - void prepareClusterInfo(const ASTPtr & ast_function, const Context & context) const; + void parseArguments(const ASTPtr & ast_function, const Context & context) override; std::string name; bool is_cluster_function; std::string help_message; bool secure; - mutable ClusterPtr cluster; - mutable StorageID remote_table_id = StorageID::createEmpty(); - mutable ASTPtr remote_table_function_ptr; + ClusterPtr cluster; + StorageID remote_table_id = StorageID::createEmpty(); + ASTPtr remote_table_function_ptr; }; } diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index b4fe1baffce..a4f403f7875 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -22,10 +22,8 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -void TableFunctionS3::parseArguments(const ASTPtr & ast_function, const Context & context) const +void TableFunctionS3::parseArguments(const ASTPtr & ast_function, const Context & context) { - - /// Parse args ASTs & args_func = ast_function->children; @@ -60,16 +58,13 @@ void TableFunctionS3::parseArguments(const ASTPtr & ast_function, const Context compression_method = args.back()->as().value.safeGet(); } -ColumnsDescription TableFunctionS3::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const +ColumnsDescription TableFunctionS3::getActualTableStructure(const Context & context) const { - parseArguments(ast_function, context); return parseColumnsListFromString(structure, context); } -StoragePtr TableFunctionS3::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const { - parseArguments(ast_function, context); - Poco::URI uri (filename); S3::URI s3_uri (uri); UInt64 min_upload_part_size = context.getSettingsRef().s3_min_upload_part_size; @@ -81,7 +76,7 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & ast_function, const Conte StorageID(getDatabaseName(), table_name), format, min_upload_part_size, - getActualTableStructure(ast_function, context), + getActualTableStructure(context), ConstraintsDescription{}, const_cast(context), compression_method); diff --git a/src/TableFunctions/TableFunctionS3.h b/src/TableFunctions/TableFunctionS3.h index 23dec9a689c..45738de8ba8 100644 --- a/src/TableFunctions/TableFunctionS3.h +++ b/src/TableFunctions/TableFunctionS3.h @@ -31,15 +31,15 @@ protected: const char * getStorageTypeName() const override { return "S3"; } - ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; - void parseArguments(const ASTPtr & ast_function, const Context & context) const; + ColumnsDescription getActualTableStructure(const Context & context) const override; + void parseArguments(const ASTPtr & ast_function, const Context & context) override; - mutable String filename; - mutable String format; - mutable String structure; - mutable String access_key_id; - mutable String secret_access_key; - mutable String compression_method = "auto"; + String filename; + String format; + String structure; + String access_key_id; + String secret_access_key; + String compression_method = "auto"; }; class TableFunctionCOS : public TableFunctionS3 diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index e867fc836f7..6139e6ffecb 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -14,7 +14,7 @@ StoragePtr TableFunctionURL::getStorage( const std::string & table_name, const String & compression_method_) const { Poco::URI uri(source); - return StorageURL::create( uri, StorageID(getDatabaseName(), table_name), format_, columns, ConstraintsDescription{}, + return StorageURL::create(uri, StorageID(getDatabaseName(), table_name), format_, columns, ConstraintsDescription{}, global_context, compression_method_); } diff --git a/src/TableFunctions/TableFunctionValues.cpp b/src/TableFunctions/TableFunctionValues.cpp index e51e28abff3..4e892c83247 100644 --- a/src/TableFunctions/TableFunctionValues.cpp +++ b/src/TableFunctions/TableFunctionValues.cpp @@ -63,7 +63,7 @@ static void parseAndInsertValues(MutableColumns & res_columns, const ASTs & args } } -void TableFunctionValues::parseArguments(const ASTPtr & ast_function, const Context & /*context*/) const +void TableFunctionValues::parseArguments(const ASTPtr & ast_function, const Context & /*context*/) { @@ -90,17 +90,14 @@ void TableFunctionValues::parseArguments(const ASTPtr & ast_function, const Cont structure = args[0]->as().value.safeGet(); } -ColumnsDescription TableFunctionValues::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const +ColumnsDescription TableFunctionValues::getActualTableStructure(const Context & context) const { - parseArguments(ast_function, context); return parseColumnsListFromString(structure, context); } StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { - parseArguments(ast_function, context); - - auto columns = getActualTableStructure(ast_function, context); + auto columns = getActualTableStructure(context); Block sample_block; for (const auto & name_type : columns.getOrdinary()) diff --git a/src/TableFunctions/TableFunctionValues.h b/src/TableFunctions/TableFunctionValues.h index 9ab2dfb047e..b66f5bb457f 100644 --- a/src/TableFunctions/TableFunctionValues.h +++ b/src/TableFunctions/TableFunctionValues.h @@ -16,10 +16,10 @@ private: StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; const char * getStorageTypeName() const override { return "Values"; } - ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; - void parseArguments(const ASTPtr & ast_function, const Context & context) const; + ColumnsDescription getActualTableStructure(const Context & context) const override; + void parseArguments(const ASTPtr & ast_function, const Context & context) override; - mutable String structure; + String structure; }; diff --git a/src/TableFunctions/TableFunctionZeros.cpp b/src/TableFunctions/TableFunctionZeros.cpp index b633260861c..5ba6c034e09 100644 --- a/src/TableFunctions/TableFunctionZeros.cpp +++ b/src/TableFunctions/TableFunctionZeros.cpp @@ -21,7 +21,7 @@ extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; template -ColumnsDescription TableFunctionZeros::getActualTableStructure(const ASTPtr & /*ast_function*/, const Context & /*context*/) const +ColumnsDescription TableFunctionZeros::getActualTableStructure(const Context & /*context*/) const { return ColumnsDescription({{"zero", std::make_shared()}}); } diff --git a/src/TableFunctions/TableFunctionZeros.h b/src/TableFunctions/TableFunctionZeros.h index aba161af4d0..c50a17d046d 100644 --- a/src/TableFunctions/TableFunctionZeros.h +++ b/src/TableFunctions/TableFunctionZeros.h @@ -23,7 +23,7 @@ private: UInt64 evaluateArgument(const Context & context, ASTPtr & argument) const; - ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; + ColumnsDescription getActualTableStructure(const Context & context) const override; }; From c7f355a8f3701ae6d090e274ebd07d945af3362c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 1 Sep 2020 18:24:21 +0300 Subject: [PATCH 06/42] add test --- src/Storages/StorageTableFunction.h | 10 +++++++ ..._expressions_in_engine_arguments.reference | 6 ++-- tests/queries/0_stateless/014 | 0 ...eate_as_table_function_structure.reference | 7 +++++ ...457_create_as_table_function_structure.sql | 28 +++++++++++++++++++ 5 files changed, 48 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/014 create mode 100644 tests/queries/0_stateless/01457_create_as_table_function_structure.reference create mode 100644 tests/queries/0_stateless/01457_create_as_table_function_structure.sql diff --git a/src/Storages/StorageTableFunction.h b/src/Storages/StorageTableFunction.h index 530e614c856..87af3a49a5c 100644 --- a/src/Storages/StorageTableFunction.h +++ b/src/Storages/StorageTableFunction.h @@ -58,6 +58,13 @@ public: nested->shutdown(); } + void drop() override + { + std::lock_guard lock{nested_mutex}; + if (nested) + nested->drop(); + } + Pipe read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -111,6 +118,9 @@ public: IStorage::renameInMemory(new_table_id); } + bool isView() const override { return false; } + void checkTableCanBeDropped() const override {} + private: mutable std::mutex nested_mutex; mutable GetNestedStorageFunc get_nested; diff --git a/tests/queries/0_stateless/01083_expressions_in_engine_arguments.reference b/tests/queries/0_stateless/01083_expressions_in_engine_arguments.reference index 933fda604d7..2a5d7e6da32 100644 --- a/tests/queries/0_stateless/01083_expressions_in_engine_arguments.reference +++ b/tests/queries/0_stateless/01083_expressions_in_engine_arguments.reference @@ -1,11 +1,11 @@ CREATE TABLE default.file\n(\n `n` Int8\n)\nENGINE = File(\'TSVWithNamesAndTypes\') CREATE TABLE default.buffer\n(\n `n` Int8\n)\nENGINE = Buffer(\'default\', \'file\', 16, 10, 200, 10000, 1000000, 10000000, 1000000000) CREATE TABLE default.merge\n(\n `n` Int8\n)\nENGINE = Merge(\'default\', \'distributed\') -CREATE TABLE default.merge_tf AS merge(\'default\', \'.*\') +CREATE TABLE default.merge_tf\n(\n `n` Int8\n) AS merge(\'default\', \'.*\') CREATE TABLE default.distributed\n(\n `n` Int8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'default\', \'file\') -CREATE TABLE default.distributed_tf AS cluster(\'test_shard_localhost\', \'default\', \'buffer\') +CREATE TABLE default.distributed_tf\n(\n `n` Int8\n) AS cluster(\'test_shard_localhost\', \'default\', \'buffer\') CREATE TABLE default.url\n(\n `n` UInt64,\n `col` String\n)\nENGINE = URL(\'https://localhost:8443/?query=select+n,+_table+from+default.merge+format+CSV\', \'CSV\') -CREATE TABLE default.rich_syntax AS remote(\'localhos{x|y|t}\', cluster(\'test_shard_localhost\', remote(\'127.0.0.{1..4}\', \'default\', \'view\'))) +CREATE TABLE default.rich_syntax\n(\n `n` Int64\n) AS remote(\'localhos{x|y|t}\', cluster(\'test_shard_localhost\', remote(\'127.0.0.{1..4}\', \'default\', \'view\'))) CREATE VIEW default.view\n(\n `n` Int64\n) AS\nSELECT toInt64(n) AS n\nFROM \n(\n SELECT toString(n) AS n\n FROM default.merge\n WHERE _table != \'qwerty\'\n ORDER BY _table ASC\n)\nUNION ALL\nSELECT *\nFROM default.file CREATE DICTIONARY default.dict\n(\n `n` UInt64,\n `col` String DEFAULT \'42\'\n)\nPRIMARY KEY n\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9440 SECURE 1 USER \'default\' TABLE \'url\'))\nLIFETIME(MIN 0 MAX 1)\nLAYOUT(CACHE(SIZE_IN_CELLS 1)) 16 diff --git a/tests/queries/0_stateless/014 b/tests/queries/0_stateless/014 new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01457_create_as_table_function_structure.reference b/tests/queries/0_stateless/01457_create_as_table_function_structure.reference new file mode 100644 index 00000000000..7169648d524 --- /dev/null +++ b/tests/queries/0_stateless/01457_create_as_table_function_structure.reference @@ -0,0 +1,7 @@ +CREATE TABLE test_01457.tf_remote\n(\n `n` Int8\n) AS remote(\'localhost\', \'default\', \'tmp\') +CREATE TABLE test_01457.tf_remote_explicit_structure\n(\n `n` UInt64\n) AS remote(\'localhost\', \'default\', \'tmp\') +CREATE TABLE test_01457.tf_numbers\n(\n `number` String\n) AS numbers(1) +0 Int8 +0 Int8 +0 UInt64 +0 String diff --git a/tests/queries/0_stateless/01457_create_as_table_function_structure.sql b/tests/queries/0_stateless/01457_create_as_table_function_structure.sql new file mode 100644 index 00000000000..1949c084960 --- /dev/null +++ b/tests/queries/0_stateless/01457_create_as_table_function_structure.sql @@ -0,0 +1,28 @@ +DROP DATABASE IF EXISTS test_01457; + +CREATE DATABASE test_01457; + +CREATE TABLE tmp (n Int8) ENGINE=Memory; + +CREATE TABLE test_01457.tf_remote AS remote('localhost', currentDatabase(), 'tmp'); +SHOW CREATE TABLE test_01457.tf_remote; +CREATE TABLE test_01457.tf_remote_explicit_structure (n UInt64) AS remote('localhost', currentDatabase(), 'tmp'); +SHOW CREATE TABLE test_01457.tf_remote_explicit_structure; +CREATE TABLE test_01457.tf_numbers (number String) AS numbers(1); +SHOW CREATE TABLE test_01457.tf_numbers; + +DROP TABLE tmp; + +DETACH DATABASE test_01457; +ATTACH DATABASE test_01457; + +CREATE TABLE tmp (n Int8) ENGINE=Memory; +INSERT INTO test_01457.tf_remote_explicit_structure VALUES (0); -- { serverError 122 } +INSERT INTO test_01457.tf_remote VALUES (0); + +SELECT (*,).1 AS c, toTypeName(c) FROM tmp; +SELECT (*,).1 AS c, toTypeName(c) FROM test_01457.tf_remote; +SELECT (*,).1 AS c, toTypeName(c) FROM test_01457.tf_remote_explicit_structure; +SELECT (*,).1 AS c, toTypeName(c) FROM test_01457.tf_numbers; + +DROP DATABASE test_01457; From 41dd39382447ff4dd88f3af8c8fda72965af551a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 1 Sep 2020 20:55:35 +0300 Subject: [PATCH 07/42] fixes --- src/Databases/DatabaseOrdinary.cpp | 7 +++++-- src/Interpreters/ya.make | 1 + src/Storages/StorageTableFunction.h | 1 + .../0_stateless/01461_alter_table_function.reference | 9 ++++----- tests/queries/0_stateless/01461_alter_table_function.sql | 4 +--- 5 files changed, 12 insertions(+), 10 deletions(-) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 0512a155418..3f536f7c995 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -254,9 +254,12 @@ void DatabaseOrdinary::alterTable(const Context & context, const StorageID & tab auto & ast_create_query = ast->as(); - if (ast_create_query.as_table_function) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot alter table {} because it was created AS table function", backQuote(table_name)); + bool has_structure = ast_create_query.columns_list && ast_create_query.columns_list->columns; + if (ast_create_query.as_table_function && !has_structure) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot alter table {} because it was created AS table function" + " and doesn't have structure in metadata", backQuote(table_name)); + assert(has_structure); ASTPtr new_columns = InterpreterCreateQuery::formatColumns(metadata.columns); ASTPtr new_indices = InterpreterCreateQuery::formatIndices(metadata.secondary_indices); ASTPtr new_constraints = InterpreterCreateQuery::formatConstraints(metadata.constraints); diff --git a/src/Interpreters/ya.make b/src/Interpreters/ya.make index 23cde61a744..cc3b63778f9 100644 --- a/src/Interpreters/ya.make +++ b/src/Interpreters/ya.make @@ -58,6 +58,7 @@ SRCS( ExtractExpressionInfoVisitor.cpp FillingRow.cpp getClusterName.cpp + getHeaderForProcessingStage.cpp getTableExpressions.cpp HashJoin.cpp IdentifierSemantic.cpp diff --git a/src/Storages/StorageTableFunction.h b/src/Storages/StorageTableFunction.h index 87af3a49a5c..4a0afdd5fb1 100644 --- a/src/Storages/StorageTableFunction.h +++ b/src/Storages/StorageTableFunction.h @@ -37,6 +37,7 @@ public: auto nested_storage = get_nested(); nested_storage->startup(); + nested_storage->renameInMemory(getStorageID()); nested = nested_storage; get_nested = {}; return nested; diff --git a/tests/queries/0_stateless/01461_alter_table_function.reference b/tests/queries/0_stateless/01461_alter_table_function.reference index 395155967a9..b552dd81b77 100644 --- a/tests/queries/0_stateless/01461_alter_table_function.reference +++ b/tests/queries/0_stateless/01461_alter_table_function.reference @@ -1,7 +1,6 @@ -CREATE TABLE default.table_from_remote AS remote(\'localhost\', \'system\', \'numbers\') -CREATE TABLE default.table_from_remote AS remote(\'localhost\', \'system\', \'numbers\') -CREATE TABLE default.table_from_numbers AS numbers(1000) -CREATE TABLE default.table_from_numbers AS numbers(1000) +CREATE TABLE default.table_from_remote\n(\n `number` UInt64\n) AS remote(\'localhost\', \'system\', \'numbers\') +CREATE TABLE default.table_from_remote\n(\n `number` UInt64,\n `col` UInt8\n) AS remote(\'localhost\', \'system\', \'numbers\') +CREATE TABLE default.table_from_numbers\n(\n `number` UInt64\n) AS numbers(1000) +CREATE TABLE default.table_from_numbers\n(\n `number` UInt64\n) AS numbers(1000) CREATE TABLE default.table_from_select\n(\n `number` UInt64\n)\nENGINE = MergeTree()\nORDER BY tuple()\nSETTINGS index_granularity = 8192 CREATE TABLE default.table_from_select\n(\n `number` UInt64,\n `col` UInt8\n)\nENGINE = MergeTree()\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -1 diff --git a/tests/queries/0_stateless/01461_alter_table_function.sql b/tests/queries/0_stateless/01461_alter_table_function.sql index e242d1f0b7b..11f643f1e3e 100644 --- a/tests/queries/0_stateless/01461_alter_table_function.sql +++ b/tests/queries/0_stateless/01461_alter_table_function.sql @@ -6,7 +6,7 @@ CREATE TABLE table_from_remote AS remote('localhost', 'system', 'numbers'); SHOW CREATE TABLE table_from_remote; -ALTER TABLE table_from_remote ADD COLUMN col UInt8; --{serverError 48} +ALTER TABLE table_from_remote ADD COLUMN col UInt8; SHOW CREATE TABLE table_from_remote; @@ -26,8 +26,6 @@ ALTER TABLE table_from_select ADD COLUMN col UInt8; SHOW CREATE TABLE table_from_select; -SELECT 1; - DROP TABLE IF EXISTS table_from_remote; DROP TABLE IF EXISTS table_from_select; DROP TABLE IF EXISTS table_from_numbers; From ee7b8a797a7aed63eee854dedba69e9403fe4d48 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 2 Sep 2020 18:07:53 +0300 Subject: [PATCH 08/42] better code, fix Distributed format of StorageFile --- src/Interpreters/InterpreterCreateQuery.cpp | 6 +- src/Storages/StorageFile.cpp | 57 +++++++++++-------- src/Storages/StorageFile.h | 2 + src/Storages/StorageProxy.h | 1 - src/Storages/StorageTableFunction.h | 18 ++++-- src/TableFunctions/ITableFunction.cpp | 11 ++-- src/TableFunctions/ITableFunction.h | 16 +++++- src/TableFunctions/ITableFunctionFileLike.cpp | 11 +++- src/TableFunctions/ITableFunctionFileLike.h | 5 +- src/TableFunctions/ITableFunctionXDBC.cpp | 3 +- src/TableFunctions/ITableFunctionXDBC.h | 2 +- .../TableFunctionGenerateRandom.cpp | 5 +- .../TableFunctionGenerateRandom.h | 3 +- src/TableFunctions/TableFunctionInput.cpp | 4 +- src/TableFunctions/TableFunctionInput.h | 3 +- src/TableFunctions/TableFunctionMerge.cpp | 3 +- src/TableFunctions/TableFunctionMerge.h | 2 +- src/TableFunctions/TableFunctionMySQL.cpp | 3 +- src/TableFunctions/TableFunctionMySQL.h | 2 +- src/TableFunctions/TableFunctionNumbers.cpp | 3 +- src/TableFunctions/TableFunctionNumbers.h | 3 +- src/TableFunctions/TableFunctionRemote.cpp | 14 +++-- src/TableFunctions/TableFunctionRemote.h | 4 +- src/TableFunctions/TableFunctionS3.cpp | 3 +- src/TableFunctions/TableFunctionS3.h | 4 +- src/TableFunctions/TableFunctionValues.cpp | 3 +- src/TableFunctions/TableFunctionValues.h | 3 +- src/TableFunctions/TableFunctionZeros.cpp | 3 +- src/TableFunctions/TableFunctionZeros.h | 3 +- tests/queries/0_stateless/014 | 0 ...eate_as_table_function_structure.reference | 3 + ...457_create_as_table_function_structure.sql | 7 ++- 32 files changed, 131 insertions(+), 79 deletions(-) delete mode 100644 tests/queries/0_stateless/014 diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index c6a010e8240..20825f35109 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -495,8 +495,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(AS /// Table function without columns list. auto table_function = TableFunctionFactory::instance().get(create.as_table_function, context); properties.columns = table_function->getActualTableStructure(context); - if (properties.columns.empty()) //FIXME TableFunctionFile may return empty structure for Distributed format - return {}; + assert(!properties.columns.empty()); } else throw Exception("Incorrect CREATE query: required list of column descriptions or AS section or SELECT.", ErrorCodes::INCORRECT_QUERY); @@ -583,6 +582,9 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const { + if (create.as_table_function) + return; + if (create.storage || create.is_view || create.is_materialized_view || create.is_live_view || create.is_dictionary) { if (create.temporary && create.storage->engine->name != "Memory") diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 6ead7a914ff..fc777b68d60 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -52,6 +52,7 @@ namespace ErrorCodes extern const int UNKNOWN_IDENTIFIER; extern const int INCORRECT_FILE_NAME; extern const int FILE_DOESNT_EXIST; + extern const int INCOMPATIBLE_COLUMNS; } namespace @@ -126,11 +127,33 @@ void checkCreationIsAllowed(const Context & context_global, const std::string & } } +Strings StorageFile::getPathsList(const String & table_path, const String & user_files_path, const Context & context) +{ + String user_files_absolute_path = Poco::Path(user_files_path).makeAbsolute().makeDirectory().toString(); + Poco::Path poco_path = Poco::Path(table_path); + if (poco_path.isRelative()) + poco_path = Poco::Path(user_files_absolute_path, poco_path); + + Strings paths; + const String path = poco_path.absolute().toString(); + if (path.find_first_of("*?{") == std::string::npos) + paths.push_back(path); + else + paths = listFilesWithRegexpMatching("/", path); + + for (const auto & cur_path : paths) + checkCreationIsAllowed(context, user_files_absolute_path, cur_path); + + return paths; +} + StorageFile::StorageFile(int table_fd_, CommonArguments args) : StorageFile(args) { if (args.context.getApplicationType() == Context::ApplicationType::SERVER) throw Exception("Using file descriptor as source of storage isn't allowed for server daemons", ErrorCodes::DATABASE_ACCESS_DENIED); + if (args.format_name == "Distributed") + throw Exception("Distributed format is allowed only with explicit file path", ErrorCodes::INCORRECT_FILE_NAME); is_db_table = false; use_table_fd = true; @@ -145,36 +168,22 @@ StorageFile::StorageFile(const std::string & table_path_, const std::string & us : StorageFile(args) { is_db_table = false; - std::string user_files_absolute_path = Poco::Path(user_files_path).makeAbsolute().makeDirectory().toString(); - Poco::Path poco_path = Poco::Path(table_path_); - if (poco_path.isRelative()) - poco_path = Poco::Path(user_files_absolute_path, poco_path); - - const std::string path = poco_path.absolute().toString(); - if (path.find_first_of("*?{") == std::string::npos) - paths.push_back(path); - else - paths = listFilesWithRegexpMatching("/", path); - - for (const auto & cur_path : paths) - checkCreationIsAllowed(args.context, user_files_absolute_path, cur_path); + paths = getPathsList(table_path_, user_files_path, args.context); if (args.format_name == "Distributed") { if (paths.empty()) - { throw Exception("Cannot get table structure from file, because no files match specified name", ErrorCodes::INCORRECT_FILE_NAME); - } - else - { - auto & first_path = paths[0]; - Block header = StorageDistributedDirectoryMonitor::createStreamFromFile(first_path)->getHeader(); + auto & first_path = paths[0]; + Block header = StorageDistributedDirectoryMonitor::createStreamFromFile(first_path)->getHeader(); - StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(ColumnsDescription(header.getNamesAndTypesList())); - setInMemoryMetadata(storage_metadata); - } + StorageInMemoryMetadata storage_metadata; + auto columns = ColumnsDescription(header.getNamesAndTypesList()); + if (!args.columns.empty() && columns != args.columns) + throw Exception("Table structure and file structure are different", ErrorCodes::INCOMPATIBLE_COLUMNS); + storage_metadata.setColumns(columns); + setInMemoryMetadata(storage_metadata); } } @@ -183,6 +192,8 @@ StorageFile::StorageFile(const std::string & relative_table_dir_path, CommonArgu { if (relative_table_dir_path.empty()) throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); + if (args.format_name == "Distributed") + throw Exception("Distributed format is allowed only with explicit file path", ErrorCodes::INCORRECT_FILE_NAME); String table_dir_path = base_path + relative_table_dir_path + "/"; Poco::File(table_dir_path).createDirectories(); diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index ea70dcd5311..f331538b4c7 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -60,6 +60,8 @@ public: NamesAndTypesList getVirtuals() const override; + static Strings getPathsList(const String & table_path, const String & user_files_path, const Context & context); + protected: friend class StorageFileSource; friend class StorageFileBlockOutputStream; diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index 81edcf27c64..92200ff5775 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -6,7 +6,6 @@ namespace DB { - class StorageProxy : public IStorage { public: diff --git a/src/Storages/StorageTableFunction.h b/src/Storages/StorageTableFunction.h index 4a0afdd5fb1..9b698cb3954 100644 --- a/src/Storages/StorageTableFunction.h +++ b/src/Storages/StorageTableFunction.h @@ -18,11 +18,14 @@ namespace ErrorCodes using GetNestedStorageFunc = std::function; +/// Lazily creates underlying storage. +/// Adds ConversionTransform in case of structure mismatch. class StorageTableFunctionProxy final : public StorageProxy { public: - StorageTableFunctionProxy(const StorageID & table_id_, GetNestedStorageFunc get_nested_, ColumnsDescription cached_columns) - : StorageProxy(table_id_), get_nested(std::move(get_nested_)) + StorageTableFunctionProxy(const StorageID & table_id_, GetNestedStorageFunc get_nested_, + ColumnsDescription cached_columns, bool add_conversion_ = true) + : StorageProxy(table_id_), get_nested(std::move(get_nested_)), add_conversion(add_conversion_) { StorageInMemoryMetadata cached_metadata; cached_metadata.setColumns(std::move(cached_columns)); @@ -80,10 +83,12 @@ public: cnames += c + " "; auto storage = getNested(); auto nested_metadata = storage->getInMemoryMetadataPtr(); - auto pipe = storage->read(column_names, nested_metadata, query_info, context, processed_stage, max_block_size, num_streams); - if (!pipe.empty()) + auto pipe = storage->read(column_names, nested_metadata, query_info, context, + processed_stage, max_block_size, num_streams); + if (!pipe.empty() && add_conversion) { - auto to_header = getHeaderForProcessingStage(*this, column_names, metadata_snapshot, query_info, context, processed_stage); + auto to_header = getHeaderForProcessingStage(*this, column_names, metadata_snapshot, + query_info, context, processed_stage); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( @@ -103,7 +108,7 @@ public: auto storage = getNested(); auto cached_structure = metadata_snapshot->getSampleBlock(); auto actual_structure = storage->getInMemoryMetadataPtr()->getSampleBlock(); - if (!blocksHaveEqualStructure(actual_structure, cached_structure)) + if (!blocksHaveEqualStructure(actual_structure, cached_structure) && add_conversion) { throw Exception("Source storage and table function have different structure", ErrorCodes::INCOMPATIBLE_COLUMNS); } @@ -126,6 +131,7 @@ private: mutable std::mutex nested_mutex; mutable GetNestedStorageFunc get_nested; mutable StoragePtr nested; + const bool add_conversion; }; } diff --git a/src/TableFunctions/ITableFunction.cpp b/src/TableFunctions/ITableFunction.cpp index 13282275322..fd5cdf6e219 100644 --- a/src/TableFunctions/ITableFunction.cpp +++ b/src/TableFunctions/ITableFunction.cpp @@ -14,21 +14,24 @@ namespace ProfileEvents namespace DB { -StoragePtr ITableFunction::execute(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const +StoragePtr ITableFunction::execute(const ASTPtr & ast_function, const Context & context, const std::string & table_name, + ColumnsDescription cached_columns) const { ProfileEvents::increment(ProfileEvents::TableFunctionExecute); context.checkAccess(AccessType::CREATE_TEMPORARY_TABLE | StorageFactory::instance().getSourceAccessType(getStorageTypeName())); bool no_conversion_required = hasStaticStructure() && cached_columns == getActualTableStructure(context); if (cached_columns.empty() || no_conversion_required) - return executeImpl(ast_function, context, table_name); + return executeImpl(ast_function, context, table_name, std::move(cached_columns)); auto get_storage = [=, tf = shared_from_this()]() -> StoragePtr { - return tf->executeImpl(ast_function, context, table_name); + return tf->executeImpl(ast_function, context, table_name, cached_columns); }; - return std::make_shared(StorageID(getDatabaseName(), table_name), std::move(get_storage), std::move(cached_columns)); + /// It will request actual table structure and create underlying storage lazily + return std::make_shared(StorageID(getDatabaseName(), table_name), std::move(get_storage), + std::move(cached_columns), needStructureConversion()); } } diff --git a/src/TableFunctions/ITableFunction.h b/src/TableFunctions/ITableFunction.h index b41213a5e09..4a73adbdf80 100644 --- a/src/TableFunctions/ITableFunction.h +++ b/src/TableFunctions/ITableFunction.h @@ -22,6 +22,15 @@ class Context; * Example: * SELECT count() FROM remote('example01-01-1', merge, hits) * - go to `example01-01-1`, in `merge` database, `hits` table. + * + * + * When creating table AS table_function(...) we probably don't know structure of the table + * and have to request if from remote server, because structure is required to create a Storage. + * To avoid failures on server startup, we write obtained structure to metadata file. + * So, table function may have two different columns lists: + * - cached_columns written to metadata + * - the list returned from getActualTableStructure(...) + * See StorageTableFunctionProxy. */ class ITableFunction : public std::enable_shared_from_this @@ -32,10 +41,15 @@ public: /// Get the main function name. virtual std::string getName() const = 0; + /// Returns true if we always know table structure when executing table function + /// (e.g. structure is specified in table function arguments) virtual bool hasStaticStructure() const { return false; } + /// Returns false if storage returned by table function supports type conversion (e.g. StorageDistributed) + virtual bool needStructureConversion() const { return true; } virtual void parseArguments(const ASTPtr & /*ast_function*/, const Context & /*context*/) {} + /// Returns actual table structure probably requested from remote server, may fail virtual ColumnsDescription getActualTableStructure(const Context & /*context*/) const = 0; /// Create storage according to the query. @@ -44,7 +58,7 @@ public: virtual ~ITableFunction() {} private: - virtual StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const = 0; + virtual StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const = 0; virtual const char * getStorageTypeName() const = 0; }; diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index b362a646f9e..f876da02fd1 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -9,6 +9,8 @@ #include #include +#include +#include #include #include @@ -20,6 +22,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int INCORRECT_FILE_NAME; } void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, const Context & context) @@ -57,7 +60,7 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, const C compression_method = args[3]->as().value.safeGet(); } -StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const +StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { auto columns = getActualTableStructure(context); StoragePtr storage = getStorage(filename, format, columns, const_cast(context), table_name, compression_method); @@ -70,7 +73,11 @@ ColumnsDescription ITableFunctionFileLike::getActualTableStructure(const Context if (structure.empty()) { assert(getName() == "file" && format == "Distributed"); - return {}; /// TODO get matching path, read structure + Strings paths = StorageFile::getPathsList(filename, context.getUserFilesPath(), context); + if (paths.empty()) + throw Exception("Cannot get table structure from file, because no files match specified name", ErrorCodes::INCORRECT_FILE_NAME); + auto read_stream = StorageDistributedDirectoryMonitor::createStreamFromFile(paths[0]); + return ColumnsDescription{read_stream->getHeader().getNamesAndTypesList()}; } return parseColumnsListFromString(structure, context); } diff --git a/src/TableFunctions/ITableFunctionFileLike.h b/src/TableFunctions/ITableFunctionFileLike.h index 40111eebc5c..f1c648ac0aa 100644 --- a/src/TableFunctions/ITableFunctionFileLike.h +++ b/src/TableFunctions/ITableFunctionFileLike.h @@ -1,7 +1,6 @@ #pragma once #include -#include namespace DB { @@ -14,7 +13,7 @@ class Context; class ITableFunctionFileLike : public ITableFunction { private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override; virtual StoragePtr getStorage( const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, @@ -24,6 +23,8 @@ private: void parseArguments(const ASTPtr & ast_function, const Context & context) override; + bool hasStaticStructure() const override { return true; } + String filename; String format; String structure; diff --git a/src/TableFunctions/ITableFunctionXDBC.cpp b/src/TableFunctions/ITableFunctionXDBC.cpp index 6ba62b8ebf1..67d1257fe4c 100644 --- a/src/TableFunctions/ITableFunctionXDBC.cpp +++ b/src/TableFunctions/ITableFunctionXDBC.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include #include #include @@ -85,7 +84,7 @@ ColumnsDescription ITableFunctionXDBC::getActualTableStructure(const Context & c return ColumnsDescription{columns}; } -StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const +StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { assert(helper); auto columns = getActualTableStructure(context); diff --git a/src/TableFunctions/ITableFunctionXDBC.h b/src/TableFunctions/ITableFunctionXDBC.h index 218f361c025..fb0a0fd1185 100644 --- a/src/TableFunctions/ITableFunctionXDBC.h +++ b/src/TableFunctions/ITableFunctionXDBC.h @@ -18,7 +18,7 @@ namespace DB class ITableFunctionXDBC : public ITableFunction { private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override; /* A factory method to create bridge helper, that will assist in remote interaction */ virtual BridgeHelperPtr createBridgeHelper(Context & context, diff --git a/src/TableFunctions/TableFunctionGenerateRandom.cpp b/src/TableFunctions/TableFunctionGenerateRandom.cpp index c88cb3229b5..15c2c2bfa1f 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.cpp +++ b/src/TableFunctions/TableFunctionGenerateRandom.cpp @@ -13,9 +13,6 @@ #include #include -#include -#include - #include "registerTableFunctions.h" @@ -82,7 +79,7 @@ ColumnsDescription TableFunctionGenerateRandom::getActualTableStructure(const Co return parseColumnsListFromString(structure, context); } -StoragePtr TableFunctionGenerateRandom::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const +StoragePtr TableFunctionGenerateRandom::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { auto columns = getActualTableStructure(context); auto res = StorageGenerateRandom::create(StorageID(getDatabaseName(), table_name), columns, max_array_length, max_string_length, random_seed); diff --git a/src/TableFunctions/TableFunctionGenerateRandom.h b/src/TableFunctions/TableFunctionGenerateRandom.h index 8c13f853a0d..1d8839ba6d4 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.h +++ b/src/TableFunctions/TableFunctionGenerateRandom.h @@ -13,8 +13,9 @@ class TableFunctionGenerateRandom : public ITableFunction public: static constexpr auto name = "generateRandom"; std::string getName() const override { return name; } + bool hasStaticStructure() const override { return true; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override; const char * getStorageTypeName() const override { return "GenerateRandom"; } ColumnsDescription getActualTableStructure(const Context & context) const override; diff --git a/src/TableFunctions/TableFunctionInput.cpp b/src/TableFunctions/TableFunctionInput.cpp index 6563d8f0b3a..41c41835434 100644 --- a/src/TableFunctions/TableFunctionInput.cpp +++ b/src/TableFunctions/TableFunctionInput.cpp @@ -7,10 +7,8 @@ #include #include #include -#include #include #include -#include #include #include "registerTableFunctions.h" @@ -45,7 +43,7 @@ ColumnsDescription TableFunctionInput::getActualTableStructure(const Context & c return parseColumnsListFromString(structure, context); } -StoragePtr TableFunctionInput::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const +StoragePtr TableFunctionInput::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { auto storage = StorageInput::create(StorageID(getDatabaseName(), table_name), getActualTableStructure(context)); storage->startup(); diff --git a/src/TableFunctions/TableFunctionInput.h b/src/TableFunctions/TableFunctionInput.h index 9190fe600f6..5809d48a77c 100644 --- a/src/TableFunctions/TableFunctionInput.h +++ b/src/TableFunctions/TableFunctionInput.h @@ -15,9 +15,10 @@ class TableFunctionInput : public ITableFunction public: static constexpr auto name = "input"; std::string getName() const override { return name; } + bool hasStaticStructure() const override { return true; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override; const char * getStorageTypeName() const override { return "Input"; } ColumnsDescription getActualTableStructure(const Context & context) const override; diff --git a/src/TableFunctions/TableFunctionMerge.cpp b/src/TableFunctions/TableFunctionMerge.cpp index 6358a0bbbef..a878909e29d 100644 --- a/src/TableFunctions/TableFunctionMerge.cpp +++ b/src/TableFunctions/TableFunctionMerge.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include #include #include @@ -74,7 +73,7 @@ ColumnsDescription TableFunctionMerge::getActualTableStructure(const Context & c return ColumnsDescription{chooseColumns(source_database, table_name_regexp, context)}; } -StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const +StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { auto res = StorageMerge::create( StorageID(getDatabaseName(), table_name), diff --git a/src/TableFunctions/TableFunctionMerge.h b/src/TableFunctions/TableFunctionMerge.h index 2d038de32fb..a9c4dd6b038 100644 --- a/src/TableFunctions/TableFunctionMerge.h +++ b/src/TableFunctions/TableFunctionMerge.h @@ -16,7 +16,7 @@ public: static constexpr auto name = "merge"; std::string getName() const override { return name; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override; const char * getStorageTypeName() const override { return "Merge"; } ColumnsDescription getActualTableStructure(const Context & context) const override; diff --git a/src/TableFunctions/TableFunctionMySQL.cpp b/src/TableFunctions/TableFunctionMySQL.cpp index c957bb8c05f..ca67cdf2b24 100644 --- a/src/TableFunctions/TableFunctionMySQL.cpp +++ b/src/TableFunctions/TableFunctionMySQL.cpp @@ -13,7 +13,6 @@ # include # include # include -# include # include # include # include @@ -121,7 +120,7 @@ ColumnsDescription TableFunctionMySQL::getActualTableStructure(const Context & c return ColumnsDescription{columns}; } -StoragePtr TableFunctionMySQL::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const +StoragePtr TableFunctionMySQL::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { assert(!parsed_host_port.first.empty()); if (!pool) diff --git a/src/TableFunctions/TableFunctionMySQL.h b/src/TableFunctions/TableFunctionMySQL.h index f3c15a5fdd9..1fe5a4aa4ac 100644 --- a/src/TableFunctions/TableFunctionMySQL.h +++ b/src/TableFunctions/TableFunctionMySQL.h @@ -24,7 +24,7 @@ public: return name; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override; const char * getStorageTypeName() const override { return "MySQL"; } ColumnsDescription getActualTableStructure(const Context & context) const override; diff --git a/src/TableFunctions/TableFunctionNumbers.cpp b/src/TableFunctions/TableFunctionNumbers.cpp index 84edd3935b3..984b26d5de4 100644 --- a/src/TableFunctions/TableFunctionNumbers.cpp +++ b/src/TableFunctions/TableFunctionNumbers.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include #include #include @@ -28,7 +27,7 @@ ColumnsDescription TableFunctionNumbers::getActualTableStructure( } template -StoragePtr TableFunctionNumbers::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +StoragePtr TableFunctionNumbers::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { if (const auto * function = ast_function->as()) { diff --git a/src/TableFunctions/TableFunctionNumbers.h b/src/TableFunctions/TableFunctionNumbers.h index 5cbde6cf0a3..9c54125551f 100644 --- a/src/TableFunctions/TableFunctionNumbers.h +++ b/src/TableFunctions/TableFunctionNumbers.h @@ -17,8 +17,9 @@ class TableFunctionNumbers : public ITableFunction public: static constexpr auto name = multithreaded ? "numbers_mt" : "numbers"; std::string getName() const override { return name; } + bool hasStaticStructure() const override { return true; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override; const char * getStorageTypeName() const override { return "SystemNumbers"; } UInt64 evaluateArgument(const Context & context, ASTPtr & argument) const; diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 594b47a3666..2e34e82ce36 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -45,7 +45,6 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, const Cont String cluster_description; String remote_database; String remote_table; - //ASTPtr remote_table_function_ptr; String username; String password; @@ -137,7 +136,6 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, const Cont for (auto ast : args) setIdentifierSpecial(ast); - //ClusterPtr cluster; if (!cluster_name.empty()) { /// Use an existing cluster from the main config @@ -190,18 +188,22 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, const Cont if (!remote_table_function_ptr && remote_table.empty()) throw Exception("The name of remote table cannot be empty", ErrorCodes::BAD_ARGUMENTS); - //auto remote_table_id = StorageID::createEmpty(); remote_table_id.database_name = remote_database; remote_table_id.table_name = remote_table; } -StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const +StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const { + /// StorageDistributed supports mismatching structure of remote table, so we can use outdated structure for CREATE ... AS remote(...) + /// without additional conversion in StorageTableFunctionProxy + if (cached_columns.empty()) + cached_columns = getActualTableStructure(context); + assert(cluster); StoragePtr res = remote_table_function_ptr ? StorageDistributed::create( StorageID(getDatabaseName(), table_name), - getActualTableStructure(context), + cached_columns, ConstraintsDescription{}, remote_table_function_ptr, String{}, @@ -213,7 +215,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & /*ast_function*/, con cluster) : StorageDistributed::create( StorageID(getDatabaseName(), table_name), - getActualTableStructure(context), + cached_columns, ConstraintsDescription{}, remote_table_id.database_name, remote_table_id.table_name, diff --git a/src/TableFunctions/TableFunctionRemote.h b/src/TableFunctions/TableFunctionRemote.h index ff214bb7332..6ec591a34ac 100644 --- a/src/TableFunctions/TableFunctionRemote.h +++ b/src/TableFunctions/TableFunctionRemote.h @@ -24,8 +24,10 @@ public: ColumnsDescription getActualTableStructure(const Context & context) const override; + bool needStructureConversion() const override { return false; } + private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override; const char * getStorageTypeName() const override { return "Distributed"; } void parseArguments(const ASTPtr & ast_function, const Context & context) override; diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index a4f403f7875..dfe1cf6e792 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -4,7 +4,6 @@ #include #include -#include #include #include #include @@ -63,7 +62,7 @@ ColumnsDescription TableFunctionS3::getActualTableStructure(const Context & cont return parseColumnsListFromString(structure, context); } -StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const +StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { Poco::URI uri (filename); S3::URI s3_uri (uri); diff --git a/src/TableFunctions/TableFunctionS3.h b/src/TableFunctions/TableFunctionS3.h index 45738de8ba8..722fb9eb23c 100644 --- a/src/TableFunctions/TableFunctionS3.h +++ b/src/TableFunctions/TableFunctionS3.h @@ -22,12 +22,14 @@ public: { return name; } + bool hasStaticStructure() const override { return true; } protected: StoragePtr executeImpl( const ASTPtr & ast_function, const Context & context, - const std::string & table_name) const override; + const std::string & table_name, + ColumnsDescription cached_columns) const override; const char * getStorageTypeName() const override { return "S3"; } diff --git a/src/TableFunctions/TableFunctionValues.cpp b/src/TableFunctions/TableFunctionValues.cpp index 4e892c83247..2d01d581fda 100644 --- a/src/TableFunctions/TableFunctionValues.cpp +++ b/src/TableFunctions/TableFunctionValues.cpp @@ -3,7 +3,6 @@ #include #include -#include #include #include @@ -95,7 +94,7 @@ ColumnsDescription TableFunctionValues::getActualTableStructure(const Context & return parseColumnsListFromString(structure, context); } -StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { auto columns = getActualTableStructure(context); diff --git a/src/TableFunctions/TableFunctionValues.h b/src/TableFunctions/TableFunctionValues.h index b66f5bb457f..549fa2de507 100644 --- a/src/TableFunctions/TableFunctionValues.h +++ b/src/TableFunctions/TableFunctionValues.h @@ -12,8 +12,9 @@ class TableFunctionValues : public ITableFunction public: static constexpr auto name = "values"; std::string getName() const override { return name; } + bool hasStaticStructure() const override { return true; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override; const char * getStorageTypeName() const override { return "Values"; } ColumnsDescription getActualTableStructure(const Context & context) const override; diff --git a/src/TableFunctions/TableFunctionZeros.cpp b/src/TableFunctions/TableFunctionZeros.cpp index 5ba6c034e09..22bf47f5e9f 100644 --- a/src/TableFunctions/TableFunctionZeros.cpp +++ b/src/TableFunctions/TableFunctionZeros.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include #include #include @@ -27,7 +26,7 @@ ColumnsDescription TableFunctionZeros::getActualTableStructure(co } template -StoragePtr TableFunctionZeros::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +StoragePtr TableFunctionZeros::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { if (const auto * function = ast_function->as()) { diff --git a/src/TableFunctions/TableFunctionZeros.h b/src/TableFunctions/TableFunctionZeros.h index c50a17d046d..2944e2c1c9d 100644 --- a/src/TableFunctions/TableFunctionZeros.h +++ b/src/TableFunctions/TableFunctionZeros.h @@ -17,8 +17,9 @@ class TableFunctionZeros : public ITableFunction public: static constexpr auto name = multithreaded ? "zeros_mt" : "zeros"; std::string getName() const override { return name; } + bool hasStaticStructure() const override { return true; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override; const char * getStorageTypeName() const override { return "SystemZeros"; } UInt64 evaluateArgument(const Context & context, ASTPtr & argument) const; diff --git a/tests/queries/0_stateless/014 b/tests/queries/0_stateless/014 deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/01457_create_as_table_function_structure.reference b/tests/queries/0_stateless/01457_create_as_table_function_structure.reference index 7169648d524..0deaa20171a 100644 --- a/tests/queries/0_stateless/01457_create_as_table_function_structure.reference +++ b/tests/queries/0_stateless/01457_create_as_table_function_structure.reference @@ -1,7 +1,10 @@ CREATE TABLE test_01457.tf_remote\n(\n `n` Int8\n) AS remote(\'localhost\', \'default\', \'tmp\') CREATE TABLE test_01457.tf_remote_explicit_structure\n(\n `n` UInt64\n) AS remote(\'localhost\', \'default\', \'tmp\') CREATE TABLE test_01457.tf_numbers\n(\n `number` String\n) AS numbers(1) +CREATE TABLE test_01457.tf_merge\n(\n `n` Int8\n) AS merge(\'default\', \'tmp\') +42 0 Int8 0 Int8 0 UInt64 0 String +0 Int8 diff --git a/tests/queries/0_stateless/01457_create_as_table_function_structure.sql b/tests/queries/0_stateless/01457_create_as_table_function_structure.sql index 1949c084960..edfab747fc4 100644 --- a/tests/queries/0_stateless/01457_create_as_table_function_structure.sql +++ b/tests/queries/0_stateless/01457_create_as_table_function_structure.sql @@ -10,6 +10,8 @@ CREATE TABLE test_01457.tf_remote_explicit_structure (n UInt64) AS remote('local SHOW CREATE TABLE test_01457.tf_remote_explicit_structure; CREATE TABLE test_01457.tf_numbers (number String) AS numbers(1); SHOW CREATE TABLE test_01457.tf_numbers; +CREATE TABLE test_01457.tf_merge AS merge(currentDatabase(), 'tmp'); +SHOW CREATE TABLE test_01457.tf_merge; DROP TABLE tmp; @@ -17,12 +19,15 @@ DETACH DATABASE test_01457; ATTACH DATABASE test_01457; CREATE TABLE tmp (n Int8) ENGINE=Memory; -INSERT INTO test_01457.tf_remote_explicit_structure VALUES (0); -- { serverError 122 } +INSERT INTO test_01457.tf_remote_explicit_structure VALUES ('42'); +SELECT * FROM tmp; +TRUNCATE TABLE tmp; INSERT INTO test_01457.tf_remote VALUES (0); SELECT (*,).1 AS c, toTypeName(c) FROM tmp; SELECT (*,).1 AS c, toTypeName(c) FROM test_01457.tf_remote; SELECT (*,).1 AS c, toTypeName(c) FROM test_01457.tf_remote_explicit_structure; SELECT (*,).1 AS c, toTypeName(c) FROM test_01457.tf_numbers; +SELECT (*,).1 AS c, toTypeName(c) FROM test_01457.tf_merge; DROP DATABASE test_01457; From 4bf3f9e4a3bdd17e4f2e2fa35e971641ea5830cd Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 2 Sep 2020 18:22:53 +0300 Subject: [PATCH 09/42] use StorageProxy for StorageMaterializeMySQL --- src/Storages/StorageMaterializeMySQL.cpp | 2 +- src/Storages/StorageMaterializeMySQL.h | 20 +++++++++++++++----- src/Storages/StorageProxy.h | 1 + 3 files changed, 17 insertions(+), 6 deletions(-) diff --git a/src/Storages/StorageMaterializeMySQL.cpp b/src/Storages/StorageMaterializeMySQL.cpp index 7d908736bdc..68981280619 100644 --- a/src/Storages/StorageMaterializeMySQL.cpp +++ b/src/Storages/StorageMaterializeMySQL.cpp @@ -23,7 +23,7 @@ namespace DB { StorageMaterializeMySQL::StorageMaterializeMySQL(const StoragePtr & nested_storage_, const DatabaseMaterializeMySQL * database_) - : IStorage(nested_storage_->getStorageID()), nested_storage(nested_storage_), database(database_) + : StorageProxy(nested_storage_->getStorageID()), nested_storage(nested_storage_), database(database_) { auto nested_memory_metadata = nested_storage->getInMemoryMetadata(); StorageInMemoryMetadata in_memory_metadata; diff --git a/src/Storages/StorageMaterializeMySQL.h b/src/Storages/StorageMaterializeMySQL.h index 4278ce64bd7..cadafaf778c 100644 --- a/src/Storages/StorageMaterializeMySQL.h +++ b/src/Storages/StorageMaterializeMySQL.h @@ -4,30 +4,40 @@ #if USE_MYSQL -#include +#include #include namespace DB { -class StorageMaterializeMySQL final : public ext::shared_ptr_helper, public IStorage +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +class StorageMaterializeMySQL final : public ext::shared_ptr_helper, public StorageProxy { friend struct ext::shared_ptr_helper; public: String getName() const override { return "MaterializeMySQL"; } - bool supportsFinal() const override { return nested_storage->supportsFinal(); } - bool supportsSampling() const override { return nested_storage->supportsSampling(); } - StorageMaterializeMySQL(const StoragePtr & nested_storage_, const DatabaseMaterializeMySQL * database_); Pipe read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; + BlockOutputStreamPtr write(const ASTPtr &, const StorageMetadataPtr &, const Context &) override { throwNotAllowed(); } + NamesAndTypesList getVirtuals() const override; private: + StoragePtr getNested() const override { return nested_storage; } + [[noreturn]] void throwNotAllowed() const + { + throw Exception("This method is not allowed for MaterializeMySQ", ErrorCodes::NOT_IMPLEMENTED); + } + StoragePtr nested_storage; const DatabaseMaterializeMySQL * database; }; diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index 92200ff5775..7b010476b22 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB From 9d98effa5e166ff15a9bf61b6b76b7c72566f173 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 2 Sep 2020 20:39:37 +0300 Subject: [PATCH 10/42] write structure for TableFunctionView --- src/TableFunctions/TableFunctionView.cpp | 42 +++++++++++++----------- src/TableFunctions/TableFunctionView.h | 9 +++-- 2 files changed, 29 insertions(+), 22 deletions(-) diff --git a/src/TableFunctions/TableFunctionView.cpp b/src/TableFunctions/TableFunctionView.cpp index 6166fa56f47..066b67de0fc 100644 --- a/src/TableFunctions/TableFunctionView.cpp +++ b/src/TableFunctions/TableFunctionView.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include @@ -16,25 +15,30 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -StoragePtr TableFunctionView::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +void TableFunctionView::parseArguments(const ASTPtr & ast_function, const Context & /*context*/) { - if (const auto * function = ast_function->as()) - { - if (function->query) - { - if (auto * select = function->query->as()) - { - auto sample = InterpreterSelectWithUnionQuery::getSampleBlock(function->query, context); - auto columns = ColumnsDescription(sample.getNamesAndTypesList()); - ASTCreateQuery create; - create.select = select; - auto res = StorageView::create(StorageID(getDatabaseName(), table_name), create, columns); - res->startup(); - return res; - } - } - } - throw Exception("Table function '" + getName() + "' requires a query argument.", ErrorCodes::BAD_ARGUMENTS); + const auto * function = ast_function->as(); + if (function && function->query && function->query->as()) + create.set(create.select, function->query); + else + throw Exception("Table function '" + getName() + "' requires a query argument.", ErrorCodes::BAD_ARGUMENTS); +} + +ColumnsDescription TableFunctionView::getActualTableStructure(const Context & context) const +{ + assert(create.select); + assert(create.children.size() == 1); + assert(create.children[0]->as()); + auto sample = InterpreterSelectWithUnionQuery::getSampleBlock(create.children[0], context); + return ColumnsDescription(sample.getNamesAndTypesList()); +} + +StoragePtr TableFunctionView::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const +{ + auto columns = getActualTableStructure(context); + auto res = StorageView::create(StorageID(getDatabaseName(), table_name), create, columns); + res->startup(); + return res; } void registerTableFunctionView(TableFunctionFactory & factory) diff --git a/src/TableFunctions/TableFunctionView.h b/src/TableFunctions/TableFunctionView.h index 49f51823735..f82d182d159 100644 --- a/src/TableFunctions/TableFunctionView.h +++ b/src/TableFunctions/TableFunctionView.h @@ -2,7 +2,7 @@ #include #include - +#include namespace DB { @@ -17,10 +17,13 @@ public: static constexpr auto name = "view"; std::string getName() const override { return name; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override; const char * getStorageTypeName() const override { return "View"; } - UInt64 evaluateArgument(const Context & context, ASTPtr & argument) const; + void parseArguments(const ASTPtr & ast_function, const Context & context) override; + ColumnsDescription getActualTableStructure(const Context & context) const override; + + ASTCreateQuery create; }; From df09bf3cb67ee130bb545693458ad464d44147ad Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 3 Sep 2020 22:50:26 +0300 Subject: [PATCH 11/42] add suppressions for leak sanitizer --- docker/test/stateful/run.sh | 1 + docker/test/stateless/run.sh | 1 + src/TableFunctions/CMakeLists.txt | 4 + src/TableFunctions/ITableFunction.cpp | 3 +- src/TableFunctions/tests/CMakeLists.txt | 2 + .../tests/test_strange_memory_leak.cpp | 113 ++++++++++++++++++ tests/config/lsan_suppressions.txt | 2 + 7 files changed, 124 insertions(+), 2 deletions(-) create mode 100644 src/TableFunctions/tests/CMakeLists.txt create mode 100644 src/TableFunctions/tests/test_strange_memory_leak.cpp create mode 100644 tests/config/lsan_suppressions.txt diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 5be14970914..e7e1fc54a0a 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -35,6 +35,7 @@ echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment echo "ASAN_SYMBOLIZER_PATH=/usr/lib/llvm-10/bin/llvm-symbolizer" >> /etc/environment echo "UBSAN_SYMBOLIZER_PATH=/usr/lib/llvm-10/bin/llvm-symbolizer" >> /etc/environment echo "LLVM_SYMBOLIZER_PATH=/usr/lib/llvm-10/bin/llvm-symbolizer" >> /etc/environment +echo "LSAN_OPTIONS='suppressions=/usr/share/clickhouse-test/config/lsan_suppressions.txt'" >> /etc/environment service zookeeper start sleep 5 diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 2ff15ca9c6a..bb587ce73c2 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -53,6 +53,7 @@ echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment echo "ASAN_SYMBOLIZER_PATH=/usr/lib/llvm-10/bin/llvm-symbolizer" >> /etc/environment echo "UBSAN_SYMBOLIZER_PATH=/usr/lib/llvm-10/bin/llvm-symbolizer" >> /etc/environment echo "LLVM_SYMBOLIZER_PATH=/usr/lib/llvm-10/bin/llvm-symbolizer" >> /etc/environment +echo "LSAN_OPTIONS='suppressions=/usr/share/clickhouse-test/config/lsan_suppressions.txt'" >> /etc/environment service zookeeper start sleep 5 diff --git a/src/TableFunctions/CMakeLists.txt b/src/TableFunctions/CMakeLists.txt index 33e900231e5..19748fefaf9 100644 --- a/src/TableFunctions/CMakeLists.txt +++ b/src/TableFunctions/CMakeLists.txt @@ -6,3 +6,7 @@ list(REMOVE_ITEM clickhouse_table_functions_headers ITableFunction.h TableFuncti add_library(clickhouse_table_functions ${clickhouse_table_functions_sources}) target_link_libraries(clickhouse_table_functions PRIVATE clickhouse_parsers clickhouse_storages_system dbms) + +if(ENABLE_TESTS) + add_subdirectory(tests) +endif() diff --git a/src/TableFunctions/ITableFunction.cpp b/src/TableFunctions/ITableFunction.cpp index fd5cdf6e219..804a5b232ec 100644 --- a/src/TableFunctions/ITableFunction.cpp +++ b/src/TableFunctions/ITableFunction.cpp @@ -20,8 +20,7 @@ StoragePtr ITableFunction::execute(const ASTPtr & ast_function, const Context & ProfileEvents::increment(ProfileEvents::TableFunctionExecute); context.checkAccess(AccessType::CREATE_TEMPORARY_TABLE | StorageFactory::instance().getSourceAccessType(getStorageTypeName())); - bool no_conversion_required = hasStaticStructure() && cached_columns == getActualTableStructure(context); - if (cached_columns.empty() || no_conversion_required) + if (cached_columns.empty() || (hasStaticStructure() && cached_columns == getActualTableStructure(context))) return executeImpl(ast_function, context, table_name, std::move(cached_columns)); auto get_storage = [=, tf = shared_from_this()]() -> StoragePtr diff --git a/src/TableFunctions/tests/CMakeLists.txt b/src/TableFunctions/tests/CMakeLists.txt new file mode 100644 index 00000000000..f2ed25a0753 --- /dev/null +++ b/src/TableFunctions/tests/CMakeLists.txt @@ -0,0 +1,2 @@ +add_executable (test-strange-memory-leak test_strange_memory_leak.cpp ${SRCS}) +#target_link_libraries (test-strange-memory-leak PRIVATE dbms) diff --git a/src/TableFunctions/tests/test_strange_memory_leak.cpp b/src/TableFunctions/tests/test_strange_memory_leak.cpp new file mode 100644 index 00000000000..a1e8af34616 --- /dev/null +++ b/src/TableFunctions/tests/test_strange_memory_leak.cpp @@ -0,0 +1,113 @@ +/// Demonstrates mysterious memory leak when built using clang version 10.0.0-4ubuntu1~18.04.2 +/// Ne leaks are found by valgrind when built using gcc version 9.2.1 20191102 + +//#include +//#include + +//class ITableFunction : public std::enable_shared_from_this +//{ +//public: +// virtual DB::ColumnsDescription getActualTableStructure() const = 0; +// virtual ~ITableFunction() {} +// +//}; +// +//class TableFunction : public ITableFunction +//{ +// DB::ColumnsDescription getActualTableStructure() const override +// { +// return DB::ColumnsDescription({{"number", DB::DataTypePtr{}}}); +// } +//}; +// +//template +//class TableFunctionTemplate : public ITableFunction +//{ +// DB::ColumnsDescription getActualTableStructure() const override +// { +// return DB::ColumnsDescription({{"number", DB::DataTypePtr{}}}); +// } +//}; + +/// Simplified version of the commented code above (without dependencies on ClickHouse code): + +#include +#include +#include +#include +#include +#include +#include + +using LolPtr = std::shared_ptr; +struct Elem +{ + Elem(std::string n, LolPtr t) : name(std::move(n)), type(std::move(t)) {} + std::string name; + LolPtr type; +}; + +using Container = boost::multi_index_container< + Elem, + boost::multi_index::indexed_by< + boost::multi_index::sequenced<>, + boost::multi_index::ordered_unique>>>; + +struct List : public std::list +{ + List(std::initializer_list init) : std::list(init) {} +}; + +struct Kek +{ + Container container; + Kek(List list) + { + for (auto & elem : list) + add(Elem(std::move(elem.name), std::move(elem.type))); + } + + void add(Elem column) + { + auto insert_it = container.cbegin(); + container.get<0>().insert(insert_it, std::move(column)); + } + +}; + +class ITableFunction : public std::enable_shared_from_this +{ +public: + virtual Kek getActualTableStructure() const = 0; + virtual ~ITableFunction() {} + +}; + +class TableFunction : public ITableFunction +{ + Kek getActualTableStructure() const override + { + return Kek({{"number", LolPtr{}}}); + } +}; + +template +class TableFunctionTemplate : public ITableFunction +{ + Kek getActualTableStructure() const override + { + return Kek({{"number", LolPtr{}}}); + } +}; + +int main() +{ + /// Works fine + const ITableFunction & tf1 = TableFunction{}; + tf1.getActualTableStructure(); + + /// ERROR: LeakSanitizer: detected memory leaks + /// and the same error with valgrind + const ITableFunction & tf2 = TableFunctionTemplate{}; + tf2.getActualTableStructure(); +} diff --git a/tests/config/lsan_suppressions.txt b/tests/config/lsan_suppressions.txt new file mode 100644 index 00000000000..870bbe5e081 --- /dev/null +++ b/tests/config/lsan_suppressions.txt @@ -0,0 +1,2 @@ +# See src/TableFunctions/tests/test-strange-memory-leak.cpp +leak:getActualTableStructure From 3cb73de0595a4ab3104e81293575a7b62f1c3292 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 4 Sep 2020 01:23:27 +0300 Subject: [PATCH 12/42] try fix fast build --- src/TableFunctions/tests/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/src/TableFunctions/tests/CMakeLists.txt b/src/TableFunctions/tests/CMakeLists.txt index f2ed25a0753..bdc6515ecfd 100644 --- a/src/TableFunctions/tests/CMakeLists.txt +++ b/src/TableFunctions/tests/CMakeLists.txt @@ -1,2 +1,3 @@ add_executable (test-strange-memory-leak test_strange_memory_leak.cpp ${SRCS}) #target_link_libraries (test-strange-memory-leak PRIVATE dbms) +target_link_libraries (test-strange-memory-leak PUBLIC boost::headers_only) From c233be29c57a694c3e64fd9182c68da8c1592975 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 4 Sep 2020 16:29:43 +0300 Subject: [PATCH 13/42] fix --- src/Interpreters/InterpreterCreateQuery.cpp | 1 + src/TableFunctions/tests/CMakeLists.txt | 2 +- .../tests/test_strange_memory_leak.cpp | 45 ++++++++++++++----- src/TableFunctions/ya.make | 1 + .../test_distributed_format/test.py | 6 +-- 5 files changed, 40 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 20825f35109..d053d8e6870 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -587,6 +587,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const if (create.storage || create.is_view || create.is_materialized_view || create.is_live_view || create.is_dictionary) { + assert(create.storage->engine); if (create.temporary && create.storage->engine->name != "Memory") throw Exception( "Temporary tables can only be created with ENGINE = Memory, not " + create.storage->engine->name, diff --git a/src/TableFunctions/tests/CMakeLists.txt b/src/TableFunctions/tests/CMakeLists.txt index bdc6515ecfd..6b70ce0beca 100644 --- a/src/TableFunctions/tests/CMakeLists.txt +++ b/src/TableFunctions/tests/CMakeLists.txt @@ -1,3 +1,3 @@ add_executable (test-strange-memory-leak test_strange_memory_leak.cpp ${SRCS}) #target_link_libraries (test-strange-memory-leak PRIVATE dbms) -target_link_libraries (test-strange-memory-leak PUBLIC boost::headers_only) +#target_link_libraries (test-strange-memory-leak PUBLIC boost::headers_only) diff --git a/src/TableFunctions/tests/test_strange_memory_leak.cpp b/src/TableFunctions/tests/test_strange_memory_leak.cpp index a1e8af34616..dcf6c779429 100644 --- a/src/TableFunctions/tests/test_strange_memory_leak.cpp +++ b/src/TableFunctions/tests/test_strange_memory_leak.cpp @@ -1,5 +1,5 @@ /// Demonstrates mysterious memory leak when built using clang version 10.0.0-4ubuntu1~18.04.2 -/// Ne leaks are found by valgrind when built using gcc version 9.2.1 20191102 +/// No leaks are found by valgrind when built using gcc version 9.2.1 20191102 //#include //#include @@ -34,10 +34,23 @@ #include #include #include -#include -#include -#include -#include +#include +#include +//#include +//#include +//#include +//#include + +struct Print +{ + std::string data = "lol"; + Print(const Print &) { std::cout << "copy" << std::endl; } + Print(Print &&) { std::cout << "move" << std::endl; } + Print & operator = (const Print &) { std::cout << "copy=" << std::endl; return *this; } + Print & operator = (Print &&) { std::cout << "move=" << std::endl; return *this; } + Print() { std::cout << "ctor" << std::endl; } + ~Print() { std::cout << "dtor" << std::endl; } +}; using LolPtr = std::shared_ptr; struct Elem @@ -45,13 +58,16 @@ struct Elem Elem(std::string n, LolPtr t) : name(std::move(n)), type(std::move(t)) {} std::string name; LolPtr type; + Print print; }; -using Container = boost::multi_index_container< - Elem, - boost::multi_index::indexed_by< - boost::multi_index::sequenced<>, - boost::multi_index::ordered_unique>>>; +//using Container = boost::multi_index_container< +// Elem, +// boost::multi_index::indexed_by< +// boost::multi_index::sequenced<>, +// boost::multi_index::ordered_unique>>>; +/// Simplified version: +using Container = std::map; struct List : public std::list { @@ -70,7 +86,8 @@ struct Kek void add(Elem column) { auto insert_it = container.cbegin(); - container.get<0>().insert(insert_it, std::move(column)); + //container.get<0>().insert(insert_it, std::move(column)); + container.insert(insert_it, {column.name, column}); } }; @@ -102,12 +119,18 @@ class TableFunctionTemplate : public ITableFunction int main() { + std::cout << "0" << std::endl; + /// Works fine const ITableFunction & tf1 = TableFunction{}; tf1.getActualTableStructure(); + std::cout << "1" << std::endl; + /// ERROR: LeakSanitizer: detected memory leaks /// and the same error with valgrind const ITableFunction & tf2 = TableFunctionTemplate{}; tf2.getActualTableStructure(); + + std::cout << "2" << std::endl; } diff --git a/src/TableFunctions/ya.make b/src/TableFunctions/ya.make index e87c96073bd..36218bee1e0 100644 --- a/src/TableFunctions/ya.make +++ b/src/TableFunctions/ya.make @@ -23,6 +23,7 @@ SRCS( TableFunctionValues.cpp TableFunctionView.cpp TableFunctionZeros.cpp + tests/test_strange_memory_leak.cpp ) diff --git a/tests/integration/test_distributed_format/test.py b/tests/integration/test_distributed_format/test.py index 291db89ae4c..08d2f0087a4 100644 --- a/tests/integration/test_distributed_format/test.py +++ b/tests/integration/test_distributed_format/test.py @@ -37,7 +37,7 @@ def test_single_file(started_cluster, cluster): assert out == '1\ta\n2\tbb\n3\tccc\n' - query = "create table t (dummy UInt32) engine = File('Distributed', '/var/lib/clickhouse/data/test/distr_1/shard1_replica1/1.bin');" \ + query = "create table t (x UInt64, s String) engine = File('Distributed', '/var/lib/clickhouse/data/test/distr_1/shard1_replica1/1.bin');" \ "select * from t" out = node.exec_in_container(['/usr/bin/clickhouse', 'local', '--stacktrace', '-q', query]) @@ -57,7 +57,7 @@ def test_two_files(started_cluster, cluster): assert out == '0\t_\n1\ta\n2\tbb\n3\tccc\n' - query = "create table t (dummy UInt32) engine = File('Distributed', '/var/lib/clickhouse/data/test/distr_2/shard1_replica1/{1,2,3,4}.bin');" \ + query = "create table t (x UInt64, s String) engine = File('Distributed', '/var/lib/clickhouse/data/test/distr_2/shard1_replica1/{1,2,3,4}.bin');" \ "select * from t order by x" out = node.exec_in_container(['/usr/bin/clickhouse', 'local', '--stacktrace', '-q', query]) @@ -76,7 +76,7 @@ def test_single_file_old(started_cluster, cluster): assert out == '1\ta\n2\tbb\n3\tccc\n' - query = "create table t (dummy UInt32) engine = File('Distributed', '/var/lib/clickhouse/data/test/distr_3/default@not_existing:9000/1.bin');" \ + query = "create table t (x UInt64, s String) engine = File('Distributed', '/var/lib/clickhouse/data/test/distr_3/default@not_existing:9000/1.bin');" \ "select * from t" out = node.exec_in_container(['/usr/bin/clickhouse', 'local', '--stacktrace', '-q', query]) From 98963deed33812d66bd4159c6ab1b98377fbee81 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 4 Sep 2020 19:31:37 +0300 Subject: [PATCH 14/42] fix --- src/Interpreters/InterpreterCreateQuery.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index d053d8e6870..0e7c2e5a48f 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -587,8 +587,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const if (create.storage || create.is_view || create.is_materialized_view || create.is_live_view || create.is_dictionary) { - assert(create.storage->engine); - if (create.temporary && create.storage->engine->name != "Memory") + if (create.temporary && create.storage && create.storage->engine && create.storage->engine->name != "Memory") throw Exception( "Temporary tables can only be created with ENGINE = Memory, not " + create.storage->engine->name, ErrorCodes::INCORRECT_QUERY); From 88e5476dbe6c9491427b7e2f4088fe81ac7d23c4 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sat, 5 Sep 2020 00:40:31 +0300 Subject: [PATCH 15/42] Docs for the cast_keep_nullable setting (English). --- docs/en/operations/settings/settings.md | 49 +++++++++++++++++++++++++ 1 file changed, 49 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 4995c04f712..9bbdcacd1ab 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1899,4 +1899,53 @@ Possible values: Default value: `120`. +## cast_keep_nullable {#cast_keep_nullable} + +Enables or disables converstion to nullable types for [CAST](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) operator. + +If set, `CAST(something_nullable AS Type)` returns `Nullable(Type)`. + +Possible values: + +- 0 — Nullable type converstion disabled. +- 1 — Nullable type converstion enabled. + +Default value: `0`. + +**Examples** + +Query with nullable type converstion disabled: + +```sql +SET cast_keep_nullable = 0; +SELECT CAST(toNullable(toInt32(0)) AS Int32) as x, toTypeName(x); +``` + +Result: + +``` +┌─x─┬─toTypeName(CAST(toNullable(toInt32(0)), 'Int32'))─┐ +│ 0 │ Int32 │ +└───┴───────────────────────────────────────────────────┘ +``` + +Query with nullable type converstion enabled: + +```sql +SET cast_keep_nullable = 1; +SELECT CAST(toNullable(toInt32(0)) AS Int32) as x, toTypeName(x); +``` + +Result: + +``` +┌─x─┬─toTypeName(CAST(toNullable(toInt32(0)), 'Int32'))─┐ +│ 0 │ Nullable(Int32) │ +└───┴───────────────────────────────────────────────────┘ +``` + +**See Also** + +- [CAST](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) operator + [Original article](https://clickhouse.tech/docs/en/operations/settings/settings/) From a76139bc223ec952c5c852b0cecd5dd285b20c9d Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sat, 5 Sep 2020 01:00:07 +0300 Subject: [PATCH 16/42] Minor fixes. --- docs/en/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 9bbdcacd1ab..dc93d31dea4 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1923,7 +1923,7 @@ SELECT CAST(toNullable(toInt32(0)) AS Int32) as x, toTypeName(x); Result: -``` +```text ┌─x─┬─toTypeName(CAST(toNullable(toInt32(0)), 'Int32'))─┐ │ 0 │ Int32 │ └───┴───────────────────────────────────────────────────┘ @@ -1938,7 +1938,7 @@ SELECT CAST(toNullable(toInt32(0)) AS Int32) as x, toTypeName(x); Result: -``` +```text ┌─x─┬─toTypeName(CAST(toNullable(toInt32(0)), 'Int32'))─┐ │ 0 │ Nullable(Int32) │ └───┴───────────────────────────────────────────────────┘ From c5df4647e6f5571b55e72f680e4ba8eb214e5c96 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 7 Sep 2020 14:30:12 +0300 Subject: [PATCH 17/42] add workaround for memory leak --- src/TableFunctions/CMakeLists.txt | 3 - src/TableFunctions/TableFunctionNumbers.cpp | 3 +- src/TableFunctions/TableFunctionZeros.cpp | 3 +- src/TableFunctions/tests/CMakeLists.txt | 3 - .../tests/test_strange_memory_leak.cpp | 136 ------------------ src/TableFunctions/ya.make | 1 - tests/config/lsan_suppressions.txt | 4 +- 7 files changed, 6 insertions(+), 147 deletions(-) delete mode 100644 src/TableFunctions/tests/CMakeLists.txt delete mode 100644 src/TableFunctions/tests/test_strange_memory_leak.cpp diff --git a/src/TableFunctions/CMakeLists.txt b/src/TableFunctions/CMakeLists.txt index 19748fefaf9..8e9eedadf53 100644 --- a/src/TableFunctions/CMakeLists.txt +++ b/src/TableFunctions/CMakeLists.txt @@ -7,6 +7,3 @@ list(REMOVE_ITEM clickhouse_table_functions_headers ITableFunction.h TableFuncti add_library(clickhouse_table_functions ${clickhouse_table_functions_sources}) target_link_libraries(clickhouse_table_functions PRIVATE clickhouse_parsers clickhouse_storages_system dbms) -if(ENABLE_TESTS) - add_subdirectory(tests) -endif() diff --git a/src/TableFunctions/TableFunctionNumbers.cpp b/src/TableFunctions/TableFunctionNumbers.cpp index 984b26d5de4..4658165735a 100644 --- a/src/TableFunctions/TableFunctionNumbers.cpp +++ b/src/TableFunctions/TableFunctionNumbers.cpp @@ -23,7 +23,8 @@ namespace ErrorCodes template ColumnsDescription TableFunctionNumbers::getActualTableStructure(const Context & /*context*/) const { - return ColumnsDescription({{"number", std::make_shared()}}); + /// NOTE: https://bugs.llvm.org/show_bug.cgi?id=47418 + return ColumnsDescription{{{"number", std::make_shared()}}}; } template diff --git a/src/TableFunctions/TableFunctionZeros.cpp b/src/TableFunctions/TableFunctionZeros.cpp index 22bf47f5e9f..9b0c6c6e78b 100644 --- a/src/TableFunctions/TableFunctionZeros.cpp +++ b/src/TableFunctions/TableFunctionZeros.cpp @@ -22,7 +22,8 @@ extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; template ColumnsDescription TableFunctionZeros::getActualTableStructure(const Context & /*context*/) const { - return ColumnsDescription({{"zero", std::make_shared()}}); + /// NOTE: https://bugs.llvm.org/show_bug.cgi?id=47418 + return ColumnsDescription{{{"zero", std::make_shared()}}}; } template diff --git a/src/TableFunctions/tests/CMakeLists.txt b/src/TableFunctions/tests/CMakeLists.txt deleted file mode 100644 index 6b70ce0beca..00000000000 --- a/src/TableFunctions/tests/CMakeLists.txt +++ /dev/null @@ -1,3 +0,0 @@ -add_executable (test-strange-memory-leak test_strange_memory_leak.cpp ${SRCS}) -#target_link_libraries (test-strange-memory-leak PRIVATE dbms) -#target_link_libraries (test-strange-memory-leak PUBLIC boost::headers_only) diff --git a/src/TableFunctions/tests/test_strange_memory_leak.cpp b/src/TableFunctions/tests/test_strange_memory_leak.cpp deleted file mode 100644 index dcf6c779429..00000000000 --- a/src/TableFunctions/tests/test_strange_memory_leak.cpp +++ /dev/null @@ -1,136 +0,0 @@ -/// Demonstrates mysterious memory leak when built using clang version 10.0.0-4ubuntu1~18.04.2 -/// No leaks are found by valgrind when built using gcc version 9.2.1 20191102 - -//#include -//#include - -//class ITableFunction : public std::enable_shared_from_this -//{ -//public: -// virtual DB::ColumnsDescription getActualTableStructure() const = 0; -// virtual ~ITableFunction() {} -// -//}; -// -//class TableFunction : public ITableFunction -//{ -// DB::ColumnsDescription getActualTableStructure() const override -// { -// return DB::ColumnsDescription({{"number", DB::DataTypePtr{}}}); -// } -//}; -// -//template -//class TableFunctionTemplate : public ITableFunction -//{ -// DB::ColumnsDescription getActualTableStructure() const override -// { -// return DB::ColumnsDescription({{"number", DB::DataTypePtr{}}}); -// } -//}; - -/// Simplified version of the commented code above (without dependencies on ClickHouse code): - -#include -#include -#include -#include -#include -//#include -//#include -//#include -//#include - -struct Print -{ - std::string data = "lol"; - Print(const Print &) { std::cout << "copy" << std::endl; } - Print(Print &&) { std::cout << "move" << std::endl; } - Print & operator = (const Print &) { std::cout << "copy=" << std::endl; return *this; } - Print & operator = (Print &&) { std::cout << "move=" << std::endl; return *this; } - Print() { std::cout << "ctor" << std::endl; } - ~Print() { std::cout << "dtor" << std::endl; } -}; - -using LolPtr = std::shared_ptr; -struct Elem -{ - Elem(std::string n, LolPtr t) : name(std::move(n)), type(std::move(t)) {} - std::string name; - LolPtr type; - Print print; -}; - -//using Container = boost::multi_index_container< -// Elem, -// boost::multi_index::indexed_by< -// boost::multi_index::sequenced<>, -// boost::multi_index::ordered_unique>>>; -/// Simplified version: -using Container = std::map; - -struct List : public std::list -{ - List(std::initializer_list init) : std::list(init) {} -}; - -struct Kek -{ - Container container; - Kek(List list) - { - for (auto & elem : list) - add(Elem(std::move(elem.name), std::move(elem.type))); - } - - void add(Elem column) - { - auto insert_it = container.cbegin(); - //container.get<0>().insert(insert_it, std::move(column)); - container.insert(insert_it, {column.name, column}); - } - -}; - -class ITableFunction : public std::enable_shared_from_this -{ -public: - virtual Kek getActualTableStructure() const = 0; - virtual ~ITableFunction() {} - -}; - -class TableFunction : public ITableFunction -{ - Kek getActualTableStructure() const override - { - return Kek({{"number", LolPtr{}}}); - } -}; - -template -class TableFunctionTemplate : public ITableFunction -{ - Kek getActualTableStructure() const override - { - return Kek({{"number", LolPtr{}}}); - } -}; - -int main() -{ - std::cout << "0" << std::endl; - - /// Works fine - const ITableFunction & tf1 = TableFunction{}; - tf1.getActualTableStructure(); - - std::cout << "1" << std::endl; - - /// ERROR: LeakSanitizer: detected memory leaks - /// and the same error with valgrind - const ITableFunction & tf2 = TableFunctionTemplate{}; - tf2.getActualTableStructure(); - - std::cout << "2" << std::endl; -} diff --git a/src/TableFunctions/ya.make b/src/TableFunctions/ya.make index 36218bee1e0..e87c96073bd 100644 --- a/src/TableFunctions/ya.make +++ b/src/TableFunctions/ya.make @@ -23,7 +23,6 @@ SRCS( TableFunctionValues.cpp TableFunctionView.cpp TableFunctionZeros.cpp - tests/test_strange_memory_leak.cpp ) diff --git a/tests/config/lsan_suppressions.txt b/tests/config/lsan_suppressions.txt index 870bbe5e081..39eb40560d7 100644 --- a/tests/config/lsan_suppressions.txt +++ b/tests/config/lsan_suppressions.txt @@ -1,2 +1,2 @@ -# See src/TableFunctions/tests/test-strange-memory-leak.cpp -leak:getActualTableStructure +# See https://bugs.llvm.org/show_bug.cgi?id=47418 +# leak:getActualTableStructure From ef7ca3788dadb352153b2f0aa35e48e61be7f45f Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 9 Sep 2020 19:16:41 +0300 Subject: [PATCH 18/42] Update arcadia_skip_list.txt --- tests/queries/0_stateless/arcadia_skip_list.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 698b38460e4..a9267c3ed5c 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -140,3 +140,4 @@ 01456_ast_optimizations_over_distributed 01460_DistributedFilesToInsert 01474_bad_global_join +01457_create_as_table_function_structure From 72d27196a0e4398a70589a433204d2516d086d6c Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Tue, 15 Sep 2020 06:10:07 +0300 Subject: [PATCH 19/42] Apply suggestions from code review Co-authored-by: BayoNet --- docs/en/operations/settings/settings.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index dc93d31dea4..798624ff5a5 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1901,20 +1901,20 @@ Default value: `120`. ## cast_keep_nullable {#cast_keep_nullable} -Enables or disables converstion to nullable types for [CAST](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) operator. +Enables or disables keeping of the `Nullable` data type in [CAST](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) operations. If set, `CAST(something_nullable AS Type)` returns `Nullable(Type)`. Possible values: -- 0 — Nullable type converstion disabled. -- 1 — Nullable type converstion enabled. +- 0 — The final type of `CAST` exactly the destination data type specified. +- 1 — The final type of `CAST` becomes `Nullable(DestinationDataType). Default value: `0`. **Examples** -Query with nullable type converstion disabled: +The following query exactly results in the destination data type: ```sql SET cast_keep_nullable = 0; From 3617ae4e4b306ce61bed491651b3b96b53ec8aa9 Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Tue, 15 Sep 2020 06:12:23 +0300 Subject: [PATCH 20/42] Update docs/en/operations/settings/settings.md Co-authored-by: BayoNet --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 798624ff5a5..7dbf7a85f0f 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1929,7 +1929,7 @@ Result: └───┴───────────────────────────────────────────────────┘ ``` -Query with nullable type converstion enabled: +The following query results in the `Nullable` modification on the destination data type: ```sql SET cast_keep_nullable = 1; From 8506fb20fe789cf9e1a89eb04597e77aa52b5bfc Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Tue, 15 Sep 2020 06:38:54 +0300 Subject: [PATCH 21/42] Translated into Russian, and typo fixed in English. --- docs/en/operations/settings/settings.md | 2 +- docs/ru/operations/settings/settings.md | 49 +++++++++++++++++++++++++ 2 files changed, 50 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 7dbf7a85f0f..d5a6bf2c0b8 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1908,7 +1908,7 @@ If set, `CAST(something_nullable AS Type)` returns `Nullable(Type)`. Possible values: - 0 — The final type of `CAST` exactly the destination data type specified. -- 1 — The final type of `CAST` becomes `Nullable(DestinationDataType). +- 1 — The final type of `CAST` becomes `Nullable(DestinationDataType)`. Default value: `0`. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 9a487b6c166..55dbaaeb4ac 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1678,4 +1678,53 @@ SELECT idx, i FROM null_in WHERE i IN (1, NULL) SETTINGS transform_null_in = 1; - [Секции и настройки запроса CREATE TABLE](../../engines/table-engines/mergetree-family/mergetree.md#mergetree-query-clauses) (настройка `merge_with_ttl_timeout`) - [Table TTL](../../engines/table-engines/mergetree-family/mergetree.md#mergetree-table-ttl) +## cast_keep_nullable {#cast_keep_nullable} + +Включает или отключает финальное преобразование аргумента функции [CAST](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) к типу `Nullable`. + +Если настройка включена, то функция `CAST(something_nullable AS Type)` возвращает `Nullable(Type)`. + +Возможные значения: + +- 0 — функция `CAST` преобразует аргумент строго к указанному типу. +- 1 — функция `CAST` преобразует аргумент к типу `Nullable` для указанного типа. + +Значение по умолчанию: `0`. + +**Примеры** + +Запрос возвращает аргумент, преобразованный строго к указанному типу: + +```sql +SET cast_keep_nullable = 0; +SELECT CAST(toNullable(toInt32(0)) AS Int32) as x, toTypeName(x); +``` + +Результат: + +```text +┌─x─┬─toTypeName(CAST(toNullable(toInt32(0)), 'Int32'))─┐ +│ 0 │ Int32 │ +└───┴───────────────────────────────────────────────────┘ +``` + +Запрос возвращает аргумент, преобразованный к типу `Nullable` для указанного типа: + +```sql +SET cast_keep_nullable = 1; +SELECT CAST(toNullable(toInt32(0)) AS Int32) as x, toTypeName(x); +``` + +Результат: + +```text +┌─x─┬─toTypeName(CAST(toNullable(toInt32(0)), 'Int32'))─┐ +│ 0 │ Nullable(Int32) │ +└───┴───────────────────────────────────────────────────┘ +``` + +**См. также** + +- Функция [CAST](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) + [Оригинальная статья](https://clickhouse.tech/docs/ru/operations/settings/settings/) From 9d6247bb7ed0064c08ba7b5ee8dda2a0335839de Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Tue, 15 Sep 2020 14:38:22 +0300 Subject: [PATCH 22/42] Links fixed --- docs/ru/interfaces/formats.md | 4 ++-- docs/ru/operations/settings/settings.md | 2 +- docs/ru/sql-reference/functions/bitmap-functions.md | 6 +++--- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 04bca115974..dd68f7eb646 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -1050,13 +1050,13 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Parquet" > {some_ Для обмена данными с экосистемой Hadoop можно использовать движки таблиц [HDFS](../engines/table-engines/integrations/hdfs.md). -## Arrow {data-format-arrow} +## Arrow {#data-format-arrow} [Apache Arrow](https://arrow.apache.org/) поставляется с двумя встроенными поколоночнами форматами хранения. ClickHouse поддерживает операции чтения и записи для этих форматов. `Arrow` — это Apache Arrow's "file mode" формат. Он предназначен для произвольного доступа в памяти. -## ArrowStream {data-format-arrow-stream} +## ArrowStream {#data-format-arrow-stream} `ArrowStream` — это Apache Arrow's "stream mode" формат. Он предназначен для обработки потоков в памяти. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 70e20ef954d..0667637c4ad 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -484,7 +484,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( См. также: -- [JOIN strictness](../../sql-reference/statements/select/join.md#select-join-strictness) +- [JOIN strictness](../../sql-reference/statements/select/join.md#join-settings) ## max\_block\_size {#setting-max_block_size} diff --git a/docs/ru/sql-reference/functions/bitmap-functions.md b/docs/ru/sql-reference/functions/bitmap-functions.md index c91725c7a39..ef59a46d59d 100644 --- a/docs/ru/sql-reference/functions/bitmap-functions.md +++ b/docs/ru/sql-reference/functions/bitmap-functions.md @@ -61,8 +61,8 @@ bitmapSubsetLimit(bitmap, range_start, cardinality_limit) **Параметры** - `bitmap` – Битмап. [Bitmap object](#bitmap_functions-bitmapbuild). -- `range_start` – Начальная точка подмножества. [UInt32](../../sql-reference/functions/bitmap-functions.md). -- `cardinality_limit` – Верхний предел подмножества. [UInt32](../../sql-reference/functions/bitmap-functions.md). +- `range_start` – Начальная точка подмножества. [UInt32](../../sql-reference/data-types/int-uint.md). +- `cardinality_limit` – Верхний предел подмножества. [UInt32](../../sql-reference/data-types/int-uint.md). **Возвращаемое значение** @@ -97,7 +97,7 @@ bitmapContains(haystack, needle) **Параметры** - `haystack` – [объект Bitmap](#bitmap_functions-bitmapbuild), в котором функция ищет значение. -- `needle` – значение, которое функция ищет. Тип — [UInt32](../../sql-reference/functions/bitmap-functions.md). +- `needle` – значение, которое функция ищет. Тип — [UInt32](../../sql-reference/data-types/int-uint.md). **Возвращаемые значения** From 536c8c2416464bbcfe09227fb30538fc79dcfbbd Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Fri, 25 Sep 2020 11:39:29 +0300 Subject: [PATCH 23/42] Russian text updated. --- docs/ru/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 1396f43a01c..878616de8ca 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1882,7 +1882,7 @@ SELECT range(number) FROM system.numbers LIMIT 5 FORMAT PrettyCompactNoEscapes; ## cast_keep_nullable {#cast_keep_nullable} -Включает или отключает финальное преобразование аргумента функции [CAST](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) к типу `Nullable`. +Включает или отключает сохранение типа `Nullable` для аргумента функции [CAST](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast). Если настройка включена, то функция `CAST(something_nullable AS Type)` возвращает `Nullable(Type)`. From 39367be57af5b53253582f31fe8c6efa209df464 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Mon, 12 Oct 2020 16:46:49 +0800 Subject: [PATCH 24/42] Add Support for OFFSET_FETCH_CLAUSE --- src/Parsers/ParserSelectQuery.cpp | 56 +++++++++++++++++++ ..._select_with_offset_fetch_clause.reference | 10 ++++ .../01525_select_with_offset_fetch_clause.sql | 8 +++ 3 files changed, 74 insertions(+) create mode 100644 tests/queries/0_stateless/01525_select_with_offset_fetch_clause.reference create mode 100644 tests/queries/0_stateless/01525_select_with_offset_fetch_clause.sql diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index 9f2df82b4b4..a64688e6a4d 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -45,6 +45,12 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_top("TOP"); ParserKeyword s_with_ties("WITH TIES"); ParserKeyword s_offset("OFFSET"); + ParserKeyword s_fetch("FETCH"); + ParserKeyword s_only("ONLY"); + ParserKeyword s_row("ROW"); + ParserKeyword s_rows("ROWS"); + ParserKeyword s_first("FIRST"); + ParserKeyword s_next("NEXT"); ParserNotEmptyExpressionList exp_list(false); ParserNotEmptyExpressionList exp_list_for_with_clause(false); @@ -191,6 +197,56 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; } + /// OFFSET offset_row_count {ROW | ROWS} FETCH {FIRST | NEXT} fetch_row_count {ROW | ROWS} {ONLY | WITH TIES} + if (s_offset.ignore(pos, expected)) + { + /// OFFSET must exists with "order by" + if (!order_expression_list) + return false; + + if (!exp_elem.parse(pos, limit_offset, expected)) + return false; + + if (s_row.ignore(pos, expected)) + { + if (s_rows.ignore(pos, expected)) + throw Exception("Can not use ROW and ROWS together", ErrorCodes::TOP_AND_LIMIT_TOGETHER); + } + else if (!s_rows.ignore(pos, expected)) + return false; + + if (!s_fetch.ignore(pos, expected)) + return false; + + if (s_first.ignore(pos, expected)) + { + if (s_next.ignore(pos, expected)) + throw Exception("Can not use ROW and ROWS together", ErrorCodes::TOP_AND_LIMIT_TOGETHER); + } + else if (!s_next.ignore(pos, expected)) + return false; + + if (!exp_elem.parse(pos, limit_length, expected)) + return false; + + if (s_row.ignore(pos, expected)) + { + if (s_rows.ignore(pos, expected)) + throw Exception("Can not use ROW and ROWS together", ErrorCodes::TOP_AND_LIMIT_TOGETHER); + } + else if (!s_rows.ignore(pos, expected)) + return false; + + if (s_with_ties.ignore(pos, expected)) + { + select_query->limit_with_ties = true; + } + else if (s_only.ignore(pos, expected)) + { + select_query->limit_with_ties = false; + } + } + /// This is needed for TOP expression, because it can also use WITH TIES. bool limit_with_ties_occured = false; diff --git a/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.reference b/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.reference new file mode 100644 index 00000000000..709116b4746 --- /dev/null +++ b/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.reference @@ -0,0 +1,10 @@ +7 +6 +5 +1 1 +2 1 +3 4 +1 1 +2 1 +3 4 +3 3 diff --git a/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.sql b/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.sql new file mode 100644 index 00000000000..fce7dd753d2 --- /dev/null +++ b/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.sql @@ -0,0 +1,8 @@ +SELECT number FROM numbers(10) ORDER BY number DESC OFFSET 2 ROWS FETCH NEXT 3 ROWS WITH TIES; + +DROP TABLE IF EXISTS test_fetch; +CREATE TABLE test_fetch(a Int32, b Int32) Engine = Memory; +INSERT INTO test_fetch VALUES(1, 1), (2, 1), (3, 4), (3, 3), (5, 4), (0, 6), (5, 7); +SELECT * FROM test_fetch ORDER BY a OFFSET 1 ROW FETCH FIRST 3 ROWS ONLY; +SELECT * FROM test_fetch ORDER BY a OFFSET 1 ROW FETCH FIRST 3 ROWS WITH TIES; +DROP TABLE test_fetch; From 2947eff87879dd5997d784d2e25c0ef4ec72a824 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 12 Oct 2020 22:01:15 +0300 Subject: [PATCH 25/42] Fix query hang with connections_with_failover_max_tries = 0 (endless loop) --- src/Common/PoolWithFailoverBase.h | 2 +- .../01521_connections_with_failover_max_tries_0.reference | 1 + .../01521_connections_with_failover_max_tries_0.sql | 3 +++ 3 files changed, 5 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01521_connections_with_failover_max_tries_0.reference create mode 100644 tests/queries/0_stateless/01521_connections_with_failover_max_tries_0.sql diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index a328e15e4e5..7779d18d969 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -229,7 +229,7 @@ PoolWithFailoverBase::getMany( ShuffledPool & shuffled_pool = shuffled_pools[i]; TryResult & result = try_results[i]; - if (shuffled_pool.error_count >= max_tries || !result.entry.isNull()) + if (max_tries && (shuffled_pool.error_count >= max_tries || !result.entry.isNull())) continue; std::string fail_message; diff --git a/tests/queries/0_stateless/01521_connections_with_failover_max_tries_0.reference b/tests/queries/0_stateless/01521_connections_with_failover_max_tries_0.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/01521_connections_with_failover_max_tries_0.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/01521_connections_with_failover_max_tries_0.sql b/tests/queries/0_stateless/01521_connections_with_failover_max_tries_0.sql new file mode 100644 index 00000000000..73e0f1ec921 --- /dev/null +++ b/tests/queries/0_stateless/01521_connections_with_failover_max_tries_0.sql @@ -0,0 +1,3 @@ +-- regression for endless loop with connections_with_failover_max_tries=0 +set connections_with_failover_max_tries=0; +select * from remote('127.2', system.one); From 9613144fa8b3cfab0bc13b139228b4762a0a1eeb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 12 Oct 2020 23:30:37 +0300 Subject: [PATCH 26/42] Move 01521_connections_with_failover_max_tries_0 to 01521_distributed_query_hang This will fix style check error: /place/sandbox-data/tasks/4/8/795063184/ClickHouse/tests/queries/0_stateless/01521_connections_with_failover_max_tries_0.reference /place/sandbox-data/tasks/4/8/795063184/ClickHouse/tests/queries/0_stateless/01521_connections_with_failover_max_tries_0.sql Tests should not be named with "fail" in their names. It makes looking at the results less convenient when you search for "fail" substring in browser. --- ...x_tries_0.reference => 01521_distributed_query_hang.reference} | 0 ..._failover_max_tries_0.sql => 01521_distributed_query_hang.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{01521_connections_with_failover_max_tries_0.reference => 01521_distributed_query_hang.reference} (100%) rename tests/queries/0_stateless/{01521_connections_with_failover_max_tries_0.sql => 01521_distributed_query_hang.sql} (100%) diff --git a/tests/queries/0_stateless/01521_connections_with_failover_max_tries_0.reference b/tests/queries/0_stateless/01521_distributed_query_hang.reference similarity index 100% rename from tests/queries/0_stateless/01521_connections_with_failover_max_tries_0.reference rename to tests/queries/0_stateless/01521_distributed_query_hang.reference diff --git a/tests/queries/0_stateless/01521_connections_with_failover_max_tries_0.sql b/tests/queries/0_stateless/01521_distributed_query_hang.sql similarity index 100% rename from tests/queries/0_stateless/01521_connections_with_failover_max_tries_0.sql rename to tests/queries/0_stateless/01521_distributed_query_hang.sql From f211d33841fecbb8172d3345a69b4c42553ba6c2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 3 Aug 2020 00:46:52 +0300 Subject: [PATCH 27/42] Avoid creating separate replxx instance This replxx object is pretty heavy and in debug build may slow down [1] (although I cannot confirm 0.5s delay for each query in debug build) the client and besides it is not required since ClickHouse-Extras/replxx#10, which changes the behaviour of history_save(), and now it will not update current session anymore, only save the history to the disk. [1]: https://github.com/ClickHouse/ClickHouse/pull/13086#issuecomment-667719026 --- base/common/ReplxxLineReader.cpp | 15 +-------------- contrib/replxx | 2 +- 2 files changed, 2 insertions(+), 15 deletions(-) diff --git a/base/common/ReplxxLineReader.cpp b/base/common/ReplxxLineReader.cpp index c4f9c65a116..33a368c27dc 100644 --- a/base/common/ReplxxLineReader.cpp +++ b/base/common/ReplxxLineReader.cpp @@ -16,19 +16,6 @@ void trim(String & s) s.erase(std::find_if(s.rbegin(), s.rend(), [](int ch) { return !std::isspace(ch); }).base(), s.end()); } -// Uses separate replxx::Replxx instance to avoid loading them again in the -// current context (replxx::Replxx::history_load() will re-load the history -// from the file), since then they will overlaps with history from the current -// session (this will make behavior compatible with other interpreters, i.e. -// bash). -void history_save(const String & history_file_path, const String & line) -{ - replxx::Replxx rx_no_overlap; - rx_no_overlap.history_load(history_file_path); - rx_no_overlap.history_add(line); - rx_no_overlap.history_save(history_file_path); -} - } ReplxxLineReader::ReplxxLineReader( @@ -128,7 +115,7 @@ void ReplxxLineReader::addToHistory(const String & line) rx.history_add(line); // flush changes to the disk - history_save(history_file_path, line); + rx.history_save(history_file_path); if (locked && 0 != flock(history_file_fd, LOCK_UN)) rx.print("Unlock of history file failed: %s\n", strerror(errno)); diff --git a/contrib/replxx b/contrib/replxx index 94b1f568d16..8cf626c04e9 160000 --- a/contrib/replxx +++ b/contrib/replxx @@ -1 +1 @@ -Subproject commit 94b1f568d16183214d26c7c0e9ce69a4ce407f65 +Subproject commit 8cf626c04e9a74313fb0b474cdbe2297c0f3cdc8 From f7bf94b5d361d0a54e85673c14e630cfdf8dbe01 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 12 Oct 2020 23:39:00 +0300 Subject: [PATCH 28/42] Check return value of the history_save/history_load The coding style is different and this is the intention, to make look like surrounding code. --- base/common/ReplxxLineReader.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/base/common/ReplxxLineReader.cpp b/base/common/ReplxxLineReader.cpp index 33a368c27dc..85b474e2021 100644 --- a/base/common/ReplxxLineReader.cpp +++ b/base/common/ReplxxLineReader.cpp @@ -45,7 +45,10 @@ ReplxxLineReader::ReplxxLineReader( } else { - rx.history_load(history_file_path); + if (!rx.history_load(history_file_path)) + { + rx.print("Loading history failed: %s\n", strerror(errno)); + } if (flock(history_file_fd, LOCK_UN)) { @@ -115,7 +118,8 @@ void ReplxxLineReader::addToHistory(const String & line) rx.history_add(line); // flush changes to the disk - rx.history_save(history_file_path); + if (!rx.history_save(history_file_path)) + rx.print("Saving history failed: %s\n", strerror(errno)); if (locked && 0 != flock(history_file_fd, LOCK_UN)) rx.print("Unlock of history file failed: %s\n", strerror(errno)); From dea965832a1ecd62475774c60edae7b65baa9ccc Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Tue, 13 Oct 2020 11:32:43 +0800 Subject: [PATCH 29/42] Fix error for previous offset test --- src/Common/ErrorCodes.cpp | 2 + src/Parsers/ParserSelectQuery.cpp | 101 +++++++++++++++--------------- 2 files changed, 53 insertions(+), 50 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index bf475bc9b21..bc4dc8435e8 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -507,6 +507,8 @@ namespace ErrorCodes extern const int CANNOT_CREATE_RABBITMQ_QUEUE_BINDING = 541; extern const int CANNOT_REMOVE_RABBITMQ_EXCHANGE = 542; extern const int UNKNOWN_MYSQL_DATATYPES_SUPPORT_LEVEL = 543; + extern const int ROW_AND_ROWS_TOGETHER = 544; + extern const int FIRST_AND_NEXT_TOGETHER = 545; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index a64688e6a4d..8f0c70308ff 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -19,6 +19,8 @@ namespace ErrorCodes extern const int TOP_AND_LIMIT_TOGETHER; extern const int WITH_TIES_WITHOUT_ORDER_BY; extern const int LIMIT_BY_WITH_TIES_IS_NOT_SUPPORTED; + extern const int ROW_AND_ROWS_TOGETHER; + extern const int FIRST_AND_NEXT_TOGETHER; } @@ -197,56 +199,6 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; } - /// OFFSET offset_row_count {ROW | ROWS} FETCH {FIRST | NEXT} fetch_row_count {ROW | ROWS} {ONLY | WITH TIES} - if (s_offset.ignore(pos, expected)) - { - /// OFFSET must exists with "order by" - if (!order_expression_list) - return false; - - if (!exp_elem.parse(pos, limit_offset, expected)) - return false; - - if (s_row.ignore(pos, expected)) - { - if (s_rows.ignore(pos, expected)) - throw Exception("Can not use ROW and ROWS together", ErrorCodes::TOP_AND_LIMIT_TOGETHER); - } - else if (!s_rows.ignore(pos, expected)) - return false; - - if (!s_fetch.ignore(pos, expected)) - return false; - - if (s_first.ignore(pos, expected)) - { - if (s_next.ignore(pos, expected)) - throw Exception("Can not use ROW and ROWS together", ErrorCodes::TOP_AND_LIMIT_TOGETHER); - } - else if (!s_next.ignore(pos, expected)) - return false; - - if (!exp_elem.parse(pos, limit_length, expected)) - return false; - - if (s_row.ignore(pos, expected)) - { - if (s_rows.ignore(pos, expected)) - throw Exception("Can not use ROW and ROWS together", ErrorCodes::TOP_AND_LIMIT_TOGETHER); - } - else if (!s_rows.ignore(pos, expected)) - return false; - - if (s_with_ties.ignore(pos, expected)) - { - select_query->limit_with_ties = true; - } - else if (s_only.ignore(pos, expected)) - { - select_query->limit_with_ties = false; - } - } - /// This is needed for TOP expression, because it can also use WITH TIES. bool limit_with_ties_occured = false; @@ -303,8 +255,57 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } else if (s_offset.ignore(pos, expected)) { + /// OFFSET offset_row_count {ROW | ROWS} FETCH {FIRST | NEXT} fetch_row_count {ROW | ROWS} {ONLY | WITH TIES} + bool offset_with_fetch_maybe = false; + if (!exp_elem.parse(pos, limit_offset, expected)) return false; + + if (s_row.ignore(pos, expected)) + { + if (s_rows.ignore(pos, expected)) + throw Exception("Can not use ROW and ROWS together", ErrorCodes::ROW_AND_ROWS_TOGETHER); + offset_with_fetch_maybe = true; + } + else if (s_rows.ignore(pos, expected)) + { + offset_with_fetch_maybe = true; + } + + if (offset_with_fetch_maybe && s_fetch.ignore(pos, expected)) + { + /// OFFSET FETCH clause must exists with "ORDER BY" + if (!order_expression_list) + return false; + + if (s_first.ignore(pos, expected)) + { + if (s_next.ignore(pos, expected)) + throw Exception("Can not use FIRST and NEXT together", ErrorCodes::FIRST_AND_NEXT_TOGETHER); + } + else if (!s_next.ignore(pos, expected)) + return false; + + if (!exp_elem.parse(pos, limit_length, expected)) + return false; + + if (s_row.ignore(pos, expected)) + { + if (s_rows.ignore(pos, expected)) + throw Exception("Can not use ROW and ROWS together", ErrorCodes::ROW_AND_ROWS_TOGETHER); + } + else if (!s_rows.ignore(pos, expected)) + return false; + + if (s_with_ties.ignore(pos, expected)) + { + select_query->limit_with_ties = true; + } + else if (s_only.ignore(pos, expected)) + { + select_query->limit_with_ties = false; + } + } } /// Because TOP n in totally equals LIMIT n From d79d0127ad7fa16f6019ae36df504bb9393ba0a9 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 13 Oct 2020 13:30:56 +0800 Subject: [PATCH 30/42] ISSUES-12513 try fix difference expressions with same alias --- src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h | 3 ++- src/Interpreters/TreeOptimizer.cpp | 8 +++----- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h b/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h index cac3e0432dc..c8d73655ca8 100644 --- a/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h +++ b/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB { @@ -86,7 +87,7 @@ public: static bool needChildVisit(const ASTPtr & node, const ASTPtr &) { - return !(node->as()); + return !node->as() && !(node->as()); } static void visit(ASTPtr & ast, Data & data) diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index bdc506b0cc6..034340da87b 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -269,7 +269,7 @@ void optimizeGroupByFunctionKeys(ASTSelectQuery * select_query) } /// Eliminates min/max/any-aggregators of functions of GROUP BY keys -void optimizeAggregateFunctionsOfGroupByKeys(ASTSelectQuery * select_query) +void optimizeAggregateFunctionsOfGroupByKeys(ASTSelectQuery * select_query, ASTPtr & node) { if (!select_query->groupBy()) return; @@ -279,10 +279,8 @@ void optimizeAggregateFunctionsOfGroupByKeys(ASTSelectQuery * select_query) GroupByKeysInfo group_by_keys_data = getGroupByKeysInfo(group_keys); - auto select = select_query->select(); - SelectAggregateFunctionOfGroupByKeysVisitor::Data visitor_data{group_by_keys_data.key_names}; - SelectAggregateFunctionOfGroupByKeysVisitor(visitor_data).visit(select); + SelectAggregateFunctionOfGroupByKeysVisitor(visitor_data).visit(node); } /// Remove duplicate items from ORDER BY. @@ -647,7 +645,7 @@ void TreeOptimizer::apply(ASTPtr & query, Aliases & aliases, const NameSet & sou /// Eliminate min/max/any aggregators of functions of GROUP BY keys if (settings.optimize_aggregators_of_group_by_keys) - optimizeAggregateFunctionsOfGroupByKeys(select_query); + optimizeAggregateFunctionsOfGroupByKeys(select_query, query); /// Remove duplicate items from ORDER BY. optimizeDuplicatesInOrderBy(select_query); From 72f8153f0183261ba9b7832e298ccf749f65f91f Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 13 Oct 2020 13:48:16 +0800 Subject: [PATCH 31/42] ISSUES-12513 add stateless test --- .../AggregateFunctionOfGroupByKeysVisitor.h | 4 +++- ...1511_different_expression_with_same_alias.reference | 10 ++++++++++ .../01511_different_expression_with_same_alias.sql | 10 ++++++++++ 3 files changed, 23 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01511_different_expression_with_same_alias.reference create mode 100644 tests/queries/0_stateless/01511_different_expression_with_same_alias.sql diff --git a/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h b/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h index c8d73655ca8..6b903ec45cf 100644 --- a/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h +++ b/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h @@ -87,7 +87,9 @@ public: static bool needChildVisit(const ASTPtr & node, const ASTPtr &) { - return !node->as() && !(node->as()); + /// Don't descent into table functions and subqueries and special case for ArrayJoin. + return !node->as() && + !(node->as() || node->as() || node->as()); } static void visit(ASTPtr & ast, Data & data) diff --git a/tests/queries/0_stateless/01511_different_expression_with_same_alias.reference b/tests/queries/0_stateless/01511_different_expression_with_same_alias.reference new file mode 100644 index 00000000000..8b1acc12b63 --- /dev/null +++ b/tests/queries/0_stateless/01511_different_expression_with_same_alias.reference @@ -0,0 +1,10 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/tests/queries/0_stateless/01511_different_expression_with_same_alias.sql b/tests/queries/0_stateless/01511_different_expression_with_same_alias.sql new file mode 100644 index 00000000000..9d5d186b85a --- /dev/null +++ b/tests/queries/0_stateless/01511_different_expression_with_same_alias.sql @@ -0,0 +1,10 @@ +DROP TABLE IF EXISTS repro_hits; + +CREATE TABLE repro_hits ( date Date, metric Float64) ENGINE = MergeTree() ORDER BY date; + +-- From https://github.com/ClickHouse/ClickHouse/issues/12513#issue-657202535 +SELECT date as period, 1 as having_check, min(date) as period_start, addDays(max(date), 1) as period_end, dateDiff('second', period_start, period_end) as total_duration, sum(metric) as metric_ FROM repro_hits GROUP BY period HAVING having_check != -1; + +SELECT min(number) as min_number FROM numbers(10) GROUP BY number HAVING 1 ORDER BY min_number; + +DROP TABLE IF EXISTS repro_hits; From 14ef354a16d2a9a2a9320eeb433548f18fcb84e5 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 13 Oct 2020 14:52:05 +0800 Subject: [PATCH 32/42] ISSUES-12513 try fix test failure --- ...1321_aggregate_functions_of_group_by_keys.reference | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01321_aggregate_functions_of_group_by_keys.reference b/tests/queries/0_stateless/01321_aggregate_functions_of_group_by_keys.reference index 92d6e5c37e6..5eaaf24208e 100644 --- a/tests/queries/0_stateless/01321_aggregate_functions_of_group_by_keys.reference +++ b/tests/queries/0_stateless/01321_aggregate_functions_of_group_by_keys.reference @@ -55,8 +55,8 @@ GROUP BY number % 2, number % 3 ORDER BY - min(number % 2) AS a ASC, - max(number % 3) AS b ASC + a ASC, + b ASC SELECT number % 2 AS a, number % 3 AS b @@ -65,14 +65,14 @@ GROUP BY number % 2, number % 3 ORDER BY - any(number % 2) AS a ASC, - anyLast(number % 3) AS b ASC + a ASC, + b ASC SELECT (number % 5) * (number % 7) AS a FROM numbers(10000000) GROUP BY number % 7, number % 5 -ORDER BY max((number % 5) * (number % 7)) AS a ASC +ORDER BY a ASC SELECT foo FROM ( From adaae8a12ce09008f078be97ca2708dbbdbf65b3 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 13 Oct 2020 13:59:43 +0300 Subject: [PATCH 33/42] Added OutputFormat setting date_time_output_format --- docs/en/operations/settings/settings.md | 27 +++++++++- docs/en/sql-reference/data-types/datetime.md | 3 +- .../en/sql-reference/data-types/datetime64.md | 1 + src/Core/Settings.h | 1 + src/Core/SettingsEnums.cpp | 5 ++ src/Core/SettingsEnums.h | 1 + src/DataTypes/DataTypeDateTime.cpp | 16 +++++- src/DataTypes/DataTypeDateTime64.cpp | 16 +++++- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 9 ++++ src/IO/WriteHelpers.h | 49 +++++++++++++++---- .../01516_date_time_output_format.reference | 12 +++++ .../01516_date_time_output_format.sql | 36 ++++++++++++++ 13 files changed, 162 insertions(+), 15 deletions(-) create mode 100644 tests/queries/0_stateless/01516_date_time_output_format.reference create mode 100644 tests/queries/0_stateless/01516_date_time_output_format.sql diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index f3d096f1e1d..baeabd0250c 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -389,6 +389,31 @@ See also: - [DateTime data type.](../../sql-reference/data-types/datetime.md) - [Functions for working with dates and times.](../../sql-reference/functions/date-time-functions.md) +## date\_time\_output_\_format {#settings-date_time_output_format} + +Allows choosing different output formats of the text representation of date and time. + +Possible values: + +- `'simple'` - Simple output format. + + Clickhouse output date and time `YYYY-MM-DD hh:mm:ss` format. For example, `'2019-08-20 10:18:56'`. Calculation is performed according to the data type's time zone (if present) or server time zone. + +- `'iso'` - ISO output format. + + Clickhouse output date and time in [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) `YYYY-MM-DDThh:mm:ssZ` format. For example, `'2019-08-20T10:18:56Z'`. Note that output is in UTC (`Z` means UTC). + +- `'unix_timestamp'` - Unix timestamp output format. + + Clickhouse output date and time in [Unix timestamp](https://en.wikipedia.org/wiki/Unix_time) format. For example `'1566285536'`. + +Default value: `'simple'`. + +See also: + +- [DateTime data type.](../../sql-reference/data-types/datetime.md) +- [Functions for working with dates and times.](../../sql-reference/functions/date-time-functions.md) + ## join\_default\_strictness {#settings-join_default_strictness} Sets default strictness for [JOIN clauses](../../sql-reference/statements/select/join.md#select-join). @@ -2066,4 +2091,4 @@ Possible values: - 1 — The bigint data type is enabled. - 0 — The bigint data type is disabled. -Default value: `0`. \ No newline at end of file +Default value: `0`. diff --git a/docs/en/sql-reference/data-types/datetime.md b/docs/en/sql-reference/data-types/datetime.md index a2ae68ebf14..db9e84205a2 100644 --- a/docs/en/sql-reference/data-types/datetime.md +++ b/docs/en/sql-reference/data-types/datetime.md @@ -27,7 +27,7 @@ You can explicitly set a time zone for `DateTime`-type columns when creating a t The [clickhouse-client](../../interfaces/cli.md) applies the server time zone by default if a time zone isn’t explicitly set when initializing the data type. To use the client time zone, run `clickhouse-client` with the `--use_client_time_zone` parameter. -ClickHouse outputs values in `YYYY-MM-DD hh:mm:ss` text format by default. You can change the output with the [formatDateTime](../../sql-reference/functions/date-time-functions.md#formatdatetime) function. +ClickHouse outputs values depending on the value of the [date\_time\_output\_format](../../operations/settings/settings.md#settings-date_time_output_format) setting. `YYYY-MM-DD hh:mm:ss` text format by default. Additionaly you can change the output with the [formatDateTime](../../sql-reference/functions/date-time-functions.md#formatdatetime) function. When inserting data into ClickHouse, you can use different formats of date and time strings, depending on the value of the [date\_time\_input\_format](../../operations/settings/settings.md#settings-date_time_input_format) setting. @@ -120,6 +120,7 @@ FROM dt - [Functions for working with dates and times](../../sql-reference/functions/date-time-functions.md) - [Functions for working with arrays](../../sql-reference/functions/array-functions.md) - [The `date_time_input_format` setting](../../operations/settings/settings.md#settings-date_time_input_format) +- [The `date_time_output_format` setting](../../operations/settings/settings.md#settings-date_time_output_format) - [The `timezone` server configuration parameter](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) - [Operators for working with dates and times](../../sql-reference/operators/index.md#operators-datetime) - [The `Date` data type](../../sql-reference/data-types/date.md) diff --git a/docs/en/sql-reference/data-types/datetime64.md b/docs/en/sql-reference/data-types/datetime64.md index 9a3b198b5e4..5cba8315090 100644 --- a/docs/en/sql-reference/data-types/datetime64.md +++ b/docs/en/sql-reference/data-types/datetime64.md @@ -96,6 +96,7 @@ FROM dt - [Functions for working with dates and times](../../sql-reference/functions/date-time-functions.md) - [Functions for working with arrays](../../sql-reference/functions/array-functions.md) - [The `date_time_input_format` setting](../../operations/settings/settings.md#settings-date_time_input_format) +- [The `date_time_output_format` setting](../../operations/settings/settings.md#settings-date_time_output_format) - [The `timezone` server configuration parameter](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) - [Operators for working with dates and times](../../sql-reference/operators/index.md#operators-datetime) - [`Date` data type](../../sql-reference/data-types/date.md) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a1a7a690e40..8f303e3fb48 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -415,6 +415,7 @@ class IColumn; M(Bool, input_format_null_as_default, false, "For text input formats initialize null fields with default values if data type of this field is not nullable", 0) \ \ M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.", 0) \ + M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \ \ M(Bool, optimize_group_by_function_keys, true, "Eliminates functions of other keys in GROUP BY section", 0) \ M(Bool, input_format_values_interpret_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser and try to interpret it as SQL expression.", 0) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index b4db51a506d..c337cd5e3ce 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -66,6 +66,11 @@ IMPLEMENT_SETTING_ENUM_WITH_RENAME(DateTimeInputFormat, ErrorCodes::BAD_ARGUMENT {"best_effort", FormatSettings::DateTimeInputFormat::BestEffort}}) +IMPLEMENT_SETTING_ENUM_WITH_RENAME(DateTimeOutputFormat, ErrorCodes::BAD_ARGUMENTS, + {{"simple", FormatSettings::DateTimeOutputFormat::Simple}, + {"iso", FormatSettings::DateTimeOutputFormat::ISO}, + {"unix_timestamp", FormatSettings::DateTimeOutputFormat::UnixTimestamp}}) + IMPLEMENT_SETTING_ENUM(LogsLevel, ErrorCodes::BAD_ARGUMENTS, {{"none", LogsLevel::none}, {"fatal", LogsLevel::fatal}, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 426497fff78..80b9bf9adde 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -83,6 +83,7 @@ DECLARE_SETTING_ENUM(DistributedProductMode) DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeInputFormat, FormatSettings::DateTimeInputFormat) +DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeOutputFormat, FormatSettings::DateTimeOutputFormat) enum class LogsLevel { diff --git a/src/DataTypes/DataTypeDateTime.cpp b/src/DataTypes/DataTypeDateTime.cpp index 9ea698d4fbb..bfb4473e429 100644 --- a/src/DataTypes/DataTypeDateTime.cpp +++ b/src/DataTypes/DataTypeDateTime.cpp @@ -59,9 +59,21 @@ String DataTypeDateTime::doGetName() const return out.str(); } -void DataTypeDateTime::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const +void DataTypeDateTime::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - writeDateTimeText(assert_cast(column).getData()[row_num], ostr, time_zone); + auto value = assert_cast(column).getData()[row_num]; + switch (settings.date_time_output_format) + { + case FormatSettings::DateTimeOutputFormat::Simple: + writeDateTimeText(value, ostr, time_zone); + return; + case FormatSettings::DateTimeOutputFormat::UnixTimestamp: + writeIntText(value, ostr); + return; + case FormatSettings::DateTimeOutputFormat::ISO: + writeDateTimeTextISO(value, ostr, utc_time_zone); + return; + } } void DataTypeDateTime::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const diff --git a/src/DataTypes/DataTypeDateTime64.cpp b/src/DataTypes/DataTypeDateTime64.cpp index ee4139c2b7a..ef1a971510a 100644 --- a/src/DataTypes/DataTypeDateTime64.cpp +++ b/src/DataTypes/DataTypeDateTime64.cpp @@ -57,9 +57,21 @@ std::string DataTypeDateTime64::doGetName() const return out.str(); } -void DataTypeDateTime64::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & /*settings*/) const +void DataTypeDateTime64::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - writeDateTimeText(assert_cast(column).getData()[row_num], scale, ostr, time_zone); + auto value = assert_cast(column).getData()[row_num]; + switch (settings.date_time_output_format) + { + case FormatSettings::DateTimeOutputFormat::Simple: + writeDateTimeText(value, scale, ostr, time_zone); + return; + case FormatSettings::DateTimeOutputFormat::UnixTimestamp: + writeDateTimeUnixTimestamp(value, scale, ostr); + return; + case FormatSettings::DateTimeOutputFormat::ISO: + writeDateTimeTextISO(value, scale, ostr, utc_time_zone); + return; + } } void DataTypeDateTime64::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 065b14f86b7..4ee5309eacd 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -127,6 +127,7 @@ static FormatSettings getOutputFormatSetting(const Settings & settings, const Co format_settings.custom.row_between_delimiter = settings.format_custom_row_between_delimiter; format_settings.avro.output_codec = settings.output_format_avro_codec; format_settings.avro.output_sync_interval = settings.output_format_avro_sync_interval; + format_settings.date_time_output_format = settings.date_time_output_format; return format_settings; } diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index a97bd9bf6c6..20c8a032231 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -99,6 +99,15 @@ struct FormatSettings DateTimeInputFormat date_time_input_format = DateTimeInputFormat::Basic; + enum class DateTimeOutputFormat + { + Simple, + ISO, + UnixTimestamp + }; + + DateTimeOutputFormat date_time_output_format = DateTimeOutputFormat::Simple; + UInt64 input_allow_errors_num = 0; Float32 input_allow_errors_ratio = 0; diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 1f0fe095059..d55fc4c5efd 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -636,6 +636,19 @@ inline void writeUUIDText(const UUID & uuid, WriteBuffer & buf) buf.write(s, sizeof(s)); } +template +inline void writeDecimalTypeFractionalText(typename DecimalType::NativeType fractional, UInt32 scale, WriteBuffer & buf) +{ + static constexpr UInt32 MaxScale = DecimalUtils::maxPrecision(); + + char data[20] = {'0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0'}; + static_assert(sizeof(data) >= MaxScale); + + for (Int32 pos = scale - 1; pos >= 0 && fractional; --pos, fractional /= DateTime64(10)) + data[pos] += fractional % DateTime64(10); + + writeString(&data[0], static_cast(scale), buf); +} static const char digits100[201] = "00010203040506070809" @@ -760,15 +773,7 @@ inline void writeDateTimeText(DateTime64 datetime64, UInt32 scale, WriteBuffer & if (scale > 0) { buf.write(fractional_time_delimiter); - - char data[20] = {'0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0'}; - static_assert(sizeof(data) >= MaxScale); - - auto fractional = c.fractional; - for (Int32 pos = scale - 1; pos >= 0 && fractional; --pos, fractional /= DateTime64(10)) - data[pos] += fractional % DateTime64(10); - - writeString(&data[0], static_cast(scale), buf); + writeDecimalTypeFractionalText(c.fractional, scale, buf); } } @@ -798,6 +803,32 @@ inline void writeDateTimeTextRFC1123(time_t datetime, WriteBuffer & buf, const D buf.write(" GMT", 4); } +inline void writeDateTimeTextISO(time_t datetime, WriteBuffer & buf, const DateLUTImpl & utc_time_zone) +{ + writeDateTimeText<'-', ':', 'T'>(datetime, buf, utc_time_zone); + buf.write('Z'); +} + +inline void writeDateTimeTextISO(DateTime64 datetime64, UInt32 scale, WriteBuffer & buf, const DateLUTImpl & utc_time_zone) +{ + writeDateTimeText<'-', ':', 'T'>(datetime64, scale, buf, utc_time_zone); + buf.write('Z'); +} + +inline void writeDateTimeUnixTimestamp(DateTime64 datetime64, UInt32 scale, WriteBuffer & buf) +{ + static constexpr UInt32 MaxScale = DecimalUtils::maxPrecision(); + scale = scale > MaxScale ? MaxScale : scale; + + auto c = DecimalUtils::split(datetime64, scale); + writeIntText(c.whole, buf); + + if (scale > 0) + { + buf.write('.'); + writeDecimalTypeFractionalText(c.fractional, scale, buf); + } +} /// Methods for output in binary format. template diff --git a/tests/queries/0_stateless/01516_date_time_output_format.reference b/tests/queries/0_stateless/01516_date_time_output_format.reference new file mode 100644 index 00000000000..8cb5ca739d3 --- /dev/null +++ b/tests/queries/0_stateless/01516_date_time_output_format.reference @@ -0,0 +1,12 @@ +2020-10-15 00:00:00 +2020-10-15 00:00:00 +2020-10-14T21:00:00Z +2020-10-14T21:00:00Z +1602709200 +1602709200 +2020-10-15 00:00:00.000 +2020-10-15 00:00:00.123 +2020-10-14T21:00:00.000Z +2020-10-14T21:00:00.123Z +1602709200.000 +1602709200.123 diff --git a/tests/queries/0_stateless/01516_date_time_output_format.sql b/tests/queries/0_stateless/01516_date_time_output_format.sql new file mode 100644 index 00000000000..224d8ef1035 --- /dev/null +++ b/tests/queries/0_stateless/01516_date_time_output_format.sql @@ -0,0 +1,36 @@ +DROP TABLE IF EXISTS test_datetime; + +CREATE TABLE test_datetime(timestamp DateTime('Europe/Moscow')) ENGINE=Log; + +INSERT INTO test_datetime VALUES ('2020-10-15 00:00:00'); + +SET date_time_output_format = 'simple'; +SELECT timestamp FROM test_datetime; +SELECT formatDateTime(toDateTime('2020-10-15 00:00:00', 'Europe/Moscow'), '%Y-%m-%d %R:%S') as formatted_simple FROM test_datetime; + +SET date_time_output_format = 'iso'; +SELECT timestamp FROM test_datetime; +SELECT formatDateTime(toDateTime('2020-10-15 00:00:00', 'Europe/Moscow'), '%Y-%m-%dT%R:%SZ', 'UTC') as formatted_iso FROM test_datetime;; + +SET date_time_output_format = 'unix_timestamp'; +SELECT timestamp FROM test_datetime; +SELECT toUnixTimestamp(timestamp) FROM test_datetime; + +SET date_time_output_format = 'simple'; +DROP TABLE test_datetime; + +CREATE TABLE test_datetime(timestamp DateTime64(3, 'Europe/Moscow')) Engine=Log; + +INSERT INTO test_datetime VALUES ('2020-10-15 00:00:00'), (1602709200123); + +SET date_time_output_format = 'simple'; +SELECT timestamp FROM test_datetime; + +SET date_time_output_format = 'iso'; +SELECT timestamp FROM test_datetime; + +SET date_time_output_format = 'unix_timestamp'; +SELECT timestamp FROM test_datetime; + +SET date_time_output_format = 'simple'; +DROP TABLE test_datetime; From 37c2a57202293b5680673cc4cffef4fedbd91993 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 13 Oct 2020 15:23:50 +0300 Subject: [PATCH 34/42] Update WriteHelpers.h --- src/IO/WriteHelpers.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index d55fc4c5efd..9d517913582 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -823,7 +823,7 @@ inline void writeDateTimeUnixTimestamp(DateTime64 datetime64, UInt32 scale, Writ auto c = DecimalUtils::split(datetime64, scale); writeIntText(c.whole, buf); - if (scale > 0) + if (scale > 0) { buf.write('.'); writeDecimalTypeFractionalText(c.fractional, scale, buf); From db4db42b65bed60512379e9ebc4dc009b7fc3666 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 13 Oct 2020 20:23:29 +0300 Subject: [PATCH 35/42] Fix broken links in docs --- docs/en/development/architecture.md | 2 +- docs/en/development/build.md | 2 +- docs/en/development/contrib.md | 2 +- docs/en/development/developer-instruction.md | 4 +- docs/en/development/style.md | 6 +- docs/en/development/tests.md | 2 +- .../table-engines/integrations/hdfs.md | 12 +- .../table-engines/integrations/kafka.md | 4 +- .../table-engines/integrations/rabbitmq.md | 2 +- .../custom-partitioning-key.md | 2 +- .../mergetree-family/graphitemergetree.md | 2 +- .../mergetree-family/mergetree.md | 12 +- .../mergetree-family/replication.md | 8 +- .../table-engines/special/distributed.md | 10 +- .../table-engines/special/external-data.md | 4 +- docs/en/engines/table-engines/special/join.md | 12 +- .../en/engines/table-engines/special/merge.md | 2 +- docs/en/engines/table-engines/special/url.md | 2 +- docs/en/faq/index.md | 2 +- docs/en/faq/integration/oracle-odbc.md | 2 +- .../example-datasets/amplab-benchmark.md | 2 +- .../example-datasets/nyc-taxi.md | 2 +- docs/en/getting-started/install.md | 4 +- docs/en/getting-started/playground.md | 8 +- docs/en/index.md | 10 +- docs/en/interfaces/formats.md | 34 +- docs/en/interfaces/http.md | 26 +- docs/en/interfaces/mysql.md | 2 +- .../third-party/client-libraries.md | 4 +- .../en/interfaces/third-party/integrations.md | 22 +- docs/en/operations/access-rights.md | 4 +- docs/en/operations/configuration-files.md | 2 +- docs/en/operations/monitoring.md | 4 +- .../sampling-query-profiler.md | 8 +- docs/en/operations/requirements.md | 2 +- .../settings.md | 130 +- .../settings/permissions-for-queries.md | 4 +- .../operations/settings/query-complexity.md | 122 +- docs/en/operations/settings/settings-users.md | 12 +- docs/en/operations/settings/settings.md | 362 +-- .../system-tables/asynchronous_metrics.md | 2 +- docs/en/operations/system-tables/clusters.md | 4 +- docs/en/operations/system-tables/events.md | 4 +- .../system-tables/graphite_retentions.md | 2 +- docs/en/operations/system-tables/index.md | 2 +- .../en/operations/system-tables/metric_log.md | 2 +- docs/en/operations/system-tables/metrics.md | 4 +- docs/en/operations/system-tables/part_log.md | 2 +- docs/en/operations/system-tables/processes.md | 2 +- docs/en/operations/system-tables/query_log.md | 8 +- .../system-tables/query_thread_log.md | 8 +- docs/en/operations/system-tables/replicas.md | 4 +- docs/en/operations/system-tables/text_log.md | 2 +- docs/en/operations/system-tables/trace_log.md | 4 +- docs/en/operations/tips.md | 2 +- docs/en/operations/troubleshooting.md | 4 +- .../parametric-functions.md | 2 +- .../aggregate-functions/reference/count.md | 2 +- docs/en/sql-reference/ansi.md | 6 +- docs/en/sql-reference/data-types/datetime.md | 2 +- .../sql-reference/data-types/fixedstring.md | 2 +- docs/en/sql-reference/data-types/index.md | 2 +- .../data-types/lowcardinality.md | 10 +- .../external-dicts-dict-hierarchical.md | 2 +- .../external-dicts-dict-layout.md | 36 +- .../external-dictionaries/external-dicts.md | 4 +- .../functions/array-functions.md | 2 +- .../functions/bitmap-functions.md | 8 +- .../functions/date-time-functions.md | 4 +- .../sql-reference/functions/hash-functions.md | 6 +- .../sql-reference/functions/introspection.md | 4 +- .../sql-reference/functions/json-functions.md | 24 +- .../functions/other-functions.md | 16 +- .../functions/string-functions.md | 6 +- .../functions/type-conversion-functions.md | 2 +- .../sql-reference/functions/url-functions.md | 8 +- .../functions/ym-dict-functions.md | 6 +- docs/en/sql-reference/operators/in.md | 8 +- .../statements/alter/partition.md | 2 +- .../sql-reference/statements/check-table.md | 2 +- .../sql-reference/statements/insert-into.md | 2 +- docs/en/sql-reference/statements/optimize.md | 2 +- .../statements/select/group-by.md | 8 +- .../sql-reference/statements/select/join.md | 10 +- docs/en/sql-reference/statements/system.md | 10 +- docs/en/sql-reference/syntax.md | 6 +- .../sql-reference/table-functions/cluster.md | 4 +- docs/en/sql-reference/table-functions/file.md | 14 +- docs/en/sql-reference/table-functions/hdfs.md | 12 +- .../en/sql-reference/table-functions/index.md | 4 +- .../sql-reference/table-functions/remote.md | 4 +- docs/en/whats-new/changelog/2017.md | 26 +- docs/en/whats-new/changelog/2018.md | 480 ++-- docs/en/whats-new/changelog/2019.md | 2536 ++++++++--------- docs/en/whats-new/security-changelog.md | 2 +- docs/es/development/architecture.md | 2 +- docs/es/development/build.md | 2 +- docs/es/development/contrib.md | 2 +- docs/es/development/developer-instruction.md | 4 +- docs/es/development/style.md | 6 +- docs/es/development/tests.md | 2 +- docs/es/engines/table-engines/index.md | 2 +- .../table-engines/integrations/hdfs.md | 12 +- .../table-engines/integrations/kafka.md | 2 +- .../custom-partitioning-key.md | 2 +- .../mergetree-family/graphitemergetree.md | 2 +- .../mergetree-family/mergetree.md | 8 +- .../mergetree-family/replication.md | 4 +- .../engines/table-engines/special/buffer.md | 4 +- .../table-engines/special/distributed.md | 12 +- .../table-engines/special/external-data.md | 6 +- docs/es/engines/table-engines/special/join.md | 4 +- .../es/engines/table-engines/special/merge.md | 2 +- docs/es/faq/general.md | 2 +- .../example-datasets/amplab-benchmark.md | 2 +- .../example-datasets/metrica.md | 2 +- .../example-datasets/nyc-taxi.md | 2 +- docs/es/getting-started/install.md | 4 +- docs/es/index.md | 10 +- docs/es/interfaces/formats.md | 28 +- docs/es/interfaces/http.md | 24 +- docs/es/interfaces/mysql.md | 2 +- .../third-party/client-libraries.md | 4 +- .../es/interfaces/third-party/integrations.md | 22 +- docs/es/operations/access-rights.md | 4 +- docs/es/operations/configuration-files.md | 2 +- docs/es/operations/monitoring.md | 4 +- .../sampling-query-profiler.md | 8 +- docs/es/operations/requirements.md | 2 +- .../settings.md | 100 +- .../operations/settings/query-complexity.md | 78 +- docs/es/operations/settings/settings-users.md | 12 +- docs/es/operations/settings/settings.md | 228 +- docs/es/operations/system-tables.md | 74 +- docs/es/operations/tips.md | 2 +- docs/es/operations/troubleshooting.md | 4 +- .../parametric-functions.md | 2 +- .../aggregate-functions/reference.md | 2 +- docs/es/sql-reference/ansi.md | 6 +- .../data-types/aggregatefunction.md | 2 +- docs/es/sql-reference/data-types/datetime.md | 2 +- .../sql-reference/data-types/fixedstring.md | 2 +- .../external-dicts-dict-hierarchical.md | 2 +- .../external-dicts-dict-layout.md | 8 +- .../external-dictionaries/external-dicts.md | 4 +- .../functions/array-functions.md | 2 +- .../functions/bitmap-functions.md | 8 +- docs/es/sql-reference/functions/geo.md | 8 +- .../sql-reference/functions/hash-functions.md | 6 +- .../sql-reference/functions/introspection.md | 4 +- .../sql-reference/functions/json-functions.md | 24 +- .../functions/other-functions.md | 8 +- .../functions/string-functions.md | 6 +- .../functions/type-conversion-functions.md | 2 +- .../sql-reference/functions/url-functions.md | 6 +- .../functions/ym-dict-functions.md | 4 +- docs/es/sql-reference/operators/in.md | 6 +- docs/es/sql-reference/statements/alter.md | 4 +- .../sql-reference/statements/insert-into.md | 2 +- docs/es/sql-reference/statements/misc.md | 4 +- .../statements/select/group-by.md | 8 +- .../sql-reference/statements/select/join.md | 2 +- docs/es/sql-reference/statements/system.md | 10 +- docs/es/sql-reference/syntax.md | 4 +- docs/es/sql-reference/table-functions/file.md | 14 +- docs/es/sql-reference/table-functions/hdfs.md | 12 +- .../es/sql-reference/table-functions/index.md | 2 +- .../sql-reference/table-functions/remote.md | 4 +- docs/es/whats-new/security-changelog.md | 2 +- docs/fa/development/build.md | 2 +- docs/fa/development/contrib.md | 2 +- docs/fa/development/developer-instruction.md | 4 +- docs/fa/development/style.md | 6 +- docs/fa/development/tests.md | 2 +- .../table-engines/integrations/hdfs.md | 12 +- .../table-engines/integrations/kafka.md | 2 +- .../custom-partitioning-key.md | 2 +- .../mergetree-family/graphitemergetree.md | 2 +- .../mergetree-family/mergetree.md | 6 +- .../mergetree-family/replication.md | 2 +- .../engines/table-engines/special/buffer.md | 4 +- .../table-engines/special/distributed.md | 10 +- .../table-engines/special/external-data.md | 6 +- docs/fa/engines/table-engines/special/join.md | 6 +- .../fa/engines/table-engines/special/merge.md | 2 +- docs/fa/faq/general.md | 2 +- .../example-datasets/amplab-benchmark.md | 2 +- .../example-datasets/metrica.md | 2 +- .../example-datasets/nyc-taxi.md | 2 +- docs/fa/getting-started/install.md | 4 +- docs/fa/index.md | 10 +- docs/fa/interfaces/formats.md | 28 +- docs/fa/interfaces/http.md | 18 +- docs/fa/interfaces/mysql.md | 2 +- .../third-party/client-libraries.md | 4 +- .../fa/interfaces/third-party/integrations.md | 18 +- docs/fa/operations/access-rights.md | 2 +- docs/fa/operations/configuration-files.md | 2 +- docs/fa/operations/monitoring.md | 4 +- .../sampling-query-profiler.md | 6 +- docs/fa/operations/requirements.md | 2 +- .../settings.md | 74 +- .../settings/permissions-for-queries.md | 4 +- .../operations/settings/query-complexity.md | 108 +- docs/fa/operations/settings/settings.md | 188 +- docs/fa/operations/system-tables.md | 80 +- docs/fa/operations/troubleshooting.md | 4 +- .../parametric-functions.md | 2 +- .../aggregate-functions/reference.md | 6 +- docs/fa/sql-reference/ansi.md | 4 +- .../data-types/aggregatefunction.md | 2 +- docs/fa/sql-reference/data-types/datetime.md | 2 +- .../sql-reference/data-types/fixedstring.md | 2 +- .../external-dicts-dict-hierarchical.md | 2 +- .../external-dicts-dict-layout.md | 10 +- .../external-dictionaries/external-dicts.md | 2 +- .../functions/array-functions.md | 2 +- .../functions/bitmap-functions.md | 6 +- .../functions/date-time-functions.md | 16 +- .../sql-reference/functions/hash-functions.md | 6 +- .../sql-reference/functions/introspection.md | 4 +- .../sql-reference/functions/json-functions.md | 24 +- .../functions/other-functions.md | 12 +- .../functions/string-functions.md | 6 +- .../functions/type-conversion-functions.md | 2 +- .../sql-reference/functions/url-functions.md | 6 +- .../functions/ym-dict-functions.md | 4 +- docs/fa/sql-reference/operators/in.md | 6 +- docs/fa/sql-reference/statements/alter.md | 2 +- .../sql-reference/statements/insert-into.md | 2 +- docs/fa/sql-reference/statements/misc.md | 4 +- .../statements/select/group-by.md | 10 +- .../sql-reference/statements/select/join.md | 6 +- docs/fa/sql-reference/statements/system.md | 8 +- docs/fa/sql-reference/syntax.md | 2 +- docs/fa/sql-reference/table-functions/file.md | 14 +- docs/fa/sql-reference/table-functions/hdfs.md | 12 +- .../fa/sql-reference/table-functions/index.md | 4 +- .../sql-reference/table-functions/remote.md | 4 +- docs/fa/whats-new/security-changelog.md | 2 +- docs/fr/development/architecture.md | 2 +- docs/fr/development/build.md | 2 +- docs/fr/development/contrib.md | 2 +- docs/fr/development/developer-instruction.md | 4 +- docs/fr/development/style.md | 6 +- docs/fr/development/tests.md | 2 +- docs/fr/engines/table-engines/index.md | 2 +- .../table-engines/integrations/hdfs.md | 12 +- .../table-engines/integrations/kafka.md | 2 +- .../custom-partitioning-key.md | 2 +- .../mergetree-family/graphitemergetree.md | 2 +- .../mergetree-family/mergetree.md | 8 +- .../mergetree-family/replication.md | 4 +- .../engines/table-engines/special/buffer.md | 4 +- .../table-engines/special/distributed.md | 10 +- .../table-engines/special/external-data.md | 6 +- docs/fr/engines/table-engines/special/join.md | 12 +- .../fr/engines/table-engines/special/merge.md | 2 +- docs/fr/engines/table-engines/special/url.md | 2 +- docs/fr/faq/general.md | 2 +- .../example-datasets/amplab-benchmark.md | 2 +- .../example-datasets/metrica.md | 2 +- .../example-datasets/nyc-taxi.md | 2 +- docs/fr/getting-started/install.md | 4 +- docs/fr/index.md | 10 +- docs/fr/interfaces/formats.md | 32 +- docs/fr/interfaces/http.md | 30 +- docs/fr/interfaces/mysql.md | 2 +- .../third-party/client-libraries.md | 4 +- .../fr/interfaces/third-party/integrations.md | 22 +- docs/fr/operations/access-rights.md | 4 +- docs/fr/operations/configuration-files.md | 2 +- docs/fr/operations/monitoring.md | 4 +- .../sampling-query-profiler.md | 8 +- docs/fr/operations/requirements.md | 2 +- .../settings.md | 110 +- .../settings/permissions-for-queries.md | 4 +- .../operations/settings/query-complexity.md | 118 +- docs/fr/operations/settings/settings-users.md | 12 +- docs/fr/operations/settings/settings.md | 272 +- docs/fr/operations/system-tables.md | 90 +- docs/fr/operations/tips.md | 2 +- docs/fr/operations/troubleshooting.md | 4 +- .../parametric-functions.md | 2 +- .../aggregate-functions/reference.md | 6 +- docs/fr/sql-reference/ansi.md | 6 +- .../data-types/aggregatefunction.md | 2 +- docs/fr/sql-reference/data-types/datetime.md | 2 +- .../sql-reference/data-types/fixedstring.md | 2 +- .../external-dicts-dict-hierarchical.md | 2 +- .../external-dicts-dict-layout.md | 26 +- .../external-dictionaries/external-dicts.md | 4 +- .../functions/array-functions.md | 2 +- .../functions/bitmap-functions.md | 8 +- .../functions/date-time-functions.md | 2 +- docs/fr/sql-reference/functions/geo.md | 8 +- .../sql-reference/functions/hash-functions.md | 6 +- .../sql-reference/functions/introspection.md | 4 +- .../sql-reference/functions/json-functions.md | 24 +- .../functions/other-functions.md | 16 +- .../functions/string-functions.md | 6 +- .../functions/type-conversion-functions.md | 2 +- .../sql-reference/functions/url-functions.md | 6 +- .../functions/ym-dict-functions.md | 6 +- docs/fr/sql-reference/operators/in.md | 6 +- docs/fr/sql-reference/statements/alter.md | 4 +- .../sql-reference/statements/insert-into.md | 2 +- docs/fr/sql-reference/statements/misc.md | 4 +- .../statements/select/group-by.md | 10 +- .../sql-reference/statements/select/join.md | 10 +- docs/fr/sql-reference/statements/system.md | 10 +- docs/fr/sql-reference/syntax.md | 4 +- docs/fr/sql-reference/table-functions/file.md | 14 +- docs/fr/sql-reference/table-functions/hdfs.md | 12 +- .../fr/sql-reference/table-functions/index.md | 4 +- .../sql-reference/table-functions/remote.md | 2 +- docs/fr/whats-new/security-changelog.md | 2 +- docs/ja/development/architecture.md | 2 +- docs/ja/development/build.md | 2 +- docs/ja/development/contrib.md | 2 +- docs/ja/development/developer-instruction.md | 4 +- docs/ja/development/style.md | 6 +- docs/ja/development/tests.md | 2 +- docs/ja/engines/table-engines/index.md | 2 +- .../table-engines/integrations/hdfs.md | 12 +- .../table-engines/integrations/kafka.md | 2 +- .../custom-partitioning-key.md | 2 +- .../mergetree-family/graphitemergetree.md | 2 +- .../mergetree-family/mergetree.md | 6 +- .../mergetree-family/replication.md | 4 +- .../engines/table-engines/special/buffer.md | 4 +- .../table-engines/special/distributed.md | 12 +- .../table-engines/special/external-data.md | 6 +- docs/ja/engines/table-engines/special/join.md | 12 +- .../ja/engines/table-engines/special/merge.md | 2 +- docs/ja/engines/table-engines/special/url.md | 2 +- docs/ja/faq/general.md | 2 +- .../example-datasets/amplab-benchmark.md | 2 +- docs/ja/getting-started/playground.md | 8 +- docs/ja/index.md | 10 +- docs/ja/interfaces/formats.md | 32 +- docs/ja/interfaces/http.md | 30 +- docs/ja/interfaces/mysql.md | 2 +- .../third-party/client-libraries.md | 4 +- .../ja/interfaces/third-party/integrations.md | 22 +- docs/ja/operations/access-rights.md | 4 +- docs/ja/operations/configuration-files.md | 2 +- docs/ja/operations/monitoring.md | 4 +- .../sampling-query-profiler.md | 8 +- docs/ja/operations/requirements.md | 2 +- .../settings.md | 110 +- .../settings/permissions-for-queries.md | 4 +- .../operations/settings/query-complexity.md | 118 +- docs/ja/operations/settings/settings-users.md | 12 +- docs/ja/operations/settings/settings.md | 268 +- docs/ja/operations/system-tables.md | 88 +- docs/ja/operations/tips.md | 2 +- docs/ja/operations/troubleshooting.md | 4 +- .../parametric-functions.md | 2 +- .../aggregate-functions/reference.md | 6 +- docs/ja/sql-reference/ansi.md | 6 +- .../data-types/aggregatefunction.md | 2 +- docs/ja/sql-reference/data-types/datetime.md | 2 +- .../sql-reference/data-types/fixedstring.md | 2 +- .../external-dicts-dict-hierarchical.md | 2 +- .../external-dicts-dict-layout.md | 22 +- .../external-dictionaries/external-dicts.md | 4 +- .../functions/array-functions.md | 2 +- .../functions/bitmap-functions.md | 8 +- .../functions/date-time-functions.md | 2 +- docs/ja/sql-reference/functions/geo.md | 6 +- .../sql-reference/functions/hash-functions.md | 6 +- .../sql-reference/functions/introspection.md | 4 +- .../sql-reference/functions/json-functions.md | 24 +- .../functions/other-functions.md | 16 +- .../functions/string-functions.md | 6 +- .../functions/type-conversion-functions.md | 2 +- .../sql-reference/functions/url-functions.md | 6 +- .../functions/ym-dict-functions.md | 6 +- docs/ja/sql-reference/statements/alter.md | 4 +- .../sql-reference/statements/insert-into.md | 2 +- docs/ja/sql-reference/statements/misc.md | 4 +- docs/ja/sql-reference/statements/optimize.md | 2 +- docs/ja/sql-reference/statements/system.md | 10 +- docs/ja/sql-reference/syntax.md | 4 +- docs/ja/sql-reference/table-functions/file.md | 14 +- docs/ja/sql-reference/table-functions/hdfs.md | 12 +- .../ja/sql-reference/table-functions/index.md | 4 +- .../sql-reference/table-functions/remote.md | 4 +- docs/ja/whats-new/security-changelog.md | 2 +- docs/ru/development/contrib.md | 2 +- docs/ru/development/developer-instruction.md | 2 +- docs/ru/development/style.md | 6 +- .../table-engines/integrations/hdfs.md | 12 +- .../table-engines/integrations/kafka.md | 2 +- .../table-engines/integrations/rabbitmq.md | 2 +- .../custom-partitioning-key.md | 4 +- .../mergetree-family/graphitemergetree.md | 2 +- .../mergetree-family/mergetree.md | 10 +- .../mergetree-family/replication.md | 4 +- .../engines/table-engines/special/buffer.md | 4 +- .../table-engines/special/distributed.md | 12 +- .../table-engines/special/external-data.md | 6 +- docs/ru/engines/table-engines/special/join.md | 12 +- docs/ru/engines/table-engines/special/url.md | 2 +- docs/ru/faq/general.md | 2 +- .../example-datasets/amplab-benchmark.md | 2 +- .../example-datasets/nyc-taxi.md | 2 +- docs/ru/getting-started/playground.md | 8 +- docs/ru/index.md | 10 +- docs/ru/interfaces/formats.md | 24 +- docs/ru/interfaces/http.md | 14 +- docs/ru/interfaces/mysql.md | 2 +- .../third-party/client-libraries.md | 4 +- .../ru/interfaces/third-party/integrations.md | 22 +- docs/ru/operations/configuration-files.md | 2 +- docs/ru/operations/monitoring.md | 2 +- .../sampling-query-profiler.md | 6 +- docs/ru/operations/requirements.md | 2 +- .../settings.md | 120 +- .../settings/merge-tree-settings.md | 30 +- .../settings/permissions-for-queries.md | 4 +- .../operations/settings/query-complexity.md | 120 +- docs/ru/operations/settings/settings-users.md | 10 +- docs/ru/operations/settings/settings.md | 318 +-- .../system-tables/asynchronous_metrics.md | 2 +- docs/ru/operations/system-tables/events.md | 4 +- .../system-tables/graphite_retentions.md | 2 +- .../ru/operations/system-tables/metric_log.md | 2 +- docs/ru/operations/system-tables/metrics.md | 4 +- docs/ru/operations/system-tables/processes.md | 2 +- docs/ru/operations/system-tables/query_log.md | 2 +- docs/ru/operations/system-tables/replicas.md | 8 +- docs/ru/operations/system-tables/trace_log.md | 2 +- docs/ru/operations/tips.md | 2 +- .../aggregate-functions/reference/count.md | 2 +- .../data-types/aggregatefunction.md | 2 +- docs/ru/sql-reference/data-types/datetime.md | 2 +- docs/ru/sql-reference/data-types/decimal.md | 2 +- .../sql-reference/data-types/fixedstring.md | 2 +- .../external-dicts-dict-hierarchical.md | 2 +- .../external-dicts-dict-layout.md | 26 +- .../external-dictionaries/external-dicts.md | 4 +- .../functions/date-time-functions.md | 2 +- .../sql-reference/functions/hash-functions.md | 6 +- .../sql-reference/functions/introspection.md | 2 +- .../sql-reference/functions/json-functions.md | 24 +- .../functions/other-functions.md | 18 +- .../functions/string-functions.md | 6 +- .../functions/type-conversion-functions.md | 2 +- .../sql-reference/functions/url-functions.md | 6 +- .../functions/ym-dict-functions.md | 6 +- docs/ru/sql-reference/operators/in.md | 6 +- .../sql-reference/statements/check-table.md | 2 +- .../sql-reference/statements/insert-into.md | 2 +- docs/ru/sql-reference/statements/optimize.md | 2 +- .../statements/select/group-by.md | 10 +- .../sql-reference/statements/select/join.md | 12 +- docs/ru/sql-reference/statements/system.md | 10 +- .../sql-reference/table-functions/cluster.md | 4 +- docs/ru/sql-reference/table-functions/file.md | 14 +- .../ru/sql-reference/table-functions/index.md | 4 +- .../sql-reference/table-functions/remote.md | 2 +- docs/ru/whats-new/extended-roadmap.md | 28 +- docs/ru/whats-new/security-changelog.md | 2 +- docs/tools/output.md | 10 +- docs/tr/development/architecture.md | 2 +- docs/tr/development/build.md | 2 +- docs/tr/development/contrib.md | 2 +- docs/tr/development/developer-instruction.md | 4 +- docs/tr/development/style.md | 6 +- docs/tr/development/tests.md | 2 +- docs/tr/engines/table-engines/index.md | 2 +- .../table-engines/integrations/hdfs.md | 12 +- .../table-engines/integrations/kafka.md | 2 +- .../custom-partitioning-key.md | 2 +- .../mergetree-family/graphitemergetree.md | 2 +- .../mergetree-family/mergetree.md | 8 +- .../mergetree-family/replication.md | 4 +- .../engines/table-engines/special/buffer.md | 4 +- .../table-engines/special/distributed.md | 10 +- .../table-engines/special/external-data.md | 6 +- docs/tr/engines/table-engines/special/join.md | 12 +- .../tr/engines/table-engines/special/merge.md | 2 +- docs/tr/engines/table-engines/special/url.md | 2 +- docs/tr/faq/general.md | 2 +- .../example-datasets/amplab-benchmark.md | 2 +- .../example-datasets/metrica.md | 2 +- .../example-datasets/nyc-taxi.md | 2 +- docs/tr/getting-started/install.md | 4 +- docs/tr/index.md | 10 +- docs/tr/interfaces/formats.md | 32 +- docs/tr/interfaces/http.md | 30 +- docs/tr/interfaces/mysql.md | 2 +- .../third-party/client-libraries.md | 4 +- .../tr/interfaces/third-party/integrations.md | 22 +- docs/tr/operations/access-rights.md | 4 +- docs/tr/operations/configuration-files.md | 2 +- docs/tr/operations/monitoring.md | 4 +- .../sampling-query-profiler.md | 8 +- docs/tr/operations/requirements.md | 2 +- .../settings.md | 110 +- .../settings/permissions-for-queries.md | 4 +- .../operations/settings/query-complexity.md | 118 +- docs/tr/operations/settings/settings-users.md | 10 +- docs/tr/operations/settings/settings.md | 272 +- docs/tr/operations/system-tables.md | 90 +- docs/tr/operations/tips.md | 2 +- docs/tr/operations/troubleshooting.md | 4 +- .../parametric-functions.md | 2 +- .../aggregate-functions/reference.md | 6 +- docs/tr/sql-reference/ansi.md | 6 +- .../data-types/aggregatefunction.md | 2 +- docs/tr/sql-reference/data-types/datetime.md | 2 +- .../sql-reference/data-types/fixedstring.md | 2 +- .../external-dicts-dict-hierarchical.md | 2 +- .../external-dicts-dict-layout.md | 22 +- .../external-dictionaries/external-dicts.md | 4 +- .../functions/array-functions.md | 2 +- .../functions/bitmap-functions.md | 8 +- .../functions/date-time-functions.md | 2 +- docs/tr/sql-reference/functions/geo.md | 8 +- .../sql-reference/functions/hash-functions.md | 6 +- .../sql-reference/functions/introspection.md | 4 +- .../sql-reference/functions/json-functions.md | 24 +- .../functions/other-functions.md | 16 +- .../functions/string-functions.md | 6 +- .../functions/type-conversion-functions.md | 2 +- .../sql-reference/functions/url-functions.md | 6 +- .../functions/ym-dict-functions.md | 6 +- docs/tr/sql-reference/statements/alter.md | 4 +- .../sql-reference/statements/insert-into.md | 2 +- docs/tr/sql-reference/statements/misc.md | 4 +- docs/tr/sql-reference/statements/system.md | 10 +- docs/tr/sql-reference/syntax.md | 4 +- docs/tr/sql-reference/table-functions/file.md | 14 +- docs/tr/sql-reference/table-functions/hdfs.md | 12 +- .../tr/sql-reference/table-functions/index.md | 4 +- .../sql-reference/table-functions/remote.md | 2 +- docs/tr/whats-new/security-changelog.md | 2 +- docs/zh/changelog/index.md | 1124 ++++---- docs/zh/development/build-cross-osx.md | 2 +- docs/zh/development/build.md | 2 +- docs/zh/development/contrib.md | 2 +- docs/zh/development/developer-instruction.md | 4 +- docs/zh/development/style.md | 6 +- docs/zh/development/tests.md | 2 +- .../table-engines/integrations/hdfs.md | 12 +- .../table-engines/integrations/kafka.md | 2 +- .../custom-partitioning-key.md | 2 +- .../mergetree-family/graphitemergetree.md | 2 +- .../mergetree-family/mergetree.md | 10 +- .../mergetree-family/replication.md | 4 +- .../engines/table-engines/special/buffer.md | 12 +- .../table-engines/special/distributed.md | 10 +- .../table-engines/special/external-data.md | 6 +- docs/zh/engines/table-engines/special/join.md | 12 +- .../zh/engines/table-engines/special/merge.md | 2 +- docs/zh/faq/general.md | 2 +- .../example-datasets/amplab-benchmark.md | 2 +- .../example-datasets/metrica.md | 2 +- .../example-datasets/nyc-taxi.md | 2 +- docs/zh/getting-started/install.md | 4 +- docs/zh/index.md | 10 +- docs/zh/interfaces/formats.md | 34 +- docs/zh/interfaces/http.md | 2 +- docs/zh/interfaces/mysql.md | 2 +- .../third-party/client-libraries.md | 4 +- .../zh/interfaces/third-party/integrations.md | 20 +- docs/zh/operations/configuration-files.md | 2 +- docs/zh/operations/monitoring.md | 2 +- .../sampling-query-profiler.md | 8 +- docs/zh/operations/requirements.md | 2 +- .../settings.md | 106 +- .../settings/permissions-for-queries.md | 4 +- .../operations/settings/query-complexity.md | 100 +- docs/zh/operations/settings/settings-users.md | 2 +- docs/zh/operations/settings/settings.md | 278 +- .../system-tables/asynchronous_metric_log.md | 4 +- .../system-tables/asynchronous_metrics.md | 4 +- docs/zh/operations/system-tables/clusters.md | 4 +- .../system-tables/data_type_families.md | 2 +- .../system-tables/detached_parts.md | 2 +- docs/zh/operations/system-tables/disks.md | 2 +- docs/zh/operations/system-tables/events.md | 4 +- .../system-tables/graphite_retentions.md | 4 +- docs/zh/operations/system-tables/index.md | 2 +- .../system-tables/merge_tree_settings.md | 2 +- .../zh/operations/system-tables/metric_log.md | 4 +- docs/zh/operations/system-tables/metrics.md | 4 +- docs/zh/operations/system-tables/mutations.md | 16 +- .../zh/operations/system-tables/numbers_mt.md | 2 +- docs/zh/operations/system-tables/part_log.md | 4 +- docs/zh/operations/system-tables/processes.md | 2 +- docs/zh/operations/system-tables/query_log.md | 10 +- .../system-tables/query_thread_log.md | 10 +- docs/zh/operations/system-tables/replicas.md | 4 +- .../system-tables/storage_policies.md | 2 +- .../operations/system-tables/table_engines.md | 2 +- docs/zh/operations/system-tables/text_log.md | 2 +- docs/zh/operations/system-tables/trace_log.md | 6 +- docs/zh/operations/tips.md | 2 +- docs/zh/operations/troubleshooting.md | 4 +- .../parametric-functions.md | 2 +- .../aggregate-functions/reference.md | 6 +- docs/zh/sql-reference/ansi.md | 4 +- .../data-types/aggregatefunction.md | 2 +- .../sql-reference/data-types/domains/index.md | 6 +- .../sql-reference/data-types/fixedstring.md | 2 +- .../external-dicts-dict-hierarchical.md | 2 +- .../external-dicts-dict-layout.md | 22 +- .../external-dictionaries/external-dicts.md | 4 +- .../functions/array-functions.md | 4 +- .../functions/bitmap-functions.md | 4 +- .../functions/conditional-functions.md | 2 +- .../functions/date-time-functions.md | 2 +- .../functions/ext-dict-functions.md | 6 +- docs/zh/sql-reference/functions/geo.md | 8 +- .../sql-reference/functions/hash-functions.md | 4 +- .../sql-reference/functions/introspection.md | 4 +- .../sql-reference/functions/json-functions.md | 22 +- .../functions/other-functions.md | 8 +- .../functions/string-functions.md | 4 +- .../functions/string-replace-functions.md | 2 +- .../functions/type-conversion-functions.md | 2 +- .../sql-reference/functions/url-functions.md | 12 +- .../functions/ym-dict-functions.md | 4 +- docs/zh/sql-reference/operators/in.md | 8 +- docs/zh/sql-reference/statements/alter.md | 2 +- .../sql-reference/statements/insert-into.md | 2 +- docs/zh/sql-reference/statements/misc.md | 4 +- .../statements/select/group-by.md | 8 +- .../sql-reference/statements/select/join.md | 10 +- docs/zh/sql-reference/statements/system.md | 12 +- docs/zh/sql-reference/syntax.md | 4 +- docs/zh/sql-reference/table-functions/file.md | 14 +- docs/zh/sql-reference/table-functions/hdfs.md | 12 +- .../zh/sql-reference/table-functions/index.md | 4 +- .../sql-reference/table-functions/remote.md | 4 +- docs/zh/whats-new/security-changelog.md | 2 +- 640 files changed, 6092 insertions(+), 6092 deletions(-) diff --git a/docs/en/development/architecture.md b/docs/en/development/architecture.md index da5b4470704..bea75140606 100644 --- a/docs/en/development/architecture.md +++ b/docs/en/development/architecture.md @@ -45,7 +45,7 @@ A `Block` is a container that represents a subset (chunk) of a table in memory. When we calculate some function over columns in a block, we add another column with its result to the block, and we don’t touch columns for arguments of the function because operations are immutable. Later, unneeded columns can be removed from the block, but not modified. It is convenient for the elimination of common subexpressions. -Blocks are created for every processed chunk of data. Note that for the same type of calculation, the column names and types remain the same for different blocks, and only column data changes. It is better to split block data from the block header because small block sizes have a high overhead of temporary strings for copying shared\_ptrs and column names. +Blocks are created for every processed chunk of data. Note that for the same type of calculation, the column names and types remain the same for different blocks, and only column data changes. It is better to split block data from the block header because small block sizes have a high overhead of temporary strings for copying shared_ptrs and column names. ## Block Streams {#block-streams} diff --git a/docs/en/development/build.md b/docs/en/development/build.md index 145dd99e8de..e9b6ecebae8 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -7,7 +7,7 @@ toc_title: Build on Linux Supported platforms: -- x86\_64 +- x86_64 - AArch64 - Power9 (experimental) diff --git a/docs/en/development/contrib.md b/docs/en/development/contrib.md index b0629479568..639b78185e4 100644 --- a/docs/en/development/contrib.md +++ b/docs/en/development/contrib.md @@ -26,7 +26,7 @@ toc_title: Third-Party Libraries Used | libpcg-random | [Apache License 2.0](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libpcg-random/LICENSE-APACHE.txt) | | libressl | [OpenSSL License](https://github.com/ClickHouse-Extras/ssl/blob/master/COPYING) | | librdkafka | [BSD 2-Clause License](https://github.com/edenhill/librdkafka/blob/363dcad5a23dc29381cc626620e68ae418b3af19/LICENSE) | -| libwidechar\_width | [CC0 1.0 Universal](https://github.com/ClickHouse/ClickHouse/blob/master/libs/libwidechar_width/LICENSE) | +| libwidechar_width | [CC0 1.0 Universal](https://github.com/ClickHouse/ClickHouse/blob/master/libs/libwidechar_width/LICENSE) | | llvm | [BSD 3-Clause License](https://github.com/ClickHouse-Extras/llvm/blob/163def217817c90fb982a6daf384744d8472b92b/llvm/LICENSE.TXT) | | lz4 | [BSD 2-Clause License](https://github.com/lz4/lz4/blob/c10863b98e1503af90616ae99725ecd120265dfb/LICENSE) | | mariadb-connector-c | [LGPL v2.1](https://github.com/ClickHouse-Extras/mariadb-connector-c/blob/3.1/COPYING.LIB) | diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index db884649c2d..3dc7c138520 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -40,7 +40,7 @@ In the command line terminal run: git clone --recursive git@github.com:your_github_username/ClickHouse.git cd ClickHouse -Note: please, substitute *your\_github\_username* with what is appropriate! +Note: please, substitute *your_github_username* with what is appropriate! This command will create a directory `ClickHouse` containing the working copy of the project. @@ -150,7 +150,7 @@ Now that you are ready to build ClickHouse we recommend you to create a separate mkdir build cd build -You can have several different directories (build\_release, build\_debug, etc.) for different types of build. +You can have several different directories (build_release, build_debug, etc.) for different types of build. While inside the `build` directory, configure your build by running CMake. Before the first run, you need to define environment variables that specify compiler (version 9 gcc compiler in this example). diff --git a/docs/en/development/style.md b/docs/en/development/style.md index b2007e8042c..828f866c23c 100644 --- a/docs/en/development/style.md +++ b/docs/en/development/style.md @@ -354,7 +354,7 @@ In all other cases, use a name that describes the meaning. bool info_successfully_loaded = false; ``` -**9.** Names of `define`s and global constants use ALL\_CAPS with underscores. +**9.** Names of `define`s and global constants use ALL_CAPS with underscores. ``` cpp #define MAX_SRC_TABLE_NAMES_TO_STORE 1000 @@ -394,7 +394,7 @@ The underscore suffix can be omitted if the argument is not used in the construc timer (not m_timer) ``` -**14.** For the constants in an `enum`, use CamelCase with a capital letter. ALL\_CAPS is also acceptable. If the `enum` is non-local, use an `enum class`. +**14.** For the constants in an `enum`, use CamelCase with a capital letter. ALL_CAPS is also acceptable. If the `enum` is non-local, use an `enum class`. ``` cpp enum class CompressionMethod @@ -707,7 +707,7 @@ The standard library is used (`libc++`). **4.**OS: Linux Ubuntu, not older than Precise. -**5.**Code is written for x86\_64 CPU architecture. +**5.**Code is written for x86_64 CPU architecture. The CPU instruction set is the minimum supported set among our servers. Currently, it is SSE 4.2. diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 1302d675d3e..8c06c868e66 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -220,7 +220,7 @@ Debug version of `jemalloc` is used for debug build. ClickHouse fuzzing is implemented both using [libFuzzer](https://llvm.org/docs/LibFuzzer.html) and random SQL queries. All the fuzz testing should be performed with sanitizers (Address and Undefined). -LibFuzzer is used for isolated fuzz testing of library code. Fuzzers are implemented as part of test code and have “\_fuzzer” name postfixes. +LibFuzzer is used for isolated fuzz testing of library code. Fuzzers are implemented as part of test code and have “_fuzzer” name postfixes. Fuzzer example can be found at `src/Parsers/tests/lexer_fuzzer.cpp`. LibFuzzer-specific configs, dictionaries and corpus are stored at `tests/fuzz`. We encourage you to write fuzz tests for every functionality that handles user input. diff --git a/docs/en/engines/table-engines/integrations/hdfs.md b/docs/en/engines/table-engines/integrations/hdfs.md index 2c01d0cf8e9..f36a4771222 100644 --- a/docs/en/engines/table-engines/integrations/hdfs.md +++ b/docs/en/engines/table-engines/integrations/hdfs.md @@ -71,12 +71,12 @@ Constructions with `{}` are similar to the [remote](../../../sql-reference/table 1. Suppose we have several files in TSV format with the following URIs on HDFS: -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_3’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_3’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_3’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_3’ 1. There are several ways to make a table consisting of all six files: diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index d0a4bc928a7..c519d6bb136 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -134,7 +134,7 @@ Example: SELECT level, sum(total) FROM daily GROUP BY level; ``` -To improve performance, received messages are grouped into blocks the size of [max\_insert\_block\_size](../../../operations/server-configuration-parameters/settings.md#settings-max_insert_block_size). If the block wasn’t formed within [stream\_flush\_interval\_ms](../../../operations/server-configuration-parameters/settings.md) milliseconds, the data will be flushed to the table regardless of the completeness of the block. +To improve performance, received messages are grouped into blocks the size of [max_insert_block_size](../../../operations/server-configuration-parameters/settings.md#settings-max_insert_block_size). If the block wasn’t formed within [stream_flush_interval_ms](../../../operations/server-configuration-parameters/settings.md) milliseconds, the data will be flushed to the table regardless of the completeness of the block. To stop receiving topic data or to change the conversion logic, detach the materialized view: @@ -192,6 +192,6 @@ Example: **See Also** - [Virtual columns](../../../engines/table-engines/index.md#table_engines-virtual_columns) -- [background\_schedule\_pool\_size](../../../operations/settings/settings.md#background_schedule_pool_size) +- [background_schedule_pool_size](../../../operations/settings/settings.md#background_schedule_pool_size) [Original article](https://clickhouse.tech/docs/en/operations/table_engines/kafka/) diff --git a/docs/en/engines/table-engines/integrations/rabbitmq.md b/docs/en/engines/table-engines/integrations/rabbitmq.md index 7ee270539cc..dd14ee3b4b1 100644 --- a/docs/en/engines/table-engines/integrations/rabbitmq.md +++ b/docs/en/engines/table-engines/integrations/rabbitmq.md @@ -109,7 +109,7 @@ Setting `rabbitmq_queue_base` may be used for the following cases: - to be able to restore reading from certain durable queues when not all messages were successfully consumed. To resume consumption from one specific queue - set its name in `rabbitmq_queue_base` setting and do not specify `rabbitmq_num_consumers` and `rabbitmq_num_queues` (defaults to 1). To resume consumption from all queues, which were declared for a specific table - just specify the same settings: `rabbitmq_queue_base`, `rabbitmq_num_consumers`, `rabbitmq_num_queues`. By default, queue names will be unique to tables. - to reuse queues as they are declared durable and not auto-deleted. (Can be deleted via any of RabbitMQ CLI tools.) -To improve performance, received messages are grouped into blocks the size of [max\_insert\_block\_size](../../../operations/server-configuration-parameters/settings.md#settings-max_insert_block_size). If the block wasn’t formed within [stream\_flush\_interval\_ms](../../../operations/server-configuration-parameters/settings.md) milliseconds, the data will be flushed to the table regardless of the completeness of the block. +To improve performance, received messages are grouped into blocks the size of [max_insert_block_size](../../../operations/server-configuration-parameters/settings.md#settings-max_insert_block_size). If the block wasn’t formed within [stream_flush_interval_ms](../../../operations/server-configuration-parameters/settings.md) milliseconds, the data will be flushed to the table regardless of the completeness of the block. If `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` settings are specified along with `rabbitmq_exchange_type`, then: diff --git a/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md b/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md index 8854ec0cdb7..855d5fdadf4 100644 --- a/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md +++ b/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md @@ -114,7 +114,7 @@ drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 5 12:09 201902_4_6_1 drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 1 16:48 detached ``` -The folders ‘201901\_1\_1\_0’, ‘201901\_1\_7\_1’ and so on are the directories of the parts. Each part relates to a corresponding partition and contains data just for a certain month (the table in this example has partitioning by month). +The folders ‘201901_1_1_0’, ‘201901_1_7_1’ and so on are the directories of the parts. Each part relates to a corresponding partition and contains data just for a certain month (the table in this example has partitioning by month). The `detached` directory contains parts that were detached from the table using the [DETACH](../../../sql-reference/statements/alter/partition.md#alter_detach-partition) query. The corrupted parts are also moved to this directory, instead of being deleted. The server does not use the parts from the `detached` directory. You can add, delete, or modify the data in this directory at any time – the server will not know about this until you run the [ATTACH](../../../sql-reference/statements/alter/partition.md#alter_attach-partition) query. diff --git a/docs/en/engines/table-engines/mergetree-family/graphitemergetree.md b/docs/en/engines/table-engines/mergetree-family/graphitemergetree.md index ebbe2b0b5b7..14df9ae130e 100644 --- a/docs/en/engines/table-engines/mergetree-family/graphitemergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/graphitemergetree.md @@ -79,7 +79,7 @@ All of the parameters excepting `config_section` have the same meaning as in `Me ## Rollup Configuration {#rollup-configuration} -The settings for rollup are defined by the [graphite\_rollup](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) parameter in the server configuration. The name of the parameter could be any. You can create several configurations and use them for different tables. +The settings for rollup are defined by the [graphite_rollup](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) parameter in the server configuration. The name of the parameter could be any. You can create several configurations and use them for different tables. Rollup configuration structure: diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index d40e5a86f7b..c96e31a6eed 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -205,7 +205,7 @@ The number of columns in the primary key is not explicitly limited. Depending on A long primary key will negatively affect the insert performance and memory consumption, but extra columns in the primary key do not affect ClickHouse performance during `SELECT` queries. -You can create a table without a primary key using the `ORDER BY tuple()` syntax. In this case, ClickHouse stores data in the order of inserting. If you want to save data order when inserting data by `INSERT ... SELECT` queries, set [max\_insert\_threads = 1](../../../operations/settings/settings.md#settings-max-insert-threads). +You can create a table without a primary key using the `ORDER BY tuple()` syntax. In this case, ClickHouse stores data in the order of inserting. If you want to save data order when inserting data by `INSERT ... SELECT` queries, set [max_insert_threads = 1](../../../operations/settings/settings.md#settings-max-insert-threads). To select data in the initial order, use [single-threaded](../../../operations/settings/settings.md#settings-max_threads) `SELECT` queries. @@ -248,7 +248,7 @@ In the example below, the index can’t be used. SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' ``` -To check whether ClickHouse can use the index when running a query, use the settings [force\_index\_by\_date](../../../operations/settings/settings.md#settings-force_index_by_date) and [force\_primary\_key](../../../operations/settings/settings.md). +To check whether ClickHouse can use the index when running a query, use the settings [force_index_by_date](../../../operations/settings/settings.md#settings-force_index_by_date) and [force_primary_key](../../../operations/settings/settings.md). The key for partitioning by month allows reading only those data blocks which contain dates from the proper range. In this case, the data block may contain data for many dates (up to an entire month). Within a block, data is sorted by primary key, which might not contain the date as the first column. Because of this, using a query with only a date condition that does not specify the primary key prefix will cause more data to be read than for a single date. @@ -339,7 +339,7 @@ Conditions in the `WHERE` clause contains calls of the functions that operate wi The `set` index can be used with all functions. Function subsets for other indexes are shown in the table below. -| Function (operator) / Index | primary key | minmax | ngrambf\_v1 | tokenbf\_v1 | bloom\_filter | +| Function (operator) / Index | primary key | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter | |------------------------------------------------------------------------------------------------------------|-------------|--------|-------------|-------------|---------------| | [equals (=, ==)](../../../sql-reference/functions/comparison-functions.md#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | | [notEquals(!=, \<\>)](../../../sql-reference/functions/comparison-functions.md#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | @@ -505,7 +505,7 @@ Data part is the minimum movable unit for `MergeTree`-engine tables. The data be - Volume — Ordered set of equal disks (similar to [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures)). - Storage policy — Set of volumes and the rules for moving data between them. -The names given to the described entities can be found in the system tables, [system.storage\_policies](../../../operations/system-tables/storage_policies.md#system_tables-storage_policies) and [system.disks](../../../operations/system-tables/disks.md#system_tables-disks). To apply one of the configured storage policies for a table, use the `storage_policy` setting of `MergeTree`-engine family tables. +The names given to the described entities can be found in the system tables, [system.storage_policies](../../../operations/system-tables/storage_policies.md#system_tables-storage_policies) and [system.disks](../../../operations/system-tables/disks.md#system_tables-disks). To apply one of the configured storage policies for a table, use the `storage_policy` setting of `MergeTree`-engine family tables. ### Configuration {#table_engine-mergetree-multiple-volumes_configure} @@ -635,7 +635,7 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' The `default` storage policy implies using only one volume, which consists of only one disk given in ``. Once a table is created, its storage policy cannot be changed. -The number of threads performing background moves of data parts can be changed by [background\_move\_pool\_size](../../../operations/settings/settings.md#background_move_pool_size) setting. +The number of threads performing background moves of data parts can be changed by [background_move_pool_size](../../../operations/settings/settings.md#background_move_pool_size) setting. ### Details {#details} @@ -654,7 +654,7 @@ In all these cases except for mutations and partition freezing, a part is stored Under the hood, mutations and partition freezing make use of [hard links](https://en.wikipedia.org/wiki/Hard_link). Hard links between different disks are not supported, therefore in such cases the resulting parts are stored on the same disks as the initial ones. In the background, parts are moved between volumes on the basis of the amount of free space (`move_factor` parameter) according to the order the volumes are declared in the configuration file. -Data is never transferred from the last one and into the first one. One may use system tables [system.part\_log](../../../operations/system-tables/part_log.md#system_tables-part-log) (field `type = MOVE_PART`) and [system.parts](../../../operations/system-tables/parts.md#system_tables-parts) (fields `path` and `disk`) to monitor background moves. Also, the detailed information can be found in server logs. +Data is never transferred from the last one and into the first one. One may use system tables [system.part_log](../../../operations/system-tables/part_log.md#system_tables-part-log) (field `type = MOVE_PART`) and [system.parts](../../../operations/system-tables/parts.md#system_tables-parts) (fields `path` and `disk`) to monitor background moves. Also, the detailed information can be found in server logs. User can force moving a part or a partition from one volume to another using the query [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](../../../sql-reference/statements/alter/partition.md#alter_move-partition), all the restrictions for background operations are taken into account. The query initiates a move on its own and does not wait for background operations to be completed. User will get an error message if not enough free space is available or if any of the required conditions are not met. diff --git a/docs/en/engines/table-engines/mergetree-family/replication.md b/docs/en/engines/table-engines/mergetree-family/replication.md index 1d380988ccc..cdb2d718415 100644 --- a/docs/en/engines/table-engines/mergetree-family/replication.md +++ b/docs/en/engines/table-engines/mergetree-family/replication.md @@ -57,19 +57,19 @@ You can specify any existing ZooKeeper cluster and the system will use a directo If ZooKeeper isn’t set in the config file, you can’t create replicated tables, and any existing replicated tables will be read-only. -ZooKeeper is not used in `SELECT` queries because replication does not affect the performance of `SELECT` and queries run just as fast as they do for non-replicated tables. When querying distributed replicated tables, ClickHouse behavior is controlled by the settings [max\_replica\_delay\_for\_distributed\_queries](../../../operations/settings/settings.md#settings-max_replica_delay_for_distributed_queries) and [fallback\_to\_stale\_replicas\_for\_distributed\_queries](../../../operations/settings/settings.md#settings-fallback_to_stale_replicas_for_distributed_queries). +ZooKeeper is not used in `SELECT` queries because replication does not affect the performance of `SELECT` and queries run just as fast as they do for non-replicated tables. When querying distributed replicated tables, ClickHouse behavior is controlled by the settings [max_replica_delay_for_distributed_queries](../../../operations/settings/settings.md#settings-max_replica_delay_for_distributed_queries) and [fallback_to_stale_replicas_for_distributed_queries](../../../operations/settings/settings.md#settings-fallback_to_stale_replicas_for_distributed_queries). For each `INSERT` query, approximately ten entries are added to ZooKeeper through several transactions. (To be more precise, this is for each inserted block of data; an INSERT query contains one block or one block per `max_insert_block_size = 1048576` rows.) This leads to slightly longer latencies for `INSERT` compared to non-replicated tables. But if you follow the recommendations to insert data in batches of no more than one `INSERT` per second, it doesn’t create any problems. The entire ClickHouse cluster used for coordinating one ZooKeeper cluster has a total of several hundred `INSERTs` per second. The throughput on data inserts (the number of rows per second) is just as high as for non-replicated data. For very large clusters, you can use different ZooKeeper clusters for different shards. However, this hasn’t proven necessary on the Yandex.Metrica cluster (approximately 300 servers). -Replication is asynchronous and multi-master. `INSERT` queries (as well as `ALTER`) can be sent to any available server. Data is inserted on the server where the query is run, and then it is copied to the other servers. Because it is asynchronous, recently inserted data appears on the other replicas with some latency. If part of the replicas are not available, the data is written when they become available. If a replica is available, the latency is the amount of time it takes to transfer the block of compressed data over the network. The number of threads performing background tasks for replicated tables can be set by [background\_schedule\_pool\_size](../../../operations/settings/settings.md#background_schedule_pool_size) setting. +Replication is asynchronous and multi-master. `INSERT` queries (as well as `ALTER`) can be sent to any available server. Data is inserted on the server where the query is run, and then it is copied to the other servers. Because it is asynchronous, recently inserted data appears on the other replicas with some latency. If part of the replicas are not available, the data is written when they become available. If a replica is available, the latency is the amount of time it takes to transfer the block of compressed data over the network. The number of threads performing background tasks for replicated tables can be set by [background_schedule_pool_size](../../../operations/settings/settings.md#background_schedule_pool_size) setting. By default, an INSERT query waits for confirmation of writing the data from only one replica. If the data was successfully written to only one replica and the server with this replica ceases to exist, the stored data will be lost. To enable getting confirmation of data writes from multiple replicas, use the `insert_quorum` option. Each block of data is written atomically. The INSERT query is divided into blocks up to `max_insert_block_size = 1048576` rows. In other words, if the `INSERT` query has less than 1048576 rows, it is made atomically. -Data blocks are deduplicated. For multiple writes of the same data block (data blocks of the same size containing the same rows in the same order), the block is only written once. The reason for this is in case of network failures when the client application doesn’t know if the data was written to the DB, so the `INSERT` query can simply be repeated. It doesn’t matter which replica INSERTs were sent to with identical data. `INSERTs` are idempotent. Deduplication parameters are controlled by [merge\_tree](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-merge_tree) server settings. +Data blocks are deduplicated. For multiple writes of the same data block (data blocks of the same size containing the same rows in the same order), the block is only written once. The reason for this is in case of network failures when the client application doesn’t know if the data was written to the DB, so the `INSERT` query can simply be repeated. It doesn’t matter which replica INSERTs were sent to with identical data. `INSERTs` are idempotent. Deduplication parameters are controlled by [merge_tree](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-merge_tree) server settings. During replication, only the source data to insert is transferred over the network. Further data transformation (merging) is coordinated and performed on all the replicas in the same way. This minimizes network usage, which means that replication works well when replicas reside in different datacenters. (Note that duplicating data in different datacenters is the main goal of replication.) @@ -217,6 +217,6 @@ If the data in ZooKeeper was lost or damaged, you can save data by moving it to **See also** -- [background\_schedule\_pool\_size](../../../operations/settings/settings.md#background_schedule_pool_size) +- [background_schedule_pool_size](../../../operations/settings/settings.md#background_schedule_pool_size) [Original article](https://clickhouse.tech/docs/en/operations/table_engines/replication/) diff --git a/docs/en/engines/table-engines/special/distributed.md b/docs/en/engines/table-engines/special/distributed.md index b1d741e9e13..9f96ca3fe8c 100644 --- a/docs/en/engines/table-engines/special/distributed.md +++ b/docs/en/engines/table-engines/special/distributed.md @@ -97,13 +97,13 @@ Cluster names must not contain dots. The parameters `host`, `port`, and optionally `user`, `password`, `secure`, `compression` are specified for each server: - `host` – The address of the remote server. You can use either the domain or the IPv4 or IPv6 address. If you specify the domain, the server makes a DNS request when it starts, and the result is stored as long as the server is running. If the DNS request fails, the server doesn’t start. If you change the DNS record, restart the server. -- `port` – The TCP port for messenger activity (`tcp_port` in the config, usually set to 9000). Do not confuse it with http\_port. +- `port` – The TCP port for messenger activity (`tcp_port` in the config, usually set to 9000). Do not confuse it with http_port. - `user` – Name of the user for connecting to a remote server. Default value: default. This user must have access to connect to the specified server. Access is configured in the users.xml file. For more information, see the section [Access rights](../../../operations/access-rights.md). - `password` – The password for connecting to a remote server (not masked). Default value: empty string. - `secure` - Use ssl for connection, usually you also should define `port` = 9440. Server should listen on `9440` and have correct certificates. - `compression` - Use data compression. Default value: true. -When specifying replicas, one of the available replicas will be selected for each of the shards when reading. You can configure the algorithm for load balancing (the preference for which replica to access) – see the [load\_balancing](../../../operations/settings/settings.md#settings-load_balancing) setting. +When specifying replicas, one of the available replicas will be selected for each of the shards when reading. You can configure the algorithm for load balancing (the preference for which replica to access) – see the [load_balancing](../../../operations/settings/settings.md#settings-load_balancing) setting. If the connection with the server is not established, there will be an attempt to connect with a short timeout. If the connection failed, the next replica will be selected, and so on for all the replicas. If the connection attempt failed for all the replicas, the attempt will be repeated the same way, several times. This works in favour of resiliency, but does not provide complete fault tolerance: a remote server might accept the connection, but might not work, or work poorly. @@ -144,11 +144,11 @@ You should be concerned about the sharding scheme in the following cases: - Queries are used that require joining data (IN or JOIN) by a specific key. If data is sharded by this key, you can use local IN or JOIN instead of GLOBAL IN or GLOBAL JOIN, which is much more efficient. - A large number of servers is used (hundreds or more) with a large number of small queries (queries of individual clients - websites, advertisers, or partners). In order for the small queries to not affect the entire cluster, it makes sense to locate data for a single client on a single shard. Alternatively, as we’ve done in Yandex.Metrica, you can set up bi-level sharding: divide the entire cluster into “layers”, where a layer may consist of multiple shards. Data for a single client is located on a single layer, but shards can be added to a layer as necessary, and data is randomly distributed within them. Distributed tables are created for each layer, and a single shared distributed table is created for global queries. -Data is written asynchronously. When inserted in the table, the data block is just written to the local file system. The data is sent to the remote servers in the background as soon as possible. The period for sending data is managed by the [distributed\_directory\_monitor\_sleep\_time\_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms) and [distributed\_directory\_monitor\_max\_sleep\_time\_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms) settings. The `Distributed` engine sends each file with inserted data separately, but you can enable batch sending of files with the [distributed\_directory\_monitor\_batch\_inserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts) setting. This setting improves cluster performance by better utilizing local server and network resources. You should check whether data is sent successfully by checking the list of files (data waiting to be sent) in the table directory: `/var/lib/clickhouse/data/database/table/`. The number of threads performing background tasks can be set by [background\_distributed\_schedule\_pool\_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size) setting. +Data is written asynchronously. When inserted in the table, the data block is just written to the local file system. The data is sent to the remote servers in the background as soon as possible. The period for sending data is managed by the [distributed_directory_monitor_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms) and [distributed_directory_monitor_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms) settings. The `Distributed` engine sends each file with inserted data separately, but you can enable batch sending of files with the [distributed_directory_monitor_batch_inserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts) setting. This setting improves cluster performance by better utilizing local server and network resources. You should check whether data is sent successfully by checking the list of files (data waiting to be sent) in the table directory: `/var/lib/clickhouse/data/database/table/`. The number of threads performing background tasks can be set by [background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size) setting. If the server ceased to exist or had a rough restart (for example, after a device failure) after an INSERT to a Distributed table, the inserted data might be lost. If a damaged data part is detected in the table directory, it is transferred to the `broken` subdirectory and no longer used. -When the `max_parallel_replicas` option is enabled, query processing is parallelized across all replicas within a single shard. For more information, see the section [max\_parallel\_replicas](../../../operations/settings/settings.md#settings-max_parallel_replicas). +When the `max_parallel_replicas` option is enabled, query processing is parallelized across all replicas within a single shard. For more information, see the section [max_parallel_replicas](../../../operations/settings/settings.md#settings-max_parallel_replicas). ## Virtual Columns {#virtual-columns} @@ -160,6 +160,6 @@ When the `max_parallel_replicas` option is enabled, query processing is parallel **See Also** - [Virtual columns](../../../engines/table-engines/special/index.md#table_engines-virtual_columns) -- [background\_distributed\_schedule\_pool\_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size) +- [background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size) [Original article](https://clickhouse.tech/docs/en/operations/table_engines/distributed/) diff --git a/docs/en/engines/table-engines/special/external-data.md b/docs/en/engines/table-engines/special/external-data.md index a2f6c076196..88d76b3805e 100644 --- a/docs/en/engines/table-engines/special/external-data.md +++ b/docs/en/engines/table-engines/special/external-data.md @@ -25,10 +25,10 @@ You may have multiple sections like this, for the number of tables being transmi **–file** – Path to the file with the table dump, or -, which refers to stdin. Only a single table can be retrieved from stdin. -The following parameters are optional: **–name**– Name of the table. If omitted, \_data is used. +The following parameters are optional: **–name**– Name of the table. If omitted, _data is used. **–format** – Data format in the file. If omitted, TabSeparated is used. -One of the following parameters is required:**–types** – A list of comma-separated column types. For example: `UInt64,String`. The columns will be named \_1, \_2, … +One of the following parameters is required:**–types** – A list of comma-separated column types. For example: `UInt64,String`. The columns will be named _1, _2, … **–structure**– The table structure in the format`UserID UInt64`, `URL String`. Defines the column names and types. The files specified in ‘file’ will be parsed by the format specified in ‘format’, using the data types specified in ‘types’ or ‘structure’. The table will be uploaded to the server and accessible there as a temporary table with the name in ‘name’. diff --git a/docs/en/engines/table-engines/special/join.md b/docs/en/engines/table-engines/special/join.md index 0572627473e..2066abec6a8 100644 --- a/docs/en/engines/table-engines/special/join.md +++ b/docs/en/engines/table-engines/special/join.md @@ -93,15 +93,15 @@ You cannot perform a `SELECT` query directly from the table. Instead, use one of When creating a table, the following settings are applied: -- [join\_use\_nulls](../../../operations/settings/settings.md#join_use_nulls) -- [max\_rows\_in\_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) -- [max\_bytes\_in\_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) -- [join\_overflow\_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) -- [join\_any\_take\_last\_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) +- [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) +- [max_rows_in_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) +- [max_bytes_in_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) +- [join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) +- [join_any_take_last_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) The `Join`-engine tables can’t be used in `GLOBAL JOIN` operations. -The `Join`-engine allows use [join\_use\_nulls](../../../operations/settings/settings.md#join_use_nulls) setting in the `CREATE TABLE` statement. And [SELECT](../../../sql-reference/statements/select/index.md) query allows use `join_use_nulls` too. If you have different `join_use_nulls` settings, you can get an error joining table. It depends on kind of JOIN. When you use [joinGet](../../../sql-reference/functions/other-functions.md#joinget) function, you have to use the same `join_use_nulls` setting in `CRATE TABLE` and `SELECT` statements. +The `Join`-engine allows use [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) setting in the `CREATE TABLE` statement. And [SELECT](../../../sql-reference/statements/select/index.md) query allows use `join_use_nulls` too. If you have different `join_use_nulls` settings, you can get an error joining table. It depends on kind of JOIN. When you use [joinGet](../../../sql-reference/functions/other-functions.md#joinget) function, you have to use the same `join_use_nulls` setting in `CRATE TABLE` and `SELECT` statements. ## Data Storage {#data-storage} diff --git a/docs/en/engines/table-engines/special/merge.md b/docs/en/engines/table-engines/special/merge.md index 53615ce07ec..07cb778c6e9 100644 --- a/docs/en/engines/table-engines/special/merge.md +++ b/docs/en/engines/table-engines/special/merge.md @@ -33,7 +33,7 @@ The typical way to use the `Merge` engine is for working with a large number of Example 2: -Let’s say you have a old table (WatchLog\_old) and decided to change partitioning without moving data to a new table (WatchLog\_new) and you need to see data from both tables. +Let’s say you have a old table (WatchLog_old) and decided to change partitioning without moving data to a new table (WatchLog_new) and you need to see data from both tables. ``` sql CREATE TABLE WatchLog_old(date Date, UserId Int64, EventType String, Cnt UInt64) diff --git a/docs/en/engines/table-engines/special/url.md b/docs/en/engines/table-engines/special/url.md index 0e22bfb7008..d3f577f8f4f 100644 --- a/docs/en/engines/table-engines/special/url.md +++ b/docs/en/engines/table-engines/special/url.md @@ -23,7 +23,7 @@ additional headers for getting a response from the server. respectively. For processing `POST` requests, the remote server must support [Chunked transfer encoding](https://en.wikipedia.org/wiki/Chunked_transfer_encoding). -You can limit the maximum number of HTTP GET redirect hops using the [max\_http\_get\_redirects](../../../operations/settings/settings.md#setting-max_http_get_redirects) setting. +You can limit the maximum number of HTTP GET redirect hops using the [max_http_get_redirects](../../../operations/settings/settings.md#setting-max_http_get_redirects) setting. ## Example {#example} diff --git a/docs/en/faq/index.md b/docs/en/faq/index.md index eb426f47cf7..1ae71cf680c 100644 --- a/docs/en/faq/index.md +++ b/docs/en/faq/index.md @@ -40,7 +40,7 @@ Question candidates: - How to implement pivot (like in pandas)? - How to remove the default ClickHouse user through users.d? - Importing MySQL dump to Clickhouse -- Window function workarounds (row\_number, lag/lead, running diff/sum/average) +- Window function workarounds (row_number, lag/lead, running diff/sum/average) ##} {## [Original article](https://clickhouse.tech/docs/en/faq) ##} diff --git a/docs/en/faq/integration/oracle-odbc.md b/docs/en/faq/integration/oracle-odbc.md index 490c622cd03..91265a3daa2 100644 --- a/docs/en/faq/integration/oracle-odbc.md +++ b/docs/en/faq/integration/oracle-odbc.md @@ -6,7 +6,7 @@ toc_priority: 20 # What If I Have a Problem with Encodings When Using Oracle Via ODBC? {#oracle-odbc-encodings} -If you use Oracle as a source of ClickHouse external dictionaries via Oracle ODBC driver, you need to set the correct value for the `NLS_LANG` environment variable in `/etc/default/clickhouse`. For more information, see the [Oracle NLS\_LANG FAQ](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html). +If you use Oracle as a source of ClickHouse external dictionaries via Oracle ODBC driver, you need to set the correct value for the `NLS_LANG` environment variable in `/etc/default/clickhouse`. For more information, see the [Oracle NLS_LANG FAQ](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html). **Example** diff --git a/docs/en/getting-started/example-datasets/amplab-benchmark.md b/docs/en/getting-started/example-datasets/amplab-benchmark.md index 95fe444a897..1b740f6194c 100644 --- a/docs/en/getting-started/example-datasets/amplab-benchmark.md +++ b/docs/en/getting-started/example-datasets/amplab-benchmark.md @@ -7,7 +7,7 @@ toc_title: AMPLab Big Data Benchmark See https://amplab.cs.berkeley.edu/benchmark/ -Sign up for a free account at https://aws.amazon.com. It requires a credit card, email, and phone number. Get a new access key at https://console.aws.amazon.com/iam/home?nc2=h\_m\_sc\#security\_credential +Sign up for a free account at https://aws.amazon.com. It requires a credit card, email, and phone number. Get a new access key at https://console.aws.amazon.com/iam/home?nc2=h_m_sc#security_credential Run the following in the console: diff --git a/docs/en/getting-started/example-datasets/nyc-taxi.md b/docs/en/getting-started/example-datasets/nyc-taxi.md index 4a069e247b3..a180f19be8a 100644 --- a/docs/en/getting-started/example-datasets/nyc-taxi.md +++ b/docs/en/getting-started/example-datasets/nyc-taxi.md @@ -195,7 +195,7 @@ The data in this table uses 142 GB. (Importing data directly from Postgres is also possible using `COPY ... TO PROGRAM`.) -Unfortunately, all the fields associated with the weather (precipitation…average\_wind\_speed) were filled with NULL. Because of this, we will remove them from the final data set. +Unfortunately, all the fields associated with the weather (precipitation…average_wind_speed) were filled with NULL. Because of this, we will remove them from the final data set. To start, we’ll create a table on a single server. Later we will make the table distributed. diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index dcecb3c3510..6600263f026 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -7,9 +7,9 @@ toc_title: Installation ## System Requirements {#system-requirements} -ClickHouse can run on any Linux, FreeBSD, or Mac OS X with x86\_64, AArch64, or PowerPC64LE CPU architecture. +ClickHouse can run on any Linux, FreeBSD, or Mac OS X with x86_64, AArch64, or PowerPC64LE CPU architecture. -Official pre-built binaries are typically compiled for x86\_64 and leverage SSE 4.2 instruction set, so unless otherwise stated usage of CPU that supports it becomes an additional system requirement. Here’s the command to check if current CPU has support for SSE 4.2: +Official pre-built binaries are typically compiled for x86_64 and leverage SSE 4.2 instruction set, so unless otherwise stated usage of CPU that supports it becomes an additional system requirement. Here’s the command to check if current CPU has support for SSE 4.2: ``` bash $ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not supported" diff --git a/docs/en/getting-started/playground.md b/docs/en/getting-started/playground.md index 26fb105099b..7838dad14ea 100644 --- a/docs/en/getting-started/playground.md +++ b/docs/en/getting-started/playground.md @@ -38,10 +38,10 @@ The queries are executed as a read-only user. It implies some limitations: The following settings are also enforced: -- [max\_result\_bytes=10485760](../operations/settings/query_complexity/#max-result-bytes) -- [max\_result\_rows=2000](../operations/settings/query_complexity/#setting-max_result_rows) -- [result\_overflow\_mode=break](../operations/settings/query_complexity/#result-overflow-mode) -- [max\_execution\_time=60000](../operations/settings/query_complexity/#max-execution-time) +- [max_result_bytes=10485760](../operations/settings/query_complexity/#max-result-bytes) +- [max_result_rows=2000](../operations/settings/query_complexity/#setting-max_result_rows) +- [result_overflow_mode=break](../operations/settings/query_complexity/#result-overflow-mode) +- [max_execution_time=60000](../operations/settings/query_complexity/#max-execution-time) ## Examples {#examples} diff --git a/docs/en/index.md b/docs/en/index.md index fb7e1eb67d9..8280d5c9f97 100644 --- a/docs/en/index.md +++ b/docs/en/index.md @@ -11,10 +11,10 @@ In a “normal” row-oriented DBMS, data is stored in this order: | Row | WatchID | JavaEnable | Title | GoodEvent | EventTime | |-----|-------------|------------|--------------------|-----------|---------------------| -| \#0 | 89354350662 | 1 | Investor Relations | 1 | 2016-05-18 05:19:20 | -| \#1 | 90329509958 | 0 | Contact us | 1 | 2016-05-18 08:10:20 | -| \#2 | 89953706054 | 1 | Mission | 1 | 2016-05-18 07:38:00 | -| \#N | … | … | … | … | … | +| #0 | 89354350662 | 1 | Investor Relations | 1 | 2016-05-18 05:19:20 | +| #1 | 90329509958 | 0 | Contact us | 1 | 2016-05-18 08:10:20 | +| #2 | 89953706054 | 1 | Mission | 1 | 2016-05-18 07:38:00 | +| #N | … | … | … | … | … | In other words, all the values related to a row are physically stored next to each other. @@ -22,7 +22,7 @@ Examples of a row-oriented DBMS are MySQL, Postgres, and MS SQL Server. In a column-oriented DBMS, data is stored like this: -| Row: | \#0 | \#1 | \#2 | \#N | +| Row: | #0 | #1 | #2 | #N | |-------------|---------------------|---------------------|---------------------|-----| | WatchID: | 89354350662 | 90329509958 | 89953706054 | … | | JavaEnable: | 1 | 0 | 1 | … | diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index bfe5b6218e4..96e20777bdb 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -210,7 +210,7 @@ Setting `format_template_resultset` specifies the path to file, which contains a - `min` is the row with minimum values in `format_template_row` format (when extremes are set to 1) - `max` is the row with maximum values in `format_template_row` format (when extremes are set to 1) - `rows` is the total number of output rows -- `rows_before_limit` is the minimal number of rows there would have been without LIMIT. Output only if the query contains LIMIT. If the query contains GROUP BY, rows\_before\_limit\_at\_least is the exact number of rows there would have been without a LIMIT. +- `rows_before_limit` is the minimal number of rows there would have been without LIMIT. Output only if the query contains LIMIT. If the query contains GROUP BY, rows_before_limit_at_least is the exact number of rows there would have been without a LIMIT. - `time` is the request execution time in seconds - `rows_read` is the number of rows has been read - `bytes_read` is the number of bytes (uncompressed) has been read @@ -361,21 +361,21 @@ Parsing allows the presence of the additional field `tskv` without the equal sig Comma Separated Values format ([RFC](https://tools.ietf.org/html/rfc4180)). -When formatting, rows are enclosed in double-quotes. A double quote inside a string is output as two double quotes in a row. There are no other rules for escaping characters. Date and date-time are enclosed in double-quotes. Numbers are output without quotes. Values are separated by a delimiter character, which is `,` by default. The delimiter character is defined in the setting [format\_csv\_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter). Rows are separated using the Unix line feed (LF). Arrays are serialized in CSV as follows: first, the array is serialized to a string as in TabSeparated format, and then the resulting string is output to CSV in double-quotes. Tuples in CSV format are serialized as separate columns (that is, their nesting in the tuple is lost). +When formatting, rows are enclosed in double-quotes. A double quote inside a string is output as two double quotes in a row. There are no other rules for escaping characters. Date and date-time are enclosed in double-quotes. Numbers are output without quotes. Values are separated by a delimiter character, which is `,` by default. The delimiter character is defined in the setting [format_csv_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter). Rows are separated using the Unix line feed (LF). Arrays are serialized in CSV as follows: first, the array is serialized to a string as in TabSeparated format, and then the resulting string is output to CSV in double-quotes. Tuples in CSV format are serialized as separate columns (that is, their nesting in the tuple is lost). ``` bash $ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV" < data.csv ``` -\*By default, the delimiter is `,`. See the [format\_csv\_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter) setting for more information. +\*By default, the delimiter is `,`. See the [format_csv_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter) setting for more information. When parsing, all values can be parsed either with or without quotes. Both double and single quotes are supported. Rows can also be arranged without quotes. In this case, they are parsed up to the delimiter character or line feed (CR or LF). In violation of the RFC, when parsing rows without quotes, the leading and trailing spaces and tabs are ignored. For the line feed, Unix (LF), Windows (CR LF) and Mac OS Classic (CR LF) types are all supported. Empty unquoted input values are replaced with default values for the respective columns, if -[input\_format\_defaults\_for\_omitted\_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) +[input_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) is enabled. -`NULL` is formatted as `\N` or `NULL` or an empty unquoted string (see settings [input\_format\_csv\_unquoted\_null\_literal\_as\_null](../operations/settings/settings.md#settings-input_format_csv_unquoted_null_literal_as_null) and [input\_format\_defaults\_for\_omitted\_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields)). +`NULL` is formatted as `\N` or `NULL` or an empty unquoted string (see settings [input_format_csv_unquoted_null_literal_as_null](../operations/settings/settings.md#settings-input_format_csv_unquoted_null_literal_as_null) and [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields)). The CSV format supports the output of totals and extremes the same way as `TabSeparated`. @@ -439,12 +439,12 @@ SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase WITH TOTA } ``` -The JSON is compatible with JavaScript. To ensure this, some characters are additionally escaped: the slash `/` is escaped as `\/`; alternative line breaks `U+2028` and `U+2029`, which break some browsers, are escaped as `\uXXXX`. ASCII control characters are escaped: backspace, form feed, line feed, carriage return, and horizontal tab are replaced with `\b`, `\f`, `\n`, `\r`, `\t` , as well as the remaining bytes in the 00-1F range using `\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) to 0. +The JSON is compatible with JavaScript. To ensure this, some characters are additionally escaped: the slash `/` is escaped as `\/`; alternative line breaks `U+2028` and `U+2029`, which break some browsers, are escaped as `\uXXXX`. ASCII control characters are escaped: backspace, form feed, line feed, carriage return, and horizontal tab are replaced with `\b`, `\f`, `\n`, `\r`, `\t` , as well as the remaining bytes in the 00-1F range using `\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) to 0. `rows` – The total number of output rows. `rows_before_limit_at_least` The minimal number of rows there would have been without LIMIT. Output only if the query contains LIMIT. -If the query contains GROUP BY, rows\_before\_limit\_at\_least is the exact number of rows there would have been without a LIMIT. +If the query contains GROUP BY, rows_before_limit_at_least is the exact number of rows there would have been without a LIMIT. `totals` – Total values (when using WITH TOTALS). @@ -452,7 +452,7 @@ If the query contains GROUP BY, rows\_before\_limit\_at\_least is the exact numb This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). -ClickHouse supports [NULL](../sql-reference/syntax.md), which is displayed as `null` in the JSON output. To enable `+nan`, `-nan`, `+inf`, `-inf` values in output, set the [output\_format\_json\_quote\_denormals](../operations/settings/settings.md#settings-output_format_json_quote_denormals) to 1. +ClickHouse supports [NULL](../sql-reference/syntax.md), which is displayed as `null` in the JSON output. To enable `+nan`, `-nan`, `+inf`, `-inf` values in output, set the [output_format_json_quote_denormals](../operations/settings/settings.md#settings-output_format_json_quote_denormals) to 1. See also the [JSONEachRow](#jsoneachrow) format. @@ -633,7 +633,7 @@ ClickHouse ignores spaces between elements and commas after the objects. You can ClickHouse substitutes omitted values with the default values for the corresponding [data types](../sql-reference/data-types/index.md). -If `DEFAULT expr` is specified, ClickHouse uses different substitution rules depending on the [input\_format\_defaults\_for\_omitted\_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) setting. +If `DEFAULT expr` is specified, ClickHouse uses different substitution rules depending on the [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) setting. Consider the following table: @@ -676,7 +676,7 @@ Unlike the [JSON](#json) format, there is no substitution of invalid UTF-8 seque ### Usage of Nested Structures {#jsoneachrow-nested} -If you have a table with [Nested](../sql-reference/data-types/nested-data-structures/nested.md) data type columns, you can insert JSON data with the same structure. Enable this feature with the [input\_format\_import\_nested\_json](../operations/settings/settings.md#settings-input_format_import_nested_json) setting. +If you have a table with [Nested](../sql-reference/data-types/nested-data-structures/nested.md) data type columns, you can insert JSON data with the same structure. Enable this feature with the [input_format_import_nested_json](../operations/settings/settings.md#settings-input_format_import_nested_json) setting. For example, consider the following table: @@ -690,7 +690,7 @@ As you can see in the `Nested` data type description, ClickHouse treats each com INSERT INTO json_each_row_nested FORMAT JSONEachRow {"n.s": ["abc", "def"], "n.i": [1, 23]} ``` -To insert data as a hierarchical JSON object, set [input\_format\_import\_nested\_json=1](../operations/settings/settings.md#settings-input_format_import_nested_json). +To insert data as a hierarchical JSON object, set [input_format_import_nested_json=1](../operations/settings/settings.md#settings-input_format_import_nested_json). ``` json { @@ -872,7 +872,7 @@ The minimum set of characters that you need to escape when passing data in Value This is the format that is used in `INSERT INTO t VALUES ...`, but you can also use it for formatting query results. -See also: [input\_format\_values\_interpret\_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions) and [input\_format\_values\_deduce\_templates\_of\_expressions](../operations/settings/settings.md#settings-input_format_values_deduce_templates_of_expressions) settings. +See also: [input_format_values_interpret_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions) and [input_format_values_deduce_templates_of_expressions](../operations/settings/settings.md#settings-input_format_values_deduce_templates_of_expressions) settings. ## Vertical {#vertical} @@ -1135,7 +1135,7 @@ Column names must: - start with `[A-Za-z_]` - subsequently contain only `[A-Za-z0-9_]` -Output Avro file compression and sync interval can be configured with [output\_format\_avro\_codec](../operations/settings/settings.md#settings-output_format_avro_codec) and [output\_format\_avro\_sync\_interval](../operations/settings/settings.md#settings-output_format_avro_sync_interval) respectively. +Output Avro file compression and sync interval can be configured with [output_format_avro_codec](../operations/settings/settings.md#settings-output_format_avro_codec) and [output_format_avro_sync_interval](../operations/settings/settings.md#settings-output_format_avro_sync_interval) respectively. ## AvroConfluent {#data-format-avro-confluent} @@ -1145,7 +1145,7 @@ Each Avro message embeds a schema id that can be resolved to the actual schema w Schemas are cached once resolved. -Schema Registry URL is configured with [format\_avro\_schema\_registry\_url](../operations/settings/settings.md#format_avro_schema_registry_url). +Schema Registry URL is configured with [format_avro_schema_registry_url](../operations/settings/settings.md#format_avro_schema_registry_url). ### Data Types Matching {#data_types-matching-1} @@ -1298,13 +1298,13 @@ can contain an absolute path or a path relative to the current directory on the If you use the client in the [batch mode](../interfaces/cli.md#cli_usage), the path to the schema must be relative due to security reasons. If you input or output data via the [HTTP interface](../interfaces/http.md) the file name specified in the format schema -should be located in the directory specified in [format\_schema\_path](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-format_schema_path) +should be located in the directory specified in [format_schema_path](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-format_schema_path) in the server configuration. ## Skipping Errors {#skippingerrors} -Some formats such as `CSV`, `TabSeparated`, `TSKV`, `JSONEachRow`, `Template`, `CustomSeparated` and `Protobuf` can skip broken row if parsing error occurred and continue parsing from the beginning of next row. See [input\_format\_allow\_errors\_num](../operations/settings/settings.md#settings-input_format_allow_errors_num) and -[input\_format\_allow\_errors\_ratio](../operations/settings/settings.md#settings-input_format_allow_errors_ratio) settings. +Some formats such as `CSV`, `TabSeparated`, `TSKV`, `JSONEachRow`, `Template`, `CustomSeparated` and `Protobuf` can skip broken row if parsing error occurred and continue parsing from the beginning of next row. See [input_format_allow_errors_num](../operations/settings/settings.md#settings-input_format_allow_errors_num) and +[input_format_allow_errors_ratio](../operations/settings/settings.md#settings-input_format_allow_errors_ratio) settings. Limitations: - In case of parsing error `JSONEachRow` skips all data until the new line (or EOF), so rows must be delimited by `\n` to count errors correctly. - `Template` and `CustomSeparated` use delimiter after the last column and delimiter between rows to find the beginning of next row, so skipping errors works only if at least one of them is not empty. diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 35c79b5ee02..4840737fd48 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -9,7 +9,7 @@ The HTTP interface lets you use ClickHouse on any platform from any programming By default, clickhouse-server listens for HTTP on port 8123 (this can be changed in the config). -If you make a GET / request without parameters, it returns 200 response code and the string which defined in [http\_server\_default\_response](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-http_server_default_response) default value “Ok.” (with a line feed at the end) +If you make a GET / request without parameters, it returns 200 response code and the string which defined in [http_server_default_response](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-http_server_default_response) default value “Ok.” (with a line feed at the end) ``` bash $ curl 'http://localhost:8123/' @@ -148,12 +148,12 @@ $ echo 'DROP TABLE t' | curl 'http://localhost:8123/' --data-binary @- For successful requests that don’t return a data table, an empty response body is returned. -You can use the internal ClickHouse compression format when transmitting data. The compressed data has a non-standard format, and you will need to use the special `clickhouse-compressor` program to work with it (it is installed with the `clickhouse-client` package). To increase the efficiency of data insertion, you can disable server-side checksum verification by using the [http\_native\_compression\_disable\_checksumming\_on\_decompress](../operations/settings/settings.md#settings-http_native_compression_disable_checksumming_on_decompress) setting. +You can use the internal ClickHouse compression format when transmitting data. The compressed data has a non-standard format, and you will need to use the special `clickhouse-compressor` program to work with it (it is installed with the `clickhouse-client` package). To increase the efficiency of data insertion, you can disable server-side checksum verification by using the [http_native_compression_disable_checksumming_on_decompress](../operations/settings/settings.md#settings-http_native_compression_disable_checksumming_on_decompress) setting. If you specified `compress=1` in the URL, the server compresses the data it sends you. If you specified `decompress=1` in the URL, the server decompresses the same data that you pass in the `POST` method. -You can also choose to use [HTTP compression](https://en.wikipedia.org/wiki/HTTP_compression). To send a compressed `POST` request, append the request header `Content-Encoding: compression_method`. In order for ClickHouse to compress the response, you must append `Accept-Encoding: compression_method`. ClickHouse supports `gzip`, `br`, and `deflate` [compression methods](https://en.wikipedia.org/wiki/HTTP_compression#Content-Encoding_tokens). To enable HTTP compression, you must use the ClickHouse [enable\_http\_compression](../operations/settings/settings.md#settings-enable_http_compression) setting. You can configure the data compression level in the [http\_zlib\_compression\_level](#settings-http_zlib_compression_level) setting for all the compression methods. +You can also choose to use [HTTP compression](https://en.wikipedia.org/wiki/HTTP_compression). To send a compressed `POST` request, append the request header `Content-Encoding: compression_method`. In order for ClickHouse to compress the response, you must append `Accept-Encoding: compression_method`. ClickHouse supports `gzip`, `br`, and `deflate` [compression methods](https://en.wikipedia.org/wiki/HTTP_compression#Content-Encoding_tokens). To enable HTTP compression, you must use the ClickHouse [enable_http_compression](../operations/settings/settings.md#settings-enable_http_compression) setting. You can configure the data compression level in the [http_zlib_compression_level](#settings-http_zlib_compression_level) setting for all the compression methods. You can use this to reduce network traffic when transmitting a large amount of data, or for creating dumps that are immediately compressed. @@ -215,7 +215,7 @@ $ echo 'SELECT 1' | curl -H 'X-ClickHouse-User: user' -H 'X-ClickHouse-Key: pass ``` If the user name is not specified, the `default` name is used. If the password is not specified, the empty password is used. -You can also use the URL parameters to specify any settings for processing a single query or entire profiles of settings. Example:http://localhost:8123/?profile=web&max\_rows\_to\_read=1000000000&query=SELECT+1 +You can also use the URL parameters to specify any settings for processing a single query or entire profiles of settings. Example:http://localhost:8123/?profile=web&max_rows_to_read=1000000000&query=SELECT+1 For more information, see the [Settings](../operations/settings/index.md) section. @@ -237,7 +237,7 @@ For information about other parameters, see the section “SET”. Similarly, you can use ClickHouse sessions in the HTTP protocol. To do this, you need to add the `session_id` GET parameter to the request. You can use any string as the session ID. By default, the session is terminated after 60 seconds of inactivity. To change this timeout, modify the `default_session_timeout` setting in the server configuration, or add the `session_timeout` GET parameter to the request. To check the session status, use the `session_check=1` parameter. Only one query at a time can be executed within a single session. -You can receive information about the progress of a query in `X-ClickHouse-Progress` response headers. To do this, enable [send\_progress\_in\_http\_headers](../operations/settings/settings.md#settings-send_progress_in_http_headers). Example of the header sequence: +You can receive information about the progress of a query in `X-ClickHouse-Progress` response headers. To do this, enable [send_progress_in_http_headers](../operations/settings/settings.md#settings-send_progress_in_http_headers). Example of the header sequence: ``` text X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128"} @@ -254,9 +254,9 @@ Possible header fields: - `written_bytes` — Volume of data written in bytes. Running requests don’t stop automatically if the HTTP connection is lost. Parsing and data formatting are performed on the server-side, and using the network might be ineffective. -The optional ‘query\_id’ parameter can be passed as the query ID (any string). For more information, see the section “Settings, replace\_running\_query”. +The optional ‘query_id’ parameter can be passed as the query ID (any string). For more information, see the section “Settings, replace_running_query”. -The optional ‘quota\_key’ parameter can be passed as the quota key (any string). For more information, see the section “Quotas”. +The optional ‘quota_key’ parameter can be passed as the quota key (any string). For more information, see the section “Quotas”. The HTTP interface allows passing external data (external temporary tables) for querying. For more information, see the section “External data for query processing”. @@ -377,7 +377,7 @@ Now `rule` can configure `method`, `headers`, `url`, `handler`: - `headers` are responsible for matching the header part of the HTTP request. It is compatible with RE2’s regular expressions. It is an optional configuration. If it is not defined in the configuration file, it does not match the header portion of the HTTP request. - `handler` contains the main processing part. Now `handler` can configure `type`, `status`, `content_type`, `response_content`, `query`, `query_param_name`. - `type` currently supports three types: [predefined\_query\_handler](#predefined_query_handler), [dynamic\_query\_handler](#dynamic_query_handler), [static](#static). + `type` currently supports three types: [predefined_query_handler](#predefined_query_handler), [dynamic_query_handler](#dynamic_query_handler), [static](#static). - `query` — use with `predefined_query_handler` type, executes query when the handler is called. @@ -391,13 +391,13 @@ Now `rule` can configure `method`, `headers`, `url`, `handler`: Next are the configuration methods for different `type`. -### predefined\_query\_handler {#predefined_query_handler} +### predefined_query_handler {#predefined_query_handler} `predefined_query_handler` supports setting `Settings` and `query_params` values. You can configure `query` in the type of `predefined_query_handler`. `query` value is a predefined query of `predefined_query_handler`, which is executed by ClickHouse when an HTTP request is matched and the result of the query is returned. It is a must configuration. -The following example defines the values of [max\_threads](../operations/settings/settings.md#settings-max_threads) and `max_alter_threads` settings, then queries the system table to check whether these settings were set successfully. +The following example defines the values of [max_threads](../operations/settings/settings.md#settings-max_threads) and `max_alter_threads` settings, then queries the system table to check whether these settings were set successfully. Example: @@ -428,13 +428,13 @@ max_alter_threads 2 !!! note "caution" In one `predefined_query_handler` only supports one `query` of an insert type. -### dynamic\_query\_handler {#dynamic_query_handler} +### dynamic_query_handler {#dynamic_query_handler} In `dynamic_query_handler`, the query is written in the form of param of the HTTP request. The difference is that in `predefined_query_handler`, the query is written in the configuration file. You can configure `query_param_name` in `dynamic_query_handler`. ClickHouse extracts and executes the value corresponding to the `query_param_name` value in the URL of the HTTP request. The default value of `query_param_name` is `/query` . It is an optional configuration. If there is no definition in the configuration file, the param is not passed in. -To experiment with this functionality, the example defines the values of [max\_threads](../operations/settings/settings.md#settings-max_threads) and `max_alter_threads` and `queries` whether the settings were set successfully. +To experiment with this functionality, the example defines the values of [max_threads](../operations/settings/settings.md#settings-max_threads) and `max_alter_threads` and `queries` whether the settings were set successfully. Example: @@ -459,7 +459,7 @@ max_alter_threads 2 ### static {#static} -`static` can return [content\_type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type), [status](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status) and `response_content`. `response_content` can return the specified content. +`static` can return [content_type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type), [status](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status) and `response_content`. `response_content` can return the specified content. Example: diff --git a/docs/en/interfaces/mysql.md b/docs/en/interfaces/mysql.md index d4f19630c10..ee9af925671 100644 --- a/docs/en/interfaces/mysql.md +++ b/docs/en/interfaces/mysql.md @@ -5,7 +5,7 @@ toc_title: MySQL Interface # MySQL Interface {#mysql-interface} -ClickHouse supports MySQL wire protocol. It can be enabled by [mysql\_port](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-mysql_port) setting in configuration file: +ClickHouse supports MySQL wire protocol. It can be enabled by [mysql_port](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-mysql_port) setting in configuration file: ``` xml 9004 diff --git a/docs/en/interfaces/third-party/client-libraries.md b/docs/en/interfaces/third-party/client-libraries.md index 128c3009f5e..bb98c5f936b 100644 --- a/docs/en/interfaces/third-party/client-libraries.md +++ b/docs/en/interfaces/third-party/client-libraries.md @@ -9,7 +9,7 @@ toc_title: Client Libraries Yandex does **not** maintain the libraries listed below and haven’t done any extensive testing to ensure their quality. - Python - - [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm) + - [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm) - [clickhouse-driver](https://github.com/mymarilyn/clickhouse-driver) - [clickhouse-client](https://github.com/yurial/clickhouse-client) - [aiochclient](https://github.com/maximdanilchenko/aiochclient) @@ -46,7 +46,7 @@ toc_title: Client Libraries - [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client) - Kotlin - [AORM](https://github.com/TanVD/AORM) -- C\# +- C# - [Octonica.ClickHouseClient](https://github.com/Octonica/ClickHouseClient) - [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net) - [ClickHouse.Client](https://github.com/DarkWanderer/ClickHouse.Client) diff --git a/docs/en/interfaces/third-party/integrations.md b/docs/en/interfaces/third-party/integrations.md index e752725d727..7588bef0230 100644 --- a/docs/en/interfaces/third-party/integrations.md +++ b/docs/en/interfaces/third-party/integrations.md @@ -17,15 +17,15 @@ toc_title: Integrations - [clickhouse-mysql-data-reader](https://github.com/Altinity/clickhouse-mysql-data-reader) - [horgh-replicator](https://github.com/larsnovikov/horgh-replicator) - [PostgreSQL](https://www.postgresql.org) - - [clickhousedb\_fdw](https://github.com/Percona-Lab/clickhousedb_fdw) - - [infi.clickhouse\_fdw](https://github.com/Infinidat/infi.clickhouse_fdw) (uses [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) + - [clickhousedb_fdw](https://github.com/Percona-Lab/clickhousedb_fdw) + - [infi.clickhouse_fdw](https://github.com/Infinidat/infi.clickhouse_fdw) (uses [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - [pg2ch](https://github.com/mkabilov/pg2ch) - - [clickhouse\_fdw](https://github.com/adjust/clickhouse_fdw) + - [clickhouse_fdw](https://github.com/adjust/clickhouse_fdw) - [MSSQL](https://en.wikipedia.org/wiki/Microsoft_SQL_Server) - [ClickHouseMigrator](https://github.com/zlzforever/ClickHouseMigrator) - Message queues - [Kafka](https://kafka.apache.org) - - [clickhouse\_sinker](https://github.com/housepower/clickhouse_sinker) (uses [Go client](https://github.com/ClickHouse/clickhouse-go/)) + - [clickhouse_sinker](https://github.com/housepower/clickhouse_sinker) (uses [Go client](https://github.com/ClickHouse/clickhouse-go/)) - [stream-loader-clickhouse](https://github.com/adform/stream-loader) - Stream processing - [Flink](https://flink.apache.org) @@ -49,12 +49,12 @@ toc_title: Integrations - [Grafana](https://grafana.com/) - [clickhouse-grafana](https://github.com/Vertamedia/clickhouse-grafana) - [Prometheus](https://prometheus.io/) - - [clickhouse\_exporter](https://github.com/f1yegor/clickhouse_exporter) + - [clickhouse_exporter](https://github.com/f1yegor/clickhouse_exporter) - [PromHouse](https://github.com/Percona-Lab/PromHouse) - - [clickhouse\_exporter](https://github.com/hot-wifi/clickhouse_exporter) (uses [Go client](https://github.com/kshvakov/clickhouse/)) + - [clickhouse_exporter](https://github.com/hot-wifi/clickhouse_exporter) (uses [Go client](https://github.com/kshvakov/clickhouse/)) - [Nagios](https://www.nagios.org/) - - [check\_clickhouse](https://github.com/exogroup/check_clickhouse/) - - [check\_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py) + - [check_clickhouse](https://github.com/exogroup/check_clickhouse/) + - [check_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py) - [Zabbix](https://www.zabbix.com) - [clickhouse-zabbix-template](https://github.com/Altinity/clickhouse-zabbix-template) - [Sematext](https://sematext.com/) @@ -74,7 +74,7 @@ toc_title: Integrations - Python - [SQLAlchemy](https://www.sqlalchemy.org) - - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (uses [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) + - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (uses [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - [pandas](https://pandas.pydata.org) - [pandahouse](https://github.com/kszucs/pandahouse) - PHP @@ -89,7 +89,7 @@ toc_title: Integrations - Scala - [Akka](https://akka.io) - [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client) -- C\# +- C# - [ADO.NET](https://docs.microsoft.com/en-us/dotnet/framework/data/adonet/ado-net-overview) - [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net) - [ClickHouse.Client](https://github.com/DarkWanderer/ClickHouse.Client) @@ -97,7 +97,7 @@ toc_title: Integrations - [ClickHouse.Net.Migrations](https://github.com/ilyabreev/ClickHouse.Net.Migrations) - Elixir - [Ecto](https://github.com/elixir-ecto/ecto) - - [clickhouse\_ecto](https://github.com/appodeal/clickhouse_ecto) + - [clickhouse_ecto](https://github.com/appodeal/clickhouse_ecto) - Ruby - [Ruby on Rails](https://rubyonrails.org/) - [activecube](https://github.com/bitquery/activecube) diff --git a/docs/en/operations/access-rights.md b/docs/en/operations/access-rights.md index 0ab5b9aa6ff..2059bb4e4ab 100644 --- a/docs/en/operations/access-rights.md +++ b/docs/en/operations/access-rights.md @@ -138,10 +138,10 @@ Management queries: - Setup a directory for configurations storage. - ClickHouse stores access entity configurations in the folder set in the [access\_control\_path](../operations/server-configuration-parameters/settings.md#access_control_path) server configuration parameter. + ClickHouse stores access entity configurations in the folder set in the [access_control_path](../operations/server-configuration-parameters/settings.md#access_control_path) server configuration parameter. - Enable SQL-driven access control and account management for at least one user account. - By default, SQL-driven access control and account management is disabled for all users. You need to configure at least one user in the `users.xml` configuration file and set the value of the [access\_management](../operations/settings/settings-users.md#access_management-user-setting) setting to 1. + By default, SQL-driven access control and account management is disabled for all users. You need to configure at least one user in the `users.xml` configuration file and set the value of the [access_management](../operations/settings/settings-users.md#access_management-user-setting) setting to 1. [Original article](https://clickhouse.tech/docs/en/operations/access_rights/) diff --git a/docs/en/operations/configuration-files.md b/docs/en/operations/configuration-files.md index 091ed3fca10..9864efd648a 100644 --- a/docs/en/operations/configuration-files.md +++ b/docs/en/operations/configuration-files.md @@ -20,7 +20,7 @@ Some settings specified in the main configuration file can be overridden in othe ## Substitution {#substitution} -The config can also define “substitutions”. If an element has the `incl` attribute, the corresponding substitution from the file will be used as the value. By default, the path to the file with substitutions is `/etc/metrika.xml`. This can be changed in the [include\_from](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-include_from) element in the server config. The substitution values are specified in `/yandex/substitution_name` elements in this file. If a substitution specified in `incl` does not exist, it is recorded in the log. To prevent ClickHouse from logging missing substitutions, specify the `optional="true"` attribute (for example, settings for [macros](../operations/server-configuration-parameters/settings.md)). +The config can also define “substitutions”. If an element has the `incl` attribute, the corresponding substitution from the file will be used as the value. By default, the path to the file with substitutions is `/etc/metrika.xml`. This can be changed in the [include_from](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-include_from) element in the server config. The substitution values are specified in `/yandex/substitution_name` elements in this file. If a substitution specified in `incl` does not exist, it is recorded in the log. To prevent ClickHouse from logging missing substitutions, specify the `optional="true"` attribute (for example, settings for [macros](../operations/server-configuration-parameters/settings.md)). Substitutions can also be performed from ZooKeeper. To do this, specify the attribute `from_zk = "/path/to/node"`. The element value is replaced with the contents of the node at `/path/to/node` in ZooKeeper. You can also put an entire XML subtree on the ZooKeeper node and it will be fully inserted into the source element. diff --git a/docs/en/operations/monitoring.md b/docs/en/operations/monitoring.md index d0d16b526ba..ffcdae16c4d 100644 --- a/docs/en/operations/monitoring.md +++ b/docs/en/operations/monitoring.md @@ -33,7 +33,7 @@ ClickHouse collects: - Different metrics of how the server uses computational resources. - Common statistics on query processing. -You can find metrics in the [system.metrics](../operations/system-tables/metrics.md#system_tables-metrics), [system.events](../operations/system-tables/events.md#system_tables-events), and [system.asynchronous\_metrics](../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) tables. +You can find metrics in the [system.metrics](../operations/system-tables/metrics.md#system_tables-metrics), [system.events](../operations/system-tables/events.md#system_tables-events), and [system.asynchronous_metrics](../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) tables. You can configure ClickHouse to export metrics to [Graphite](https://github.com/graphite-project). See the [Graphite section](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) in the ClickHouse server configuration file. Before configuring export of metrics, you should set up Graphite by following their official [guide](https://graphite.readthedocs.io/en/latest/install.html). @@ -41,4 +41,4 @@ You can configure ClickHouse to export metrics to [Prometheus](https://prometheu Additionally, you can monitor server availability through the HTTP API. Send the `HTTP GET` request to `/ping`. If the server is available, it responds with `200 OK`. -To monitor servers in a cluster configuration, you should set the [max\_replica\_delay\_for\_distributed\_queries](../operations/settings/settings.md#settings-max_replica_delay_for_distributed_queries) parameter and use the HTTP resource `/replicas_status`. A request to `/replicas_status` returns `200 OK` if the replica is available and is not delayed behind the other replicas. If a replica is delayed, it returns `503 HTTP_SERVICE_UNAVAILABLE` with information about the gap. +To monitor servers in a cluster configuration, you should set the [max_replica_delay_for_distributed_queries](../operations/settings/settings.md#settings-max_replica_delay_for_distributed_queries) parameter and use the HTTP resource `/replicas_status`. A request to `/replicas_status` returns `200 OK` if the replica is available and is not delayed behind the other replicas. If a replica is delayed, it returns `503 HTTP_SERVICE_UNAVAILABLE` with information about the gap. diff --git a/docs/en/operations/optimizing-performance/sampling-query-profiler.md b/docs/en/operations/optimizing-performance/sampling-query-profiler.md index 5a907fb3332..0c075180530 100644 --- a/docs/en/operations/optimizing-performance/sampling-query-profiler.md +++ b/docs/en/operations/optimizing-performance/sampling-query-profiler.md @@ -9,11 +9,11 @@ ClickHouse runs sampling profiler that allows analyzing query execution. Using p To use profiler: -- Setup the [trace\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) section of the server configuration. +- Setup the [trace_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) section of the server configuration. - This section configures the [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) system table containing the results of the profiler functioning. It is configured by default. Remember that data in this table is valid only for a running server. After the server restart, ClickHouse doesn’t clean up the table and all the stored virtual memory address may become invalid. + This section configures the [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) system table containing the results of the profiler functioning. It is configured by default. Remember that data in this table is valid only for a running server. After the server restart, ClickHouse doesn’t clean up the table and all the stored virtual memory address may become invalid. -- Setup the [query\_profiler\_cpu\_time\_period\_ns](../../operations/settings/settings.md#query_profiler_cpu_time_period_ns) or [query\_profiler\_real\_time\_period\_ns](../../operations/settings/settings.md#query_profiler_real_time_period_ns) settings. Both settings can be used simultaneously. +- Setup the [query_profiler_cpu_time_period_ns](../../operations/settings/settings.md#query_profiler_cpu_time_period_ns) or [query_profiler_real_time_period_ns](../../operations/settings/settings.md#query_profiler_real_time_period_ns) settings. Both settings can be used simultaneously. These settings allow you to configure profiler timers. As these are the session settings, you can get different sampling frequency for the whole server, individual users or user profiles, for your interactive session, and for each individual query. @@ -23,7 +23,7 @@ To analyze the `trace_log` system table: - Install the `clickhouse-common-static-dbg` package. See [Install from DEB Packages](../../getting-started/install.md#install-from-deb-packages). -- Allow introspection functions by the [allow\_introspection\_functions](../../operations/settings/settings.md#settings-allow_introspection_functions) setting. +- Allow introspection functions by the [allow_introspection_functions](../../operations/settings/settings.md#settings-allow_introspection_functions) setting. For security reasons, introspection functions are disabled by default. diff --git a/docs/en/operations/requirements.md b/docs/en/operations/requirements.md index e759849eaaa..a3e61b1152b 100644 --- a/docs/en/operations/requirements.md +++ b/docs/en/operations/requirements.md @@ -7,7 +7,7 @@ toc_title: Requirements ## CPU {#cpu} -For installation from prebuilt deb packages, use a CPU with x86\_64 architecture and support for SSE 4.2 instructions. To run ClickHouse with processors that do not support SSE 4.2 or have AArch64 or PowerPC64LE architecture, you should build ClickHouse from sources. +For installation from prebuilt deb packages, use a CPU with x86_64 architecture and support for SSE 4.2 instructions. To run ClickHouse with processors that do not support SSE 4.2 or have AArch64 or PowerPC64LE architecture, you should build ClickHouse from sources. ClickHouse implements parallel data processing and uses all the hardware resources available. When choosing a processor, take into account that ClickHouse works more efficiently at configurations with a large number of cores but a lower clock rate than at configurations with fewer cores and a higher clock rate. For example, 16 cores with 2600 MHz is preferable to 8 cores with 3600 MHz. diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index e914d013d1c..368ef8e6f7c 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -5,7 +5,7 @@ toc_title: Server Settings # Server Settings {#server-settings} -## builtin\_dictionaries\_reload\_interval {#builtin-dictionaries-reload-interval} +## builtin_dictionaries_reload_interval {#builtin-dictionaries-reload-interval} The interval in seconds before reloading built-in dictionaries. @@ -92,7 +92,7 @@ Configures soft limit for core dump file size, one gigabyte by default. (Hard limit is configured via system tools) -## default\_database {#default-database} +## default_database {#default-database} The default database. @@ -104,7 +104,7 @@ To get a list of databases, use the [SHOW DATABASES](../../sql-reference/stateme default ``` -## default\_profile {#default-profile} +## default_profile {#default-profile} Default settings profile. @@ -116,7 +116,7 @@ Settings profiles are located in the file specified in the parameter `user_confi default ``` -## dictionaries\_config {#server_configuration_parameters-dictionaries_config} +## dictionaries_config {#server_configuration_parameters-dictionaries_config} The path to the config file for external dictionaries. @@ -133,7 +133,7 @@ See also “[External dictionaries](../../sql-reference/dictionaries/external-di *_dictionary.xml ``` -## dictionaries\_lazy\_load {#server_configuration_parameters-dictionaries_lazy_load} +## dictionaries_lazy_load {#server_configuration_parameters-dictionaries_lazy_load} Lazy loading of dictionaries. @@ -149,7 +149,7 @@ The default is `true`. true ``` -## format\_schema\_path {#server_configuration_parameters-format_schema_path} +## format_schema_path {#server_configuration_parameters-format_schema_path} The path to the directory with the schemes for the input data, such as schemas for the [CapnProto](../../interfaces/formats.md#capnproto) format. @@ -170,11 +170,11 @@ Settings: - port – The port on the Graphite server. - interval – The interval for sending, in seconds. - timeout – The timeout for sending data, in seconds. -- root\_path – Prefix for keys. +- root_path – Prefix for keys. - metrics – Sending data from the [system.metrics](../../operations/system-tables/metrics.md#system_tables-metrics) table. - events – Sending deltas data accumulated for the time period from the [system.events](../../operations/system-tables/events.md#system_tables-events) table. -- events\_cumulative – Sending cumulative data from the [system.events](../../operations/system-tables/events.md#system_tables-events) table. -- asynchronous\_metrics – Sending data from the [system.asynchronous\_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) table. +- events_cumulative – Sending cumulative data from the [system.events](../../operations/system-tables/events.md#system_tables-events) table. +- asynchronous_metrics – Sending data from the [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) table. You can configure multiple `` clauses. For instance, you can use this for sending different data at different intervals. @@ -194,7 +194,7 @@ You can configure multiple `` clauses. For instance, you can use this ``` -## graphite\_rollup {#server_configuration_parameters-graphite-rollup} +## graphite_rollup {#server_configuration_parameters-graphite-rollup} Settings for thinning data for Graphite. @@ -222,7 +222,7 @@ For more details, see [GraphiteMergeTree](../../engines/table-engines/mergetree- ``` -## http\_port/https\_port {#http-porthttps-port} +## http_port/https_port {#http-porthttps-port} The port for connecting to the server over HTTP(s). @@ -236,7 +236,7 @@ If `http_port` is specified, the OpenSSL configuration is ignored even if it is 9999 ``` -## http\_server\_default\_response {#server_configuration_parameters-http_server_default_response} +## http_server_default_response {#server_configuration_parameters-http_server_default_response} The page that is shown by default when you access the ClickHouse HTTP(s) server. The default value is “Ok.” (with a line feed at the end) @@ -251,7 +251,7 @@ Opens `https://tabix.io/` when accessing `http://localhost: http_port`. ``` -## include\_from {#server_configuration_parameters-include_from} +## include_from {#server_configuration_parameters-include_from} The path to the file with substitutions. @@ -263,7 +263,7 @@ For more information, see the section “[Configuration files](../../operations/ /etc/metrica.xml ``` -## interserver\_http\_port {#interserver-http-port} +## interserver_http_port {#interserver-http-port} Port for exchanging data between ClickHouse servers. @@ -273,7 +273,7 @@ Port for exchanging data between ClickHouse servers. 9009 ``` -## interserver\_http\_host {#interserver-http-host} +## interserver_http_host {#interserver-http-host} The hostname that can be used by other servers to access this server. @@ -287,7 +287,7 @@ Useful for breaking away from a specific network interface. example.yandex.ru ``` -## interserver\_http\_credentials {#server-settings-interserver-http-credentials} +## interserver_http_credentials {#server-settings-interserver-http-credentials} The username and password used to authenticate during [replication](../../engines/table-engines/mergetree-family/replication.md) with the Replicated\* engines. These credentials are used only for communication between replicas and are unrelated to credentials for ClickHouse clients. The server is checking these credentials for connecting replicas and use the same credentials when connecting to other replicas. So, these credentials should be set the same for all replicas in a cluster. By default, the authentication is not used. @@ -306,7 +306,7 @@ This section contains the following parameters: ``` -## keep\_alive\_timeout {#keep-alive-timeout} +## keep_alive_timeout {#keep-alive-timeout} The number of seconds that ClickHouse waits for incoming requests before closing the connection. Defaults to 3 seconds. @@ -316,7 +316,7 @@ The number of seconds that ClickHouse waits for incoming requests before closing 3 ``` -## listen\_host {#server_configuration_parameters-listen_host} +## listen_host {#server_configuration_parameters-listen_host} Restriction on hosts that requests can come from. If you want the server to answer all of them, specify `::`. @@ -367,14 +367,14 @@ Writing to the syslog is also supported. Config example: Keys for syslog: -- use\_syslog — Required setting if you want to write to the syslog. +- use_syslog — Required setting if you want to write to the syslog. - address — The host\[:port\] of syslogd. If omitted, the local daemon is used. - hostname — Optional. The name of the host that logs are sent from. -- facility — [The syslog facility keyword](https://en.wikipedia.org/wiki/Syslog#Facility) in uppercase letters with the “LOG\_” prefix: (`LOG_USER`, `LOG_DAEMON`, `LOG_LOCAL3`, and so on). +- facility — [The syslog facility keyword](https://en.wikipedia.org/wiki/Syslog#Facility) in uppercase letters with the “LOG_” prefix: (`LOG_USER`, `LOG_DAEMON`, `LOG_LOCAL3`, and so on). Default value: `LOG_USER` if `address` is specified, `LOG_DAEMON` otherwise. - format – Message format. Possible values: `bsd` and `syslog.` -## send\_crash\_reports {#server_configuration_parameters-logger} +## send_crash_reports {#server_configuration_parameters-logger} Settings for opt-in sending crash reports to the ClickHouse core developers team via [Sentry](https://sentry.io). Enabling it, especially in pre-production environments, is highly appreciated. @@ -412,7 +412,7 @@ For more information, see the section “[Creating replicated tables](../../engi ``` -## mark\_cache\_size {#server-mark-cache-size} +## mark_cache_size {#server-mark-cache-size} Approximate size (in bytes) of the cache of marks used by table engines of the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) family. @@ -424,7 +424,7 @@ The cache is shared for the server and memory is allocated as needed. The cache 5368709120 ``` -## max\_server\_memory\_usage {#max_server_memory_usage} +## max_server_memory_usage {#max_server_memory_usage} Limits total RAM usage by the ClickHouse server. @@ -441,7 +441,7 @@ The default `max_server_memory_usage` value is calculated as `memory_amount * ma **See also** -- [max\_memory\_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) +- [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) - [max_server_memory_usage_to_ram_ratio](#max_server_memory_usage_to_ram_ratio) ## max_server_memory_usage_to_ram_ratio {#max_server_memory_usage_to_ram_ratio} @@ -469,7 +469,7 @@ On hosts with low RAM and swap, you possibly need setting `max_server_memory_usa - [max_server_memory_usage](#max_server_memory_usage) -## max\_concurrent\_queries {#max-concurrent-queries} +## max_concurrent_queries {#max-concurrent-queries} The maximum number of simultaneously processed requests. @@ -479,7 +479,7 @@ The maximum number of simultaneously processed requests. 100 ``` -## max\_connections {#max-connections} +## max_connections {#max-connections} The maximum number of inbound connections. @@ -489,7 +489,7 @@ The maximum number of inbound connections. 4096 ``` -## max\_open\_files {#max-open-files} +## max_open_files {#max-open-files} The maximum number of open files. @@ -503,7 +503,7 @@ We recommend using this option in Mac OS X since the `getrlimit()` function retu 262144 ``` -## max\_table\_size\_to\_drop {#max-table-size-to-drop} +## max_table_size_to_drop {#max-table-size-to-drop} Restriction on deleting tables. @@ -521,7 +521,7 @@ The value 0 means that you can delete all tables without any restrictions. 0 ``` -## max\_thread\_pool\_size {#max-thread-pool-size} +## max_thread_pool_size {#max-thread-pool-size} The maximum number of threads in the Global Thread pool. @@ -533,7 +533,7 @@ Default value: 10000. 12000 ``` -## merge\_tree {#server_configuration_parameters-merge_tree} +## merge_tree {#server_configuration_parameters-merge_tree} Fine tuning for tables in the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). @@ -547,7 +547,7 @@ For more information, see the MergeTreeSettings.h header file. ``` -## replicated\_merge\_tree {#server_configuration_parameters-replicated_merge_tree} +## replicated_merge_tree {#server_configuration_parameters-replicated_merge_tree} Fine tuning for tables in the [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/mergetree.md). @@ -584,7 +584,7 @@ Keys for server/client settings: - sessionTimeout – Time for caching the session on the server. - extendedVerification – Automatically extended verification of certificates after the session ends. Acceptable values: `true`, `false`. - requireTLSv1 – Require a TLSv1 connection. Acceptable values: `true`, `false`. -- requireTLSv1\_1 – Require a TLSv1.1 connection. Acceptable values: `true`, `false`. +- requireTLSv1_1 – Require a TLSv1.1 connection. Acceptable values: `true`, `false`. - requireTLSv1 – Require a TLSv1.2 connection. Acceptable values: `true`, `false`. - fips – Activates OpenSSL FIPS mode. Supported if the library’s OpenSSL version supports FIPS. - privateKeyPassphraseHandler – Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: ``, `KeyFileHandler`, `test`, ``. @@ -622,11 +622,11 @@ Keys for server/client settings: ``` -## part\_log {#server_configuration_parameters-part-log} +## part_log {#server_configuration_parameters-part-log} Logging events that are associated with [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). For instance, adding or merging data. You can use the log to simulate merge algorithms and compare their characteristics. You can visualize the merge process. -Queries are logged in the [system.part\_log](../../operations/system-tables/part_log.md#system_tables-part-log) table, not in a separate file. You can configure the name of this table in the `table` parameter (see below). +Queries are logged in the [system.part_log](../../operations/system-tables/part_log.md#system_tables-part-log) table, not in a separate file. You can configure the name of this table in the `table` parameter (see below). Use the following parameters to configure logging: @@ -670,7 +670,7 @@ Settings: - `port` – Port for `endpoint`. - `metrics` – Flag that sets to expose metrics from the [system.metrics](../../operations/system-tables/metrics.md#system_tables-metrics) table. - `events` – Flag that sets to expose metrics from the [system.events](../../operations/system-tables/events.md#system_tables-events) table. -- `asynchronous_metrics` – Flag that sets to expose current metrics values from the [system.asynchronous\_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) table. +- `asynchronous_metrics` – Flag that sets to expose current metrics values from the [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) table. **Example** @@ -684,11 +684,11 @@ Settings: ``` -## query\_log {#server_configuration_parameters-query-log} +## query_log {#server_configuration_parameters-query-log} -Setting for logging queries received with the [log\_queries=1](../../operations/settings/settings.md) setting. +Setting for logging queries received with the [log_queries=1](../../operations/settings/settings.md) setting. -Queries are logged in the [system.query\_log](../../operations/system-tables/query_log.md#system_tables-query_log) table, not in a separate file. You can change the name of the table in the `table` parameter (see below). +Queries are logged in the [system.query_log](../../operations/system-tables/query_log.md#system_tables-query_log) table, not in a separate file. You can change the name of the table in the `table` parameter (see below). Use the following parameters to configure logging: @@ -711,11 +711,11 @@ If the table doesn’t exist, ClickHouse will create it. If the structure of the ``` -## query\_thread\_log {#server_configuration_parameters-query_thread_log} +## query_thread_log {#server_configuration_parameters-query_thread_log} -Setting for logging threads of queries received with the [log\_query\_threads=1](../../operations/settings/settings.md#settings-log-query-threads) setting. +Setting for logging threads of queries received with the [log_query_threads=1](../../operations/settings/settings.md#settings-log-query-threads) setting. -Queries are logged in the [system.query\_thread\_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log) table, not in a separate file. You can change the name of the table in the `table` parameter (see below). +Queries are logged in the [system.query_thread_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log) table, not in a separate file. You can change the name of the table in the `table` parameter (see below). Use the following parameters to configure logging: @@ -738,9 +738,9 @@ If the table doesn’t exist, ClickHouse will create it. If the structure of the ``` -## text\_log {#server_configuration_parameters-text_log} +## text_log {#server_configuration_parameters-text_log} -Settings for the [text\_log](../../operations/system-tables/text_log.md#system_tables-text_log) system table for logging text messages. +Settings for the [text_log](../../operations/system-tables/text_log.md#system_tables-text_log) system table for logging text messages. Parameters: @@ -766,9 +766,9 @@ Parameters: ``` -## trace\_log {#server_configuration_parameters-trace_log} +## trace_log {#server_configuration_parameters-trace_log} -Settings for the [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) system table operation. +Settings for the [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) system table operation. Parameters: @@ -789,7 +789,7 @@ The default server configuration file `config.xml` contains the following settin ``` -## query\_masking\_rules {#query-masking-rules} +## query_masking_rules {#query-masking-rules} Regexp-based rules, which will be applied to queries as well as all log messages before storing them in server logs, `system.query_log`, `system.text_log`, `system.processes` tables, and in logs sent to the client. That allows preventing @@ -820,7 +820,7 @@ The masking rules are applied to the whole query (to prevent leaks of sensitive For distributed queries each server have to be configured separately, otherwise, subqueries passed to other nodes will be stored without masking. -## remote\_servers {#server-settings-remote-servers} +## remote_servers {#server-settings-remote-servers} Configuration of clusters used by the [Distributed](../../engines/table-engines/special/distributed.md) table engine and by the `cluster` table function. @@ -834,7 +834,7 @@ For the value of the `incl` attribute, see the section “[Configuration files]( **See Also** -- [skip\_unavailable\_shards](../../operations/settings/settings.md#settings-skip_unavailable_shards) +- [skip_unavailable_shards](../../operations/settings/settings.md#settings-skip_unavailable_shards) ## timezone {#server_configuration_parameters-timezone} @@ -850,7 +850,7 @@ The time zone is necessary for conversions between String and DateTime formats w Europe/Moscow ``` -## tcp\_port {#server_configuration_parameters-tcp_port} +## tcp_port {#server_configuration_parameters-tcp_port} Port for communicating with clients over the TCP protocol. @@ -860,7 +860,7 @@ Port for communicating with clients over the TCP protocol. 9000 ``` -## tcp\_port\_secure {#server_configuration_parameters-tcp_port_secure} +## tcp_port_secure {#server_configuration_parameters-tcp_port_secure} TCP port for secure communication with clients. Use it with [OpenSSL](#server_configuration_parameters-openssl) settings. @@ -874,7 +874,7 @@ Positive integer. 9440 ``` -## mysql\_port {#server_configuration_parameters-mysql_port} +## mysql_port {#server_configuration_parameters-mysql_port} Port for communicating with clients over MySQL protocol. @@ -888,7 +888,7 @@ Example 9004 ``` -## tmp\_path {#tmp-path} +## tmp_path {#tmp-path} Path to temporary data for processing large queries. @@ -901,11 +901,11 @@ Path to temporary data for processing large queries. /var/lib/clickhouse/tmp/ ``` -## tmp\_policy {#tmp-policy} +## tmp_policy {#tmp-policy} -Policy from [storage\_configuration](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) to store temporary files. +Policy from [storage_configuration](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) to store temporary files. -If not set, [tmp\_path](#tmp-path) is used, otherwise it is ignored. +If not set, [tmp_path](#tmp-path) is used, otherwise it is ignored. !!! note "Note" - `move_factor` is ignored. @@ -913,11 +913,11 @@ If not set, [tmp\_path](#tmp-path) is used, otherwise it is ignored. - `max_data_part_size_bytes` is ignored. - Уou must have exactly one volume in that policy. -## uncompressed\_cache\_size {#server-settings-uncompressed_cache_size} +## uncompressed_cache_size {#server-settings-uncompressed_cache_size} Cache size (in bytes) for uncompressed data used by table engines from the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). -There is one shared cache for the server. Memory is allocated on demand. The cache is used if the option [use\_uncompressed\_cache](../../operations/settings/settings.md#setting-use_uncompressed_cache) is enabled. +There is one shared cache for the server. Memory is allocated on demand. The cache is used if the option [use_uncompressed_cache](../../operations/settings/settings.md#setting-use_uncompressed_cache) is enabled. The uncompressed cache is advantageous for very short queries in individual cases. @@ -927,7 +927,7 @@ The uncompressed cache is advantageous for very short queries in individual case 8589934592 ``` -## user\_files\_path {#server_configuration_parameters-user_files_path} +## user_files_path {#server_configuration_parameters-user_files_path} The directory with user files. Used in the table function [file()](../../sql-reference/table-functions/file.md). @@ -937,7 +937,7 @@ The directory with user files. Used in the table function [file()](../../sql-ref /var/lib/clickhouse/user_files/ ``` -## users\_config {#users-config} +## users_config {#users-config} Path to the file that contains: @@ -1005,13 +1005,13 @@ This section contains the following parameters: - [Replication](../../engines/table-engines/mergetree-family/replication.md) - [ZooKeeper Programmer’s Guide](http://zookeeper.apache.org/doc/current/zookeeperProgrammers.html) -## use\_minimalistic\_part\_header\_in\_zookeeper {#server-settings-use_minimalistic_part_header_in_zookeeper} +## use_minimalistic_part_header_in_zookeeper {#server-settings-use_minimalistic_part_header_in_zookeeper} Storage method for data part headers in ZooKeeper. This setting only applies to the `MergeTree` family. It can be specified: -- Globally in the [merge\_tree](#server_configuration_parameters-merge_tree) section of the `config.xml` file. +- Globally in the [merge_tree](#server_configuration_parameters-merge_tree) section of the `config.xml` file. ClickHouse uses the setting for all the tables on the server. You can change the setting at any time. Existing tables change their behaviour when the setting changes. @@ -1033,14 +1033,14 @@ If `use_minimalistic_part_header_in_zookeeper = 1`, then [replicated](../../engi **Default value:** 0. -## disable\_internal\_dns\_cache {#server-settings-disable-internal-dns-cache} +## disable_internal_dns_cache {#server-settings-disable-internal-dns-cache} Disables the internal DNS cache. Recommended for operating ClickHouse in systems with frequently changing infrastructure such as Kubernetes. **Default value:** 0. -## dns\_cache\_update\_period {#server-settings-dns-cache-update-period} +## dns_cache_update_period {#server-settings-dns-cache-update-period} The period of updating IP addresses stored in the ClickHouse internal DNS cache (in seconds). The update is performed asynchronously, in a separate system thread. @@ -1049,9 +1049,9 @@ The update is performed asynchronously, in a separate system thread. **See also** -- [background\_schedule\_pool\_size](../../operations/settings/settings.md#background_schedule_pool_size) +- [background_schedule_pool_size](../../operations/settings/settings.md#background_schedule_pool_size) -## access\_control\_path {#access_control_path} +## access_control_path {#access_control_path} Path to a folder where a ClickHouse server stores user and role configurations created by SQL commands. diff --git a/docs/en/operations/settings/permissions-for-queries.md b/docs/en/operations/settings/permissions-for-queries.md index b94301bf103..2ab13216d9e 100644 --- a/docs/en/operations/settings/permissions-for-queries.md +++ b/docs/en/operations/settings/permissions-for-queries.md @@ -16,7 +16,7 @@ Queries in ClickHouse can be divided into several types: The following settings regulate user permissions by the type of query: - [readonly](#settings_readonly) — Restricts permissions for all types of queries except DDL queries. -- [allow\_ddl](#settings_allow_ddl) — Restricts permissions for DDL queries. +- [allow_ddl](#settings_allow_ddl) — Restricts permissions for DDL queries. `KILL QUERY` can be performed with any settings. @@ -41,7 +41,7 @@ from changing only specific settings, for details see [constraints on settings]( Default value: 0 -## allow\_ddl {#settings_allow_ddl} +## allow_ddl {#settings_allow_ddl} Allows or denies [DDL](https://en.wikipedia.org/wiki/Data_definition_language) queries. diff --git a/docs/en/operations/settings/query-complexity.md b/docs/en/operations/settings/query-complexity.md index f803e694eb7..2ecf50762d5 100644 --- a/docs/en/operations/settings/query-complexity.md +++ b/docs/en/operations/settings/query-complexity.md @@ -12,8 +12,8 @@ Almost all the restrictions only apply to `SELECT`. For distributed query proces ClickHouse checks the restrictions for data parts, not for each row. It means that you can exceed the value of restriction with the size of the data part. Restrictions on the “maximum amount of something” can take the value 0, which means “unrestricted”. -Most restrictions also have an ‘overflow\_mode’ setting, meaning what to do when the limit is exceeded. -It can take one of two values: `throw` or `break`. Restrictions on aggregation (group\_by\_overflow\_mode) also have the value `any`. +Most restrictions also have an ‘overflow_mode’ setting, meaning what to do when the limit is exceeded. +It can take one of two values: `throw` or `break`. Restrictions on aggregation (group_by_overflow_mode) also have the value `any`. `throw` – Throw an exception (default). @@ -21,7 +21,7 @@ It can take one of two values: `throw` or `break`. Restrictions on aggregation ( `any (only for group_by_overflow_mode)` – Continuing aggregation for the keys that got into the set, but don’t add new keys to the set. -## max\_memory\_usage {#settings_max_memory_usage} +## max_memory_usage {#settings_max_memory_usage} The maximum amount of RAM to use for running a query on a single server. @@ -36,31 +36,31 @@ Memory usage is not monitored for the states of certain aggregate functions. Memory usage is not fully tracked for states of the aggregate functions `min`, `max`, `any`, `anyLast`, `argMin`, `argMax` from `String` and `Array` arguments. -Memory consumption is also restricted by the parameters `max_memory_usage_for_user` and [max\_server\_memory\_usage](../../operations/server-configuration-parameters/settings.md#max_server_memory_usage). +Memory consumption is also restricted by the parameters `max_memory_usage_for_user` and [max_server_memory_usage](../../operations/server-configuration-parameters/settings.md#max_server_memory_usage). -## max\_memory\_usage\_for\_user {#max-memory-usage-for-user} +## max_memory_usage_for_user {#max-memory-usage-for-user} The maximum amount of RAM to use for running a user’s queries on a single server. Default values are defined in [Settings.h](https://github.com/ClickHouse/ClickHouse/blob/master/src/Core/Settings.h#L288). By default, the amount is not restricted (`max_memory_usage_for_user = 0`). -See also the description of [max\_memory\_usage](#settings_max_memory_usage). +See also the description of [max_memory_usage](#settings_max_memory_usage). -## max\_rows\_to\_read {#max-rows-to-read} +## max_rows_to_read {#max-rows-to-read} The following restrictions can be checked on each block (instead of on each row). That is, the restrictions can be broken a little. A maximum number of rows that can be read from a table when running a query. -## max\_bytes\_to\_read {#max-bytes-to-read} +## max_bytes_to_read {#max-bytes-to-read} A maximum number of bytes (uncompressed data) that can be read from a table when running a query. -## read\_overflow\_mode {#read-overflow-mode} +## read_overflow_mode {#read-overflow-mode} What to do when the volume of data read exceeds one of the limits: ‘throw’ or ‘break’. By default, throw. -## max\_rows\_to\_read_leaf {#max-rows-to-read-leaf} +## max_rows_to_read_leaf {#max-rows-to-read-leaf} The following restrictions can be checked on each block (instead of on each row). That is, the restrictions can be broken a little. @@ -71,7 +71,7 @@ and each shard contains a table with 100 rows. Then distributed query which supp tables with setting `max_rows_to_read=150` will fail as in total it will be 200 rows. While query with `max_rows_to_read_leaf=150` will succeed since leaf nodes will read 100 rows at max. -## max\_bytes\_to\_read_leaf {#max-bytes-to-read-leaf} +## max_bytes_to_read_leaf {#max-bytes-to-read-leaf} A maximum number of bytes (uncompressed data) that can be read from a local table on a leaf node when running a distributed query. While distributed queries can issue a multiple sub-queries to each shard (leaf) - this limit will @@ -81,20 +81,20 @@ Then distributed query which suppose to read all the data from both tables with as in total it will be 200 bytes. While query with `max_bytes_to_read_leaf=150` will succeed since leaf nodes will read 100 bytes at max. -## read\_overflow\_mode_leaf {#read-overflow-mode-leaf} +## read_overflow_mode_leaf {#read-overflow-mode-leaf} What to do when the volume of data read exceeds one of the leaf limits: ‘throw’ or ‘break’. By default, throw. -## max\_rows\_to\_group\_by {#settings-max-rows-to-group-by} +## max_rows_to_group_by {#settings-max-rows-to-group-by} A maximum number of unique keys received from aggregation. This setting lets you limit memory consumption when aggregating. -## group\_by\_overflow\_mode {#group-by-overflow-mode} +## group_by_overflow_mode {#group-by-overflow-mode} What to do when the number of unique keys for aggregation exceeds the limit: ‘throw’, ‘break’, or ‘any’. By default, throw. Using the ‘any’ value lets you run an approximation of GROUP BY. The quality of this approximation depends on the statistical nature of the data. -## max\_bytes\_before\_external\_group\_by {#settings-max_bytes_before_external_group_by} +## max_bytes_before_external_group_by {#settings-max_bytes_before_external_group_by} Enables or disables execution of `GROUP BY` clauses in external memory. See [GROUP BY in external memory](../../sql-reference/statements/select/group-by.md#select-group-by-in-external-memory). @@ -105,31 +105,31 @@ Possible values: Default value: 0. -## max\_rows\_to\_sort {#max-rows-to-sort} +## max_rows_to_sort {#max-rows-to-sort} A maximum number of rows before sorting. This allows you to limit memory consumption when sorting. -## max\_bytes\_to\_sort {#max-bytes-to-sort} +## max_bytes_to_sort {#max-bytes-to-sort} A maximum number of bytes before sorting. -## sort\_overflow\_mode {#sort-overflow-mode} +## sort_overflow_mode {#sort-overflow-mode} What to do if the number of rows received before sorting exceeds one of the limits: ‘throw’ or ‘break’. By default, throw. -## max\_result\_rows {#setting-max_result_rows} +## max_result_rows {#setting-max_result_rows} Limit on the number of rows in the result. Also checked for subqueries, and on remote servers when running parts of a distributed query. -## max\_result\_bytes {#max-result-bytes} +## max_result_bytes {#max-result-bytes} Limit on the number of bytes in the result. The same as the previous setting. -## result\_overflow\_mode {#result-overflow-mode} +## result_overflow_mode {#result-overflow-mode} What to do if the volume of the result exceeds one of the limits: ‘throw’ or ‘break’. By default, throw. -Using ‘break’ is similar to using LIMIT. `Break` interrupts execution only at the block level. This means that amount of returned rows is greater than [max\_result\_rows](#setting-max_result_rows), multiple of [max\_block\_size](../../operations/settings/settings.md#setting-max_block_size) and depends on [max\_threads](../../operations/settings/settings.md#settings-max_threads). +Using ‘break’ is similar to using LIMIT. `Break` interrupts execution only at the block level. This means that amount of returned rows is greater than [max_result_rows](#setting-max_result_rows), multiple of [max_block_size](../../operations/settings/settings.md#setting-max_block_size) and depends on [max_threads](../../operations/settings/settings.md#settings-max_threads). Example: @@ -148,103 +148,103 @@ Result: 6666 rows in set. ... ``` -## max\_execution\_time {#max-execution-time} +## max_execution_time {#max-execution-time} Maximum query execution time in seconds. At this time, it is not checked for one of the sorting stages, or when merging and finalizing aggregate functions. -## timeout\_overflow\_mode {#timeout-overflow-mode} +## timeout_overflow_mode {#timeout-overflow-mode} -What to do if the query is run longer than ‘max\_execution\_time’: ‘throw’ or ‘break’. By default, throw. +What to do if the query is run longer than ‘max_execution_time’: ‘throw’ or ‘break’. By default, throw. -## min\_execution\_speed {#min-execution-speed} +## min_execution_speed {#min-execution-speed} -Minimal execution speed in rows per second. Checked on every data block when ‘timeout\_before\_checking\_execution\_speed’ expires. If the execution speed is lower, an exception is thrown. +Minimal execution speed in rows per second. Checked on every data block when ‘timeout_before_checking_execution_speed’ expires. If the execution speed is lower, an exception is thrown. -## min\_execution\_speed\_bytes {#min-execution-speed-bytes} +## min_execution_speed_bytes {#min-execution-speed-bytes} -A minimum number of execution bytes per second. Checked on every data block when ‘timeout\_before\_checking\_execution\_speed’ expires. If the execution speed is lower, an exception is thrown. +A minimum number of execution bytes per second. Checked on every data block when ‘timeout_before_checking_execution_speed’ expires. If the execution speed is lower, an exception is thrown. -## max\_execution\_speed {#max-execution-speed} +## max_execution_speed {#max-execution-speed} -A maximum number of execution rows per second. Checked on every data block when ‘timeout\_before\_checking\_execution\_speed’ expires. If the execution speed is high, the execution speed will be reduced. +A maximum number of execution rows per second. Checked on every data block when ‘timeout_before_checking_execution_speed’ expires. If the execution speed is high, the execution speed will be reduced. -## max\_execution\_speed\_bytes {#max-execution-speed-bytes} +## max_execution_speed_bytes {#max-execution-speed-bytes} -A maximum number of execution bytes per second. Checked on every data block when ‘timeout\_before\_checking\_execution\_speed’ expires. If the execution speed is high, the execution speed will be reduced. +A maximum number of execution bytes per second. Checked on every data block when ‘timeout_before_checking_execution_speed’ expires. If the execution speed is high, the execution speed will be reduced. -## timeout\_before\_checking\_execution\_speed {#timeout-before-checking-execution-speed} +## timeout_before_checking_execution_speed {#timeout-before-checking-execution-speed} -Checks that execution speed is not too slow (no less than ‘min\_execution\_speed’), after the specified time in seconds has expired. +Checks that execution speed is not too slow (no less than ‘min_execution_speed’), after the specified time in seconds has expired. -## max\_columns\_to\_read {#max-columns-to-read} +## max_columns_to_read {#max-columns-to-read} A maximum number of columns that can be read from a table in a single query. If a query requires reading a greater number of columns, it throws an exception. -## max\_temporary\_columns {#max-temporary-columns} +## max_temporary_columns {#max-temporary-columns} A maximum number of temporary columns that must be kept in RAM at the same time when running a query, including constant columns. If there are more temporary columns than this, it throws an exception. -## max\_temporary\_non\_const\_columns {#max-temporary-non-const-columns} +## max_temporary_non_const_columns {#max-temporary-non-const-columns} -The same thing as ‘max\_temporary\_columns’, but without counting constant columns. +The same thing as ‘max_temporary_columns’, but without counting constant columns. Note that constant columns are formed fairly often when running a query, but they require approximately zero computing resources. -## max\_subquery\_depth {#max-subquery-depth} +## max_subquery_depth {#max-subquery-depth} Maximum nesting depth of subqueries. If subqueries are deeper, an exception is thrown. By default, 100. -## max\_pipeline\_depth {#max-pipeline-depth} +## max_pipeline_depth {#max-pipeline-depth} Maximum pipeline depth. Corresponds to the number of transformations that each data block goes through during query processing. Counted within the limits of a single server. If the pipeline depth is greater, an exception is thrown. By default, 1000. -## max\_ast\_depth {#max-ast-depth} +## max_ast_depth {#max-ast-depth} Maximum nesting depth of a query syntactic tree. If exceeded, an exception is thrown. At this time, it isn’t checked during parsing, but only after parsing the query. That is, a syntactic tree that is too deep can be created during parsing, but the query will fail. By default, 1000. -## max\_ast\_elements {#max-ast-elements} +## max_ast_elements {#max-ast-elements} A maximum number of elements in a query syntactic tree. If exceeded, an exception is thrown. In the same way as the previous setting, it is checked only after parsing the query. By default, 50,000. -## max\_rows\_in\_set {#max-rows-in-set} +## max_rows_in_set {#max-rows-in-set} A maximum number of rows for a data set in the IN clause created from a subquery. -## max\_bytes\_in\_set {#max-bytes-in-set} +## max_bytes_in_set {#max-bytes-in-set} A maximum number of bytes (uncompressed data) used by a set in the IN clause created from a subquery. -## set\_overflow\_mode {#set-overflow-mode} +## set_overflow_mode {#set-overflow-mode} What to do when the amount of data exceeds one of the limits: ‘throw’ or ‘break’. By default, throw. -## max\_rows\_in\_distinct {#max-rows-in-distinct} +## max_rows_in_distinct {#max-rows-in-distinct} A maximum number of different rows when using DISTINCT. -## max\_bytes\_in\_distinct {#max-bytes-in-distinct} +## max_bytes_in_distinct {#max-bytes-in-distinct} A maximum number of bytes used by a hash table when using DISTINCT. -## distinct\_overflow\_mode {#distinct-overflow-mode} +## distinct_overflow_mode {#distinct-overflow-mode} What to do when the amount of data exceeds one of the limits: ‘throw’ or ‘break’. By default, throw. -## max\_rows\_to\_transfer {#max-rows-to-transfer} +## max_rows_to_transfer {#max-rows-to-transfer} A maximum number of rows that can be passed to a remote server or saved in a temporary table when using GLOBAL IN. -## max\_bytes\_to\_transfer {#max-bytes-to-transfer} +## max_bytes_to_transfer {#max-bytes-to-transfer} A maximum number of bytes (uncompressed data) that can be passed to a remote server or saved in a temporary table when using GLOBAL IN. -## transfer\_overflow\_mode {#transfer-overflow-mode} +## transfer_overflow_mode {#transfer-overflow-mode} What to do when the amount of data exceeds one of the limits: ‘throw’ or ‘break’. By default, throw. -## max\_rows\_in\_join {#settings-max_rows_in_join} +## max_rows_in_join {#settings-max_rows_in_join} Limits the number of rows in the hash table that is used when joining tables. @@ -252,7 +252,7 @@ This settings applies to [SELECT … JOIN](../../sql-reference/statements/select If a query contains multiple joins, ClickHouse checks this setting for every intermediate result. -ClickHouse can proceed with different actions when the limit is reached. Use the [join\_overflow\_mode](#settings-join_overflow_mode) setting to choose the action. +ClickHouse can proceed with different actions when the limit is reached. Use the [join_overflow_mode](#settings-join_overflow_mode) setting to choose the action. Possible values: @@ -261,7 +261,7 @@ Possible values: Default value: 0. -## max\_bytes\_in\_join {#settings-max_bytes_in_join} +## max_bytes_in_join {#settings-max_bytes_in_join} Limits the size in bytes of the hash table used when joining tables. @@ -269,7 +269,7 @@ This settings applies to [SELECT … JOIN](../../sql-reference/statements/select If the query contains joins, ClickHouse checks this setting for every intermediate result. -ClickHouse can proceed with different actions when the limit is reached. Use [join\_overflow\_mode](#settings-join_overflow_mode) settings to choose the action. +ClickHouse can proceed with different actions when the limit is reached. Use [join_overflow_mode](#settings-join_overflow_mode) settings to choose the action. Possible values: @@ -278,12 +278,12 @@ Possible values: Default value: 0. -## join\_overflow\_mode {#settings-join_overflow_mode} +## join_overflow_mode {#settings-join_overflow_mode} Defines what action ClickHouse performs when any of the following join limits is reached: -- [max\_bytes\_in\_join](#settings-max_bytes_in_join) -- [max\_rows\_in\_join](#settings-max_rows_in_join) +- [max_bytes_in_join](#settings-max_bytes_in_join) +- [max_rows_in_join](#settings-max_rows_in_join) Possible values: @@ -297,7 +297,7 @@ Default value: `THROW`. - [JOIN clause](../../sql-reference/statements/select/join.md#select-join) - [Join table engine](../../engines/table-engines/special/join.md) -## max\_partitions\_per\_insert\_block {#max-partitions-per-insert-block} +## max_partitions_per_insert_block {#max-partitions-per-insert-block} Limits the maximum number of partitions in a single inserted block. @@ -310,6 +310,6 @@ Default value: 100. When inserting data, ClickHouse calculates the number of partitions in the inserted block. If the number of partitions is more than `max_partitions_per_insert_block`, ClickHouse throws an exception with the following text: -> “Too many partitions for single INSERT block (more than” + toString(max\_parts) + “). The limit is controlled by ‘max\_partitions\_per\_insert\_block’ setting. A large number of partitions is a common misconception. It will lead to severe negative performance impact, including slow server startup, slow INSERT queries and slow SELECT queries. Recommended total number of partitions for a table is under 1000..10000. Please note, that partitioning is not intended to speed up SELECT queries (ORDER BY key is sufficient to make range queries fast). Partitions are intended for data manipulation (DROP PARTITION, etc).” +> “Too many partitions for single INSERT block (more than” + toString(max_parts) + “). The limit is controlled by ‘max_partitions_per_insert_block’ setting. A large number of partitions is a common misconception. It will lead to severe negative performance impact, including slow server startup, slow INSERT queries and slow SELECT queries. Recommended total number of partitions for a table is under 1000..10000. Please note, that partitioning is not intended to speed up SELECT queries (ORDER BY key is sufficient to make range queries fast). Partitions are intended for data manipulation (DROP PARTITION, etc).” [Original article](https://clickhouse.tech/docs/en/operations/settings/query_complexity/) diff --git a/docs/en/operations/settings/settings-users.md b/docs/en/operations/settings/settings-users.md index 7d6d39e6d88..3e15d9e6dea 100644 --- a/docs/en/operations/settings/settings-users.md +++ b/docs/en/operations/settings/settings-users.md @@ -41,7 +41,7 @@ Structure of the `users` section: ``` -### user\_name/password {#user-namepassword} +### user_name/password {#user-namepassword} Password can be specified in plaintext or in SHA256 (hex format). @@ -73,7 +73,7 @@ Password can be specified in plaintext or in SHA256 (hex format). The first line of the result is the password. The second line is the corresponding double SHA1 hash. -### access\_management {#access_management-user-setting} +### access_management {#access_management-user-setting} This setting enables or disables using of SQL-driven [access control and account management](../../operations/access-rights.md#access-control) for the user. @@ -84,7 +84,7 @@ Possible values: Default value: 0. -### user\_name/networks {#user-namenetworks} +### user_name/networks {#user-namenetworks} List of networks from which the user can connect to the ClickHouse server. @@ -126,18 +126,18 @@ To open access only from localhost, specify: 127.0.0.1 ``` -### user\_name/profile {#user-nameprofile} +### user_name/profile {#user-nameprofile} You can assign a settings profile for the user. Settings profiles are configured in a separate section of the `users.xml` file. For more information, see [Profiles of Settings](../../operations/settings/settings-profiles.md). -### user\_name/quota {#user-namequota} +### user_name/quota {#user-namequota} Quotas allow you to track or limit resource usage over a period of time. Quotas are configured in the `quotas` section of the `users.xml` configuration file. You can assign a quotas set for the user. For a detailed description of quotas configuration, see [Quotas](../../operations/quotas.md#quotas). -### user\_name/databases {#user-namedatabases} +### user_name/databases {#user-namedatabases} In this section, you can you can limit rows that are returned by ClickHouse for `SELECT` queries made by the current user, thus implementing basic row-level security. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 46d0c76dd8d..31bad664c19 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1,6 +1,6 @@ # Settings {#settings} -## distributed\_product\_mode {#distributed-product-mode} +## distributed_product_mode {#distributed-product-mode} Changes the behaviour of [distributed subqueries](../../sql-reference/operators/in.md). @@ -20,7 +20,7 @@ Possible values: - `global` — Replaces the `IN`/`JOIN` query with `GLOBAL IN`/`GLOBAL JOIN.` - `allow` — Allows the use of these types of subqueries. -## enable\_optimize\_predicate\_expression {#enable-optimize-predicate-expression} +## enable_optimize_predicate_expression {#enable-optimize-predicate-expression} Turns on predicate pushdown in `SELECT` queries. @@ -44,7 +44,7 @@ If `enable_optimize_predicate_expression = 1`, then the execution time of these If `enable_optimize_predicate_expression = 0`, then the execution time of the second query is much longer because the `WHERE` clause applies to all the data after the subquery finishes. -## fallback\_to\_stale\_replicas\_for\_distributed\_queries {#settings-fallback_to_stale_replicas_for_distributed_queries} +## fallback_to_stale_replicas_for_distributed_queries {#settings-fallback_to_stale_replicas_for_distributed_queries} Forces a query to an out-of-date replica if updated data is not available. See [Replication](../../engines/table-engines/mergetree-family/replication.md). @@ -54,7 +54,7 @@ Used when performing `SELECT` from a distributed table that points to replicated By default, 1 (enabled). -## force\_index\_by\_date {#settings-force_index_by_date} +## force_index_by_date {#settings-force_index_by_date} Disables query execution if the index can’t be used by date. @@ -62,7 +62,7 @@ Works with tables in the MergeTree family. If `force_index_by_date=1`, ClickHouse checks whether the query has a date key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition reduces the amount of data to read. For example, the condition `Date != ' 2000-01-01 '` is acceptable even when it matches all the data in the table (i.e., running the query requires a full scan). For more information about ranges of data in MergeTree tables, see [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). -## force\_primary\_key {#force-primary-key} +## force_primary_key {#force-primary-key} Disables query execution if indexing by the primary key is not possible. @@ -70,7 +70,7 @@ Works with tables in the MergeTree family. If `force_primary_key=1`, ClickHouse checks to see if the query has a primary key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition reduces the amount of data to read. For more information about data ranges in MergeTree tables, see [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). -## force\_data\_skipping\_indices {#settings-force_data_skipping_indices} +## force_data_skipping_indices {#settings-force_data_skipping_indices} Disables query execution if passed data skipping indices wasn't used. @@ -99,17 +99,17 @@ SELECT * FROM data_01515 WHERE d1 = 0 AND assumeNotNull(d1_null) = 0 SETTINGS fo Works with tables in the MergeTree family. -## format\_schema {#format-schema} +## format_schema {#format-schema} This parameter is useful when you are using formats that require a schema definition, such as [Cap’n Proto](https://capnproto.org/) or [Protobuf](https://developers.google.com/protocol-buffers/). The value depends on the format. -## fsync\_metadata {#fsync-metadata} +## fsync_metadata {#fsync-metadata} Enables or disables [fsync](http://pubs.opengroup.org/onlinepubs/9699919799/functions/fsync.html) when writing `.sql` files. Enabled by default. It makes sense to disable it if the server has millions of tiny tables that are constantly being created and destroyed. -## enable\_http\_compression {#settings-enable_http_compression} +## enable_http_compression {#settings-enable_http_compression} Enables or disables data compression in the response to an HTTP request. @@ -122,15 +122,15 @@ Possible values: Default value: 0. -## http\_zlib\_compression\_level {#settings-http_zlib_compression_level} +## http_zlib_compression_level {#settings-http_zlib_compression_level} -Sets the level of data compression in the response to an HTTP request if [enable\_http\_compression = 1](#settings-enable_http_compression). +Sets the level of data compression in the response to an HTTP request if [enable_http_compression = 1](#settings-enable_http_compression). Possible values: Numbers from 1 to 9. Default value: 3. -## http\_native\_compression\_disable\_checksumming\_on\_decompress {#settings-http_native_compression_disable_checksumming_on_decompress} +## http_native_compression_disable_checksumming_on_decompress {#settings-http_native_compression_disable_checksumming_on_decompress} Enables or disables checksum verification when decompressing the HTTP POST data from the client. Used only for ClickHouse native compression format (not used with `gzip` or `deflate`). @@ -143,7 +143,7 @@ Possible values: Default value: 0. -## send\_progress\_in\_http\_headers {#settings-send_progress_in_http_headers} +## send_progress_in_http_headers {#settings-send_progress_in_http_headers} Enables or disables `X-ClickHouse-Progress` HTTP response headers in `clickhouse-server` responses. @@ -156,7 +156,7 @@ Possible values: Default value: 0. -## max\_http\_get\_redirects {#setting-max_http_get_redirects} +## max_http_get_redirects {#setting-max_http_get_redirects} Limits the maximum number of HTTP GET redirect hops for [URL](../../engines/table-engines/special/url.md)-engine tables. The setting applies to both types of tables: those created by the [CREATE TABLE](../../sql-reference/statements/create/table.md) query and by the [url](../../sql-reference/table-functions/url.md) table function. @@ -167,7 +167,7 @@ Possible values: Default value: 0. -## input\_format\_allow\_errors\_num {#settings-input_format_allow_errors_num} +## input_format_allow_errors_num {#settings-input_format_allow_errors_num} Sets the maximum number of acceptable errors when reading from text formats (CSV, TSV, etc.). @@ -179,7 +179,7 @@ If an error occurred while reading rows but the error counter is still less than If both `input_format_allow_errors_num` and `input_format_allow_errors_ratio` are exceeded, ClickHouse throws an exception. -## input\_format\_allow\_errors\_ratio {#settings-input_format_allow_errors_ratio} +## input_format_allow_errors_ratio {#settings-input_format_allow_errors_ratio} Sets the maximum percentage of errors allowed when reading from text formats (CSV, TSV, etc.). The percentage of errors is set as a floating-point number between 0 and 1. @@ -192,7 +192,7 @@ If an error occurred while reading rows but the error counter is still less than If both `input_format_allow_errors_num` and `input_format_allow_errors_ratio` are exceeded, ClickHouse throws an exception. -## input\_format\_values\_interpret\_expressions {#settings-input_format_values_interpret_expressions} +## input_format_values_interpret_expressions {#settings-input_format_values_interpret_expressions} Enables or disables the full SQL parser if the fast stream parser can’t parse the data. This setting is used only for the [Values](../../interfaces/formats.md#data-format-values) format at the data insertion. For more information about syntax parsing, see the [Syntax](../../sql-reference/syntax.md) section. @@ -242,7 +242,7 @@ INSERT INTO datetime_t SELECT now() Ok. ``` -## input\_format\_values\_deduce\_templates\_of\_expressions {#settings-input_format_values_deduce_templates_of_expressions} +## input_format_values_deduce_templates_of_expressions {#settings-input_format_values_deduce_templates_of_expressions} Enables or disables template deduction for SQL expressions in [Values](../../interfaces/formats.md#data-format-values) format. It allows parsing and interpreting expressions in `Values` much faster if expressions in consecutive rows have the same structure. ClickHouse tries to deduce the template of an expression, parse the following rows using this template and evaluate the expression on a batch of successfully parsed rows. @@ -263,7 +263,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( - If `input_format_values_interpret_expressions=0` and `format_values_deduce_templates_of_expressions=1`, expressions in the first, second and third rows are parsed using template `lower(String)` and interpreted together, expression in the forth row is parsed with another template (`upper(String)`). - If `input_format_values_interpret_expressions=1` and `format_values_deduce_templates_of_expressions=1`, the same as in previous case, but also allows fallback to interpreting expressions separately if it’s not possible to deduce template. -## input\_format\_values\_accurate\_types\_of\_literals {#settings-input-format-values-accurate-types-of-literals} +## input_format_values_accurate_types_of_literals {#settings-input-format-values-accurate-types-of-literals} This setting is used only when `input_format_values_deduce_templates_of_expressions = 1`. Expressions for some column may have the same structure, but contain numeric literals of different types, e.g. @@ -285,7 +285,7 @@ Possible values: Default value: 1. -## input\_format\_defaults\_for\_omitted\_fields {#session_settings-input_format_defaults_for_omitted_fields} +## input_format_defaults_for_omitted_fields {#session_settings-input_format_defaults_for_omitted_fields} When performing `INSERT` queries, replace omitted input column values with default values of the respective columns. This option only applies to [JSONEachRow](../../interfaces/formats.md#jsoneachrow), [CSV](../../interfaces/formats.md#csv) and [TabSeparated](../../interfaces/formats.md#tabseparated) formats. @@ -299,17 +299,17 @@ Possible values: Default value: 1. -## input\_format\_tsv\_empty\_as\_default {#settings-input-format-tsv-empty-as-default} +## input_format_tsv_empty_as_default {#settings-input-format-tsv-empty-as-default} When enabled, replace empty input fields in TSV with default values. For complex default expressions `input_format_defaults_for_omitted_fields` must be enabled too. Disabled by default. -## input\_format\_null\_as\_default {#settings-input-format-null-as-default} +## input_format_null_as_default {#settings-input-format-null-as-default} Enables or disables using default values if input data contain `NULL`, but the data type of the corresponding column in not `Nullable(T)` (for text input formats). -## input\_format\_skip\_unknown\_fields {#settings-input-format-skip-unknown-fields} +## input_format_skip_unknown_fields {#settings-input-format-skip-unknown-fields} Enables or disables skipping insertion of extra data. @@ -329,7 +329,7 @@ Possible values: Default value: 0. -## input\_format\_import\_nested\_json {#settings-input_format_import_nested_json} +## input_format_import_nested_json {#settings-input_format_import_nested_json} Enables or disables the insertion of JSON data with nested objects. @@ -348,7 +348,7 @@ See also: - [Usage of Nested Structures](../../interfaces/formats.md#jsoneachrow-nested) with the `JSONEachRow` format. -## input\_format\_with\_names\_use\_header {#settings-input-format-with-names-use-header} +## input_format_with_names_use_header {#settings-input-format-with-names-use-header} Enables or disables checking the column order when inserting data. @@ -366,7 +366,7 @@ Possible values: Default value: 1. -## date\_time\_input\_format {#settings-date_time_input_format} +## date_time_input_format {#settings-date_time_input_format} Allows choosing a parser of the text representation of date and time. @@ -389,7 +389,7 @@ See also: - [DateTime data type.](../../sql-reference/data-types/datetime.md) - [Functions for working with dates and times.](../../sql-reference/functions/date-time-functions.md) -## join\_default\_strictness {#settings-join_default_strictness} +## join_default_strictness {#settings-join_default_strictness} Sets default strictness for [JOIN clauses](../../sql-reference/statements/select/join.md#select-join). @@ -402,7 +402,7 @@ Possible values: Default value: `ALL`. -## join\_any\_take\_last\_row {#settings-join_any_take_last_row} +## join_any_take_last_row {#settings-join_any_take_last_row} Changes behaviour of join operations with `ANY` strictness. @@ -420,9 +420,9 @@ See also: - [JOIN clause](../../sql-reference/statements/select/join.md#select-join) - [Join table engine](../../engines/table-engines/special/join.md) -- [join\_default\_strictness](#settings-join_default_strictness) +- [join_default_strictness](#settings-join_default_strictness) -## join\_use\_nulls {#join_use_nulls} +## join_use_nulls {#join_use_nulls} Sets the type of [JOIN](../../sql-reference/statements/select/join.md) behaviour. When merging tables, empty cells may appear. ClickHouse fills them differently based on this setting. @@ -433,7 +433,7 @@ Possible values: Default value: 0. -## partial\_merge\_join\_optimizations {#partial_merge_join_optimizations} +## partial_merge_join_optimizations {#partial_merge_join_optimizations} Disables optimizations in partial merge join algorithm for [JOIN](../../sql-reference/statements/select/join.md) queries. @@ -446,7 +446,7 @@ Possible values: Default value: 1. -## partial\_merge\_join\_rows\_in\_right\_blocks {#partial_merge_join_rows_in_right_blocks} +## partial_merge_join_rows_in_right_blocks {#partial_merge_join_rows_in_right_blocks} Limits sizes of right-hand join data blocks in partial merge join algorithm for [JOIN](../../sql-reference/statements/select/join.md) queries. @@ -462,7 +462,7 @@ Possible values: Default value: 65536. -## join\_on\_disk\_max\_files\_to\_merge {#join_on_disk_max_files_to_merge} +## join_on_disk_max_files_to_merge {#join_on_disk_max_files_to_merge} Limits the number of files allowed for parallel sorting in MergeJoin operations when they are executed on disk. @@ -474,7 +474,7 @@ Possible values: Default value: 64. -## any\_join\_distinct\_right\_table\_keys {#any_join_distinct_right_table_keys} +## any_join_distinct_right_table_keys {#any_join_distinct_right_table_keys} Enables legacy ClickHouse server behaviour in `ANY INNER|LEFT JOIN` operations. @@ -502,7 +502,7 @@ See also: - [JOIN strictness](../../sql-reference/statements/select/join.md#join-settings) -## temporary\_files\_codec {#temporary_files_codec} +## temporary_files_codec {#temporary_files_codec} Sets compression codec for temporary files used in sorting and joining operations on disk. @@ -513,7 +513,7 @@ Possible values: Default value: LZ4. -## max\_block\_size {#setting-max_block_size} +## max_block_size {#setting-max_block_size} In ClickHouse, data is processed by blocks (sets of column parts). The internal processing cycles for a single block are efficient enough, but there are noticeable expenditures on each block. The `max_block_size` setting is a recommendation for what size of the block (in a count of rows) to load from tables. The block size shouldn’t be too small, so that the expenditures on each block are still noticeable, but not too large so that the query with LIMIT that is completed after the first block is processed quickly. The goal is to avoid consuming too much memory when extracting a large number of columns in multiple threads and to preserve at least some cache locality. @@ -521,13 +521,13 @@ Default value: 65,536. Blocks the size of `max_block_size` are not always loaded from the table. If it is obvious that less data needs to be retrieved, a smaller block is processed. -## preferred\_block\_size\_bytes {#preferred-block-size-bytes} +## preferred_block_size_bytes {#preferred-block-size-bytes} Used for the same purpose as `max_block_size`, but it sets the recommended block size in bytes by adapting it to the number of rows in the block. However, the block size cannot be more than `max_block_size` rows. By default: 1,000,000. It only works when reading from MergeTree engines. -## merge\_tree\_min\_rows\_for\_concurrent\_read {#setting-merge-tree-min-rows-for-concurrent-read} +## merge_tree_min_rows_for_concurrent_read {#setting-merge-tree-min-rows-for-concurrent-read} If the number of rows to be read from a file of a [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table exceeds `merge_tree_min_rows_for_concurrent_read` then ClickHouse tries to perform a concurrent reading from this file on several threads. @@ -537,7 +537,7 @@ Possible values: Default value: 163840. -## merge\_tree\_min\_bytes\_for\_concurrent\_read {#setting-merge-tree-min-bytes-for-concurrent-read} +## merge_tree_min_bytes_for_concurrent_read {#setting-merge-tree-min-bytes-for-concurrent-read} If the number of bytes to read from one file of a [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md)-engine table exceeds `merge_tree_min_bytes_for_concurrent_read`, then ClickHouse tries to concurrently read from this file in several threads. @@ -547,7 +547,7 @@ Possible value: Default value: 251658240. -## merge\_tree\_min\_rows\_for\_seek {#setting-merge-tree-min-rows-for-seek} +## merge_tree_min_rows_for_seek {#setting-merge-tree-min-rows-for-seek} If the distance between two data blocks to be read in one file is less than `merge_tree_min_rows_for_seek` rows, then ClickHouse does not seek through the file but reads the data sequentially. @@ -557,7 +557,7 @@ Possible values: Default value: 0. -## merge\_tree\_min\_bytes\_for\_seek {#setting-merge-tree-min-bytes-for-seek} +## merge_tree_min_bytes_for_seek {#setting-merge-tree-min-bytes-for-seek} If the distance between two data blocks to be read in one file is less than `merge_tree_min_bytes_for_seek` bytes, then ClickHouse sequentially reads a range of file that contains both blocks, thus avoiding extra seek. @@ -567,7 +567,7 @@ Possible values: Default value: 0. -## merge\_tree\_coarse\_index\_granularity {#setting-merge-tree-coarse-index-granularity} +## merge_tree_coarse_index_granularity {#setting-merge-tree-coarse-index-granularity} When searching for data, ClickHouse checks the data marks in the index file. If ClickHouse finds that required keys are in some range, it divides this range into `merge_tree_coarse_index_granularity` subranges and searches the required keys there recursively. @@ -577,11 +577,11 @@ Possible values: Default value: 8. -## merge\_tree\_max\_rows\_to\_use\_cache {#setting-merge-tree-max-rows-to-use-cache} +## merge_tree_max_rows_to_use_cache {#setting-merge-tree-max-rows-to-use-cache} If ClickHouse should read more than `merge_tree_max_rows_to_use_cache` rows in one query, it doesn’t use the cache of uncompressed blocks. -The cache of uncompressed blocks stores data extracted for queries. ClickHouse uses this cache to speed up responses to repeated small queries. This setting protects the cache from trashing by queries that read a large amount of data. The [uncompressed\_cache\_size](../../operations/server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) server setting defines the size of the cache of uncompressed blocks. +The cache of uncompressed blocks stores data extracted for queries. ClickHouse uses this cache to speed up responses to repeated small queries. This setting protects the cache from trashing by queries that read a large amount of data. The [uncompressed_cache_size](../../operations/server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) server setting defines the size of the cache of uncompressed blocks. Possible values: @@ -589,11 +589,11 @@ Possible values: Default value: 128 ✕ 8192. -## merge\_tree\_max\_bytes\_to\_use\_cache {#setting-merge-tree-max-bytes-to-use-cache} +## merge_tree_max_bytes_to_use_cache {#setting-merge-tree-max-bytes-to-use-cache} If ClickHouse should read more than `merge_tree_max_bytes_to_use_cache` bytes in one query, it doesn’t use the cache of uncompressed blocks. -The cache of uncompressed blocks stores data extracted for queries. ClickHouse uses this cache to speed up responses to repeated small queries. This setting protects the cache from trashing by queries that read a large amount of data. The [uncompressed\_cache\_size](../../operations/server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) server setting defines the size of the cache of uncompressed blocks. +The cache of uncompressed blocks stores data extracted for queries. ClickHouse uses this cache to speed up responses to repeated small queries. This setting protects the cache from trashing by queries that read a large amount of data. The [uncompressed_cache_size](../../operations/server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) server setting defines the size of the cache of uncompressed blocks. Possible value: @@ -601,7 +601,7 @@ Possible value: Default value: 2013265920. -## min\_bytes\_to\_use\_direct\_io {#settings-min-bytes-to-use-direct-io} +## min_bytes_to_use_direct_io {#settings-min-bytes-to-use-direct-io} The minimum data volume required for using direct I/O access to the storage disk. @@ -639,11 +639,11 @@ Possible values: Default value: `1`. -## log\_queries {#settings-log-queries} +## log_queries {#settings-log-queries} Setting up query logging. -Queries sent to ClickHouse with this setup are logged according to the rules in the [query\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query-log) server configuration parameter. +Queries sent to ClickHouse with this setup are logged according to the rules in the [query_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query-log) server configuration parameter. Example: @@ -651,7 +651,7 @@ Example: log_queries=1 ``` -## log\_queries\_min\_type {#settings-log-queries-min-type} +## log_queries_min_type {#settings-log-queries-min-type} `query_log` minimal type to log. @@ -669,11 +669,11 @@ Can be used to limit which entities will go to `query_log`, say you are interest log_queries_min_type='EXCEPTION_WHILE_PROCESSING' ``` -## log\_query\_threads {#settings-log-query-threads} +## log_query_threads {#settings-log-query-threads} Setting up query threads logging. -Queries’ threads runned by ClickHouse with this setup are logged according to the rules in the [query\_thread\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) server configuration parameter. +Queries’ threads runned by ClickHouse with this setup are logged according to the rules in the [query_thread_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) server configuration parameter. Example: @@ -681,19 +681,19 @@ Example: log_query_threads=1 ``` -## max\_insert\_block\_size {#settings-max_insert_block_size} +## max_insert_block_size {#settings-max_insert_block_size} The size of blocks (in a count of rows) to form for insertion into a table. This setting only applies in cases when the server forms the blocks. For example, for an INSERT via the HTTP interface, the server parses the data format and forms blocks of the specified size. -But when using clickhouse-client, the client parses the data itself, and the ‘max\_insert\_block\_size’ setting on the server doesn’t affect the size of the inserted blocks. +But when using clickhouse-client, the client parses the data itself, and the ‘max_insert_block_size’ setting on the server doesn’t affect the size of the inserted blocks. The setting also doesn’t have a purpose when using INSERT SELECT, since data is inserted using the same blocks that are formed after SELECT. Default value: 1,048,576. The default is slightly more than `max_block_size`. The reason for this is because certain table engines (`*MergeTree`) form a data part on the disk for each inserted block, which is a fairly large entity. Similarly, `*MergeTree` tables sort data during insertion, and a large enough block size allow sorting more data in RAM. -## min\_insert\_block\_size\_rows {#min-insert-block-size-rows} +## min_insert_block_size_rows {#min-insert-block-size-rows} Sets the minimum number of rows in the block which can be inserted into a table by an `INSERT` query. Smaller-sized blocks are squashed into bigger ones. @@ -704,7 +704,7 @@ Possible values: Default value: 1048576. -## min\_insert\_block\_size\_bytes {#min-insert-block-size-bytes} +## min_insert_block_size_bytes {#min-insert-block-size-bytes} Sets the minimum number of bytes in the block which can be inserted into a table by an `INSERT` query. Smaller-sized blocks are squashed into bigger ones. @@ -715,7 +715,7 @@ Possible values: Default value: 268435456. -## max\_replica\_delay\_for\_distributed\_queries {#settings-max_replica_delay_for_distributed_queries} +## max_replica_delay_for_distributed_queries {#settings-max_replica_delay_for_distributed_queries} Disables lagging replicas for distributed queries. See [Replication](../../engines/table-engines/mergetree-family/replication.md). @@ -725,22 +725,22 @@ Default value: 300. Used when performing `SELECT` from a distributed table that points to replicated tables. -## max\_threads {#settings-max_threads} +## max_threads {#settings-max_threads} -The maximum number of query processing threads, excluding threads for retrieving data from remote servers (see the ‘max\_distributed\_connections’ parameter). +The maximum number of query processing threads, excluding threads for retrieving data from remote servers (see the ‘max_distributed_connections’ parameter). This parameter applies to threads that perform the same stages of the query processing pipeline in parallel. -For example, when reading from a table, if it is possible to evaluate expressions with functions, filter with WHERE and pre-aggregate for GROUP BY in parallel using at least ‘max\_threads’ number of threads, then ‘max\_threads’ are used. +For example, when reading from a table, if it is possible to evaluate expressions with functions, filter with WHERE and pre-aggregate for GROUP BY in parallel using at least ‘max_threads’ number of threads, then ‘max_threads’ are used. Default value: the number of physical CPU cores. If less than one SELECT query is normally run on a server at a time, set this parameter to a value slightly less than the actual number of processor cores. -For queries that are completed quickly because of a LIMIT, you can set a lower ‘max\_threads’. For example, if the necessary number of entries are located in every block and max\_threads = 8, then 8 blocks are retrieved, although it would have been enough to read just one. +For queries that are completed quickly because of a LIMIT, you can set a lower ‘max_threads’. For example, if the necessary number of entries are located in every block and max_threads = 8, then 8 blocks are retrieved, although it would have been enough to read just one. The smaller the `max_threads` value, the less memory is consumed. -## max\_insert\_threads {#settings-max-insert-threads} +## max_insert_threads {#settings-max-insert-threads} The maximum number of threads to execute the `INSERT SELECT` query. @@ -751,37 +751,37 @@ Possible values: Default value: 0. -Parallel `INSERT SELECT` has effect only if the `SELECT` part is executed in parallel, see [max\_threads](#settings-max_threads) setting. +Parallel `INSERT SELECT` has effect only if the `SELECT` part is executed in parallel, see [max_threads](#settings-max_threads) setting. Higher values will lead to higher memory usage. -## max\_compress\_block\_size {#max-compress-block-size} +## max_compress_block_size {#max-compress-block-size} The maximum size of blocks of uncompressed data before compressing for writing to a table. By default, 1,048,576 (1 MiB). If the size is reduced, the compression rate is significantly reduced, the compression and decompression speed increases slightly due to cache locality, and memory consumption is reduced. There usually isn’t any reason to change this setting. Don’t confuse blocks for compression (a chunk of memory consisting of bytes) with blocks for query processing (a set of rows from a table). -## min\_compress\_block\_size {#min-compress-block-size} +## min_compress_block_size {#min-compress-block-size} -For [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md)" tables. In order to reduce latency when processing queries, a block is compressed when writing the next mark if its size is at least ‘min\_compress\_block\_size’. By default, 65,536. +For [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md)" tables. In order to reduce latency when processing queries, a block is compressed when writing the next mark if its size is at least ‘min_compress_block_size’. By default, 65,536. -The actual size of the block, if the uncompressed data is less than ‘max\_compress\_block\_size’, is no less than this value and no less than the volume of data for one mark. +The actual size of the block, if the uncompressed data is less than ‘max_compress_block_size’, is no less than this value and no less than the volume of data for one mark. -Let’s look at an example. Assume that ‘index\_granularity’ was set to 8192 during table creation. +Let’s look at an example. Assume that ‘index_granularity’ was set to 8192 during table creation. -We are writing a UInt32-type column (4 bytes per value). When writing 8192 rows, the total will be 32 KB of data. Since min\_compress\_block\_size = 65,536, a compressed block will be formed for every two marks. +We are writing a UInt32-type column (4 bytes per value). When writing 8192 rows, the total will be 32 KB of data. Since min_compress_block_size = 65,536, a compressed block will be formed for every two marks. We are writing a URL column with the String type (average size of 60 bytes per value). When writing 8192 rows, the average will be slightly less than 500 KB of data. Since this is more than 65,536, a compressed block will be formed for each mark. In this case, when reading data from the disk in the range of a single mark, extra data won’t be decompressed. There usually isn’t any reason to change this setting. -## max\_query\_size {#settings-max_query_size} +## max_query_size {#settings-max_query_size} The maximum part of a query that can be taken to RAM for parsing with the SQL parser. The INSERT query also contains data for INSERT that is processed by a separate stream parser (that consumes O(1) RAM), which is not included in this restriction. Default value: 256 KiB. -## max\_parser\_depth {#max_parser_depth} +## max_parser_depth {#max_parser_depth} Limits maximum recursion depth in the recursive descent parser. Allows controlling the stack size. @@ -792,31 +792,31 @@ Possible values: Default value: 1000. -## interactive\_delay {#interactive-delay} +## interactive_delay {#interactive-delay} The interval in microseconds for checking whether request execution has been cancelled and sending the progress. Default value: 100,000 (checks for cancelling and sends the progress ten times per second). -## connect\_timeout, receive\_timeout, send\_timeout {#connect-timeout-receive-timeout-send-timeout} +## connect_timeout, receive_timeout, send_timeout {#connect-timeout-receive-timeout-send-timeout} Timeouts in seconds on the socket used for communicating with the client. Default value: 10, 300, 300. -## cancel\_http\_readonly\_queries\_on\_client\_close {#cancel-http-readonly-queries-on-client-close} +## cancel_http_readonly_queries_on_client_close {#cancel-http-readonly-queries-on-client-close} Cancels HTTP read-only queries (e.g. SELECT) when a client closes the connection without waiting for the response. Default value: 0 -## poll\_interval {#poll-interval} +## poll_interval {#poll-interval} Lock in a wait loop for the specified number of seconds. Default value: 10. -## max\_distributed\_connections {#max-distributed-connections} +## max_distributed_connections {#max-distributed-connections} The maximum number of simultaneous connections with remote servers for distributed processing of a single query to a single Distributed table. We recommend setting a value no less than the number of servers in the cluster. @@ -824,20 +824,20 @@ Default value: 1024. The following parameters are only used when creating Distributed tables (and when launching a server), so there is no reason to change them at runtime. -## distributed\_connections\_pool\_size {#distributed-connections-pool-size} +## distributed_connections_pool_size {#distributed-connections-pool-size} The maximum number of simultaneous connections with remote servers for distributed processing of all queries to a single Distributed table. We recommend setting a value no less than the number of servers in the cluster. Default value: 1024. -## connect\_timeout\_with\_failover\_ms {#connect-timeout-with-failover-ms} +## connect_timeout_with_failover_ms {#connect-timeout-with-failover-ms} The timeout in milliseconds for connecting to a remote server for a Distributed table engine, if the ‘shard’ and ‘replica’ sections are used in the cluster definition. If unsuccessful, several attempts are made to connect to various replicas. Default value: 50. -## connection\_pool\_max\_wait\_ms {#connection-pool-max-wait-ms} +## connection_pool_max_wait_ms {#connection-pool-max-wait-ms} The wait time in milliseconds for a connection when the connection pool is full. @@ -848,7 +848,7 @@ Possible values: Default value: 0. -## connections\_with\_failover\_max\_tries {#connections-with-failover-max-tries} +## connections_with_failover_max_tries {#connections-with-failover-max-tries} The maximum number of connection attempts with each replica for the Distributed table engine. @@ -859,7 +859,7 @@ Default value: 3. Whether to count extreme values (the minimums and maximums in columns of a query result). Accepts 0 or 1. By default, 0 (disabled). For more information, see the section “Extreme values”. -## kafka\_max\_wait\_ms {#kafka-max-wait-ms} +## kafka_max_wait_ms {#kafka-max-wait-ms} The wait time in milliseconds for reading messages from [Kafka](../../engines/table-engines/integrations/kafka.md#kafka) before retry. @@ -874,25 +874,25 @@ See also: - [Apache Kafka](https://kafka.apache.org/) -## use\_uncompressed\_cache {#setting-use_uncompressed_cache} +## use_uncompressed_cache {#setting-use_uncompressed_cache} Whether to use a cache of uncompressed blocks. Accepts 0 or 1. By default, 0 (disabled). -Using the uncompressed cache (only for tables in the MergeTree family) can significantly reduce latency and increase throughput when working with a large number of short queries. Enable this setting for users who send frequent short requests. Also pay attention to the [uncompressed\_cache\_size](../../operations/server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuration parameter (only set in the config file) – the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed and the least-used data is automatically deleted. +Using the uncompressed cache (only for tables in the MergeTree family) can significantly reduce latency and increase throughput when working with a large number of short queries. Enable this setting for users who send frequent short requests. Also pay attention to the [uncompressed_cache_size](../../operations/server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuration parameter (only set in the config file) – the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed and the least-used data is automatically deleted. -For queries that read at least a somewhat large volume of data (one million rows or more), the uncompressed cache is disabled automatically to save space for truly small queries. This means that you can keep the ‘use\_uncompressed\_cache’ setting always set to 1. +For queries that read at least a somewhat large volume of data (one million rows or more), the uncompressed cache is disabled automatically to save space for truly small queries. This means that you can keep the ‘use_uncompressed_cache’ setting always set to 1. -## replace\_running\_query {#replace-running-query} +## replace_running_query {#replace-running-query} -When using the HTTP interface, the ‘query\_id’ parameter can be passed. This is any string that serves as the query identifier. -If a query from the same user with the same ‘query\_id’ already exists at this time, the behaviour depends on the ‘replace\_running\_query’ parameter. +When using the HTTP interface, the ‘query_id’ parameter can be passed. This is any string that serves as the query identifier. +If a query from the same user with the same ‘query_id’ already exists at this time, the behaviour depends on the ‘replace_running_query’ parameter. -`0` (default) – Throw an exception (don’t allow the query to run if a query with the same ‘query\_id’ is already running). +`0` (default) – Throw an exception (don’t allow the query to run if a query with the same ‘query_id’ is already running). `1` – Cancel the old query and start running the new one. Yandex.Metrica uses this parameter set to 1 for implementing suggestions for segmentation conditions. After entering the next character, if the old query hasn’t finished yet, it should be cancelled. -## replace\_running\_query\_max\_wait\_ms {#replace-running-query-max-wait-ms} +## replace_running_query_max_wait_ms {#replace-running-query-max-wait-ms} The wait time for running the query with the same `query_id` to finish, when the [replace_running_query](#replace-running-query) setting is active. @@ -903,15 +903,15 @@ Possible values: Default value: 5000. -## stream\_flush\_interval\_ms {#stream-flush-interval-ms} +## stream_flush_interval_ms {#stream-flush-interval-ms} -Works for tables with streaming in the case of a timeout, or when a thread generates [max\_insert\_block\_size](#settings-max_insert_block_size) rows. +Works for tables with streaming in the case of a timeout, or when a thread generates [max_insert_block_size](#settings-max_insert_block_size) rows. The default value is 7500. The smaller the value, the more often data is flushed into the table. Setting the value too low leads to poor performance. -## load\_balancing {#settings-load_balancing} +## load_balancing {#settings-load_balancing} Specifies the algorithm of replicas selection that is used for distributed query processing. @@ -925,7 +925,7 @@ ClickHouse supports the following algorithms of choosing replicas: See also: -- [distributed\_replica\_max\_ignored\_errors](#settings-distributed_replica_max_ignored_errors) +- [distributed_replica_max_ignored_errors](#settings-distributed_replica_max_ignored_errors) ### Random (by Default) {#load_balancing-random} @@ -979,31 +979,31 @@ load_balancing = round_robin This algorithm uses a round-robin policy across replicas with the same number of errors (only the queries with `round_robin` policy is accounted). -## prefer\_localhost\_replica {#settings-prefer-localhost-replica} +## prefer_localhost_replica {#settings-prefer-localhost-replica} Enables/disables preferable using the localhost replica when processing distributed queries. Possible values: - 1 — ClickHouse always sends a query to the localhost replica if it exists. -- 0 — ClickHouse uses the balancing strategy specified by the [load\_balancing](#settings-load_balancing) setting. +- 0 — ClickHouse uses the balancing strategy specified by the [load_balancing](#settings-load_balancing) setting. Default value: 1. !!! warning "Warning" - Disable this setting if you use [max\_parallel\_replicas](#settings-max_parallel_replicas). + Disable this setting if you use [max_parallel_replicas](#settings-max_parallel_replicas). -## totals\_mode {#totals-mode} +## totals_mode {#totals-mode} -How to calculate TOTALS when HAVING is present, as well as when max\_rows\_to\_group\_by and group\_by\_overflow\_mode = ‘any’ are present. +How to calculate TOTALS when HAVING is present, as well as when max_rows_to_group_by and group_by_overflow_mode = ‘any’ are present. See the section “WITH TOTALS modifier”. -## totals\_auto\_threshold {#totals-auto-threshold} +## totals_auto_threshold {#totals-auto-threshold} The threshold for `totals_mode = 'auto'`. See the section “WITH TOTALS modifier”. -## max\_parallel\_replicas {#settings-max_parallel_replicas} +## max_parallel_replicas {#settings-max_parallel_replicas} The maximum number of replicas for each shard when executing a query. For consistency (to get different parts of the same data split), this option only works when the sampling key is set. @@ -1016,7 +1016,7 @@ Enable compilation of queries. By default, 0 (disabled). The compilation is only used for part of the query-processing pipeline: for the first stage of aggregation (GROUP BY). If this portion of the pipeline was compiled, the query may run faster due to the deployment of short cycles and inlining aggregate function calls. The maximum performance improvement (up to four times faster in rare cases) is seen for queries with multiple simple aggregate functions. Typically, the performance gain is insignificant. In very rare cases, it may slow down query execution. -## min\_count\_to\_compile {#min-count-to-compile} +## min_count_to_compile {#min-count-to-compile} How many times to potentially use a compiled chunk of code before running compilation. By default, 3. For testing, the value can be set to 0: compilation runs synchronously and the query waits for the end of the compilation process before continuing execution. For all other cases, use values ​​starting with 1. Compilation normally takes about 5-10 seconds. @@ -1025,11 +1025,11 @@ If the value is 1 or more, compilation occurs asynchronously in a separate threa Compiled code is required for each different combination of aggregate functions used in the query and the type of keys in the GROUP BY clause. The results of the compilation are saved in the build directory in the form of .so files. There is no restriction on the number of compilation results since they don’t use very much space. Old results will be used after server restarts, except in the case of a server upgrade – in this case, the old results are deleted. -## output\_format\_json\_quote\_64bit\_integers {#session_settings-output_format_json_quote_64bit_integers} +## output_format_json_quote_64bit_integers {#session_settings-output_format_json_quote_64bit_integers} If the value is true, integers appear in quotes when using JSON\* Int64 and UInt64 formats (for compatibility with most JavaScript implementations); otherwise, integers are output without the quotes. -## output\_format\_json\_quote\_denormals {#settings-output_format_json_quote_denormals} +## output_format_json_quote_denormals {#settings-output_format_json_quote_denormals} Enables `+nan`, `-nan`, `+inf`, `-inf` outputs in [JSON](../../interfaces/formats.md#json) output format. @@ -1128,23 +1128,23 @@ When `output_format_json_quote_denormals = 1`, the query returns: } ``` -## format\_csv\_delimiter {#settings-format_csv_delimiter} +## format_csv_delimiter {#settings-format_csv_delimiter} The character is interpreted as a delimiter in the CSV data. By default, the delimiter is `,`. -## input\_format\_csv\_unquoted\_null\_literal\_as\_null {#settings-input_format_csv_unquoted_null_literal_as_null} +## input_format_csv_unquoted_null_literal_as_null {#settings-input_format_csv_unquoted_null_literal_as_null} For CSV input format enables or disables parsing of unquoted `NULL` as literal (synonym for `\N`). -## output\_format\_csv\_crlf\_end\_of\_line {#settings-output-format-csv-crlf-end-of-line} +## output_format_csv_crlf_end_of_line {#settings-output-format-csv-crlf-end-of-line} Use DOS/Windows-style line separator (CRLF) in CSV instead of Unix style (LF). -## output\_format\_tsv\_crlf\_end\_of\_line {#settings-output-format-tsv-crlf-end-of-line} +## output_format_tsv_crlf_end_of_line {#settings-output-format-tsv-crlf-end-of-line} Use DOC/Windows-style line separator (CRLF) in TSV instead of Unix style (LF). -## insert\_quorum {#settings-insert_quorum} +## insert_quorum {#settings-insert_quorum} Enables the quorum writes. @@ -1159,7 +1159,7 @@ Quorum writes All the replicas in the quorum are consistent, i.e., they contain data from all previous `INSERT` queries. The `INSERT` sequence is linearized. -When reading the data written from the `insert_quorum`, you can use the [select\_sequential\_consistency](#settings-select_sequential_consistency) option. +When reading the data written from the `insert_quorum`, you can use the [select_sequential_consistency](#settings-select_sequential_consistency) option. ClickHouse generates an exception @@ -1168,10 +1168,10 @@ ClickHouse generates an exception See also: -- [insert\_quorum\_timeout](#settings-insert_quorum_timeout) -- [select\_sequential\_consistency](#settings-select_sequential_consistency) +- [insert_quorum_timeout](#settings-insert_quorum_timeout) +- [select_sequential_consistency](#settings-select_sequential_consistency) -## insert\_quorum\_timeout {#settings-insert_quorum_timeout} +## insert_quorum_timeout {#settings-insert_quorum_timeout} Write to a quorum timeout in milliseconds. If the timeout has passed and no write has taken place yet, ClickHouse will generate an exception and the client must repeat the query to write the same block to the same or any other replica. @@ -1179,10 +1179,10 @@ Default value: 600000 milliseconds (ten minutes). See also: -- [insert\_quorum](#settings-insert_quorum) -- [select\_sequential\_consistency](#settings-select_sequential_consistency) +- [insert_quorum](#settings-insert_quorum) +- [select_sequential_consistency](#settings-select_sequential_consistency) -## select\_sequential\_consistency {#settings-select_sequential_consistency} +## select_sequential_consistency {#settings-select_sequential_consistency} Enables or disables sequential consistency for `SELECT` queries: @@ -1199,10 +1199,10 @@ When sequential consistency is enabled, ClickHouse allows the client to execute See also: -- [insert\_quorum](#settings-insert_quorum) -- [insert\_quorum\_timeout](#settings-insert_quorum_timeout) +- [insert_quorum](#settings-insert_quorum) +- [insert_quorum_timeout](#settings-insert_quorum_timeout) -## insert\_deduplicate {#settings-insert-deduplicate} +## insert_deduplicate {#settings-insert-deduplicate} Enables or disables block deduplication of `INSERT` (for Replicated\* tables). @@ -1215,7 +1215,7 @@ Default value: 1. By default, blocks inserted into replicated tables by the `INSERT` statement are deduplicated (see [Data Replication](../../engines/table-engines/mergetree-family/replication.md)). -## deduplicate\_blocks\_in\_dependent\_materialized\_views {#settings-deduplicate-blocks-in-dependent-materialized-views} +## deduplicate_blocks_in_dependent_materialized_views {#settings-deduplicate-blocks-in-dependent-materialized-views} Enables or disables the deduplication check for materialized views that receive data from Replicated\* tables. @@ -1233,7 +1233,7 @@ If an INSERTed block is skipped due to deduplication in the source table, there At the same time, this behaviour “breaks” `INSERT` idempotency. If an `INSERT` into the main table was successful and `INSERT` into a materialized view failed (e.g. because of communication failure with Zookeeper) a client will get an error and can retry the operation. However, the materialized view won’t receive the second insert because it will be discarded by deduplication in the main (source) table. The setting `deduplicate_blocks_in_dependent_materialized_views` allows for changing this behaviour. On retry, a materialized view will receive the repeat insert and will perform a deduplication check by itself, ignoring check result for the source table, and will insert rows lost because of the first failure. -## max\_network\_bytes {#settings-max-network-bytes} +## max_network_bytes {#settings-max-network-bytes} Limits the data volume (in bytes) that is received or transmitted over the network when executing a query. This setting applies to every individual query. @@ -1244,7 +1244,7 @@ Possible values: Default value: 0. -## max\_network\_bandwidth {#settings-max-network-bandwidth} +## max_network_bandwidth {#settings-max-network-bandwidth} Limits the speed of the data exchange over the network in bytes per second. This setting applies to every query. @@ -1255,7 +1255,7 @@ Possible values: Default value: 0. -## max\_network\_bandwidth\_for\_user {#settings-max-network-bandwidth-for-user} +## max_network_bandwidth_for_user {#settings-max-network-bandwidth-for-user} Limits the speed of the data exchange over the network in bytes per second. This setting applies to all concurrently running queries performed by a single user. @@ -1266,7 +1266,7 @@ Possible values: Default value: 0. -## max\_network\_bandwidth\_for\_all\_users {#settings-max-network-bandwidth-for-all-users} +## max_network_bandwidth_for_all_users {#settings-max-network-bandwidth-for-all-users} Limits the speed that data is exchanged at over the network in bytes per second. This setting applies to all concurrently running queries on the server. @@ -1277,7 +1277,7 @@ Possible values: Default value: 0. -## count\_distinct\_implementation {#settings-count_distinct_implementation} +## count_distinct_implementation {#settings-count_distinct_implementation} Specifies which of the `uniq*` functions should be used to perform the [COUNT(DISTINCT …)](../../sql-reference/aggregate-functions/reference/count.md#agg_function-count) construction. @@ -1291,7 +1291,7 @@ Possible values: Default value: `uniqExact`. -## skip\_unavailable\_shards {#settings-skip_unavailable_shards} +## skip_unavailable_shards {#settings-skip_unavailable_shards} Enables or disables silently skipping of unavailable shards. @@ -1321,7 +1321,7 @@ Possible values: Default value: 0. -## distributed\_group\_by\_no\_merge {#distributed-group-by-no-merge} +## distributed_group_by_no_merge {#distributed-group-by-no-merge} Do not merge aggregation states from different servers for distributed query processing, you can use this in case it is for certain that there are different keys on different shards @@ -1362,7 +1362,7 @@ FORMAT PrettyCompactMonoBlock Default value: 0 -## optimize\_skip\_unused\_shards {#optimize-skip-unused-shards} +## optimize_skip_unused_shards {#optimize-skip-unused-shards} Enables or disables skipping of unused shards for [SELECT](../../sql-reference/statements/select/index.md) queries that have sharding key condition in `WHERE/PREWHERE` (assuming that the data is distributed by sharding key, otherwise does nothing). @@ -1373,7 +1373,7 @@ Possible values: Default value: 0 -## optimize\_skip\_unused\_shards\_nesting {#optimize-skip-unused-shards-nesting} +## optimize_skip_unused_shards_nesting {#optimize-skip-unused-shards-nesting} Controls [`optimize_skip_unused_shards`](#optimize-skip-unused-shards) (hence still requires [`optimize_skip_unused_shards`](#optimize-skip-unused-shards)) depends on the nesting level of the distributed query (case when you have `Distributed` table that look into another `Distributed` table). @@ -1385,9 +1385,9 @@ Possible values: Default value: 0 -## force\_optimize\_skip\_unused\_shards {#force-optimize-skip-unused-shards} +## force_optimize_skip_unused_shards {#force-optimize-skip-unused-shards} -Enables or disables query execution if [optimize\_skip\_unused\_shards](#optimize-skip-unused-shards) is enabled and skipping of unused shards is not possible. If the skipping is not possible and the setting is enabled, an exception will be thrown. +Enables or disables query execution if [optimize_skip_unused_shards](#optimize-skip-unused-shards) is enabled and skipping of unused shards is not possible. If the skipping is not possible and the setting is enabled, an exception will be thrown. Possible values: @@ -1397,7 +1397,7 @@ Possible values: Default value: 0 -## force\_optimize\_skip\_unused\_shards\_nesting {#settings-force_optimize_skip_unused_shards_nesting} +## force_optimize_skip_unused_shards_nesting {#settings-force_optimize_skip_unused_shards_nesting} Controls [`force_optimize_skip_unused_shards`](#force-optimize-skip-unused-shards) (hence still requires [`force_optimize_skip_unused_shards`](#force-optimize-skip-unused-shards)) depends on the nesting level of the distributed query (case when you have `Distributed` table that look into another `Distributed` table). @@ -1409,7 +1409,7 @@ Possible values: Default value: 0 -## optimize\_distributed\_group\_by\_sharding\_key {#optimize-distributed-group-by-sharding-key} +## optimize_distributed_group_by_sharding_key {#optimize-distributed-group-by-sharding-key} Optimize `GROUP BY sharding_key` queries, by avoiding costly aggregation on the initiator server (which will reduce memory usage for the query on the initiator server). @@ -1437,13 +1437,13 @@ Default value: 0 See also: -- [distributed\_group\_by\_no\_merge](#distributed-group-by-no-merge) -- [optimize\_skip\_unused\_shards](#optimize-skip-unused-shards) +- [distributed_group_by_no_merge](#distributed-group-by-no-merge) +- [optimize_skip_unused_shards](#optimize-skip-unused-shards) !!! note "Note" Right now it requires `optimize_skip_unused_shards` (the reason behind this is that one day it may be enabled by default, and it will work correctly only if data was inserted via Distributed table, i.e. data is distributed according to sharding_key). -## optimize\_throw\_if\_noop {#setting-optimize_throw_if_noop} +## optimize_throw_if_noop {#setting-optimize_throw_if_noop} Enables or disables throwing an exception if an [OPTIMIZE](../../sql-reference/statements/misc.md#misc_operations-optimize) query didn’t perform a merge. @@ -1456,21 +1456,21 @@ Possible values: Default value: 0. -## distributed\_replica\_error\_half\_life {#settings-distributed_replica_error_half_life} +## distributed_replica_error_half_life {#settings-distributed_replica_error_half_life} - Type: seconds - Default value: 60 seconds -Controls how fast errors in distributed tables are zeroed. If a replica is unavailable for some time, accumulates 5 errors, and distributed\_replica\_error\_half\_life is set to 1 second, then the replica is considered normal 3 seconds after the last error. +Controls how fast errors in distributed tables are zeroed. If a replica is unavailable for some time, accumulates 5 errors, and distributed_replica_error_half_life is set to 1 second, then the replica is considered normal 3 seconds after the last error. See also: -- [load\_balancing](#load_balancing-round_robin) +- [load_balancing](#load_balancing-round_robin) - [Table engine Distributed](../../engines/table-engines/special/distributed.md) -- [distributed\_replica\_error\_cap](#settings-distributed_replica_error_cap) -- [distributed\_replica\_max\_ignored\_errors](#settings-distributed_replica_max_ignored_errors) +- [distributed_replica_error_cap](#settings-distributed_replica_error_cap) +- [distributed_replica_max_ignored_errors](#settings-distributed_replica_max_ignored_errors) -## distributed\_replica\_error\_cap {#settings-distributed_replica_error_cap} +## distributed_replica_error_cap {#settings-distributed_replica_error_cap} - Type: unsigned int - Default value: 1000 @@ -1479,12 +1479,12 @@ The error count of each replica is capped at this value, preventing a single rep See also: -- [load\_balancing](#load_balancing-round_robin) +- [load_balancing](#load_balancing-round_robin) - [Table engine Distributed](../../engines/table-engines/special/distributed.md) -- [distributed\_replica\_error\_half\_life](#settings-distributed_replica_error_half_life) -- [distributed\_replica\_max\_ignored\_errors](#settings-distributed_replica_max_ignored_errors) +- [distributed_replica_error_half_life](#settings-distributed_replica_error_half_life) +- [distributed_replica_max_ignored_errors](#settings-distributed_replica_max_ignored_errors) -## distributed\_replica\_max\_ignored\_errors {#settings-distributed_replica_max_ignored_errors} +## distributed_replica_max_ignored_errors {#settings-distributed_replica_max_ignored_errors} - Type: unsigned int - Default value: 0 @@ -1493,12 +1493,12 @@ The number of errors that will be ignored while choosing replicas (according to See also: -- [load\_balancing](#load_balancing-round_robin) +- [load_balancing](#load_balancing-round_robin) - [Table engine Distributed](../../engines/table-engines/special/distributed.md) -- [distributed\_replica\_error\_cap](#settings-distributed_replica_error_cap) -- [distributed\_replica\_error\_half\_life](#settings-distributed_replica_error_half_life) +- [distributed_replica_error_cap](#settings-distributed_replica_error_cap) +- [distributed_replica_error_half_life](#settings-distributed_replica_error_half_life) -## distributed\_directory\_monitor\_sleep\_time\_ms {#distributed_directory_monitor_sleep_time_ms} +## distributed_directory_monitor_sleep_time_ms {#distributed_directory_monitor_sleep_time_ms} Base interval for the [Distributed](../../engines/table-engines/special/distributed.md) table engine to send data. The actual interval grows exponentially in the event of errors. @@ -1508,9 +1508,9 @@ Possible values: Default value: 100 milliseconds. -## distributed\_directory\_monitor\_max\_sleep\_time\_ms {#distributed_directory_monitor_max_sleep_time_ms} +## distributed_directory_monitor_max_sleep_time_ms {#distributed_directory_monitor_max_sleep_time_ms} -Maximum interval for the [Distributed](../../engines/table-engines/special/distributed.md) table engine to send data. Limits exponential growth of the interval set in the [distributed\_directory\_monitor\_sleep\_time\_ms](#distributed_directory_monitor_sleep_time_ms) setting. +Maximum interval for the [Distributed](../../engines/table-engines/special/distributed.md) table engine to send data. Limits exponential growth of the interval set in the [distributed_directory_monitor_sleep_time_ms](#distributed_directory_monitor_sleep_time_ms) setting. Possible values: @@ -1518,7 +1518,7 @@ Possible values: Default value: 30000 milliseconds (30 seconds). -## distributed\_directory\_monitor\_batch\_inserts {#distributed_directory_monitor_batch_inserts} +## distributed_directory_monitor_batch_inserts {#distributed_directory_monitor_batch_inserts} Enables/disables inserted data sending in batches. @@ -1531,7 +1531,7 @@ Possible values: Default value: 0. -## os\_thread\_priority {#setting-os-thread-priority} +## os_thread_priority {#setting-os-thread-priority} Sets the priority ([nice](https://en.wikipedia.org/wiki/Nice_(Unix))) for threads that execute queries. The OS scheduler considers this priority when choosing the next thread to run on each available CPU core. @@ -1546,7 +1546,7 @@ Lower values mean higher priority. Threads with low `nice` priority values are e Default value: 0. -## query\_profiler\_real\_time\_period\_ns {#query_profiler_real_time_period_ns} +## query_profiler_real_time_period_ns {#query_profiler_real_time_period_ns} Sets the period for a real clock timer of the [query profiler](../../operations/optimizing-performance/sampling-query-profiler.md). Real clock timer counts wall-clock time. @@ -1567,9 +1567,9 @@ Default value: 1000000000 nanoseconds (once a second). See also: -- System table [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) +- System table [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) -## query\_profiler\_cpu\_time\_period\_ns {#query_profiler_cpu_time_period_ns} +## query_profiler_cpu_time_period_ns {#query_profiler_cpu_time_period_ns} Sets the period for a CPU clock timer of the [query profiler](../../operations/optimizing-performance/sampling-query-profiler.md). This timer counts only CPU time. @@ -1590,9 +1590,9 @@ Default value: 1000000000 nanoseconds. See also: -- System table [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) +- System table [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) -## allow\_introspection\_functions {#settings-allow_introspection_functions} +## allow_introspection_functions {#settings-allow_introspection_functions} Enables or disables [introspections functions](../../sql-reference/functions/introspection.md) for query profiling. @@ -1606,23 +1606,23 @@ Default value: 0. **See Also** - [Sampling Query Profiler](../../operations/optimizing-performance/sampling-query-profiler.md) -- System table [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) +- System table [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) -## input\_format\_parallel\_parsing {#input-format-parallel-parsing} +## input_format_parallel_parsing {#input-format-parallel-parsing} - Type: bool - Default value: True Enable order-preserving parallel parsing of data formats. Supported only for TSV, TKSV, CSV, and JSONEachRow formats. -## min\_chunk\_bytes\_for\_parallel\_parsing {#min-chunk-bytes-for-parallel-parsing} +## min_chunk_bytes_for_parallel_parsing {#min-chunk-bytes-for-parallel-parsing} - Type: unsigned int - Default value: 1 MiB The minimum chunk size in bytes, which each thread will parse in parallel. -## output\_format\_avro\_codec {#settings-output_format_avro_codec} +## output_format_avro_codec {#settings-output_format_avro_codec} Sets the compression codec used for output Avro file. @@ -1636,7 +1636,7 @@ Possible values: Default value: `snappy` (if available) or `deflate`. -## output\_format\_avro\_sync\_interval {#settings-output_format_avro_sync_interval} +## output_format_avro_sync_interval {#settings-output_format_avro_sync_interval} Sets minimum data size (in bytes) between synchronization markers for output Avro file. @@ -1646,7 +1646,7 @@ Possible values: 32 (32 bytes) - 1073741824 (1 GiB) Default value: 32768 (32 KiB) -## format\_avro\_schema\_registry\_url {#format_avro_schema_registry_url} +## format_avro_schema_registry_url {#format_avro_schema_registry_url} Sets [Confluent Schema Registry](https://docs.confluent.io/current/schema-registry/index.html) URL to use with [AvroConfluent](../../interfaces/formats.md#data-format-avro-confluent) format. @@ -1663,7 +1663,7 @@ Possible values: Default value: 0. -## background\_pool\_size {#background_pool_size} +## background_pool_size {#background_pool_size} Sets the number of threads performing background operations in table engines (for example, merges in [MergeTree engine](../../engines/table-engines/mergetree-family/index.md) tables). This setting is applied from the `default` profile at the ClickHouse server start and can’t be changed in a user session. By adjusting this setting, you manage CPU and disk load. Smaller pool size utilizes less CPU and disk resources, but background processes advance slower which might eventually impact query performance. @@ -1706,7 +1706,7 @@ Default value: `0`. - [Distributed Table Engine](../../engines/table-engines/special/distributed.md#distributed) - [Managing Distributed Tables](../../sql-reference/statements/system.md#query-language-system-distributed) -## background\_buffer\_flush\_schedule\_pool\_size {#background_buffer_flush_schedule_pool_size} +## background_buffer_flush_schedule_pool_size {#background_buffer_flush_schedule_pool_size} Sets the number of threads performing background flush in [Buffer](../../engines/table-engines/special/buffer.md)-engine tables. This setting is applied at the ClickHouse server start and can’t be changed in a user session. @@ -1716,7 +1716,7 @@ Possible values: Default value: 16. -## background\_move\_pool\_size {#background_move_pool_size} +## background_move_pool_size {#background_move_pool_size} Sets the number of threads performing background moves of data parts for [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes)-engine tables. This setting is applied at the ClickHouse server start and can’t be changed in a user session. @@ -1726,7 +1726,7 @@ Possible values: Default value: 8. -## background\_schedule\_pool\_size {#background_schedule_pool_size} +## background_schedule_pool_size {#background_schedule_pool_size} Sets the number of threads performing background tasks for [replicated](../../engines/table-engines/mergetree-family/replication.md) tables, [Kafka](../../engines/table-engines/integrations/kafka.md) streaming, [DNS cache updates](../../operations/server-configuration-parameters/settings.md#server-settings-dns-cache-update-period). This setting is applied at ClickHouse server start and can’t be changed in a user session. @@ -1736,7 +1736,7 @@ Possible values: Default value: 16. -## always\_fetch\_merged\_part {#always_fetch_merged_part} +## always_fetch_merged_part {#always_fetch_merged_part} Prohibits data parts merging in [Replicated\*MergeTree](../../engines/table-engines/mergetree-family/replication.md)-engine tables. @@ -1753,7 +1753,7 @@ Default value: 0. - [Data Replication](../../engines/table-engines/mergetree-family/replication.md) -## background\_distributed\_schedule\_pool\_size {#background_distributed_schedule_pool_size} +## background_distributed_schedule_pool_size {#background_distributed_schedule_pool_size} Sets the number of threads performing background tasks for [distributed](../../engines/table-engines/special/distributed.md) sends. This setting is applied at the ClickHouse server start and can’t be changed in a user session. @@ -1763,7 +1763,7 @@ Possible values: Default value: 16. -## validate\_polygons {#validate_polygons} +## validate_polygons {#validate_polygons} Enables or disables throwing an exception in the [pointInPolygon](../../sql-reference/functions/geo/index.md#pointinpolygon) function, if the polygon is self-intersecting or self-tangent. @@ -1774,7 +1774,7 @@ Possible values: Default value: 1. -## transform\_null\_in {#transform_null_in} +## transform_null_in {#transform_null_in} Enables equality of [NULL](../../sql-reference/syntax.md#null-literal) values for [IN](../../sql-reference/operators/in.md) operator. @@ -1832,7 +1832,7 @@ Result: - [NULL Processing in IN Operators](../../sql-reference/operators/in.md#in-null-processing) -## low\_cardinality\_max\_dictionary\_size {#low_cardinality_max_dictionary_size} +## low_cardinality_max_dictionary_size {#low_cardinality_max_dictionary_size} Sets a maximum size in rows of a shared global dictionary for the [LowCardinality](../../sql-reference/data-types/lowcardinality.md) data type that can be written to a storage file system. This setting prevents issues with RAM in case of unlimited dictionary growth. All the data that can’t be encoded due to maximum dictionary size limitation ClickHouse writes in an ordinary method. @@ -1842,7 +1842,7 @@ Possible values: Default value: 8192. -## low\_cardinality\_use\_single\_dictionary\_for\_part {#low_cardinality_use_single_dictionary_for_part} +## low_cardinality_use_single_dictionary_for_part {#low_cardinality_use_single_dictionary_for_part} Turns on or turns off using of single dictionary for the data part. @@ -1855,7 +1855,7 @@ Possible values: Default value: 0. -## low\_cardinality\_allow\_in\_native\_format {#low_cardinality_allow_in_native_format} +## low_cardinality_allow_in_native_format {#low_cardinality_allow_in_native_format} Allows or restricts using the [LowCardinality](../../sql-reference/data-types/lowcardinality.md) data type with the [Native](../../interfaces/formats.md#native) format. @@ -1870,7 +1870,7 @@ Possible values: Default value: 1. -## allow\_suspicious\_low\_cardinality\_types {#allow_suspicious_low_cardinality_types} +## allow_suspicious_low_cardinality_types {#allow_suspicious_low_cardinality_types} Allows or restricts using [LowCardinality](../../sql-reference/data-types/lowcardinality.md) with data types with fixed size of 8 bytes or less: numeric data types and `FixedString(8_bytes_or_less)`. @@ -1889,7 +1889,7 @@ Possible values: Default value: 0. -## min\_insert\_block\_size\_rows\_for\_materialized\_views {#min-insert-block-size-rows-for-materialized-views} +## min_insert_block_size_rows_for_materialized_views {#min-insert-block-size-rows-for-materialized-views} Sets the minimum number of rows in the block which can be inserted into a table by an `INSERT` query. Smaller-sized blocks are squashed into bigger ones. This setting is applied only for blocks inserted into [materialized view](../../sql-reference/statements/create/view.md). By adjusting this setting, you control blocks squashing while pushing to materialized view and avoid excessive memory usage. @@ -1902,9 +1902,9 @@ Default value: 1048576. **See Also** -- [min\_insert\_block\_size\_rows](#min-insert-block-size-rows) +- [min_insert_block_size_rows](#min-insert-block-size-rows) -## min\_insert\_block\_size\_bytes\_for\_materialized\_views {#min-insert-block-size-bytes-for-materialized-views} +## min_insert_block_size_bytes_for_materialized_views {#min-insert-block-size-bytes-for-materialized-views} Sets the minimum number of bytes in the block which can be inserted into a table by an `INSERT` query. Smaller-sized blocks are squashed into bigger ones. This setting is applied only for blocks inserted into [materialized view](../../sql-reference/statements/create/view.md). By adjusting this setting, you control blocks squashing while pushing to materialized view and avoid excessive memory usage. @@ -1917,9 +1917,9 @@ Default value: 268435456. **See also** -- [min\_insert\_block\_size\_bytes](#min-insert-block-size-bytes) +- [min_insert_block_size_bytes](#min-insert-block-size-bytes) -## output\_format\_pretty\_grid\_charset {#output-format-pretty-grid-charset} +## output_format_pretty_grid_charset {#output-format-pretty-grid-charset} Allows changing a charset which is used for printing grids borders. Available charsets are UTF-8, ASCII. diff --git a/docs/en/operations/system-tables/asynchronous_metrics.md b/docs/en/operations/system-tables/asynchronous_metrics.md index a401c7a723b..b27434793c7 100644 --- a/docs/en/operations/system-tables/asynchronous_metrics.md +++ b/docs/en/operations/system-tables/asynchronous_metrics.md @@ -33,6 +33,6 @@ SELECT * FROM system.asynchronous_metrics LIMIT 10 - [Monitoring](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring. - [system.metrics](../../operations/system-tables/metrics.md#system_tables-metrics) — Contains instantly calculated metrics. - [system.events](../../operations/system-tables/events.md#system_tables-events) — Contains a number of events that have occurred. -- [system.metric\_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [system.metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. [Original article](https://clickhouse.tech/docs/en/operations/system_tables/asynchronous_metrics) \ No newline at end of file diff --git a/docs/en/operations/system-tables/clusters.md b/docs/en/operations/system-tables/clusters.md index e9721379d7b..f18dfb3d1c0 100644 --- a/docs/en/operations/system-tables/clusters.md +++ b/docs/en/operations/system-tables/clusters.md @@ -20,7 +20,7 @@ Please note that `errors_count` is updated once per query to the cluster, but `e **See also** - [Table engine Distributed](../../engines/table-engines/special/distributed.md) -- [distributed\_replica\_error\_cap setting](../../operations/settings/settings.md#settings-distributed_replica_error_cap) -- [distributed\_replica\_error\_half\_life setting](../../operations/settings/settings.md#settings-distributed_replica_error_half_life) +- [distributed_replica_error_cap setting](../../operations/settings/settings.md#settings-distributed_replica_error_cap) +- [distributed_replica_error_half_life setting](../../operations/settings/settings.md#settings-distributed_replica_error_half_life) [Original article](https://clickhouse.tech/docs/en/operations/system_tables/clusters) diff --git a/docs/en/operations/system-tables/events.md b/docs/en/operations/system-tables/events.md index d23533189c7..b4ced6e6bf6 100644 --- a/docs/en/operations/system-tables/events.md +++ b/docs/en/operations/system-tables/events.md @@ -26,9 +26,9 @@ SELECT * FROM system.events LIMIT 5 **See Also** -- [system.asynchronous\_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. - [system.metrics](../../operations/system-tables/metrics.md#system_tables-metrics) — Contains instantly calculated metrics. -- [system.metric\_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [system.metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. - [Monitoring](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring. [Original article](https://clickhouse.tech/docs/en/operations/system_tables/events) diff --git a/docs/en/operations/system-tables/graphite_retentions.md b/docs/en/operations/system-tables/graphite_retentions.md index f5d65bbe3fe..7ae5e0e36a8 100644 --- a/docs/en/operations/system-tables/graphite_retentions.md +++ b/docs/en/operations/system-tables/graphite_retentions.md @@ -1,6 +1,6 @@ # system.graphite_retentions {#system-graphite-retentions} -Contains information about parameters [graphite\_rollup](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) which are used in tables with [\*GraphiteMergeTree](../../engines/table-engines/mergetree-family/graphitemergetree.md) engines. +Contains information about parameters [graphite_rollup](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) which are used in tables with [\*GraphiteMergeTree](../../engines/table-engines/mergetree-family/graphitemergetree.md) engines. Columns: diff --git a/docs/en/operations/system-tables/index.md b/docs/en/operations/system-tables/index.md index 0af6763ad3b..7a9e386d419 100644 --- a/docs/en/operations/system-tables/index.md +++ b/docs/en/operations/system-tables/index.md @@ -20,7 +20,7 @@ System tables: Most of system tables store their data in RAM. A ClickHouse server creates such system tables at the start. -Unlike other system tables, the system tables [metric\_log](../../operations/system-tables/metric_log.md#system_tables-metric_log), [query\_log](../../operations/system-tables/query_log.md#system_tables-query_log), [query\_thread\_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log), [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) are served by [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table engine and store their data in a storage filesystem. If you remove a table from a filesystem, the ClickHouse server creates the empty one again at the time of the next data writing. If system table schema changed in a new release, then ClickHouse renames the current table and creates a new one. +Unlike other system tables, the system tables [metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log), [query_log](../../operations/system-tables/query_log.md#system_tables-query_log), [query_thread_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log), [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) are served by [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table engine and store their data in a storage filesystem. If you remove a table from a filesystem, the ClickHouse server creates the empty one again at the time of the next data writing. If system table schema changed in a new release, then ClickHouse renames the current table and creates a new one. By default, table growth is unlimited. To control a size of a table, you can use [TTL](../../sql-reference/statements/alter/ttl.md#manipulations-with-table-ttl) settings for removing outdated log records. Also you can use the partitioning feature of `MergeTree`-engine tables. diff --git a/docs/en/operations/system-tables/metric_log.md b/docs/en/operations/system-tables/metric_log.md index 063fe81923b..f1910407949 100644 --- a/docs/en/operations/system-tables/metric_log.md +++ b/docs/en/operations/system-tables/metric_log.md @@ -49,7 +49,7 @@ CurrentMetric_DistributedFilesToInsert: 0 **See also** -- [system.asynchronous\_metrics](../../operations/system-tables/asynchronous_metrics.md) — Contains periodically calculated metrics. +- [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md) — Contains periodically calculated metrics. - [system.events](../../operations/system-tables/events.md#system_tables-events) — Contains a number of events that occurred. - [system.metrics](../../operations/system-tables/metrics.md) — Contains instantly calculated metrics. - [Monitoring](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring. diff --git a/docs/en/operations/system-tables/metrics.md b/docs/en/operations/system-tables/metrics.md index cf4c6efe8d4..decae8ea7fb 100644 --- a/docs/en/operations/system-tables/metrics.md +++ b/docs/en/operations/system-tables/metrics.md @@ -33,9 +33,9 @@ SELECT * FROM system.metrics LIMIT 10 **See Also** -- [system.asynchronous\_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. - [system.events](../../operations/system-tables/events.md#system_tables-events) — Contains a number of events that occurred. -- [system.metric\_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [system.metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. - [Monitoring](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring. [Original article](https://clickhouse.tech/docs/en/operations/system_tables/metrics) diff --git a/docs/en/operations/system-tables/part_log.md b/docs/en/operations/system-tables/part_log.md index e7c157077e3..9aa95b1a493 100644 --- a/docs/en/operations/system-tables/part_log.md +++ b/docs/en/operations/system-tables/part_log.md @@ -1,6 +1,6 @@ # system.part_log {#system_tables-part-log} -The `system.part_log` table is created only if the [part\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-part-log) server setting is specified. +The `system.part_log` table is created only if the [part_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-part-log) server setting is specified. This table contains information about events that occurred with [data parts](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) in the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) family tables, such as adding or merging data. diff --git a/docs/en/operations/system-tables/processes.md b/docs/en/operations/system-tables/processes.md index 0e44c61a4f6..2af39eff862 100644 --- a/docs/en/operations/system-tables/processes.md +++ b/docs/en/operations/system-tables/processes.md @@ -10,7 +10,7 @@ Columns: - `rows_read` (UInt64) – The number of rows read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. - `bytes_read` (UInt64) – The number of uncompressed bytes read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. - `total_rows_approx` (UInt64) – The approximation of the total number of rows that should be read. For distributed processing, on the requestor server, this is the total for all remote servers. It can be updated during request processing, when new sources to process become known. -- `memory_usage` (UInt64) – Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max\_memory\_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) setting. +- `memory_usage` (UInt64) – Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) setting. - `query` (String) – The query text. For `INSERT`, it doesn’t include the data to insert. - `query_id` (String) – Query ID, if defined. diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index 8b663475fa8..ddd5d327d59 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -5,11 +5,11 @@ Contains information about executed queries, for example, start time, duration o !!! note "Note" This table doesn’t contain the ingested data for `INSERT` queries. -You can change settings of queries logging in the [query\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query-log) section of the server configuration. +You can change settings of queries logging in the [query_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query-log) section of the server configuration. -You can disable queries logging by setting [log\_queries = 0](../../operations/settings/settings.md#settings-log-queries). We don’t recommend to turn off logging because information in this table is important for solving issues. +You can disable queries logging by setting [log_queries = 0](../../operations/settings/settings.md#settings-log-queries). We don’t recommend to turn off logging because information in this table is important for solving issues. -The flushing period of data is set in `flush_interval_milliseconds` parameter of the [query\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query-log) server settings section. To force flushing, use the [SYSTEM FLUSH LOGS](../../sql-reference/statements/system.md#query_language-system-flush_logs) query. +The flushing period of data is set in `flush_interval_milliseconds` parameter of the [query_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query-log) server settings section. To force flushing, use the [SYSTEM FLUSH LOGS](../../sql-reference/statements/system.md#query_language-system-flush_logs) query. ClickHouse doesn’t delete data from the table automatically. See [Introduction](../../operations/system-tables/index.md#system-tables-introduction) for more details. @@ -140,7 +140,7 @@ Settings.Values: ['0','random','1','10000000000','1'] **See Also** -- [system.query\_thread\_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log) — This table contains information about each query execution thread. +- [system.query_thread_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log) — This table contains information about each query execution thread. [Original article](https://clickhouse.tech/docs/en/operations/system_tables/query_log) diff --git a/docs/en/operations/system-tables/query_thread_log.md b/docs/en/operations/system-tables/query_thread_log.md index 84408edd117..a6f090e99f3 100644 --- a/docs/en/operations/system-tables/query_thread_log.md +++ b/docs/en/operations/system-tables/query_thread_log.md @@ -4,10 +4,10 @@ Contains information about threads which execute queries, for example, thread na To start logging: -1. Configure parameters in the [query\_thread\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) section. -2. Set [log\_query\_threads](../../operations/settings/settings.md#settings-log-query-threads) to 1. +1. Configure parameters in the [query_thread_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) section. +2. Set [log_query_threads](../../operations/settings/settings.md#settings-log-query-threads) to 1. -The flushing period of data is set in `flush_interval_milliseconds` parameter of the [query\_thread\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) server settings section. To force flushing, use the [SYSTEM FLUSH LOGS](../../sql-reference/statements/system.md#query_language-system-flush_logs) query. +The flushing period of data is set in `flush_interval_milliseconds` parameter of the [query_thread_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) server settings section. To force flushing, use the [SYSTEM FLUSH LOGS](../../sql-reference/statements/system.md#query_language-system-flush_logs) query. ClickHouse doesn’t delete data from the table automatically. See [Introduction](../../operations/system-tables/index.md#system-tables-introduction) for more details. @@ -113,6 +113,6 @@ ProfileEvents.Values: [1,1,11,11,591,148,3,71,29,6533808,1,11,72,18,47, **See Also** -- [system.query\_log](../../operations/system-tables/query_log.md#system_tables-query_log) — Description of the `query_log` system table which contains common information about queries execution. +- [system.query_log](../../operations/system-tables/query_log.md#system_tables-query_log) — Description of the `query_log` system table which contains common information about queries execution. [Original article](https://clickhouse.tech/docs/en/operations/system_tables/query_thread_log) diff --git a/docs/en/operations/system-tables/replicas.md b/docs/en/operations/system-tables/replicas.md index 17519690951..f31f7fd27ea 100644 --- a/docs/en/operations/system-tables/replicas.md +++ b/docs/en/operations/system-tables/replicas.md @@ -63,7 +63,7 @@ Columns: - `parts_to_check` (`UInt32`) - The number of data parts in the queue for verification. A part is put in the verification queue if there is suspicion that it might be damaged. - `zookeeper_path` (`String`) - Path to table data in ZooKeeper. - `replica_name` (`String`) - Replica name in ZooKeeper. Different replicas of the same table have different names. -- `replica_path` (`String`) - Path to replica data in ZooKeeper. The same as concatenating ‘zookeeper\_path/replicas/replica\_path’. +- `replica_path` (`String`) - Path to replica data in ZooKeeper. The same as concatenating ‘zookeeper_path/replicas/replica_path’. - `columns_version` (`Int32`) - Version number of the table structure. Indicates how many times ALTER was performed. If replicas have different versions, it means some replicas haven’t made all of the ALTERs yet. - `queue_size` (`UInt32`) - Size of the queue for operations waiting to be performed. Operations include inserting blocks of data, merges, and certain other actions. It usually coincides with `future_parts`. - `inserts_in_queue` (`UInt32`) - Number of inserts of blocks of data that need to be made. Insertions are usually replicated fairly quickly. If this number is large, it means something is wrong. @@ -84,7 +84,7 @@ The next 4 columns have a non-zero value only where there is an active session w - `active_replicas` (`UInt8`) - The number of replicas of this table that have a session in ZooKeeper (i.e., the number of functioning replicas). If you request all the columns, the table may work a bit slowly, since several reads from ZooKeeper are made for each row. -If you don’t request the last 4 columns (log\_max\_index, log\_pointer, total\_replicas, active\_replicas), the table works quickly. +If you don’t request the last 4 columns (log_max_index, log_pointer, total_replicas, active_replicas), the table works quickly. For example, you can check that everything is working correctly like this: diff --git a/docs/en/operations/system-tables/text_log.md b/docs/en/operations/system-tables/text_log.md index 3c3281ff8c6..34f37c2cae7 100644 --- a/docs/en/operations/system-tables/text_log.md +++ b/docs/en/operations/system-tables/text_log.md @@ -1,4 +1,4 @@ -# system.text\_log {#system_tables-text_log} +# system.text_log {#system_tables-text_log} Contains logging entries. Logging level which goes to this table can be limited with `text_log.level` server setting. diff --git a/docs/en/operations/system-tables/trace_log.md b/docs/en/operations/system-tables/trace_log.md index 0fb967822d2..bc955ebe4de 100644 --- a/docs/en/operations/system-tables/trace_log.md +++ b/docs/en/operations/system-tables/trace_log.md @@ -2,7 +2,7 @@ Contains stack traces collected by the sampling query profiler. -ClickHouse creates this table when the [trace\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) server configuration section is set. Also the [query\_profiler\_real\_time\_period\_ns](../../operations/settings/settings.md#query_profiler_real_time_period_ns) and [query\_profiler\_cpu\_time\_period\_ns](../../operations/settings/settings.md#query_profiler_cpu_time_period_ns) settings should be set. +ClickHouse creates this table when the [trace_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) server configuration section is set. Also the [query_profiler_real_time_period_ns](../../operations/settings/settings.md#query_profiler_real_time_period_ns) and [query_profiler_cpu_time_period_ns](../../operations/settings/settings.md#query_profiler_cpu_time_period_ns) settings should be set. To analyze logs, use the `addressToLine`, `addressToSymbol` and `demangle` introspection functions. @@ -27,7 +27,7 @@ Columns: - `thread_number` ([UInt32](../../sql-reference/data-types/int-uint.md)) — Thread identifier. -- `query_id` ([String](../../sql-reference/data-types/string.md)) — Query identifier that can be used to get details about a query that was running from the [query\_log](#system_tables-query_log) system table. +- `query_id` ([String](../../sql-reference/data-types/string.md)) — Query identifier that can be used to get details about a query that was running from the [query_log](#system_tables-query_log) system table. - `trace` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — Stack trace at the moment of sampling. Each element is a virtual memory address inside ClickHouse server process. diff --git a/docs/en/operations/tips.md b/docs/en/operations/tips.md index 56510ee09cc..024eba1c899 100644 --- a/docs/en/operations/tips.md +++ b/docs/en/operations/tips.md @@ -57,7 +57,7 @@ When creating RAID-10, select the `far` layout. If your budget allows, choose RAID-10. If you have more than 4 disks, use RAID-6 (preferred) or RAID-50, instead of RAID-5. -When using RAID-5, RAID-6 or RAID-50, always increase stripe\_cache\_size, since the default value is usually not the best choice. +When using RAID-5, RAID-6 or RAID-50, always increase stripe_cache_size, since the default value is usually not the best choice. ``` bash $ echo 4096 | sudo tee /sys/block/md2/md/stripe_cache_size diff --git a/docs/en/operations/troubleshooting.md b/docs/en/operations/troubleshooting.md index 6be27f8cb9f..39449afccef 100644 --- a/docs/en/operations/troubleshooting.md +++ b/docs/en/operations/troubleshooting.md @@ -103,7 +103,7 @@ Check: - Endpoint settings. - Check [listen\_host](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-listen_host) and [tcp\_port](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) settings. + Check [listen_host](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-listen_host) and [tcp_port](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) settings. ClickHouse server accepts localhost connections only by default. @@ -115,7 +115,7 @@ Check: Check: - - The [tcp\_port\_secure](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) setting. + - The [tcp_port_secure](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) setting. - Settings for [SSL certificates](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-openssl). Use proper parameters while connecting. For example, use the `port_secure` parameter with `clickhouse_client`. diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index fb9d390d2e7..3b02e145ff4 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -492,6 +492,6 @@ Solution: Write in the GROUP BY query SearchPhrase HAVING uniqUpTo(4)(UserID) >= [Original article](https://clickhouse.tech/docs/en/query_language/agg_functions/parametric_functions/) -## sumMapFiltered(keys\_to\_keep)(keys, values) {#summapfilteredkeys-to-keepkeys-values} +## sumMapFiltered(keys_to_keep)(keys, values) {#summapfilteredkeys-to-keepkeys-values} Same behavior as [sumMap](../../sql-reference/aggregate-functions/reference/summap.md#agg_functions-summap) except that an array of keys is passed as a parameter. This can be especially useful when working with a high cardinality of keys. diff --git a/docs/en/sql-reference/aggregate-functions/reference/count.md b/docs/en/sql-reference/aggregate-functions/reference/count.md index 68ad69fe264..e5d31429e12 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/count.md +++ b/docs/en/sql-reference/aggregate-functions/reference/count.md @@ -26,7 +26,7 @@ In both cases the type of the returned value is [UInt64](../../../sql-reference/ **Details** -ClickHouse supports the `COUNT(DISTINCT ...)` syntax. The behavior of this construction depends on the [count\_distinct\_implementation](../../../operations/settings/settings.md#settings-count_distinct_implementation) setting. It defines which of the [uniq\*](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) functions is used to perform the operation. The default is the [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact) function. +ClickHouse supports the `COUNT(DISTINCT ...)` syntax. The behavior of this construction depends on the [count_distinct_implementation](../../../operations/settings/settings.md#settings-count_distinct_implementation) setting. It defines which of the [uniq\*](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) functions is used to perform the operation. The default is the [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact) function. The `SELECT count() FROM table` query is not optimized, because the number of entries in the table is not stored separately. It chooses a small column from the table and counts the number of values in it. diff --git a/docs/en/sql-reference/ansi.md b/docs/en/sql-reference/ansi.md index 2cd9142c2f9..61d0e274207 100644 --- a/docs/en/sql-reference/ansi.md +++ b/docs/en/sql-reference/ansi.md @@ -34,8 +34,8 @@ The following table lists cases when query feature works in ClickHouse, but beha | E021-01 | CHARACTER data type | No{.text-danger} | | | E021-02 | CHARACTER VARYING data type | No{.text-danger} | `String` behaves similarly, but without length limit in parentheses | | E021-03 | Character literals | Partial{.text-warning} | No automatic concatenation of consecutive literals and character set support | -| E021-04 | CHARACTER\_LENGTH function | Partial{.text-warning} | No `USING` clause | -| E021-05 | OCTET\_LENGTH function | No{.text-danger} | `LENGTH` behaves similarly | +| E021-04 | CHARACTER_LENGTH function | Partial{.text-warning} | No `USING` clause | +| E021-05 | OCTET_LENGTH function | No{.text-danger} | `LENGTH` behaves similarly | | E021-06 | SUBSTRING | Partial{.text-warning} | No support for `SIMILAR` and `ESCAPE` clauses, no `SUBSTRING_REGEX` variant | | E021-07 | Character concatenation | Partial{.text-warning} | No `COLLATE` clause | | E021-08 | UPPER and LOWER functions | Yes{.text-success} | | @@ -152,7 +152,7 @@ The following table lists cases when query feature works in ClickHouse, but beha | F051-03 | TIMESTAMP data type (including support of TIMESTAMP literal) with fractional seconds precision of at least 0 and 6 | No{.text-danger} | `DateTime64` time provides similar functionality | | F051-04 | Comparison predicate on DATE, TIME, and TIMESTAMP data types | Partial{.text-warning} | Only one data type available | | F051-05 | Explicit CAST between datetime types and character string types | Yes{.text-success} | | -| F051-06 | CURRENT\_DATE | No{.text-danger} | `today()` is similar | +| F051-06 | CURRENT_DATE | No{.text-danger} | `today()` is similar | | F051-07 | LOCALTIME | No{.text-danger} | `now()` is similar | | F051-08 | LOCALTIMESTAMP | No{.text-danger} | | | **F081** | **UNION and EXCEPT in views** | **Partial**{.text-warning} | | diff --git a/docs/en/sql-reference/data-types/datetime.md b/docs/en/sql-reference/data-types/datetime.md index a2ae68ebf14..c032b2f30fc 100644 --- a/docs/en/sql-reference/data-types/datetime.md +++ b/docs/en/sql-reference/data-types/datetime.md @@ -29,7 +29,7 @@ The [clickhouse-client](../../interfaces/cli.md) applies the server time zone by ClickHouse outputs values in `YYYY-MM-DD hh:mm:ss` text format by default. You can change the output with the [formatDateTime](../../sql-reference/functions/date-time-functions.md#formatdatetime) function. -When inserting data into ClickHouse, you can use different formats of date and time strings, depending on the value of the [date\_time\_input\_format](../../operations/settings/settings.md#settings-date_time_input_format) setting. +When inserting data into ClickHouse, you can use different formats of date and time strings, depending on the value of the [date_time_input_format](../../operations/settings/settings.md#settings-date_time_input_format) setting. ## Examples {#examples} diff --git a/docs/en/sql-reference/data-types/fixedstring.md b/docs/en/sql-reference/data-types/fixedstring.md index c813bd66a67..6f5725b017c 100644 --- a/docs/en/sql-reference/data-types/fixedstring.md +++ b/docs/en/sql-reference/data-types/fixedstring.md @@ -20,7 +20,7 @@ The `FixedString` type is efficient when data has the length of precisely `N` by Examples of the values that can be efficiently stored in `FixedString`-typed columns: - The binary representation of IP addresses (`FixedString(16)` for IPv6). -- Language codes (ru\_RU, en\_US … ). +- Language codes (ru_RU, en_US … ). - Currency codes (USD, RUB … ). - Binary representation of hashes (`FixedString(16)` for MD5, `FixedString(32)` for SHA256). diff --git a/docs/en/sql-reference/data-types/index.md b/docs/en/sql-reference/data-types/index.md index 202d5122ab6..92a4d034278 100644 --- a/docs/en/sql-reference/data-types/index.md +++ b/docs/en/sql-reference/data-types/index.md @@ -10,6 +10,6 @@ ClickHouse can store various kinds of data in table cells. This section describes the supported data types and special considerations for using and/or implementing them if any. -You can check whether data type name is case-sensitive in the [system.data\_type\_families](../../operations/system-tables/data_type_families.md#system_tables-data_type_families) table. +You can check whether data type name is case-sensitive in the [system.data_type_families](../../operations/system-tables/data_type_families.md#system_tables-data_type_families) table. [Original article](https://clickhouse.tech/docs/en/data_types/) diff --git a/docs/en/sql-reference/data-types/lowcardinality.md b/docs/en/sql-reference/data-types/lowcardinality.md index 1a0cedb99c7..36c86bf443c 100644 --- a/docs/en/sql-reference/data-types/lowcardinality.md +++ b/docs/en/sql-reference/data-types/lowcardinality.md @@ -15,7 +15,7 @@ LowCardinality(data_type) **Parameters** -- `data_type` — [String](../../sql-reference/data-types/string.md), [FixedString](../../sql-reference/data-types/fixedstring.md), [Date](../../sql-reference/data-types/date.md), [DateTime](../../sql-reference/data-types/datetime.md), and numbers excepting [Decimal](../../sql-reference/data-types/decimal.md). `LowCardinality` is not efficient for some data types, see the [allow\_suspicious\_low\_cardinality\_types](../../operations/settings/settings.md#allow_suspicious_low_cardinality_types) setting description. +- `data_type` — [String](../../sql-reference/data-types/string.md), [FixedString](../../sql-reference/data-types/fixedstring.md), [Date](../../sql-reference/data-types/date.md), [DateTime](../../sql-reference/data-types/datetime.md), and numbers excepting [Decimal](../../sql-reference/data-types/decimal.md). `LowCardinality` is not efficient for some data types, see the [allow_suspicious_low_cardinality_types](../../operations/settings/settings.md#allow_suspicious_low_cardinality_types) setting description. ## Description {#lowcardinality-dscr} @@ -43,10 +43,10 @@ ORDER BY id Settings: -- [low\_cardinality\_max\_dictionary\_size](../../operations/settings/settings.md#low_cardinality_max_dictionary_size) -- [low\_cardinality\_use\_single\_dictionary\_for\_part](../../operations/settings/settings.md#low_cardinality_use_single_dictionary_for_part) -- [low\_cardinality\_allow\_in\_native\_format](../../operations/settings/settings.md#low_cardinality_allow_in_native_format) -- [allow\_suspicious\_low\_cardinality\_types](../../operations/settings/settings.md#allow_suspicious_low_cardinality_types) +- [low_cardinality_max_dictionary_size](../../operations/settings/settings.md#low_cardinality_max_dictionary_size) +- [low_cardinality_use_single_dictionary_for_part](../../operations/settings/settings.md#low_cardinality_use_single_dictionary_for_part) +- [low_cardinality_allow_in_native_format](../../operations/settings/settings.md#low_cardinality_allow_in_native_format) +- [allow_suspicious_low_cardinality_types](../../operations/settings/settings.md#allow_suspicious_low_cardinality_types) Functions: diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md index 894ffe38d47..a5e105d2e13 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md @@ -25,7 +25,7 @@ Look at the following hierarchical structure: This hierarchy can be expressed as the following dictionary table. -| region\_id | parent\_region | region\_name | +| region_id | parent_region | region_name | |------------|----------------|---------------| | 1 | 0 | Russia | | 2 | 1 | Moscow | diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 251ec532e17..bbf45a577e0 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -7,7 +7,7 @@ toc_title: Storing Dictionaries in Memory There are a variety of ways to store dictionaries in memory. -We recommend [flat](#flat), [hashed](#dicts-external_dicts_dict_layout-hashed) and [complex\_key\_hashed](#complex-key-hashed). which provide optimal processing speed. +We recommend [flat](#flat), [hashed](#dicts-external_dicts_dict_layout-hashed) and [complex_key_hashed](#complex-key-hashed). which provide optimal processing speed. Caching is not recommended because of potentially poor performance and difficulties in selecting optimal parameters. Read more in the section “[cache](#cache)”. @@ -52,16 +52,16 @@ LAYOUT(LAYOUT_TYPE(param value)) -- layout settings - [flat](#flat) - [hashed](#dicts-external_dicts_dict_layout-hashed) -- [sparse\_hashed](#dicts-external_dicts_dict_layout-sparse_hashed) +- [sparse_hashed](#dicts-external_dicts_dict_layout-sparse_hashed) - [cache](#cache) -- [ssd\_cache](#ssd-cache) +- [ssd_cache](#ssd-cache) - [direct](#direct) -- [range\_hashed](#range-hashed) -- [complex\_key\_hashed](#complex-key-hashed) -- [complex\_key\_cache](#complex-key-cache) -- [ssd\_complex\_key\_cache](#ssd-cache) -- [complex\_key\_direct](#complex-key-direct) -- [ip\_trie](#ip-trie) +- [range_hashed](#range-hashed) +- [complex_key_hashed](#complex-key-hashed) +- [complex_key_cache](#complex-key-cache) +- [ssd_complex_key_cache](#ssd-cache) +- [complex_key_direct](#complex-key-direct) +- [ip_trie](#ip-trie) ### flat {#flat} @@ -107,7 +107,7 @@ or LAYOUT(HASHED()) ``` -### sparse\_hashed {#dicts-external_dicts_dict_layout-sparse_hashed} +### sparse_hashed {#dicts-external_dicts_dict_layout-sparse_hashed} Similar to `hashed`, but uses less memory in favor more CPU usage. @@ -123,7 +123,7 @@ Configuration example: LAYOUT(SPARSE_HASHED()) ``` -### complex\_key\_hashed {#complex-key-hashed} +### complex_key_hashed {#complex-key-hashed} This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to `hashed`. @@ -139,7 +139,7 @@ Configuration example: LAYOUT(COMPLEX_KEY_HASHED()) ``` -### range\_hashed {#range-hashed} +### range_hashed {#range-hashed} The dictionary is stored in memory in the form of a hash table with an ordered array of ranges and their corresponding values. @@ -294,11 +294,11 @@ Set a large enough cache size. You need to experiment to select the number of ce !!! warning "Warning" Do not use ClickHouse as a source, because it is slow to process queries with random reads. -### complex\_key\_cache {#complex-key-cache} +### complex_key_cache {#complex-key-cache} This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to `cache`. -### ssd\_cache {#ssd-cache} +### ssd_cache {#ssd-cache} Similar to `cache`, but stores data on SSD and index in RAM. @@ -328,9 +328,9 @@ LAYOUT(CACHE(BLOCK_SIZE 4096 FILE_SIZE 16777216 READ_BUFFER_SIZE 1048576 PATH /var/lib/clickhouse/clickhouse_dictionaries/test_dict MAX_STORED_KEYS 1048576)) ``` -### complex\_key\_ssd\_cache {#complex-key-ssd-cache} +### complex_key_ssd_cache {#complex-key-ssd-cache} -This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to `ssd\_cache`. +This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to `ssd_cache`. ### direct {#direct} @@ -354,11 +354,11 @@ or LAYOUT(DIRECT()) ``` -### complex\_key\_direct {#complex-key-direct} +### complex_key_direct {#complex-key-direct} This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to `direct`. -### ip\_trie {#ip-trie} +### ip_trie {#ip-trie} This type of storage is for mapping network prefixes (IP addresses) to metadata such as ASN. diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md index c6eb2cb3f62..99a62002822 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md @@ -13,9 +13,9 @@ ClickHouse: - Periodically updates dictionaries and dynamically loads missing values. In other words, dictionaries can be loaded dynamically. - Allows to create external dictionaries with xml files or [DDL queries](../../../sql-reference/statements/create/dictionary.md). -The configuration of external dictionaries can be located in one or more xml-files. The path to the configuration is specified in the [dictionaries\_config](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config) parameter. +The configuration of external dictionaries can be located in one or more xml-files. The path to the configuration is specified in the [dictionaries_config](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config) parameter. -Dictionaries can be loaded at server startup or at first use, depending on the [dictionaries\_lazy\_load](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load) setting. +Dictionaries can be loaded at server startup or at first use, depending on the [dictionaries_lazy_load](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load) setting. The [dictionaries](../../../operations/system-tables/dictionaries.md#system_tables-dictionaries) system table contains information about dictionaries configured at server. For each dictionary you can find there: diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 82700a109b5..50d1ae2aed4 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -1099,7 +1099,7 @@ Result: ## arrayAUC {#arrayauc} -Calculate AUC (Area Under the Curve, which is a concept in machine learning, see more details: https://en.wikipedia.org/wiki/Receiver\_operating\_characteristic\#Area\_under\_the\_curve). +Calculate AUC (Area Under the Curve, which is a concept in machine learning, see more details: https://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve). **Syntax** diff --git a/docs/en/sql-reference/functions/bitmap-functions.md b/docs/en/sql-reference/functions/bitmap-functions.md index 1edace1de69..a66098beffb 100644 --- a/docs/en/sql-reference/functions/bitmap-functions.md +++ b/docs/en/sql-reference/functions/bitmap-functions.md @@ -63,7 +63,7 @@ SELECT bitmapToArray(bitmapBuild([1, 2, 3, 4, 5])) AS res ## bitmapSubsetInRange {#bitmap-functions-bitmapsubsetinrange} -Return subset in specified range (not include the range\_end). +Return subset in specified range (not include the range_end). ``` sql bitmapSubsetInRange(bitmap, range_start, range_end) @@ -239,7 +239,7 @@ SELECT bitmapCardinality(bitmapBuild([1, 2, 3, 4, 5])) AS res ## bitmapMin {#bitmapmin} -Retrun the smallest value of type UInt64 in the set, UINT32\_MAX if the set is empty. +Retrun the smallest value of type UInt64 in the set, UINT32_MAX if the set is empty. bitmapMin(bitmap) @@ -286,8 +286,8 @@ Transform an array of values in a bitmap to another array of values, the result **Parameters** - `bitmap` – bitmap object. -- `from_array` – UInt32 array. For idx in range \[0, from\_array.size()), if bitmap contains from\_array\[idx\], then replace it with to\_array\[idx\]. Note that the result depends on array ordering if there are common elements between from\_array and to\_array. -- `to_array` – UInt32 array, its size shall be the same to from\_array. +- `from_array` – UInt32 array. For idx in range \[0, from_array.size()), if bitmap contains from_array\[idx\], then replace it with to_array\[idx\]. Note that the result depends on array ordering if there are common elements between from_array and to_array. +- `to_array` – UInt32 array, its size shall be the same to from_array. **Example** diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 910937b3fa9..c4186c47f45 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -218,7 +218,7 @@ Rounds down a date with time to the start of the ten-minute interval. Rounds down the date with time to the start of the fifteen-minute interval. -## toStartOfInterval(time\_or\_data, INTERVAL x unit \[, time\_zone\]) {#tostartofintervaltime-or-data-interval-x-unit-time-zone} +## toStartOfInterval(time_or_data, INTERVAL x unit \[, time_zone\]) {#tostartofintervaltime-or-data-interval-x-unit-time-zone} This is a generalization of other functions named `toStartOf*`. For example, `toStartOfInterval(t, INTERVAL 1 year)` returns the same as `toStartOfYear(t)`, @@ -339,7 +339,7 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d └────────────┴───────────┴───────────┴───────────┘ ``` -## date_trunc(datepart, time\_or\_data\[, time\_zone\]), dateTrunc(datepart, time\_or\_data\[, time\_zone\]) {#date_trunc} +## date_trunc(datepart, time_or_data\[, time_zone\]), dateTrunc(datepart, time_or_data\[, time_zone\]) {#date_trunc} Truncates a date or date with time based on the specified datepart, such as - `second` diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 50dfe1d2110..73e9ef6254e 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -315,7 +315,7 @@ Calculates JumpConsistentHash form a UInt64. Accepts two arguments: a UInt64-type key and the number of buckets. Returns Int32. For more information, see the link: [JumpConsistentHash](https://arxiv.org/pdf/1406.2294.pdf) -## murmurHash2\_32, murmurHash2\_64 {#murmurhash2-32-murmurhash2-64} +## murmurHash2_32, murmurHash2_64 {#murmurhash2-32-murmurhash2-64} Produces a [MurmurHash2](https://github.com/aappleby/smhasher) hash value. @@ -383,7 +383,7 @@ Result: └──────────────────────┴─────────────────────┘ ``` -## murmurHash3\_32, murmurHash3\_64 {#murmurhash3-32-murmurhash3-64} +## murmurHash3_32, murmurHash3_64 {#murmurhash3-32-murmurhash3-64} Produces a [MurmurHash3](https://github.com/aappleby/smhasher) hash value. @@ -413,7 +413,7 @@ SELECT murmurHash3_32(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23: └─────────────┴────────┘ ``` -## murmurHash3\_128 {#murmurhash3-128} +## murmurHash3_128 {#murmurhash3-128} Produces a 128-bit [MurmurHash3](https://github.com/aappleby/smhasher) hash value. diff --git a/docs/en/sql-reference/functions/introspection.md b/docs/en/sql-reference/functions/introspection.md index 1fd39c704c5..3e63fc9946b 100644 --- a/docs/en/sql-reference/functions/introspection.md +++ b/docs/en/sql-reference/functions/introspection.md @@ -14,11 +14,11 @@ For proper operation of introspection functions: - Install the `clickhouse-common-static-dbg` package. -- Set the [allow\_introspection\_functions](../../operations/settings/settings.md#settings-allow_introspection_functions) setting to 1. +- Set the [allow_introspection_functions](../../operations/settings/settings.md#settings-allow_introspection_functions) setting to 1. For security reasons introspection functions are disabled by default. -ClickHouse saves profiler reports to the [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) system table. Make sure the table and profiler are configured properly. +ClickHouse saves profiler reports to the [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) system table. Make sure the table and profiler are configured properly. ## addressToLine {#addresstoline} diff --git a/docs/en/sql-reference/functions/json-functions.md b/docs/en/sql-reference/functions/json-functions.md index 6f9c643c565..05e755eaddc 100644 --- a/docs/en/sql-reference/functions/json-functions.md +++ b/docs/en/sql-reference/functions/json-functions.md @@ -73,7 +73,7 @@ SELECT isValidJSON('{"a": "hello", "b": [-100, 200.0, 300]}') = 1 SELECT isValidJSON('not a json') = 0 ``` -## JSONHas(json\[, indices\_or\_keys\]…) {#jsonhasjson-indices-or-keys} +## JSONHas(json\[, indices_or_keys\]…) {#jsonhasjson-indices-or-keys} If the value exists in the JSON document, `1` will be returned. @@ -106,7 +106,7 @@ SELECT JSONExtractKey('{"a": "hello", "b": [-100, 200.0, 300]}', -2) = 'a' SELECT JSONExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 1) = 'hello' ``` -## JSONLength(json\[, indices\_or\_keys\]…) {#jsonlengthjson-indices-or-keys} +## JSONLength(json\[, indices_or_keys\]…) {#jsonlengthjson-indices-or-keys} Return the length of a JSON array or a JSON object. @@ -119,7 +119,7 @@ SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 3 SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}') = 2 ``` -## JSONType(json\[, indices\_or\_keys\]…) {#jsontypejson-indices-or-keys} +## JSONType(json\[, indices_or_keys\]…) {#jsontypejson-indices-or-keys} Return the type of a JSON value. @@ -133,13 +133,13 @@ SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'a') = 'String' SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 'Array' ``` -## JSONExtractUInt(json\[, indices\_or\_keys\]…) {#jsonextractuintjson-indices-or-keys} +## JSONExtractUInt(json\[, indices_or_keys\]…) {#jsonextractuintjson-indices-or-keys} -## JSONExtractInt(json\[, indices\_or\_keys\]…) {#jsonextractintjson-indices-or-keys} +## JSONExtractInt(json\[, indices_or_keys\]…) {#jsonextractintjson-indices-or-keys} -## JSONExtractFloat(json\[, indices\_or\_keys\]…) {#jsonextractfloatjson-indices-or-keys} +## JSONExtractFloat(json\[, indices_or_keys\]…) {#jsonextractfloatjson-indices-or-keys} -## JSONExtractBool(json\[, indices\_or\_keys\]…) {#jsonextractbooljson-indices-or-keys} +## JSONExtractBool(json\[, indices_or_keys\]…) {#jsonextractbooljson-indices-or-keys} Parses a JSON and extract a value. These functions are similar to `visitParam` functions. @@ -153,7 +153,7 @@ SELECT JSONExtractFloat('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 2) = 200 SELECT JSONExtractUInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', -1) = 300 ``` -## JSONExtractString(json\[, indices\_or\_keys\]…) {#jsonextractstringjson-indices-or-keys} +## JSONExtractString(json\[, indices_or_keys\]…) {#jsonextractstringjson-indices-or-keys} Parses a JSON and extract a string. This function is similar to `visitParamExtractString` functions. @@ -171,7 +171,7 @@ SELECT JSONExtractString('{"abc":"\\u263"}', 'abc') = '' SELECT JSONExtractString('{"abc":"hello}', 'abc') = '' ``` -## JSONExtract(json\[, indices\_or\_keys…\], Return\_type) {#jsonextractjson-indices-or-keys-return-type} +## JSONExtract(json\[, indices_or_keys…\], Return_type) {#jsonextractjson-indices-or-keys-return-type} Parses a JSON and extract a value of the given ClickHouse data type. @@ -192,7 +192,7 @@ SELECT JSONExtract('{"day": "Thursday"}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday SELECT JSONExtract('{"day": 5}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \'Tuesday\' = 2, \'Wednesday\' = 3, \'Thursday\' = 4, \'Friday\' = 5, \'Saturday\' = 6)') = 'Friday' ``` -## JSONExtractKeysAndValues(json\[, indices\_or\_keys…\], Value\_type) {#jsonextractkeysandvaluesjson-indices-or-keys-value-type} +## JSONExtractKeysAndValues(json\[, indices_or_keys…\], Value_type) {#jsonextractkeysandvaluesjson-indices-or-keys-value-type} Parses key-value pairs from a JSON where the values are of the given ClickHouse data type. @@ -202,7 +202,7 @@ Example: SELECT JSONExtractKeysAndValues('{"x": {"a": 5, "b": 7, "c": 11}}', 'x', 'Int8') = [('a',5),('b',7),('c',11)] ``` -## JSONExtractRaw(json\[, indices\_or\_keys\]…) {#jsonextractrawjson-indices-or-keys} +## JSONExtractRaw(json\[, indices_or_keys\]…) {#jsonextractrawjson-indices-or-keys} Returns a part of JSON as unparsed string. @@ -214,7 +214,7 @@ Example: SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = '[-100, 200.0, 300]' ``` -## JSONExtractArrayRaw(json\[, indices\_or\_keys…\]) {#jsonextractarrayrawjson-indices-or-keys} +## JSONExtractArrayRaw(json\[, indices_or_keys…\]) {#jsonextractarrayrawjson-indices-or-keys} Returns an array with elements of JSON array, each represented as unparsed string. diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 00b30d1aefa..de208f2c209 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -417,7 +417,7 @@ ORDER BY h ASC Transforms a value according to the explicitly defined mapping of some elements to other ones. There are two variations of this function: -### transform(x, array\_from, array\_to, default) {#transformx-array-from-array-to-default} +### transform(x, array_from, array_to, default) {#transformx-array-from-array-to-default} `x` – What to transform. @@ -437,7 +437,7 @@ Types: Where the same letter is indicated (T or U), for numeric types these might not be matching types, but types that have a common type. For example, the first argument can have the Int64 type, while the second has the Array(UInt16) type. -If the ‘x’ value is equal to one of the elements in the ‘array\_from’ array, it returns the existing element (that is numbered the same) from the ‘array\_to’ array. Otherwise, it returns ‘default’. If there are multiple matching elements in ‘array\_from’, it returns one of the matches. +If the ‘x’ value is equal to one of the elements in the ‘array_from’ array, it returns the existing element (that is numbered the same) from the ‘array_to’ array. Otherwise, it returns ‘default’. If there are multiple matching elements in ‘array_from’, it returns one of the matches. Example: @@ -459,10 +459,10 @@ ORDER BY c DESC └───────────┴────────┘ ``` -### transform(x, array\_from, array\_to) {#transformx-array-from-array-to} +### transform(x, array_from, array_to) {#transformx-array-from-array-to} Differs from the first variation in that the ‘default’ argument is omitted. -If the ‘x’ value is equal to one of the elements in the ‘array\_from’ array, it returns the matching element (that is numbered the same) from the ‘array\_to’ array. Otherwise, it returns ‘x’. +If the ‘x’ value is equal to one of the elements in the ‘array_from’ array, it returns the matching element (that is numbered the same) from the ‘array_to’ array. Otherwise, it returns ‘x’. Types: @@ -1248,7 +1248,7 @@ joinGet(join_storage_table_name, `value_column`, join_keys) Returns list of values corresponded to list of keys. -If certain doesn’t exist in source table then `0` or `null` will be returned based on [join\_use\_nulls](../../operations/settings/settings.md#join_use_nulls) setting. +If certain doesn’t exist in source table then `0` or `null` will be returned based on [join_use_nulls](../../operations/settings/settings.md#join_use_nulls) setting. More info about `join_use_nulls` in [Join operation](../../engines/table-engines/special/join.md). @@ -1287,15 +1287,15 @@ Result: └──────────────────────────────────────────────────┘ ``` -## modelEvaluate(model\_name, …) {#function-modelevaluate} +## modelEvaluate(model_name, …) {#function-modelevaluate} Evaluate external model. Accepts a model name and model arguments. Returns Float64. -## throwIf(x\[, custom\_message\]) {#throwifx-custom-message} +## throwIf(x\[, custom_message\]) {#throwifx-custom-message} Throw an exception if the argument is non zero. -custom\_message - is an optional parameter: a constant string, provides an error message +custom_message - is an optional parameter: a constant string, provides an error message ``` sql SELECT throwIf(number = 3, 'Too many') FROM numbers(10); diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index a31237ecfb5..a846a01f11f 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -32,12 +32,12 @@ The function also works for arrays. Returns the length of a string in Unicode code points (not in characters), assuming that the string contains a set of bytes that make up UTF-8 encoded text. If this assumption is not met, it returns some result (it doesn’t throw an exception). The result type is UInt64. -## char\_length, CHAR\_LENGTH {#char-length} +## char_length, CHAR_LENGTH {#char-length} Returns the length of a string in Unicode code points (not in characters), assuming that the string contains a set of bytes that make up UTF-8 encoded text. If this assumption is not met, it returns some result (it doesn’t throw an exception). The result type is UInt64. -## character\_length, CHARACTER\_LENGTH {#character-length} +## character_length, CHARACTER_LENGTH {#character-length} Returns the length of a string in Unicode code points (not in characters), assuming that the string contains a set of bytes that make up UTF-8 encoded text. If this assumption is not met, it returns some result (it doesn’t throw an exception). The result type is UInt64. @@ -78,7 +78,7 @@ toValidUTF8( input_string ) Parameters: -- input\_string — Any set of bytes represented as the [String](../../sql-reference/data-types/string.md) data type object. +- input_string — Any set of bytes represented as the [String](../../sql-reference/data-types/string.md) data type object. Returned value: Valid UTF-8 string. diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index f3ae7c85cf4..7b68eab7f8a 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -250,7 +250,7 @@ YYYY-MM-DD YYYY-MM-DD hh:mm:ss ``` -As an exception, if converting from UInt32, Int32, UInt64, or Int64 numeric types to Date, and if the number is greater than or equal to 65536, the number is interpreted as a Unix timestamp (and not as the number of days) and is rounded to the date. This allows support for the common occurrence of writing ‘toDate(unix\_timestamp)’, which otherwise would be an error and would require writing the more cumbersome ‘toDate(toDateTime(unix\_timestamp))’. +As an exception, if converting from UInt32, Int32, UInt64, or Int64 numeric types to Date, and if the number is greater than or equal to 65536, the number is interpreted as a Unix timestamp (and not as the number of days) and is rounded to the date. This allows support for the common occurrence of writing ‘toDate(unix_timestamp)’, which otherwise would be an error and would require writing the more cumbersome ‘toDate(toDateTime(unix_timestamp))’. Conversion between a date and date with time is performed the natural way: by adding a null time or dropping the time. diff --git a/docs/en/sql-reference/functions/url-functions.md b/docs/en/sql-reference/functions/url-functions.md index 89154ac3d02..ad63a5b72ac 100644 --- a/docs/en/sql-reference/functions/url-functions.md +++ b/docs/en/sql-reference/functions/url-functions.md @@ -117,7 +117,7 @@ Returns the part of the domain that includes top-level subdomains up to the “f For example, `cutToFirstSignificantSubdomain('https://news.yandex.com.tr/') = 'yandex.com.tr'`. -### port(URL\[, default\_port = 0\]) {#port} +### port(URL\[, default_port = 0\]) {#port} Returns the port or `default_port` if there is no port in the URL (or in case of validation error). @@ -127,11 +127,11 @@ Returns the path. Example: `/top/news.html` The path does not include the query ### pathFull {#pathfull} -The same as above, but including query string and fragment. Example: /top/news.html?page=2\#comments +The same as above, but including query string and fragment. Example: /top/news.html?page=2#comments ### queryString {#querystring} -Returns the query string. Example: page=1&lr=213. query-string does not include the initial question mark, as well as \# and everything after \#. +Returns the query string. Example: page=1&lr=213. query-string does not include the initial question mark, as well as # and everything after #. ### fragment {#fragment} @@ -139,7 +139,7 @@ Returns the fragment identifier. fragment does not include the initial hash symb ### queryStringAndFragment {#querystringandfragment} -Returns the query string and fragment identifier. Example: page=1\#29390. +Returns the query string and fragment identifier. Example: page=1#29390. ### extractURLParameter(URL, name) {#extracturlparameterurl-name} diff --git a/docs/en/sql-reference/functions/ym-dict-functions.md b/docs/en/sql-reference/functions/ym-dict-functions.md index 79de37bf3d3..f70532252c7 100644 --- a/docs/en/sql-reference/functions/ym-dict-functions.md +++ b/docs/en/sql-reference/functions/ym-dict-functions.md @@ -15,12 +15,12 @@ ClickHouse supports working with multiple alternative geobases (regional hierarc The ‘clickhouse-server’ config specifies the file with the regional hierarchy::`/opt/geo/regions_hierarchy.txt` -Besides this file, it also searches for files nearby that have the \_ symbol and any suffix appended to the name (before the file extension). +Besides this file, it also searches for files nearby that have the _ symbol and any suffix appended to the name (before the file extension). For example, it will also find the file `/opt/geo/regions_hierarchy_ua.txt`, if present. `ua` is called the dictionary key. For a dictionary without a suffix, the key is an empty string. -All the dictionaries are re-loaded in runtime (once every certain number of seconds, as defined in the builtin\_dictionaries\_reload\_interval config parameter, or once an hour by default). However, the list of available dictionaries is defined one time, when the server starts. +All the dictionaries are re-loaded in runtime (once every certain number of seconds, as defined in the builtin_dictionaries_reload_interval config parameter, or once an hour by default). However, the list of available dictionaries is defined one time, when the server starts. All functions for working with regions have an optional argument at the end – the dictionary key. It is referred to as the geobase. Example: @@ -105,7 +105,7 @@ Example: `regionToCountry(toUInt32(213)) = 225` converts Moscow (213) to Russia Converts a region to a continent. In every other way, this function is the same as ‘regionToCity’. Example: `regionToContinent(toUInt32(213)) = 10001` converts Moscow (213) to Eurasia (10001). -### regionToTopContinent (\#regiontotopcontinent) {#regiontotopcontinent-regiontotopcontinent} +### regionToTopContinent (#regiontotopcontinent) {#regiontotopcontinent-regiontotopcontinent} Finds the highest continent in the hierarchy for the region. diff --git a/docs/en/sql-reference/operators/in.md b/docs/en/sql-reference/operators/in.md index d67f1474f10..eca95dbc652 100644 --- a/docs/en/sql-reference/operators/in.md +++ b/docs/en/sql-reference/operators/in.md @@ -64,7 +64,7 @@ A subquery in the IN clause is always run just one time on a single server. Ther ## NULL Processing {#in-null-processing} -During request processing, the `IN` operator assumes that the result of an operation with [NULL](../../sql-reference/syntax.md#null-literal) always equals `0`, regardless of whether `NULL` is on the right or left side of the operator. `NULL` values are not included in any dataset, do not correspond to each other and cannot be compared if [transform\_null\_in = 0](../../operations/settings/settings.md#transform_null_in). +During request processing, the `IN` operator assumes that the result of an operation with [NULL](../../sql-reference/syntax.md#null-literal) always equals `0`, regardless of whether `NULL` is on the right or left side of the operator. `NULL` values are not included in any dataset, do not correspond to each other and cannot be compared if [transform_null_in = 0](../../operations/settings/settings.md#transform_null_in). Here is an example with the `t_null` table: @@ -112,9 +112,9 @@ For a non-distributed query, use the regular `IN` / `JOIN`. Be careful when using subqueries in the `IN` / `JOIN` clauses for distributed query processing. -Let’s look at some examples. Assume that each server in the cluster has a normal **local\_table**. Each server also has a **distributed\_table** table with the **Distributed** type, which looks at all the servers in the cluster. +Let’s look at some examples. Assume that each server in the cluster has a normal **local_table**. Each server also has a **distributed_table** table with the **Distributed** type, which looks at all the servers in the cluster. -For a query to the **distributed\_table**, the query will be sent to all the remote servers and run on them using the **local\_table**. +For a query to the **distributed_table**, the query will be sent to all the remote servers and run on them using the **local_table**. For example, the query @@ -148,7 +148,7 @@ In other words, the data set in the IN clause will be collected on each server i This will work correctly and optimally if you are prepared for this case and have spread data across the cluster servers such that the data for a single UserID resides entirely on a single server. In this case, all the necessary data will be available locally on each server. Otherwise, the result will be inaccurate. We refer to this variation of the query as “local IN”. -To correct how the query works when data is spread randomly across the cluster servers, you could specify **distributed\_table** inside a subquery. The query would look like this: +To correct how the query works when data is spread randomly across the cluster servers, you could specify **distributed_table** inside a subquery. The query would look like this: ``` sql SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM distributed_table WHERE CounterID = 34) diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index f33ecb63bf1..d2dd1c638cc 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -240,7 +240,7 @@ You can specify the partition expression in `ALTER ... PARTITION` queries in dif - As a value from the `partition` column of the `system.parts` table. For example, `ALTER TABLE visits DETACH PARTITION 201901`. - As the expression from the table column. Constants and constant expressions are supported. For example, `ALTER TABLE visits DETACH PARTITION toYYYYMM(toDate('2019-01-25'))`. - Using the partition ID. Partition ID is a string identifier of the partition (human-readable, if possible) that is used as the names of partitions in the file system and in ZooKeeper. The partition ID must be specified in the `PARTITION ID` clause, in a single quotes. For example, `ALTER TABLE visits DETACH PARTITION ID '201901'`. -- In the [ALTER ATTACH PART](#alter_attach-partition) and [DROP DETACHED PART](#alter_drop-detached) query, to specify the name of a part, use string literal with a value from the `name` column of the [system.detached\_parts](../../../operations/system-tables/detached_parts.md#system_tables-detached_parts) table. For example, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. +- In the [ALTER ATTACH PART](#alter_attach-partition) and [DROP DETACHED PART](#alter_drop-detached) query, to specify the name of a part, use string literal with a value from the `name` column of the [system.detached_parts](../../../operations/system-tables/detached_parts.md#system_tables-detached_parts) table. For example, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. Usage of quotes when specifying the partition depends on the type of partition expression. For example, for the `String` type, you have to specify its name in quotes (`'`). For the `Date` and `Int*` types no quotes are needed. diff --git a/docs/en/sql-reference/statements/check-table.md b/docs/en/sql-reference/statements/check-table.md index cfb18161d64..b55955eeb40 100644 --- a/docs/en/sql-reference/statements/check-table.md +++ b/docs/en/sql-reference/statements/check-table.md @@ -37,6 +37,6 @@ For `MergeTree` family engines, the `CHECK TABLE` query shows a check status for If the table is corrupted, you can copy the non-corrupted data to another table. To do this: 1. Create a new table with the same structure as damaged table. To do this execute the query `CREATE TABLE AS `. -2. Set the [max\_threads](../../operations/settings/settings.md#settings-max_threads) value to 1 to process the next query in a single thread. To do this run the query `SET max_threads = 1`. +2. Set the [max_threads](../../operations/settings/settings.md#settings-max_threads) value to 1 to process the next query in a single thread. To do this run the query `SET max_threads = 1`. 3. Execute the query `INSERT INTO SELECT * FROM `. This request copies the non-corrupted data from the damaged table to another table. Only the data before the corrupted part will be copied. 4. Restart the `clickhouse-client` to reset the `max_threads` value. diff --git a/docs/en/sql-reference/statements/insert-into.md b/docs/en/sql-reference/statements/insert-into.md index 8ad4212d59b..a2afbb2e10a 100644 --- a/docs/en/sql-reference/statements/insert-into.md +++ b/docs/en/sql-reference/statements/insert-into.md @@ -18,7 +18,7 @@ The query can specify a list of columns to insert `[(c1, c2, c3)]`. In this case - The values calculated from the `DEFAULT` expressions specified in the table definition. - Zeros and empty strings, if `DEFAULT` expressions are not defined. -If [strict\_insert\_defaults=1](../../operations/settings/settings.md), columns that do not have `DEFAULT` defined must be listed in the query. +If [strict_insert_defaults=1](../../operations/settings/settings.md), columns that do not have `DEFAULT` defined must be listed in the query. Data can be passed to the INSERT in any [format](../../interfaces/formats.md#formats) supported by ClickHouse. The format must be specified explicitly in the query: diff --git a/docs/en/sql-reference/statements/optimize.md b/docs/en/sql-reference/statements/optimize.md index 0f834f30db1..374ef926d6c 100644 --- a/docs/en/sql-reference/statements/optimize.md +++ b/docs/en/sql-reference/statements/optimize.md @@ -15,7 +15,7 @@ The `OPTMIZE` query is also supported for the [MaterializedView](../../engines/t When `OPTIMIZE` is used with the [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/replication.md) family of table engines, ClickHouse creates a task for merging and waits for execution on all nodes (if the `replication_alter_partitions_sync` setting is enabled). -- If `OPTIMIZE` doesn’t perform a merge for any reason, it doesn’t notify the client. To enable notifications, use the [optimize\_throw\_if\_noop](../../operations/settings/settings.md#setting-optimize_throw_if_noop) setting. +- If `OPTIMIZE` doesn’t perform a merge for any reason, it doesn’t notify the client. To enable notifications, use the [optimize_throw_if_noop](../../operations/settings/settings.md#setting-optimize_throw_if_noop) setting. - If you specify a `PARTITION`, only the specified partition is optimized. [How to set partition expression](../../sql-reference/statements/alter/index.md#alter-how-to-specify-part-expr). - If you specify `FINAL`, optimization is performed even when all the data is already in one part. - If you specify `DEDUPLICATE`, then completely identical rows will be deduplicated (all columns are compared), it makes sense only for the MergeTree engine. diff --git a/docs/en/sql-reference/statements/select/group-by.md b/docs/en/sql-reference/statements/select/group-by.md index 0c2b0a2dc4a..6cb99f285f2 100644 --- a/docs/en/sql-reference/statements/select/group-by.md +++ b/docs/en/sql-reference/statements/select/group-by.md @@ -66,9 +66,9 @@ The other alternatives include only the rows that pass through HAVING in ‘tota `after_having_exclusive` – Don’t include rows that didn’t pass through `max_rows_to_group_by`. In other words, ‘totals’ will have less than or the same number of rows as it would if `max_rows_to_group_by` were omitted. -`after_having_inclusive` – Include all the rows that didn’t pass through ‘max\_rows\_to\_group\_by’ in ‘totals’. In other words, ‘totals’ will have more than or the same number of rows as it would if `max_rows_to_group_by` were omitted. +`after_having_inclusive` – Include all the rows that didn’t pass through ‘max_rows_to_group_by’ in ‘totals’. In other words, ‘totals’ will have more than or the same number of rows as it would if `max_rows_to_group_by` were omitted. -`after_having_auto` – Count the number of rows that passed through HAVING. If it is more than a certain amount (by default, 50%), include all the rows that didn’t pass through ‘max\_rows\_to\_group\_by’ in ‘totals’. Otherwise, do not include them. +`after_having_auto` – Count the number of rows that passed through HAVING. If it is more than a certain amount (by default, 50%), include all the rows that didn’t pass through ‘max_rows_to_group_by’ in ‘totals’. Otherwise, do not include them. `totals_auto_threshold` – By default, 0.5. The coefficient for `after_having_auto`. @@ -116,11 +116,11 @@ Aggregation is one of the most important features of a column-oriented DBMS, and ### GROUP BY in External Memory {#select-group-by-in-external-memory} You can enable dumping temporary data to the disk to restrict memory usage during `GROUP BY`. -The [max\_bytes\_before\_external\_group\_by](../../../operations/settings/settings.md#settings-max_bytes_before_external_group_by) setting determines the threshold RAM consumption for dumping `GROUP BY` temporary data to the file system. If set to 0 (the default), it is disabled. +The [max_bytes_before_external_group_by](../../../operations/settings/settings.md#settings-max_bytes_before_external_group_by) setting determines the threshold RAM consumption for dumping `GROUP BY` temporary data to the file system. If set to 0 (the default), it is disabled. When using `max_bytes_before_external_group_by`, we recommend that you set `max_memory_usage` about twice as high. This is necessary because there are two stages to aggregation: reading the data and forming intermediate data (1) and merging the intermediate data (2). Dumping data to the file system can only occur during stage 1. If the temporary data wasn’t dumped, then stage 2 might require up to the same amount of memory as in stage 1. -For example, if [max\_memory\_usage](../../../operations/settings/settings.md#settings_max_memory_usage) was set to 10000000000 and you want to use external aggregation, it makes sense to set `max_bytes_before_external_group_by` to 10000000000, and `max_memory_usage` to 20000000000. When external aggregation is triggered (if there was at least one dump of temporary data), maximum consumption of RAM is only slightly more than `max_bytes_before_external_group_by`. +For example, if [max_memory_usage](../../../operations/settings/settings.md#settings_max_memory_usage) was set to 10000000000 and you want to use external aggregation, it makes sense to set `max_bytes_before_external_group_by` to 10000000000, and `max_memory_usage` to 20000000000. When external aggregation is triggered (if there was at least one dump of temporary data), maximum consumption of RAM is only slightly more than `max_bytes_before_external_group_by`. With distributed query processing, external aggregation is performed on remote servers. In order for the requester server to use only a small amount of RAM, set `distributed_aggregation_memory_efficient` to 1. diff --git a/docs/en/sql-reference/statements/select/join.md b/docs/en/sql-reference/statements/select/join.md index 7d41930aab6..c90b4bf0eaa 100644 --- a/docs/en/sql-reference/statements/select/join.md +++ b/docs/en/sql-reference/statements/select/join.md @@ -39,7 +39,7 @@ Additional join types available in ClickHouse: ## Setting {#join-settings} !!! note "Note" - The default join type can be overriden using [join\_default\_strictness](../../../operations/settings/settings.md#settings-join_default_strictness) setting. + The default join type can be overriden using [join_default_strictness](../../../operations/settings/settings.md#settings-join_default_strictness) setting. Also the behavior of ClickHouse server for `ANY JOIN` operations depends on the [any_join_distinct_right_table_keys](../../../operations/settings/settings.md#any_join_distinct_right_table_keys) setting. @@ -106,7 +106,7 @@ Be careful when using `GLOBAL`. For more information, see the [Distributed subqu ### Processing of Empty or NULL Cells {#processing-of-empty-or-null-cells} -While joining tables, the empty cells may appear. The setting [join\_use\_nulls](../../../operations/settings/settings.md#join_use_nulls) define how ClickHouse fills these cells. +While joining tables, the empty cells may appear. The setting [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) define how ClickHouse fills these cells. If the `JOIN` keys are [Nullable](../../../sql-reference/data-types/nullable.md) fields, the rows where at least one of the keys has the value [NULL](../../../sql-reference/syntax.md#null-literal) are not joined. @@ -143,10 +143,10 @@ By default, ClickHouse uses the [hash join](https://en.wikipedia.org/wiki/Hash_j If you need to restrict join operation memory consumption use the following settings: -- [max\_rows\_in\_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) — Limits number of rows in the hash table. -- [max\_bytes\_in\_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) — Limits size of the hash table. +- [max_rows_in_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) — Limits number of rows in the hash table. +- [max_bytes_in_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) — Limits size of the hash table. -When any of these limits is reached, ClickHouse acts as the [join\_overflow\_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) setting instructs. +When any of these limits is reached, ClickHouse acts as the [join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) setting instructs. ## Examples {#examples} diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index a9f9b718de6..8230177f249 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -47,12 +47,12 @@ Always returns `Ok.` regardless of the result of the internal dictionary update. ## RELOAD DICTIONARIES {#query_language-system-reload-dictionaries} Reloads all dictionaries that have been successfully loaded before. -By default, dictionaries are loaded lazily (see [dictionaries\_lazy\_load](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load)), so instead of being loaded automatically at startup, they are initialized on first access through dictGet function or SELECT from tables with ENGINE = Dictionary. The `SYSTEM RELOAD DICTIONARIES` query reloads such dictionaries (LOADED). +By default, dictionaries are loaded lazily (see [dictionaries_lazy_load](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load)), so instead of being loaded automatically at startup, they are initialized on first access through dictGet function or SELECT from tables with ENGINE = Dictionary. The `SYSTEM RELOAD DICTIONARIES` query reloads such dictionaries (LOADED). Always returns `Ok.` regardless of the result of the dictionary update. ## RELOAD DICTIONARY {#query_language-system-reload-dictionary} -Completely reloads a dictionary `dictionary_name`, regardless of the state of the dictionary (LOADED / NOT\_LOADED / FAILED). +Completely reloads a dictionary `dictionary_name`, regardless of the state of the dictionary (LOADED / NOT_LOADED / FAILED). Always returns `Ok.` regardless of the result of updating the dictionary. The status of the dictionary can be checked by querying the `system.dictionaries` table. @@ -64,7 +64,7 @@ SELECT name, status FROM system.dictionaries; Resets ClickHouse’s internal DNS cache. Sometimes (for old ClickHouse versions) it is necessary to use this command when changing the infrastructure (changing the IP address of another ClickHouse server or the server used by dictionaries). -For more convenient (automatic) cache management, see disable\_internal\_dns\_cache, dns\_cache\_update\_period parameters. +For more convenient (automatic) cache management, see disable_internal_dns_cache, dns_cache_update_period parameters. ## DROP MARK CACHE {#query_language-system-drop-mark-cache} @@ -91,7 +91,7 @@ The fourth one is useful to remove metadata of dead replica when all other repli ## DROP UNCOMPRESSED CACHE {#query_language-system-drop-uncompressed-cache} Reset the uncompressed data cache. Used in development of ClickHouse and performance tests. -For manage uncompressed data cache parameters use following server level settings [uncompressed\_cache\_size](../../operations/server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) and query/user/profile level settings [use\_uncompressed\_cache](../../operations/settings/settings.md#setting-use_uncompressed_cache) +For manage uncompressed data cache parameters use following server level settings [uncompressed_cache_size](../../operations/server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) and query/user/profile level settings [use_uncompressed_cache](../../operations/settings/settings.md#setting-use_uncompressed_cache) ## DROP COMPILED EXPRESSION CACHE {#query_language-system-drop-compiled-expression-cache} @@ -100,7 +100,7 @@ Complied expression cache used when query/user/profile enable option [compile](. ## FLUSH LOGS {#query_language-system-flush_logs} -Flushes buffers of log messages to system tables (e.g. system.query\_log). Allows you to not wait 7.5 seconds when debugging. +Flushes buffers of log messages to system tables (e.g. system.query_log). Allows you to not wait 7.5 seconds when debugging. This will also create system tables even if message queue is empty. ## RELOAD CONFIG {#query_language-system-reload-config} diff --git a/docs/en/sql-reference/syntax.md b/docs/en/sql-reference/syntax.md index 65a4f7b7332..fecea3ab30b 100644 --- a/docs/en/sql-reference/syntax.md +++ b/docs/en/sql-reference/syntax.md @@ -13,9 +13,9 @@ The `INSERT` query uses both parsers: INSERT INTO t VALUES (1, 'Hello, world'), (2, 'abc'), (3, 'def') ``` -The `INSERT INTO t VALUES` fragment is parsed by the full parser, and the data `(1, 'Hello, world'), (2, 'abc'), (3, 'def')` is parsed by the fast stream parser. You can also turn on the full parser for the data by using the [input\_format\_values\_interpret\_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions) setting. When `input_format_values_interpret_expressions = 1`, ClickHouse first tries to parse values with the fast stream parser. If it fails, ClickHouse tries to use the full parser for the data, treating it like an SQL [expression](#syntax-expressions). +The `INSERT INTO t VALUES` fragment is parsed by the full parser, and the data `(1, 'Hello, world'), (2, 'abc'), (3, 'def')` is parsed by the fast stream parser. You can also turn on the full parser for the data by using the [input_format_values_interpret_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions) setting. When `input_format_values_interpret_expressions = 1`, ClickHouse first tries to parse values with the fast stream parser. If it fails, ClickHouse tries to use the full parser for the data, treating it like an SQL [expression](#syntax-expressions). -Data can have any format. When a query is received, the server calculates no more than [max\_query\_size](../operations/settings/settings.md#settings-max_query_size) bytes of the request in RAM (by default, 1 MB), and the rest is stream parsed. +Data can have any format. When a query is received, the server calculates no more than [max_query_size](../operations/settings/settings.md#settings-max_query_size) bytes of the request in RAM (by default, 1 MB), and the rest is stream parsed. It allows for avoiding issues with large `INSERT` queries. When using the `Values` format in an `INSERT` query, it may seem that data is parsed the same as expressions in a `SELECT` query, but this is not true. The `Values` format is much more limited. @@ -40,7 +40,7 @@ Keywords are case-insensitive when they correspond to: - SQL standard. For example, `SELECT`, `select` and `SeLeCt` are all valid. - Implementation in some popular DBMS (MySQL or Postgres). For example, `DateTime` is the same as `datetime`. -You can check whether a data type name is case-sensitive in the [system.data\_type\_families](../operations/system-tables/data_type_families.md#system_tables-data_type_families) table. +You can check whether a data type name is case-sensitive in the [system.data_type_families](../operations/system-tables/data_type_families.md#system_tables-data_type_families) table. In contrast to standard SQL, all other keywords (including functions names) are **case-sensitive**. diff --git a/docs/en/sql-reference/table-functions/cluster.md b/docs/en/sql-reference/table-functions/cluster.md index 9fe50a2ad2e..84f534c02dd 100644 --- a/docs/en/sql-reference/table-functions/cluster.md +++ b/docs/en/sql-reference/table-functions/cluster.md @@ -34,5 +34,5 @@ Connection settings like `host`, `port`, `user`, `password`, `compression`, `sec **See Also** -- [skip\_unavailable\_shards](../../operations/settings/settings.md#settings-skip_unavailable_shards) -- [load\_balancing](../../operations/settings/settings.md#settings-load_balancing) +- [skip_unavailable_shards](../../operations/settings/settings.md#settings-skip_unavailable_shards) +- [load_balancing](../../operations/settings/settings.md#settings-load_balancing) diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index a67d9bbfd90..beab691ad0e 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -13,7 +13,7 @@ file(path, format, structure) **Input parameters** -- `path` — The relative path to the file from [user\_files\_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Path to file support following globs in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, \``'abc', 'def'` — strings. +- `path` — The relative path to the file from [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Path to file support following globs in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, \``'abc', 'def'` — strings. - `format` — The [format](../../interfaces/formats.md#formats) of the file. - `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. @@ -70,12 +70,12 @@ Constructions with `{}` are similar to the [remote table function](../../sql-ref 1. Suppose we have several files with the following relative paths: -- ‘some\_dir/some\_file\_1’ -- ‘some\_dir/some\_file\_2’ -- ‘some\_dir/some\_file\_3’ -- ‘another\_dir/some\_file\_1’ -- ‘another\_dir/some\_file\_2’ -- ‘another\_dir/some\_file\_3’ +- ‘some_dir/some_file_1’ +- ‘some_dir/some_file_2’ +- ‘some_dir/some_file_3’ +- ‘another_dir/some_file_1’ +- ‘another_dir/some_file_2’ +- ‘another_dir/some_file_3’ 1. Query the amount of rows in these files: diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index 9114bcb5a14..512f47a2b46 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -53,12 +53,12 @@ Constructions with `{}` are similar to the [remote table function](../../sql-ref 1. Suppose that we have several files with following URIs on HDFS: -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_3’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_3’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_3’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_3’ 2. Query the amount of rows in these files: diff --git a/docs/en/sql-reference/table-functions/index.md b/docs/en/sql-reference/table-functions/index.md index c3208d132b3..1ab1321170d 100644 --- a/docs/en/sql-reference/table-functions/index.md +++ b/docs/en/sql-reference/table-functions/index.md @@ -14,12 +14,12 @@ You can use table functions in: The method for creating a temporary table that is available only in the current query. The table is deleted when the query finishes. -- [CREATE TABLE AS \](../../sql-reference/statements/create/table.md) query. +- [CREATE TABLE AS \](../../sql-reference/statements/create/table.md) query. It's one of the methods of creating a table. !!! warning "Warning" - You can’t use table functions if the [allow\_ddl](../../operations/settings/permissions-for-queries.md#settings_allow_ddl) setting is disabled. + You can’t use table functions if the [allow_ddl](../../operations/settings/permissions-for-queries.md#settings_allow_ddl) setting is disabled. | Function | Description | |-----------------------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------| diff --git a/docs/en/sql-reference/table-functions/remote.md b/docs/en/sql-reference/table-functions/remote.md index a7bdca6218e..6da28362e23 100644 --- a/docs/en/sql-reference/table-functions/remote.md +++ b/docs/en/sql-reference/table-functions/remote.md @@ -54,7 +54,7 @@ example01-{01..02}-1 If you have multiple pairs of curly brackets, it generates the direct product of the corresponding sets. -Addresses and parts of addresses in curly brackets can be separated by the pipe symbol (\|). In this case, the corresponding sets of addresses are interpreted as replicas, and the query will be sent to the first healthy replica. However, the replicas are iterated in the order currently set in the [load\_balancing](../../operations/settings/settings.md) setting. +Addresses and parts of addresses in curly brackets can be separated by the pipe symbol (\|). In this case, the corresponding sets of addresses are interpreted as replicas, and the query will be sent to the first healthy replica. However, the replicas are iterated in the order currently set in the [load_balancing](../../operations/settings/settings.md) setting. Example: @@ -78,6 +78,6 @@ The `remote` table function can be useful in the following cases: If the user is not specified, `default` is used. If the password is not specified, an empty password is used. -`remoteSecure` - same as `remote` but with secured connection. Default port — [tcp\_port\_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) from config or 9440. +`remoteSecure` - same as `remote` but with secured connection. Default port — [tcp_port_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) from config or 9440. [Original article](https://clickhouse.tech/docs/en/query_language/table_functions/remote/) diff --git a/docs/en/whats-new/changelog/2017.md b/docs/en/whats-new/changelog/2017.md index 3b48e23233f..17d3efe7bab 100644 --- a/docs/en/whats-new/changelog/2017.md +++ b/docs/en/whats-new/changelog/2017.md @@ -157,13 +157,13 @@ This release contains bug fixes for the previous release 1.1.54276: #### New Features: {#new-features-4} - Added an optional WITH section for a SELECT query. Example query: `WITH 1+1 AS a SELECT a, a*a` -- INSERT can be performed synchronously in a Distributed table: OK is returned only after all the data is saved on all the shards. This is activated by the setting insert\_distributed\_sync=1. +- INSERT can be performed synchronously in a Distributed table: OK is returned only after all the data is saved on all the shards. This is activated by the setting insert_distributed_sync=1. - Added the UUID data type for working with 16-byte identifiers. - Added aliases of CHAR, FLOAT and other types for compatibility with the Tableau. - Added the functions toYYYYMM, toYYYYMMDD, and toYYYYMMDDhhmmss for converting time into numbers. - You can use IP addresses (together with the hostname) to identify servers for clustered DDL queries. - Added support for non-constant arguments and negative offsets in the function `substring(str, pos, len).` -- Added the max\_size parameter for the `groupArray(max_size)(column)` aggregate function, and optimized its performance. +- Added the max_size parameter for the `groupArray(max_size)(column)` aggregate function, and optimized its performance. #### Main Changes: {#main-changes} @@ -171,7 +171,7 @@ This release contains bug fixes for the previous release 1.1.54276: - Improved error messages for queries with invalid syntax. - Significantly reduced memory consumption and improved performance when merging large sections of MergeTree data. - Significantly increased the performance of data merges for the ReplacingMergeTree engine. -- Improved performance for asynchronous inserts from a Distributed table by combining multiple source inserts. To enable this functionality, use the setting distributed\_directory\_monitor\_batch\_inserts=1. +- Improved performance for asynchronous inserts from a Distributed table by combining multiple source inserts. To enable this functionality, use the setting distributed_directory_monitor_batch_inserts=1. #### Backward Incompatible Changes: {#backward-incompatible-changes-1} @@ -182,17 +182,17 @@ This release contains bug fixes for the previous release 1.1.54276: - Added the `output_format_json_quote_denormals` setting, which enables outputting nan and inf values in JSON format. - Optimized stream allocation when reading from a Distributed table. - Settings can be configured in readonly mode if the value doesn’t change. -- Added the ability to retrieve non-integer granules of the MergeTree engine in order to meet restrictions on the block size specified in the preferred\_block\_size\_bytes setting. The purpose is to reduce the consumption of RAM and increase cache locality when processing queries from tables with large columns. +- Added the ability to retrieve non-integer granules of the MergeTree engine in order to meet restrictions on the block size specified in the preferred_block_size_bytes setting. The purpose is to reduce the consumption of RAM and increase cache locality when processing queries from tables with large columns. - Efficient use of indexes that contain expressions like `toStartOfHour(x)` for conditions like `toStartOfHour(x) op сonstexpr.` -- Added new settings for MergeTree engines (the merge\_tree section in config.xml): - - replicated\_deduplication\_window\_seconds sets the number of seconds allowed for deduplicating inserts in Replicated tables. - - cleanup\_delay\_period sets how often to start cleanup to remove outdated data. - - replicated\_can\_become\_leader can prevent a replica from becoming the leader (and assigning merges). +- Added new settings for MergeTree engines (the merge_tree section in config.xml): + - replicated_deduplication_window_seconds sets the number of seconds allowed for deduplicating inserts in Replicated tables. + - cleanup_delay_period sets how often to start cleanup to remove outdated data. + - replicated_can_become_leader can prevent a replica from becoming the leader (and assigning merges). - Accelerated cleanup to remove outdated data from ZooKeeper. -- Multiple improvements and fixes for clustered DDL queries. Of particular interest is the new setting distributed\_ddl\_task\_timeout, which limits the time to wait for a response from the servers in the cluster. If a ddl request has not been performed on all hosts, a response will contain a timeout error and a request will be executed in an async mode. +- Multiple improvements and fixes for clustered DDL queries. Of particular interest is the new setting distributed_ddl_task_timeout, which limits the time to wait for a response from the servers in the cluster. If a ddl request has not been performed on all hosts, a response will contain a timeout error and a request will be executed in an async mode. - Improved display of stack traces in the server logs. - Added the “none” value for the compression method. -- You can use multiple dictionaries\_config sections in config.xml. +- You can use multiple dictionaries_config sections in config.xml. - It is possible to connect to MySQL through a socket in the file system. - The system.parts table has a new column with information about the size of marks, in bytes. @@ -201,19 +201,19 @@ This release contains bug fixes for the previous release 1.1.54276: - Distributed tables using a Merge table now work correctly for a SELECT query with a condition on the `_table` field. - Fixed a rare race condition in ReplicatedMergeTree when checking data parts. - Fixed possible freezing on “leader election” when starting a server. -- The max\_replica\_delay\_for\_distributed\_queries setting was ignored when using a local replica of the data source. This has been fixed. +- The max_replica_delay_for_distributed_queries setting was ignored when using a local replica of the data source. This has been fixed. - Fixed incorrect behavior of `ALTER TABLE CLEAR COLUMN IN PARTITION` when attempting to clean a non-existing column. - Fixed an exception in the multiIf function when using empty arrays or strings. - Fixed excessive memory allocations when deserializing Native format. - Fixed incorrect auto-update of Trie dictionaries. - Fixed an exception when running queries with a GROUP BY clause from a Merge table when using SAMPLE. -- Fixed a crash of GROUP BY when using distributed\_aggregation\_memory\_efficient=1. +- Fixed a crash of GROUP BY when using distributed_aggregation_memory_efficient=1. - Now you can specify the database.table in the right side of IN and JOIN. - Too many threads were used for parallel aggregation. This has been fixed. - Fixed how the “if” function works with FixedString arguments. - SELECT worked incorrectly from a Distributed table for shards with a weight of 0. This has been fixed. - Running `CREATE VIEW IF EXISTS no longer causes crashes.` -- Fixed incorrect behavior when input\_format\_skip\_unknown\_fields=1 is set and there are negative numbers. +- Fixed incorrect behavior when input_format_skip_unknown_fields=1 is set and there are negative numbers. - Fixed an infinite loop in the `dictGetHierarchy()` function if there is some invalid data in the dictionary. - Fixed `Syntax error: unexpected (...)` errors when running distributed queries with subqueries in an IN or JOIN clause and Merge tables. - Fixed an incorrect interpretation of a SELECT query from Dictionary tables. diff --git a/docs/en/whats-new/changelog/2018.md b/docs/en/whats-new/changelog/2018.md index 5ddd9a10f0f..b0c4e147352 100644 --- a/docs/en/whats-new/changelog/2018.md +++ b/docs/en/whats-new/changelog/2018.md @@ -9,12 +9,12 @@ toc_title: '2018' #### Bug Fixes: {#bug-fixes} -- Fixed an error that led to problems with updating dictionaries with the ODBC source. [\#3825](https://github.com/ClickHouse/ClickHouse/issues/3825), [\#3829](https://github.com/ClickHouse/ClickHouse/issues/3829) -- JIT compilation of aggregate functions now works with LowCardinality columns. [\#3838](https://github.com/ClickHouse/ClickHouse/issues/3838) +- Fixed an error that led to problems with updating dictionaries with the ODBC source. [#3825](https://github.com/ClickHouse/ClickHouse/issues/3825), [#3829](https://github.com/ClickHouse/ClickHouse/issues/3829) +- JIT compilation of aggregate functions now works with LowCardinality columns. [#3838](https://github.com/ClickHouse/ClickHouse/issues/3838) #### Improvements: {#improvements} -- Added the `low_cardinality_allow_in_native_format` setting (enabled by default). When disabled, LowCardinality columns will be converted to ordinary columns for SELECT queries and ordinary columns will be expected for INSERT queries. [\#3879](https://github.com/ClickHouse/ClickHouse/pull/3879) +- Added the `low_cardinality_allow_in_native_format` setting (enabled by default). When disabled, LowCardinality columns will be converted to ordinary columns for SELECT queries and ordinary columns will be expected for INSERT queries. [#3879](https://github.com/ClickHouse/ClickHouse/pull/3879) #### Build Improvements: {#build-improvements} @@ -24,79 +24,79 @@ toc_title: '2018' #### New Features: {#new-features} -- `DEFAULT` expressions are evaluated for missing fields when loading data in semi-structured input formats (`JSONEachRow`, `TSKV`). The feature is enabled with the `insert_sample_with_metadata` setting. [\#3555](https://github.com/ClickHouse/ClickHouse/pull/3555) -- The `ALTER TABLE` query now has the `MODIFY ORDER BY` action for changing the sorting key when adding or removing a table column. This is useful for tables in the `MergeTree` family that perform additional tasks when merging based on this sorting key, such as `SummingMergeTree`, `AggregatingMergeTree`, and so on. [\#3581](https://github.com/ClickHouse/ClickHouse/pull/3581) [\#3755](https://github.com/ClickHouse/ClickHouse/pull/3755) -- For tables in the `MergeTree` family, now you can specify a different sorting key (`ORDER BY`) and index (`PRIMARY KEY`). The sorting key can be longer than the index. [\#3581](https://github.com/ClickHouse/ClickHouse/pull/3581) +- `DEFAULT` expressions are evaluated for missing fields when loading data in semi-structured input formats (`JSONEachRow`, `TSKV`). The feature is enabled with the `insert_sample_with_metadata` setting. [#3555](https://github.com/ClickHouse/ClickHouse/pull/3555) +- The `ALTER TABLE` query now has the `MODIFY ORDER BY` action for changing the sorting key when adding or removing a table column. This is useful for tables in the `MergeTree` family that perform additional tasks when merging based on this sorting key, such as `SummingMergeTree`, `AggregatingMergeTree`, and so on. [#3581](https://github.com/ClickHouse/ClickHouse/pull/3581) [#3755](https://github.com/ClickHouse/ClickHouse/pull/3755) +- For tables in the `MergeTree` family, now you can specify a different sorting key (`ORDER BY`) and index (`PRIMARY KEY`). The sorting key can be longer than the index. [#3581](https://github.com/ClickHouse/ClickHouse/pull/3581) - Added the `hdfs` table function and the `HDFS` table engine for importing and exporting data to HDFS. [chenxing-xc](https://github.com/ClickHouse/ClickHouse/pull/3617) - Added functions for working with base64: `base64Encode`, `base64Decode`, `tryBase64Decode`. [Alexander Krasheninnikov](https://github.com/ClickHouse/ClickHouse/pull/3350) -- Now you can use a parameter to configure the precision of the `uniqCombined` aggregate function (select the number of HyperLogLog cells). [\#3406](https://github.com/ClickHouse/ClickHouse/pull/3406) -- Added the `system.contributors` table that contains the names of everyone who made commits in ClickHouse. [\#3452](https://github.com/ClickHouse/ClickHouse/pull/3452) -- Added the ability to omit the partition for the `ALTER TABLE ... FREEZE` query in order to back up all partitions at once. [\#3514](https://github.com/ClickHouse/ClickHouse/pull/3514) +- Now you can use a parameter to configure the precision of the `uniqCombined` aggregate function (select the number of HyperLogLog cells). [#3406](https://github.com/ClickHouse/ClickHouse/pull/3406) +- Added the `system.contributors` table that contains the names of everyone who made commits in ClickHouse. [#3452](https://github.com/ClickHouse/ClickHouse/pull/3452) +- Added the ability to omit the partition for the `ALTER TABLE ... FREEZE` query in order to back up all partitions at once. [#3514](https://github.com/ClickHouse/ClickHouse/pull/3514) - Added `dictGet` and `dictGetOrDefault` functions that don’t require specifying the type of return value. The type is determined automatically from the dictionary description. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3564) -- Now you can specify comments for a column in the table description and change it using `ALTER`. [\#3377](https://github.com/ClickHouse/ClickHouse/pull/3377) +- Now you can specify comments for a column in the table description and change it using `ALTER`. [#3377](https://github.com/ClickHouse/ClickHouse/pull/3377) - Reading is supported for `Join` type tables with simple keys. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3728) - Now you can specify the options `join_use_nulls`, `max_rows_in_join`, `max_bytes_in_join`, and `join_overflow_mode` when creating a `Join` type table. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3728) - Added the `joinGet` function that allows you to use a `Join` type table like a dictionary. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3728) -- Added the `partition_key`, `sorting_key`, `primary_key`, and `sampling_key` columns to the `system.tables` table in order to provide information about table keys. [\#3609](https://github.com/ClickHouse/ClickHouse/pull/3609) -- Added the `is_in_partition_key`, `is_in_sorting_key`, `is_in_primary_key`, and `is_in_sampling_key` columns to the `system.columns` table. [\#3609](https://github.com/ClickHouse/ClickHouse/pull/3609) +- Added the `partition_key`, `sorting_key`, `primary_key`, and `sampling_key` columns to the `system.tables` table in order to provide information about table keys. [#3609](https://github.com/ClickHouse/ClickHouse/pull/3609) +- Added the `is_in_partition_key`, `is_in_sorting_key`, `is_in_primary_key`, and `is_in_sampling_key` columns to the `system.columns` table. [#3609](https://github.com/ClickHouse/ClickHouse/pull/3609) - Added the `min_time` and `max_time` columns to the `system.parts` table. These columns are populated when the partitioning key is an expression consisting of `DateTime` columns. [Emmanuel Donin de Rosière](https://github.com/ClickHouse/ClickHouse/pull/3800) #### Bug Fixes: {#bug-fixes-1} -- Fixes and performance improvements for the `LowCardinality` data type. `GROUP BY` using `LowCardinality(Nullable(...))`. Getting the values of `extremes`. Processing high-order functions. `LEFT ARRAY JOIN`. Distributed `GROUP BY`. Functions that return `Array`. Execution of `ORDER BY`. Writing to `Distributed` tables (nicelulu). Backward compatibility for `INSERT` queries from old clients that implement the `Native` protocol. Support for `LowCardinality` for `JOIN`. Improved performance when working in a single stream. [\#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) -- Fixed how the `select_sequential_consistency` option works. Previously, when this setting was enabled, an incomplete result was sometimes returned after beginning to write to a new partition. [\#2863](https://github.com/ClickHouse/ClickHouse/pull/2863) -- Databases are correctly specified when executing DDL `ON CLUSTER` queries and `ALTER UPDATE/DELETE`. [\#3772](https://github.com/ClickHouse/ClickHouse/pull/3772) [\#3460](https://github.com/ClickHouse/ClickHouse/pull/3460) -- Databases are correctly specified for subqueries inside a VIEW. [\#3521](https://github.com/ClickHouse/ClickHouse/pull/3521) +- Fixes and performance improvements for the `LowCardinality` data type. `GROUP BY` using `LowCardinality(Nullable(...))`. Getting the values of `extremes`. Processing high-order functions. `LEFT ARRAY JOIN`. Distributed `GROUP BY`. Functions that return `Array`. Execution of `ORDER BY`. Writing to `Distributed` tables (nicelulu). Backward compatibility for `INSERT` queries from old clients that implement the `Native` protocol. Support for `LowCardinality` for `JOIN`. Improved performance when working in a single stream. [#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) +- Fixed how the `select_sequential_consistency` option works. Previously, when this setting was enabled, an incomplete result was sometimes returned after beginning to write to a new partition. [#2863](https://github.com/ClickHouse/ClickHouse/pull/2863) +- Databases are correctly specified when executing DDL `ON CLUSTER` queries and `ALTER UPDATE/DELETE`. [#3772](https://github.com/ClickHouse/ClickHouse/pull/3772) [#3460](https://github.com/ClickHouse/ClickHouse/pull/3460) +- Databases are correctly specified for subqueries inside a VIEW. [#3521](https://github.com/ClickHouse/ClickHouse/pull/3521) - Fixed a bug in `PREWHERE` with `FINAL` for `VersionedCollapsingMergeTree`. [7167bfd7](https://github.com/ClickHouse/ClickHouse/commit/7167bfd7b365538f7a91c4307ad77e552ab4e8c1) -- Now you can use `KILL QUERY` to cancel queries that have not started yet because they are waiting for the table to be locked. [\#3517](https://github.com/ClickHouse/ClickHouse/pull/3517) -- Corrected date and time calculations if the clocks were moved back at midnight (this happens in Iran, and happened in Moscow from 1981 to 1983). Previously, this led to the time being reset a day earlier than necessary, and also caused incorrect formatting of the date and time in text format. [\#3819](https://github.com/ClickHouse/ClickHouse/pull/3819) +- Now you can use `KILL QUERY` to cancel queries that have not started yet because they are waiting for the table to be locked. [#3517](https://github.com/ClickHouse/ClickHouse/pull/3517) +- Corrected date and time calculations if the clocks were moved back at midnight (this happens in Iran, and happened in Moscow from 1981 to 1983). Previously, this led to the time being reset a day earlier than necessary, and also caused incorrect formatting of the date and time in text format. [#3819](https://github.com/ClickHouse/ClickHouse/pull/3819) - Fixed bugs in some cases of `VIEW` and subqueries that omit the database. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/3521) -- Fixed a race condition when simultaneously reading from a `MATERIALIZED VIEW` and deleting a `MATERIALIZED VIEW` due to not locking the internal `MATERIALIZED VIEW`. [\#3404](https://github.com/ClickHouse/ClickHouse/pull/3404) [\#3694](https://github.com/ClickHouse/ClickHouse/pull/3694) -- Fixed the error `Lock handler cannot be nullptr.` [\#3689](https://github.com/ClickHouse/ClickHouse/pull/3689) -- Fixed query processing when the `compile_expressions` option is enabled (it’s enabled by default). Nondeterministic constant expressions like the `now` function are no longer unfolded. [\#3457](https://github.com/ClickHouse/ClickHouse/pull/3457) +- Fixed a race condition when simultaneously reading from a `MATERIALIZED VIEW` and deleting a `MATERIALIZED VIEW` due to not locking the internal `MATERIALIZED VIEW`. [#3404](https://github.com/ClickHouse/ClickHouse/pull/3404) [#3694](https://github.com/ClickHouse/ClickHouse/pull/3694) +- Fixed the error `Lock handler cannot be nullptr.` [#3689](https://github.com/ClickHouse/ClickHouse/pull/3689) +- Fixed query processing when the `compile_expressions` option is enabled (it’s enabled by default). Nondeterministic constant expressions like the `now` function are no longer unfolded. [#3457](https://github.com/ClickHouse/ClickHouse/pull/3457) - Fixed a crash when specifying a non-constant scale argument in `toDecimal32/64/128` functions. -- Fixed an error when trying to insert an array with `NULL` elements in the `Values` format into a column of type `Array` without `Nullable` (if `input_format_values_interpret_expressions` = 1). [\#3487](https://github.com/ClickHouse/ClickHouse/pull/3487) [\#3503](https://github.com/ClickHouse/ClickHouse/pull/3503) +- Fixed an error when trying to insert an array with `NULL` elements in the `Values` format into a column of type `Array` without `Nullable` (if `input_format_values_interpret_expressions` = 1). [#3487](https://github.com/ClickHouse/ClickHouse/pull/3487) [#3503](https://github.com/ClickHouse/ClickHouse/pull/3503) - Fixed continuous error logging in `DDLWorker` if ZooKeeper is not available. [8f50c620](https://github.com/ClickHouse/ClickHouse/commit/8f50c620334988b28018213ec0092fe6423847e2) -- Fixed the return type for `quantile*` functions from `Date` and `DateTime` types of arguments. [\#3580](https://github.com/ClickHouse/ClickHouse/pull/3580) -- Fixed the `WITH` clause if it specifies a simple alias without expressions. [\#3570](https://github.com/ClickHouse/ClickHouse/pull/3570) +- Fixed the return type for `quantile*` functions from `Date` and `DateTime` types of arguments. [#3580](https://github.com/ClickHouse/ClickHouse/pull/3580) +- Fixed the `WITH` clause if it specifies a simple alias without expressions. [#3570](https://github.com/ClickHouse/ClickHouse/pull/3570) - Fixed processing of queries with named sub-queries and qualified column names when `enable_optimize_predicate_expression` is enabled. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/3588) - Fixed the error `Attempt to attach to nullptr thread group` when working with materialized views. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3623) - Fixed a crash when passing certain incorrect arguments to the `arrayReverse` function. [73e3a7b6](https://github.com/ClickHouse/ClickHouse/commit/73e3a7b662161d6005e7727d8a711b930386b871) - Fixed the buffer overflow in the `extractURLParameter` function. Improved performance. Added correct processing of strings containing zero bytes. [141e9799](https://github.com/ClickHouse/ClickHouse/commit/141e9799e49201d84ea8e951d1bed4fb6d3dacb5) -- Fixed buffer overflow in the `lowerUTF8` and `upperUTF8` functions. Removed the ability to execute these functions over `FixedString` type arguments. [\#3662](https://github.com/ClickHouse/ClickHouse/pull/3662) -- Fixed a rare race condition when deleting `MergeTree` tables. [\#3680](https://github.com/ClickHouse/ClickHouse/pull/3680) -- Fixed a race condition when reading from `Buffer` tables and simultaneously performing `ALTER` or `DROP` on the target tables. [\#3719](https://github.com/ClickHouse/ClickHouse/pull/3719) -- Fixed a segfault if the `max_temporary_non_const_columns` limit was exceeded. [\#3788](https://github.com/ClickHouse/ClickHouse/pull/3788) +- Fixed buffer overflow in the `lowerUTF8` and `upperUTF8` functions. Removed the ability to execute these functions over `FixedString` type arguments. [#3662](https://github.com/ClickHouse/ClickHouse/pull/3662) +- Fixed a rare race condition when deleting `MergeTree` tables. [#3680](https://github.com/ClickHouse/ClickHouse/pull/3680) +- Fixed a race condition when reading from `Buffer` tables and simultaneously performing `ALTER` or `DROP` on the target tables. [#3719](https://github.com/ClickHouse/ClickHouse/pull/3719) +- Fixed a segfault if the `max_temporary_non_const_columns` limit was exceeded. [#3788](https://github.com/ClickHouse/ClickHouse/pull/3788) #### Improvements: {#improvements-1} -- The server does not write the processed configuration files to the `/etc/clickhouse-server/` directory. Instead, it saves them in the `preprocessed_configs` directory inside `path`. This means that the `/etc/clickhouse-server/` directory doesn’t have write access for the `clickhouse` user, which improves security. [\#2443](https://github.com/ClickHouse/ClickHouse/pull/2443) -- The `min_merge_bytes_to_use_direct_io` option is set to 10 GiB by default. A merge that forms large parts of tables from the MergeTree family will be performed in `O_DIRECT` mode, which prevents excessive page cache eviction. [\#3504](https://github.com/ClickHouse/ClickHouse/pull/3504) -- Accelerated server start when there is a very large number of tables. [\#3398](https://github.com/ClickHouse/ClickHouse/pull/3398) -- Added a connection pool and HTTP `Keep-Alive` for connections between replicas. [\#3594](https://github.com/ClickHouse/ClickHouse/pull/3594) +- The server does not write the processed configuration files to the `/etc/clickhouse-server/` directory. Instead, it saves them in the `preprocessed_configs` directory inside `path`. This means that the `/etc/clickhouse-server/` directory doesn’t have write access for the `clickhouse` user, which improves security. [#2443](https://github.com/ClickHouse/ClickHouse/pull/2443) +- The `min_merge_bytes_to_use_direct_io` option is set to 10 GiB by default. A merge that forms large parts of tables from the MergeTree family will be performed in `O_DIRECT` mode, which prevents excessive page cache eviction. [#3504](https://github.com/ClickHouse/ClickHouse/pull/3504) +- Accelerated server start when there is a very large number of tables. [#3398](https://github.com/ClickHouse/ClickHouse/pull/3398) +- Added a connection pool and HTTP `Keep-Alive` for connections between replicas. [#3594](https://github.com/ClickHouse/ClickHouse/pull/3594) - If the query syntax is invalid, the `400 Bad Request` code is returned in the `HTTP` interface (500 was returned previously). [31bc680a](https://github.com/ClickHouse/ClickHouse/commit/31bc680ac5f4bb1d0360a8ba4696fa84bb47d6ab) - The `join_default_strictness` option is set to `ALL` by default for compatibility. [120e2cbe](https://github.com/ClickHouse/ClickHouse/commit/120e2cbe2ff4fbad626c28042d9b28781c805afe) -- Removed logging to `stderr` from the `re2` library for invalid or complex regular expressions. [\#3723](https://github.com/ClickHouse/ClickHouse/pull/3723) -- Added for the `Kafka` table engine: checks for subscriptions before beginning to read from Kafka; the kafka\_max\_block\_size setting for the table. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3396) -- The `cityHash64`, `farmHash64`, `metroHash64`, `sipHash64`, `halfMD5`, `murmurHash2_32`, `murmurHash2_64`, `murmurHash3_32`, and `murmurHash3_64` functions now work for any number of arguments and for arguments in the form of tuples. [\#3451](https://github.com/ClickHouse/ClickHouse/pull/3451) [\#3519](https://github.com/ClickHouse/ClickHouse/pull/3519) +- Removed logging to `stderr` from the `re2` library for invalid or complex regular expressions. [#3723](https://github.com/ClickHouse/ClickHouse/pull/3723) +- Added for the `Kafka` table engine: checks for subscriptions before beginning to read from Kafka; the kafka_max_block_size setting for the table. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3396) +- The `cityHash64`, `farmHash64`, `metroHash64`, `sipHash64`, `halfMD5`, `murmurHash2_32`, `murmurHash2_64`, `murmurHash3_32`, and `murmurHash3_64` functions now work for any number of arguments and for arguments in the form of tuples. [#3451](https://github.com/ClickHouse/ClickHouse/pull/3451) [#3519](https://github.com/ClickHouse/ClickHouse/pull/3519) - The `arrayReverse` function now works with any types of arrays. [73e3a7b6](https://github.com/ClickHouse/ClickHouse/commit/73e3a7b662161d6005e7727d8a711b930386b871) - Added an optional parameter: the slot size for the `timeSlots` function. [Kirill Shvakov](https://github.com/ClickHouse/ClickHouse/pull/3724) - For `FULL` and `RIGHT JOIN`, the `max_block_size` setting is used for a stream of non-joined data from the right table. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3699) -- Added the `--secure` command line parameter in `clickhouse-benchmark` and `clickhouse-performance-test` to enable TLS. [\#3688](https://github.com/ClickHouse/ClickHouse/pull/3688) [\#3690](https://github.com/ClickHouse/ClickHouse/pull/3690) +- Added the `--secure` command line parameter in `clickhouse-benchmark` and `clickhouse-performance-test` to enable TLS. [#3688](https://github.com/ClickHouse/ClickHouse/pull/3688) [#3690](https://github.com/ClickHouse/ClickHouse/pull/3690) - Type conversion when the structure of a `Buffer` type table does not match the structure of the destination table. [Vitaly Baranov](https://github.com/ClickHouse/ClickHouse/pull/3603) -- Added the `tcp_keep_alive_timeout` option to enable keep-alive packets after inactivity for the specified time interval. [\#3441](https://github.com/ClickHouse/ClickHouse/pull/3441) -- Removed unnecessary quoting of values for the partition key in the `system.parts` table if it consists of a single column. [\#3652](https://github.com/ClickHouse/ClickHouse/pull/3652) -- The modulo function works for `Date` and `DateTime` data types. [\#3385](https://github.com/ClickHouse/ClickHouse/pull/3385) -- Added synonyms for the `POWER`, `LN`, `LCASE`, `UCASE`, `REPLACE`, `LOCATE`, `SUBSTR`, and `MID` functions. [\#3774](https://github.com/ClickHouse/ClickHouse/pull/3774) [\#3763](https://github.com/ClickHouse/ClickHouse/pull/3763) Some function names are case-insensitive for compatibility with the SQL standard. Added syntactic sugar `SUBSTRING(expr FROM start FOR length)` for compatibility with SQL. [\#3804](https://github.com/ClickHouse/ClickHouse/pull/3804) -- Added the ability to `mlock` memory pages corresponding to `clickhouse-server` executable code to prevent it from being forced out of memory. This feature is disabled by default. [\#3553](https://github.com/ClickHouse/ClickHouse/pull/3553) -- Improved performance when reading from `O_DIRECT` (with the `min_bytes_to_use_direct_io` option enabled). [\#3405](https://github.com/ClickHouse/ClickHouse/pull/3405) +- Added the `tcp_keep_alive_timeout` option to enable keep-alive packets after inactivity for the specified time interval. [#3441](https://github.com/ClickHouse/ClickHouse/pull/3441) +- Removed unnecessary quoting of values for the partition key in the `system.parts` table if it consists of a single column. [#3652](https://github.com/ClickHouse/ClickHouse/pull/3652) +- The modulo function works for `Date` and `DateTime` data types. [#3385](https://github.com/ClickHouse/ClickHouse/pull/3385) +- Added synonyms for the `POWER`, `LN`, `LCASE`, `UCASE`, `REPLACE`, `LOCATE`, `SUBSTR`, and `MID` functions. [#3774](https://github.com/ClickHouse/ClickHouse/pull/3774) [#3763](https://github.com/ClickHouse/ClickHouse/pull/3763) Some function names are case-insensitive for compatibility with the SQL standard. Added syntactic sugar `SUBSTRING(expr FROM start FOR length)` for compatibility with SQL. [#3804](https://github.com/ClickHouse/ClickHouse/pull/3804) +- Added the ability to `mlock` memory pages corresponding to `clickhouse-server` executable code to prevent it from being forced out of memory. This feature is disabled by default. [#3553](https://github.com/ClickHouse/ClickHouse/pull/3553) +- Improved performance when reading from `O_DIRECT` (with the `min_bytes_to_use_direct_io` option enabled). [#3405](https://github.com/ClickHouse/ClickHouse/pull/3405) - Improved performance of the `dictGet...OrDefault` function for a constant key argument and a non-constant default argument. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3563) -- The `firstSignificantSubdomain` function now processes the domains `gov`, `mil`, and `edu`. [Igor Hatarist](https://github.com/ClickHouse/ClickHouse/pull/3601) Improved performance. [\#3628](https://github.com/ClickHouse/ClickHouse/pull/3628) +- The `firstSignificantSubdomain` function now processes the domains `gov`, `mil`, and `edu`. [Igor Hatarist](https://github.com/ClickHouse/ClickHouse/pull/3601) Improved performance. [#3628](https://github.com/ClickHouse/ClickHouse/pull/3628) - Ability to specify custom environment variables for starting `clickhouse-server` using the `SYS-V init.d` script by defining `CLICKHOUSE_PROGRAM_ENV` in `/etc/default/clickhouse`. [Pavlo Bashynskyi](https://github.com/ClickHouse/ClickHouse/pull/3612) -- Correct return code for the clickhouse-server init script. [\#3516](https://github.com/ClickHouse/ClickHouse/pull/3516) -- The `system.metrics` table now has the `VersionInteger` metric, and `system.build_options` has the added line `VERSION_INTEGER`, which contains the numeric form of the ClickHouse version, such as `18016000`. [\#3644](https://github.com/ClickHouse/ClickHouse/pull/3644) -- Removed the ability to compare the `Date` type with a number to avoid potential errors like `date = 2018-12-17`, where quotes around the date are omitted by mistake. [\#3687](https://github.com/ClickHouse/ClickHouse/pull/3687) +- Correct return code for the clickhouse-server init script. [#3516](https://github.com/ClickHouse/ClickHouse/pull/3516) +- The `system.metrics` table now has the `VersionInteger` metric, and `system.build_options` has the added line `VERSION_INTEGER`, which contains the numeric form of the ClickHouse version, such as `18016000`. [#3644](https://github.com/ClickHouse/ClickHouse/pull/3644) +- Removed the ability to compare the `Date` type with a number to avoid potential errors like `date = 2018-12-17`, where quotes around the date are omitted by mistake. [#3687](https://github.com/ClickHouse/ClickHouse/pull/3687) - Fixed the behavior of stateful functions like `rowNumberInAllBlocks`. They previously output a result that was one number larger due to starting during query analysis. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3729) - If the `force_restore_data` file can’t be deleted, an error message is displayed. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3794) @@ -104,16 +104,16 @@ toc_title: '2018' - Updated the `jemalloc` library, which fixes a potential memory leak. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3557) - Profiling with `jemalloc` is enabled by default in order to debug builds. [2cc82f5c](https://github.com/ClickHouse/ClickHouse/commit/2cc82f5cbe266421cd4c1165286c2c47e5ffcb15) -- Added the ability to run integration tests when only `Docker` is installed on the system. [\#3650](https://github.com/ClickHouse/ClickHouse/pull/3650) -- Added the fuzz expression test in SELECT queries. [\#3442](https://github.com/ClickHouse/ClickHouse/pull/3442) -- Added a stress test for commits, which performs functional tests in parallel and in random order to detect more race conditions. [\#3438](https://github.com/ClickHouse/ClickHouse/pull/3438) +- Added the ability to run integration tests when only `Docker` is installed on the system. [#3650](https://github.com/ClickHouse/ClickHouse/pull/3650) +- Added the fuzz expression test in SELECT queries. [#3442](https://github.com/ClickHouse/ClickHouse/pull/3442) +- Added a stress test for commits, which performs functional tests in parallel and in random order to detect more race conditions. [#3438](https://github.com/ClickHouse/ClickHouse/pull/3438) - Improved the method for starting clickhouse-server in a Docker image. [Elghazal Ahmed](https://github.com/ClickHouse/ClickHouse/pull/3663) - For a Docker image, added support for initializing databases using files in the `/docker-entrypoint-initdb.d` directory. [Konstantin Lebedev](https://github.com/ClickHouse/ClickHouse/pull/3695) -- Fixes for builds on ARM. [\#3709](https://github.com/ClickHouse/ClickHouse/pull/3709) +- Fixes for builds on ARM. [#3709](https://github.com/ClickHouse/ClickHouse/pull/3709) #### Backward Incompatible Changes: {#backward-incompatible-changes} -- Removed the ability to compare the `Date` type with a number. Instead of `toDate('2018-12-18') = 17883`, you must use explicit type conversion `= toDate(17883)` [\#3687](https://github.com/ClickHouse/ClickHouse/pull/3687) +- Removed the ability to compare the `Date` type with a number. Instead of `toDate('2018-12-18') = 17883`, you must use explicit type conversion `= toDate(17883)` [#3687](https://github.com/ClickHouse/ClickHouse/pull/3687) ## ClickHouse Release 18.14 {#clickhouse-release-18-14} @@ -121,9 +121,9 @@ toc_title: '2018' #### Bug Fixes: {#bug-fixes-2} -- Fixed an error that led to problems with updating dictionaries with the ODBC source. [\#3825](https://github.com/ClickHouse/ClickHouse/issues/3825), [\#3829](https://github.com/ClickHouse/ClickHouse/issues/3829) -- Databases are correctly specified when executing DDL `ON CLUSTER` queries. [\#3460](https://github.com/ClickHouse/ClickHouse/pull/3460) -- Fixed a segfault if the `max_temporary_non_const_columns` limit was exceeded. [\#3788](https://github.com/ClickHouse/ClickHouse/pull/3788) +- Fixed an error that led to problems with updating dictionaries with the ODBC source. [#3825](https://github.com/ClickHouse/ClickHouse/issues/3825), [#3829](https://github.com/ClickHouse/ClickHouse/issues/3829) +- Databases are correctly specified when executing DDL `ON CLUSTER` queries. [#3460](https://github.com/ClickHouse/ClickHouse/pull/3460) +- Fixed a segfault if the `max_temporary_non_const_columns` limit was exceeded. [#3788](https://github.com/ClickHouse/ClickHouse/pull/3788) #### Build Improvements: {#build-improvements-2} @@ -133,64 +133,64 @@ toc_title: '2018' #### Bug Fixes: {#bug-fixes-3} -- Fixed error in `dictGet...` function for dictionaries of type `range`, if one of the arguments is constant and other is not. [\#3751](https://github.com/ClickHouse/ClickHouse/pull/3751) -- Fixed error that caused messages `netlink: '...': attribute type 1 has an invalid length` to be printed in Linux kernel log, that was happening only on fresh enough versions of Linux kernel. [\#3749](https://github.com/ClickHouse/ClickHouse/pull/3749) +- Fixed error in `dictGet...` function for dictionaries of type `range`, if one of the arguments is constant and other is not. [#3751](https://github.com/ClickHouse/ClickHouse/pull/3751) +- Fixed error that caused messages `netlink: '...': attribute type 1 has an invalid length` to be printed in Linux kernel log, that was happening only on fresh enough versions of Linux kernel. [#3749](https://github.com/ClickHouse/ClickHouse/pull/3749) - Fixed segfault in function `empty` for argument of `FixedString` type. [Daniel, Dao Quang Minh](https://github.com/ClickHouse/ClickHouse/pull/3703) -- Fixed excessive memory allocation when using large value of `max_query_size` setting (a memory chunk of `max_query_size` bytes was preallocated at once). [\#3720](https://github.com/ClickHouse/ClickHouse/pull/3720) +- Fixed excessive memory allocation when using large value of `max_query_size` setting (a memory chunk of `max_query_size` bytes was preallocated at once). [#3720](https://github.com/ClickHouse/ClickHouse/pull/3720) #### Build Changes: {#build-changes} -- Fixed build with LLVM/Clang libraries of version 7 from the OS packages (these libraries are used for runtime query compilation). [\#3582](https://github.com/ClickHouse/ClickHouse/pull/3582) +- Fixed build with LLVM/Clang libraries of version 7 from the OS packages (these libraries are used for runtime query compilation). [#3582](https://github.com/ClickHouse/ClickHouse/pull/3582) ### ClickHouse Release 18.14.17, 2018-11-30 {#clickhouse-release-18-14-17-2018-11-30} #### Bug Fixes: {#bug-fixes-4} -- Fixed cases when the ODBC bridge process did not terminate with the main server process. [\#3642](https://github.com/ClickHouse/ClickHouse/pull/3642) -- Fixed synchronous insertion into the `Distributed` table with a columns list that differs from the column list of the remote table. [\#3673](https://github.com/ClickHouse/ClickHouse/pull/3673) -- Fixed a rare race condition that can lead to a crash when dropping a MergeTree table. [\#3643](https://github.com/ClickHouse/ClickHouse/pull/3643) -- Fixed a query deadlock in case when query thread creation fails with the `Resource temporarily unavailable` error. [\#3643](https://github.com/ClickHouse/ClickHouse/pull/3643) -- Fixed parsing of the `ENGINE` clause when the `CREATE AS table` syntax was used and the `ENGINE` clause was specified before the `AS table` (the error resulted in ignoring the specified engine). [\#3692](https://github.com/ClickHouse/ClickHouse/pull/3692) +- Fixed cases when the ODBC bridge process did not terminate with the main server process. [#3642](https://github.com/ClickHouse/ClickHouse/pull/3642) +- Fixed synchronous insertion into the `Distributed` table with a columns list that differs from the column list of the remote table. [#3673](https://github.com/ClickHouse/ClickHouse/pull/3673) +- Fixed a rare race condition that can lead to a crash when dropping a MergeTree table. [#3643](https://github.com/ClickHouse/ClickHouse/pull/3643) +- Fixed a query deadlock in case when query thread creation fails with the `Resource temporarily unavailable` error. [#3643](https://github.com/ClickHouse/ClickHouse/pull/3643) +- Fixed parsing of the `ENGINE` clause when the `CREATE AS table` syntax was used and the `ENGINE` clause was specified before the `AS table` (the error resulted in ignoring the specified engine). [#3692](https://github.com/ClickHouse/ClickHouse/pull/3692) ### ClickHouse Release 18.14.15, 2018-11-21 {#clickhouse-release-18-14-15-2018-11-21} #### Bug Fixes: {#bug-fixes-5} -- The size of memory chunk was overestimated while deserializing the column of type `Array(String)` that leads to “Memory limit exceeded” errors. The issue appeared in version 18.12.13. [\#3589](https://github.com/ClickHouse/ClickHouse/issues/3589) +- The size of memory chunk was overestimated while deserializing the column of type `Array(String)` that leads to “Memory limit exceeded” errors. The issue appeared in version 18.12.13. [#3589](https://github.com/ClickHouse/ClickHouse/issues/3589) ### ClickHouse Release 18.14.14, 2018-11-20 {#clickhouse-release-18-14-14-2018-11-20} #### Bug Fixes: {#bug-fixes-6} -- Fixed `ON CLUSTER` queries when cluster configured as secure (flag ``). [\#3599](https://github.com/ClickHouse/ClickHouse/pull/3599) +- Fixed `ON CLUSTER` queries when cluster configured as secure (flag ``). [#3599](https://github.com/ClickHouse/ClickHouse/pull/3599) #### Build Changes: {#build-changes-1} -- Fixed problems (llvm-7 from system, macos) [\#3582](https://github.com/ClickHouse/ClickHouse/pull/3582) +- Fixed problems (llvm-7 from system, macos) [#3582](https://github.com/ClickHouse/ClickHouse/pull/3582) ### ClickHouse Release 18.14.13, 2018-11-08 {#clickhouse-release-18-14-13-2018-11-08} #### Bug Fixes: {#bug-fixes-7} -- Fixed the `Block structure mismatch in MergingSorted stream` error. [\#3162](https://github.com/ClickHouse/ClickHouse/issues/3162) -- Fixed `ON CLUSTER` queries in case when secure connections were turned on in the cluster config (the `` flag). [\#3465](https://github.com/ClickHouse/ClickHouse/pull/3465) -- Fixed an error in queries that used `SAMPLE`, `PREWHERE` and alias columns. [\#3543](https://github.com/ClickHouse/ClickHouse/pull/3543) +- Fixed the `Block structure mismatch in MergingSorted stream` error. [#3162](https://github.com/ClickHouse/ClickHouse/issues/3162) +- Fixed `ON CLUSTER` queries in case when secure connections were turned on in the cluster config (the `` flag). [#3465](https://github.com/ClickHouse/ClickHouse/pull/3465) +- Fixed an error in queries that used `SAMPLE`, `PREWHERE` and alias columns. [#3543](https://github.com/ClickHouse/ClickHouse/pull/3543) - Fixed a rare `unknown compression method` error when the `min_bytes_to_use_direct_io` setting was enabled. [3544](https://github.com/ClickHouse/ClickHouse/pull/3544) #### Performance Improvements: {#performance-improvements} - Fixed performance regression of queries with `GROUP BY` of columns of UInt16 or Date type when executing on AMD EPYC processors. [Igor Lapko](https://github.com/ClickHouse/ClickHouse/pull/3512) -- Fixed performance regression of queries that process long strings. [\#3530](https://github.com/ClickHouse/ClickHouse/pull/3530) +- Fixed performance regression of queries that process long strings. [#3530](https://github.com/ClickHouse/ClickHouse/pull/3530) #### Build Improvements: {#build-improvements-3} -- Improvements for simplifying the Arcadia build. [\#3475](https://github.com/ClickHouse/ClickHouse/pull/3475), [\#3535](https://github.com/ClickHouse/ClickHouse/pull/3535) +- Improvements for simplifying the Arcadia build. [#3475](https://github.com/ClickHouse/ClickHouse/pull/3475), [#3535](https://github.com/ClickHouse/ClickHouse/pull/3535) ### ClickHouse Release 18.14.12, 2018-11-02 {#clickhouse-release-18-14-12-2018-11-02} #### Bug Fixes: {#bug-fixes-8} -- Fixed a crash on joining two unnamed subqueries. [\#3505](https://github.com/ClickHouse/ClickHouse/pull/3505) +- Fixed a crash on joining two unnamed subqueries. [#3505](https://github.com/ClickHouse/ClickHouse/pull/3505) - Fixed generating incorrect queries (with an empty `WHERE` clause) when querying external databases. [hotid](https://github.com/ClickHouse/ClickHouse/pull/3477) - Fixed using an incorrect timeout value in ODBC dictionaries. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3511) @@ -198,106 +198,106 @@ toc_title: '2018' #### Bug Fixes: {#bug-fixes-9} -- Fixed the error `Block structure mismatch in UNION stream: different number of columns` in LIMIT queries. [\#2156](https://github.com/ClickHouse/ClickHouse/issues/2156) -- Fixed errors when merging data in tables containing arrays inside Nested structures. [\#3397](https://github.com/ClickHouse/ClickHouse/pull/3397) -- Fixed incorrect query results if the `merge_tree_uniform_read_distribution` setting is disabled (it is enabled by default). [\#3429](https://github.com/ClickHouse/ClickHouse/pull/3429) -- Fixed an error on inserts to a Distributed table in Native format. [\#3411](https://github.com/ClickHouse/ClickHouse/issues/3411) +- Fixed the error `Block structure mismatch in UNION stream: different number of columns` in LIMIT queries. [#2156](https://github.com/ClickHouse/ClickHouse/issues/2156) +- Fixed errors when merging data in tables containing arrays inside Nested structures. [#3397](https://github.com/ClickHouse/ClickHouse/pull/3397) +- Fixed incorrect query results if the `merge_tree_uniform_read_distribution` setting is disabled (it is enabled by default). [#3429](https://github.com/ClickHouse/ClickHouse/pull/3429) +- Fixed an error on inserts to a Distributed table in Native format. [#3411](https://github.com/ClickHouse/ClickHouse/issues/3411) ### ClickHouse Release 18.14.10, 2018-10-23 {#clickhouse-release-18-14-10-2018-10-23} -- The `compile_expressions` setting (JIT compilation of expressions) is disabled by default. [\#3410](https://github.com/ClickHouse/ClickHouse/pull/3410) +- The `compile_expressions` setting (JIT compilation of expressions) is disabled by default. [#3410](https://github.com/ClickHouse/ClickHouse/pull/3410) - The `enable_optimize_predicate_expression` setting is disabled by default. ### ClickHouse Release 18.14.9, 2018-10-16 {#clickhouse-release-18-14-9-2018-10-16} #### New Features: {#new-features-1} -- The `WITH CUBE` modifier for `GROUP BY` (the alternative syntax `GROUP BY CUBE(...)` is also available). [\#3172](https://github.com/ClickHouse/ClickHouse/pull/3172) +- The `WITH CUBE` modifier for `GROUP BY` (the alternative syntax `GROUP BY CUBE(...)` is also available). [#3172](https://github.com/ClickHouse/ClickHouse/pull/3172) - Added the `formatDateTime` function. [Alexandr Krasheninnikov](https://github.com/ClickHouse/ClickHouse/pull/2770) - Added the `JDBC` table engine and `jdbc` table function (requires installing clickhouse-jdbc-bridge). [Alexandr Krasheninnikov](https://github.com/ClickHouse/ClickHouse/pull/3210) -- Added functions for working with the ISO week number: `toISOWeek`, `toISOYear`, `toStartOfISOYear`, and `toDayOfYear`. [\#3146](https://github.com/ClickHouse/ClickHouse/pull/3146) -- Now you can use `Nullable` columns for `MySQL` and `ODBC` tables. [\#3362](https://github.com/ClickHouse/ClickHouse/pull/3362) +- Added functions for working with the ISO week number: `toISOWeek`, `toISOYear`, `toStartOfISOYear`, and `toDayOfYear`. [#3146](https://github.com/ClickHouse/ClickHouse/pull/3146) +- Now you can use `Nullable` columns for `MySQL` and `ODBC` tables. [#3362](https://github.com/ClickHouse/ClickHouse/pull/3362) - Nested data structures can be read as nested objects in `JSONEachRow` format. Added the `input_format_import_nested_json` setting. [Veloman Yunkan](https://github.com/ClickHouse/ClickHouse/pull/3144) - Parallel processing is available for many `MATERIALIZED VIEW`s when inserting data. See the `parallel_view_processing` setting. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3208) -- Added the `SYSTEM FLUSH LOGS` query (forced log flushes to system tables such as `query_log`) [\#3321](https://github.com/ClickHouse/ClickHouse/pull/3321) -- Now you can use pre-defined `database` and `table` macros when declaring `Replicated` tables. [\#3251](https://github.com/ClickHouse/ClickHouse/pull/3251) -- Added the ability to read `Decimal` type values in engineering notation (indicating powers of ten). [\#3153](https://github.com/ClickHouse/ClickHouse/pull/3153) +- Added the `SYSTEM FLUSH LOGS` query (forced log flushes to system tables such as `query_log`) [#3321](https://github.com/ClickHouse/ClickHouse/pull/3321) +- Now you can use pre-defined `database` and `table` macros when declaring `Replicated` tables. [#3251](https://github.com/ClickHouse/ClickHouse/pull/3251) +- Added the ability to read `Decimal` type values in engineering notation (indicating powers of ten). [#3153](https://github.com/ClickHouse/ClickHouse/pull/3153) #### Experimental Features: {#experimental-features} -- Optimization of the GROUP BY clause for `LowCardinality data types.` [\#3138](https://github.com/ClickHouse/ClickHouse/pull/3138) -- Optimized calculation of expressions for `LowCardinality data types.` [\#3200](https://github.com/ClickHouse/ClickHouse/pull/3200) +- Optimization of the GROUP BY clause for `LowCardinality data types.` [#3138](https://github.com/ClickHouse/ClickHouse/pull/3138) +- Optimized calculation of expressions for `LowCardinality data types.` [#3200](https://github.com/ClickHouse/ClickHouse/pull/3200) #### Improvements: {#improvements-2} -- Significantly reduced memory consumption for queries with `ORDER BY` and `LIMIT`. See the `max_bytes_before_remerge_sort` setting. [\#3205](https://github.com/ClickHouse/ClickHouse/pull/3205) -- In the absence of `JOIN` (`LEFT`, `INNER`, …), `INNER JOIN` is assumed. [\#3147](https://github.com/ClickHouse/ClickHouse/pull/3147) +- Significantly reduced memory consumption for queries with `ORDER BY` and `LIMIT`. See the `max_bytes_before_remerge_sort` setting. [#3205](https://github.com/ClickHouse/ClickHouse/pull/3205) +- In the absence of `JOIN` (`LEFT`, `INNER`, …), `INNER JOIN` is assumed. [#3147](https://github.com/ClickHouse/ClickHouse/pull/3147) - Qualified asterisks work correctly in queries with `JOIN`. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/3202) - The `ODBC` table engine correctly chooses the method for quoting identifiers in the SQL dialect of a remote database. [Alexandr Krasheninnikov](https://github.com/ClickHouse/ClickHouse/pull/3210) - The `compile_expressions` setting (JIT compilation of expressions) is enabled by default. -- Fixed behavior for simultaneous DROP DATABASE/TABLE IF EXISTS and CREATE DATABASE/TABLE IF NOT EXISTS. Previously, a `CREATE DATABASE ... IF NOT EXISTS` query could return the error message “File … already exists”, and the `CREATE TABLE ... IF NOT EXISTS` and `DROP TABLE IF EXISTS` queries could return `Table ... is creating or attaching right now`. [\#3101](https://github.com/ClickHouse/ClickHouse/pull/3101) -- LIKE and IN expressions with a constant right half are passed to the remote server when querying from MySQL or ODBC tables. [\#3182](https://github.com/ClickHouse/ClickHouse/pull/3182) -- Comparisons with constant expressions in a WHERE clause are passed to the remote server when querying from MySQL and ODBC tables. Previously, only comparisons with constants were passed. [\#3182](https://github.com/ClickHouse/ClickHouse/pull/3182) +- Fixed behavior for simultaneous DROP DATABASE/TABLE IF EXISTS and CREATE DATABASE/TABLE IF NOT EXISTS. Previously, a `CREATE DATABASE ... IF NOT EXISTS` query could return the error message “File … already exists”, and the `CREATE TABLE ... IF NOT EXISTS` and `DROP TABLE IF EXISTS` queries could return `Table ... is creating or attaching right now`. [#3101](https://github.com/ClickHouse/ClickHouse/pull/3101) +- LIKE and IN expressions with a constant right half are passed to the remote server when querying from MySQL or ODBC tables. [#3182](https://github.com/ClickHouse/ClickHouse/pull/3182) +- Comparisons with constant expressions in a WHERE clause are passed to the remote server when querying from MySQL and ODBC tables. Previously, only comparisons with constants were passed. [#3182](https://github.com/ClickHouse/ClickHouse/pull/3182) - Correct calculation of row width in the terminal for `Pretty` formats, including strings with hieroglyphs. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3257). - `ON CLUSTER` can be specified for `ALTER UPDATE` queries. -- Improved performance for reading data in `JSONEachRow` format. [\#3332](https://github.com/ClickHouse/ClickHouse/pull/3332) -- Added synonyms for the `LENGTH` and `CHARACTER_LENGTH` functions for compatibility. The `CONCAT` function is no longer case-sensitive. [\#3306](https://github.com/ClickHouse/ClickHouse/pull/3306) -- Added the `TIMESTAMP` synonym for the `DateTime` type. [\#3390](https://github.com/ClickHouse/ClickHouse/pull/3390) -- There is always space reserved for query\_id in the server logs, even if the log line is not related to a query. This makes it easier to parse server text logs with third-party tools. -- Memory consumption by a query is logged when it exceeds the next level of an integer number of gigabytes. [\#3205](https://github.com/ClickHouse/ClickHouse/pull/3205) -- Added compatibility mode for the case when the client library that uses the Native protocol sends fewer columns by mistake than the server expects for the INSERT query. This scenario was possible when using the clickhouse-cpp library. Previously, this scenario caused the server to crash. [\#3171](https://github.com/ClickHouse/ClickHouse/pull/3171) -- In a user-defined WHERE expression in `clickhouse-copier`, you can now use a `partition_key` alias (for additional filtering by source table partition). This is useful if the partitioning scheme changes during copying, but only changes slightly. [\#3166](https://github.com/ClickHouse/ClickHouse/pull/3166) +- Improved performance for reading data in `JSONEachRow` format. [#3332](https://github.com/ClickHouse/ClickHouse/pull/3332) +- Added synonyms for the `LENGTH` and `CHARACTER_LENGTH` functions for compatibility. The `CONCAT` function is no longer case-sensitive. [#3306](https://github.com/ClickHouse/ClickHouse/pull/3306) +- Added the `TIMESTAMP` synonym for the `DateTime` type. [#3390](https://github.com/ClickHouse/ClickHouse/pull/3390) +- There is always space reserved for query_id in the server logs, even if the log line is not related to a query. This makes it easier to parse server text logs with third-party tools. +- Memory consumption by a query is logged when it exceeds the next level of an integer number of gigabytes. [#3205](https://github.com/ClickHouse/ClickHouse/pull/3205) +- Added compatibility mode for the case when the client library that uses the Native protocol sends fewer columns by mistake than the server expects for the INSERT query. This scenario was possible when using the clickhouse-cpp library. Previously, this scenario caused the server to crash. [#3171](https://github.com/ClickHouse/ClickHouse/pull/3171) +- In a user-defined WHERE expression in `clickhouse-copier`, you can now use a `partition_key` alias (for additional filtering by source table partition). This is useful if the partitioning scheme changes during copying, but only changes slightly. [#3166](https://github.com/ClickHouse/ClickHouse/pull/3166) - The workflow of the `Kafka` engine has been moved to a background thread pool in order to automatically reduce the speed of data reading at high loads. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3215). - Support for reading `Tuple` and `Nested` values of structures like `struct` in the `Cap'n'Proto format`. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3216) - The list of top-level domains for the `firstSignificantSubdomain` function now includes the domain `biz`. [decaseal](https://github.com/ClickHouse/ClickHouse/pull/3219) -- In the configuration of external dictionaries, `null_value` is interpreted as the value of the default data type. [\#3330](https://github.com/ClickHouse/ClickHouse/pull/3330) +- In the configuration of external dictionaries, `null_value` is interpreted as the value of the default data type. [#3330](https://github.com/ClickHouse/ClickHouse/pull/3330) - Support for the `intDiv` and `intDivOrZero` functions for `Decimal`. [b48402e8](https://github.com/ClickHouse/ClickHouse/commit/b48402e8712e2b9b151e0eef8193811d433a1264) -- Support for the `Date`, `DateTime`, `UUID`, and `Decimal` types as a key for the `sumMap` aggregate function. [\#3281](https://github.com/ClickHouse/ClickHouse/pull/3281) -- Support for the `Decimal` data type in external dictionaries. [\#3324](https://github.com/ClickHouse/ClickHouse/pull/3324) -- Support for the `Decimal` data type in `SummingMergeTree` tables. [\#3348](https://github.com/ClickHouse/ClickHouse/pull/3348) -- Added specializations for `UUID` in `if`. [\#3366](https://github.com/ClickHouse/ClickHouse/pull/3366) -- Reduced the number of `open` and `close` system calls when reading from a `MergeTree table`. [\#3283](https://github.com/ClickHouse/ClickHouse/pull/3283) +- Support for the `Date`, `DateTime`, `UUID`, and `Decimal` types as a key for the `sumMap` aggregate function. [#3281](https://github.com/ClickHouse/ClickHouse/pull/3281) +- Support for the `Decimal` data type in external dictionaries. [#3324](https://github.com/ClickHouse/ClickHouse/pull/3324) +- Support for the `Decimal` data type in `SummingMergeTree` tables. [#3348](https://github.com/ClickHouse/ClickHouse/pull/3348) +- Added specializations for `UUID` in `if`. [#3366](https://github.com/ClickHouse/ClickHouse/pull/3366) +- Reduced the number of `open` and `close` system calls when reading from a `MergeTree table`. [#3283](https://github.com/ClickHouse/ClickHouse/pull/3283) - A `TRUNCATE TABLE` query can be executed on any replica (the query is passed to the leader replica). [Kirill Shvakov](https://github.com/ClickHouse/ClickHouse/pull/3375) #### Bug Fixes: {#bug-fixes-10} -- Fixed an issue with `Dictionary` tables for `range_hashed` dictionaries. This error occurred in version 18.12.17. [\#1702](https://github.com/ClickHouse/ClickHouse/pull/1702) -- Fixed an error when loading `range_hashed` dictionaries (the message `Unsupported type Nullable (...)`). This error occurred in version 18.12.17. [\#3362](https://github.com/ClickHouse/ClickHouse/pull/3362) -- Fixed errors in the `pointInPolygon` function due to the accumulation of inaccurate calculations for polygons with a large number of vertices located close to each other. [\#3331](https://github.com/ClickHouse/ClickHouse/pull/3331) [\#3341](https://github.com/ClickHouse/ClickHouse/pull/3341) -- If after merging data parts, the checksum for the resulting part differs from the result of the same merge in another replica, the result of the merge is deleted and the data part is downloaded from the other replica (this is the correct behavior). But after downloading the data part, it couldn’t be added to the working set because of an error that the part already exists (because the data part was deleted with some delay after the merge). This led to cyclical attempts to download the same data. [\#3194](https://github.com/ClickHouse/ClickHouse/pull/3194) +- Fixed an issue with `Dictionary` tables for `range_hashed` dictionaries. This error occurred in version 18.12.17. [#1702](https://github.com/ClickHouse/ClickHouse/pull/1702) +- Fixed an error when loading `range_hashed` dictionaries (the message `Unsupported type Nullable (...)`). This error occurred in version 18.12.17. [#3362](https://github.com/ClickHouse/ClickHouse/pull/3362) +- Fixed errors in the `pointInPolygon` function due to the accumulation of inaccurate calculations for polygons with a large number of vertices located close to each other. [#3331](https://github.com/ClickHouse/ClickHouse/pull/3331) [#3341](https://github.com/ClickHouse/ClickHouse/pull/3341) +- If after merging data parts, the checksum for the resulting part differs from the result of the same merge in another replica, the result of the merge is deleted and the data part is downloaded from the other replica (this is the correct behavior). But after downloading the data part, it couldn’t be added to the working set because of an error that the part already exists (because the data part was deleted with some delay after the merge). This led to cyclical attempts to download the same data. [#3194](https://github.com/ClickHouse/ClickHouse/pull/3194) - Fixed incorrect calculation of total memory consumption by queries (because of incorrect calculation, the `max_memory_usage_for_all_queries` setting worked incorrectly and the `MemoryTracking` metric had an incorrect value). This error occurred in version 18.12.13. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3344) -- Fixed the functionality of `CREATE TABLE ... ON CLUSTER ... AS SELECT ...` This error occurred in version 18.12.13. [\#3247](https://github.com/ClickHouse/ClickHouse/pull/3247) -- Fixed unnecessary preparation of data structures for `JOIN`s on the server that initiates the query if the `JOIN` is only performed on remote servers. [\#3340](https://github.com/ClickHouse/ClickHouse/pull/3340) +- Fixed the functionality of `CREATE TABLE ... ON CLUSTER ... AS SELECT ...` This error occurred in version 18.12.13. [#3247](https://github.com/ClickHouse/ClickHouse/pull/3247) +- Fixed unnecessary preparation of data structures for `JOIN`s on the server that initiates the query if the `JOIN` is only performed on remote servers. [#3340](https://github.com/ClickHouse/ClickHouse/pull/3340) - Fixed bugs in the `Kafka` engine: deadlocks after exceptions when starting to read data, and locks upon completion [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3215). - For `Kafka` tables, the optional `schema` parameter was not passed (the schema of the `Cap'n'Proto` format). [Vojtech Splichal](https://github.com/ClickHouse/ClickHouse/pull/3150) - If the ensemble of ZooKeeper servers has servers that accept the connection but then immediately close it instead of responding to the handshake, ClickHouse chooses to connect another server. Previously, this produced the error `Cannot read all data. Bytes read: 0. Bytes expected: 4.` and the server couldn’t start. [8218cf3a](https://github.com/ClickHouse/ClickHouse/commit/8218cf3a5f39a43401953769d6d12a0bb8d29da9) - If the ensemble of ZooKeeper servers contains servers for which the DNS query returns an error, these servers are ignored. [17b8e209](https://github.com/ClickHouse/ClickHouse/commit/17b8e209221061325ad7ba0539f03c6e65f87f29) -- Fixed type conversion between `Date` and `DateTime` when inserting data in the `VALUES` format (if `input_format_values_interpret_expressions = 1`). Previously, the conversion was performed between the numerical value of the number of days in Unix Epoch time and the Unix timestamp, which led to unexpected results. [\#3229](https://github.com/ClickHouse/ClickHouse/pull/3229) -- Corrected type conversion between `Decimal` and integer numbers. [\#3211](https://github.com/ClickHouse/ClickHouse/pull/3211) +- Fixed type conversion between `Date` and `DateTime` when inserting data in the `VALUES` format (if `input_format_values_interpret_expressions = 1`). Previously, the conversion was performed between the numerical value of the number of days in Unix Epoch time and the Unix timestamp, which led to unexpected results. [#3229](https://github.com/ClickHouse/ClickHouse/pull/3229) +- Corrected type conversion between `Decimal` and integer numbers. [#3211](https://github.com/ClickHouse/ClickHouse/pull/3211) - Fixed errors in the `enable_optimize_predicate_expression` setting. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/3231) -- Fixed a parsing error in CSV format with floating-point numbers if a non-default CSV separator is used, such as `;` [\#3155](https://github.com/ClickHouse/ClickHouse/pull/3155) +- Fixed a parsing error in CSV format with floating-point numbers if a non-default CSV separator is used, such as `;` [#3155](https://github.com/ClickHouse/ClickHouse/pull/3155) - Fixed the `arrayCumSumNonNegative` function (it does not accumulate negative values if the accumulator is less than zero). [Aleksey Studnev](https://github.com/ClickHouse/ClickHouse/pull/3163) -- Fixed how `Merge` tables work on top of `Distributed` tables when using `PREWHERE`. [\#3165](https://github.com/ClickHouse/ClickHouse/pull/3165) +- Fixed how `Merge` tables work on top of `Distributed` tables when using `PREWHERE`. [#3165](https://github.com/ClickHouse/ClickHouse/pull/3165) - Bug fixes in the `ALTER UPDATE` query. -- Fixed bugs in the `odbc` table function that appeared in version 18.12. [\#3197](https://github.com/ClickHouse/ClickHouse/pull/3197) -- Fixed the operation of aggregate functions with `StateArray` combinators. [\#3188](https://github.com/ClickHouse/ClickHouse/pull/3188) +- Fixed bugs in the `odbc` table function that appeared in version 18.12. [#3197](https://github.com/ClickHouse/ClickHouse/pull/3197) +- Fixed the operation of aggregate functions with `StateArray` combinators. [#3188](https://github.com/ClickHouse/ClickHouse/pull/3188) - Fixed a crash when dividing a `Decimal` value by zero. [69dd6609](https://github.com/ClickHouse/ClickHouse/commit/69dd6609193beb4e7acd3e6ad216eca0ccfb8179) -- Fixed output of types for operations using `Decimal` and integer arguments. [\#3224](https://github.com/ClickHouse/ClickHouse/pull/3224) +- Fixed output of types for operations using `Decimal` and integer arguments. [#3224](https://github.com/ClickHouse/ClickHouse/pull/3224) - Fixed the segfault during `GROUP BY` on `Decimal128`. [3359ba06](https://github.com/ClickHouse/ClickHouse/commit/3359ba06c39fcd05bfdb87d6c64154819621e13a) -- The `log_query_threads` setting (logging information about each thread of query execution) now takes effect only if the `log_queries` option (logging information about queries) is set to 1. Since the `log_query_threads` option is enabled by default, information about threads was previously logged even if query logging was disabled. [\#3241](https://github.com/ClickHouse/ClickHouse/pull/3241) +- The `log_query_threads` setting (logging information about each thread of query execution) now takes effect only if the `log_queries` option (logging information about queries) is set to 1. Since the `log_query_threads` option is enabled by default, information about threads was previously logged even if query logging was disabled. [#3241](https://github.com/ClickHouse/ClickHouse/pull/3241) - Fixed an error in the distributed operation of the quantiles aggregate function (the error message `Not found column quantile...`). [292a8855](https://github.com/ClickHouse/ClickHouse/commit/292a885533b8e3b41ce8993867069d14cbd5a664) -- Fixed the compatibility problem when working on a cluster of version 18.12.17 servers and older servers at the same time. For distributed queries with GROUP BY keys of both fixed and non-fixed length, if there was a large amount of data to aggregate, the returned data was not always fully aggregated (two different rows contained the same aggregation keys). [\#3254](https://github.com/ClickHouse/ClickHouse/pull/3254) -- Fixed handling of substitutions in `clickhouse-performance-test`, if the query contains only part of the substitutions declared in the test. [\#3263](https://github.com/ClickHouse/ClickHouse/pull/3263) -- Fixed an error when using `FINAL` with `PREWHERE`. [\#3298](https://github.com/ClickHouse/ClickHouse/pull/3298) -- Fixed an error when using `PREWHERE` over columns that were added during `ALTER`. [\#3298](https://github.com/ClickHouse/ClickHouse/pull/3298) -- Added a check for the absence of `arrayJoin` for `DEFAULT` and `MATERIALIZED` expressions. Previously, `arrayJoin` led to an error when inserting data. [\#3337](https://github.com/ClickHouse/ClickHouse/pull/3337) -- Added a check for the absence of `arrayJoin` in a `PREWHERE` clause. Previously, this led to messages like `Size ... doesn't match` or `Unknown compression method` when executing queries. [\#3357](https://github.com/ClickHouse/ClickHouse/pull/3357) +- Fixed the compatibility problem when working on a cluster of version 18.12.17 servers and older servers at the same time. For distributed queries with GROUP BY keys of both fixed and non-fixed length, if there was a large amount of data to aggregate, the returned data was not always fully aggregated (two different rows contained the same aggregation keys). [#3254](https://github.com/ClickHouse/ClickHouse/pull/3254) +- Fixed handling of substitutions in `clickhouse-performance-test`, if the query contains only part of the substitutions declared in the test. [#3263](https://github.com/ClickHouse/ClickHouse/pull/3263) +- Fixed an error when using `FINAL` with `PREWHERE`. [#3298](https://github.com/ClickHouse/ClickHouse/pull/3298) +- Fixed an error when using `PREWHERE` over columns that were added during `ALTER`. [#3298](https://github.com/ClickHouse/ClickHouse/pull/3298) +- Added a check for the absence of `arrayJoin` for `DEFAULT` and `MATERIALIZED` expressions. Previously, `arrayJoin` led to an error when inserting data. [#3337](https://github.com/ClickHouse/ClickHouse/pull/3337) +- Added a check for the absence of `arrayJoin` in a `PREWHERE` clause. Previously, this led to messages like `Size ... doesn't match` or `Unknown compression method` when executing queries. [#3357](https://github.com/ClickHouse/ClickHouse/pull/3357) - Fixed segfault that could occur in rare cases after optimization that replaced AND chains from equality evaluations with the corresponding IN expression. [liuyimin-bytedance](https://github.com/ClickHouse/ClickHouse/pull/3339) -- Minor corrections to `clickhouse-benchmark`: previously, client information was not sent to the server; now the number of queries executed is calculated more accurately when shutting down and for limiting the number of iterations. [\#3351](https://github.com/ClickHouse/ClickHouse/pull/3351) [\#3352](https://github.com/ClickHouse/ClickHouse/pull/3352) +- Minor corrections to `clickhouse-benchmark`: previously, client information was not sent to the server; now the number of queries executed is calculated more accurately when shutting down and for limiting the number of iterations. [#3351](https://github.com/ClickHouse/ClickHouse/pull/3351) [#3352](https://github.com/ClickHouse/ClickHouse/pull/3352) #### Backward Incompatible Changes: {#backward-incompatible-changes-1} -- Removed the `allow_experimental_decimal_type` option. The `Decimal` data type is available for default use. [\#3329](https://github.com/ClickHouse/ClickHouse/pull/3329) +- Removed the `allow_experimental_decimal_type` option. The `Decimal` data type is available for default use. [#3329](https://github.com/ClickHouse/ClickHouse/pull/3329) ## ClickHouse Release 18.12 {#clickhouse-release-18-12} @@ -305,18 +305,18 @@ toc_title: '2018' #### New Features: {#new-features-2} -- `invalidate_query` (the ability to specify a query to check whether an external dictionary needs to be updated) is implemented for the `clickhouse` source. [\#3126](https://github.com/ClickHouse/ClickHouse/pull/3126) +- `invalidate_query` (the ability to specify a query to check whether an external dictionary needs to be updated) is implemented for the `clickhouse` source. [#3126](https://github.com/ClickHouse/ClickHouse/pull/3126) - Added the ability to use `UInt*`, `Int*`, and `DateTime` data types (along with the `Date` type) as a `range_hashed` external dictionary key that defines the boundaries of ranges. Now `NULL` can be used to designate an open range. [Vasily Nemkov](https://github.com/ClickHouse/ClickHouse/pull/3123) -- The `Decimal` type now supports `var*` and `stddev*` aggregate functions. [\#3129](https://github.com/ClickHouse/ClickHouse/pull/3129) -- The `Decimal` type now supports mathematical functions (`exp`, `sin` and so on.) [\#3129](https://github.com/ClickHouse/ClickHouse/pull/3129) -- The `system.part_log` table now has the `partition_id` column. [\#3089](https://github.com/ClickHouse/ClickHouse/pull/3089) +- The `Decimal` type now supports `var*` and `stddev*` aggregate functions. [#3129](https://github.com/ClickHouse/ClickHouse/pull/3129) +- The `Decimal` type now supports mathematical functions (`exp`, `sin` and so on.) [#3129](https://github.com/ClickHouse/ClickHouse/pull/3129) +- The `system.part_log` table now has the `partition_id` column. [#3089](https://github.com/ClickHouse/ClickHouse/pull/3089) #### Bug Fixes: {#bug-fixes-11} - `Merge` now works correctly on `Distributed` tables. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/3159) -- Fixed incompatibility (unnecessary dependency on the `glibc` version) that made it impossible to run ClickHouse on `Ubuntu Precise` and older versions. The incompatibility arose in version 18.12.13. [\#3130](https://github.com/ClickHouse/ClickHouse/pull/3130) +- Fixed incompatibility (unnecessary dependency on the `glibc` version) that made it impossible to run ClickHouse on `Ubuntu Precise` and older versions. The incompatibility arose in version 18.12.13. [#3130](https://github.com/ClickHouse/ClickHouse/pull/3130) - Fixed errors in the `enable_optimize_predicate_expression` setting. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/3107) -- Fixed a minor issue with backwards compatibility that appeared when working with a cluster of replicas on versions earlier than 18.12.13 and simultaneously creating a new replica of a table on a server with a newer version (shown in the message `Can not clone replica, because the ... updated to new ClickHouse version`, which is logical, but shouldn’t happen). [\#3122](https://github.com/ClickHouse/ClickHouse/pull/3122) +- Fixed a minor issue with backwards compatibility that appeared when working with a cluster of replicas on versions earlier than 18.12.13 and simultaneously creating a new replica of a table on a server with a newer version (shown in the message `Can not clone replica, because the ... updated to new ClickHouse version`, which is logical, but shouldn’t happen). [#3122](https://github.com/ClickHouse/ClickHouse/pull/3122) #### Backward Incompatible Changes: {#backward-incompatible-changes-2} @@ -326,19 +326,19 @@ toc_title: '2018' #### New Features: {#new-features-3} -- Added support for `ALTER UPDATE` queries. [\#3035](https://github.com/ClickHouse/ClickHouse/pull/3035) -- Added the `allow_ddl` option, which restricts the user’s access to DDL queries. [\#3104](https://github.com/ClickHouse/ClickHouse/pull/3104) -- Added the `min_merge_bytes_to_use_direct_io` option for `MergeTree` engines, which allows you to set a threshold for the total size of the merge (when above the threshold, data part files will be handled using O\_DIRECT). [\#3117](https://github.com/ClickHouse/ClickHouse/pull/3117) -- The `system.merges` system table now contains the `partition_id` column. [\#3099](https://github.com/ClickHouse/ClickHouse/pull/3099) +- Added support for `ALTER UPDATE` queries. [#3035](https://github.com/ClickHouse/ClickHouse/pull/3035) +- Added the `allow_ddl` option, which restricts the user’s access to DDL queries. [#3104](https://github.com/ClickHouse/ClickHouse/pull/3104) +- Added the `min_merge_bytes_to_use_direct_io` option for `MergeTree` engines, which allows you to set a threshold for the total size of the merge (when above the threshold, data part files will be handled using O_DIRECT). [#3117](https://github.com/ClickHouse/ClickHouse/pull/3117) +- The `system.merges` system table now contains the `partition_id` column. [#3099](https://github.com/ClickHouse/ClickHouse/pull/3099) #### Improvements {#improvements-3} -- If a data part remains unchanged during mutation, it isn’t downloaded by replicas. [\#3103](https://github.com/ClickHouse/ClickHouse/pull/3103) -- Autocomplete is available for names of settings when working with `clickhouse-client`. [\#3106](https://github.com/ClickHouse/ClickHouse/pull/3106) +- If a data part remains unchanged during mutation, it isn’t downloaded by replicas. [#3103](https://github.com/ClickHouse/ClickHouse/pull/3103) +- Autocomplete is available for names of settings when working with `clickhouse-client`. [#3106](https://github.com/ClickHouse/ClickHouse/pull/3106) #### Bug Fixes: {#bug-fixes-12} -- Added a check for the sizes of arrays that are elements of `Nested` type fields when inserting. [\#3118](https://github.com/ClickHouse/ClickHouse/pull/3118) +- Added a check for the sizes of arrays that are elements of `Nested` type fields when inserting. [#3118](https://github.com/ClickHouse/ClickHouse/pull/3118) - Fixed an error updating external dictionaries with the `ODBC` source and `hashed` storage. This error occurred in version 18.12.13. - Fixed a crash when creating a temporary table from a query with an `IN` condition. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/3098) - Fixed an error in aggregate functions for arrays that can have `NULL` elements. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/3097) @@ -347,90 +347,90 @@ toc_title: '2018' #### New Features: {#new-features-4} -- Added the `DECIMAL(digits, scale)` data type (`Decimal32(scale)`, `Decimal64(scale)`, `Decimal128(scale)`). To enable it, use the setting `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) -- New `WITH ROLLUP` modifier for `GROUP BY` (alternative syntax: `GROUP BY ROLLUP(...)`). [\#2948](https://github.com/ClickHouse/ClickHouse/pull/2948) +- Added the `DECIMAL(digits, scale)` data type (`Decimal32(scale)`, `Decimal64(scale)`, `Decimal128(scale)`). To enable it, use the setting `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) +- New `WITH ROLLUP` modifier for `GROUP BY` (alternative syntax: `GROUP BY ROLLUP(...)`). [#2948](https://github.com/ClickHouse/ClickHouse/pull/2948) - In queries with JOIN, the star character expands to a list of columns in all tables, in compliance with the SQL standard. You can restore the old behavior by setting `asterisk_left_columns_only` to 1 on the user configuration level. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2787) - Added support for JOIN with table functions. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2907) - Autocomplete by pressing Tab in clickhouse-client. [Sergey Shcherbin](https://github.com/ClickHouse/ClickHouse/pull/2447) -- Ctrl+C in clickhouse-client clears a query that was entered. [\#2877](https://github.com/ClickHouse/ClickHouse/pull/2877) -- Added the `join_default_strictness` setting (values: `"`, `'any'`, `'all'`). This allows you to not specify `ANY` or `ALL` for `JOIN`. [\#2982](https://github.com/ClickHouse/ClickHouse/pull/2982) -- Each line of the server log related to query processing shows the query ID. [\#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) -- Now you can get query execution logs in clickhouse-client (use the `send_logs_level` setting). With distributed query processing, logs are cascaded from all the servers. [\#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) -- The `system.query_log` and `system.processes` (`SHOW PROCESSLIST`) tables now have information about all changed settings when you run a query (the nested structure of the `Settings` data). Added the `log_query_settings` setting. [\#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) -- The `system.query_log` and `system.processes` tables now show information about the number of threads that are participating in query execution (see the `thread_numbers` column). [\#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) -- Added `ProfileEvents` counters that measure the time spent on reading and writing over the network and reading and writing to disk, the number of network errors, and the time spent waiting when network bandwidth is limited. [\#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) -- Added `ProfileEvents`counters that contain the system metrics from rusage (you can use them to get information about CPU usage in userspace and the kernel, page faults, and context switches), as well as taskstats metrics (use these to obtain information about I/O wait time, CPU wait time, and the amount of data read and recorded, both with and without page cache). [\#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) -- The `ProfileEvents` counters are applied globally and for each query, as well as for each query execution thread, which allows you to profile resource consumption by query in detail. [\#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) -- Added the `system.query_thread_log` table, which contains information about each query execution thread. Added the `log_query_threads` setting. [\#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) -- The `system.metrics` and `system.events` tables now have built-in documentation. [\#3016](https://github.com/ClickHouse/ClickHouse/pull/3016) +- Ctrl+C in clickhouse-client clears a query that was entered. [#2877](https://github.com/ClickHouse/ClickHouse/pull/2877) +- Added the `join_default_strictness` setting (values: `"`, `'any'`, `'all'`). This allows you to not specify `ANY` or `ALL` for `JOIN`. [#2982](https://github.com/ClickHouse/ClickHouse/pull/2982) +- Each line of the server log related to query processing shows the query ID. [#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) +- Now you can get query execution logs in clickhouse-client (use the `send_logs_level` setting). With distributed query processing, logs are cascaded from all the servers. [#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) +- The `system.query_log` and `system.processes` (`SHOW PROCESSLIST`) tables now have information about all changed settings when you run a query (the nested structure of the `Settings` data). Added the `log_query_settings` setting. [#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) +- The `system.query_log` and `system.processes` tables now show information about the number of threads that are participating in query execution (see the `thread_numbers` column). [#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) +- Added `ProfileEvents` counters that measure the time spent on reading and writing over the network and reading and writing to disk, the number of network errors, and the time spent waiting when network bandwidth is limited. [#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) +- Added `ProfileEvents`counters that contain the system metrics from rusage (you can use them to get information about CPU usage in userspace and the kernel, page faults, and context switches), as well as taskstats metrics (use these to obtain information about I/O wait time, CPU wait time, and the amount of data read and recorded, both with and without page cache). [#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) +- The `ProfileEvents` counters are applied globally and for each query, as well as for each query execution thread, which allows you to profile resource consumption by query in detail. [#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) +- Added the `system.query_thread_log` table, which contains information about each query execution thread. Added the `log_query_threads` setting. [#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) +- The `system.metrics` and `system.events` tables now have built-in documentation. [#3016](https://github.com/ClickHouse/ClickHouse/pull/3016) - Added the `arrayEnumerateDense` function. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2975) - Added the `arrayCumSumNonNegative` and `arrayDifference` functions. [Aleksey Studnev](https://github.com/ClickHouse/ClickHouse/pull/2942) - Added the `retention` aggregate function. [Sundy Li](https://github.com/ClickHouse/ClickHouse/pull/2887) -- Now you can add (merge) states of aggregate functions by using the plus operator, and multiply the states of aggregate functions by a nonnegative constant. [\#3062](https://github.com/ClickHouse/ClickHouse/pull/3062) [\#3034](https://github.com/ClickHouse/ClickHouse/pull/3034) -- Tables in the MergeTree family now have the virtual column `_partition_id`. [\#3089](https://github.com/ClickHouse/ClickHouse/pull/3089) +- Now you can add (merge) states of aggregate functions by using the plus operator, and multiply the states of aggregate functions by a nonnegative constant. [#3062](https://github.com/ClickHouse/ClickHouse/pull/3062) [#3034](https://github.com/ClickHouse/ClickHouse/pull/3034) +- Tables in the MergeTree family now have the virtual column `_partition_id`. [#3089](https://github.com/ClickHouse/ClickHouse/pull/3089) #### Experimental Features: {#experimental-features-1} -- Added the `LowCardinality(T)` data type. This data type automatically creates a local dictionary of values and allows data processing without unpacking the dictionary. [\#2830](https://github.com/ClickHouse/ClickHouse/pull/2830) -- Added a cache of JIT-compiled functions and a counter for the number of uses before compiling. To JIT compile expressions, enable the `compile_expressions` setting. [\#2990](https://github.com/ClickHouse/ClickHouse/pull/2990) [\#3077](https://github.com/ClickHouse/ClickHouse/pull/3077) +- Added the `LowCardinality(T)` data type. This data type automatically creates a local dictionary of values and allows data processing without unpacking the dictionary. [#2830](https://github.com/ClickHouse/ClickHouse/pull/2830) +- Added a cache of JIT-compiled functions and a counter for the number of uses before compiling. To JIT compile expressions, enable the `compile_expressions` setting. [#2990](https://github.com/ClickHouse/ClickHouse/pull/2990) [#3077](https://github.com/ClickHouse/ClickHouse/pull/3077) #### Improvements: {#improvements-4} - Fixed the problem with unlimited accumulation of the replication log when there are abandoned replicas. Added an effective recovery mode for replicas with a long lag. - Improved performance of `GROUP BY` with multiple aggregation fields when one of them is string and the others are fixed length. - Improved performance when using `PREWHERE` and with implicit transfer of expressions in `PREWHERE`. -- Improved parsing performance for text formats (`CSV`, `TSV`). [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2977) [\#2980](https://github.com/ClickHouse/ClickHouse/pull/2980) +- Improved parsing performance for text formats (`CSV`, `TSV`). [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2977) [#2980](https://github.com/ClickHouse/ClickHouse/pull/2980) - Improved performance of reading strings and arrays in binary formats. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2955) -- Increased performance and reduced memory consumption for queries to `system.tables` and `system.columns` when there is a very large number of tables on a single server. [\#2953](https://github.com/ClickHouse/ClickHouse/pull/2953) -- Fixed a performance problem in the case of a large stream of queries that result in an error (the `_dl_addr` function is visible in `perf top`, but the server isn’t using much CPU). [\#2938](https://github.com/ClickHouse/ClickHouse/pull/2938) +- Increased performance and reduced memory consumption for queries to `system.tables` and `system.columns` when there is a very large number of tables on a single server. [#2953](https://github.com/ClickHouse/ClickHouse/pull/2953) +- Fixed a performance problem in the case of a large stream of queries that result in an error (the `_dl_addr` function is visible in `perf top`, but the server isn’t using much CPU). [#2938](https://github.com/ClickHouse/ClickHouse/pull/2938) - Conditions are cast into the View (when `enable_optimize_predicate_expression` is enabled). [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2907) -- Improvements to the functionality for the `UUID` data type. [\#3074](https://github.com/ClickHouse/ClickHouse/pull/3074) [\#2985](https://github.com/ClickHouse/ClickHouse/pull/2985) -- The `UUID` data type is supported in The-Alchemist dictionaries. [\#2822](https://github.com/ClickHouse/ClickHouse/pull/2822) +- Improvements to the functionality for the `UUID` data type. [#3074](https://github.com/ClickHouse/ClickHouse/pull/3074) [#2985](https://github.com/ClickHouse/ClickHouse/pull/2985) +- The `UUID` data type is supported in The-Alchemist dictionaries. [#2822](https://github.com/ClickHouse/ClickHouse/pull/2822) - The `visitParamExtractRaw` function works correctly with nested structures. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2974) - When the `input_format_skip_unknown_fields` setting is enabled, object fields in `JSONEachRow` format are skipped correctly. [BlahGeek](https://github.com/ClickHouse/ClickHouse/pull/2958) -- For a `CASE` expression with conditions, you can now omit `ELSE`, which is equivalent to `ELSE NULL`. [\#2920](https://github.com/ClickHouse/ClickHouse/pull/2920) +- For a `CASE` expression with conditions, you can now omit `ELSE`, which is equivalent to `ELSE NULL`. [#2920](https://github.com/ClickHouse/ClickHouse/pull/2920) - The operation timeout can now be configured when working with ZooKeeper. [urykhy](https://github.com/ClickHouse/ClickHouse/pull/2971) -- You can specify an offset for `LIMIT n, m` as `LIMIT n OFFSET m`. [\#2840](https://github.com/ClickHouse/ClickHouse/pull/2840) -- You can use the `SELECT TOP n` syntax as an alternative for `LIMIT`. [\#2840](https://github.com/ClickHouse/ClickHouse/pull/2840) +- You can specify an offset for `LIMIT n, m` as `LIMIT n OFFSET m`. [#2840](https://github.com/ClickHouse/ClickHouse/pull/2840) +- You can use the `SELECT TOP n` syntax as an alternative for `LIMIT`. [#2840](https://github.com/ClickHouse/ClickHouse/pull/2840) - Increased the size of the queue to write to system tables, so the `SystemLog parameter queue is full` error doesn’t happen as often. - The `windowFunnel` aggregate function now supports events that meet multiple conditions. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2801) -- Duplicate columns can be used in a `USING` clause for `JOIN`. [\#3006](https://github.com/ClickHouse/ClickHouse/pull/3006) -- `Pretty` formats now have a limit on column alignment by width. Use the `output_format_pretty_max_column_pad_width` setting. If a value is wider, it will still be displayed in its entirety, but the other cells in the table will not be too wide. [\#3003](https://github.com/ClickHouse/ClickHouse/pull/3003) +- Duplicate columns can be used in a `USING` clause for `JOIN`. [#3006](https://github.com/ClickHouse/ClickHouse/pull/3006) +- `Pretty` formats now have a limit on column alignment by width. Use the `output_format_pretty_max_column_pad_width` setting. If a value is wider, it will still be displayed in its entirety, but the other cells in the table will not be too wide. [#3003](https://github.com/ClickHouse/ClickHouse/pull/3003) - The `odbc` table function now allows you to specify the database/schema name. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2885) - Added the ability to use a username specified in the `clickhouse-client` config file. [Vladimir Kozbin](https://github.com/ClickHouse/ClickHouse/pull/2909) - The `ZooKeeperExceptions` counter has been split into three counters: `ZooKeeperUserExceptions`, `ZooKeeperHardwareExceptions`, and `ZooKeeperOtherExceptions`. - `ALTER DELETE` queries work for materialized views. - Added randomization when running the cleanup thread periodically for `ReplicatedMergeTree` tables in order to avoid periodic load spikes when there are a very large number of `ReplicatedMergeTree` tables. -- Support for `ATTACH TABLE ... ON CLUSTER` queries. [\#3025](https://github.com/ClickHouse/ClickHouse/pull/3025) +- Support for `ATTACH TABLE ... ON CLUSTER` queries. [#3025](https://github.com/ClickHouse/ClickHouse/pull/3025) #### Bug Fixes: {#bug-fixes-13} -- Fixed an issue with `Dictionary` tables (throws the `Size of offsets doesn't match size of column` or `Unknown compression method` exception). This bug appeared in version 18.10.3. [\#2913](https://github.com/ClickHouse/ClickHouse/issues/2913) -- Fixed a bug when merging `CollapsingMergeTree` tables if one of the data parts is empty (these parts are formed during merge or `ALTER DELETE` if all data was deleted), and the `vertical` algorithm was used for the merge. [\#3049](https://github.com/ClickHouse/ClickHouse/pull/3049) -- Fixed a race condition during `DROP` or `TRUNCATE` for `Memory` tables with a simultaneous `SELECT`, which could lead to server crashes. This bug appeared in version 1.1.54388. [\#3038](https://github.com/ClickHouse/ClickHouse/pull/3038) -- Fixed the possibility of data loss when inserting in `Replicated` tables if the `Session is expired` error is returned (data loss can be detected by the `ReplicatedDataLoss` metric). This error occurred in version 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) -- Fixed a segfault during `JOIN ... ON`. [\#3000](https://github.com/ClickHouse/ClickHouse/pull/3000) -- Fixed the error searching column names when the `WHERE` expression consists entirely of a qualified column name, such as `WHERE table.column`. [\#2994](https://github.com/ClickHouse/ClickHouse/pull/2994) -- Fixed the “Not found column” error that occurred when executing distributed queries if a single column consisting of an IN expression with a subquery is requested from a remote server. [\#3087](https://github.com/ClickHouse/ClickHouse/pull/3087) -- Fixed the `Block structure mismatch in UNION stream: different number of columns` error that occurred for distributed queries if one of the shards is local and the other is not, and optimization of the move to `PREWHERE` is triggered. [\#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) -- Fixed the `pointInPolygon` function for certain cases of non-convex polygons. [\#2910](https://github.com/ClickHouse/ClickHouse/pull/2910) -- Fixed the incorrect result when comparing `nan` with integers. [\#3024](https://github.com/ClickHouse/ClickHouse/pull/3024) -- Fixed an error in the `zlib-ng` library that could lead to segfault in rare cases. [\#2854](https://github.com/ClickHouse/ClickHouse/pull/2854) -- Fixed a memory leak when inserting into a table with `AggregateFunction` columns, if the state of the aggregate function is not simple (allocates memory separately), and if a single insertion request results in multiple small blocks. [\#3084](https://github.com/ClickHouse/ClickHouse/pull/3084) +- Fixed an issue with `Dictionary` tables (throws the `Size of offsets doesn't match size of column` or `Unknown compression method` exception). This bug appeared in version 18.10.3. [#2913](https://github.com/ClickHouse/ClickHouse/issues/2913) +- Fixed a bug when merging `CollapsingMergeTree` tables if one of the data parts is empty (these parts are formed during merge or `ALTER DELETE` if all data was deleted), and the `vertical` algorithm was used for the merge. [#3049](https://github.com/ClickHouse/ClickHouse/pull/3049) +- Fixed a race condition during `DROP` or `TRUNCATE` for `Memory` tables with a simultaneous `SELECT`, which could lead to server crashes. This bug appeared in version 1.1.54388. [#3038](https://github.com/ClickHouse/ClickHouse/pull/3038) +- Fixed the possibility of data loss when inserting in `Replicated` tables if the `Session is expired` error is returned (data loss can be detected by the `ReplicatedDataLoss` metric). This error occurred in version 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) +- Fixed a segfault during `JOIN ... ON`. [#3000](https://github.com/ClickHouse/ClickHouse/pull/3000) +- Fixed the error searching column names when the `WHERE` expression consists entirely of a qualified column name, such as `WHERE table.column`. [#2994](https://github.com/ClickHouse/ClickHouse/pull/2994) +- Fixed the “Not found column” error that occurred when executing distributed queries if a single column consisting of an IN expression with a subquery is requested from a remote server. [#3087](https://github.com/ClickHouse/ClickHouse/pull/3087) +- Fixed the `Block structure mismatch in UNION stream: different number of columns` error that occurred for distributed queries if one of the shards is local and the other is not, and optimization of the move to `PREWHERE` is triggered. [#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) +- Fixed the `pointInPolygon` function for certain cases of non-convex polygons. [#2910](https://github.com/ClickHouse/ClickHouse/pull/2910) +- Fixed the incorrect result when comparing `nan` with integers. [#3024](https://github.com/ClickHouse/ClickHouse/pull/3024) +- Fixed an error in the `zlib-ng` library that could lead to segfault in rare cases. [#2854](https://github.com/ClickHouse/ClickHouse/pull/2854) +- Fixed a memory leak when inserting into a table with `AggregateFunction` columns, if the state of the aggregate function is not simple (allocates memory separately), and if a single insertion request results in multiple small blocks. [#3084](https://github.com/ClickHouse/ClickHouse/pull/3084) - Fixed a race condition when creating and deleting the same `Buffer` or `MergeTree` table simultaneously. -- Fixed the possibility of a segfault when comparing tuples made up of certain non-trivial types, such as tuples. [\#2989](https://github.com/ClickHouse/ClickHouse/pull/2989) +- Fixed the possibility of a segfault when comparing tuples made up of certain non-trivial types, such as tuples. [#2989](https://github.com/ClickHouse/ClickHouse/pull/2989) - Fixed the possibility of a segfault when running certain `ON CLUSTER` queries. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2960) -- Fixed an error in the `arrayDistinct` function for `Nullable` array elements. [\#2845](https://github.com/ClickHouse/ClickHouse/pull/2845) [\#2937](https://github.com/ClickHouse/ClickHouse/pull/2937) +- Fixed an error in the `arrayDistinct` function for `Nullable` array elements. [#2845](https://github.com/ClickHouse/ClickHouse/pull/2845) [#2937](https://github.com/ClickHouse/ClickHouse/pull/2937) - The `enable_optimize_predicate_expression` option now correctly supports cases with `SELECT *`. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2929) -- Fixed the segfault when re-initializing the ZooKeeper session. [\#2917](https://github.com/ClickHouse/ClickHouse/pull/2917) +- Fixed the segfault when re-initializing the ZooKeeper session. [#2917](https://github.com/ClickHouse/ClickHouse/pull/2917) - Fixed potential blocking when working with ZooKeeper. - Fixed incorrect code for adding nested data structures in a `SummingMergeTree`. - When allocating memory for states of aggregate functions, alignment is correctly taken into account, which makes it possible to use operations that require alignment when implementing states of aggregate functions. [chenxing-xc](https://github.com/ClickHouse/ClickHouse/pull/2808) #### Security Fix: {#security-fix} -- Safe use of ODBC data sources. Interaction with ODBC drivers uses a separate `clickhouse-odbc-bridge` process. Errors in third-party ODBC drivers no longer cause problems with server stability or vulnerabilities. [\#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) -- Fixed incorrect validation of the file path in the `catBoostPool` table function. [\#2894](https://github.com/ClickHouse/ClickHouse/pull/2894) +- Safe use of ODBC data sources. Interaction with ODBC drivers uses a separate `clickhouse-odbc-bridge` process. Errors in third-party ODBC drivers no longer cause problems with server stability or vulnerabilities. [#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) +- Fixed incorrect validation of the file path in the `catBoostPool` table function. [#2894](https://github.com/ClickHouse/ClickHouse/pull/2894) - The contents of system tables (`tables`, `databases`, `parts`, `columns`, `parts_columns`, `merges`, `mutations`, `replicas`, and `replication_queue`) are filtered according to the user’s configured access to databases (`allow_databases`). [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2856) #### Backward Incompatible Changes: {#backward-incompatible-changes-3} @@ -442,9 +442,9 @@ toc_title: '2018' - Most integration tests can now be run by commit. - Code style checks can also be run by commit. - The `memcpy` implementation is chosen correctly when building on CentOS7/Fedora. [Etienne Champetier](https://github.com/ClickHouse/ClickHouse/pull/2912) -- When using clang to build, some warnings from `-Weverything` have been added, in addition to the regular `-Wall-Wextra -Werror`. [\#2957](https://github.com/ClickHouse/ClickHouse/pull/2957) +- When using clang to build, some warnings from `-Weverything` have been added, in addition to the regular `-Wall-Wextra -Werror`. [#2957](https://github.com/ClickHouse/ClickHouse/pull/2957) - Debugging the build uses the `jemalloc` debug option. -- The interface of the library for interacting with ZooKeeper is declared abstract. [\#2950](https://github.com/ClickHouse/ClickHouse/pull/2950) +- The interface of the library for interacting with ZooKeeper is declared abstract. [#2950](https://github.com/ClickHouse/ClickHouse/pull/2950) ## ClickHouse Release 18.10 {#clickhouse-release-18-10} @@ -452,37 +452,37 @@ toc_title: '2018' #### New Features: {#new-features-5} -- HTTPS can be used for replication. [\#2760](https://github.com/ClickHouse/ClickHouse/pull/2760) -- Added the functions `murmurHash2_64`, `murmurHash3_32`, `murmurHash3_64`, and `murmurHash3_128` in addition to the existing `murmurHash2_32`. [\#2791](https://github.com/ClickHouse/ClickHouse/pull/2791) -- Support for Nullable types in the ClickHouse ODBC driver (`ODBCDriver2` output format). [\#2834](https://github.com/ClickHouse/ClickHouse/pull/2834) +- HTTPS can be used for replication. [#2760](https://github.com/ClickHouse/ClickHouse/pull/2760) +- Added the functions `murmurHash2_64`, `murmurHash3_32`, `murmurHash3_64`, and `murmurHash3_128` in addition to the existing `murmurHash2_32`. [#2791](https://github.com/ClickHouse/ClickHouse/pull/2791) +- Support for Nullable types in the ClickHouse ODBC driver (`ODBCDriver2` output format). [#2834](https://github.com/ClickHouse/ClickHouse/pull/2834) - Support for `UUID` in the key columns. #### Improvements: {#improvements-5} -- Clusters can be removed without restarting the server when they are deleted from the config files. [\#2777](https://github.com/ClickHouse/ClickHouse/pull/2777) -- External dictionaries can be removed without restarting the server when they are removed from config files. [\#2779](https://github.com/ClickHouse/ClickHouse/pull/2779) +- Clusters can be removed without restarting the server when they are deleted from the config files. [#2777](https://github.com/ClickHouse/ClickHouse/pull/2777) +- External dictionaries can be removed without restarting the server when they are removed from config files. [#2779](https://github.com/ClickHouse/ClickHouse/pull/2779) - Added `SETTINGS` support for the `Kafka` table engine. [Alexander Marshalov](https://github.com/ClickHouse/ClickHouse/pull/2781) -- Improvements for the `UUID` data type (not yet complete). [\#2618](https://github.com/ClickHouse/ClickHouse/pull/2618) -- Support for empty parts after merges in the `SummingMergeTree`, `CollapsingMergeTree` and `VersionedCollapsingMergeTree` engines. [\#2815](https://github.com/ClickHouse/ClickHouse/pull/2815) -- Old records of completed mutations are deleted (`ALTER DELETE`). [\#2784](https://github.com/ClickHouse/ClickHouse/pull/2784) +- Improvements for the `UUID` data type (not yet complete). [#2618](https://github.com/ClickHouse/ClickHouse/pull/2618) +- Support for empty parts after merges in the `SummingMergeTree`, `CollapsingMergeTree` and `VersionedCollapsingMergeTree` engines. [#2815](https://github.com/ClickHouse/ClickHouse/pull/2815) +- Old records of completed mutations are deleted (`ALTER DELETE`). [#2784](https://github.com/ClickHouse/ClickHouse/pull/2784) - Added the `system.merge_tree_settings` table. [Kirill Shvakov](https://github.com/ClickHouse/ClickHouse/pull/2841) - The `system.tables` table now has dependency columns: `dependencies_database` and `dependencies_table`. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2851) -- Added the `max_partition_size_to_drop` config option. [\#2782](https://github.com/ClickHouse/ClickHouse/pull/2782) +- Added the `max_partition_size_to_drop` config option. [#2782](https://github.com/ClickHouse/ClickHouse/pull/2782) - Added the `output_format_json_escape_forward_slashes` option. [Alexander Bocharov](https://github.com/ClickHouse/ClickHouse/pull/2812) -- Added the `max_fetch_partition_retries_count` setting. [\#2831](https://github.com/ClickHouse/ClickHouse/pull/2831) -- Added the `prefer_localhost_replica` setting for disabling the preference for a local replica and going to a local replica without inter-process interaction. [\#2832](https://github.com/ClickHouse/ClickHouse/pull/2832) +- Added the `max_fetch_partition_retries_count` setting. [#2831](https://github.com/ClickHouse/ClickHouse/pull/2831) +- Added the `prefer_localhost_replica` setting for disabling the preference for a local replica and going to a local replica without inter-process interaction. [#2832](https://github.com/ClickHouse/ClickHouse/pull/2832) - The `quantileExact` aggregate function returns `nan` in the case of aggregation on an empty `Float32` or `Float64` set. [Sundy Li](https://github.com/ClickHouse/ClickHouse/pull/2855) #### Bug Fixes: {#bug-fixes-14} - Removed unnecessary escaping of the connection string parameters for ODBC, which made it impossible to establish a connection. This error occurred in version 18.6.0. -- Fixed the logic for processing `REPLACE PARTITION` commands in the replication queue. If there are two `REPLACE` commands for the same partition, the incorrect logic could cause one of them to remain in the replication queue and not be executed. [\#2814](https://github.com/ClickHouse/ClickHouse/pull/2814) -- Fixed a merge bug when all data parts were empty (parts that were formed from a merge or from `ALTER DELETE` if all data was deleted). This bug appeared in version 18.1.0. [\#2930](https://github.com/ClickHouse/ClickHouse/pull/2930) +- Fixed the logic for processing `REPLACE PARTITION` commands in the replication queue. If there are two `REPLACE` commands for the same partition, the incorrect logic could cause one of them to remain in the replication queue and not be executed. [#2814](https://github.com/ClickHouse/ClickHouse/pull/2814) +- Fixed a merge bug when all data parts were empty (parts that were formed from a merge or from `ALTER DELETE` if all data was deleted). This bug appeared in version 18.1.0. [#2930](https://github.com/ClickHouse/ClickHouse/pull/2930) - Fixed an error for concurrent `Set` or `Join`. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2823) - Fixed the `Block structure mismatch in UNION stream: different number of columns` error that occurred for `UNION ALL` queries inside a sub-query if one of the `SELECT` queries contains duplicate column names. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2094) - Fixed a memory leak if an exception occurred when connecting to a MySQL server. - Fixed incorrect clickhouse-client response code in case of a query error. -- Fixed incorrect behavior of materialized views containing DISTINCT. [\#2795](https://github.com/ClickHouse/ClickHouse/issues/2795) +- Fixed incorrect behavior of materialized views containing DISTINCT. [#2795](https://github.com/ClickHouse/ClickHouse/issues/2795) #### Backward Incompatible Changes {#backward-incompatible-changes-4} @@ -490,10 +490,10 @@ toc_title: '2018' #### Build Changes: {#build-changes-3} -- The allocator has been replaced: `jemalloc` is now used instead of `tcmalloc`. In some scenarios, this increases speed up to 20%. However, there are queries that have slowed by up to 20%. Memory consumption has been reduced by approximately 10% in some scenarios, with improved stability. With highly competitive loads, CPU usage in userspace and in system shows just a slight increase. [\#2773](https://github.com/ClickHouse/ClickHouse/pull/2773) -- Use of libressl from a submodule. [\#1983](https://github.com/ClickHouse/ClickHouse/pull/1983) [\#2807](https://github.com/ClickHouse/ClickHouse/pull/2807) -- Use of unixodbc from a submodule. [\#2789](https://github.com/ClickHouse/ClickHouse/pull/2789) -- Use of mariadb-connector-c from a submodule. [\#2785](https://github.com/ClickHouse/ClickHouse/pull/2785) +- The allocator has been replaced: `jemalloc` is now used instead of `tcmalloc`. In some scenarios, this increases speed up to 20%. However, there are queries that have slowed by up to 20%. Memory consumption has been reduced by approximately 10% in some scenarios, with improved stability. With highly competitive loads, CPU usage in userspace and in system shows just a slight increase. [#2773](https://github.com/ClickHouse/ClickHouse/pull/2773) +- Use of libressl from a submodule. [#1983](https://github.com/ClickHouse/ClickHouse/pull/1983) [#2807](https://github.com/ClickHouse/ClickHouse/pull/2807) +- Use of unixodbc from a submodule. [#2789](https://github.com/ClickHouse/ClickHouse/pull/2789) +- Use of mariadb-connector-c from a submodule. [#2785](https://github.com/ClickHouse/ClickHouse/pull/2785) - Added functional test files to the repository that depend on the availability of test data (for the time being, without the test data itself). ## ClickHouse Release 18.6 {#clickhouse-release-18-6} @@ -504,12 +504,12 @@ toc_title: '2018' - Added support for ON expressions for the JOIN ON syntax: `JOIN ON Expr([table.]column ...) = Expr([table.]column, ...) [AND Expr([table.]column, ...) = Expr([table.]column, ...) ...]` - The expression must be a chain of equalities joined by the AND operator. Each side of the equality can be an arbitrary expression over the columns of one of the tables. The use of fully qualified column names is supported (`table.name`, `database.table.name`, `table_alias.name`, `subquery_alias.name`) for the right table. [\#2742](https://github.com/ClickHouse/ClickHouse/pull/2742) -- HTTPS can be enabled for replication. [\#2760](https://github.com/ClickHouse/ClickHouse/pull/2760) + The expression must be a chain of equalities joined by the AND operator. Each side of the equality can be an arbitrary expression over the columns of one of the tables. The use of fully qualified column names is supported (`table.name`, `database.table.name`, `table_alias.name`, `subquery_alias.name`) for the right table. [#2742](https://github.com/ClickHouse/ClickHouse/pull/2742) +- HTTPS can be enabled for replication. [#2760](https://github.com/ClickHouse/ClickHouse/pull/2760) #### Improvements: {#improvements-6} -- The server passes the patch component of its version to the client. Data about the patch version component is in `system.processes` and `query_log`. [\#2646](https://github.com/ClickHouse/ClickHouse/pull/2646) +- The server passes the patch component of its version to the client. Data about the patch version component is in `system.processes` and `query_log`. [#2646](https://github.com/ClickHouse/ClickHouse/pull/2646) ## ClickHouse Release 18.5 {#clickhouse-release-18-5} @@ -517,16 +517,16 @@ toc_title: '2018' #### New Features: {#new-features-7} -- Added the hash function `murmurHash2_32` [\#2756](https://github.com/ClickHouse/ClickHouse/pull/2756). +- Added the hash function `murmurHash2_32` [#2756](https://github.com/ClickHouse/ClickHouse/pull/2756). #### Improvements: {#improvements-7} -- Now you can use the `from_env` [\#2741](https://github.com/ClickHouse/ClickHouse/pull/2741) attribute to set values in config files from environment variables. -- Added case-insensitive versions of the `coalesce`, `ifNull`, and `nullIf functions` [\#2752](https://github.com/ClickHouse/ClickHouse/pull/2752). +- Now you can use the `from_env` [#2741](https://github.com/ClickHouse/ClickHouse/pull/2741) attribute to set values in config files from environment variables. +- Added case-insensitive versions of the `coalesce`, `ifNull`, and `nullIf functions` [#2752](https://github.com/ClickHouse/ClickHouse/pull/2752). #### Bug Fixes: {#bug-fixes-15} -- Fixed a possible bug when starting a replica [\#2759](https://github.com/ClickHouse/ClickHouse/pull/2759). +- Fixed a possible bug when starting a replica [#2759](https://github.com/ClickHouse/ClickHouse/pull/2759). ## ClickHouse Release 18.4 {#clickhouse-release-18-4} @@ -534,15 +534,15 @@ toc_title: '2018' #### New Features: {#new-features-8} -- Added system tables: `formats`, `data_type_families`, `aggregate_function_combinators`, `table_functions`, `table_engines`, `collations` [\#2721](https://github.com/ClickHouse/ClickHouse/pull/2721). -- Added the ability to use a table function instead of a table as an argument of a `remote` or `cluster table function` [\#2708](https://github.com/ClickHouse/ClickHouse/pull/2708). -- Support for `HTTP Basic` authentication in the replication protocol [\#2727](https://github.com/ClickHouse/ClickHouse/pull/2727). +- Added system tables: `formats`, `data_type_families`, `aggregate_function_combinators`, `table_functions`, `table_engines`, `collations` [#2721](https://github.com/ClickHouse/ClickHouse/pull/2721). +- Added the ability to use a table function instead of a table as an argument of a `remote` or `cluster table function` [#2708](https://github.com/ClickHouse/ClickHouse/pull/2708). +- Support for `HTTP Basic` authentication in the replication protocol [#2727](https://github.com/ClickHouse/ClickHouse/pull/2727). - The `has` function now allows searching for a numeric value in an array of `Enum` values [Maxim Khrisanfov](https://github.com/ClickHouse/ClickHouse/pull/2699). - Support for adding arbitrary message separators when reading from `Kafka` [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2701). #### Improvements: {#improvements-8} -- The `ALTER TABLE t DELETE WHERE` query does not rewrite data parts that were not affected by the WHERE condition [\#2694](https://github.com/ClickHouse/ClickHouse/pull/2694). +- The `ALTER TABLE t DELETE WHERE` query does not rewrite data parts that were not affected by the WHERE condition [#2694](https://github.com/ClickHouse/ClickHouse/pull/2694). - The `use_minimalistic_checksums_in_zookeeper` option for `ReplicatedMergeTree` tables is enabled by default. This setting was added in version 1.1.54378, 2018-04-16. Versions that are older than 1.1.54378 can no longer be installed. - Support for running `KILL` and `OPTIMIZE` queries that specify `ON CLUSTER` [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2689). @@ -563,11 +563,11 @@ toc_title: '2018' #### New Features: {#new-features-9} -- Support for the `ALTER TABLE t DELETE WHERE` query for non-replicated MergeTree tables ([\#2634](https://github.com/ClickHouse/ClickHouse/pull/2634)). -- Support for arbitrary types for the `uniq*` family of aggregate functions ([\#2010](https://github.com/ClickHouse/ClickHouse/issues/2010)). -- Support for arbitrary types in comparison operators ([\#2026](https://github.com/ClickHouse/ClickHouse/issues/2026)). -- The `users.xml` file allows setting a subnet mask in the format `10.0.0.1/255.255.255.0`. This is necessary for using masks for IPv6 networks with zeros in the middle ([\#2637](https://github.com/ClickHouse/ClickHouse/pull/2637)). -- Added the `arrayDistinct` function ([\#2670](https://github.com/ClickHouse/ClickHouse/pull/2670)). +- Support for the `ALTER TABLE t DELETE WHERE` query for non-replicated MergeTree tables ([#2634](https://github.com/ClickHouse/ClickHouse/pull/2634)). +- Support for arbitrary types for the `uniq*` family of aggregate functions ([#2010](https://github.com/ClickHouse/ClickHouse/issues/2010)). +- Support for arbitrary types in comparison operators ([#2026](https://github.com/ClickHouse/ClickHouse/issues/2026)). +- The `users.xml` file allows setting a subnet mask in the format `10.0.0.1/255.255.255.0`. This is necessary for using masks for IPv6 networks with zeros in the middle ([#2637](https://github.com/ClickHouse/ClickHouse/pull/2637)). +- Added the `arrayDistinct` function ([#2670](https://github.com/ClickHouse/ClickHouse/pull/2670)). - The SummingMergeTree engine can now work with AggregateFunction type columns ([Constantin S. Pan](https://github.com/ClickHouse/ClickHouse/pull/2566)). #### Improvements: {#improvements-9} @@ -580,12 +580,12 @@ toc_title: '2018' - Fixed the TRUNCATE command for temporary tables ([Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2624)). - Fixed a rare deadlock in the ZooKeeper client library that occurred when there was a network error while reading the response ([c315200](https://github.com/ClickHouse/ClickHouse/commit/c315200e64b87e44bdf740707fc857d1fdf7e947)). -- Fixed an error during a CAST to Nullable types ([\#1322](https://github.com/ClickHouse/ClickHouse/issues/1322)). +- Fixed an error during a CAST to Nullable types ([#1322](https://github.com/ClickHouse/ClickHouse/issues/1322)). - Fixed the incorrect result of the `maxIntersection()` function when the boundaries of intervals coincided ([Michael Furmur](https://github.com/ClickHouse/ClickHouse/pull/2657)). - Fixed incorrect transformation of the OR expression chain in a function argument ([chenxing-xc](https://github.com/ClickHouse/ClickHouse/pull/2663)). -- Fixed performance degradation for queries containing `IN (subquery)` expressions inside another subquery ([\#2571](https://github.com/ClickHouse/ClickHouse/issues/2571)). +- Fixed performance degradation for queries containing `IN (subquery)` expressions inside another subquery ([#2571](https://github.com/ClickHouse/ClickHouse/issues/2571)). - Fixed incompatibility between servers with different versions in distributed queries that use a `CAST` function that isn’t in uppercase letters ([fe8c4d6](https://github.com/ClickHouse/ClickHouse/commit/fe8c4d64e434cacd4ceef34faa9005129f2190a5)). -- Added missing quoting of identifiers for queries to an external DBMS ([\#2635](https://github.com/ClickHouse/ClickHouse/issues/2635)). +- Added missing quoting of identifiers for queries to an external DBMS ([#2635](https://github.com/ClickHouse/ClickHouse/issues/2635)). #### Backward Incompatible Changes: {#backward-incompatible-changes-6} @@ -604,9 +604,9 @@ toc_title: '2018' - Fixed a problem with a very small timeout for sockets (one second) for reading and writing when sending and downloading replicated data, which made it impossible to download larger parts if there is a load on the network or disk (it resulted in cyclical attempts to download parts). This error occurred in version 1.1.54388. - Fixed issues when using chroot in ZooKeeper if you inserted duplicate data blocks in the table. -- The `has` function now works correctly for an array with Nullable elements ([\#2115](https://github.com/ClickHouse/ClickHouse/issues/2115)). +- The `has` function now works correctly for an array with Nullable elements ([#2115](https://github.com/ClickHouse/ClickHouse/issues/2115)). - The `system.tables` table now works correctly when used in distributed queries. The `metadata_modification_time` and `engine_full` columns are now non-virtual. Fixed an error that occurred if only these columns were queried from the table. -- Fixed how an empty `TinyLog` table works after inserting an empty data block ([\#2563](https://github.com/ClickHouse/ClickHouse/issues/2563)). +- Fixed how an empty `TinyLog` table works after inserting an empty data block ([#2563](https://github.com/ClickHouse/ClickHouse/issues/2563)). - The `system.zookeeper` table works if the value of the node in ZooKeeper is NULL. ### ClickHouse Release 1.1.54390, 2018-07-06 {#clickhouse-release-1-1-54390-2018-07-06} @@ -619,12 +619,12 @@ toc_title: '2018' #### Improvements: {#improvements-10} -- Improved performance, reduced memory consumption, and correct memory consumption tracking with use of the IN operator when a table index could be used ([\#2584](https://github.com/ClickHouse/ClickHouse/pull/2584)). +- Improved performance, reduced memory consumption, and correct memory consumption tracking with use of the IN operator when a table index could be used ([#2584](https://github.com/ClickHouse/ClickHouse/pull/2584)). - Removed redundant checking of checksums when adding a data part. This is important when there are a large number of replicas, because in these cases the total number of checks was equal to N^2. -- Added support for `Array(Tuple(...))` arguments for the `arrayEnumerateUniq` function ([\#2573](https://github.com/ClickHouse/ClickHouse/pull/2573)). -- Added `Nullable` support for the `runningDifference` function ([\#2594](https://github.com/ClickHouse/ClickHouse/pull/2594)). -- Improved query analysis performance when there is a very large number of expressions ([\#2572](https://github.com/ClickHouse/ClickHouse/pull/2572)). -- Faster selection of data parts for merging in `ReplicatedMergeTree` tables. Faster recovery of the ZooKeeper session ([\#2597](https://github.com/ClickHouse/ClickHouse/pull/2597)). +- Added support for `Array(Tuple(...))` arguments for the `arrayEnumerateUniq` function ([#2573](https://github.com/ClickHouse/ClickHouse/pull/2573)). +- Added `Nullable` support for the `runningDifference` function ([#2594](https://github.com/ClickHouse/ClickHouse/pull/2594)). +- Improved query analysis performance when there is a very large number of expressions ([#2572](https://github.com/ClickHouse/ClickHouse/pull/2572)). +- Faster selection of data parts for merging in `ReplicatedMergeTree` tables. Faster recovery of the ZooKeeper session ([#2597](https://github.com/ClickHouse/ClickHouse/pull/2597)). - The `format_version.txt` file for `MergeTree` tables is re-created if it is missing, which makes sense if ClickHouse is launched after copying the directory structure without files ([Ciprian Hacman](https://github.com/ClickHouse/ClickHouse/pull/2593)). #### Bug Fixes: {#bug-fixes-19} @@ -632,9 +632,9 @@ toc_title: '2018' - Fixed a bug when working with ZooKeeper that could make it impossible to recover the session and readonly states of tables before restarting the server. - Fixed a bug when working with ZooKeeper that could result in old nodes not being deleted if the session is interrupted. - Fixed an error in the `quantileTDigest` function for Float arguments (this bug was introduced in version 1.1.54388) ([Mikhail Surin](https://github.com/ClickHouse/ClickHouse/pull/2553)). -- Fixed a bug in the index for MergeTree tables if the primary key column is located inside the function for converting types between signed and unsigned integers of the same size ([\#2603](https://github.com/ClickHouse/ClickHouse/pull/2603)). -- Fixed segfault if `macros` are used but they aren’t in the config file ([\#2570](https://github.com/ClickHouse/ClickHouse/pull/2570)). -- Fixed switching to the default database when reconnecting the client ([\#2583](https://github.com/ClickHouse/ClickHouse/pull/2583)). +- Fixed a bug in the index for MergeTree tables if the primary key column is located inside the function for converting types between signed and unsigned integers of the same size ([#2603](https://github.com/ClickHouse/ClickHouse/pull/2603)). +- Fixed segfault if `macros` are used but they aren’t in the config file ([#2570](https://github.com/ClickHouse/ClickHouse/pull/2570)). +- Fixed switching to the default database when reconnecting the client ([#2583](https://github.com/ClickHouse/ClickHouse/pull/2583)). - Fixed a bug that occurred when the `use_index_for_in_with_subqueries` setting was disabled. #### Security Fix: {#security-fix-1} @@ -670,23 +670,23 @@ toc_title: '2018' - Duplicates no longer appear for a query with `DISTINCT` and `ORDER BY`. - Queries with `ARRAY JOIN` and `arrayFilter` no longer return an incorrect result. -- Fixed an error when reading an array column from a Nested structure ([\#2066](https://github.com/ClickHouse/ClickHouse/issues/2066)). +- Fixed an error when reading an array column from a Nested structure ([#2066](https://github.com/ClickHouse/ClickHouse/issues/2066)). - Fixed an error when analyzing queries with a HAVING clause like `HAVING tuple IN (...)`. - Fixed an error when analyzing queries with recursive aliases. -- Fixed an error when reading from ReplacingMergeTree with a condition in PREWHERE that filters all rows ([\#2525](https://github.com/ClickHouse/ClickHouse/issues/2525)). +- Fixed an error when reading from ReplacingMergeTree with a condition in PREWHERE that filters all rows ([#2525](https://github.com/ClickHouse/ClickHouse/issues/2525)). - User profile settings were not applied when using sessions in the HTTP interface. - Fixed how settings are applied from the command line parameters in clickhouse-local. - The ZooKeeper client library now uses the session timeout received from the server. - Fixed a bug in the ZooKeeper client library when the client waited for the server response longer than the timeout. -- Fixed pruning of parts for queries with conditions on partition key columns ([\#2342](https://github.com/ClickHouse/ClickHouse/issues/2342)). -- Merges are now possible after `CLEAR COLUMN IN PARTITION` ([\#2315](https://github.com/ClickHouse/ClickHouse/issues/2315)). +- Fixed pruning of parts for queries with conditions on partition key columns ([#2342](https://github.com/ClickHouse/ClickHouse/issues/2342)). +- Merges are now possible after `CLEAR COLUMN IN PARTITION` ([#2315](https://github.com/ClickHouse/ClickHouse/issues/2315)). - Type mapping in the ODBC table function has been fixed ([sundy-li](https://github.com/ClickHouse/ClickHouse/pull/2268)). - Type comparisons have been fixed for `DateTime` with and without the time zone ([Alexander Bocharov](https://github.com/ClickHouse/ClickHouse/pull/2400)). - Fixed syntactic parsing and formatting of the `CAST` operator. - Fixed insertion into a materialized view for the Distributed table engine ([Babacar Diassé](https://github.com/ClickHouse/ClickHouse/pull/2411)). - Fixed a race condition when writing data from the `Kafka` engine to materialized views ([Yangkuan Liu](https://github.com/ClickHouse/ClickHouse/pull/2448)). - Fixed SSRF in the remote() table function. -- Fixed exit behavior of `clickhouse-client` in multiline mode ([\#2510](https://github.com/ClickHouse/ClickHouse/issues/2510)). +- Fixed exit behavior of `clickhouse-client` in multiline mode ([#2510](https://github.com/ClickHouse/ClickHouse/issues/2510)). #### Improvements: {#improvements-11} @@ -710,7 +710,7 @@ toc_title: '2018' - Fixed the build using the vectorclass library ([Babacar Diassé](https://github.com/ClickHouse/ClickHouse/pull/2274)). - Cmake now generates files for ninja by default (like when using `-G Ninja`). - Added the ability to use the libtinfo library instead of libtermcap ([Georgy Kondratiev](https://github.com/ClickHouse/ClickHouse/pull/2519)). -- Fixed a header file conflict in Fedora Rawhide ([\#2520](https://github.com/ClickHouse/ClickHouse/issues/2520)). +- Fixed a header file conflict in Fedora Rawhide ([#2520](https://github.com/ClickHouse/ClickHouse/issues/2520)). #### Backward Incompatible Changes: {#backward-incompatible-changes-7} @@ -793,10 +793,10 @@ toc_title: '2018' - The `max_execution_time` limit now works correctly with distributed queries. - Fixed errors when calculating the size of composite columns in the `system.columns` table. - Fixed an error when creating a temporary table `CREATE TEMPORARY TABLE IF NOT EXISTS.` -- Fixed errors in `StorageKafka` (\#\#2075) +- Fixed errors in `StorageKafka` (##2075) - Fixed server crashes from invalid arguments of certain aggregate functions. - Fixed the error that prevented the `DETACH DATABASE` query from stopping background tasks for `ReplicatedMergeTree` tables. -- `Too many parts` state is less likely to happen when inserting into aggregated materialized views (\#\#2084). +- `Too many parts` state is less likely to happen when inserting into aggregated materialized views (##2084). - Corrected recursive handling of substitutions in the config if a substitution must be followed by another substitution on the same level. - Corrected the syntax in the metadata file when creating a `VIEW` that uses a query with `UNION ALL`. - `SummingMergeTree` now works correctly for summation of nested data structures with a composite key. diff --git a/docs/en/whats-new/changelog/2019.md b/docs/en/whats-new/changelog/2019.md index ee4e66bc2f3..eacd522390f 100644 --- a/docs/en/whats-new/changelog/2019.md +++ b/docs/en/whats-new/changelog/2019.md @@ -9,404 +9,404 @@ toc_title: '2019' #### Bug Fix {#bug-fix} -- Fixed potential buffer overflow in decompress. Malicious user can pass fabricated compressed data that could cause read after buffer. This issue was found by Eldar Zaitov from Yandex information security team. [\#8404](https://github.com/ClickHouse/ClickHouse/pull/8404) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed possible server crash (`std::terminate`) when the server cannot send or write data in JSON or XML format with values of String data type (that require UTF-8 validation) or when compressing result data with Brotli algorithm or in some other rare cases. [\#8384](https://github.com/ClickHouse/ClickHouse/pull/8384) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed dictionaries with source from a clickhouse `VIEW`, now reading such dictionaries doesn’t cause the error `There is no query`. [\#8351](https://github.com/ClickHouse/ClickHouse/pull/8351) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Fixed checking if a client host is allowed by host\_regexp specified in users.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` for a distributed table now renames the folder containing inserted data before sending to shards. This fixes an issue with successive renames `tableA->tableB`, `tableC->tableA`. [\#8306](https://github.com/ClickHouse/ClickHouse/pull/8306) ([tavplubix](https://github.com/tavplubix)) -- `range_hashed` external dictionaries created by DDL queries now allow ranges of arbitrary numeric types. [\#8275](https://github.com/ClickHouse/ClickHouse/pull/8275) ([alesapin](https://github.com/alesapin)) -- Fixed `INSERT INTO table SELECT ... FROM mysql(...)` table function. [\#8234](https://github.com/ClickHouse/ClickHouse/pull/8234) ([tavplubix](https://github.com/tavplubix)) -- Fixed segfault in `INSERT INTO TABLE FUNCTION file()` while inserting into a file which doesn’t exist. Now in this case file would be created and then insert would be processed. [\#8177](https://github.com/ClickHouse/ClickHouse/pull/8177) ([Olga Khvostikova](https://github.com/stavrolia)) -- Fixed bitmapAnd error when intersecting an aggregated bitmap and a scalar bitmap. [\#8082](https://github.com/ClickHouse/ClickHouse/pull/8082) ([Yue Huang](https://github.com/moon03432)) -- Fixed segfault when `EXISTS` query was used without `TABLE` or `DICTIONARY` qualifier, just like `EXISTS t`. [\#8213](https://github.com/ClickHouse/ClickHouse/pull/8213) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed return type for functions `rand` and `randConstant` in case of nullable argument. Now functions always return `UInt32` and never `Nullable(UInt32)`. [\#8204](https://github.com/ClickHouse/ClickHouse/pull/8204) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Fixed `DROP DICTIONARY IF EXISTS db.dict`, now it doesn’t throw exception if `db` doesn’t exist. [\#8185](https://github.com/ClickHouse/ClickHouse/pull/8185) ([Vitaly Baranov](https://github.com/vitlibar)) -- If a table wasn’t completely dropped because of server crash, the server will try to restore and load it [\#8176](https://github.com/ClickHouse/ClickHouse/pull/8176) ([tavplubix](https://github.com/tavplubix)) -- Fixed a trivial count query for a distributed table if there are more than two shard local table. [\#8164](https://github.com/ClickHouse/ClickHouse/pull/8164) ([小路](https://github.com/nicelulu)) -- Fixed bug that lead to a data race in DB::BlockStreamProfileInfo::calculateRowsBeforeLimit() [\#8143](https://github.com/ClickHouse/ClickHouse/pull/8143) ([Alexander Kazakov](https://github.com/Akazz)) -- Fixed `ALTER table MOVE part` executed immediately after merging the specified part, which could cause moving a part which the specified part merged into. Now it correctly moves the specified part. [\#8104](https://github.com/ClickHouse/ClickHouse/pull/8104) ([Vladimir Chebotarev](https://github.com/excitoon)) -- Expressions for dictionaries can be specified as strings now. This is useful for calculation of attributes while extracting data from non-ClickHouse sources because it allows to use non-ClickHouse syntax for those expressions. [\#8098](https://github.com/ClickHouse/ClickHouse/pull/8098) ([alesapin](https://github.com/alesapin)) -- Fixed a very rare race in `clickhouse-copier` because of an overflow in ZXid. [\#8088](https://github.com/ClickHouse/ClickHouse/pull/8088) ([Ding Xiang Fei](https://github.com/dingxiangfei2009)) +- Fixed potential buffer overflow in decompress. Malicious user can pass fabricated compressed data that could cause read after buffer. This issue was found by Eldar Zaitov from Yandex information security team. [#8404](https://github.com/ClickHouse/ClickHouse/pull/8404) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed possible server crash (`std::terminate`) when the server cannot send or write data in JSON or XML format with values of String data type (that require UTF-8 validation) or when compressing result data with Brotli algorithm or in some other rare cases. [#8384](https://github.com/ClickHouse/ClickHouse/pull/8384) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed dictionaries with source from a clickhouse `VIEW`, now reading such dictionaries doesn’t cause the error `There is no query`. [#8351](https://github.com/ClickHouse/ClickHouse/pull/8351) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fixed checking if a client host is allowed by host_regexp specified in users.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` for a distributed table now renames the folder containing inserted data before sending to shards. This fixes an issue with successive renames `tableA->tableB`, `tableC->tableA`. [#8306](https://github.com/ClickHouse/ClickHouse/pull/8306) ([tavplubix](https://github.com/tavplubix)) +- `range_hashed` external dictionaries created by DDL queries now allow ranges of arbitrary numeric types. [#8275](https://github.com/ClickHouse/ClickHouse/pull/8275) ([alesapin](https://github.com/alesapin)) +- Fixed `INSERT INTO table SELECT ... FROM mysql(...)` table function. [#8234](https://github.com/ClickHouse/ClickHouse/pull/8234) ([tavplubix](https://github.com/tavplubix)) +- Fixed segfault in `INSERT INTO TABLE FUNCTION file()` while inserting into a file which doesn’t exist. Now in this case file would be created and then insert would be processed. [#8177](https://github.com/ClickHouse/ClickHouse/pull/8177) ([Olga Khvostikova](https://github.com/stavrolia)) +- Fixed bitmapAnd error when intersecting an aggregated bitmap and a scalar bitmap. [#8082](https://github.com/ClickHouse/ClickHouse/pull/8082) ([Yue Huang](https://github.com/moon03432)) +- Fixed segfault when `EXISTS` query was used without `TABLE` or `DICTIONARY` qualifier, just like `EXISTS t`. [#8213](https://github.com/ClickHouse/ClickHouse/pull/8213) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed return type for functions `rand` and `randConstant` in case of nullable argument. Now functions always return `UInt32` and never `Nullable(UInt32)`. [#8204](https://github.com/ClickHouse/ClickHouse/pull/8204) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fixed `DROP DICTIONARY IF EXISTS db.dict`, now it doesn’t throw exception if `db` doesn’t exist. [#8185](https://github.com/ClickHouse/ClickHouse/pull/8185) ([Vitaly Baranov](https://github.com/vitlibar)) +- If a table wasn’t completely dropped because of server crash, the server will try to restore and load it [#8176](https://github.com/ClickHouse/ClickHouse/pull/8176) ([tavplubix](https://github.com/tavplubix)) +- Fixed a trivial count query for a distributed table if there are more than two shard local table. [#8164](https://github.com/ClickHouse/ClickHouse/pull/8164) ([小路](https://github.com/nicelulu)) +- Fixed bug that lead to a data race in DB::BlockStreamProfileInfo::calculateRowsBeforeLimit() [#8143](https://github.com/ClickHouse/ClickHouse/pull/8143) ([Alexander Kazakov](https://github.com/Akazz)) +- Fixed `ALTER table MOVE part` executed immediately after merging the specified part, which could cause moving a part which the specified part merged into. Now it correctly moves the specified part. [#8104](https://github.com/ClickHouse/ClickHouse/pull/8104) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Expressions for dictionaries can be specified as strings now. This is useful for calculation of attributes while extracting data from non-ClickHouse sources because it allows to use non-ClickHouse syntax for those expressions. [#8098](https://github.com/ClickHouse/ClickHouse/pull/8098) ([alesapin](https://github.com/alesapin)) +- Fixed a very rare race in `clickhouse-copier` because of an overflow in ZXid. [#8088](https://github.com/ClickHouse/ClickHouse/pull/8088) ([Ding Xiang Fei](https://github.com/dingxiangfei2009)) - Fixed the bug when after the query failed (due to “Too many simultaneous queries” for example) it would not read external tables info, and the - next request would interpret this info as the beginning of the next query causing an error like `Unknown packet from client`. [\#8084](https://github.com/ClickHouse/ClickHouse/pull/8084) ([Azat Khuzhin](https://github.com/azat)) -- Avoid null dereference after “Unknown packet X from server” [\#8071](https://github.com/ClickHouse/ClickHouse/pull/8071) ([Azat Khuzhin](https://github.com/azat)) -- Restore support of all ICU locales, add the ability to apply collations for constant expressions and add language name to system.collations table. [\#8051](https://github.com/ClickHouse/ClickHouse/pull/8051) ([alesapin](https://github.com/alesapin)) -- Number of streams for read from `StorageFile` and `StorageHDFS` is now limited, to avoid exceeding the memory limit. [\#7981](https://github.com/ClickHouse/ClickHouse/pull/7981) ([alesapin](https://github.com/alesapin)) -- Fixed `CHECK TABLE` query for `*MergeTree` tables without key. [\#7979](https://github.com/ClickHouse/ClickHouse/pull/7979) ([alesapin](https://github.com/alesapin)) -- Removed the mutation number from a part name in case there were no mutations. This removing improved the compatibility with older versions. [\#8250](https://github.com/ClickHouse/ClickHouse/pull/8250) ([alesapin](https://github.com/alesapin)) -- Fixed the bug that mutations are skipped for some attached parts due to their data\_version are larger than the table mutation version. [\#7812](https://github.com/ClickHouse/ClickHouse/pull/7812) ([Zhichang Yu](https://github.com/yuzhichang)) -- Allow starting the server with redundant copies of parts after moving them to another device. [\#7810](https://github.com/ClickHouse/ClickHouse/pull/7810) ([Vladimir Chebotarev](https://github.com/excitoon)) -- Fixed the error “Sizes of columns doesn’t match” that might appear when using aggregate function columns. [\#7790](https://github.com/ClickHouse/ClickHouse/pull/7790) ([Boris Granveaud](https://github.com/bgranvea)) -- Now an exception will be thrown in case of using WITH TIES alongside LIMIT BY. And now it’s possible to use TOP with LIMIT BY. [\#7637](https://github.com/ClickHouse/ClickHouse/pull/7637) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -- Fix dictionary reload if it has `invalidate_query`, which stopped updates and some exception on previous update tries. [\#8029](https://github.com/ClickHouse/ClickHouse/pull/8029) ([alesapin](https://github.com/alesapin)) + next request would interpret this info as the beginning of the next query causing an error like `Unknown packet from client`. [#8084](https://github.com/ClickHouse/ClickHouse/pull/8084) ([Azat Khuzhin](https://github.com/azat)) +- Avoid null dereference after “Unknown packet X from server” [#8071](https://github.com/ClickHouse/ClickHouse/pull/8071) ([Azat Khuzhin](https://github.com/azat)) +- Restore support of all ICU locales, add the ability to apply collations for constant expressions and add language name to system.collations table. [#8051](https://github.com/ClickHouse/ClickHouse/pull/8051) ([alesapin](https://github.com/alesapin)) +- Number of streams for read from `StorageFile` and `StorageHDFS` is now limited, to avoid exceeding the memory limit. [#7981](https://github.com/ClickHouse/ClickHouse/pull/7981) ([alesapin](https://github.com/alesapin)) +- Fixed `CHECK TABLE` query for `*MergeTree` tables without key. [#7979](https://github.com/ClickHouse/ClickHouse/pull/7979) ([alesapin](https://github.com/alesapin)) +- Removed the mutation number from a part name in case there were no mutations. This removing improved the compatibility with older versions. [#8250](https://github.com/ClickHouse/ClickHouse/pull/8250) ([alesapin](https://github.com/alesapin)) +- Fixed the bug that mutations are skipped for some attached parts due to their data_version are larger than the table mutation version. [#7812](https://github.com/ClickHouse/ClickHouse/pull/7812) ([Zhichang Yu](https://github.com/yuzhichang)) +- Allow starting the server with redundant copies of parts after moving them to another device. [#7810](https://github.com/ClickHouse/ClickHouse/pull/7810) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Fixed the error “Sizes of columns doesn’t match” that might appear when using aggregate function columns. [#7790](https://github.com/ClickHouse/ClickHouse/pull/7790) ([Boris Granveaud](https://github.com/bgranvea)) +- Now an exception will be thrown in case of using WITH TIES alongside LIMIT BY. And now it’s possible to use TOP with LIMIT BY. [#7637](https://github.com/ClickHouse/ClickHouse/pull/7637) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +- Fix dictionary reload if it has `invalidate_query`, which stopped updates and some exception on previous update tries. [#8029](https://github.com/ClickHouse/ClickHouse/pull/8029) ([alesapin](https://github.com/alesapin)) ### ClickHouse Release 19.17.4.11, 2019-11-22 {#clickhouse-release-v19-17-4-11-2019-11-22} #### Backward Incompatible Change {#backward-incompatible-change} -- Using column instead of AST to store scalar subquery results for better performance. Setting `enable_scalar_subquery_optimization` was added in 19.17 and it was enabled by default. It leads to errors like [this](https://github.com/ClickHouse/ClickHouse/issues/7851) during upgrade to 19.17.2 or 19.17.3 from previous versions. This setting was disabled by default in 19.17.4, to make possible upgrading from 19.16 and older versions without errors. [\#7392](https://github.com/ClickHouse/ClickHouse/pull/7392) ([Amos Bird](https://github.com/amosbird)) +- Using column instead of AST to store scalar subquery results for better performance. Setting `enable_scalar_subquery_optimization` was added in 19.17 and it was enabled by default. It leads to errors like [this](https://github.com/ClickHouse/ClickHouse/issues/7851) during upgrade to 19.17.2 or 19.17.3 from previous versions. This setting was disabled by default in 19.17.4, to make possible upgrading from 19.16 and older versions without errors. [#7392](https://github.com/ClickHouse/ClickHouse/pull/7392) ([Amos Bird](https://github.com/amosbird)) #### New Feature {#new-feature} -- Add the ability to create dictionaries with DDL queries. [\#7360](https://github.com/ClickHouse/ClickHouse/pull/7360) ([alesapin](https://github.com/alesapin)) -- Make `bloom_filter` type of index supporting `LowCardinality` and `Nullable` [\#7363](https://github.com/ClickHouse/ClickHouse/issues/7363) [\#7561](https://github.com/ClickHouse/ClickHouse/pull/7561) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Add function `isValidJSON` to check that passed string is a valid json. [\#5910](https://github.com/ClickHouse/ClickHouse/issues/5910) [\#7293](https://github.com/ClickHouse/ClickHouse/pull/7293) ([Vdimir](https://github.com/Vdimir)) -- Implement `arrayCompact` function [\#7328](https://github.com/ClickHouse/ClickHouse/pull/7328) ([Memo](https://github.com/Joeywzr)) -- Created function `hex` for Decimal numbers. It works like `hex(reinterpretAsString())`, but doesn’t delete last zero bytes. [\#7355](https://github.com/ClickHouse/ClickHouse/pull/7355) ([Mikhail Korotov](https://github.com/millb)) -- Add `arrayFill` and `arrayReverseFill` functions, which replace elements by other elements in front/back of them in the array. [\#7380](https://github.com/ClickHouse/ClickHouse/pull/7380) ([hcz](https://github.com/hczhcz)) -- Add `CRC32IEEE()`/`CRC64()` support [\#7480](https://github.com/ClickHouse/ClickHouse/pull/7480) ([Azat Khuzhin](https://github.com/azat)) -- Implement `char` function similar to one in [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)) -- Add `bitmapTransform` function. It transforms an array of values in a bitmap to another array of values, the result is a new bitmap [\#7598](https://github.com/ClickHouse/ClickHouse/pull/7598) ([Zhichang Yu](https://github.com/yuzhichang)) -- Implemented `javaHashUTF16LE()` function [\#7651](https://github.com/ClickHouse/ClickHouse/pull/7651) ([achimbab](https://github.com/achimbab)) -- Add `_shard_num` virtual column for the Distributed engine [\#7624](https://github.com/ClickHouse/ClickHouse/pull/7624) ([Azat Khuzhin](https://github.com/azat)) +- Add the ability to create dictionaries with DDL queries. [#7360](https://github.com/ClickHouse/ClickHouse/pull/7360) ([alesapin](https://github.com/alesapin)) +- Make `bloom_filter` type of index supporting `LowCardinality` and `Nullable` [#7363](https://github.com/ClickHouse/ClickHouse/issues/7363) [#7561](https://github.com/ClickHouse/ClickHouse/pull/7561) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Add function `isValidJSON` to check that passed string is a valid json. [#5910](https://github.com/ClickHouse/ClickHouse/issues/5910) [#7293](https://github.com/ClickHouse/ClickHouse/pull/7293) ([Vdimir](https://github.com/Vdimir)) +- Implement `arrayCompact` function [#7328](https://github.com/ClickHouse/ClickHouse/pull/7328) ([Memo](https://github.com/Joeywzr)) +- Created function `hex` for Decimal numbers. It works like `hex(reinterpretAsString())`, but doesn’t delete last zero bytes. [#7355](https://github.com/ClickHouse/ClickHouse/pull/7355) ([Mikhail Korotov](https://github.com/millb)) +- Add `arrayFill` and `arrayReverseFill` functions, which replace elements by other elements in front/back of them in the array. [#7380](https://github.com/ClickHouse/ClickHouse/pull/7380) ([hcz](https://github.com/hczhcz)) +- Add `CRC32IEEE()`/`CRC64()` support [#7480](https://github.com/ClickHouse/ClickHouse/pull/7480) ([Azat Khuzhin](https://github.com/azat)) +- Implement `char` function similar to one in [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)) +- Add `bitmapTransform` function. It transforms an array of values in a bitmap to another array of values, the result is a new bitmap [#7598](https://github.com/ClickHouse/ClickHouse/pull/7598) ([Zhichang Yu](https://github.com/yuzhichang)) +- Implemented `javaHashUTF16LE()` function [#7651](https://github.com/ClickHouse/ClickHouse/pull/7651) ([achimbab](https://github.com/achimbab)) +- Add `_shard_num` virtual column for the Distributed engine [#7624](https://github.com/ClickHouse/ClickHouse/pull/7624) ([Azat Khuzhin](https://github.com/azat)) #### Experimental Feature {#experimental-feature} -- Support for processors (new query execution pipeline) in `MergeTree`. [\#7181](https://github.com/ClickHouse/ClickHouse/pull/7181) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Support for processors (new query execution pipeline) in `MergeTree`. [#7181](https://github.com/ClickHouse/ClickHouse/pull/7181) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) #### Bug Fix {#bug-fix-1} -- Fix incorrect float parsing in `Values` [\#7817](https://github.com/ClickHouse/ClickHouse/issues/7817) [\#7870](https://github.com/ClickHouse/ClickHouse/pull/7870) ([tavplubix](https://github.com/tavplubix)) -- Fix rare deadlock which can happen when trace\_log is enabled. [\#7838](https://github.com/ClickHouse/ClickHouse/pull/7838) ([filimonov](https://github.com/filimonov)) -- Prevent message duplication when producing Kafka table has any MVs selecting from it [\#7265](https://github.com/ClickHouse/ClickHouse/pull/7265) ([Ivan](https://github.com/abyss7)) -- Support for `Array(LowCardinality(Nullable(String)))` in `IN`. Resolves [\#7364](https://github.com/ClickHouse/ClickHouse/issues/7364) [\#7366](https://github.com/ClickHouse/ClickHouse/pull/7366) ([achimbab](https://github.com/achimbab)) -- Add handling of `SQL_TINYINT` and `SQL_BIGINT`, and fix handling of `SQL_FLOAT` data source types in ODBC Bridge. [\#7491](https://github.com/ClickHouse/ClickHouse/pull/7491) ([Denis Glazachev](https://github.com/traceon)) -- Fix aggregation (`avg` and quantiles) over empty decimal columns [\#7431](https://github.com/ClickHouse/ClickHouse/pull/7431) ([Andrey Konyaev](https://github.com/akonyaev90)) -- Fix `INSERT` into Distributed with `MATERIALIZED` columns [\#7377](https://github.com/ClickHouse/ClickHouse/pull/7377) ([Azat Khuzhin](https://github.com/azat)) -- Make `MOVE PARTITION` work if some parts of partition are already on destination disk or volume [\#7434](https://github.com/ClickHouse/ClickHouse/pull/7434) ([Vladimir Chebotarev](https://github.com/excitoon)) -- Fixed bug with hardlinks failing to be created during mutations in `ReplicatedMergeTree` in multi-disk configurations. [\#7558](https://github.com/ClickHouse/ClickHouse/pull/7558) ([Vladimir Chebotarev](https://github.com/excitoon)) -- Fixed a bug with a mutation on a MergeTree when whole part remains unchanged and best space is being found on another disk [\#7602](https://github.com/ClickHouse/ClickHouse/pull/7602) ([Vladimir Chebotarev](https://github.com/excitoon)) -- Fixed bug with `keep_free_space_ratio` not being read from disks configuration [\#7645](https://github.com/ClickHouse/ClickHouse/pull/7645) ([Vladimir Chebotarev](https://github.com/excitoon)) -- Fix bug with table contains only `Tuple` columns or columns with complex paths. Fixes [7541](https://github.com/ClickHouse/ClickHouse/issues/7541). [\#7545](https://github.com/ClickHouse/ClickHouse/pull/7545) ([alesapin](https://github.com/alesapin)) -- Do not account memory for Buffer engine in max\_memory\_usage limit [\#7552](https://github.com/ClickHouse/ClickHouse/pull/7552) ([Azat Khuzhin](https://github.com/azat)) -- Fix final mark usage in `MergeTree` tables ordered by `tuple()`. In rare cases it could lead to `Can't adjust last granule` error while select. [\#7639](https://github.com/ClickHouse/ClickHouse/pull/7639) ([Anton Popov](https://github.com/CurtizJ)) -- Fix bug in mutations that have predicate with actions that require context (for example functions for json), which may lead to crashes or strange exceptions. [\#7664](https://github.com/ClickHouse/ClickHouse/pull/7664) ([alesapin](https://github.com/alesapin)) -- Fix mismatch of database and table names escaping in `data/` and `shadow/` directories [\#7575](https://github.com/ClickHouse/ClickHouse/pull/7575) ([Alexander 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`. Fix crash in this case. [\#7586](https://github.com/ClickHouse/ClickHouse/pull/7586) ([Artem Zuikov](https://github.com/4ertus2)) -- Fix `Not found column in block` when joining on expression with RIGHT or FULL JOIN. [\#7641](https://github.com/ClickHouse/ClickHouse/pull/7641) ([Artem Zuikov](https://github.com/4ertus2)) -- One more attempt to fix infinite loop in `PrettySpace` format [\#7591](https://github.com/ClickHouse/ClickHouse/pull/7591) ([Olga Khvostikova](https://github.com/stavrolia)) -- Fix bug in `concat` function when all arguments were `FixedString` of the same size. [\#7635](https://github.com/ClickHouse/ClickHouse/pull/7635) ([alesapin](https://github.com/alesapin)) -- Fixed exception in case of using 1 argument while defining S3, URL and HDFS storages. [\#7618](https://github.com/ClickHouse/ClickHouse/pull/7618) ([Vladimir Chebotarev](https://github.com/excitoon)) -- Fix scope of the InterpreterSelectQuery for views with query [\#7601](https://github.com/ClickHouse/ClickHouse/pull/7601) ([Azat Khuzhin](https://github.com/azat)) +- Fix incorrect float parsing in `Values` [#7817](https://github.com/ClickHouse/ClickHouse/issues/7817) [#7870](https://github.com/ClickHouse/ClickHouse/pull/7870) ([tavplubix](https://github.com/tavplubix)) +- Fix rare deadlock which can happen when trace_log is enabled. [#7838](https://github.com/ClickHouse/ClickHouse/pull/7838) ([filimonov](https://github.com/filimonov)) +- Prevent message duplication when producing Kafka table has any MVs selecting from it [#7265](https://github.com/ClickHouse/ClickHouse/pull/7265) ([Ivan](https://github.com/abyss7)) +- Support for `Array(LowCardinality(Nullable(String)))` in `IN`. Resolves [#7364](https://github.com/ClickHouse/ClickHouse/issues/7364) [#7366](https://github.com/ClickHouse/ClickHouse/pull/7366) ([achimbab](https://github.com/achimbab)) +- Add handling of `SQL_TINYINT` and `SQL_BIGINT`, and fix handling of `SQL_FLOAT` data source types in ODBC Bridge. [#7491](https://github.com/ClickHouse/ClickHouse/pull/7491) ([Denis Glazachev](https://github.com/traceon)) +- Fix aggregation (`avg` and quantiles) over empty decimal columns [#7431](https://github.com/ClickHouse/ClickHouse/pull/7431) ([Andrey Konyaev](https://github.com/akonyaev90)) +- Fix `INSERT` into Distributed with `MATERIALIZED` columns [#7377](https://github.com/ClickHouse/ClickHouse/pull/7377) ([Azat Khuzhin](https://github.com/azat)) +- Make `MOVE PARTITION` work if some parts of partition are already on destination disk or volume [#7434](https://github.com/ClickHouse/ClickHouse/pull/7434) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Fixed bug with hardlinks failing to be created during mutations in `ReplicatedMergeTree` in multi-disk configurations. [#7558](https://github.com/ClickHouse/ClickHouse/pull/7558) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Fixed a bug with a mutation on a MergeTree when whole part remains unchanged and best space is being found on another disk [#7602](https://github.com/ClickHouse/ClickHouse/pull/7602) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Fixed bug with `keep_free_space_ratio` not being read from disks configuration [#7645](https://github.com/ClickHouse/ClickHouse/pull/7645) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Fix bug with table contains only `Tuple` columns or columns with complex paths. Fixes [7541](https://github.com/ClickHouse/ClickHouse/issues/7541). [#7545](https://github.com/ClickHouse/ClickHouse/pull/7545) ([alesapin](https://github.com/alesapin)) +- Do not account memory for Buffer engine in max_memory_usage limit [#7552](https://github.com/ClickHouse/ClickHouse/pull/7552) ([Azat Khuzhin](https://github.com/azat)) +- Fix final mark usage in `MergeTree` tables ordered by `tuple()`. In rare cases it could lead to `Can't adjust last granule` error while select. [#7639](https://github.com/ClickHouse/ClickHouse/pull/7639) ([Anton Popov](https://github.com/CurtizJ)) +- Fix bug in mutations that have predicate with actions that require context (for example functions for json), which may lead to crashes or strange exceptions. [#7664](https://github.com/ClickHouse/ClickHouse/pull/7664) ([alesapin](https://github.com/alesapin)) +- Fix mismatch of database and table names escaping in `data/` and `shadow/` directories [#7575](https://github.com/ClickHouse/ClickHouse/pull/7575) ([Alexander 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`. Fix crash in this case. [#7586](https://github.com/ClickHouse/ClickHouse/pull/7586) ([Artem Zuikov](https://github.com/4ertus2)) +- Fix `Not found column in block` when joining on expression with RIGHT or FULL JOIN. [#7641](https://github.com/ClickHouse/ClickHouse/pull/7641) ([Artem Zuikov](https://github.com/4ertus2)) +- One more attempt to fix infinite loop in `PrettySpace` format [#7591](https://github.com/ClickHouse/ClickHouse/pull/7591) ([Olga Khvostikova](https://github.com/stavrolia)) +- Fix bug in `concat` function when all arguments were `FixedString` of the same size. [#7635](https://github.com/ClickHouse/ClickHouse/pull/7635) ([alesapin](https://github.com/alesapin)) +- Fixed exception in case of using 1 argument while defining S3, URL and HDFS storages. [#7618](https://github.com/ClickHouse/ClickHouse/pull/7618) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Fix scope of the InterpreterSelectQuery for views with query [#7601](https://github.com/ClickHouse/ClickHouse/pull/7601) ([Azat Khuzhin](https://github.com/azat)) #### Improvement {#improvement} -- `Nullable` columns recognized and NULL-values handled correctly by ODBC-bridge [\#7402](https://github.com/ClickHouse/ClickHouse/pull/7402) ([Vasily Nemkov](https://github.com/Enmk)) -- Write current batch for distributed send atomically [\#7600](https://github.com/ClickHouse/ClickHouse/pull/7600) ([Azat Khuzhin](https://github.com/azat)) -- Throw an exception if we cannot detect table for column name in query. [\#7358](https://github.com/ClickHouse/ClickHouse/pull/7358) ([Artem Zuikov](https://github.com/4ertus2)) -- Add `merge_max_block_size` setting to `MergeTreeSettings` [\#7412](https://github.com/ClickHouse/ClickHouse/pull/7412) ([Artem Zuikov](https://github.com/4ertus2)) -- Queries with `HAVING` and without `GROUP BY` assume group by constant. So, `SELECT 1 HAVING 1` now returns a result. [\#7496](https://github.com/ClickHouse/ClickHouse/pull/7496) ([Amos Bird](https://github.com/amosbird)) -- Support parsing `(X,)` as tuple similar to python. [\#7501](https://github.com/ClickHouse/ClickHouse/pull/7501), [\#7562](https://github.com/ClickHouse/ClickHouse/pull/7562) ([Amos Bird](https://github.com/amosbird)) -- Make `range` function behaviors almost like pythonic one. [\#7518](https://github.com/ClickHouse/ClickHouse/pull/7518) ([sundyli](https://github.com/sundy-li)) -- Add `constraints` columns to table `system.settings` [\#7553](https://github.com/ClickHouse/ClickHouse/pull/7553) ([Vitaly Baranov](https://github.com/vitlibar)) -- Better Null format for tcp handler, so that it’s possible to use `select ignore() from table format Null` for perf measure via clickhouse-client [\#7606](https://github.com/ClickHouse/ClickHouse/pull/7606) ([Amos Bird](https://github.com/amosbird)) -- Queries like `CREATE TABLE ... AS (SELECT (1, 2))` are parsed correctly [\#7542](https://github.com/ClickHouse/ClickHouse/pull/7542) ([hcz](https://github.com/hczhcz)) +- `Nullable` columns recognized and NULL-values handled correctly by ODBC-bridge [#7402](https://github.com/ClickHouse/ClickHouse/pull/7402) ([Vasily Nemkov](https://github.com/Enmk)) +- Write current batch for distributed send atomically [#7600](https://github.com/ClickHouse/ClickHouse/pull/7600) ([Azat Khuzhin](https://github.com/azat)) +- Throw an exception if we cannot detect table for column name in query. [#7358](https://github.com/ClickHouse/ClickHouse/pull/7358) ([Artem Zuikov](https://github.com/4ertus2)) +- Add `merge_max_block_size` setting to `MergeTreeSettings` [#7412](https://github.com/ClickHouse/ClickHouse/pull/7412) ([Artem Zuikov](https://github.com/4ertus2)) +- Queries with `HAVING` and without `GROUP BY` assume group by constant. So, `SELECT 1 HAVING 1` now returns a result. [#7496](https://github.com/ClickHouse/ClickHouse/pull/7496) ([Amos Bird](https://github.com/amosbird)) +- Support parsing `(X,)` as tuple similar to python. [#7501](https://github.com/ClickHouse/ClickHouse/pull/7501), [#7562](https://github.com/ClickHouse/ClickHouse/pull/7562) ([Amos Bird](https://github.com/amosbird)) +- Make `range` function behaviors almost like pythonic one. [#7518](https://github.com/ClickHouse/ClickHouse/pull/7518) ([sundyli](https://github.com/sundy-li)) +- Add `constraints` columns to table `system.settings` [#7553](https://github.com/ClickHouse/ClickHouse/pull/7553) ([Vitaly Baranov](https://github.com/vitlibar)) +- Better Null format for tcp handler, so that it’s possible to use `select ignore() from table format Null` for perf measure via clickhouse-client [#7606](https://github.com/ClickHouse/ClickHouse/pull/7606) ([Amos Bird](https://github.com/amosbird)) +- Queries like `CREATE TABLE ... AS (SELECT (1, 2))` are parsed correctly [#7542](https://github.com/ClickHouse/ClickHouse/pull/7542) ([hcz](https://github.com/hczhcz)) #### Performance Improvement {#performance-improvement} -- The performance of aggregation over short string keys is improved. [\#6243](https://github.com/ClickHouse/ClickHouse/pull/6243) ([Alexander Kuzmenkov](https://github.com/akuzm), [Amos Bird](https://github.com/amosbird)) -- Run another pass of syntax/expression analysis to get potential optimizations after constant predicates are folded. [\#7497](https://github.com/ClickHouse/ClickHouse/pull/7497) ([Amos Bird](https://github.com/amosbird)) -- Use storage meta info to evaluate trivial `SELECT count() FROM table;` [\#7510](https://github.com/ClickHouse/ClickHouse/pull/7510) ([Amos Bird](https://github.com/amosbird), [alexey-milovidov](https://github.com/alexey-milovidov)) -- Vectorize processing `arrayReduce` similar to Aggregator `addBatch`. [\#7608](https://github.com/ClickHouse/ClickHouse/pull/7608) ([Amos Bird](https://github.com/amosbird)) -- Minor improvements in performance of `Kafka` consumption [\#7475](https://github.com/ClickHouse/ClickHouse/pull/7475) ([Ivan](https://github.com/abyss7)) +- The performance of aggregation over short string keys is improved. [#6243](https://github.com/ClickHouse/ClickHouse/pull/6243) ([Alexander Kuzmenkov](https://github.com/akuzm), [Amos Bird](https://github.com/amosbird)) +- Run another pass of syntax/expression analysis to get potential optimizations after constant predicates are folded. [#7497](https://github.com/ClickHouse/ClickHouse/pull/7497) ([Amos Bird](https://github.com/amosbird)) +- Use storage meta info to evaluate trivial `SELECT count() FROM table;` [#7510](https://github.com/ClickHouse/ClickHouse/pull/7510) ([Amos Bird](https://github.com/amosbird), [alexey-milovidov](https://github.com/alexey-milovidov)) +- Vectorize processing `arrayReduce` similar to Aggregator `addBatch`. [#7608](https://github.com/ClickHouse/ClickHouse/pull/7608) ([Amos Bird](https://github.com/amosbird)) +- Minor improvements in performance of `Kafka` consumption [#7475](https://github.com/ClickHouse/ClickHouse/pull/7475) ([Ivan](https://github.com/abyss7)) #### Build/Testing/Packaging Improvement {#buildtestingpackaging-improvement} -- Add support for cross-compiling to the CPU architecture AARCH64. Refactor packager script. [\#7370](https://github.com/ClickHouse/ClickHouse/pull/7370) [\#7539](https://github.com/ClickHouse/ClickHouse/pull/7539) ([Ivan](https://github.com/abyss7)) -- Unpack darwin-x86\_64 and linux-aarch64 toolchains into mounted Docker volume when building packages [\#7534](https://github.com/ClickHouse/ClickHouse/pull/7534) ([Ivan](https://github.com/abyss7)) -- Update Docker Image for Binary Packager [\#7474](https://github.com/ClickHouse/ClickHouse/pull/7474) ([Ivan](https://github.com/abyss7)) -- Fixed compile errors on MacOS Catalina [\#7585](https://github.com/ClickHouse/ClickHouse/pull/7585) ([Ernest Poletaev](https://github.com/ernestp)) -- Some refactoring in query analysis logic: split complex class into several simple ones. [\#7454](https://github.com/ClickHouse/ClickHouse/pull/7454) ([Artem Zuikov](https://github.com/4ertus2)) -- Fix build without submodules [\#7295](https://github.com/ClickHouse/ClickHouse/pull/7295) ([proller](https://github.com/proller)) -- Better `add_globs` in CMake files [\#7418](https://github.com/ClickHouse/ClickHouse/pull/7418) ([Amos Bird](https://github.com/amosbird)) -- Remove hardcoded paths in `unwind` target [\#7460](https://github.com/ClickHouse/ClickHouse/pull/7460) ([Konstantin Podshumok](https://github.com/podshumok)) -- Allow to use mysql format without ssl [\#7524](https://github.com/ClickHouse/ClickHouse/pull/7524) ([proller](https://github.com/proller)) +- Add support for cross-compiling to the CPU architecture AARCH64. Refactor packager script. [#7370](https://github.com/ClickHouse/ClickHouse/pull/7370) [#7539](https://github.com/ClickHouse/ClickHouse/pull/7539) ([Ivan](https://github.com/abyss7)) +- Unpack darwin-x86_64 and linux-aarch64 toolchains into mounted Docker volume when building packages [#7534](https://github.com/ClickHouse/ClickHouse/pull/7534) ([Ivan](https://github.com/abyss7)) +- Update Docker Image for Binary Packager [#7474](https://github.com/ClickHouse/ClickHouse/pull/7474) ([Ivan](https://github.com/abyss7)) +- Fixed compile errors on MacOS Catalina [#7585](https://github.com/ClickHouse/ClickHouse/pull/7585) ([Ernest Poletaev](https://github.com/ernestp)) +- Some refactoring in query analysis logic: split complex class into several simple ones. [#7454](https://github.com/ClickHouse/ClickHouse/pull/7454) ([Artem Zuikov](https://github.com/4ertus2)) +- Fix build without submodules [#7295](https://github.com/ClickHouse/ClickHouse/pull/7295) ([proller](https://github.com/proller)) +- Better `add_globs` in CMake files [#7418](https://github.com/ClickHouse/ClickHouse/pull/7418) ([Amos Bird](https://github.com/amosbird)) +- Remove hardcoded paths in `unwind` target [#7460](https://github.com/ClickHouse/ClickHouse/pull/7460) ([Konstantin Podshumok](https://github.com/podshumok)) +- Allow to use mysql format without ssl [#7524](https://github.com/ClickHouse/ClickHouse/pull/7524) ([proller](https://github.com/proller)) #### Other {#other} -- Added ANTLR4 grammar for ClickHouse SQL dialect [\#7595](https://github.com/ClickHouse/ClickHouse/issues/7595) [\#7596](https://github.com/ClickHouse/ClickHouse/pull/7596) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Added ANTLR4 grammar for ClickHouse SQL dialect [#7595](https://github.com/ClickHouse/ClickHouse/issues/7595) [#7596](https://github.com/ClickHouse/ClickHouse/pull/7596) ([alexey-milovidov](https://github.com/alexey-milovidov)) ## ClickHouse Release 19.16 {#clickhouse-release-v19-16} #### ClickHouse Release 19.16.14.65, 2020-03-25 {#clickhouse-release-v19-16-14-65-2020-03-25} -- Fixed up a bug in batched calculations of ternary logical OPs on multiple arguments (more than 10). [\#8718](https://github.com/ClickHouse/ClickHouse/pull/8718) ([Alexander Kazakov](https://github.com/Akazz)) This bugfix was backported to version 19.16 by a special request from Altinity. +- Fixed up a bug in batched calculations of ternary logical OPs on multiple arguments (more than 10). [#8718](https://github.com/ClickHouse/ClickHouse/pull/8718) ([Alexander Kazakov](https://github.com/Akazz)) This bugfix was backported to version 19.16 by a special request from Altinity. #### ClickHouse Release 19.16.14.65, 2020-03-05 {#clickhouse-release-v19-16-14-65-2020-03-05} -- Fix distributed subqueries incompatibility with older CH versions. Fixes [\#7851](https://github.com/ClickHouse/ClickHouse/issues/7851) +- Fix distributed subqueries incompatibility with older CH versions. Fixes [#7851](https://github.com/ClickHouse/ClickHouse/issues/7851) [(tabplubix)](https://github.com/tavplubix) -- When executing `CREATE` query, fold constant expressions in storage engine arguments. Replace empty database name with current database. Fixes [\#6508](https://github.com/ClickHouse/ClickHouse/issues/6508), [\#3492](https://github.com/ClickHouse/ClickHouse/issues/3492). Also fix check for local address in `ClickHouseDictionarySource`. - [\#9262](https://github.com/ClickHouse/ClickHouse/pull/9262) [(tabplubix)](https://github.com/tavplubix) +- When executing `CREATE` query, fold constant expressions in storage engine arguments. Replace empty database name with current database. Fixes [#6508](https://github.com/ClickHouse/ClickHouse/issues/6508), [#3492](https://github.com/ClickHouse/ClickHouse/issues/3492). Also fix check for local address in `ClickHouseDictionarySource`. + [#9262](https://github.com/ClickHouse/ClickHouse/pull/9262) [(tabplubix)](https://github.com/tavplubix) - Now background merges in `*MergeTree` table engines family preserve storage policy volume order more accurately. - [\#8549](https://github.com/ClickHouse/ClickHouse/pull/8549) ([Vladimir Chebotarev](https://github.com/excitoon)) -- Prevent losing data in `Kafka` in rare cases when exception happens after reading suffix but before commit. Fixes [\#9378](https://github.com/ClickHouse/ClickHouse/issues/9378). Related: [\#7175](https://github.com/ClickHouse/ClickHouse/issues/7175) - [\#9507](https://github.com/ClickHouse/ClickHouse/pull/9507) [(filimonov)](https://github.com/filimonov) -- Fix bug leading to server termination when trying to use / drop `Kafka` table created with wrong parameters. Fixes [\#9494](https://github.com/ClickHouse/ClickHouse/issues/9494). Incorporates [\#9507](https://github.com/ClickHouse/ClickHouse/issues/9507). - [\#9513](https://github.com/ClickHouse/ClickHouse/pull/9513) [(filimonov)](https://github.com/filimonov) + [#8549](https://github.com/ClickHouse/ClickHouse/pull/8549) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Prevent losing data in `Kafka` in rare cases when exception happens after reading suffix but before commit. Fixes [#9378](https://github.com/ClickHouse/ClickHouse/issues/9378). Related: [#7175](https://github.com/ClickHouse/ClickHouse/issues/7175) + [#9507](https://github.com/ClickHouse/ClickHouse/pull/9507) [(filimonov)](https://github.com/filimonov) +- Fix bug leading to server termination when trying to use / drop `Kafka` table created with wrong parameters. Fixes [#9494](https://github.com/ClickHouse/ClickHouse/issues/9494). Incorporates [#9507](https://github.com/ClickHouse/ClickHouse/issues/9507). + [#9513](https://github.com/ClickHouse/ClickHouse/pull/9513) [(filimonov)](https://github.com/filimonov) - Allow using `MaterializedView` with subqueries above `Kafka` tables. - [\#8197](https://github.com/ClickHouse/ClickHouse/pull/8197) ([filimonov](https://github.com/filimonov)) + [#8197](https://github.com/ClickHouse/ClickHouse/pull/8197) ([filimonov](https://github.com/filimonov)) #### New Feature {#new-feature-1} - Add `deduplicate_blocks_in_dependent_materialized_views` option to control the behaviour of idempotent inserts into tables with materialized views. This new feature was added to the bugfix release by a special request from Altinity. - [\#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) [(urykhy)](https://github.com/urykhy) + [#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) [(urykhy)](https://github.com/urykhy) ### ClickHouse Release 19.16.2.2, 2019-10-30 {#clickhouse-release-v19-16-2-2-2019-10-30} #### Backward Incompatible Change {#backward-incompatible-change-1} - Add missing arity validation for count/counIf. - [\#7095](https://github.com/ClickHouse/ClickHouse/issues/7095) - [\#7298](https://github.com/ClickHouse/ClickHouse/pull/7298) ([Vdimir](https://github.com/Vdimir)) + [#7095](https://github.com/ClickHouse/ClickHouse/issues/7095) + [#7298](https://github.com/ClickHouse/ClickHouse/pull/7298) ([Vdimir](https://github.com/Vdimir)) - Remove legacy `asterisk_left_columns_only` setting (it was disabled by default). - [\#7335](https://github.com/ClickHouse/ClickHouse/pull/7335) ([Artem + [#7335](https://github.com/ClickHouse/ClickHouse/pull/7335) ([Artem Zuikov](https://github.com/4ertus2)) - Format strings for Template data format are now specified in files. - [\#7118](https://github.com/ClickHouse/ClickHouse/pull/7118) + [#7118](https://github.com/ClickHouse/ClickHouse/pull/7118) ([tavplubix](https://github.com/tavplubix)) #### New Feature {#new-feature-2} -- Introduce uniqCombined64() to calculate cardinality greater than UINT\_MAX. - [\#7213](https://github.com/ClickHouse/ClickHouse/pull/7213), - [\#7222](https://github.com/ClickHouse/ClickHouse/pull/7222) ([Azat +- Introduce uniqCombined64() to calculate cardinality greater than UINT_MAX. + [#7213](https://github.com/ClickHouse/ClickHouse/pull/7213), + [#7222](https://github.com/ClickHouse/ClickHouse/pull/7222) ([Azat Khuzhin](https://github.com/azat)) - Support Bloom filter indexes on Array columns. - [\#6984](https://github.com/ClickHouse/ClickHouse/pull/6984) + [#6984](https://github.com/ClickHouse/ClickHouse/pull/6984) ([achimbab](https://github.com/achimbab)) - Add a function `getMacro(name)` that returns String with the value of corresponding `` - from server configuration. [\#7240](https://github.com/ClickHouse/ClickHouse/pull/7240) + from server configuration. [#7240](https://github.com/ClickHouse/ClickHouse/pull/7240) ([alexey-milovidov](https://github.com/alexey-milovidov)) - Set two configuration options for a dictionary based on an HTTP source: `credentials` and - `http-headers`. [\#7092](https://github.com/ClickHouse/ClickHouse/pull/7092) ([Guillaume + `http-headers`. [#7092](https://github.com/ClickHouse/ClickHouse/pull/7092) ([Guillaume Tassery](https://github.com/YiuRULE)) - Add a new ProfileEvent `Merge` that counts the number of launched background merges. - [\#7093](https://github.com/ClickHouse/ClickHouse/pull/7093) ([Mikhail + [#7093](https://github.com/ClickHouse/ClickHouse/pull/7093) ([Mikhail Korotov](https://github.com/millb)) - Add fullHostName function that returns a fully qualified domain name. - [\#7263](https://github.com/ClickHouse/ClickHouse/issues/7263) - [\#7291](https://github.com/ClickHouse/ClickHouse/pull/7291) ([sundyli](https://github.com/sundy-li)) + [#7263](https://github.com/ClickHouse/ClickHouse/issues/7263) + [#7291](https://github.com/ClickHouse/ClickHouse/pull/7291) ([sundyli](https://github.com/sundy-li)) - Add function `arraySplit` and `arrayReverseSplit` which split an array by “cut off” conditions. They are useful in time sequence handling. - [\#7294](https://github.com/ClickHouse/ClickHouse/pull/7294) ([hcz](https://github.com/hczhcz)) + [#7294](https://github.com/ClickHouse/ClickHouse/pull/7294) ([hcz](https://github.com/hczhcz)) - Add new functions that return the Array of all matched indices in multiMatch family of functions. - [\#7299](https://github.com/ClickHouse/ClickHouse/pull/7299) ([Danila + [#7299](https://github.com/ClickHouse/ClickHouse/pull/7299) ([Danila Kutenin](https://github.com/danlark1)) - Add a new database engine `Lazy` that is optimized for storing a large number of small -Log - tables. [\#7171](https://github.com/ClickHouse/ClickHouse/pull/7171) ([Nikita + tables. [#7171](https://github.com/ClickHouse/ClickHouse/pull/7171) ([Nikita Vasilev](https://github.com/nikvas0)) -- Add aggregate functions groupBitmapAnd, -Or, -Xor for bitmap columns. [\#7109](https://github.com/ClickHouse/ClickHouse/pull/7109) ([Zhichang +- Add aggregate functions groupBitmapAnd, -Or, -Xor for bitmap columns. [#7109](https://github.com/ClickHouse/ClickHouse/pull/7109) ([Zhichang Yu](https://github.com/yuzhichang)) - Add aggregate function combinators -OrNull and -OrDefault, which return null or default values when there is nothing to aggregate. - [\#7331](https://github.com/ClickHouse/ClickHouse/pull/7331) + [#7331](https://github.com/ClickHouse/ClickHouse/pull/7331) ([hcz](https://github.com/hczhcz)) - Introduce CustomSeparated data format that supports custom escaping and - delimiter rules. [\#7118](https://github.com/ClickHouse/ClickHouse/pull/7118) + delimiter rules. [#7118](https://github.com/ClickHouse/ClickHouse/pull/7118) ([tavplubix](https://github.com/tavplubix)) -- Support Redis as source of external dictionary. [\#4361](https://github.com/ClickHouse/ClickHouse/pull/4361) [\#6962](https://github.com/ClickHouse/ClickHouse/pull/6962) ([comunodi](https://github.com/comunodi), [Anton +- Support Redis as source of external dictionary. [#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)) #### Bug Fix {#bug-fix-2} - Fix wrong query result if it has `WHERE IN (SELECT ...)` section and `optimize_read_in_order` is - used. [\#7371](https://github.com/ClickHouse/ClickHouse/pull/7371) ([Anton + used. [#7371](https://github.com/ClickHouse/ClickHouse/pull/7371) ([Anton Popov](https://github.com/CurtizJ)) - Disabled MariaDB authentication plugin, which depends on files outside of project. - [\#7140](https://github.com/ClickHouse/ClickHouse/pull/7140) ([Yuriy + [#7140](https://github.com/ClickHouse/ClickHouse/pull/7140) ([Yuriy Baranov](https://github.com/yurriy)) - Fix exception `Cannot convert column ... because it is constant but values of constants are different in source and result` which could rarely happen when functions `now()`, `today()`, `yesterday()`, `randConstant()` are used. - [\#7156](https://github.com/ClickHouse/ClickHouse/pull/7156) ([Nikolai + [#7156](https://github.com/ClickHouse/ClickHouse/pull/7156) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) - Fixed issue of using HTTP keep alive timeout instead of TCP keep alive timeout. - [\#7351](https://github.com/ClickHouse/ClickHouse/pull/7351) ([Vasily + [#7351](https://github.com/ClickHouse/ClickHouse/pull/7351) ([Vasily Nemkov](https://github.com/Enmk)) -- Fixed a segmentation fault in groupBitmapOr (issue [\#7109](https://github.com/ClickHouse/ClickHouse/issues/7109)). - [\#7289](https://github.com/ClickHouse/ClickHouse/pull/7289) ([Zhichang +- Fixed a segmentation fault in groupBitmapOr (issue [#7109](https://github.com/ClickHouse/ClickHouse/issues/7109)). + [#7289](https://github.com/ClickHouse/ClickHouse/pull/7289) ([Zhichang Yu](https://github.com/yuzhichang)) - For materialized views the commit for Kafka is called after all data were written. - [\#7175](https://github.com/ClickHouse/ClickHouse/pull/7175) ([Ivan](https://github.com/abyss7)) + [#7175](https://github.com/ClickHouse/ClickHouse/pull/7175) ([Ivan](https://github.com/abyss7)) - Fixed wrong `duration_ms` value in `system.part_log` table. It was ten times off. - [\#7172](https://github.com/ClickHouse/ClickHouse/pull/7172) ([Vladimir + [#7172](https://github.com/ClickHouse/ClickHouse/pull/7172) ([Vladimir Chebotarev](https://github.com/excitoon)) - A quick fix to resolve crash in LIVE VIEW table and re-enabling all LIVE VIEW tests. - [\#7201](https://github.com/ClickHouse/ClickHouse/pull/7201) + [#7201](https://github.com/ClickHouse/ClickHouse/pull/7201) ([vzakaznikov](https://github.com/vzakaznikov)) - Serialize NULL values correctly in min/max indexes of MergeTree parts. - [\#7234](https://github.com/ClickHouse/ClickHouse/pull/7234) ([Alexander + [#7234](https://github.com/ClickHouse/ClickHouse/pull/7234) ([Alexander Kuzmenkov](https://github.com/akuzm)) - Don’t put virtual columns to .sql metadata when table is created as `CREATE TABLE AS`. - [\#7183](https://github.com/ClickHouse/ClickHouse/pull/7183) ([Ivan](https://github.com/abyss7)) + [#7183](https://github.com/ClickHouse/ClickHouse/pull/7183) ([Ivan](https://github.com/abyss7)) - Fix segmentation fault in `ATTACH PART` query. - [\#7185](https://github.com/ClickHouse/ClickHouse/pull/7185) + [#7185](https://github.com/ClickHouse/ClickHouse/pull/7185) ([alesapin](https://github.com/alesapin)) - Fix wrong result for some queries given by the optimization of empty IN subqueries and empty - INNER/RIGHT JOIN. [\#7284](https://github.com/ClickHouse/ClickHouse/pull/7284) ([Nikolai + INNER/RIGHT JOIN. [#7284](https://github.com/ClickHouse/ClickHouse/pull/7284) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) - Fixing AddressSanitizer error in the LIVE VIEW getHeader() method. - [\#7271](https://github.com/ClickHouse/ClickHouse/pull/7271) + [#7271](https://github.com/ClickHouse/ClickHouse/pull/7271) ([vzakaznikov](https://github.com/vzakaznikov)) #### Improvement {#improvement-1} -- Add a message in case of queue\_wait\_max\_ms wait takes place. - [\#7390](https://github.com/ClickHouse/ClickHouse/pull/7390) ([Azat +- Add a message in case of queue_wait_max_ms wait takes place. + [#7390](https://github.com/ClickHouse/ClickHouse/pull/7390) ([Azat Khuzhin](https://github.com/azat)) - Made setting `s3_min_upload_part_size` table-level. - [\#7059](https://github.com/ClickHouse/ClickHouse/pull/7059) ([Vladimir + [#7059](https://github.com/ClickHouse/ClickHouse/pull/7059) ([Vladimir Chebotarev](https://github.com/excitoon)) -- Check TTL in StorageFactory. [\#7304](https://github.com/ClickHouse/ClickHouse/pull/7304) +- Check TTL in StorageFactory. [#7304](https://github.com/ClickHouse/ClickHouse/pull/7304) ([sundyli](https://github.com/sundy-li)) - Squash left-hand blocks in partial merge join (optimization). - [\#7122](https://github.com/ClickHouse/ClickHouse/pull/7122) ([Artem + [#7122](https://github.com/ClickHouse/ClickHouse/pull/7122) ([Artem Zuikov](https://github.com/4ertus2)) - Do not allow non-deterministic functions in mutations of Replicated table engines, because this can introduce inconsistencies between replicas. - [\#7247](https://github.com/ClickHouse/ClickHouse/pull/7247) ([Alexander + [#7247](https://github.com/ClickHouse/ClickHouse/pull/7247) ([Alexander Kazakov](https://github.com/Akazz)) - Disable memory tracker while converting exception stack trace to string. It can prevent the loss - of error messages of type `Memory limit exceeded` on server, which caused the `Attempt to read after eof` exception on client. [\#7264](https://github.com/ClickHouse/ClickHouse/pull/7264) + of error messages of type `Memory limit exceeded` on server, which caused the `Attempt to read after eof` exception on client. [#7264](https://github.com/ClickHouse/ClickHouse/pull/7264) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) - Miscellaneous format improvements. Resolves - [\#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) + [#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 ignores values on the right side of IN operator that are not convertible to the left side type. Make it work properly for compound types – Array and Tuple. - [\#7283](https://github.com/ClickHouse/ClickHouse/pull/7283) ([Alexander + [#7283](https://github.com/ClickHouse/ClickHouse/pull/7283) ([Alexander Kuzmenkov](https://github.com/akuzm)) - Support missing inequalities for ASOF JOIN. It’s possible to join less-or-equal variant and strict greater and less variants for ASOF column in ON syntax. - [\#7282](https://github.com/ClickHouse/ClickHouse/pull/7282) ([Artem + [#7282](https://github.com/ClickHouse/ClickHouse/pull/7282) ([Artem Zuikov](https://github.com/4ertus2)) -- Optimize partial merge join. [\#7070](https://github.com/ClickHouse/ClickHouse/pull/7070) +- Optimize partial merge join. [#7070](https://github.com/ClickHouse/ClickHouse/pull/7070) ([Artem Zuikov](https://github.com/4ertus2)) - Do not use more than 98K of memory in uniqCombined functions. - [\#7236](https://github.com/ClickHouse/ClickHouse/pull/7236), - [\#7270](https://github.com/ClickHouse/ClickHouse/pull/7270) ([Azat + [#7236](https://github.com/ClickHouse/ClickHouse/pull/7236), + [#7270](https://github.com/ClickHouse/ClickHouse/pull/7270) ([Azat Khuzhin](https://github.com/azat)) - Flush parts of right-hand joining table on disk in PartialMergeJoin (if there is not enough - memory). Load data back when needed. [\#7186](https://github.com/ClickHouse/ClickHouse/pull/7186) + memory). Load data back when needed. [#7186](https://github.com/ClickHouse/ClickHouse/pull/7186) ([Artem Zuikov](https://github.com/4ertus2)) #### Performance Improvement {#performance-improvement-1} - Speed up joinGet with const arguments by avoiding data duplication. - [\#7359](https://github.com/ClickHouse/ClickHouse/pull/7359) ([Amos + [#7359](https://github.com/ClickHouse/ClickHouse/pull/7359) ([Amos Bird](https://github.com/amosbird)) - Return early if the subquery is empty. - [\#7007](https://github.com/ClickHouse/ClickHouse/pull/7007) ([小路](https://github.com/nicelulu)) + [#7007](https://github.com/ClickHouse/ClickHouse/pull/7007) ([小路](https://github.com/nicelulu)) - Optimize parsing of SQL expression in Values. - [\#6781](https://github.com/ClickHouse/ClickHouse/pull/6781) + [#6781](https://github.com/ClickHouse/ClickHouse/pull/6781) ([tavplubix](https://github.com/tavplubix)) #### Build/Testing/Packaging Improvement {#buildtestingpackaging-improvement-1} - Disable some contribs for cross-compilation to Mac OS. - [\#7101](https://github.com/ClickHouse/ClickHouse/pull/7101) ([Ivan](https://github.com/abyss7)) -- Add missing linking with PocoXML for clickhouse\_common\_io. - [\#7200](https://github.com/ClickHouse/ClickHouse/pull/7200) ([Azat + [#7101](https://github.com/ClickHouse/ClickHouse/pull/7101) ([Ivan](https://github.com/abyss7)) +- Add missing linking with PocoXML for clickhouse_common_io. + [#7200](https://github.com/ClickHouse/ClickHouse/pull/7200) ([Azat Khuzhin](https://github.com/azat)) - Accept multiple test filter arguments in clickhouse-test. - [\#7226](https://github.com/ClickHouse/ClickHouse/pull/7226) ([Alexander + [#7226](https://github.com/ClickHouse/ClickHouse/pull/7226) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- Enable musl and jemalloc for ARM. [\#7300](https://github.com/ClickHouse/ClickHouse/pull/7300) +- Enable musl and jemalloc for ARM. [#7300](https://github.com/ClickHouse/ClickHouse/pull/7300) ([Amos Bird](https://github.com/amosbird)) - Added `--client-option` parameter to `clickhouse-test` to pass additional parameters to client. - [\#7277](https://github.com/ClickHouse/ClickHouse/pull/7277) ([Nikolai + [#7277](https://github.com/ClickHouse/ClickHouse/pull/7277) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) - Preserve existing configs on rpm package upgrade. - [\#7103](https://github.com/ClickHouse/ClickHouse/pull/7103) + [#7103](https://github.com/ClickHouse/ClickHouse/pull/7103) ([filimonov](https://github.com/filimonov)) -- Fix errors detected by PVS. [\#7153](https://github.com/ClickHouse/ClickHouse/pull/7153) ([Artem +- Fix errors detected by PVS. [#7153](https://github.com/ClickHouse/ClickHouse/pull/7153) ([Artem Zuikov](https://github.com/4ertus2)) -- Fix build for Darwin. [\#7149](https://github.com/ClickHouse/ClickHouse/pull/7149) +- Fix build for Darwin. [#7149](https://github.com/ClickHouse/ClickHouse/pull/7149) ([Ivan](https://github.com/abyss7)) -- glibc 2.29 compatibility. [\#7142](https://github.com/ClickHouse/ClickHouse/pull/7142) ([Amos +- glibc 2.29 compatibility. [#7142](https://github.com/ClickHouse/ClickHouse/pull/7142) ([Amos Bird](https://github.com/amosbird)) -- Make sure dh\_clean does not touch potential source files. - [\#7205](https://github.com/ClickHouse/ClickHouse/pull/7205) ([Amos +- Make sure dh_clean does not touch potential source files. + [#7205](https://github.com/ClickHouse/ClickHouse/pull/7205) ([Amos Bird](https://github.com/amosbird)) - Attempt to avoid conflict when updating from altinity rpm - it has config file packaged separately - in clickhouse-server-common. [\#7073](https://github.com/ClickHouse/ClickHouse/pull/7073) + in clickhouse-server-common. [#7073](https://github.com/ClickHouse/ClickHouse/pull/7073) ([filimonov](https://github.com/filimonov)) - Optimize some header files for faster rebuilds. - [\#7212](https://github.com/ClickHouse/ClickHouse/pull/7212), - [\#7231](https://github.com/ClickHouse/ClickHouse/pull/7231) ([Alexander + [#7212](https://github.com/ClickHouse/ClickHouse/pull/7212), + [#7231](https://github.com/ClickHouse/ClickHouse/pull/7231) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- Add performance tests for Date and DateTime. [\#7332](https://github.com/ClickHouse/ClickHouse/pull/7332) ([Vasily +- Add performance tests for Date and DateTime. [#7332](https://github.com/ClickHouse/ClickHouse/pull/7332) ([Vasily Nemkov](https://github.com/Enmk)) - Fix some tests that contained non-deterministic mutations. - [\#7132](https://github.com/ClickHouse/ClickHouse/pull/7132) ([Alexander + [#7132](https://github.com/ClickHouse/ClickHouse/pull/7132) ([Alexander Kazakov](https://github.com/Akazz)) -- Add build with MemorySanitizer to CI. [\#7066](https://github.com/ClickHouse/ClickHouse/pull/7066) +- Add build with MemorySanitizer to CI. [#7066](https://github.com/ClickHouse/ClickHouse/pull/7066) ([Alexander Kuzmenkov](https://github.com/akuzm)) - Avoid use of uninitialized values in MetricsTransmitter. - [\#7158](https://github.com/ClickHouse/ClickHouse/pull/7158) ([Azat + [#7158](https://github.com/ClickHouse/ClickHouse/pull/7158) ([Azat Khuzhin](https://github.com/azat)) - Fix some issues in Fields found by MemorySanitizer. - [\#7135](https://github.com/ClickHouse/ClickHouse/pull/7135), - [\#7179](https://github.com/ClickHouse/ClickHouse/pull/7179) ([Alexander - Kuzmenkov](https://github.com/akuzm)), [\#7376](https://github.com/ClickHouse/ClickHouse/pull/7376) + [#7135](https://github.com/ClickHouse/ClickHouse/pull/7135), + [#7179](https://github.com/ClickHouse/ClickHouse/pull/7179) ([Alexander + Kuzmenkov](https://github.com/akuzm)), [#7376](https://github.com/ClickHouse/ClickHouse/pull/7376) ([Amos Bird](https://github.com/amosbird)) -- Fix undefined behavior in murmurhash32. [\#7388](https://github.com/ClickHouse/ClickHouse/pull/7388) ([Amos +- Fix undefined behavior in murmurhash32. [#7388](https://github.com/ClickHouse/ClickHouse/pull/7388) ([Amos Bird](https://github.com/amosbird)) -- Fix undefined behavior in StoragesInfoStream. [\#7384](https://github.com/ClickHouse/ClickHouse/pull/7384) +- Fix undefined behavior in StoragesInfoStream. [#7384](https://github.com/ClickHouse/ClickHouse/pull/7384) ([tavplubix](https://github.com/tavplubix)) - Fixed constant expressions folding for external database engines (MySQL, ODBC, JDBC). In previous versions it wasn’t working for multiple constant expressions and was not working at all for Date, - DateTime and UUID. This fixes [\#7245](https://github.com/ClickHouse/ClickHouse/issues/7245) - [\#7252](https://github.com/ClickHouse/ClickHouse/pull/7252) + DateTime and UUID. This fixes [#7245](https://github.com/ClickHouse/ClickHouse/issues/7245) + [#7252](https://github.com/ClickHouse/ClickHouse/pull/7252) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixing ThreadSanitizer data race error in the LIVE VIEW when accessing no\_users\_thread variable. - [\#7353](https://github.com/ClickHouse/ClickHouse/pull/7353) +- Fixing ThreadSanitizer data race error in the LIVE VIEW when accessing no_users_thread variable. + [#7353](https://github.com/ClickHouse/ClickHouse/pull/7353) ([vzakaznikov](https://github.com/vzakaznikov)) - Get rid of malloc symbols in libcommon - [\#7134](https://github.com/ClickHouse/ClickHouse/pull/7134), - [\#7065](https://github.com/ClickHouse/ClickHouse/pull/7065) ([Amos + [#7134](https://github.com/ClickHouse/ClickHouse/pull/7134), + [#7065](https://github.com/ClickHouse/ClickHouse/pull/7065) ([Amos Bird](https://github.com/amosbird)) -- Add global flag ENABLE\_LIBRARIES for disabling all libraries. - [\#7063](https://github.com/ClickHouse/ClickHouse/pull/7063) +- Add global flag ENABLE_LIBRARIES for disabling all libraries. + [#7063](https://github.com/ClickHouse/ClickHouse/pull/7063) ([proller](https://github.com/proller)) #### Code Cleanup {#code-cleanup} -- Generalize configuration repository to prepare for DDL for Dictionaries. [\#7155](https://github.com/ClickHouse/ClickHouse/pull/7155) +- Generalize configuration repository to prepare for DDL for Dictionaries. [#7155](https://github.com/ClickHouse/ClickHouse/pull/7155) ([alesapin](https://github.com/alesapin)) - Parser for dictionaries DDL without any semantic. - [\#7209](https://github.com/ClickHouse/ClickHouse/pull/7209) + [#7209](https://github.com/ClickHouse/ClickHouse/pull/7209) ([alesapin](https://github.com/alesapin)) - Split ParserCreateQuery into different smaller parsers. - [\#7253](https://github.com/ClickHouse/ClickHouse/pull/7253) + [#7253](https://github.com/ClickHouse/ClickHouse/pull/7253) ([alesapin](https://github.com/alesapin)) - Small refactoring and renaming near external dictionaries. - [\#7111](https://github.com/ClickHouse/ClickHouse/pull/7111) + [#7111](https://github.com/ClickHouse/ClickHouse/pull/7111) ([alesapin](https://github.com/alesapin)) -- Refactor some code to prepare for role-based access control. [\#7235](https://github.com/ClickHouse/ClickHouse/pull/7235) ([Vitaly +- Refactor some code to prepare for role-based access control. [#7235](https://github.com/ClickHouse/ClickHouse/pull/7235) ([Vitaly Baranov](https://github.com/vitlibar)) - Some improvements in DatabaseOrdinary code. - [\#7086](https://github.com/ClickHouse/ClickHouse/pull/7086) ([Nikita + [#7086](https://github.com/ClickHouse/ClickHouse/pull/7086) ([Nikita Vasilev](https://github.com/nikvas0)) - Do not use iterators in find() and emplace() methods of hash tables. - [\#7026](https://github.com/ClickHouse/ClickHouse/pull/7026) ([Alexander + [#7026](https://github.com/ClickHouse/ClickHouse/pull/7026) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- Fix getMultipleValuesFromConfig in case when parameter root is not empty. [\#7374](https://github.com/ClickHouse/ClickHouse/pull/7374) +- Fix getMultipleValuesFromConfig in case when parameter root is not empty. [#7374](https://github.com/ClickHouse/ClickHouse/pull/7374) ([Mikhail Korotov](https://github.com/millb)) - Remove some copy-paste (TemporaryFile and TemporaryFileStream) - [\#7166](https://github.com/ClickHouse/ClickHouse/pull/7166) ([Artem + [#7166](https://github.com/ClickHouse/ClickHouse/pull/7166) ([Artem Zuikov](https://github.com/4ertus2)) - Improved code readability a little bit (`MergeTreeData::getActiveContainingPart`). - [\#7361](https://github.com/ClickHouse/ClickHouse/pull/7361) ([Vladimir + [#7361](https://github.com/ClickHouse/ClickHouse/pull/7361) ([Vladimir Chebotarev](https://github.com/excitoon)) - Wait for all scheduled jobs, which are using local objects, if `ThreadPool::schedule(...)` throws an exception. Rename `ThreadPool::schedule(...)` to `ThreadPool::scheduleOrThrowOnError(...)` and fix comments to make obvious that it may throw. - [\#7350](https://github.com/ClickHouse/ClickHouse/pull/7350) + [#7350](https://github.com/ClickHouse/ClickHouse/pull/7350) ([tavplubix](https://github.com/tavplubix)) ## ClickHouse Release 19.15 {#clickhouse-release-19-15} @@ -415,99 +415,99 @@ toc_title: '2019' #### Bug Fix {#bug-fix-3} -- Added handling of SQL\_TINYINT and SQL\_BIGINT, and fix handling of SQL\_FLOAT data source types in ODBC Bridge. - [\#7491](https://github.com/ClickHouse/ClickHouse/pull/7491) ([Denis Glazachev](https://github.com/traceon)) +- Added handling of SQL_TINYINT and SQL_BIGINT, and fix handling of SQL_FLOAT data source types in ODBC Bridge. + [#7491](https://github.com/ClickHouse/ClickHouse/pull/7491) ([Denis Glazachev](https://github.com/traceon)) - Allowed to have some parts on destination disk or volume in MOVE PARTITION. - [\#7434](https://github.com/ClickHouse/ClickHouse/pull/7434) ([Vladimir Chebotarev](https://github.com/excitoon)) + [#7434](https://github.com/ClickHouse/ClickHouse/pull/7434) ([Vladimir Chebotarev](https://github.com/excitoon)) - Fixed NULL-values in nullable columns through ODBC-bridge. - [\#7402](https://github.com/ClickHouse/ClickHouse/pull/7402) ([Vasily Nemkov](https://github.com/Enmk)) + [#7402](https://github.com/ClickHouse/ClickHouse/pull/7402) ([Vasily Nemkov](https://github.com/Enmk)) - Fixed INSERT into Distributed non local node with MATERIALIZED columns. - [\#7377](https://github.com/ClickHouse/ClickHouse/pull/7377) ([Azat Khuzhin](https://github.com/azat)) + [#7377](https://github.com/ClickHouse/ClickHouse/pull/7377) ([Azat Khuzhin](https://github.com/azat)) - Fixed function getMultipleValuesFromConfig. - [\#7374](https://github.com/ClickHouse/ClickHouse/pull/7374) ([Mikhail Korotov](https://github.com/millb)) + [#7374](https://github.com/ClickHouse/ClickHouse/pull/7374) ([Mikhail Korotov](https://github.com/millb)) - Fixed issue of using HTTP keep alive timeout instead of TCP keep alive timeout. - [\#7351](https://github.com/ClickHouse/ClickHouse/pull/7351) ([Vasily Nemkov](https://github.com/Enmk)) + [#7351](https://github.com/ClickHouse/ClickHouse/pull/7351) ([Vasily Nemkov](https://github.com/Enmk)) - Wait for all jobs to finish on exception (fixes rare segfaults). - [\#7350](https://github.com/ClickHouse/ClickHouse/pull/7350) ([tavplubix](https://github.com/tavplubix)) + [#7350](https://github.com/ClickHouse/ClickHouse/pull/7350) ([tavplubix](https://github.com/tavplubix)) - Don’t push to MVs when inserting into Kafka table. - [\#7265](https://github.com/ClickHouse/ClickHouse/pull/7265) ([Ivan](https://github.com/abyss7)) + [#7265](https://github.com/ClickHouse/ClickHouse/pull/7265) ([Ivan](https://github.com/abyss7)) - Disable memory tracker for exception stack. - [\#7264](https://github.com/ClickHouse/ClickHouse/pull/7264) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) + [#7264](https://github.com/ClickHouse/ClickHouse/pull/7264) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) - Fixed bad code in transforming query for external database. - [\#7252](https://github.com/ClickHouse/ClickHouse/pull/7252) ([alexey-milovidov](https://github.com/alexey-milovidov)) + [#7252](https://github.com/ClickHouse/ClickHouse/pull/7252) ([alexey-milovidov](https://github.com/alexey-milovidov)) - Avoid use of uninitialized values in MetricsTransmitter. - [\#7158](https://github.com/ClickHouse/ClickHouse/pull/7158) ([Azat Khuzhin](https://github.com/azat)) + [#7158](https://github.com/ClickHouse/ClickHouse/pull/7158) ([Azat Khuzhin](https://github.com/azat)) - Added example config with macros for tests ([alexey-milovidov](https://github.com/alexey-milovidov)) ### ClickHouse Release 19.15.3.6, 2019-10-09 {#clickhouse-release-19-15-3-6-2019-10-09} #### Bug Fix {#bug-fix-4} -- Fixed bad\_variant in hashed dictionary. +- Fixed bad_variant in hashed dictionary. ([alesapin](https://github.com/alesapin)) - Fixed up bug with segmentation fault in ATTACH PART query. ([alesapin](https://github.com/alesapin)) - Fixed time calculation in `MergeTreeData`. ([Vladimir Chebotarev](https://github.com/excitoon)) - Commit to Kafka explicitly after the writing is finalized. - [\#7175](https://github.com/ClickHouse/ClickHouse/pull/7175) ([Ivan](https://github.com/abyss7)) + [#7175](https://github.com/ClickHouse/ClickHouse/pull/7175) ([Ivan](https://github.com/abyss7)) - Serialize NULL values correctly in min/max indexes of MergeTree parts. - [\#7234](https://github.com/ClickHouse/ClickHouse/pull/7234) ([Alexander Kuzmenkov](https://github.com/akuzm)) + [#7234](https://github.com/ClickHouse/ClickHouse/pull/7234) ([Alexander Kuzmenkov](https://github.com/akuzm)) ### ClickHouse Release 19.15.2.2, 2019-10-01 {#clickhouse-release-19-15-2-2-2019-10-01} #### New Feature {#new-feature-3} -- Tiered storage: support to use multiple storage volumes for tables with MergeTree engine. It’s possible to store fresh data on SSD and automatically move old data to HDD. ([example](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)) -- Add table function `input` for reading incoming data in `INSERT SELECT` query. [\#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)) -- Add a `sparse_hashed` dictionary layout, that is functionally equivalent to the `hashed` layout, but is more memory efficient. It uses about twice as less memory at the cost of slower value retrieval. [\#6894](https://github.com/ClickHouse/ClickHouse/pull/6894) ([Azat Khuzhin](https://github.com/azat)) -- Implement ability to define list of users for access to dictionaries. Only current connected database using. [\#6907](https://github.com/ClickHouse/ClickHouse/pull/6907) ([Guillaume Tassery](https://github.com/YiuRULE)) -- Add `LIMIT` option to `SHOW` query. [\#6944](https://github.com/ClickHouse/ClickHouse/pull/6944) ([Philipp Malkovsky](https://github.com/malkfilipp)) -- Add `bitmapSubsetLimit(bitmap, range_start, limit)` function, that returns subset of the smallest `limit` values in set that is no smaller than `range_start`. [\#6957](https://github.com/ClickHouse/ClickHouse/pull/6957) ([Zhichang Yu](https://github.com/yuzhichang)) -- Add `bitmapMin` and `bitmapMax` functions. [\#6970](https://github.com/ClickHouse/ClickHouse/pull/6970) ([Zhichang Yu](https://github.com/yuzhichang)) -- Add function `repeat` related to [issue-6648](https://github.com/ClickHouse/ClickHouse/issues/6648) [\#6999](https://github.com/ClickHouse/ClickHouse/pull/6999) ([flynn](https://github.com/ucasFL)) +- Tiered storage: support to use multiple storage volumes for tables with MergeTree engine. It’s possible to store fresh data on SSD and automatically move old data to HDD. ([example](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)) +- Add table function `input` for reading incoming data in `INSERT SELECT` query. [#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)) +- Add a `sparse_hashed` dictionary layout, that is functionally equivalent to the `hashed` layout, but is more memory efficient. It uses about twice as less memory at the cost of slower value retrieval. [#6894](https://github.com/ClickHouse/ClickHouse/pull/6894) ([Azat Khuzhin](https://github.com/azat)) +- Implement ability to define list of users for access to dictionaries. Only current connected database using. [#6907](https://github.com/ClickHouse/ClickHouse/pull/6907) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Add `LIMIT` option to `SHOW` query. [#6944](https://github.com/ClickHouse/ClickHouse/pull/6944) ([Philipp Malkovsky](https://github.com/malkfilipp)) +- Add `bitmapSubsetLimit(bitmap, range_start, limit)` function, that returns subset of the smallest `limit` values in set that is no smaller than `range_start`. [#6957](https://github.com/ClickHouse/ClickHouse/pull/6957) ([Zhichang Yu](https://github.com/yuzhichang)) +- Add `bitmapMin` and `bitmapMax` functions. [#6970](https://github.com/ClickHouse/ClickHouse/pull/6970) ([Zhichang Yu](https://github.com/yuzhichang)) +- Add function `repeat` related to [issue-6648](https://github.com/ClickHouse/ClickHouse/issues/6648) [#6999](https://github.com/ClickHouse/ClickHouse/pull/6999) ([flynn](https://github.com/ucasFL)) #### Experimental Feature {#experimental-feature-1} -- Implement (in memory) Merge Join variant that does not change current pipeline. Result is partially sorted by merge key. Set `partial_merge_join = 1` to use this feature. The Merge Join is still in development. [\#6940](https://github.com/ClickHouse/ClickHouse/pull/6940) ([Artem Zuikov](https://github.com/4ertus2)) -- Add `S3` engine and table function. It is still in development (no authentication support yet). [\#5596](https://github.com/ClickHouse/ClickHouse/pull/5596) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Implement (in memory) Merge Join variant that does not change current pipeline. Result is partially sorted by merge key. Set `partial_merge_join = 1` to use this feature. The Merge Join is still in development. [#6940](https://github.com/ClickHouse/ClickHouse/pull/6940) ([Artem Zuikov](https://github.com/4ertus2)) +- Add `S3` engine and table function. It is still in development (no authentication support yet). [#5596](https://github.com/ClickHouse/ClickHouse/pull/5596) ([Vladimir Chebotarev](https://github.com/excitoon)) #### Improvement {#improvement-2} -- Every message read from Kafka is inserted atomically. This resolves almost all known issues with Kafka engine. [\#6950](https://github.com/ClickHouse/ClickHouse/pull/6950) ([Ivan](https://github.com/abyss7)) -- Improvements for failover of Distributed queries. Shorten recovery time, also it is now configurable and can be seen in `system.clusters`. [\#6399](https://github.com/ClickHouse/ClickHouse/pull/6399) ([Vasily Nemkov](https://github.com/Enmk)) -- Support numeric values for Enums directly in `IN` section. \#6766 [\#6941](https://github.com/ClickHouse/ClickHouse/pull/6941) ([dimarub2000](https://github.com/dimarub2000)) -- Support (optional, disabled by default) redirects on URL storage. [\#6914](https://github.com/ClickHouse/ClickHouse/pull/6914) ([maqroll](https://github.com/maqroll)) -- Add information message when client with an older version connects to a server. [\#6893](https://github.com/ClickHouse/ClickHouse/pull/6893) ([Philipp Malkovsky](https://github.com/malkfilipp)) -- Remove maximum backoff sleep time limit for sending data in Distributed tables [\#6895](https://github.com/ClickHouse/ClickHouse/pull/6895) ([Azat Khuzhin](https://github.com/azat)) -- Add ability to send profile events (counters) with cumulative values to graphite. It can be enabled under `` in server `config.xml`. [\#6969](https://github.com/ClickHouse/ClickHouse/pull/6969) ([Azat Khuzhin](https://github.com/azat)) -- Add automatically cast type `T` to `LowCardinality(T)` while inserting data in column of type `LowCardinality(T)` in Native format via HTTP. [\#6891](https://github.com/ClickHouse/ClickHouse/pull/6891) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Add ability to use function `hex` without using `reinterpretAsString` for `Float32`, `Float64`. [\#7024](https://github.com/ClickHouse/ClickHouse/pull/7024) ([Mikhail Korotov](https://github.com/millb)) +- Every message read from Kafka is inserted atomically. This resolves almost all known issues with Kafka engine. [#6950](https://github.com/ClickHouse/ClickHouse/pull/6950) ([Ivan](https://github.com/abyss7)) +- Improvements for failover of Distributed queries. Shorten recovery time, also it is now configurable and can be seen in `system.clusters`. [#6399](https://github.com/ClickHouse/ClickHouse/pull/6399) ([Vasily Nemkov](https://github.com/Enmk)) +- Support numeric values for Enums directly in `IN` section. #6766 [#6941](https://github.com/ClickHouse/ClickHouse/pull/6941) ([dimarub2000](https://github.com/dimarub2000)) +- Support (optional, disabled by default) redirects on URL storage. [#6914](https://github.com/ClickHouse/ClickHouse/pull/6914) ([maqroll](https://github.com/maqroll)) +- Add information message when client with an older version connects to a server. [#6893](https://github.com/ClickHouse/ClickHouse/pull/6893) ([Philipp Malkovsky](https://github.com/malkfilipp)) +- Remove maximum backoff sleep time limit for sending data in Distributed tables [#6895](https://github.com/ClickHouse/ClickHouse/pull/6895) ([Azat Khuzhin](https://github.com/azat)) +- Add ability to send profile events (counters) with cumulative values to graphite. It can be enabled under `` in server `config.xml`. [#6969](https://github.com/ClickHouse/ClickHouse/pull/6969) ([Azat Khuzhin](https://github.com/azat)) +- Add automatically cast type `T` to `LowCardinality(T)` while inserting data in column of type `LowCardinality(T)` in Native format via HTTP. [#6891](https://github.com/ClickHouse/ClickHouse/pull/6891) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Add ability to use function `hex` without using `reinterpretAsString` for `Float32`, `Float64`. [#7024](https://github.com/ClickHouse/ClickHouse/pull/7024) ([Mikhail Korotov](https://github.com/millb)) #### Build/Testing/Packaging Improvement {#buildtestingpackaging-improvement-2} -- Add gdb-index to clickhouse binary with debug info. It will speed up startup time of `gdb`. [\#6947](https://github.com/ClickHouse/ClickHouse/pull/6947) ([alesapin](https://github.com/alesapin)) -- Speed up deb packaging with patched dpkg-deb which uses `pigz`. [\#6960](https://github.com/ClickHouse/ClickHouse/pull/6960) ([alesapin](https://github.com/alesapin)) -- Set `enable_fuzzing = 1` to enable libfuzzer instrumentation of all the project code. [\#7042](https://github.com/ClickHouse/ClickHouse/pull/7042) ([kyprizel](https://github.com/kyprizel)) -- Add split build smoke test in CI. [\#7061](https://github.com/ClickHouse/ClickHouse/pull/7061) ([alesapin](https://github.com/alesapin)) -- Add build with MemorySanitizer to CI. [\#7066](https://github.com/ClickHouse/ClickHouse/pull/7066) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- Replace `libsparsehash` with `sparsehash-c11` [\#6965](https://github.com/ClickHouse/ClickHouse/pull/6965) ([Azat Khuzhin](https://github.com/azat)) +- Add gdb-index to clickhouse binary with debug info. It will speed up startup time of `gdb`. [#6947](https://github.com/ClickHouse/ClickHouse/pull/6947) ([alesapin](https://github.com/alesapin)) +- Speed up deb packaging with patched dpkg-deb which uses `pigz`. [#6960](https://github.com/ClickHouse/ClickHouse/pull/6960) ([alesapin](https://github.com/alesapin)) +- Set `enable_fuzzing = 1` to enable libfuzzer instrumentation of all the project code. [#7042](https://github.com/ClickHouse/ClickHouse/pull/7042) ([kyprizel](https://github.com/kyprizel)) +- Add split build smoke test in CI. [#7061](https://github.com/ClickHouse/ClickHouse/pull/7061) ([alesapin](https://github.com/alesapin)) +- Add build with MemorySanitizer to CI. [#7066](https://github.com/ClickHouse/ClickHouse/pull/7066) ([Alexander Kuzmenkov](https://github.com/akuzm)) +- Replace `libsparsehash` with `sparsehash-c11` [#6965](https://github.com/ClickHouse/ClickHouse/pull/6965) ([Azat Khuzhin](https://github.com/azat)) #### Bug Fix {#bug-fix-5} -- Fixed performance degradation of index analysis on complex keys on large tables. This fixes \#6924. [\#7075](https://github.com/ClickHouse/ClickHouse/pull/7075) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix logical error causing segfaults when selecting from Kafka empty topic. [\#6909](https://github.com/ClickHouse/ClickHouse/pull/6909) ([Ivan](https://github.com/abyss7)) -- Fix too early MySQL connection close in `MySQLBlockInputStream.cpp`. [\#6882](https://github.com/ClickHouse/ClickHouse/pull/6882) ([Clément Rodriguez](https://github.com/clemrodriguez)) -- Returned support for very old Linux kernels (fix [\#6841](https://github.com/ClickHouse/ClickHouse/issues/6841)) [\#6853](https://github.com/ClickHouse/ClickHouse/pull/6853) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix possible data loss in `insert select` query in case of empty block in input stream. \#6834 \#6862 [\#6911](https://github.com/ClickHouse/ClickHouse/pull/6911) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Fix for function `АrrayEnumerateUniqRanked` with empty arrays in params [\#6928](https://github.com/ClickHouse/ClickHouse/pull/6928) ([proller](https://github.com/proller)) -- Fix complex queries with array joins and global subqueries. [\#6934](https://github.com/ClickHouse/ClickHouse/pull/6934) ([Ivan](https://github.com/abyss7)) -- Fix `Unknown identifier` error in ORDER BY and GROUP BY with multiple JOINs [\#7022](https://github.com/ClickHouse/ClickHouse/pull/7022) ([Artem Zuikov](https://github.com/4ertus2)) -- Fixed `MSan` warning while executing function with `LowCardinality` argument. [\#7062](https://github.com/ClickHouse/ClickHouse/pull/7062) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fixed performance degradation of index analysis on complex keys on large tables. This fixes #6924. [#7075](https://github.com/ClickHouse/ClickHouse/pull/7075) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix logical error causing segfaults when selecting from Kafka empty topic. [#6909](https://github.com/ClickHouse/ClickHouse/pull/6909) ([Ivan](https://github.com/abyss7)) +- Fix too early MySQL connection close in `MySQLBlockInputStream.cpp`. [#6882](https://github.com/ClickHouse/ClickHouse/pull/6882) ([Clément Rodriguez](https://github.com/clemrodriguez)) +- Returned support for very old Linux kernels (fix [#6841](https://github.com/ClickHouse/ClickHouse/issues/6841)) [#6853](https://github.com/ClickHouse/ClickHouse/pull/6853) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix possible data loss in `insert select` query in case of empty block in input stream. #6834 #6862 [#6911](https://github.com/ClickHouse/ClickHouse/pull/6911) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fix for function `АrrayEnumerateUniqRanked` with empty arrays in params [#6928](https://github.com/ClickHouse/ClickHouse/pull/6928) ([proller](https://github.com/proller)) +- Fix complex queries with array joins and global subqueries. [#6934](https://github.com/ClickHouse/ClickHouse/pull/6934) ([Ivan](https://github.com/abyss7)) +- Fix `Unknown identifier` error in ORDER BY and GROUP BY with multiple JOINs [#7022](https://github.com/ClickHouse/ClickHouse/pull/7022) ([Artem Zuikov](https://github.com/4ertus2)) +- Fixed `MSan` warning while executing function with `LowCardinality` argument. [#7062](https://github.com/ClickHouse/ClickHouse/pull/7062) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) #### Backward Incompatible Change {#backward-incompatible-change-2} -- Changed serialization format of bitmap\* aggregate function states to improve performance. Serialized states of bitmap\* from previous versions cannot be read. [\#6908](https://github.com/ClickHouse/ClickHouse/pull/6908) ([Zhichang Yu](https://github.com/yuzhichang)) +- Changed serialization format of bitmap\* aggregate function states to improve performance. Serialized states of bitmap\* from previous versions cannot be read. [#6908](https://github.com/ClickHouse/ClickHouse/pull/6908) ([Zhichang Yu](https://github.com/yuzhichang)) ## ClickHouse Release 19.14 {#clickhouse-release-19-14} @@ -516,273 +516,273 @@ toc_title: '2019' #### Bug Fix {#bug-fix-6} - This release also contains all bug fixes from 19.11.12.69. -- Fixed compatibility for distributed queries between 19.14 and earlier versions. This fixes [\#7068](https://github.com/ClickHouse/ClickHouse/issues/7068). [\#7069](https://github.com/ClickHouse/ClickHouse/pull/7069) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed compatibility for distributed queries between 19.14 and earlier versions. This fixes [#7068](https://github.com/ClickHouse/ClickHouse/issues/7068). [#7069](https://github.com/ClickHouse/ClickHouse/pull/7069) ([alexey-milovidov](https://github.com/alexey-milovidov)) ### ClickHouse Release 19.14.6.12, 2019-09-19 {#clickhouse-release-19-14-6-12-2019-09-19} #### Bug Fix {#bug-fix-7} -- Fix for function `АrrayEnumerateUniqRanked` with empty arrays in params. [\#6928](https://github.com/ClickHouse/ClickHouse/pull/6928) ([proller](https://github.com/proller)) -- Fixed subquery name in queries with `ARRAY JOIN` and `GLOBAL IN subquery` with alias. Use subquery alias for external table name if it is specified. [\#6934](https://github.com/ClickHouse/ClickHouse/pull/6934) ([Ivan](https://github.com/abyss7)) +- Fix for function `АrrayEnumerateUniqRanked` with empty arrays in params. [#6928](https://github.com/ClickHouse/ClickHouse/pull/6928) ([proller](https://github.com/proller)) +- Fixed subquery name in queries with `ARRAY JOIN` and `GLOBAL IN subquery` with alias. Use subquery alias for external table name if it is specified. [#6934](https://github.com/ClickHouse/ClickHouse/pull/6934) ([Ivan](https://github.com/abyss7)) #### Build/Testing/Packaging Improvement {#buildtestingpackaging-improvement-3} -- Fix [flapping](https://clickhouse-test-reports.s3.yandex.net/6944/aab95fd5175a513413c7395a73a82044bdafb906/functional_stateless_tests_(debug).html) test `00715_fetch_merged_or_mutated_part_zookeeper` by rewriting it to a shell scripts because it needs to wait for mutations to apply. [\#6977](https://github.com/ClickHouse/ClickHouse/pull/6977) ([Alexander Kazakov](https://github.com/Akazz)) -- Fixed UBSan and MemSan failure in function `groupUniqArray` with emtpy array argument. It was caused by placing of empty `PaddedPODArray` into hash table zero cell because constructor for zero cell value was not called. [\#6937](https://github.com/ClickHouse/ClickHouse/pull/6937) ([Amos Bird](https://github.com/amosbird)) +- Fix [flapping](https://clickhouse-test-reports.s3.yandex.net/6944/aab95fd5175a513413c7395a73a82044bdafb906/functional_stateless_tests_(debug).html) test `00715_fetch_merged_or_mutated_part_zookeeper` by rewriting it to a shell scripts because it needs to wait for mutations to apply. [#6977](https://github.com/ClickHouse/ClickHouse/pull/6977) ([Alexander Kazakov](https://github.com/Akazz)) +- Fixed UBSan and MemSan failure in function `groupUniqArray` with emtpy array argument. It was caused by placing of empty `PaddedPODArray` into hash table zero cell because constructor for zero cell value was not called. [#6937](https://github.com/ClickHouse/ClickHouse/pull/6937) ([Amos Bird](https://github.com/amosbird)) ### ClickHouse Release 19.14.3.3, 2019-09-10 {#clickhouse-release-19-14-3-3-2019-09-10} #### New Feature {#new-feature-4} -- `WITH FILL` modifier for `ORDER BY`. (continuation of [\#5069](https://github.com/ClickHouse/ClickHouse/issues/5069)) [\#6610](https://github.com/ClickHouse/ClickHouse/pull/6610) ([Anton Popov](https://github.com/CurtizJ)) -- `WITH TIES` modifier for `LIMIT`. (continuation of [\#5069](https://github.com/ClickHouse/ClickHouse/issues/5069)) [\#6610](https://github.com/ClickHouse/ClickHouse/pull/6610) ([Anton Popov](https://github.com/CurtizJ)) -- Parse unquoted `NULL` literal as NULL (if setting `format_csv_unquoted_null_literal_as_null=1`). Initialize null fields with default values if data type of this field is not nullable (if setting `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)) -- Support for wildcards in paths of table functions `file` and `hdfs`. If the path contains wildcards, the table will be readonly. Example of usage: `select * from hdfs('hdfs://hdfs1:9000/some_dir/another_dir/*/file{0..9}{0..9}')` and `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)) -- New `system.metric_log` table which stores values of `system.events` and `system.metrics` with specified time interval. [\#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)) -- Allow to write ClickHouse text logs to `system.text_log` table. [\#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)) -- Show private symbols in stack traces (this is done via parsing symbol tables of ELF files). Added information about file and line number in stack traces if debug info is present. Speedup symbol name lookup with indexing symbols present in program. Added new SQL functions for introspection: `demangle` and `addressToLine`. Renamed function `symbolizeAddress` to `addressToSymbol` for consistency. Function `addressToSymbol` will return mangled name for performance reasons and you have to apply `demangle`. Added setting `allow_introspection_functions` which is turned off by default. [\#6201](https://github.com/ClickHouse/ClickHouse/pull/6201) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Table function `values` (the name is case-insensitive). It allows to read from `VALUES` list proposed in [\#5984](https://github.com/ClickHouse/ClickHouse/issues/5984). Example: `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)) -- Added an ability to alter storage settings. Syntax: `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)) -- Support for removing of detached parts. Syntax: `ALTER TABLE DROP DETACHED PART ''`. [\#6158](https://github.com/ClickHouse/ClickHouse/pull/6158) ([tavplubix](https://github.com/tavplubix)) -- Table constraints. Allows to add constraint to table definition which will be checked at insert. [\#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)) -- Suppport for cascaded materialized views. [\#6324](https://github.com/ClickHouse/ClickHouse/pull/6324) ([Amos Bird](https://github.com/amosbird)) -- Turn on query profiler by default to sample every query execution thread once a second. [\#6283](https://github.com/ClickHouse/ClickHouse/pull/6283) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Input format `ORC`. [\#6454](https://github.com/ClickHouse/ClickHouse/pull/6454) [\#6703](https://github.com/ClickHouse/ClickHouse/pull/6703) ([akonyaev90](https://github.com/akonyaev90)) -- Added two new functions: `sigmoid` and `tanh` (that are useful for machine learning applications). [\#6254](https://github.com/ClickHouse/ClickHouse/pull/6254) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Function `hasToken(haystack, token)`, `hasTokenCaseInsensitive(haystack, token)` to check if given token is in haystack. Token is a maximal length substring between two non alphanumeric ASCII characters (or boundaries of haystack). Token must be a constant string. Supported by tokenbf\_v1 index specialization. [\#6596](https://github.com/ClickHouse/ClickHouse/pull/6596), [\#6662](https://github.com/ClickHouse/ClickHouse/pull/6662) ([Vasily Nemkov](https://github.com/Enmk)) -- New function `neighbor(value, offset[, default_value])`. Allows to reach prev/next value within column in a block of data. [\#5925](https://github.com/ClickHouse/ClickHouse/pull/5925) ([Alex Krash](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) -- Created a function `currentUser()`, returning login of authorized user. Added alias `user()` for compatibility with MySQL. [\#6470](https://github.com/ClickHouse/ClickHouse/pull/6470) ([Alex Krash](https://github.com/alex-krash)) -- New aggregate functions `quantilesExactInclusive` and `quantilesExactExclusive` which were proposed in [\#5885](https://github.com/ClickHouse/ClickHouse/issues/5885). [\#6477](https://github.com/ClickHouse/ClickHouse/pull/6477) ([dimarub2000](https://github.com/dimarub2000)) -- Function `bitmapRange(bitmap, range_begin, range_end)` which returns new set with specified range (not include the `range_end`). [\#6314](https://github.com/ClickHouse/ClickHouse/pull/6314) ([Zhichang Yu](https://github.com/yuzhichang)) -- Function `geohashesInBox(longitude_min, latitude_min, longitude_max, latitude_max, precision)` which creates array of precision-long strings of geohash-boxes covering provided area. [\#6127](https://github.com/ClickHouse/ClickHouse/pull/6127) ([Vasily Nemkov](https://github.com/Enmk)) -- Implement support for INSERT query with `Kafka` tables. [\#6012](https://github.com/ClickHouse/ClickHouse/pull/6012) ([Ivan](https://github.com/abyss7)) -- Added support for `_partition` and `_timestamp` virtual columns to Kafka engine. [\#6400](https://github.com/ClickHouse/ClickHouse/pull/6400) ([Ivan](https://github.com/abyss7)) -- Possibility to remove sensitive data from `query_log`, server logs, process list with regexp-based rules. [\#5710](https://github.com/ClickHouse/ClickHouse/pull/5710) ([filimonov](https://github.com/filimonov)) +- `WITH FILL` modifier for `ORDER BY`. (continuation of [#5069](https://github.com/ClickHouse/ClickHouse/issues/5069)) [#6610](https://github.com/ClickHouse/ClickHouse/pull/6610) ([Anton Popov](https://github.com/CurtizJ)) +- `WITH TIES` modifier for `LIMIT`. (continuation of [#5069](https://github.com/ClickHouse/ClickHouse/issues/5069)) [#6610](https://github.com/ClickHouse/ClickHouse/pull/6610) ([Anton Popov](https://github.com/CurtizJ)) +- Parse unquoted `NULL` literal as NULL (if setting `format_csv_unquoted_null_literal_as_null=1`). Initialize null fields with default values if data type of this field is not nullable (if setting `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)) +- Support for wildcards in paths of table functions `file` and `hdfs`. If the path contains wildcards, the table will be readonly. Example of usage: `select * from hdfs('hdfs://hdfs1:9000/some_dir/another_dir/*/file{0..9}{0..9}')` and `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)) +- New `system.metric_log` table which stores values of `system.events` and `system.metrics` with specified time interval. [#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)) +- Allow to write ClickHouse text logs to `system.text_log` table. [#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)) +- Show private symbols in stack traces (this is done via parsing symbol tables of ELF files). Added information about file and line number in stack traces if debug info is present. Speedup symbol name lookup with indexing symbols present in program. Added new SQL functions for introspection: `demangle` and `addressToLine`. Renamed function `symbolizeAddress` to `addressToSymbol` for consistency. Function `addressToSymbol` will return mangled name for performance reasons and you have to apply `demangle`. Added setting `allow_introspection_functions` which is turned off by default. [#6201](https://github.com/ClickHouse/ClickHouse/pull/6201) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Table function `values` (the name is case-insensitive). It allows to read from `VALUES` list proposed in [#5984](https://github.com/ClickHouse/ClickHouse/issues/5984). Example: `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)) +- Added an ability to alter storage settings. Syntax: `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)) +- Support for removing of detached parts. Syntax: `ALTER TABLE DROP DETACHED PART ''`. [#6158](https://github.com/ClickHouse/ClickHouse/pull/6158) ([tavplubix](https://github.com/tavplubix)) +- Table constraints. Allows to add constraint to table definition which will be checked at insert. [#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)) +- Suppport for cascaded materialized views. [#6324](https://github.com/ClickHouse/ClickHouse/pull/6324) ([Amos Bird](https://github.com/amosbird)) +- Turn on query profiler by default to sample every query execution thread once a second. [#6283](https://github.com/ClickHouse/ClickHouse/pull/6283) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Input format `ORC`. [#6454](https://github.com/ClickHouse/ClickHouse/pull/6454) [#6703](https://github.com/ClickHouse/ClickHouse/pull/6703) ([akonyaev90](https://github.com/akonyaev90)) +- Added two new functions: `sigmoid` and `tanh` (that are useful for machine learning applications). [#6254](https://github.com/ClickHouse/ClickHouse/pull/6254) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Function `hasToken(haystack, token)`, `hasTokenCaseInsensitive(haystack, token)` to check if given token is in haystack. Token is a maximal length substring between two non alphanumeric ASCII characters (or boundaries of haystack). Token must be a constant string. Supported by tokenbf_v1 index specialization. [#6596](https://github.com/ClickHouse/ClickHouse/pull/6596), [#6662](https://github.com/ClickHouse/ClickHouse/pull/6662) ([Vasily Nemkov](https://github.com/Enmk)) +- New function `neighbor(value, offset[, default_value])`. Allows to reach prev/next value within column in a block of data. [#5925](https://github.com/ClickHouse/ClickHouse/pull/5925) ([Alex Krash](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) +- Created a function `currentUser()`, returning login of authorized user. Added alias `user()` for compatibility with MySQL. [#6470](https://github.com/ClickHouse/ClickHouse/pull/6470) ([Alex Krash](https://github.com/alex-krash)) +- New aggregate functions `quantilesExactInclusive` and `quantilesExactExclusive` which were proposed in [#5885](https://github.com/ClickHouse/ClickHouse/issues/5885). [#6477](https://github.com/ClickHouse/ClickHouse/pull/6477) ([dimarub2000](https://github.com/dimarub2000)) +- Function `bitmapRange(bitmap, range_begin, range_end)` which returns new set with specified range (not include the `range_end`). [#6314](https://github.com/ClickHouse/ClickHouse/pull/6314) ([Zhichang Yu](https://github.com/yuzhichang)) +- Function `geohashesInBox(longitude_min, latitude_min, longitude_max, latitude_max, precision)` which creates array of precision-long strings of geohash-boxes covering provided area. [#6127](https://github.com/ClickHouse/ClickHouse/pull/6127) ([Vasily Nemkov](https://github.com/Enmk)) +- Implement support for INSERT query with `Kafka` tables. [#6012](https://github.com/ClickHouse/ClickHouse/pull/6012) ([Ivan](https://github.com/abyss7)) +- Added support for `_partition` and `_timestamp` virtual columns to Kafka engine. [#6400](https://github.com/ClickHouse/ClickHouse/pull/6400) ([Ivan](https://github.com/abyss7)) +- Possibility to remove sensitive data from `query_log`, server logs, process list with regexp-based rules. [#5710](https://github.com/ClickHouse/ClickHouse/pull/5710) ([filimonov](https://github.com/filimonov)) #### Experimental Feature {#experimental-feature-2} -- Input and output data format `Template`. It allows to specify custom format string for input and output. [\#4354](https://github.com/ClickHouse/ClickHouse/issues/4354) [\#6727](https://github.com/ClickHouse/ClickHouse/pull/6727) ([tavplubix](https://github.com/tavplubix)) -- Implementation of `LIVE VIEW` tables that were originally proposed in [\#2898](https://github.com/ClickHouse/ClickHouse/pull/2898), prepared in [\#3925](https://github.com/ClickHouse/ClickHouse/issues/3925), and then updated in [\#5541](https://github.com/ClickHouse/ClickHouse/issues/5541). See [\#5541](https://github.com/ClickHouse/ClickHouse/issues/5541) for detailed description. [\#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)) Note that `LIVE VIEW` feature may be removed in next versions. +- Input and output data format `Template`. It allows to specify custom format string for input and output. [#4354](https://github.com/ClickHouse/ClickHouse/issues/4354) [#6727](https://github.com/ClickHouse/ClickHouse/pull/6727) ([tavplubix](https://github.com/tavplubix)) +- Implementation of `LIVE VIEW` tables that were originally proposed in [#2898](https://github.com/ClickHouse/ClickHouse/pull/2898), prepared in [#3925](https://github.com/ClickHouse/ClickHouse/issues/3925), and then updated in [#5541](https://github.com/ClickHouse/ClickHouse/issues/5541). See [#5541](https://github.com/ClickHouse/ClickHouse/issues/5541) for detailed description. [#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)) Note that `LIVE VIEW` feature may be removed in next versions. #### Bug Fix {#bug-fix-8} - This release also contains all bug fixes from 19.13 and 19.11. -- Fix segmentation fault when the table has skip indices and vertical merge happens. [\#6723](https://github.com/ClickHouse/ClickHouse/pull/6723) ([alesapin](https://github.com/alesapin)) -- Fix per-column TTL with non-trivial column defaults. Previously in case of force TTL merge with `OPTIMIZE ... FINAL` query, expired values was replaced by type defaults instead of user-specified column defaults. [\#6796](https://github.com/ClickHouse/ClickHouse/pull/6796) ([Anton Popov](https://github.com/CurtizJ)) -- Fix Kafka messages duplication problem on normal server restart. [\#6597](https://github.com/ClickHouse/ClickHouse/pull/6597) ([Ivan](https://github.com/abyss7)) -- Fixed infinite loop when reading Kafka messages. Do not pause/resume consumer on subscription at all - otherwise it may get paused indefinitely in some scenarios. [\#6354](https://github.com/ClickHouse/ClickHouse/pull/6354) ([Ivan](https://github.com/abyss7)) -- Fix `Key expression contains comparison between inconvertible types` exception in `bitmapContains` function. [\#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)) -- Fix segfault with enabled `optimize_skip_unused_shards` and missing sharding key. [\#6384](https://github.com/ClickHouse/ClickHouse/pull/6384) ([Anton Popov](https://github.com/CurtizJ)) -- Fixed wrong code in mutations that may lead to memory corruption. Fixed segfault with read of address `0x14c0` that may happed due to concurrent `DROP TABLE` and `SELECT` from `system.parts` or `system.parts_columns`. Fixed race condition in preparation of mutation queries. Fixed deadlock caused by `OPTIMIZE` of Replicated tables and concurrent modification operations like ALTERs. [\#6514](https://github.com/ClickHouse/ClickHouse/pull/6514) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Removed extra verbose logging in MySQL interface [\#6389](https://github.com/ClickHouse/ClickHouse/pull/6389) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Return the ability to parse boolean settings from ‘true’ and ‘false’ in the configuration file. [\#6278](https://github.com/ClickHouse/ClickHouse/pull/6278) ([alesapin](https://github.com/alesapin)) -- Fix crash in `quantile` and `median` function over `Nullable(Decimal128)`. [\#6378](https://github.com/ClickHouse/ClickHouse/pull/6378) ([Artem Zuikov](https://github.com/4ertus2)) -- Fixed possible incomplete result returned by `SELECT` query with `WHERE` condition on primary key contained conversion to Float type. It was caused by incorrect checking of monotonicity in `toFloat` function. [\#6248](https://github.com/ClickHouse/ClickHouse/issues/6248) [\#6374](https://github.com/ClickHouse/ClickHouse/pull/6374) ([dimarub2000](https://github.com/dimarub2000)) -- Check `max_expanded_ast_elements` setting for mutations. Clear mutations after `TRUNCATE TABLE`. [\#6205](https://github.com/ClickHouse/ClickHouse/pull/6205) ([Winter Zhang](https://github.com/zhang2014)) -- Fix JOIN results for key columns when used with `join_use_nulls`. Attach Nulls instead of columns defaults. [\#6249](https://github.com/ClickHouse/ClickHouse/pull/6249) ([Artem Zuikov](https://github.com/4ertus2)) -- Fix for skip indices with vertical merge and alter. Fix for `Bad size of marks file` exception. [\#6594](https://github.com/ClickHouse/ClickHouse/issues/6594) [\#6713](https://github.com/ClickHouse/ClickHouse/pull/6713) ([alesapin](https://github.com/alesapin)) -- Fix rare crash in `ALTER MODIFY COLUMN` and vertical merge when one of merged/altered parts is empty (0 rows) [\#6746](https://github.com/ClickHouse/ClickHouse/issues/6746) [\#6780](https://github.com/ClickHouse/ClickHouse/pull/6780) ([alesapin](https://github.com/alesapin)) -- Fixed bug in conversion of `LowCardinality` types in `AggregateFunctionFactory`. This fixes [\#6257](https://github.com/ClickHouse/ClickHouse/issues/6257). [\#6281](https://github.com/ClickHouse/ClickHouse/pull/6281) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Fix wrong behavior and possible segfaults in `topK` and `topKWeighted` aggregated functions. [\#6404](https://github.com/ClickHouse/ClickHouse/pull/6404) ([Anton Popov](https://github.com/CurtizJ)) -- Fixed unsafe code around `getIdentifier` function. [\#6401](https://github.com/ClickHouse/ClickHouse/issues/6401) [\#6409](https://github.com/ClickHouse/ClickHouse/pull/6409) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed bug in MySQL wire protocol (is used while connecting to ClickHouse form MySQL client). Caused by heap buffer overflow in `PacketPayloadWriteBuffer`. [\#6212](https://github.com/ClickHouse/ClickHouse/pull/6212) ([Yuriy Baranov](https://github.com/yurriy)) -- Fixed memory leak in `bitmapSubsetInRange` function. [\#6819](https://github.com/ClickHouse/ClickHouse/pull/6819) ([Zhichang Yu](https://github.com/yuzhichang)) -- Fix rare bug when mutation executed after granularity change. [\#6816](https://github.com/ClickHouse/ClickHouse/pull/6816) ([alesapin](https://github.com/alesapin)) -- Allow protobuf message with all fields by default. [\#6132](https://github.com/ClickHouse/ClickHouse/pull/6132) ([Vitaly Baranov](https://github.com/vitlibar)) -- Resolve a bug with `nullIf` function when we send a `NULL` argument on the second argument. [\#6446](https://github.com/ClickHouse/ClickHouse/pull/6446) ([Guillaume Tassery](https://github.com/YiuRULE)) -- Fix rare bug with wrong memory allocation/deallocation in complex key cache dictionaries with string fields which leads to infinite memory consumption (looks like memory leak). Bug reproduces when string size was a power of two starting from eight (8, 16, 32, etc). [\#6447](https://github.com/ClickHouse/ClickHouse/pull/6447) ([alesapin](https://github.com/alesapin)) -- Fixed Gorilla encoding on small sequences which caused exception `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)) -- Allow to use not nullable types in JOINs with `join_use_nulls` enabled. [\#6705](https://github.com/ClickHouse/ClickHouse/pull/6705) ([Artem Zuikov](https://github.com/4ertus2)) -- Disable `Poco::AbstractConfiguration` substitutions in query in `clickhouse-client`. [\#6706](https://github.com/ClickHouse/ClickHouse/pull/6706) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Avoid deadlock in `REPLACE PARTITION`. [\#6677](https://github.com/ClickHouse/ClickHouse/pull/6677) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Using `arrayReduce` for constant arguments may lead to segfault. [\#6242](https://github.com/ClickHouse/ClickHouse/issues/6242) [\#6326](https://github.com/ClickHouse/ClickHouse/pull/6326) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix inconsistent parts which can appear if replica was restored after `DROP PARTITION`. [\#6522](https://github.com/ClickHouse/ClickHouse/issues/6522) [\#6523](https://github.com/ClickHouse/ClickHouse/pull/6523) ([tavplubix](https://github.com/tavplubix)) -- Fixed hang in `JSONExtractRaw` function. [\#6195](https://github.com/ClickHouse/ClickHouse/issues/6195) [\#6198](https://github.com/ClickHouse/ClickHouse/pull/6198) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix bug with incorrect skip indices serialization and aggregation with adaptive granularity. [\#6594](https://github.com/ClickHouse/ClickHouse/issues/6594). [\#6748](https://github.com/ClickHouse/ClickHouse/pull/6748) ([alesapin](https://github.com/alesapin)) -- Fix `WITH ROLLUP` and `WITH CUBE` modifiers of `GROUP BY` with two-level aggregation. [\#6225](https://github.com/ClickHouse/ClickHouse/pull/6225) ([Anton Popov](https://github.com/CurtizJ)) -- Fix bug with writing secondary indices marks with adaptive granularity. [\#6126](https://github.com/ClickHouse/ClickHouse/pull/6126) ([alesapin](https://github.com/alesapin)) -- Fix initialization order while server startup. Since `StorageMergeTree::background_task_handle` is initialized in `startup()` the `MergeTreeBlockOutputStream::write()` may try to use it before initialization. Just check if it is initialized. [\#6080](https://github.com/ClickHouse/ClickHouse/pull/6080) ([Ivan](https://github.com/abyss7)) -- Clearing the data buffer from the previous read operation that was completed with an error. [\#6026](https://github.com/ClickHouse/ClickHouse/pull/6026) ([Nikolay](https://github.com/bopohaa)) -- Fix bug with enabling adaptive granularity when creating a new replica for Replicated\*MergeTree table. [\#6394](https://github.com/ClickHouse/ClickHouse/issues/6394) [\#6452](https://github.com/ClickHouse/ClickHouse/pull/6452) ([alesapin](https://github.com/alesapin)) -- Fixed possible crash during server startup in case of exception happened in `libunwind` during exception at access to uninitialized `ThreadStatus` structure. [\#6456](https://github.com/ClickHouse/ClickHouse/pull/6456) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -- Fix crash in `yandexConsistentHash` function. Found by fuzz test. [\#6304](https://github.com/ClickHouse/ClickHouse/issues/6304) [\#6305](https://github.com/ClickHouse/ClickHouse/pull/6305) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed the possibility of hanging queries when server is overloaded and global thread pool becomes near full. This have higher chance to happen on clusters with large number of shards (hundreds), because distributed queries allocate a thread per connection to each shard. For example, this issue may reproduce if a cluster of 330 shards is processing 30 concurrent distributed queries. This issue affects all versions starting from 19.2. [\#6301](https://github.com/ClickHouse/ClickHouse/pull/6301) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed logic of `arrayEnumerateUniqRanked` function. [\#6423](https://github.com/ClickHouse/ClickHouse/pull/6423) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix segfault when decoding symbol table. [\#6603](https://github.com/ClickHouse/ClickHouse/pull/6603) ([Amos Bird](https://github.com/amosbird)) -- Fixed irrelevant exception in cast of `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)) -- Removed extra quoting of description in `system.settings` table. [\#6696](https://github.com/ClickHouse/ClickHouse/issues/6696) [\#6699](https://github.com/ClickHouse/ClickHouse/pull/6699) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Avoid possible deadlock in `TRUNCATE` of Replicated table. [\#6695](https://github.com/ClickHouse/ClickHouse/pull/6695) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix reading in order of sorting key. [\#6189](https://github.com/ClickHouse/ClickHouse/pull/6189) ([Anton Popov](https://github.com/CurtizJ)) -- Fix `ALTER TABLE ... UPDATE` query for tables with `enable_mixed_granularity_parts=1`. [\#6543](https://github.com/ClickHouse/ClickHouse/pull/6543) ([alesapin](https://github.com/alesapin)) -- Fix bug opened by [\#4405](https://github.com/ClickHouse/ClickHouse/pull/4405) (since 19.4.0). Reproduces in queries to Distributed tables over MergeTree tables when we doesn’t query any columns (`SELECT 1`). [\#6236](https://github.com/ClickHouse/ClickHouse/pull/6236) ([alesapin](https://github.com/alesapin)) -- Fixed overflow in integer division of signed type to unsigned type. The behaviour was exactly as in C or C++ language (integer promotion rules) that may be surprising. Please note that the overflow is still possible when dividing large signed number to large unsigned number or vice-versa (but that case is less usual). The issue existed in all server versions. [\#6214](https://github.com/ClickHouse/ClickHouse/issues/6214) [\#6233](https://github.com/ClickHouse/ClickHouse/pull/6233) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Limit maximum sleep time for throttling when `max_execution_speed` or `max_execution_speed_bytes` is set. Fixed false errors like `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)) -- Fixed issues about using `MATERIALIZED` columns and aliases in `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 Bird](https://github.com/amosbird)) [\#6316](https://github.com/ClickHouse/ClickHouse/pull/6316) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix `FormatFactory` behaviour for input streams which are not implemented as processor. [\#6495](https://github.com/ClickHouse/ClickHouse/pull/6495) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Fixed typo. [\#6631](https://github.com/ClickHouse/ClickHouse/pull/6631) ([Alex Ryndin](https://github.com/alexryndin)) -- Typo in the error message ( is -\> are ). [\#6839](https://github.com/ClickHouse/ClickHouse/pull/6839) ([Denis Zhuravlev](https://github.com/den-crane)) -- Fixed error while parsing of columns list from string if type contained a comma (this issue was relevant for `File`, `URL`, `HDFS` storages) [\#6217](https://github.com/ClickHouse/ClickHouse/issues/6217). [\#6209](https://github.com/ClickHouse/ClickHouse/pull/6209) ([dimarub2000](https://github.com/dimarub2000)) +- Fix segmentation fault when the table has skip indices and vertical merge happens. [#6723](https://github.com/ClickHouse/ClickHouse/pull/6723) ([alesapin](https://github.com/alesapin)) +- Fix per-column TTL with non-trivial column defaults. Previously in case of force TTL merge with `OPTIMIZE ... FINAL` query, expired values was replaced by type defaults instead of user-specified column defaults. [#6796](https://github.com/ClickHouse/ClickHouse/pull/6796) ([Anton Popov](https://github.com/CurtizJ)) +- Fix Kafka messages duplication problem on normal server restart. [#6597](https://github.com/ClickHouse/ClickHouse/pull/6597) ([Ivan](https://github.com/abyss7)) +- Fixed infinite loop when reading Kafka messages. Do not pause/resume consumer on subscription at all - otherwise it may get paused indefinitely in some scenarios. [#6354](https://github.com/ClickHouse/ClickHouse/pull/6354) ([Ivan](https://github.com/abyss7)) +- Fix `Key expression contains comparison between inconvertible types` exception in `bitmapContains` function. [#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)) +- Fix segfault with enabled `optimize_skip_unused_shards` and missing sharding key. [#6384](https://github.com/ClickHouse/ClickHouse/pull/6384) ([Anton Popov](https://github.com/CurtizJ)) +- Fixed wrong code in mutations that may lead to memory corruption. Fixed segfault with read of address `0x14c0` that may happed due to concurrent `DROP TABLE` and `SELECT` from `system.parts` or `system.parts_columns`. Fixed race condition in preparation of mutation queries. Fixed deadlock caused by `OPTIMIZE` of Replicated tables and concurrent modification operations like ALTERs. [#6514](https://github.com/ClickHouse/ClickHouse/pull/6514) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Removed extra verbose logging in MySQL interface [#6389](https://github.com/ClickHouse/ClickHouse/pull/6389) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Return the ability to parse boolean settings from ‘true’ and ‘false’ in the configuration file. [#6278](https://github.com/ClickHouse/ClickHouse/pull/6278) ([alesapin](https://github.com/alesapin)) +- Fix crash in `quantile` and `median` function over `Nullable(Decimal128)`. [#6378](https://github.com/ClickHouse/ClickHouse/pull/6378) ([Artem Zuikov](https://github.com/4ertus2)) +- Fixed possible incomplete result returned by `SELECT` query with `WHERE` condition on primary key contained conversion to Float type. It was caused by incorrect checking of monotonicity in `toFloat` function. [#6248](https://github.com/ClickHouse/ClickHouse/issues/6248) [#6374](https://github.com/ClickHouse/ClickHouse/pull/6374) ([dimarub2000](https://github.com/dimarub2000)) +- Check `max_expanded_ast_elements` setting for mutations. Clear mutations after `TRUNCATE TABLE`. [#6205](https://github.com/ClickHouse/ClickHouse/pull/6205) ([Winter Zhang](https://github.com/zhang2014)) +- Fix JOIN results for key columns when used with `join_use_nulls`. Attach Nulls instead of columns defaults. [#6249](https://github.com/ClickHouse/ClickHouse/pull/6249) ([Artem Zuikov](https://github.com/4ertus2)) +- Fix for skip indices with vertical merge and alter. Fix for `Bad size of marks file` exception. [#6594](https://github.com/ClickHouse/ClickHouse/issues/6594) [#6713](https://github.com/ClickHouse/ClickHouse/pull/6713) ([alesapin](https://github.com/alesapin)) +- Fix rare crash in `ALTER MODIFY COLUMN` and vertical merge when one of merged/altered parts is empty (0 rows) [#6746](https://github.com/ClickHouse/ClickHouse/issues/6746) [#6780](https://github.com/ClickHouse/ClickHouse/pull/6780) ([alesapin](https://github.com/alesapin)) +- Fixed bug in conversion of `LowCardinality` types in `AggregateFunctionFactory`. This fixes [#6257](https://github.com/ClickHouse/ClickHouse/issues/6257). [#6281](https://github.com/ClickHouse/ClickHouse/pull/6281) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fix wrong behavior and possible segfaults in `topK` and `topKWeighted` aggregated functions. [#6404](https://github.com/ClickHouse/ClickHouse/pull/6404) ([Anton Popov](https://github.com/CurtizJ)) +- Fixed unsafe code around `getIdentifier` function. [#6401](https://github.com/ClickHouse/ClickHouse/issues/6401) [#6409](https://github.com/ClickHouse/ClickHouse/pull/6409) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed bug in MySQL wire protocol (is used while connecting to ClickHouse form MySQL client). Caused by heap buffer overflow in `PacketPayloadWriteBuffer`. [#6212](https://github.com/ClickHouse/ClickHouse/pull/6212) ([Yuriy Baranov](https://github.com/yurriy)) +- Fixed memory leak in `bitmapSubsetInRange` function. [#6819](https://github.com/ClickHouse/ClickHouse/pull/6819) ([Zhichang Yu](https://github.com/yuzhichang)) +- Fix rare bug when mutation executed after granularity change. [#6816](https://github.com/ClickHouse/ClickHouse/pull/6816) ([alesapin](https://github.com/alesapin)) +- Allow protobuf message with all fields by default. [#6132](https://github.com/ClickHouse/ClickHouse/pull/6132) ([Vitaly Baranov](https://github.com/vitlibar)) +- Resolve a bug with `nullIf` function when we send a `NULL` argument on the second argument. [#6446](https://github.com/ClickHouse/ClickHouse/pull/6446) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Fix rare bug with wrong memory allocation/deallocation in complex key cache dictionaries with string fields which leads to infinite memory consumption (looks like memory leak). Bug reproduces when string size was a power of two starting from eight (8, 16, 32, etc). [#6447](https://github.com/ClickHouse/ClickHouse/pull/6447) ([alesapin](https://github.com/alesapin)) +- Fixed Gorilla encoding on small sequences which caused exception `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)) +- Allow to use not nullable types in JOINs with `join_use_nulls` enabled. [#6705](https://github.com/ClickHouse/ClickHouse/pull/6705) ([Artem Zuikov](https://github.com/4ertus2)) +- Disable `Poco::AbstractConfiguration` substitutions in query in `clickhouse-client`. [#6706](https://github.com/ClickHouse/ClickHouse/pull/6706) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Avoid deadlock in `REPLACE PARTITION`. [#6677](https://github.com/ClickHouse/ClickHouse/pull/6677) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Using `arrayReduce` for constant arguments may lead to segfault. [#6242](https://github.com/ClickHouse/ClickHouse/issues/6242) [#6326](https://github.com/ClickHouse/ClickHouse/pull/6326) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix inconsistent parts which can appear if replica was restored after `DROP PARTITION`. [#6522](https://github.com/ClickHouse/ClickHouse/issues/6522) [#6523](https://github.com/ClickHouse/ClickHouse/pull/6523) ([tavplubix](https://github.com/tavplubix)) +- Fixed hang in `JSONExtractRaw` function. [#6195](https://github.com/ClickHouse/ClickHouse/issues/6195) [#6198](https://github.com/ClickHouse/ClickHouse/pull/6198) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix bug with incorrect skip indices serialization and aggregation with adaptive granularity. [#6594](https://github.com/ClickHouse/ClickHouse/issues/6594). [#6748](https://github.com/ClickHouse/ClickHouse/pull/6748) ([alesapin](https://github.com/alesapin)) +- Fix `WITH ROLLUP` and `WITH CUBE` modifiers of `GROUP BY` with two-level aggregation. [#6225](https://github.com/ClickHouse/ClickHouse/pull/6225) ([Anton Popov](https://github.com/CurtizJ)) +- Fix bug with writing secondary indices marks with adaptive granularity. [#6126](https://github.com/ClickHouse/ClickHouse/pull/6126) ([alesapin](https://github.com/alesapin)) +- Fix initialization order while server startup. Since `StorageMergeTree::background_task_handle` is initialized in `startup()` the `MergeTreeBlockOutputStream::write()` may try to use it before initialization. Just check if it is initialized. [#6080](https://github.com/ClickHouse/ClickHouse/pull/6080) ([Ivan](https://github.com/abyss7)) +- Clearing the data buffer from the previous read operation that was completed with an error. [#6026](https://github.com/ClickHouse/ClickHouse/pull/6026) ([Nikolay](https://github.com/bopohaa)) +- Fix bug with enabling adaptive granularity when creating a new replica for Replicated\*MergeTree table. [#6394](https://github.com/ClickHouse/ClickHouse/issues/6394) [#6452](https://github.com/ClickHouse/ClickHouse/pull/6452) ([alesapin](https://github.com/alesapin)) +- Fixed possible crash during server startup in case of exception happened in `libunwind` during exception at access to uninitialized `ThreadStatus` structure. [#6456](https://github.com/ClickHouse/ClickHouse/pull/6456) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +- Fix crash in `yandexConsistentHash` function. Found by fuzz test. [#6304](https://github.com/ClickHouse/ClickHouse/issues/6304) [#6305](https://github.com/ClickHouse/ClickHouse/pull/6305) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed the possibility of hanging queries when server is overloaded and global thread pool becomes near full. This have higher chance to happen on clusters with large number of shards (hundreds), because distributed queries allocate a thread per connection to each shard. For example, this issue may reproduce if a cluster of 330 shards is processing 30 concurrent distributed queries. This issue affects all versions starting from 19.2. [#6301](https://github.com/ClickHouse/ClickHouse/pull/6301) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed logic of `arrayEnumerateUniqRanked` function. [#6423](https://github.com/ClickHouse/ClickHouse/pull/6423) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix segfault when decoding symbol table. [#6603](https://github.com/ClickHouse/ClickHouse/pull/6603) ([Amos Bird](https://github.com/amosbird)) +- Fixed irrelevant exception in cast of `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)) +- Removed extra quoting of description in `system.settings` table. [#6696](https://github.com/ClickHouse/ClickHouse/issues/6696) [#6699](https://github.com/ClickHouse/ClickHouse/pull/6699) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Avoid possible deadlock in `TRUNCATE` of Replicated table. [#6695](https://github.com/ClickHouse/ClickHouse/pull/6695) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix reading in order of sorting key. [#6189](https://github.com/ClickHouse/ClickHouse/pull/6189) ([Anton Popov](https://github.com/CurtizJ)) +- Fix `ALTER TABLE ... UPDATE` query for tables with `enable_mixed_granularity_parts=1`. [#6543](https://github.com/ClickHouse/ClickHouse/pull/6543) ([alesapin](https://github.com/alesapin)) +- Fix bug opened by [#4405](https://github.com/ClickHouse/ClickHouse/pull/4405) (since 19.4.0). Reproduces in queries to Distributed tables over MergeTree tables when we doesn’t query any columns (`SELECT 1`). [#6236](https://github.com/ClickHouse/ClickHouse/pull/6236) ([alesapin](https://github.com/alesapin)) +- Fixed overflow in integer division of signed type to unsigned type. The behaviour was exactly as in C or C++ language (integer promotion rules) that may be surprising. Please note that the overflow is still possible when dividing large signed number to large unsigned number or vice-versa (but that case is less usual). The issue existed in all server versions. [#6214](https://github.com/ClickHouse/ClickHouse/issues/6214) [#6233](https://github.com/ClickHouse/ClickHouse/pull/6233) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Limit maximum sleep time for throttling when `max_execution_speed` or `max_execution_speed_bytes` is set. Fixed false errors like `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)) +- Fixed issues about using `MATERIALIZED` columns and aliases in `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 Bird](https://github.com/amosbird)) [#6316](https://github.com/ClickHouse/ClickHouse/pull/6316) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix `FormatFactory` behaviour for input streams which are not implemented as processor. [#6495](https://github.com/ClickHouse/ClickHouse/pull/6495) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fixed typo. [#6631](https://github.com/ClickHouse/ClickHouse/pull/6631) ([Alex Ryndin](https://github.com/alexryndin)) +- Typo in the error message ( is -\> are ). [#6839](https://github.com/ClickHouse/ClickHouse/pull/6839) ([Denis Zhuravlev](https://github.com/den-crane)) +- Fixed error while parsing of columns list from string if type contained a comma (this issue was relevant for `File`, `URL`, `HDFS` storages) [#6217](https://github.com/ClickHouse/ClickHouse/issues/6217). [#6209](https://github.com/ClickHouse/ClickHouse/pull/6209) ([dimarub2000](https://github.com/dimarub2000)) #### Security Fix {#security-fix} - This release also contains all bug security fixes from 19.13 and 19.11. -- Fixed the possibility of a fabricated query to cause server crash due to stack overflow in SQL parser. Fixed the possibility of stack overflow in Merge and Distributed tables, materialized views and conditions for row-level security that involve subqueries. [\#6433](https://github.com/ClickHouse/ClickHouse/pull/6433) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed the possibility of a fabricated query to cause server crash due to stack overflow in SQL parser. Fixed the possibility of stack overflow in Merge and Distributed tables, materialized views and conditions for row-level security that involve subqueries. [#6433](https://github.com/ClickHouse/ClickHouse/pull/6433) ([alexey-milovidov](https://github.com/alexey-milovidov)) #### Improvement {#improvement-3} -- Correct implementation of ternary logic for `AND/OR`. [\#6048](https://github.com/ClickHouse/ClickHouse/pull/6048) ([Alexander Kazakov](https://github.com/Akazz)) -- Now values and rows with expired TTL will be removed after `OPTIMIZE ... FINAL` query from old parts without TTL infos or with outdated TTL infos, e.g. after `ALTER ... MODIFY TTL` query. Added queries `SYSTEM STOP/START TTL MERGES` to disallow/allow assign merges with TTL and filter expired values in all merges. [\#6274](https://github.com/ClickHouse/ClickHouse/pull/6274) ([Anton Popov](https://github.com/CurtizJ)) -- Possibility to change the location of ClickHouse history file for client using `CLICKHOUSE_HISTORY_FILE` env. [\#6840](https://github.com/ClickHouse/ClickHouse/pull/6840) ([filimonov](https://github.com/filimonov)) -- Remove `dry_run` flag from `InterpreterSelectQuery`. … [\#6375](https://github.com/ClickHouse/ClickHouse/pull/6375) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Support `ASOF JOIN` with `ON` section. [\#6211](https://github.com/ClickHouse/ClickHouse/pull/6211) ([Artem Zuikov](https://github.com/4ertus2)) -- Better support of skip indexes for mutations and replication. Support for `MATERIALIZE/CLEAR INDEX ... IN PARTITION` query. `UPDATE x = x` recalculates all indices that use column `x`. [\#5053](https://github.com/ClickHouse/ClickHouse/pull/5053) ([Nikita Vasilev](https://github.com/nikvas0)) -- Allow to `ATTACH` live views (for example, at the server startup) regardless to `allow_experimental_live_view` setting. [\#6754](https://github.com/ClickHouse/ClickHouse/pull/6754) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- For stack traces gathered by query profiler, do not include stack frames generated by the query profiler itself. [\#6250](https://github.com/ClickHouse/ClickHouse/pull/6250) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Now table functions `values`, `file`, `url`, `hdfs` have support for ALIAS columns. [\#6255](https://github.com/ClickHouse/ClickHouse/pull/6255) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Throw an exception if `config.d` file doesn’t have the corresponding root element as the config file. [\#6123](https://github.com/ClickHouse/ClickHouse/pull/6123) ([dimarub2000](https://github.com/dimarub2000)) -- Print extra info in exception message for `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)) -- When determining shards of a `Distributed` table to be covered by a read query (for `optimize_skip_unused_shards` = 1) ClickHouse now checks conditions from both `prewhere` and `where` clauses of select statement. [\#6521](https://github.com/ClickHouse/ClickHouse/pull/6521) ([Alexander Kazakov](https://github.com/Akazz)) -- Enabled `SIMDJSON` for machines without AVX2 but with SSE 4.2 and PCLMUL instruction set. [\#6285](https://github.com/ClickHouse/ClickHouse/issues/6285) [\#6320](https://github.com/ClickHouse/ClickHouse/pull/6320) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- ClickHouse can work on filesystems without `O_DIRECT` support (such as ZFS and BtrFS) without additional tuning. [\#4449](https://github.com/ClickHouse/ClickHouse/issues/4449) [\#6730](https://github.com/ClickHouse/ClickHouse/pull/6730) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Support push down predicate for final subquery. [\#6120](https://github.com/ClickHouse/ClickHouse/pull/6120) ([TCeason](https://github.com/TCeason)) [\#6162](https://github.com/ClickHouse/ClickHouse/pull/6162) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Better `JOIN ON` keys extraction [\#6131](https://github.com/ClickHouse/ClickHouse/pull/6131) ([Artem Zuikov](https://github.com/4ertus2)) -- Upated `SIMDJSON`. [\#6285](https://github.com/ClickHouse/ClickHouse/issues/6285). [\#6306](https://github.com/ClickHouse/ClickHouse/pull/6306) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Optimize selecting of smallest column for `SELECT count()` query. [\#6344](https://github.com/ClickHouse/ClickHouse/pull/6344) ([Amos Bird](https://github.com/amosbird)) -- Added `strict` parameter in `windowFunnel()`. When the `strict` is set, the `windowFunnel()` applies conditions only for the unique values. [\#6548](https://github.com/ClickHouse/ClickHouse/pull/6548) ([achimbab](https://github.com/achimbab)) -- Safer interface of `mysqlxx::Pool`. [\#6150](https://github.com/ClickHouse/ClickHouse/pull/6150) ([avasiliev](https://github.com/avasiliev)) -- Options line size when executing with `--help` option now corresponds with terminal size. [\#6590](https://github.com/ClickHouse/ClickHouse/pull/6590) ([dimarub2000](https://github.com/dimarub2000)) -- Disable “read in order” optimization for aggregation without keys. [\#6599](https://github.com/ClickHouse/ClickHouse/pull/6599) ([Anton Popov](https://github.com/CurtizJ)) -- HTTP status code for `INCORRECT_DATA` and `TYPE_MISMATCH` error codes was changed from default `500 Internal Server Error` to `400 Bad Request`. [\#6271](https://github.com/ClickHouse/ClickHouse/pull/6271) ([Alexander Rodin](https://github.com/a-rodin)) -- Move Join object from `ExpressionAction` into `AnalyzedJoin`. `ExpressionAnalyzer` and `ExpressionAction` do not know about `Join` class anymore. Its logic is hidden by `AnalyzedJoin` iface. [\#6801](https://github.com/ClickHouse/ClickHouse/pull/6801) ([Artem Zuikov](https://github.com/4ertus2)) -- Fixed possible deadlock of distributed queries when one of shards is localhost but the query is sent via network connection. [\#6759](https://github.com/ClickHouse/ClickHouse/pull/6759) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Changed semantic of multiple tables `RENAME` to avoid possible deadlocks. [\#6757](https://github.com/ClickHouse/ClickHouse/issues/6757). [\#6756](https://github.com/ClickHouse/ClickHouse/pull/6756) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Rewritten MySQL compatibility server to prevent loading full packet payload in memory. Decreased memory consumption for each connection to approximately `2 * DBMS_DEFAULT_BUFFER_SIZE` (read/write buffers). [\#5811](https://github.com/ClickHouse/ClickHouse/pull/5811) ([Yuriy Baranov](https://github.com/yurriy)) -- Move AST alias interpreting logic out of parser that doesn’t have to know anything about query semantics. [\#6108](https://github.com/ClickHouse/ClickHouse/pull/6108) ([Artem Zuikov](https://github.com/4ertus2)) -- Slightly more safe parsing of `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`: Allow use `where_condition` from config with `partition_key` alias in query for checking partition existence (Earlier it was used only in reading data queries). [\#6577](https://github.com/ClickHouse/ClickHouse/pull/6577) ([proller](https://github.com/proller)) -- Added optional message argument in `throwIf`. ([\#5772](https://github.com/ClickHouse/ClickHouse/issues/5772)) [\#6329](https://github.com/ClickHouse/ClickHouse/pull/6329) ([Vdimir](https://github.com/Vdimir)) -- Server exception got while sending insertion data is now being processed in client as well. [\#5891](https://github.com/ClickHouse/ClickHouse/issues/5891) [\#6711](https://github.com/ClickHouse/ClickHouse/pull/6711) ([dimarub2000](https://github.com/dimarub2000)) -- Added a metric `DistributedFilesToInsert` that shows the total number of files in filesystem that are selected to send to remote servers by Distributed tables. The number is summed across all shards. [\#6600](https://github.com/ClickHouse/ClickHouse/pull/6600) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Move most of JOINs prepare logic from `ExpressionAction/ExpressionAnalyzer` to `AnalyzedJoin`. [\#6785](https://github.com/ClickHouse/ClickHouse/pull/6785) ([Artem Zuikov](https://github.com/4ertus2)) -- Fix TSan [warning](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)) -- Better information messages about lack of Linux capabilities. Logging fatal errors with “fatal” level, that will make it easier to find in `system.text_log`. [\#6441](https://github.com/ClickHouse/ClickHouse/pull/6441) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- When enable dumping temporary data to the disk to restrict memory usage during `GROUP BY`, `ORDER BY`, it didn’t check the free disk space. The fix add a new setting `min_free_disk_space`, when the free disk space it smaller then the threshold, the query will stop and throw `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)) -- Removed recursive rwlock by thread. It makes no sense, because threads are reused between queries. `SELECT` query may acquire a lock in one thread, hold a lock from another thread and exit from first thread. In the same time, first thread can be reused by `DROP` query. This will lead to false “Attempt to acquire exclusive lock recursively” messages. [\#6771](https://github.com/ClickHouse/ClickHouse/pull/6771) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Split `ExpressionAnalyzer.appendJoin()`. Prepare a place in `ExpressionAnalyzer` for `MergeJoin`. [\#6524](https://github.com/ClickHouse/ClickHouse/pull/6524) ([Artem Zuikov](https://github.com/4ertus2)) -- Added `mysql_native_password` authentication plugin to MySQL compatibility server. [\#6194](https://github.com/ClickHouse/ClickHouse/pull/6194) ([Yuriy Baranov](https://github.com/yurriy)) -- Less number of `clock_gettime` calls; fixed ABI compatibility between debug/release in `Allocator` (insignificant issue). [\#6197](https://github.com/ClickHouse/ClickHouse/pull/6197) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Move `collectUsedColumns` from `ExpressionAnalyzer` to `SyntaxAnalyzer`. `SyntaxAnalyzer` makes `required_source_columns` itself now. [\#6416](https://github.com/ClickHouse/ClickHouse/pull/6416) ([Artem Zuikov](https://github.com/4ertus2)) -- Add setting `joined_subquery_requires_alias` to require aliases for subselects and table functions in `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)) -- Extract `GetAggregatesVisitor` class from `ExpressionAnalyzer`. [\#6458](https://github.com/ClickHouse/ClickHouse/pull/6458) ([Artem Zuikov](https://github.com/4ertus2)) -- `system.query_log`: change data type of `type` column to `Enum`. [\#6265](https://github.com/ClickHouse/ClickHouse/pull/6265) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -- Static linking of `sha256_password` authentication plugin. [\#6512](https://github.com/ClickHouse/ClickHouse/pull/6512) ([Yuriy Baranov](https://github.com/yurriy)) -- Avoid extra dependency for the setting `compile` to work. In previous versions, the user may get error like `cannot open crti.o`, `unable to find library -lc` etc. [\#6309](https://github.com/ClickHouse/ClickHouse/pull/6309) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- More validation of the input that may come from malicious replica. [\#6303](https://github.com/ClickHouse/ClickHouse/pull/6303) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Now `clickhouse-obfuscator` file is available in `clickhouse-client` package. In previous versions it was available as `clickhouse obfuscator` (with whitespace). [\#5816](https://github.com/ClickHouse/ClickHouse/issues/5816) [\#6609](https://github.com/ClickHouse/ClickHouse/pull/6609) ([dimarub2000](https://github.com/dimarub2000)) -- Fixed deadlock when we have at least two queries that read at least two tables in different order and another query that performs DDL operation on one of tables. Fixed another very rare deadlock. [\#6764](https://github.com/ClickHouse/ClickHouse/pull/6764) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Added `os_thread_ids` column to `system.processes` and `system.query_log` for better debugging possibilities. [\#6763](https://github.com/ClickHouse/ClickHouse/pull/6763) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- A workaround for PHP mysqlnd extension bugs which occur when `sha256_password` is used as a default authentication plugin (described in [\#6031](https://github.com/ClickHouse/ClickHouse/issues/6031)). [\#6113](https://github.com/ClickHouse/ClickHouse/pull/6113) ([Yuriy Baranov](https://github.com/yurriy)) -- Remove unneeded place with changed nullability columns. [\#6693](https://github.com/ClickHouse/ClickHouse/pull/6693) ([Artem Zuikov](https://github.com/4ertus2)) -- Set default value of `queue_max_wait_ms` to zero, because current value (five seconds) makes no sense. There are rare circumstances when this settings has any use. Added settings `replace_running_query_max_wait_ms`, `kafka_max_wait_ms` and `connection_pool_max_wait_ms` for disambiguation. [\#6692](https://github.com/ClickHouse/ClickHouse/pull/6692) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Extract `SelectQueryExpressionAnalyzer` from `ExpressionAnalyzer`. Keep the last one for non-select queries. [\#6499](https://github.com/ClickHouse/ClickHouse/pull/6499) ([Artem Zuikov](https://github.com/4ertus2)) -- Removed duplicating input and output formats. [\#6239](https://github.com/ClickHouse/ClickHouse/pull/6239) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Allow user to override `poll_interval` and `idle_connection_timeout` settings on connection. [\#6230](https://github.com/ClickHouse/ClickHouse/pull/6230) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- `MergeTree` now has an additional option `ttl_only_drop_parts` (disabled by default) to avoid partial pruning of parts, so that they dropped completely when all the rows in a part are expired. [\#6191](https://github.com/ClickHouse/ClickHouse/pull/6191) ([Sergi Vladykin](https://github.com/svladykin)) -- Type checks for set index functions. Throw exception if function got a wrong type. This fixes fuzz test with UBSan. [\#6511](https://github.com/ClickHouse/ClickHouse/pull/6511) ([Nikita Vasilev](https://github.com/nikvas0)) +- Correct implementation of ternary logic for `AND/OR`. [#6048](https://github.com/ClickHouse/ClickHouse/pull/6048) ([Alexander Kazakov](https://github.com/Akazz)) +- Now values and rows with expired TTL will be removed after `OPTIMIZE ... FINAL` query from old parts without TTL infos or with outdated TTL infos, e.g. after `ALTER ... MODIFY TTL` query. Added queries `SYSTEM STOP/START TTL MERGES` to disallow/allow assign merges with TTL and filter expired values in all merges. [#6274](https://github.com/ClickHouse/ClickHouse/pull/6274) ([Anton Popov](https://github.com/CurtizJ)) +- Possibility to change the location of ClickHouse history file for client using `CLICKHOUSE_HISTORY_FILE` env. [#6840](https://github.com/ClickHouse/ClickHouse/pull/6840) ([filimonov](https://github.com/filimonov)) +- Remove `dry_run` flag from `InterpreterSelectQuery`. … [#6375](https://github.com/ClickHouse/ClickHouse/pull/6375) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Support `ASOF JOIN` with `ON` section. [#6211](https://github.com/ClickHouse/ClickHouse/pull/6211) ([Artem Zuikov](https://github.com/4ertus2)) +- Better support of skip indexes for mutations and replication. Support for `MATERIALIZE/CLEAR INDEX ... IN PARTITION` query. `UPDATE x = x` recalculates all indices that use column `x`. [#5053](https://github.com/ClickHouse/ClickHouse/pull/5053) ([Nikita Vasilev](https://github.com/nikvas0)) +- Allow to `ATTACH` live views (for example, at the server startup) regardless to `allow_experimental_live_view` setting. [#6754](https://github.com/ClickHouse/ClickHouse/pull/6754) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- For stack traces gathered by query profiler, do not include stack frames generated by the query profiler itself. [#6250](https://github.com/ClickHouse/ClickHouse/pull/6250) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Now table functions `values`, `file`, `url`, `hdfs` have support for ALIAS columns. [#6255](https://github.com/ClickHouse/ClickHouse/pull/6255) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Throw an exception if `config.d` file doesn’t have the corresponding root element as the config file. [#6123](https://github.com/ClickHouse/ClickHouse/pull/6123) ([dimarub2000](https://github.com/dimarub2000)) +- Print extra info in exception message for `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)) +- When determining shards of a `Distributed` table to be covered by a read query (for `optimize_skip_unused_shards` = 1) ClickHouse now checks conditions from both `prewhere` and `where` clauses of select statement. [#6521](https://github.com/ClickHouse/ClickHouse/pull/6521) ([Alexander Kazakov](https://github.com/Akazz)) +- Enabled `SIMDJSON` for machines without AVX2 but with SSE 4.2 and PCLMUL instruction set. [#6285](https://github.com/ClickHouse/ClickHouse/issues/6285) [#6320](https://github.com/ClickHouse/ClickHouse/pull/6320) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- ClickHouse can work on filesystems without `O_DIRECT` support (such as ZFS and BtrFS) without additional tuning. [#4449](https://github.com/ClickHouse/ClickHouse/issues/4449) [#6730](https://github.com/ClickHouse/ClickHouse/pull/6730) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Support push down predicate for final subquery. [#6120](https://github.com/ClickHouse/ClickHouse/pull/6120) ([TCeason](https://github.com/TCeason)) [#6162](https://github.com/ClickHouse/ClickHouse/pull/6162) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Better `JOIN ON` keys extraction [#6131](https://github.com/ClickHouse/ClickHouse/pull/6131) ([Artem Zuikov](https://github.com/4ertus2)) +- Upated `SIMDJSON`. [#6285](https://github.com/ClickHouse/ClickHouse/issues/6285). [#6306](https://github.com/ClickHouse/ClickHouse/pull/6306) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Optimize selecting of smallest column for `SELECT count()` query. [#6344](https://github.com/ClickHouse/ClickHouse/pull/6344) ([Amos Bird](https://github.com/amosbird)) +- Added `strict` parameter in `windowFunnel()`. When the `strict` is set, the `windowFunnel()` applies conditions only for the unique values. [#6548](https://github.com/ClickHouse/ClickHouse/pull/6548) ([achimbab](https://github.com/achimbab)) +- Safer interface of `mysqlxx::Pool`. [#6150](https://github.com/ClickHouse/ClickHouse/pull/6150) ([avasiliev](https://github.com/avasiliev)) +- Options line size when executing with `--help` option now corresponds with terminal size. [#6590](https://github.com/ClickHouse/ClickHouse/pull/6590) ([dimarub2000](https://github.com/dimarub2000)) +- Disable “read in order” optimization for aggregation without keys. [#6599](https://github.com/ClickHouse/ClickHouse/pull/6599) ([Anton Popov](https://github.com/CurtizJ)) +- HTTP status code for `INCORRECT_DATA` and `TYPE_MISMATCH` error codes was changed from default `500 Internal Server Error` to `400 Bad Request`. [#6271](https://github.com/ClickHouse/ClickHouse/pull/6271) ([Alexander Rodin](https://github.com/a-rodin)) +- Move Join object from `ExpressionAction` into `AnalyzedJoin`. `ExpressionAnalyzer` and `ExpressionAction` do not know about `Join` class anymore. Its logic is hidden by `AnalyzedJoin` iface. [#6801](https://github.com/ClickHouse/ClickHouse/pull/6801) ([Artem Zuikov](https://github.com/4ertus2)) +- Fixed possible deadlock of distributed queries when one of shards is localhost but the query is sent via network connection. [#6759](https://github.com/ClickHouse/ClickHouse/pull/6759) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Changed semantic of multiple tables `RENAME` to avoid possible deadlocks. [#6757](https://github.com/ClickHouse/ClickHouse/issues/6757). [#6756](https://github.com/ClickHouse/ClickHouse/pull/6756) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Rewritten MySQL compatibility server to prevent loading full packet payload in memory. Decreased memory consumption for each connection to approximately `2 * DBMS_DEFAULT_BUFFER_SIZE` (read/write buffers). [#5811](https://github.com/ClickHouse/ClickHouse/pull/5811) ([Yuriy Baranov](https://github.com/yurriy)) +- Move AST alias interpreting logic out of parser that doesn’t have to know anything about query semantics. [#6108](https://github.com/ClickHouse/ClickHouse/pull/6108) ([Artem Zuikov](https://github.com/4ertus2)) +- Slightly more safe parsing of `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`: Allow use `where_condition` from config with `partition_key` alias in query for checking partition existence (Earlier it was used only in reading data queries). [#6577](https://github.com/ClickHouse/ClickHouse/pull/6577) ([proller](https://github.com/proller)) +- Added optional message argument in `throwIf`. ([#5772](https://github.com/ClickHouse/ClickHouse/issues/5772)) [#6329](https://github.com/ClickHouse/ClickHouse/pull/6329) ([Vdimir](https://github.com/Vdimir)) +- Server exception got while sending insertion data is now being processed in client as well. [#5891](https://github.com/ClickHouse/ClickHouse/issues/5891) [#6711](https://github.com/ClickHouse/ClickHouse/pull/6711) ([dimarub2000](https://github.com/dimarub2000)) +- Added a metric `DistributedFilesToInsert` that shows the total number of files in filesystem that are selected to send to remote servers by Distributed tables. The number is summed across all shards. [#6600](https://github.com/ClickHouse/ClickHouse/pull/6600) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Move most of JOINs prepare logic from `ExpressionAction/ExpressionAnalyzer` to `AnalyzedJoin`. [#6785](https://github.com/ClickHouse/ClickHouse/pull/6785) ([Artem Zuikov](https://github.com/4ertus2)) +- Fix TSan [warning](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)) +- Better information messages about lack of Linux capabilities. Logging fatal errors with “fatal” level, that will make it easier to find in `system.text_log`. [#6441](https://github.com/ClickHouse/ClickHouse/pull/6441) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- When enable dumping temporary data to the disk to restrict memory usage during `GROUP BY`, `ORDER BY`, it didn’t check the free disk space. The fix add a new setting `min_free_disk_space`, when the free disk space it smaller then the threshold, the query will stop and throw `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)) +- Removed recursive rwlock by thread. It makes no sense, because threads are reused between queries. `SELECT` query may acquire a lock in one thread, hold a lock from another thread and exit from first thread. In the same time, first thread can be reused by `DROP` query. This will lead to false “Attempt to acquire exclusive lock recursively” messages. [#6771](https://github.com/ClickHouse/ClickHouse/pull/6771) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Split `ExpressionAnalyzer.appendJoin()`. Prepare a place in `ExpressionAnalyzer` for `MergeJoin`. [#6524](https://github.com/ClickHouse/ClickHouse/pull/6524) ([Artem Zuikov](https://github.com/4ertus2)) +- Added `mysql_native_password` authentication plugin to MySQL compatibility server. [#6194](https://github.com/ClickHouse/ClickHouse/pull/6194) ([Yuriy Baranov](https://github.com/yurriy)) +- Less number of `clock_gettime` calls; fixed ABI compatibility between debug/release in `Allocator` (insignificant issue). [#6197](https://github.com/ClickHouse/ClickHouse/pull/6197) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Move `collectUsedColumns` from `ExpressionAnalyzer` to `SyntaxAnalyzer`. `SyntaxAnalyzer` makes `required_source_columns` itself now. [#6416](https://github.com/ClickHouse/ClickHouse/pull/6416) ([Artem Zuikov](https://github.com/4ertus2)) +- Add setting `joined_subquery_requires_alias` to require aliases for subselects and table functions in `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)) +- Extract `GetAggregatesVisitor` class from `ExpressionAnalyzer`. [#6458](https://github.com/ClickHouse/ClickHouse/pull/6458) ([Artem Zuikov](https://github.com/4ertus2)) +- `system.query_log`: change data type of `type` column to `Enum`. [#6265](https://github.com/ClickHouse/ClickHouse/pull/6265) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +- Static linking of `sha256_password` authentication plugin. [#6512](https://github.com/ClickHouse/ClickHouse/pull/6512) ([Yuriy Baranov](https://github.com/yurriy)) +- Avoid extra dependency for the setting `compile` to work. In previous versions, the user may get error like `cannot open crti.o`, `unable to find library -lc` etc. [#6309](https://github.com/ClickHouse/ClickHouse/pull/6309) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- More validation of the input that may come from malicious replica. [#6303](https://github.com/ClickHouse/ClickHouse/pull/6303) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Now `clickhouse-obfuscator` file is available in `clickhouse-client` package. In previous versions it was available as `clickhouse obfuscator` (with whitespace). [#5816](https://github.com/ClickHouse/ClickHouse/issues/5816) [#6609](https://github.com/ClickHouse/ClickHouse/pull/6609) ([dimarub2000](https://github.com/dimarub2000)) +- Fixed deadlock when we have at least two queries that read at least two tables in different order and another query that performs DDL operation on one of tables. Fixed another very rare deadlock. [#6764](https://github.com/ClickHouse/ClickHouse/pull/6764) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Added `os_thread_ids` column to `system.processes` and `system.query_log` for better debugging possibilities. [#6763](https://github.com/ClickHouse/ClickHouse/pull/6763) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- A workaround for PHP mysqlnd extension bugs which occur when `sha256_password` is used as a default authentication plugin (described in [#6031](https://github.com/ClickHouse/ClickHouse/issues/6031)). [#6113](https://github.com/ClickHouse/ClickHouse/pull/6113) ([Yuriy Baranov](https://github.com/yurriy)) +- Remove unneeded place with changed nullability columns. [#6693](https://github.com/ClickHouse/ClickHouse/pull/6693) ([Artem Zuikov](https://github.com/4ertus2)) +- Set default value of `queue_max_wait_ms` to zero, because current value (five seconds) makes no sense. There are rare circumstances when this settings has any use. Added settings `replace_running_query_max_wait_ms`, `kafka_max_wait_ms` and `connection_pool_max_wait_ms` for disambiguation. [#6692](https://github.com/ClickHouse/ClickHouse/pull/6692) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Extract `SelectQueryExpressionAnalyzer` from `ExpressionAnalyzer`. Keep the last one for non-select queries. [#6499](https://github.com/ClickHouse/ClickHouse/pull/6499) ([Artem Zuikov](https://github.com/4ertus2)) +- Removed duplicating input and output formats. [#6239](https://github.com/ClickHouse/ClickHouse/pull/6239) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Allow user to override `poll_interval` and `idle_connection_timeout` settings on connection. [#6230](https://github.com/ClickHouse/ClickHouse/pull/6230) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- `MergeTree` now has an additional option `ttl_only_drop_parts` (disabled by default) to avoid partial pruning of parts, so that they dropped completely when all the rows in a part are expired. [#6191](https://github.com/ClickHouse/ClickHouse/pull/6191) ([Sergi Vladykin](https://github.com/svladykin)) +- Type checks for set index functions. Throw exception if function got a wrong type. This fixes fuzz test with UBSan. [#6511](https://github.com/ClickHouse/ClickHouse/pull/6511) ([Nikita Vasilev](https://github.com/nikvas0)) #### Performance Improvement {#performance-improvement-2} -- Optimize queries with `ORDER BY expressions` clause, where `expressions` have coinciding prefix with sorting key in `MergeTree` tables. This optimization is controlled by `optimize_read_in_order` setting. [\#6054](https://github.com/ClickHouse/ClickHouse/pull/6054) [\#6629](https://github.com/ClickHouse/ClickHouse/pull/6629) ([Anton Popov](https://github.com/CurtizJ)) -- Allow to use multiple threads during parts loading and removal. [\#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)) -- Implemented batch variant of updating aggregate function states. It may lead to performance benefits. [\#6435](https://github.com/ClickHouse/ClickHouse/pull/6435) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Using `FastOps` library for functions `exp`, `log`, `sigmoid`, `tanh`. FastOps is a fast vector math library from Michael Parakhin (Yandex CTO). Improved performance of `exp` and `log` functions more than 6 times. The functions `exp` and `log` from `Float32` argument will return `Float32` (in previous versions they always return `Float64`). Now `exp(nan)` may return `inf`. The result of `exp` and `log` functions may be not the nearest machine representable number to the true answer. [\#6254](https://github.com/ClickHouse/ClickHouse/pull/6254) ([alexey-milovidov](https://github.com/alexey-milovidov)) Using Danila Kutenin variant to make fastops working [\#6317](https://github.com/ClickHouse/ClickHouse/pull/6317) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Disable consecutive key optimization for `UInt8/16`. [\#6298](https://github.com/ClickHouse/ClickHouse/pull/6298) [\#6701](https://github.com/ClickHouse/ClickHouse/pull/6701) ([akuzm](https://github.com/akuzm)) -- Improved performance of `simdjson` library by getting rid of dynamic allocation in `ParsedJson::Iterator`. [\#6479](https://github.com/ClickHouse/ClickHouse/pull/6479) ([Vitaly Baranov](https://github.com/vitlibar)) -- Pre-fault pages when allocating memory with `mmap()`. [\#6667](https://github.com/ClickHouse/ClickHouse/pull/6667) ([akuzm](https://github.com/akuzm)) -- Fix performance bug in `Decimal` comparison. [\#6380](https://github.com/ClickHouse/ClickHouse/pull/6380) ([Artem Zuikov](https://github.com/4ertus2)) +- Optimize queries with `ORDER BY expressions` clause, where `expressions` have coinciding prefix with sorting key in `MergeTree` tables. This optimization is controlled by `optimize_read_in_order` setting. [#6054](https://github.com/ClickHouse/ClickHouse/pull/6054) [#6629](https://github.com/ClickHouse/ClickHouse/pull/6629) ([Anton Popov](https://github.com/CurtizJ)) +- Allow to use multiple threads during parts loading and removal. [#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)) +- Implemented batch variant of updating aggregate function states. It may lead to performance benefits. [#6435](https://github.com/ClickHouse/ClickHouse/pull/6435) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Using `FastOps` library for functions `exp`, `log`, `sigmoid`, `tanh`. FastOps is a fast vector math library from Michael Parakhin (Yandex CTO). Improved performance of `exp` and `log` functions more than 6 times. The functions `exp` and `log` from `Float32` argument will return `Float32` (in previous versions they always return `Float64`). Now `exp(nan)` may return `inf`. The result of `exp` and `log` functions may be not the nearest machine representable number to the true answer. [#6254](https://github.com/ClickHouse/ClickHouse/pull/6254) ([alexey-milovidov](https://github.com/alexey-milovidov)) Using Danila Kutenin variant to make fastops working [#6317](https://github.com/ClickHouse/ClickHouse/pull/6317) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Disable consecutive key optimization for `UInt8/16`. [#6298](https://github.com/ClickHouse/ClickHouse/pull/6298) [#6701](https://github.com/ClickHouse/ClickHouse/pull/6701) ([akuzm](https://github.com/akuzm)) +- Improved performance of `simdjson` library by getting rid of dynamic allocation in `ParsedJson::Iterator`. [#6479](https://github.com/ClickHouse/ClickHouse/pull/6479) ([Vitaly Baranov](https://github.com/vitlibar)) +- Pre-fault pages when allocating memory with `mmap()`. [#6667](https://github.com/ClickHouse/ClickHouse/pull/6667) ([akuzm](https://github.com/akuzm)) +- Fix performance bug in `Decimal` comparison. [#6380](https://github.com/ClickHouse/ClickHouse/pull/6380) ([Artem Zuikov](https://github.com/4ertus2)) #### Build/Testing/Packaging Improvement {#buildtestingpackaging-improvement-4} -- Remove Compiler (runtime template instantiation) because we’ve win over it’s performance. [\#6646](https://github.com/ClickHouse/ClickHouse/pull/6646) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Added performance test to show degradation of performance in gcc-9 in more isolated way. [\#6302](https://github.com/ClickHouse/ClickHouse/pull/6302) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Added table function `numbers_mt`, which is multithreaded version of `numbers`. Updated performance tests with hash functions. [\#6554](https://github.com/ClickHouse/ClickHouse/pull/6554) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Comparison mode in `clickhouse-benchmark` [\#6220](https://github.com/ClickHouse/ClickHouse/issues/6220) [\#6343](https://github.com/ClickHouse/ClickHouse/pull/6343) ([dimarub2000](https://github.com/dimarub2000)) -- Best effort for printing stack traces. Also added `SIGPROF` as a debugging signal to print stack trace of a running thread. [\#6529](https://github.com/ClickHouse/ClickHouse/pull/6529) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Every function in its own file, part 10. [\#6321](https://github.com/ClickHouse/ClickHouse/pull/6321) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Remove doubled const `TABLE_IS_READ_ONLY`. [\#6566](https://github.com/ClickHouse/ClickHouse/pull/6566) ([filimonov](https://github.com/filimonov)) -- Formatting changes for `StringHashMap` PR [\#5417](https://github.com/ClickHouse/ClickHouse/issues/5417). [\#6700](https://github.com/ClickHouse/ClickHouse/pull/6700) ([akuzm](https://github.com/akuzm)) -- Better subquery for join creation in `ExpressionAnalyzer`. [\#6824](https://github.com/ClickHouse/ClickHouse/pull/6824) ([Artem Zuikov](https://github.com/4ertus2)) -- Remove a redundant condition (found by PVS Studio). [\#6775](https://github.com/ClickHouse/ClickHouse/pull/6775) ([akuzm](https://github.com/akuzm)) -- Separate the hash table interface for `ReverseIndex`. [\#6672](https://github.com/ClickHouse/ClickHouse/pull/6672) ([akuzm](https://github.com/akuzm)) -- Refactoring of settings. [\#6689](https://github.com/ClickHouse/ClickHouse/pull/6689) ([alesapin](https://github.com/alesapin)) -- Add comments for `set` index functions. [\#6319](https://github.com/ClickHouse/ClickHouse/pull/6319) ([Nikita Vasilev](https://github.com/nikvas0)) -- Increase OOM score in debug version on Linux. [\#6152](https://github.com/ClickHouse/ClickHouse/pull/6152) ([akuzm](https://github.com/akuzm)) -- HDFS HA now work in debug build. [\#6650](https://github.com/ClickHouse/ClickHouse/pull/6650) ([Weiqing Xu](https://github.com/weiqxu)) -- Added a test to `transform_query_for_external_database`. [\#6388](https://github.com/ClickHouse/ClickHouse/pull/6388) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Add test for multiple materialized views for Kafka table. [\#6509](https://github.com/ClickHouse/ClickHouse/pull/6509) ([Ivan](https://github.com/abyss7)) -- Make a better build scheme. [\#6500](https://github.com/ClickHouse/ClickHouse/pull/6500) ([Ivan](https://github.com/abyss7)) -- Fixed `test_external_dictionaries` integration in case it was executed under non root user. [\#6507](https://github.com/ClickHouse/ClickHouse/pull/6507) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- The bug reproduces when total size of written packets exceeds `DBMS_DEFAULT_BUFFER_SIZE`. [\#6204](https://github.com/ClickHouse/ClickHouse/pull/6204) ([Yuriy Baranov](https://github.com/yurriy)) -- Added a test for `RENAME` table race condition [\#6752](https://github.com/ClickHouse/ClickHouse/pull/6752) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Avoid data race on Settings in `KILL QUERY`. [\#6753](https://github.com/ClickHouse/ClickHouse/pull/6753) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Add integration test for handling errors by a cache dictionary. [\#6755](https://github.com/ClickHouse/ClickHouse/pull/6755) ([Vitaly Baranov](https://github.com/vitlibar)) -- Disable parsing of ELF object files on Mac OS, because it makes no sense. [\#6578](https://github.com/ClickHouse/ClickHouse/pull/6578) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Attempt to make changelog generator better. [\#6327](https://github.com/ClickHouse/ClickHouse/pull/6327) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Adding `-Wshadow` switch to the GCC. [\#6325](https://github.com/ClickHouse/ClickHouse/pull/6325) ([kreuzerkrieg](https://github.com/kreuzerkrieg)) -- Removed obsolete code for `mimalloc` support. [\#6715](https://github.com/ClickHouse/ClickHouse/pull/6715) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- `zlib-ng` determines x86 capabilities and saves this info to global variables. This is done in defalteInit call, which may be made by different threads simultaneously. To avoid multithreaded writes, do it on library startup. [\#6141](https://github.com/ClickHouse/ClickHouse/pull/6141) ([akuzm](https://github.com/akuzm)) -- Regression test for a bug which in join which was fixed in [\#5192](https://github.com/ClickHouse/ClickHouse/issues/5192). [\#6147](https://github.com/ClickHouse/ClickHouse/pull/6147) ([Bakhtiyor Ruziev](https://github.com/theruziev)) -- Fixed MSan report. [\#6144](https://github.com/ClickHouse/ClickHouse/pull/6144) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix flapping TTL test. [\#6782](https://github.com/ClickHouse/ClickHouse/pull/6782) ([Anton Popov](https://github.com/CurtizJ)) -- Fixed false data race in `MergeTreeDataPart::is_frozen` field. [\#6583](https://github.com/ClickHouse/ClickHouse/pull/6583) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed timeouts in fuzz test. In previous version, it managed to find false hangup in query `SELECT * FROM numbers_mt(gccMurmurHash(''))`. [\#6582](https://github.com/ClickHouse/ClickHouse/pull/6582) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Added debug checks to `static_cast` of columns. [\#6581](https://github.com/ClickHouse/ClickHouse/pull/6581) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Support for Oracle Linux in official RPM packages. [\#6356](https://github.com/ClickHouse/ClickHouse/issues/6356) [\#6585](https://github.com/ClickHouse/ClickHouse/pull/6585) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Changed json perftests from `once` to `loop` type. [\#6536](https://github.com/ClickHouse/ClickHouse/pull/6536) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- `odbc-bridge.cpp` defines `main()` so it should not be included in `clickhouse-lib`. [\#6538](https://github.com/ClickHouse/ClickHouse/pull/6538) ([Orivej Desh](https://github.com/orivej)) -- Test for crash in `FULL|RIGHT JOIN` with nulls in right table’s keys. [\#6362](https://github.com/ClickHouse/ClickHouse/pull/6362) ([Artem Zuikov](https://github.com/4ertus2)) -- Added a test for the limit on expansion of aliases just in case. [\#6442](https://github.com/ClickHouse/ClickHouse/pull/6442) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Switched from `boost::filesystem` to `std::filesystem` where appropriate. [\#6253](https://github.com/ClickHouse/ClickHouse/pull/6253) [\#6385](https://github.com/ClickHouse/ClickHouse/pull/6385) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Added RPM packages to website. [\#6251](https://github.com/ClickHouse/ClickHouse/pull/6251) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Add a test for fixed `Unknown identifier` exception in `IN` section. [\#6708](https://github.com/ClickHouse/ClickHouse/pull/6708) ([Artem Zuikov](https://github.com/4ertus2)) -- Simplify `shared_ptr_helper` because people facing difficulties understanding it. [\#6675](https://github.com/ClickHouse/ClickHouse/pull/6675) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Added performance tests for fixed Gorilla and DoubleDelta codec. [\#6179](https://github.com/ClickHouse/ClickHouse/pull/6179) ([Vasily Nemkov](https://github.com/Enmk)) -- Split the integration test `test_dictionaries` into 4 separate tests. [\#6776](https://github.com/ClickHouse/ClickHouse/pull/6776) ([Vitaly Baranov](https://github.com/vitlibar)) -- Fix PVS-Studio warning in `PipelineExecutor`. [\#6777](https://github.com/ClickHouse/ClickHouse/pull/6777) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Allow to use `library` dictionary source with ASan. [\#6482](https://github.com/ClickHouse/ClickHouse/pull/6482) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Added option to generate changelog from a list of PRs. [\#6350](https://github.com/ClickHouse/ClickHouse/pull/6350) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Lock the `TinyLog` storage when reading. [\#6226](https://github.com/ClickHouse/ClickHouse/pull/6226) ([akuzm](https://github.com/akuzm)) -- Check for broken symlinks in CI. [\#6634](https://github.com/ClickHouse/ClickHouse/pull/6634) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Increase timeout for “stack overflow” test because it may take a long time in debug build. [\#6637](https://github.com/ClickHouse/ClickHouse/pull/6637) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Added a check for double whitespaces. [\#6643](https://github.com/ClickHouse/ClickHouse/pull/6643) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix `new/delete` memory tracking when build with sanitizers. Tracking is not clear. It only prevents memory limit exceptions in tests. [\#6450](https://github.com/ClickHouse/ClickHouse/pull/6450) ([Artem Zuikov](https://github.com/4ertus2)) -- Enable back the check of undefined symbols while linking. [\#6453](https://github.com/ClickHouse/ClickHouse/pull/6453) ([Ivan](https://github.com/abyss7)) -- Avoid rebuilding `hyperscan` every day. [\#6307](https://github.com/ClickHouse/ClickHouse/pull/6307) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed UBSan report in `ProtobufWriter`. [\#6163](https://github.com/ClickHouse/ClickHouse/pull/6163) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Don’t allow to use query profiler with sanitizers because it is not compatible. [\#6769](https://github.com/ClickHouse/ClickHouse/pull/6769) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Add test for reloading a dictionary after fail by timer. [\#6114](https://github.com/ClickHouse/ClickHouse/pull/6114) ([Vitaly Baranov](https://github.com/vitlibar)) -- Fix inconsistency in `PipelineExecutor::prepareProcessor` argument type. [\#6494](https://github.com/ClickHouse/ClickHouse/pull/6494) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Added a test for bad URIs. [\#6493](https://github.com/ClickHouse/ClickHouse/pull/6493) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Added more checks to `CAST` function. This should get more information about segmentation fault in fuzzy test. [\#6346](https://github.com/ClickHouse/ClickHouse/pull/6346) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Added `gcc-9` support to `docker/builder` container that builds image locally. [\#6333](https://github.com/ClickHouse/ClickHouse/pull/6333) ([Gleb Novikov](https://github.com/NanoBjorn)) -- Test for primary key with `LowCardinality(String)`. [\#5044](https://github.com/ClickHouse/ClickHouse/issues/5044) [\#6219](https://github.com/ClickHouse/ClickHouse/pull/6219) ([dimarub2000](https://github.com/dimarub2000)) -- Fixed tests affected by slow stack traces printing. [\#6315](https://github.com/ClickHouse/ClickHouse/pull/6315) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Add a test case for crash in `groupUniqArray` fixed in [\#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)) -- Fixed indices mutations tests. [\#6645](https://github.com/ClickHouse/ClickHouse/pull/6645) ([Nikita Vasilev](https://github.com/nikvas0)) -- In performance test, do not read query log for queries we didn’t run. [\#6427](https://github.com/ClickHouse/ClickHouse/pull/6427) ([akuzm](https://github.com/akuzm)) -- Materialized view now could be created with any low cardinality types regardless to the setting about suspicious low cardinality types. [\#6428](https://github.com/ClickHouse/ClickHouse/pull/6428) ([Olga Khvostikova](https://github.com/stavrolia)) -- Updated tests for `send_logs_level` setting. [\#6207](https://github.com/ClickHouse/ClickHouse/pull/6207) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Fix build under gcc-8.2. [\#6196](https://github.com/ClickHouse/ClickHouse/pull/6196) ([Max Akhmedov](https://github.com/zlobober)) -- Fix build with internal libc++. [\#6724](https://github.com/ClickHouse/ClickHouse/pull/6724) ([Ivan](https://github.com/abyss7)) -- Fix shared build with `rdkafka` library [\#6101](https://github.com/ClickHouse/ClickHouse/pull/6101) ([Ivan](https://github.com/abyss7)) -- Fixes for Mac OS build (incomplete). [\#6390](https://github.com/ClickHouse/ClickHouse/pull/6390) ([alexey-milovidov](https://github.com/alexey-milovidov)) [\#6429](https://github.com/ClickHouse/ClickHouse/pull/6429) ([alex-zaitsev](https://github.com/alex-zaitsev)) -- Fix “splitted” build. [\#6618](https://github.com/ClickHouse/ClickHouse/pull/6618) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Other build fixes: [\#6186](https://github.com/ClickHouse/ClickHouse/pull/6186) ([Amos Bird](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) ([proller](https://github.com/proller)) +- Remove Compiler (runtime template instantiation) because we’ve win over it’s performance. [#6646](https://github.com/ClickHouse/ClickHouse/pull/6646) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Added performance test to show degradation of performance in gcc-9 in more isolated way. [#6302](https://github.com/ClickHouse/ClickHouse/pull/6302) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Added table function `numbers_mt`, which is multithreaded version of `numbers`. Updated performance tests with hash functions. [#6554](https://github.com/ClickHouse/ClickHouse/pull/6554) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Comparison mode in `clickhouse-benchmark` [#6220](https://github.com/ClickHouse/ClickHouse/issues/6220) [#6343](https://github.com/ClickHouse/ClickHouse/pull/6343) ([dimarub2000](https://github.com/dimarub2000)) +- Best effort for printing stack traces. Also added `SIGPROF` as a debugging signal to print stack trace of a running thread. [#6529](https://github.com/ClickHouse/ClickHouse/pull/6529) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Every function in its own file, part 10. [#6321](https://github.com/ClickHouse/ClickHouse/pull/6321) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Remove doubled const `TABLE_IS_READ_ONLY`. [#6566](https://github.com/ClickHouse/ClickHouse/pull/6566) ([filimonov](https://github.com/filimonov)) +- Formatting changes for `StringHashMap` PR [#5417](https://github.com/ClickHouse/ClickHouse/issues/5417). [#6700](https://github.com/ClickHouse/ClickHouse/pull/6700) ([akuzm](https://github.com/akuzm)) +- Better subquery for join creation in `ExpressionAnalyzer`. [#6824](https://github.com/ClickHouse/ClickHouse/pull/6824) ([Artem Zuikov](https://github.com/4ertus2)) +- Remove a redundant condition (found by PVS Studio). [#6775](https://github.com/ClickHouse/ClickHouse/pull/6775) ([akuzm](https://github.com/akuzm)) +- Separate the hash table interface for `ReverseIndex`. [#6672](https://github.com/ClickHouse/ClickHouse/pull/6672) ([akuzm](https://github.com/akuzm)) +- Refactoring of settings. [#6689](https://github.com/ClickHouse/ClickHouse/pull/6689) ([alesapin](https://github.com/alesapin)) +- Add comments for `set` index functions. [#6319](https://github.com/ClickHouse/ClickHouse/pull/6319) ([Nikita Vasilev](https://github.com/nikvas0)) +- Increase OOM score in debug version on Linux. [#6152](https://github.com/ClickHouse/ClickHouse/pull/6152) ([akuzm](https://github.com/akuzm)) +- HDFS HA now work in debug build. [#6650](https://github.com/ClickHouse/ClickHouse/pull/6650) ([Weiqing Xu](https://github.com/weiqxu)) +- Added a test to `transform_query_for_external_database`. [#6388](https://github.com/ClickHouse/ClickHouse/pull/6388) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Add test for multiple materialized views for Kafka table. [#6509](https://github.com/ClickHouse/ClickHouse/pull/6509) ([Ivan](https://github.com/abyss7)) +- Make a better build scheme. [#6500](https://github.com/ClickHouse/ClickHouse/pull/6500) ([Ivan](https://github.com/abyss7)) +- Fixed `test_external_dictionaries` integration in case it was executed under non root user. [#6507](https://github.com/ClickHouse/ClickHouse/pull/6507) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- The bug reproduces when total size of written packets exceeds `DBMS_DEFAULT_BUFFER_SIZE`. [#6204](https://github.com/ClickHouse/ClickHouse/pull/6204) ([Yuriy Baranov](https://github.com/yurriy)) +- Added a test for `RENAME` table race condition [#6752](https://github.com/ClickHouse/ClickHouse/pull/6752) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Avoid data race on Settings in `KILL QUERY`. [#6753](https://github.com/ClickHouse/ClickHouse/pull/6753) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Add integration test for handling errors by a cache dictionary. [#6755](https://github.com/ClickHouse/ClickHouse/pull/6755) ([Vitaly Baranov](https://github.com/vitlibar)) +- Disable parsing of ELF object files on Mac OS, because it makes no sense. [#6578](https://github.com/ClickHouse/ClickHouse/pull/6578) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Attempt to make changelog generator better. [#6327](https://github.com/ClickHouse/ClickHouse/pull/6327) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Adding `-Wshadow` switch to the GCC. [#6325](https://github.com/ClickHouse/ClickHouse/pull/6325) ([kreuzerkrieg](https://github.com/kreuzerkrieg)) +- Removed obsolete code for `mimalloc` support. [#6715](https://github.com/ClickHouse/ClickHouse/pull/6715) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- `zlib-ng` determines x86 capabilities and saves this info to global variables. This is done in defalteInit call, which may be made by different threads simultaneously. To avoid multithreaded writes, do it on library startup. [#6141](https://github.com/ClickHouse/ClickHouse/pull/6141) ([akuzm](https://github.com/akuzm)) +- Regression test for a bug which in join which was fixed in [#5192](https://github.com/ClickHouse/ClickHouse/issues/5192). [#6147](https://github.com/ClickHouse/ClickHouse/pull/6147) ([Bakhtiyor Ruziev](https://github.com/theruziev)) +- Fixed MSan report. [#6144](https://github.com/ClickHouse/ClickHouse/pull/6144) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix flapping TTL test. [#6782](https://github.com/ClickHouse/ClickHouse/pull/6782) ([Anton Popov](https://github.com/CurtizJ)) +- Fixed false data race in `MergeTreeDataPart::is_frozen` field. [#6583](https://github.com/ClickHouse/ClickHouse/pull/6583) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed timeouts in fuzz test. In previous version, it managed to find false hangup in query `SELECT * FROM numbers_mt(gccMurmurHash(''))`. [#6582](https://github.com/ClickHouse/ClickHouse/pull/6582) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Added debug checks to `static_cast` of columns. [#6581](https://github.com/ClickHouse/ClickHouse/pull/6581) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Support for Oracle Linux in official RPM packages. [#6356](https://github.com/ClickHouse/ClickHouse/issues/6356) [#6585](https://github.com/ClickHouse/ClickHouse/pull/6585) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Changed json perftests from `once` to `loop` type. [#6536](https://github.com/ClickHouse/ClickHouse/pull/6536) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- `odbc-bridge.cpp` defines `main()` so it should not be included in `clickhouse-lib`. [#6538](https://github.com/ClickHouse/ClickHouse/pull/6538) ([Orivej Desh](https://github.com/orivej)) +- Test for crash in `FULL|RIGHT JOIN` with nulls in right table’s keys. [#6362](https://github.com/ClickHouse/ClickHouse/pull/6362) ([Artem Zuikov](https://github.com/4ertus2)) +- Added a test for the limit on expansion of aliases just in case. [#6442](https://github.com/ClickHouse/ClickHouse/pull/6442) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Switched from `boost::filesystem` to `std::filesystem` where appropriate. [#6253](https://github.com/ClickHouse/ClickHouse/pull/6253) [#6385](https://github.com/ClickHouse/ClickHouse/pull/6385) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Added RPM packages to website. [#6251](https://github.com/ClickHouse/ClickHouse/pull/6251) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Add a test for fixed `Unknown identifier` exception in `IN` section. [#6708](https://github.com/ClickHouse/ClickHouse/pull/6708) ([Artem Zuikov](https://github.com/4ertus2)) +- Simplify `shared_ptr_helper` because people facing difficulties understanding it. [#6675](https://github.com/ClickHouse/ClickHouse/pull/6675) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Added performance tests for fixed Gorilla and DoubleDelta codec. [#6179](https://github.com/ClickHouse/ClickHouse/pull/6179) ([Vasily Nemkov](https://github.com/Enmk)) +- Split the integration test `test_dictionaries` into 4 separate tests. [#6776](https://github.com/ClickHouse/ClickHouse/pull/6776) ([Vitaly Baranov](https://github.com/vitlibar)) +- Fix PVS-Studio warning in `PipelineExecutor`. [#6777](https://github.com/ClickHouse/ClickHouse/pull/6777) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Allow to use `library` dictionary source with ASan. [#6482](https://github.com/ClickHouse/ClickHouse/pull/6482) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Added option to generate changelog from a list of PRs. [#6350](https://github.com/ClickHouse/ClickHouse/pull/6350) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Lock the `TinyLog` storage when reading. [#6226](https://github.com/ClickHouse/ClickHouse/pull/6226) ([akuzm](https://github.com/akuzm)) +- Check for broken symlinks in CI. [#6634](https://github.com/ClickHouse/ClickHouse/pull/6634) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Increase timeout for “stack overflow” test because it may take a long time in debug build. [#6637](https://github.com/ClickHouse/ClickHouse/pull/6637) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Added a check for double whitespaces. [#6643](https://github.com/ClickHouse/ClickHouse/pull/6643) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix `new/delete` memory tracking when build with sanitizers. Tracking is not clear. It only prevents memory limit exceptions in tests. [#6450](https://github.com/ClickHouse/ClickHouse/pull/6450) ([Artem Zuikov](https://github.com/4ertus2)) +- Enable back the check of undefined symbols while linking. [#6453](https://github.com/ClickHouse/ClickHouse/pull/6453) ([Ivan](https://github.com/abyss7)) +- Avoid rebuilding `hyperscan` every day. [#6307](https://github.com/ClickHouse/ClickHouse/pull/6307) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed UBSan report in `ProtobufWriter`. [#6163](https://github.com/ClickHouse/ClickHouse/pull/6163) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Don’t allow to use query profiler with sanitizers because it is not compatible. [#6769](https://github.com/ClickHouse/ClickHouse/pull/6769) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Add test for reloading a dictionary after fail by timer. [#6114](https://github.com/ClickHouse/ClickHouse/pull/6114) ([Vitaly Baranov](https://github.com/vitlibar)) +- Fix inconsistency in `PipelineExecutor::prepareProcessor` argument type. [#6494](https://github.com/ClickHouse/ClickHouse/pull/6494) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Added a test for bad URIs. [#6493](https://github.com/ClickHouse/ClickHouse/pull/6493) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Added more checks to `CAST` function. This should get more information about segmentation fault in fuzzy test. [#6346](https://github.com/ClickHouse/ClickHouse/pull/6346) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Added `gcc-9` support to `docker/builder` container that builds image locally. [#6333](https://github.com/ClickHouse/ClickHouse/pull/6333) ([Gleb Novikov](https://github.com/NanoBjorn)) +- Test for primary key with `LowCardinality(String)`. [#5044](https://github.com/ClickHouse/ClickHouse/issues/5044) [#6219](https://github.com/ClickHouse/ClickHouse/pull/6219) ([dimarub2000](https://github.com/dimarub2000)) +- Fixed tests affected by slow stack traces printing. [#6315](https://github.com/ClickHouse/ClickHouse/pull/6315) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Add a test case for crash in `groupUniqArray` fixed in [#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)) +- Fixed indices mutations tests. [#6645](https://github.com/ClickHouse/ClickHouse/pull/6645) ([Nikita Vasilev](https://github.com/nikvas0)) +- In performance test, do not read query log for queries we didn’t run. [#6427](https://github.com/ClickHouse/ClickHouse/pull/6427) ([akuzm](https://github.com/akuzm)) +- Materialized view now could be created with any low cardinality types regardless to the setting about suspicious low cardinality types. [#6428](https://github.com/ClickHouse/ClickHouse/pull/6428) ([Olga Khvostikova](https://github.com/stavrolia)) +- Updated tests for `send_logs_level` setting. [#6207](https://github.com/ClickHouse/ClickHouse/pull/6207) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fix build under gcc-8.2. [#6196](https://github.com/ClickHouse/ClickHouse/pull/6196) ([Max Akhmedov](https://github.com/zlobober)) +- Fix build with internal libc++. [#6724](https://github.com/ClickHouse/ClickHouse/pull/6724) ([Ivan](https://github.com/abyss7)) +- Fix shared build with `rdkafka` library [#6101](https://github.com/ClickHouse/ClickHouse/pull/6101) ([Ivan](https://github.com/abyss7)) +- Fixes for Mac OS build (incomplete). [#6390](https://github.com/ClickHouse/ClickHouse/pull/6390) ([alexey-milovidov](https://github.com/alexey-milovidov)) [#6429](https://github.com/ClickHouse/ClickHouse/pull/6429) ([alex-zaitsev](https://github.com/alex-zaitsev)) +- Fix “splitted” build. [#6618](https://github.com/ClickHouse/ClickHouse/pull/6618) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Other build fixes: [#6186](https://github.com/ClickHouse/ClickHouse/pull/6186) ([Amos Bird](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) ([proller](https://github.com/proller)) #### Backward Incompatible Change {#backward-incompatible-change-3} -- Removed rarely used table function `catBoostPool` and storage `CatBoostPool`. If you have used this table function, please write email to `clickhouse-feedback@yandex-team.com`. Note that CatBoost integration remains and will be supported. [\#6279](https://github.com/ClickHouse/ClickHouse/pull/6279) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Disable `ANY RIGHT JOIN` and `ANY FULL JOIN` by default. Set `any_join_distinct_right_table_keys` setting to enable them. [\#5126](https://github.com/ClickHouse/ClickHouse/issues/5126) [\#6351](https://github.com/ClickHouse/ClickHouse/pull/6351) ([Artem Zuikov](https://github.com/4ertus2)) +- Removed rarely used table function `catBoostPool` and storage `CatBoostPool`. If you have used this table function, please write email to `clickhouse-feedback@yandex-team.com`. Note that CatBoost integration remains and will be supported. [#6279](https://github.com/ClickHouse/ClickHouse/pull/6279) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Disable `ANY RIGHT JOIN` and `ANY FULL JOIN` by default. Set `any_join_distinct_right_table_keys` setting to enable them. [#5126](https://github.com/ClickHouse/ClickHouse/issues/5126) [#6351](https://github.com/ClickHouse/ClickHouse/pull/6351) ([Artem Zuikov](https://github.com/4ertus2)) ## ClickHouse Release 19.13 {#clickhouse-release-19-13} @@ -797,89 +797,89 @@ toc_title: '2019' #### Bug Fix {#bug-fix-10} - This release also contains all bug fixes from 19.14.6.12. -- Fixed possible inconsistent state of table while executing `DROP` query for replicated table while zookeeper is not accessible. [\#6045](https://github.com/ClickHouse/ClickHouse/issues/6045) [\#6413](https://github.com/ClickHouse/ClickHouse/pull/6413) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -- Fix for data race in StorageMerge [\#6717](https://github.com/ClickHouse/ClickHouse/pull/6717) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix bug introduced in query profiler which leads to endless recv from socket. [\#6386](https://github.com/ClickHouse/ClickHouse/pull/6386) ([alesapin](https://github.com/alesapin)) -- Fix excessive CPU usage while executing `JSONExtractRaw` function over a boolean value. [\#6208](https://github.com/ClickHouse/ClickHouse/pull/6208) ([Vitaly Baranov](https://github.com/vitlibar)) -- Fixes the regression while pushing to materialized view. [\#6415](https://github.com/ClickHouse/ClickHouse/pull/6415) ([Ivan](https://github.com/abyss7)) -- Table function `url` had the vulnerability allowed the attacker to inject arbitrary HTTP headers in the request. This issue was found by [Nikita Tikhomirov](https://github.com/NSTikhomirov). [\#6466](https://github.com/ClickHouse/ClickHouse/pull/6466) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix useless `AST` check in Set index. [\#6510](https://github.com/ClickHouse/ClickHouse/issues/6510) [\#6651](https://github.com/ClickHouse/ClickHouse/pull/6651) ([Nikita Vasilev](https://github.com/nikvas0)) -- Fixed parsing of `AggregateFunction` values embedded in query. [\#6575](https://github.com/ClickHouse/ClickHouse/issues/6575) [\#6773](https://github.com/ClickHouse/ClickHouse/pull/6773) ([Zhichang Yu](https://github.com/yuzhichang)) -- Fixed wrong behaviour of `trim` functions family. [\#6647](https://github.com/ClickHouse/ClickHouse/pull/6647) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed possible inconsistent state of table while executing `DROP` query for replicated table while zookeeper is not accessible. [#6045](https://github.com/ClickHouse/ClickHouse/issues/6045) [#6413](https://github.com/ClickHouse/ClickHouse/pull/6413) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +- Fix for data race in StorageMerge [#6717](https://github.com/ClickHouse/ClickHouse/pull/6717) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix bug introduced in query profiler which leads to endless recv from socket. [#6386](https://github.com/ClickHouse/ClickHouse/pull/6386) ([alesapin](https://github.com/alesapin)) +- Fix excessive CPU usage while executing `JSONExtractRaw` function over a boolean value. [#6208](https://github.com/ClickHouse/ClickHouse/pull/6208) ([Vitaly Baranov](https://github.com/vitlibar)) +- Fixes the regression while pushing to materialized view. [#6415](https://github.com/ClickHouse/ClickHouse/pull/6415) ([Ivan](https://github.com/abyss7)) +- Table function `url` had the vulnerability allowed the attacker to inject arbitrary HTTP headers in the request. This issue was found by [Nikita Tikhomirov](https://github.com/NSTikhomirov). [#6466](https://github.com/ClickHouse/ClickHouse/pull/6466) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix useless `AST` check in Set index. [#6510](https://github.com/ClickHouse/ClickHouse/issues/6510) [#6651](https://github.com/ClickHouse/ClickHouse/pull/6651) ([Nikita Vasilev](https://github.com/nikvas0)) +- Fixed parsing of `AggregateFunction` values embedded in query. [#6575](https://github.com/ClickHouse/ClickHouse/issues/6575) [#6773](https://github.com/ClickHouse/ClickHouse/pull/6773) ([Zhichang Yu](https://github.com/yuzhichang)) +- Fixed wrong behaviour of `trim` functions family. [#6647](https://github.com/ClickHouse/ClickHouse/pull/6647) ([alexey-milovidov](https://github.com/alexey-milovidov)) ### ClickHouse Release 19.13.4.32, 2019-09-10 {#clickhouse-release-19-13-4-32-2019-09-10} #### Bug Fix {#bug-fix-11} - This release also contains all bug security fixes from 19.11.9.52 and 19.11.10.54. -- Fixed data race in `system.parts` table and `ALTER` query. [\#6245](https://github.com/ClickHouse/ClickHouse/issues/6245) [\#6513](https://github.com/ClickHouse/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed mismatched header in streams happened in case of reading from empty distributed table with sample and 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)) -- Fixed crash when using `IN` clause with a subquery with a tuple. [\#6125](https://github.com/ClickHouse/ClickHouse/issues/6125) [\#6550](https://github.com/ClickHouse/ClickHouse/pull/6550) ([tavplubix](https://github.com/tavplubix)) -- Fix case with same column names in `GLOBAL JOIN ON` section. [\#6181](https://github.com/ClickHouse/ClickHouse/pull/6181) ([Artem Zuikov](https://github.com/4ertus2)) -- Fix crash when casting types to `Decimal` that do not support it. Throw exception instead. [\#6297](https://github.com/ClickHouse/ClickHouse/pull/6297) ([Artem Zuikov](https://github.com/4ertus2)) -- Fixed crash in `extractAll()` function. [\#6644](https://github.com/ClickHouse/ClickHouse/pull/6644) ([Artem Zuikov](https://github.com/4ertus2)) -- Query transformation for `MySQL`, `ODBC`, `JDBC` table functions now works properly for `SELECT WHERE` queries with multiple `AND` expressions. [\#6381](https://github.com/ClickHouse/ClickHouse/issues/6381) [\#6676](https://github.com/ClickHouse/ClickHouse/pull/6676) ([dimarub2000](https://github.com/dimarub2000)) -- Added previous declaration checks for MySQL 8 integration. [\#6569](https://github.com/ClickHouse/ClickHouse/pull/6569) ([Rafael David Tinoco](https://github.com/rafaeldtinoco)) +- Fixed data race in `system.parts` table and `ALTER` query. [#6245](https://github.com/ClickHouse/ClickHouse/issues/6245) [#6513](https://github.com/ClickHouse/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed mismatched header in streams happened in case of reading from empty distributed table with sample and 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)) +- Fixed crash when using `IN` clause with a subquery with a tuple. [#6125](https://github.com/ClickHouse/ClickHouse/issues/6125) [#6550](https://github.com/ClickHouse/ClickHouse/pull/6550) ([tavplubix](https://github.com/tavplubix)) +- Fix case with same column names in `GLOBAL JOIN ON` section. [#6181](https://github.com/ClickHouse/ClickHouse/pull/6181) ([Artem Zuikov](https://github.com/4ertus2)) +- Fix crash when casting types to `Decimal` that do not support it. Throw exception instead. [#6297](https://github.com/ClickHouse/ClickHouse/pull/6297) ([Artem Zuikov](https://github.com/4ertus2)) +- Fixed crash in `extractAll()` function. [#6644](https://github.com/ClickHouse/ClickHouse/pull/6644) ([Artem Zuikov](https://github.com/4ertus2)) +- Query transformation for `MySQL`, `ODBC`, `JDBC` table functions now works properly for `SELECT WHERE` queries with multiple `AND` expressions. [#6381](https://github.com/ClickHouse/ClickHouse/issues/6381) [#6676](https://github.com/ClickHouse/ClickHouse/pull/6676) ([dimarub2000](https://github.com/dimarub2000)) +- Added previous declaration checks for MySQL 8 integration. [#6569](https://github.com/ClickHouse/ClickHouse/pull/6569) ([Rafael David Tinoco](https://github.com/rafaeldtinoco)) #### Security Fix {#security-fix-1} -- Fix two vulnerabilities in codecs in decompression phase (malicious user can fabricate compressed data that will lead to buffer overflow in decompression). [\#6670](https://github.com/ClickHouse/ClickHouse/pull/6670) ([Artem Zuikov](https://github.com/4ertus2)) +- Fix two vulnerabilities in codecs in decompression phase (malicious user can fabricate compressed data that will lead to buffer overflow in decompression). [#6670](https://github.com/ClickHouse/ClickHouse/pull/6670) ([Artem Zuikov](https://github.com/4ertus2)) ### ClickHouse Release 19.13.3.26, 2019-08-22 {#clickhouse-release-19-13-3-26-2019-08-22} #### Bug Fix {#bug-fix-12} -- Fix `ALTER TABLE ... UPDATE` query for tables with `enable_mixed_granularity_parts=1`. [\#6543](https://github.com/ClickHouse/ClickHouse/pull/6543) ([alesapin](https://github.com/alesapin)) -- Fix NPE when using IN clause with a subquery with a tuple. [\#6125](https://github.com/ClickHouse/ClickHouse/issues/6125) [\#6550](https://github.com/ClickHouse/ClickHouse/pull/6550) ([tavplubix](https://github.com/tavplubix)) -- Fixed an issue that if a stale replica becomes alive, it may still have data parts that were removed by DROP PARTITION. [\#6522](https://github.com/ClickHouse/ClickHouse/issues/6522) [\#6523](https://github.com/ClickHouse/ClickHouse/pull/6523) ([tavplubix](https://github.com/tavplubix)) -- Fixed issue with parsing CSV [\#6426](https://github.com/ClickHouse/ClickHouse/issues/6426) [\#6559](https://github.com/ClickHouse/ClickHouse/pull/6559) ([tavplubix](https://github.com/tavplubix)) -- Fixed data race in system.parts table and ALTER query. This fixes [\#6245](https://github.com/ClickHouse/ClickHouse/issues/6245). [\#6513](https://github.com/ClickHouse/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed wrong code in mutations that may lead to memory corruption. Fixed segfault with read of address `0x14c0` that may happed due to concurrent `DROP TABLE` and `SELECT` from `system.parts` or `system.parts_columns`. Fixed race condition in preparation of mutation queries. Fixed deadlock caused by `OPTIMIZE` of Replicated tables and concurrent modification operations like ALTERs. [\#6514](https://github.com/ClickHouse/ClickHouse/pull/6514) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed possible data loss after `ALTER DELETE` query on table with skipping index. [\#6224](https://github.com/ClickHouse/ClickHouse/issues/6224) [\#6282](https://github.com/ClickHouse/ClickHouse/pull/6282) ([Nikita Vasilev](https://github.com/nikvas0)) +- Fix `ALTER TABLE ... UPDATE` query for tables with `enable_mixed_granularity_parts=1`. [#6543](https://github.com/ClickHouse/ClickHouse/pull/6543) ([alesapin](https://github.com/alesapin)) +- Fix NPE when using IN clause with a subquery with a tuple. [#6125](https://github.com/ClickHouse/ClickHouse/issues/6125) [#6550](https://github.com/ClickHouse/ClickHouse/pull/6550) ([tavplubix](https://github.com/tavplubix)) +- Fixed an issue that if a stale replica becomes alive, it may still have data parts that were removed by DROP PARTITION. [#6522](https://github.com/ClickHouse/ClickHouse/issues/6522) [#6523](https://github.com/ClickHouse/ClickHouse/pull/6523) ([tavplubix](https://github.com/tavplubix)) +- Fixed issue with parsing CSV [#6426](https://github.com/ClickHouse/ClickHouse/issues/6426) [#6559](https://github.com/ClickHouse/ClickHouse/pull/6559) ([tavplubix](https://github.com/tavplubix)) +- Fixed data race in system.parts table and ALTER query. This fixes [#6245](https://github.com/ClickHouse/ClickHouse/issues/6245). [#6513](https://github.com/ClickHouse/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed wrong code in mutations that may lead to memory corruption. Fixed segfault with read of address `0x14c0` that may happed due to concurrent `DROP TABLE` and `SELECT` from `system.parts` or `system.parts_columns`. Fixed race condition in preparation of mutation queries. Fixed deadlock caused by `OPTIMIZE` of Replicated tables and concurrent modification operations like ALTERs. [#6514](https://github.com/ClickHouse/ClickHouse/pull/6514) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed possible data loss after `ALTER DELETE` query on table with skipping index. [#6224](https://github.com/ClickHouse/ClickHouse/issues/6224) [#6282](https://github.com/ClickHouse/ClickHouse/pull/6282) ([Nikita Vasilev](https://github.com/nikvas0)) #### Security Fix {#security-fix-2} -- If the attacker has write access to ZooKeeper and is able to run custom server available from the network where ClickHouse run, it can create custom-built malicious server that will act as ClickHouse replica and register it in ZooKeeper. When another replica will fetch data part from malicious replica, it can force clickhouse-server to write to arbitrary path on filesystem. Found by Eldar Zaitov, information security team at Yandex. [\#6247](https://github.com/ClickHouse/ClickHouse/pull/6247) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- If the attacker has write access to ZooKeeper and is able to run custom server available from the network where ClickHouse run, it can create custom-built malicious server that will act as ClickHouse replica and register it in ZooKeeper. When another replica will fetch data part from malicious replica, it can force clickhouse-server to write to arbitrary path on filesystem. Found by Eldar Zaitov, information security team at Yandex. [#6247](https://github.com/ClickHouse/ClickHouse/pull/6247) ([alexey-milovidov](https://github.com/alexey-milovidov)) ### ClickHouse Release 19.13.2.19, 2019-08-14 {#clickhouse-release-19-13-2-19-2019-08-14} #### New Feature {#new-feature-5} -- Sampling profiler on query level. [Example](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) -- Allow to specify a list of columns with `COLUMNS('regexp')` expression that works like a more sophisticated variant of `*` asterisk. [\#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()` is now possible [\#6057](https://github.com/ClickHouse/ClickHouse/pull/6057) ([dimarub2000](https://github.com/dimarub2000)) -- Adam optimizer for stochastic gradient descent is used by default in `stochasticLinearRegression()` and `stochasticLogisticRegression()` aggregate functions, because it shows good quality without almost any tuning. [\#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) ([Andy Yang](https://github.com/andyyzh)) -- `RENAME` queries now work with all storages. [\#5953](https://github.com/ClickHouse/ClickHouse/pull/5953) ([Ivan](https://github.com/abyss7)) -- Now client receive logs from server with any desired level by setting `send_logs_level` regardless to the log level specified in server settings. [\#5964](https://github.com/ClickHouse/ClickHouse/pull/5964) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +- Sampling profiler on query level. [Example](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) +- Allow to specify a list of columns with `COLUMNS('regexp')` expression that works like a more sophisticated variant of `*` asterisk. [#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()` is now possible [#6057](https://github.com/ClickHouse/ClickHouse/pull/6057) ([dimarub2000](https://github.com/dimarub2000)) +- Adam optimizer for stochastic gradient descent is used by default in `stochasticLinearRegression()` and `stochasticLogisticRegression()` aggregate functions, because it shows good quality without almost any tuning. [#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) ([Andy Yang](https://github.com/andyyzh)) +- `RENAME` queries now work with all storages. [#5953](https://github.com/ClickHouse/ClickHouse/pull/5953) ([Ivan](https://github.com/abyss7)) +- Now client receive logs from server with any desired level by setting `send_logs_level` regardless to the log level specified in server settings. [#5964](https://github.com/ClickHouse/ClickHouse/pull/5964) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) #### Backward Incompatible Change {#backward-incompatible-change-4} -- The setting `input_format_defaults_for_omitted_fields` is enabled by default. Inserts in Distributed tables need this setting to be the same on cluster (you need to set it before rolling update). It enables calculation of complex default expressions for omitted fields in `JSONEachRow` and `CSV*` formats. It should be the expected behavior but may lead to negligible performance difference. [\#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)) +- The setting `input_format_defaults_for_omitted_fields` is enabled by default. Inserts in Distributed tables need this setting to be the same on cluster (you need to set it before rolling update). It enables calculation of complex default expressions for omitted fields in `JSONEachRow` and `CSV*` formats. It should be the expected behavior but may lead to negligible performance difference. [#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 Features {#experimental-features} -- New query processing pipeline. Use `experimental_use_processors=1` option to enable it. Use for your own trouble. [\#4914](https://github.com/ClickHouse/ClickHouse/pull/4914) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- New query processing pipeline. Use `experimental_use_processors=1` option to enable it. Use for your own trouble. [#4914](https://github.com/ClickHouse/ClickHouse/pull/4914) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) #### Bug Fix {#bug-fix-13} - Kafka integration has been fixed in this version. -- Fixed `DoubleDelta` encoding of `Int64` for large `DoubleDelta` values, improved `DoubleDelta` encoding for random data for `Int32`. [\#5998](https://github.com/ClickHouse/ClickHouse/pull/5998) ([Vasily Nemkov](https://github.com/Enmk)) -- Fixed overestimation of `max_rows_to_read` if the setting `merge_tree_uniform_read_distribution` is set to 0. [\#6019](https://github.com/ClickHouse/ClickHouse/pull/6019) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed `DoubleDelta` encoding of `Int64` for large `DoubleDelta` values, improved `DoubleDelta` encoding for random data for `Int32`. [#5998](https://github.com/ClickHouse/ClickHouse/pull/5998) ([Vasily Nemkov](https://github.com/Enmk)) +- Fixed overestimation of `max_rows_to_read` if the setting `merge_tree_uniform_read_distribution` is set to 0. [#6019](https://github.com/ClickHouse/ClickHouse/pull/6019) ([alexey-milovidov](https://github.com/alexey-milovidov)) #### Improvement {#improvement-4} -- Throws an exception if `config.d` file doesn’t have the corresponding root element as the config file [\#6123](https://github.com/ClickHouse/ClickHouse/pull/6123) ([dimarub2000](https://github.com/dimarub2000)) +- Throws an exception if `config.d` file doesn’t have the corresponding root element as the config file [#6123](https://github.com/ClickHouse/ClickHouse/pull/6123) ([dimarub2000](https://github.com/dimarub2000)) #### Performance Improvement {#performance-improvement-3} -- Optimize `count()`. Now it uses the smallest column (if possible). [\#6028](https://github.com/ClickHouse/ClickHouse/pull/6028) ([Amos Bird](https://github.com/amosbird)) +- Optimize `count()`. Now it uses the smallest column (if possible). [#6028](https://github.com/ClickHouse/ClickHouse/pull/6028) ([Amos Bird](https://github.com/amosbird)) #### Build/Testing/Packaging Improvement {#buildtestingpackaging-improvement-5} -- Report memory usage in performance tests. [\#5899](https://github.com/ClickHouse/ClickHouse/pull/5899) ([akuzm](https://github.com/akuzm)) -- Fix build with external `libcxx` [\#6010](https://github.com/ClickHouse/ClickHouse/pull/6010) ([Ivan](https://github.com/abyss7)) -- Fix shared build with `rdkafka` library [\#6101](https://github.com/ClickHouse/ClickHouse/pull/6101) ([Ivan](https://github.com/abyss7)) +- Report memory usage in performance tests. [#5899](https://github.com/ClickHouse/ClickHouse/pull/5899) ([akuzm](https://github.com/akuzm)) +- Fix build with external `libcxx` [#6010](https://github.com/ClickHouse/ClickHouse/pull/6010) ([Ivan](https://github.com/abyss7)) +- Fix shared build with `rdkafka` library [#6101](https://github.com/ClickHouse/ClickHouse/pull/6101) ([Ivan](https://github.com/abyss7)) ## ClickHouse Release 19.11 {#clickhouse-release-19-11} @@ -887,90 +887,90 @@ toc_title: '2019' #### Bug Fix {#bug-fix-14} -- Fixed rare crash in `ALTER MODIFY COLUMN` and vertical merge when one of merged/altered parts is empty (0 rows). [\#6780](https://github.com/ClickHouse/ClickHouse/pull/6780) ([alesapin](https://github.com/alesapin)) -- Manual update of `SIMDJSON`. This fixes possible flooding of stderr files with bogus json diagnostic messages. [\#7548](https://github.com/ClickHouse/ClickHouse/pull/7548) ([Alexander Kazakov](https://github.com/Akazz)) +- Fixed rare crash in `ALTER MODIFY COLUMN` and vertical merge when one of merged/altered parts is empty (0 rows). [#6780](https://github.com/ClickHouse/ClickHouse/pull/6780) ([alesapin](https://github.com/alesapin)) +- Manual update of `SIMDJSON`. This fixes possible flooding of stderr files with bogus json diagnostic messages. [#7548](https://github.com/ClickHouse/ClickHouse/pull/7548) ([Alexander Kazakov](https://github.com/Akazz)) - Fixed bug with `mrk` file extension for mutations ([alesapin](https://github.com/alesapin)) ### ClickHouse Release 19.11.12.69, 2019-10-02 {#clickhouse-release-19-11-12-69-2019-10-02} #### Bug Fix {#bug-fix-15} -- Fixed performance degradation of index analysis on complex keys on large tables. This fixes [\#6924](https://github.com/ClickHouse/ClickHouse/issues/6924). [\#7075](https://github.com/ClickHouse/ClickHouse/pull/7075) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Avoid rare SIGSEGV while sending data in tables with Distributed engine (`Failed to send batch: file with index XXXXX is absent`). [\#7032](https://github.com/ClickHouse/ClickHouse/pull/7032) ([Azat Khuzhin](https://github.com/azat)) -- Fix `Unknown identifier` with multiple joins. This fixes [\#5254](https://github.com/ClickHouse/ClickHouse/issues/5254). [\#7022](https://github.com/ClickHouse/ClickHouse/pull/7022) ([Artem Zuikov](https://github.com/4ertus2)) +- Fixed performance degradation of index analysis on complex keys on large tables. This fixes [#6924](https://github.com/ClickHouse/ClickHouse/issues/6924). [#7075](https://github.com/ClickHouse/ClickHouse/pull/7075) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Avoid rare SIGSEGV while sending data in tables with Distributed engine (`Failed to send batch: file with index XXXXX is absent`). [#7032](https://github.com/ClickHouse/ClickHouse/pull/7032) ([Azat Khuzhin](https://github.com/azat)) +- Fix `Unknown identifier` with multiple joins. This fixes [#5254](https://github.com/ClickHouse/ClickHouse/issues/5254). [#7022](https://github.com/ClickHouse/ClickHouse/pull/7022) ([Artem Zuikov](https://github.com/4ertus2)) ### ClickHouse Release 19.11.11.57, 2019-09-13 {#clickhouse-release-19-11-11-57-2019-09-13} -- Fix logical error causing segfaults when selecting from Kafka empty topic. [\#6902](https://github.com/ClickHouse/ClickHouse/issues/6902) [\#6909](https://github.com/ClickHouse/ClickHouse/pull/6909) ([Ivan](https://github.com/abyss7)) -- Fix for function `АrrayEnumerateUniqRanked` with empty arrays in params. [\#6928](https://github.com/ClickHouse/ClickHouse/pull/6928) ([proller](https://github.com/proller)) +- Fix logical error causing segfaults when selecting from Kafka empty topic. [#6902](https://github.com/ClickHouse/ClickHouse/issues/6902) [#6909](https://github.com/ClickHouse/ClickHouse/pull/6909) ([Ivan](https://github.com/abyss7)) +- Fix for function `АrrayEnumerateUniqRanked` with empty arrays in params. [#6928](https://github.com/ClickHouse/ClickHouse/pull/6928) ([proller](https://github.com/proller)) ### ClickHouse Release 19.11.10.54, 2019-09-10 {#clickhouse-release-19-11-10-54-2019-09-10} #### Bug Fix {#bug-fix-16} -- Do store offsets for Kafka messages manually to be able to commit them all at once for all partitions. Fixes potential duplication in “one consumer - many partitions” scenario. [\#6872](https://github.com/ClickHouse/ClickHouse/pull/6872) ([Ivan](https://github.com/abyss7)) +- Do store offsets for Kafka messages manually to be able to commit them all at once for all partitions. Fixes potential duplication in “one consumer - many partitions” scenario. [#6872](https://github.com/ClickHouse/ClickHouse/pull/6872) ([Ivan](https://github.com/abyss7)) ### ClickHouse Release 19.11.9.52, 2019-09-6 {#clickhouse-release-19-11-9-52-2019-09-6} -- Improve error handling in cache dictionaries. [\#6737](https://github.com/ClickHouse/ClickHouse/pull/6737) ([Vitaly Baranov](https://github.com/vitlibar)) -- Fixed bug in function `arrayEnumerateUniqRanked`. [\#6779](https://github.com/ClickHouse/ClickHouse/pull/6779) ([proller](https://github.com/proller)) -- Fix `JSONExtract` function while extracting a `Tuple` from JSON. [\#6718](https://github.com/ClickHouse/ClickHouse/pull/6718) ([Vitaly Baranov](https://github.com/vitlibar)) -- Fixed possible data loss after `ALTER DELETE` query on table with skipping index. [\#6224](https://github.com/ClickHouse/ClickHouse/issues/6224) [\#6282](https://github.com/ClickHouse/ClickHouse/pull/6282) ([Nikita Vasilev](https://github.com/nikvas0)) -- Fixed performance test. [\#6392](https://github.com/ClickHouse/ClickHouse/pull/6392) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Parquet: Fix reading boolean columns. [\#6579](https://github.com/ClickHouse/ClickHouse/pull/6579) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed wrong behaviour of `nullIf` function for constant arguments. [\#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)) -- Fix Kafka messages duplication problem on normal server restart. [\#6597](https://github.com/ClickHouse/ClickHouse/pull/6597) ([Ivan](https://github.com/abyss7)) -- Fixed an issue when long `ALTER UPDATE` or `ALTER DELETE` may prevent regular merges to run. Prevent mutations from executing if there is no enough free threads available. [\#6502](https://github.com/ClickHouse/ClickHouse/issues/6502) [\#6617](https://github.com/ClickHouse/ClickHouse/pull/6617) ([tavplubix](https://github.com/tavplubix)) -- Fixed error with processing “timezone” in server configuration file. [\#6709](https://github.com/ClickHouse/ClickHouse/pull/6709) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix kafka tests. [\#6805](https://github.com/ClickHouse/ClickHouse/pull/6805) ([Ivan](https://github.com/abyss7)) +- Improve error handling in cache dictionaries. [#6737](https://github.com/ClickHouse/ClickHouse/pull/6737) ([Vitaly Baranov](https://github.com/vitlibar)) +- Fixed bug in function `arrayEnumerateUniqRanked`. [#6779](https://github.com/ClickHouse/ClickHouse/pull/6779) ([proller](https://github.com/proller)) +- Fix `JSONExtract` function while extracting a `Tuple` from JSON. [#6718](https://github.com/ClickHouse/ClickHouse/pull/6718) ([Vitaly Baranov](https://github.com/vitlibar)) +- Fixed possible data loss after `ALTER DELETE` query on table with skipping index. [#6224](https://github.com/ClickHouse/ClickHouse/issues/6224) [#6282](https://github.com/ClickHouse/ClickHouse/pull/6282) ([Nikita Vasilev](https://github.com/nikvas0)) +- Fixed performance test. [#6392](https://github.com/ClickHouse/ClickHouse/pull/6392) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Parquet: Fix reading boolean columns. [#6579](https://github.com/ClickHouse/ClickHouse/pull/6579) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed wrong behaviour of `nullIf` function for constant arguments. [#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)) +- Fix Kafka messages duplication problem on normal server restart. [#6597](https://github.com/ClickHouse/ClickHouse/pull/6597) ([Ivan](https://github.com/abyss7)) +- Fixed an issue when long `ALTER UPDATE` or `ALTER DELETE` may prevent regular merges to run. Prevent mutations from executing if there is no enough free threads available. [#6502](https://github.com/ClickHouse/ClickHouse/issues/6502) [#6617](https://github.com/ClickHouse/ClickHouse/pull/6617) ([tavplubix](https://github.com/tavplubix)) +- Fixed error with processing “timezone” in server configuration file. [#6709](https://github.com/ClickHouse/ClickHouse/pull/6709) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix kafka tests. [#6805](https://github.com/ClickHouse/ClickHouse/pull/6805) ([Ivan](https://github.com/abyss7)) #### Security Fix {#security-fix-3} -- If the attacker has write access to ZooKeeper and is able to run custom server available from the network where ClickHouse runs, it can create custom-built malicious server that will act as ClickHouse replica and register it in ZooKeeper. When another replica will fetch data part from malicious replica, it can force clickhouse-server to write to arbitrary path on filesystem. Found by Eldar Zaitov, information security team at Yandex. [\#6247](https://github.com/ClickHouse/ClickHouse/pull/6247) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- If the attacker has write access to ZooKeeper and is able to run custom server available from the network where ClickHouse runs, it can create custom-built malicious server that will act as ClickHouse replica and register it in ZooKeeper. When another replica will fetch data part from malicious replica, it can force clickhouse-server to write to arbitrary path on filesystem. Found by Eldar Zaitov, information security team at Yandex. [#6247](https://github.com/ClickHouse/ClickHouse/pull/6247) ([alexey-milovidov](https://github.com/alexey-milovidov)) ### ClickHouse Release 19.11.8.46, 2019-08-22 {#clickhouse-release-19-11-8-46-2019-08-22} #### Bug Fix {#bug-fix-17} -- Fix `ALTER TABLE ... UPDATE` query for tables with `enable_mixed_granularity_parts=1`. [\#6543](https://github.com/ClickHouse/ClickHouse/pull/6543) ([alesapin](https://github.com/alesapin)) -- Fix NPE when using IN clause with a subquery with a tuple. [\#6125](https://github.com/ClickHouse/ClickHouse/issues/6125) [\#6550](https://github.com/ClickHouse/ClickHouse/pull/6550) ([tavplubix](https://github.com/tavplubix)) -- Fixed an issue that if a stale replica becomes alive, it may still have data parts that were removed by DROP PARTITION. [\#6522](https://github.com/ClickHouse/ClickHouse/issues/6522) [\#6523](https://github.com/ClickHouse/ClickHouse/pull/6523) ([tavplubix](https://github.com/tavplubix)) -- Fixed issue with parsing CSV [\#6426](https://github.com/ClickHouse/ClickHouse/issues/6426) [\#6559](https://github.com/ClickHouse/ClickHouse/pull/6559) ([tavplubix](https://github.com/tavplubix)) -- Fixed data race in system.parts table and ALTER query. This fixes [\#6245](https://github.com/ClickHouse/ClickHouse/issues/6245). [\#6513](https://github.com/ClickHouse/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed wrong code in mutations that may lead to memory corruption. Fixed segfault with read of address `0x14c0` that may happed due to concurrent `DROP TABLE` and `SELECT` from `system.parts` or `system.parts_columns`. Fixed race condition in preparation of mutation queries. Fixed deadlock caused by `OPTIMIZE` of Replicated tables and concurrent modification operations like ALTERs. [\#6514](https://github.com/ClickHouse/ClickHouse/pull/6514) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix `ALTER TABLE ... UPDATE` query for tables with `enable_mixed_granularity_parts=1`. [#6543](https://github.com/ClickHouse/ClickHouse/pull/6543) ([alesapin](https://github.com/alesapin)) +- Fix NPE when using IN clause with a subquery with a tuple. [#6125](https://github.com/ClickHouse/ClickHouse/issues/6125) [#6550](https://github.com/ClickHouse/ClickHouse/pull/6550) ([tavplubix](https://github.com/tavplubix)) +- Fixed an issue that if a stale replica becomes alive, it may still have data parts that were removed by DROP PARTITION. [#6522](https://github.com/ClickHouse/ClickHouse/issues/6522) [#6523](https://github.com/ClickHouse/ClickHouse/pull/6523) ([tavplubix](https://github.com/tavplubix)) +- Fixed issue with parsing CSV [#6426](https://github.com/ClickHouse/ClickHouse/issues/6426) [#6559](https://github.com/ClickHouse/ClickHouse/pull/6559) ([tavplubix](https://github.com/tavplubix)) +- Fixed data race in system.parts table and ALTER query. This fixes [#6245](https://github.com/ClickHouse/ClickHouse/issues/6245). [#6513](https://github.com/ClickHouse/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed wrong code in mutations that may lead to memory corruption. Fixed segfault with read of address `0x14c0` that may happed due to concurrent `DROP TABLE` and `SELECT` from `system.parts` or `system.parts_columns`. Fixed race condition in preparation of mutation queries. Fixed deadlock caused by `OPTIMIZE` of Replicated tables and concurrent modification operations like ALTERs. [#6514](https://github.com/ClickHouse/ClickHouse/pull/6514) ([alexey-milovidov](https://github.com/alexey-milovidov)) ### ClickHouse Release 19.11.7.40, 2019-08-14 {#clickhouse-release-19-11-7-40-2019-08-14} #### Bug Fix {#bug-fix-18} - Kafka integration has been fixed in this version. -- Fix segfault when using `arrayReduce` for constant arguments. [\#6326](https://github.com/ClickHouse/ClickHouse/pull/6326) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed `toFloat()` monotonicity. [\#6374](https://github.com/ClickHouse/ClickHouse/pull/6374) ([dimarub2000](https://github.com/dimarub2000)) -- Fix segfault with enabled `optimize_skip_unused_shards` and missing sharding key. [\#6384](https://github.com/ClickHouse/ClickHouse/pull/6384) ([CurtizJ](https://github.com/CurtizJ)) -- Fixed logic of `arrayEnumerateUniqRanked` function. [\#6423](https://github.com/ClickHouse/ClickHouse/pull/6423) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Removed extra verbose logging from MySQL handler. [\#6389](https://github.com/ClickHouse/ClickHouse/pull/6389) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix wrong behavior and possible segfaults in `topK` and `topKWeighted` aggregated functions. [\#6404](https://github.com/ClickHouse/ClickHouse/pull/6404) ([CurtizJ](https://github.com/CurtizJ)) -- Do not expose virtual columns in `system.columns` table. This is required for backward compatibility. [\#6406](https://github.com/ClickHouse/ClickHouse/pull/6406) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix bug with memory allocation for string fields in complex key cache dictionary. [\#6447](https://github.com/ClickHouse/ClickHouse/pull/6447) ([alesapin](https://github.com/alesapin)) -- Fix bug with enabling adaptive granularity when creating new replica for `Replicated*MergeTree` table. [\#6452](https://github.com/ClickHouse/ClickHouse/pull/6452) ([alesapin](https://github.com/alesapin)) -- Fix infinite loop when reading Kafka messages. [\#6354](https://github.com/ClickHouse/ClickHouse/pull/6354) ([abyss7](https://github.com/abyss7)) -- Fixed the possibility of a fabricated query to cause server crash due to stack overflow in SQL parser and possibility of stack overflow in `Merge` and `Distributed` tables [\#6433](https://github.com/ClickHouse/ClickHouse/pull/6433) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed Gorilla encoding error on small sequences. [\#6444](https://github.com/ClickHouse/ClickHouse/pull/6444) ([Enmk](https://github.com/Enmk)) +- Fix segfault when using `arrayReduce` for constant arguments. [#6326](https://github.com/ClickHouse/ClickHouse/pull/6326) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed `toFloat()` monotonicity. [#6374](https://github.com/ClickHouse/ClickHouse/pull/6374) ([dimarub2000](https://github.com/dimarub2000)) +- Fix segfault with enabled `optimize_skip_unused_shards` and missing sharding key. [#6384](https://github.com/ClickHouse/ClickHouse/pull/6384) ([CurtizJ](https://github.com/CurtizJ)) +- Fixed logic of `arrayEnumerateUniqRanked` function. [#6423](https://github.com/ClickHouse/ClickHouse/pull/6423) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Removed extra verbose logging from MySQL handler. [#6389](https://github.com/ClickHouse/ClickHouse/pull/6389) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix wrong behavior and possible segfaults in `topK` and `topKWeighted` aggregated functions. [#6404](https://github.com/ClickHouse/ClickHouse/pull/6404) ([CurtizJ](https://github.com/CurtizJ)) +- Do not expose virtual columns in `system.columns` table. This is required for backward compatibility. [#6406](https://github.com/ClickHouse/ClickHouse/pull/6406) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix bug with memory allocation for string fields in complex key cache dictionary. [#6447](https://github.com/ClickHouse/ClickHouse/pull/6447) ([alesapin](https://github.com/alesapin)) +- Fix bug with enabling adaptive granularity when creating new replica for `Replicated*MergeTree` table. [#6452](https://github.com/ClickHouse/ClickHouse/pull/6452) ([alesapin](https://github.com/alesapin)) +- Fix infinite loop when reading Kafka messages. [#6354](https://github.com/ClickHouse/ClickHouse/pull/6354) ([abyss7](https://github.com/abyss7)) +- Fixed the possibility of a fabricated query to cause server crash due to stack overflow in SQL parser and possibility of stack overflow in `Merge` and `Distributed` tables [#6433](https://github.com/ClickHouse/ClickHouse/pull/6433) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed Gorilla encoding error on small sequences. [#6444](https://github.com/ClickHouse/ClickHouse/pull/6444) ([Enmk](https://github.com/Enmk)) #### Improvement {#improvement-5} -- Allow user to override `poll_interval` and `idle_connection_timeout` settings on connection. [\#6230](https://github.com/ClickHouse/ClickHouse/pull/6230) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Allow user to override `poll_interval` and `idle_connection_timeout` settings on connection. [#6230](https://github.com/ClickHouse/ClickHouse/pull/6230) ([alexey-milovidov](https://github.com/alexey-milovidov)) ### ClickHouse Release 19.11.5.28, 2019-08-05 {#clickhouse-release-19-11-5-28-2019-08-05} #### Bug Fix {#bug-fix-19} -- Fixed the possibility of hanging queries when server is overloaded. [\#6301](https://github.com/ClickHouse/ClickHouse/pull/6301) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix FPE in yandexConsistentHash function. This fixes [\#6304](https://github.com/ClickHouse/ClickHouse/issues/6304). [\#6126](https://github.com/ClickHouse/ClickHouse/pull/6126) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed bug in conversion of `LowCardinality` types in `AggregateFunctionFactory`. This fixes [\#6257](https://github.com/ClickHouse/ClickHouse/issues/6257). [\#6281](https://github.com/ClickHouse/ClickHouse/pull/6281) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Fix parsing of `bool` settings from `true` and `false` strings in configuration files. [\#6278](https://github.com/ClickHouse/ClickHouse/pull/6278) ([alesapin](https://github.com/alesapin)) -- Fix rare bug with incompatible stream headers in queries to `Distributed` table over `MergeTree` table when part of `WHERE` moves to `PREWHERE`. [\#6236](https://github.com/ClickHouse/ClickHouse/pull/6236) ([alesapin](https://github.com/alesapin)) -- Fixed overflow in integer division of signed type to unsigned type. This fixes [\#6214](https://github.com/ClickHouse/ClickHouse/issues/6214). [\#6233](https://github.com/ClickHouse/ClickHouse/pull/6233) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed the possibility of hanging queries when server is overloaded. [#6301](https://github.com/ClickHouse/ClickHouse/pull/6301) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix FPE in yandexConsistentHash function. This fixes [#6304](https://github.com/ClickHouse/ClickHouse/issues/6304). [#6126](https://github.com/ClickHouse/ClickHouse/pull/6126) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed bug in conversion of `LowCardinality` types in `AggregateFunctionFactory`. This fixes [#6257](https://github.com/ClickHouse/ClickHouse/issues/6257). [#6281](https://github.com/ClickHouse/ClickHouse/pull/6281) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fix parsing of `bool` settings from `true` and `false` strings in configuration files. [#6278](https://github.com/ClickHouse/ClickHouse/pull/6278) ([alesapin](https://github.com/alesapin)) +- Fix rare bug with incompatible stream headers in queries to `Distributed` table over `MergeTree` table when part of `WHERE` moves to `PREWHERE`. [#6236](https://github.com/ClickHouse/ClickHouse/pull/6236) ([alesapin](https://github.com/alesapin)) +- Fixed overflow in integer division of signed type to unsigned type. This fixes [#6214](https://github.com/ClickHouse/ClickHouse/issues/6214). [#6233](https://github.com/ClickHouse/ClickHouse/pull/6233) ([alexey-milovidov](https://github.com/alexey-milovidov)) #### Backward Incompatible Change {#backward-incompatible-change-5} @@ -980,25 +980,25 @@ toc_title: '2019' #### Bug Fix {#bug-fix-20} -- Fix bug with writing secondary indices marks with adaptive granularity. [\#6126](https://github.com/ClickHouse/ClickHouse/pull/6126) ([alesapin](https://github.com/alesapin)) -- Fix `WITH ROLLUP` and `WITH CUBE` modifiers of `GROUP BY` with two-level aggregation. [\#6225](https://github.com/ClickHouse/ClickHouse/pull/6225) ([Anton Popov](https://github.com/CurtizJ)) -- Fixed hang in `JSONExtractRaw` function. Fixed [\#6195](https://github.com/ClickHouse/ClickHouse/issues/6195) [\#6198](https://github.com/ClickHouse/ClickHouse/pull/6198) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix segfault in ExternalLoader::reloadOutdated(). [\#6082](https://github.com/ClickHouse/ClickHouse/pull/6082) ([Vitaly Baranov](https://github.com/vitlibar)) -- Fixed the case when server may close listening sockets but not shutdown and continue serving remaining queries. You may end up with two running clickhouse-server processes. Sometimes, the server may return an error `bad_function_call` for remaining queries. [\#6231](https://github.com/ClickHouse/ClickHouse/pull/6231) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed useless and incorrect condition on update field for initial loading of external dictionaries via ODBC, MySQL, ClickHouse and HTTP. This fixes [\#6069](https://github.com/ClickHouse/ClickHouse/issues/6069) [\#6083](https://github.com/ClickHouse/ClickHouse/pull/6083) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed irrelevant exception in cast of `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)) -- Fix non-deterministic result of “uniq” aggregate function in extreme rare cases. The bug was present in all ClickHouse versions. [\#6058](https://github.com/ClickHouse/ClickHouse/pull/6058) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Segfault when we set a little bit too high CIDR on the function `IPv6CIDRToRange`. [\#6068](https://github.com/ClickHouse/ClickHouse/pull/6068) ([Guillaume Tassery](https://github.com/YiuRULE)) -- Fixed small memory leak when server throw many exceptions from many different contexts. [\#6144](https://github.com/ClickHouse/ClickHouse/pull/6144) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix the situation when consumer got paused before subscription and not resumed afterwards. [\#6075](https://github.com/ClickHouse/ClickHouse/pull/6075) ([Ivan](https://github.com/abyss7)) Note that Kafka is broken in this version. -- Clearing the Kafka data buffer from the previous read operation that was completed with an error [\#6026](https://github.com/ClickHouse/ClickHouse/pull/6026) ([Nikolay](https://github.com/bopohaa)) Note that Kafka is broken in this version. -- Since `StorageMergeTree::background_task_handle` is initialized in `startup()` the `MergeTreeBlockOutputStream::write()` may try to use it before initialization. Just check if it is initialized. [\#6080](https://github.com/ClickHouse/ClickHouse/pull/6080) ([Ivan](https://github.com/abyss7)) +- Fix bug with writing secondary indices marks with adaptive granularity. [#6126](https://github.com/ClickHouse/ClickHouse/pull/6126) ([alesapin](https://github.com/alesapin)) +- Fix `WITH ROLLUP` and `WITH CUBE` modifiers of `GROUP BY` with two-level aggregation. [#6225](https://github.com/ClickHouse/ClickHouse/pull/6225) ([Anton Popov](https://github.com/CurtizJ)) +- Fixed hang in `JSONExtractRaw` function. Fixed [#6195](https://github.com/ClickHouse/ClickHouse/issues/6195) [#6198](https://github.com/ClickHouse/ClickHouse/pull/6198) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix segfault in ExternalLoader::reloadOutdated(). [#6082](https://github.com/ClickHouse/ClickHouse/pull/6082) ([Vitaly Baranov](https://github.com/vitlibar)) +- Fixed the case when server may close listening sockets but not shutdown and continue serving remaining queries. You may end up with two running clickhouse-server processes. Sometimes, the server may return an error `bad_function_call` for remaining queries. [#6231](https://github.com/ClickHouse/ClickHouse/pull/6231) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed useless and incorrect condition on update field for initial loading of external dictionaries via ODBC, MySQL, ClickHouse and HTTP. This fixes [#6069](https://github.com/ClickHouse/ClickHouse/issues/6069) [#6083](https://github.com/ClickHouse/ClickHouse/pull/6083) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed irrelevant exception in cast of `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)) +- Fix non-deterministic result of “uniq” aggregate function in extreme rare cases. The bug was present in all ClickHouse versions. [#6058](https://github.com/ClickHouse/ClickHouse/pull/6058) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Segfault when we set a little bit too high CIDR on the function `IPv6CIDRToRange`. [#6068](https://github.com/ClickHouse/ClickHouse/pull/6068) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Fixed small memory leak when server throw many exceptions from many different contexts. [#6144](https://github.com/ClickHouse/ClickHouse/pull/6144) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix the situation when consumer got paused before subscription and not resumed afterwards. [#6075](https://github.com/ClickHouse/ClickHouse/pull/6075) ([Ivan](https://github.com/abyss7)) Note that Kafka is broken in this version. +- Clearing the Kafka data buffer from the previous read operation that was completed with an error [#6026](https://github.com/ClickHouse/ClickHouse/pull/6026) ([Nikolay](https://github.com/bopohaa)) Note that Kafka is broken in this version. +- Since `StorageMergeTree::background_task_handle` is initialized in `startup()` the `MergeTreeBlockOutputStream::write()` may try to use it before initialization. Just check if it is initialized. [#6080](https://github.com/ClickHouse/ClickHouse/pull/6080) ([Ivan](https://github.com/abyss7)) #### Build/Testing/Packaging Improvement {#buildtestingpackaging-improvement-6} -- Added official `rpm` packages. [\#5740](https://github.com/ClickHouse/ClickHouse/pull/5740) ([proller](https://github.com/proller)) ([alesapin](https://github.com/alesapin)) -- Add an ability to build `.rpm` and `.tgz` packages with `packager` script. [\#5769](https://github.com/ClickHouse/ClickHouse/pull/5769) ([alesapin](https://github.com/alesapin)) -- Fixes for “Arcadia” build system. [\#6223](https://github.com/ClickHouse/ClickHouse/pull/6223) ([proller](https://github.com/proller)) +- Added official `rpm` packages. [#5740](https://github.com/ClickHouse/ClickHouse/pull/5740) ([proller](https://github.com/proller)) ([alesapin](https://github.com/alesapin)) +- Add an ability to build `.rpm` and `.tgz` packages with `packager` script. [#5769](https://github.com/ClickHouse/ClickHouse/pull/5769) ([alesapin](https://github.com/alesapin)) +- Fixes for “Arcadia” build system. [#6223](https://github.com/ClickHouse/ClickHouse/pull/6223) ([proller](https://github.com/proller)) #### Backward Incompatible Change {#backward-incompatible-change-6} @@ -1008,112 +1008,112 @@ toc_title: '2019' #### New Feature {#new-feature-6} -- Added support for prepared statements. [\#5331](https://github.com/ClickHouse/ClickHouse/pull/5331/) ([Alexander](https://github.com/sanych73)) [\#5630](https://github.com/ClickHouse/ClickHouse/pull/5630) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- `DoubleDelta` and `Gorilla` column codecs [\#5600](https://github.com/ClickHouse/ClickHouse/pull/5600) ([Vasily Nemkov](https://github.com/Enmk)) -- Added `os_thread_priority` setting that allows to control the “nice” value of query processing threads that is used by OS to adjust dynamic scheduling priority. It requires `CAP_SYS_NICE` capabilities to work. This implements [\#5858](https://github.com/ClickHouse/ClickHouse/issues/5858) [\#5909](https://github.com/ClickHouse/ClickHouse/pull/5909) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Implement `_topic`, `_offset`, `_key` columns for Kafka engine [\#5382](https://github.com/ClickHouse/ClickHouse/pull/5382) ([Ivan](https://github.com/abyss7)) Note that Kafka is broken in this version. -- Add aggregate function combinator `-Resample` [\#5590](https://github.com/ClickHouse/ClickHouse/pull/5590) ([hcz](https://github.com/hczhcz)) -- Aggregate functions `groupArrayMovingSum(win_size)(x)` and `groupArrayMovingAvg(win_size)(x)`, which calculate moving sum/avg with or without window-size limitation. [\#5595](https://github.com/ClickHouse/ClickHouse/pull/5595) ([inv2004](https://github.com/inv2004)) -- Add synonim `arrayFlatten` \<-\> `flatten` [\#5764](https://github.com/ClickHouse/ClickHouse/pull/5764) ([hcz](https://github.com/hczhcz)) -- Intergate H3 function `geoToH3` from 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)) +- Added support for prepared statements. [#5331](https://github.com/ClickHouse/ClickHouse/pull/5331/) ([Alexander](https://github.com/sanych73)) [#5630](https://github.com/ClickHouse/ClickHouse/pull/5630) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- `DoubleDelta` and `Gorilla` column codecs [#5600](https://github.com/ClickHouse/ClickHouse/pull/5600) ([Vasily Nemkov](https://github.com/Enmk)) +- Added `os_thread_priority` setting that allows to control the “nice” value of query processing threads that is used by OS to adjust dynamic scheduling priority. It requires `CAP_SYS_NICE` capabilities to work. This implements [#5858](https://github.com/ClickHouse/ClickHouse/issues/5858) [#5909](https://github.com/ClickHouse/ClickHouse/pull/5909) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Implement `_topic`, `_offset`, `_key` columns for Kafka engine [#5382](https://github.com/ClickHouse/ClickHouse/pull/5382) ([Ivan](https://github.com/abyss7)) Note that Kafka is broken in this version. +- Add aggregate function combinator `-Resample` [#5590](https://github.com/ClickHouse/ClickHouse/pull/5590) ([hcz](https://github.com/hczhcz)) +- Aggregate functions `groupArrayMovingSum(win_size)(x)` and `groupArrayMovingAvg(win_size)(x)`, which calculate moving sum/avg with or without window-size limitation. [#5595](https://github.com/ClickHouse/ClickHouse/pull/5595) ([inv2004](https://github.com/inv2004)) +- Add synonim `arrayFlatten` \<-\> `flatten` [#5764](https://github.com/ClickHouse/ClickHouse/pull/5764) ([hcz](https://github.com/hczhcz)) +- Intergate H3 function `geoToH3` from 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)) #### Bug Fix {#bug-fix-21} -- Implement DNS cache with asynchronous update. Separate thread resolves all hosts and updates DNS cache with period (setting `dns_cache_update_period`). It should help, when ip of hosts changes frequently. [\#5857](https://github.com/ClickHouse/ClickHouse/pull/5857) ([Anton Popov](https://github.com/CurtizJ)) -- Fix segfault in `Delta` codec which affects columns with values less than 32 bits size. The bug led to random memory corruption. [\#5786](https://github.com/ClickHouse/ClickHouse/pull/5786) ([alesapin](https://github.com/alesapin)) -- Fix segfault in TTL merge with non-physical columns in block. [\#5819](https://github.com/ClickHouse/ClickHouse/pull/5819) ([Anton Popov](https://github.com/CurtizJ)) -- Fix rare bug in checking of part with `LowCardinality` column. Previously `checkDataPart` always fails for part with `LowCardinality` column. [\#5832](https://github.com/ClickHouse/ClickHouse/pull/5832) ([alesapin](https://github.com/alesapin)) -- Avoid hanging connections when server thread pool is full. It is important for connections from `remote` table function or connections to a shard without replicas when there is long connection timeout. This fixes [\#5878](https://github.com/ClickHouse/ClickHouse/issues/5878) [\#5881](https://github.com/ClickHouse/ClickHouse/pull/5881) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Support for constant arguments to `evalMLModel` function. This fixes [\#5817](https://github.com/ClickHouse/ClickHouse/issues/5817) [\#5820](https://github.com/ClickHouse/ClickHouse/pull/5820) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed the issue when ClickHouse determines default time zone as `UCT` instead of `UTC`. This fixes [\#5804](https://github.com/ClickHouse/ClickHouse/issues/5804). [\#5828](https://github.com/ClickHouse/ClickHouse/pull/5828) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed buffer underflow in `visitParamExtractRaw`. This fixes [\#5901](https://github.com/ClickHouse/ClickHouse/issues/5901) [\#5902](https://github.com/ClickHouse/ClickHouse/pull/5902) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Now distributed `DROP/ALTER/TRUNCATE/OPTIMIZE ON CLUSTER` queries will be executed directly on leader replica. [\#5757](https://github.com/ClickHouse/ClickHouse/pull/5757) ([alesapin](https://github.com/alesapin)) -- Fix `coalesce` for `ColumnConst` with `ColumnNullable` + related changes. [\#5755](https://github.com/ClickHouse/ClickHouse/pull/5755) ([Artem Zuikov](https://github.com/4ertus2)) -- Fix the `ReadBufferFromKafkaConsumer` so that it keeps reading new messages after `commit()` even if it was stalled before [\#5852](https://github.com/ClickHouse/ClickHouse/pull/5852) ([Ivan](https://github.com/abyss7)) -- Fix `FULL` and `RIGHT` JOIN results when joining on `Nullable` keys in right table. [\#5859](https://github.com/ClickHouse/ClickHouse/pull/5859) ([Artem Zuikov](https://github.com/4ertus2)) -- Possible fix of infinite sleeping of low-priority queries. [\#5842](https://github.com/ClickHouse/ClickHouse/pull/5842) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix race condition, which cause that some queries may not appear in query\_log after `SYSTEM FLUSH LOGS` query. [\#5456](https://github.com/ClickHouse/ClickHouse/issues/5456) [\#5685](https://github.com/ClickHouse/ClickHouse/pull/5685) ([Anton Popov](https://github.com/CurtizJ)) -- Fixed `heap-use-after-free` ASan warning in ClusterCopier caused by watch which try to use already removed copier object. [\#5871](https://github.com/ClickHouse/ClickHouse/pull/5871) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Fixed wrong `StringRef` pointer returned by some implementations of `IColumn::deserializeAndInsertFromArena`. This bug affected only unit-tests. [\#5973](https://github.com/ClickHouse/ClickHouse/pull/5973) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Prevent source and intermediate array join columns of masking same name columns. [\#5941](https://github.com/ClickHouse/ClickHouse/pull/5941) ([Artem Zuikov](https://github.com/4ertus2)) -- Fix insert and select query to MySQL engine with MySQL style identifier quoting. [\#5704](https://github.com/ClickHouse/ClickHouse/pull/5704) ([Winter Zhang](https://github.com/zhang2014)) -- Now `CHECK TABLE` query can work with MergeTree engine family. It returns check status and message if any for each part (or file in case of simplier engines). Also, fix bug in fetch of a broken part. [\#5865](https://github.com/ClickHouse/ClickHouse/pull/5865) ([alesapin](https://github.com/alesapin)) -- Fix SPLIT\_SHARED\_LIBRARIES runtime [\#5793](https://github.com/ClickHouse/ClickHouse/pull/5793) ([Danila Kutenin](https://github.com/danlark1)) -- Fixed time zone initialization when `/etc/localtime` is a relative symlink like `../usr/share/zoneinfo/Europe/Moscow` [\#5922](https://github.com/ClickHouse/ClickHouse/pull/5922) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- clickhouse-copier: Fix use-after free on shutdown [\#5752](https://github.com/ClickHouse/ClickHouse/pull/5752) ([proller](https://github.com/proller)) -- Updated `simdjson`. Fixed the issue that some invalid JSONs with zero bytes successfully parse. [\#5938](https://github.com/ClickHouse/ClickHouse/pull/5938) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix shutdown of SystemLogs [\#5802](https://github.com/ClickHouse/ClickHouse/pull/5802) ([Anton Popov](https://github.com/CurtizJ)) -- Fix hanging when condition in invalidate\_query depends on a dictionary. [\#6011](https://github.com/ClickHouse/ClickHouse/pull/6011) ([Vitaly Baranov](https://github.com/vitlibar)) +- Implement DNS cache with asynchronous update. Separate thread resolves all hosts and updates DNS cache with period (setting `dns_cache_update_period`). It should help, when ip of hosts changes frequently. [#5857](https://github.com/ClickHouse/ClickHouse/pull/5857) ([Anton Popov](https://github.com/CurtizJ)) +- Fix segfault in `Delta` codec which affects columns with values less than 32 bits size. The bug led to random memory corruption. [#5786](https://github.com/ClickHouse/ClickHouse/pull/5786) ([alesapin](https://github.com/alesapin)) +- Fix segfault in TTL merge with non-physical columns in block. [#5819](https://github.com/ClickHouse/ClickHouse/pull/5819) ([Anton Popov](https://github.com/CurtizJ)) +- Fix rare bug in checking of part with `LowCardinality` column. Previously `checkDataPart` always fails for part with `LowCardinality` column. [#5832](https://github.com/ClickHouse/ClickHouse/pull/5832) ([alesapin](https://github.com/alesapin)) +- Avoid hanging connections when server thread pool is full. It is important for connections from `remote` table function or connections to a shard without replicas when there is long connection timeout. This fixes [#5878](https://github.com/ClickHouse/ClickHouse/issues/5878) [#5881](https://github.com/ClickHouse/ClickHouse/pull/5881) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Support for constant arguments to `evalMLModel` function. This fixes [#5817](https://github.com/ClickHouse/ClickHouse/issues/5817) [#5820](https://github.com/ClickHouse/ClickHouse/pull/5820) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed the issue when ClickHouse determines default time zone as `UCT` instead of `UTC`. This fixes [#5804](https://github.com/ClickHouse/ClickHouse/issues/5804). [#5828](https://github.com/ClickHouse/ClickHouse/pull/5828) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed buffer underflow in `visitParamExtractRaw`. This fixes [#5901](https://github.com/ClickHouse/ClickHouse/issues/5901) [#5902](https://github.com/ClickHouse/ClickHouse/pull/5902) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Now distributed `DROP/ALTER/TRUNCATE/OPTIMIZE ON CLUSTER` queries will be executed directly on leader replica. [#5757](https://github.com/ClickHouse/ClickHouse/pull/5757) ([alesapin](https://github.com/alesapin)) +- Fix `coalesce` for `ColumnConst` with `ColumnNullable` + related changes. [#5755](https://github.com/ClickHouse/ClickHouse/pull/5755) ([Artem Zuikov](https://github.com/4ertus2)) +- Fix the `ReadBufferFromKafkaConsumer` so that it keeps reading new messages after `commit()` even if it was stalled before [#5852](https://github.com/ClickHouse/ClickHouse/pull/5852) ([Ivan](https://github.com/abyss7)) +- Fix `FULL` and `RIGHT` JOIN results when joining on `Nullable` keys in right table. [#5859](https://github.com/ClickHouse/ClickHouse/pull/5859) ([Artem Zuikov](https://github.com/4ertus2)) +- Possible fix of infinite sleeping of low-priority queries. [#5842](https://github.com/ClickHouse/ClickHouse/pull/5842) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix race condition, which cause that some queries may not appear in query_log after `SYSTEM FLUSH LOGS` query. [#5456](https://github.com/ClickHouse/ClickHouse/issues/5456) [#5685](https://github.com/ClickHouse/ClickHouse/pull/5685) ([Anton Popov](https://github.com/CurtizJ)) +- Fixed `heap-use-after-free` ASan warning in ClusterCopier caused by watch which try to use already removed copier object. [#5871](https://github.com/ClickHouse/ClickHouse/pull/5871) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fixed wrong `StringRef` pointer returned by some implementations of `IColumn::deserializeAndInsertFromArena`. This bug affected only unit-tests. [#5973](https://github.com/ClickHouse/ClickHouse/pull/5973) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Prevent source and intermediate array join columns of masking same name columns. [#5941](https://github.com/ClickHouse/ClickHouse/pull/5941) ([Artem Zuikov](https://github.com/4ertus2)) +- Fix insert and select query to MySQL engine with MySQL style identifier quoting. [#5704](https://github.com/ClickHouse/ClickHouse/pull/5704) ([Winter Zhang](https://github.com/zhang2014)) +- Now `CHECK TABLE` query can work with MergeTree engine family. It returns check status and message if any for each part (or file in case of simplier engines). Also, fix bug in fetch of a broken part. [#5865](https://github.com/ClickHouse/ClickHouse/pull/5865) ([alesapin](https://github.com/alesapin)) +- Fix SPLIT_SHARED_LIBRARIES runtime [#5793](https://github.com/ClickHouse/ClickHouse/pull/5793) ([Danila Kutenin](https://github.com/danlark1)) +- Fixed time zone initialization when `/etc/localtime` is a relative symlink like `../usr/share/zoneinfo/Europe/Moscow` [#5922](https://github.com/ClickHouse/ClickHouse/pull/5922) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- clickhouse-copier: Fix use-after free on shutdown [#5752](https://github.com/ClickHouse/ClickHouse/pull/5752) ([proller](https://github.com/proller)) +- Updated `simdjson`. Fixed the issue that some invalid JSONs with zero bytes successfully parse. [#5938](https://github.com/ClickHouse/ClickHouse/pull/5938) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix shutdown of SystemLogs [#5802](https://github.com/ClickHouse/ClickHouse/pull/5802) ([Anton Popov](https://github.com/CurtizJ)) +- Fix hanging when condition in invalidate_query depends on a dictionary. [#6011](https://github.com/ClickHouse/ClickHouse/pull/6011) ([Vitaly Baranov](https://github.com/vitlibar)) #### Improvement {#improvement-6} -- Allow unresolvable addresses in cluster configuration. They will be considered unavailable and tried to resolve at every connection attempt. This is especially useful for Kubernetes. This fixes [\#5714](https://github.com/ClickHouse/ClickHouse/issues/5714) [\#5924](https://github.com/ClickHouse/ClickHouse/pull/5924) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Close idle TCP connections (with one hour timeout by default). This is especially important for large clusters with multiple distributed tables on every server, because every server can possibly keep a connection pool to every other server, and after peak query concurrency, connections will stall. This fixes [\#5879](https://github.com/ClickHouse/ClickHouse/issues/5879) [\#5880](https://github.com/ClickHouse/ClickHouse/pull/5880) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Better quality of `topK` function. Changed the SavingSpace set behavior to remove the last element if the new element have a bigger weight. [\#5833](https://github.com/ClickHouse/ClickHouse/issues/5833) [\#5850](https://github.com/ClickHouse/ClickHouse/pull/5850) ([Guillaume Tassery](https://github.com/YiuRULE)) -- URL functions to work with domains now can work for incomplete URLs without scheme [\#5725](https://github.com/ClickHouse/ClickHouse/pull/5725) ([alesapin](https://github.com/alesapin)) -- Checksums added to the `system.parts_columns` table. [\#5874](https://github.com/ClickHouse/ClickHouse/pull/5874) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -- Added `Enum` data type as a synonim for `Enum8` or `Enum16`. [\#5886](https://github.com/ClickHouse/ClickHouse/pull/5886) ([dimarub2000](https://github.com/dimarub2000)) -- Full bit transpose variant for `T64` codec. Could lead to better compression with `zstd`. [\#5742](https://github.com/ClickHouse/ClickHouse/pull/5742) ([Artem Zuikov](https://github.com/4ertus2)) -- Condition on `startsWith` function now can uses primary key. This fixes [\#5310](https://github.com/ClickHouse/ClickHouse/issues/5310) and [\#5882](https://github.com/ClickHouse/ClickHouse/issues/5882) [\#5919](https://github.com/ClickHouse/ClickHouse/pull/5919) ([dimarub2000](https://github.com/dimarub2000)) -- Allow to use `clickhouse-copier` with cross-replication cluster topology by permitting empty database name. [\#5745](https://github.com/ClickHouse/ClickHouse/pull/5745) ([nvartolomei](https://github.com/nvartolomei)) -- Use `UTC` as default timezone on a system without `tzdata` (e.g. bare Docker container). Before this patch, error message `Could not determine local time zone` was printed and server or client refused to start. [\#5827](https://github.com/ClickHouse/ClickHouse/pull/5827) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Returned back support for floating point argument in function `quantileTiming` for backward compatibility. [\#5911](https://github.com/ClickHouse/ClickHouse/pull/5911) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Show which table is missing column in error messages. [\#5768](https://github.com/ClickHouse/ClickHouse/pull/5768) ([Ivan](https://github.com/abyss7)) -- Disallow run query with same query\_id by various users [\#5430](https://github.com/ClickHouse/ClickHouse/pull/5430) ([proller](https://github.com/proller)) -- More robust code for sending metrics to Graphite. It will work even during long multiple `RENAME TABLE` operation. [\#5875](https://github.com/ClickHouse/ClickHouse/pull/5875) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- More informative error messages will be displayed when ThreadPool cannot schedule a task for execution. This fixes [\#5305](https://github.com/ClickHouse/ClickHouse/issues/5305) [\#5801](https://github.com/ClickHouse/ClickHouse/pull/5801) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Inverting ngramSearch to be more intuitive [\#5807](https://github.com/ClickHouse/ClickHouse/pull/5807) ([Danila Kutenin](https://github.com/danlark1)) -- Add user parsing in HDFS engine builder [\#5946](https://github.com/ClickHouse/ClickHouse/pull/5946) ([akonyaev90](https://github.com/akonyaev90)) -- Update default value of `max_ast_elements parameter` [\#5933](https://github.com/ClickHouse/ClickHouse/pull/5933) ([Artem Konovalov](https://github.com/izebit)) +- Allow unresolvable addresses in cluster configuration. They will be considered unavailable and tried to resolve at every connection attempt. This is especially useful for Kubernetes. This fixes [#5714](https://github.com/ClickHouse/ClickHouse/issues/5714) [#5924](https://github.com/ClickHouse/ClickHouse/pull/5924) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Close idle TCP connections (with one hour timeout by default). This is especially important for large clusters with multiple distributed tables on every server, because every server can possibly keep a connection pool to every other server, and after peak query concurrency, connections will stall. This fixes [#5879](https://github.com/ClickHouse/ClickHouse/issues/5879) [#5880](https://github.com/ClickHouse/ClickHouse/pull/5880) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Better quality of `topK` function. Changed the SavingSpace set behavior to remove the last element if the new element have a bigger weight. [#5833](https://github.com/ClickHouse/ClickHouse/issues/5833) [#5850](https://github.com/ClickHouse/ClickHouse/pull/5850) ([Guillaume Tassery](https://github.com/YiuRULE)) +- URL functions to work with domains now can work for incomplete URLs without scheme [#5725](https://github.com/ClickHouse/ClickHouse/pull/5725) ([alesapin](https://github.com/alesapin)) +- Checksums added to the `system.parts_columns` table. [#5874](https://github.com/ClickHouse/ClickHouse/pull/5874) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +- Added `Enum` data type as a synonim for `Enum8` or `Enum16`. [#5886](https://github.com/ClickHouse/ClickHouse/pull/5886) ([dimarub2000](https://github.com/dimarub2000)) +- Full bit transpose variant for `T64` codec. Could lead to better compression with `zstd`. [#5742](https://github.com/ClickHouse/ClickHouse/pull/5742) ([Artem Zuikov](https://github.com/4ertus2)) +- Condition on `startsWith` function now can uses primary key. This fixes [#5310](https://github.com/ClickHouse/ClickHouse/issues/5310) and [#5882](https://github.com/ClickHouse/ClickHouse/issues/5882) [#5919](https://github.com/ClickHouse/ClickHouse/pull/5919) ([dimarub2000](https://github.com/dimarub2000)) +- Allow to use `clickhouse-copier` with cross-replication cluster topology by permitting empty database name. [#5745](https://github.com/ClickHouse/ClickHouse/pull/5745) ([nvartolomei](https://github.com/nvartolomei)) +- Use `UTC` as default timezone on a system without `tzdata` (e.g. bare Docker container). Before this patch, error message `Could not determine local time zone` was printed and server or client refused to start. [#5827](https://github.com/ClickHouse/ClickHouse/pull/5827) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Returned back support for floating point argument in function `quantileTiming` for backward compatibility. [#5911](https://github.com/ClickHouse/ClickHouse/pull/5911) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Show which table is missing column in error messages. [#5768](https://github.com/ClickHouse/ClickHouse/pull/5768) ([Ivan](https://github.com/abyss7)) +- Disallow run query with same query_id by various users [#5430](https://github.com/ClickHouse/ClickHouse/pull/5430) ([proller](https://github.com/proller)) +- More robust code for sending metrics to Graphite. It will work even during long multiple `RENAME TABLE` operation. [#5875](https://github.com/ClickHouse/ClickHouse/pull/5875) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- More informative error messages will be displayed when ThreadPool cannot schedule a task for execution. This fixes [#5305](https://github.com/ClickHouse/ClickHouse/issues/5305) [#5801](https://github.com/ClickHouse/ClickHouse/pull/5801) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Inverting ngramSearch to be more intuitive [#5807](https://github.com/ClickHouse/ClickHouse/pull/5807) ([Danila Kutenin](https://github.com/danlark1)) +- Add user parsing in HDFS engine builder [#5946](https://github.com/ClickHouse/ClickHouse/pull/5946) ([akonyaev90](https://github.com/akonyaev90)) +- Update default value of `max_ast_elements parameter` [#5933](https://github.com/ClickHouse/ClickHouse/pull/5933) ([Artem Konovalov](https://github.com/izebit)) - Added a notion of obsolete settings. The obsolete setting `allow_experimental_low_cardinality_type` can be used with no effect. [0f15c01c6802f7ce1a1494c12c846be8c98944cd](https://github.com/ClickHouse/ClickHouse/commit/0f15c01c6802f7ce1a1494c12c846be8c98944cd) [Alexey Milovidov](https://github.com/alexey-milovidov) #### Performance Improvement {#performance-improvement-4} -- Increase number of streams to SELECT from Merge table for more uniform distribution of threads. Added setting `max_streams_multiplier_for_merge_tables`. This fixes [\#5797](https://github.com/ClickHouse/ClickHouse/issues/5797) [\#5915](https://github.com/ClickHouse/ClickHouse/pull/5915) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Increase number of streams to SELECT from Merge table for more uniform distribution of threads. Added setting `max_streams_multiplier_for_merge_tables`. This fixes [#5797](https://github.com/ClickHouse/ClickHouse/issues/5797) [#5915](https://github.com/ClickHouse/ClickHouse/pull/5915) ([alexey-milovidov](https://github.com/alexey-milovidov)) #### Build/Testing/Packaging Improvement {#buildtestingpackaging-improvement-7} -- Add a backward compatibility test for client-server interaction with different versions of clickhouse. [\#5868](https://github.com/ClickHouse/ClickHouse/pull/5868) ([alesapin](https://github.com/alesapin)) -- Test coverage information in every commit and pull request. [\#5896](https://github.com/ClickHouse/ClickHouse/pull/5896) ([alesapin](https://github.com/alesapin)) -- Cooperate with address sanitizer to support our custom allocators (`Arena` and `ArenaWithFreeLists`) for better debugging of “use-after-free” errors. [\#5728](https://github.com/ClickHouse/ClickHouse/pull/5728) ([akuzm](https://github.com/akuzm)) -- Switch to [LLVM libunwind implementation](https://github.com/llvm-mirror/libunwind) for C++ exception handling and for stack traces printing [\#4828](https://github.com/ClickHouse/ClickHouse/pull/4828) ([Nikita Lapkov](https://github.com/laplab)) -- Add two more warnings from -Weverything [\#5923](https://github.com/ClickHouse/ClickHouse/pull/5923) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Allow to build ClickHouse with Memory Sanitizer. [\#3949](https://github.com/ClickHouse/ClickHouse/pull/3949) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed ubsan report about `bitTest` function in fuzz test. [\#5943](https://github.com/ClickHouse/ClickHouse/pull/5943) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Docker: added possibility to init a ClickHouse instance which requires authentication. [\#5727](https://github.com/ClickHouse/ClickHouse/pull/5727) ([Korviakov Andrey](https://github.com/shurshun)) -- Update librdkafka to version 1.1.0 [\#5872](https://github.com/ClickHouse/ClickHouse/pull/5872) ([Ivan](https://github.com/abyss7)) -- Add global timeout for integration tests and disable some of them in tests code. [\#5741](https://github.com/ClickHouse/ClickHouse/pull/5741) ([alesapin](https://github.com/alesapin)) -- Fix some ThreadSanitizer failures. [\#5854](https://github.com/ClickHouse/ClickHouse/pull/5854) ([akuzm](https://github.com/akuzm)) -- The `--no-undefined` option forces the linker to check all external names for existence while linking. It’s very useful to track real dependencies between libraries in the split build mode. [\#5855](https://github.com/ClickHouse/ClickHouse/pull/5855) ([Ivan](https://github.com/abyss7)) -- Added performance test for [\#5797](https://github.com/ClickHouse/ClickHouse/issues/5797) [\#5914](https://github.com/ClickHouse/ClickHouse/pull/5914) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed compatibility with gcc-7. [\#5840](https://github.com/ClickHouse/ClickHouse/pull/5840) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Added support for gcc-9. This fixes [\#5717](https://github.com/ClickHouse/ClickHouse/issues/5717) [\#5774](https://github.com/ClickHouse/ClickHouse/pull/5774) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed error when libunwind can be linked incorrectly. [\#5948](https://github.com/ClickHouse/ClickHouse/pull/5948) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed a few warnings found by PVS-Studio. [\#5921](https://github.com/ClickHouse/ClickHouse/pull/5921) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Added initial support for `clang-tidy` static analyzer. [\#5806](https://github.com/ClickHouse/ClickHouse/pull/5806) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Convert BSD/Linux endian macros( ‘be64toh’ and ‘htobe64’) to the Mac OS X equivalents [\#5785](https://github.com/ClickHouse/ClickHouse/pull/5785) ([Fu Chen](https://github.com/fredchenbj)) -- Improved integration tests guide. [\#5796](https://github.com/ClickHouse/ClickHouse/pull/5796) ([Vladimir Chebotarev](https://github.com/excitoon)) -- Fixing build at macosx + gcc9 [\#5822](https://github.com/ClickHouse/ClickHouse/pull/5822) ([filimonov](https://github.com/filimonov)) -- Fix a hard-to-spot typo: aggreAGte -\> aggregate. [\#5753](https://github.com/ClickHouse/ClickHouse/pull/5753) ([akuzm](https://github.com/akuzm)) -- Fix freebsd build [\#5760](https://github.com/ClickHouse/ClickHouse/pull/5760) ([proller](https://github.com/proller)) -- Add link to experimental YouTube channel to website [\#5845](https://github.com/ClickHouse/ClickHouse/pull/5845) ([Ivan Blinkov](https://github.com/blinkov)) -- CMake: add option for coverage flags: WITH\_COVERAGE [\#5776](https://github.com/ClickHouse/ClickHouse/pull/5776) ([proller](https://github.com/proller)) -- Fix initial size of some inline PODArray’s. [\#5787](https://github.com/ClickHouse/ClickHouse/pull/5787) ([akuzm](https://github.com/akuzm)) -- clickhouse-server.postinst: fix os detection for centos 6 [\#5788](https://github.com/ClickHouse/ClickHouse/pull/5788) ([proller](https://github.com/proller)) -- Added Arch linux package generation. [\#5719](https://github.com/ClickHouse/ClickHouse/pull/5719) ([Vladimir Chebotarev](https://github.com/excitoon)) -- Split Common/config.h by libs (dbms) [\#5715](https://github.com/ClickHouse/ClickHouse/pull/5715) ([proller](https://github.com/proller)) -- Fixes for “Arcadia” build platform [\#5795](https://github.com/ClickHouse/ClickHouse/pull/5795) ([proller](https://github.com/proller)) -- Fixes for unconventional build (gcc9, no submodules) [\#5792](https://github.com/ClickHouse/ClickHouse/pull/5792) ([proller](https://github.com/proller)) -- Require explicit type in unalignedStore because it was proven to be bug-prone [\#5791](https://github.com/ClickHouse/ClickHouse/pull/5791) ([akuzm](https://github.com/akuzm)) -- Fixes MacOS build [\#5830](https://github.com/ClickHouse/ClickHouse/pull/5830) ([filimonov](https://github.com/filimonov)) -- Performance test concerning the new JIT feature with bigger dataset, as requested here [\#5263](https://github.com/ClickHouse/ClickHouse/issues/5263) [\#5887](https://github.com/ClickHouse/ClickHouse/pull/5887) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Add a backward compatibility test for client-server interaction with different versions of clickhouse. [#5868](https://github.com/ClickHouse/ClickHouse/pull/5868) ([alesapin](https://github.com/alesapin)) +- Test coverage information in every commit and pull request. [#5896](https://github.com/ClickHouse/ClickHouse/pull/5896) ([alesapin](https://github.com/alesapin)) +- Cooperate with address sanitizer to support our custom allocators (`Arena` and `ArenaWithFreeLists`) for better debugging of “use-after-free” errors. [#5728](https://github.com/ClickHouse/ClickHouse/pull/5728) ([akuzm](https://github.com/akuzm)) +- Switch to [LLVM libunwind implementation](https://github.com/llvm-mirror/libunwind) for C++ exception handling and for stack traces printing [#4828](https://github.com/ClickHouse/ClickHouse/pull/4828) ([Nikita Lapkov](https://github.com/laplab)) +- Add two more warnings from -Weverything [#5923](https://github.com/ClickHouse/ClickHouse/pull/5923) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Allow to build ClickHouse with Memory Sanitizer. [#3949](https://github.com/ClickHouse/ClickHouse/pull/3949) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed ubsan report about `bitTest` function in fuzz test. [#5943](https://github.com/ClickHouse/ClickHouse/pull/5943) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Docker: added possibility to init a ClickHouse instance which requires authentication. [#5727](https://github.com/ClickHouse/ClickHouse/pull/5727) ([Korviakov Andrey](https://github.com/shurshun)) +- Update librdkafka to version 1.1.0 [#5872](https://github.com/ClickHouse/ClickHouse/pull/5872) ([Ivan](https://github.com/abyss7)) +- Add global timeout for integration tests and disable some of them in tests code. [#5741](https://github.com/ClickHouse/ClickHouse/pull/5741) ([alesapin](https://github.com/alesapin)) +- Fix some ThreadSanitizer failures. [#5854](https://github.com/ClickHouse/ClickHouse/pull/5854) ([akuzm](https://github.com/akuzm)) +- The `--no-undefined` option forces the linker to check all external names for existence while linking. It’s very useful to track real dependencies between libraries in the split build mode. [#5855](https://github.com/ClickHouse/ClickHouse/pull/5855) ([Ivan](https://github.com/abyss7)) +- Added performance test for [#5797](https://github.com/ClickHouse/ClickHouse/issues/5797) [#5914](https://github.com/ClickHouse/ClickHouse/pull/5914) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed compatibility with gcc-7. [#5840](https://github.com/ClickHouse/ClickHouse/pull/5840) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Added support for gcc-9. This fixes [#5717](https://github.com/ClickHouse/ClickHouse/issues/5717) [#5774](https://github.com/ClickHouse/ClickHouse/pull/5774) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed error when libunwind can be linked incorrectly. [#5948](https://github.com/ClickHouse/ClickHouse/pull/5948) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed a few warnings found by PVS-Studio. [#5921](https://github.com/ClickHouse/ClickHouse/pull/5921) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Added initial support for `clang-tidy` static analyzer. [#5806](https://github.com/ClickHouse/ClickHouse/pull/5806) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Convert BSD/Linux endian macros( ‘be64toh’ and ‘htobe64’) to the Mac OS X equivalents [#5785](https://github.com/ClickHouse/ClickHouse/pull/5785) ([Fu Chen](https://github.com/fredchenbj)) +- Improved integration tests guide. [#5796](https://github.com/ClickHouse/ClickHouse/pull/5796) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Fixing build at macosx + gcc9 [#5822](https://github.com/ClickHouse/ClickHouse/pull/5822) ([filimonov](https://github.com/filimonov)) +- Fix a hard-to-spot typo: aggreAGte -\> aggregate. [#5753](https://github.com/ClickHouse/ClickHouse/pull/5753) ([akuzm](https://github.com/akuzm)) +- Fix freebsd build [#5760](https://github.com/ClickHouse/ClickHouse/pull/5760) ([proller](https://github.com/proller)) +- Add link to experimental YouTube channel to website [#5845](https://github.com/ClickHouse/ClickHouse/pull/5845) ([Ivan Blinkov](https://github.com/blinkov)) +- CMake: add option for coverage flags: WITH_COVERAGE [#5776](https://github.com/ClickHouse/ClickHouse/pull/5776) ([proller](https://github.com/proller)) +- Fix initial size of some inline PODArray’s. [#5787](https://github.com/ClickHouse/ClickHouse/pull/5787) ([akuzm](https://github.com/akuzm)) +- clickhouse-server.postinst: fix os detection for centos 6 [#5788](https://github.com/ClickHouse/ClickHouse/pull/5788) ([proller](https://github.com/proller)) +- Added Arch linux package generation. [#5719](https://github.com/ClickHouse/ClickHouse/pull/5719) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Split Common/config.h by libs (dbms) [#5715](https://github.com/ClickHouse/ClickHouse/pull/5715) ([proller](https://github.com/proller)) +- Fixes for “Arcadia” build platform [#5795](https://github.com/ClickHouse/ClickHouse/pull/5795) ([proller](https://github.com/proller)) +- Fixes for unconventional build (gcc9, no submodules) [#5792](https://github.com/ClickHouse/ClickHouse/pull/5792) ([proller](https://github.com/proller)) +- Require explicit type in unalignedStore because it was proven to be bug-prone [#5791](https://github.com/ClickHouse/ClickHouse/pull/5791) ([akuzm](https://github.com/akuzm)) +- Fixes MacOS build [#5830](https://github.com/ClickHouse/ClickHouse/pull/5830) ([filimonov](https://github.com/filimonov)) +- Performance test concerning the new JIT feature with bigger dataset, as requested here [#5263](https://github.com/ClickHouse/ClickHouse/issues/5263) [#5887](https://github.com/ClickHouse/ClickHouse/pull/5887) ([Guillaume Tassery](https://github.com/YiuRULE)) - Run stateful tests in stress test [12693e568722f11e19859742f56428455501fd2a](https://github.com/ClickHouse/ClickHouse/commit/12693e568722f11e19859742f56428455501fd2a) ([alesapin](https://github.com/alesapin)) #### Backward Incompatible Change {#backward-incompatible-change-7} - `Kafka` is broken in this version. -- Enable `adaptive_index_granularity` = 10MB by default for new `MergeTree` tables. If you created new MergeTree tables on version 19.11+, downgrade to versions prior to 19.6 will be impossible. [\#5628](https://github.com/ClickHouse/ClickHouse/pull/5628) ([alesapin](https://github.com/alesapin)) -- Removed obsolete undocumented embedded dictionaries that were used by Yandex.Metrica. The functions `OSIn`, `SEIn`, `OSToRoot`, `SEToRoot`, `OSHierarchy`, `SEHierarchy` are no longer available. If you are using these functions, write email to clickhouse-feedback@yandex-team.com. Note: at the last moment we decided to keep these functions for a while. [\#5780](https://github.com/ClickHouse/ClickHouse/pull/5780) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Enable `adaptive_index_granularity` = 10MB by default for new `MergeTree` tables. If you created new MergeTree tables on version 19.11+, downgrade to versions prior to 19.6 will be impossible. [#5628](https://github.com/ClickHouse/ClickHouse/pull/5628) ([alesapin](https://github.com/alesapin)) +- Removed obsolete undocumented embedded dictionaries that were used by Yandex.Metrica. The functions `OSIn`, `SEIn`, `OSToRoot`, `SEToRoot`, `OSHierarchy`, `SEHierarchy` are no longer available. If you are using these functions, write email to clickhouse-feedback@yandex-team.com. Note: at the last moment we decided to keep these functions for a while. [#5780](https://github.com/ClickHouse/ClickHouse/pull/5780) ([alexey-milovidov](https://github.com/alexey-milovidov)) ## ClickHouse Release 19.10 {#clickhouse-release-19-10} @@ -1121,47 +1121,47 @@ toc_title: '2019' #### New Feature {#new-feature-7} -- Add new column codec: `T64`. Made for (U)IntX/EnumX/Data(Time)/DecimalX columns. It should be good for columns with constant or small range values. Codec itself allows enlarge or shrink data type without re-compression. [\#5557](https://github.com/ClickHouse/ClickHouse/pull/5557) ([Artem Zuikov](https://github.com/4ertus2)) -- Add database engine `MySQL` that allow to view all the tables in remote MySQL server [\#5599](https://github.com/ClickHouse/ClickHouse/pull/5599) ([Winter Zhang](https://github.com/zhang2014)) -- `bitmapContains` implementation. It’s 2x faster than `bitmapHasAny` if the second bitmap contains one element. [\#5535](https://github.com/ClickHouse/ClickHouse/pull/5535) ([Zhichang Yu](https://github.com/yuzhichang)) -- Support for `crc32` function (with behaviour exactly as in MySQL or PHP). Do not use it if you need a hash function. [\#5661](https://github.com/ClickHouse/ClickHouse/pull/5661) ([Remen Ivan](https://github.com/BHYCHIK)) -- Implemented `SYSTEM START/STOP DISTRIBUTED SENDS` queries to control asynchronous inserts into `Distributed` tables. [\#4935](https://github.com/ClickHouse/ClickHouse/pull/4935) ([Winter Zhang](https://github.com/zhang2014)) +- Add new column codec: `T64`. Made for (U)IntX/EnumX/Data(Time)/DecimalX columns. It should be good for columns with constant or small range values. Codec itself allows enlarge or shrink data type without re-compression. [#5557](https://github.com/ClickHouse/ClickHouse/pull/5557) ([Artem Zuikov](https://github.com/4ertus2)) +- Add database engine `MySQL` that allow to view all the tables in remote MySQL server [#5599](https://github.com/ClickHouse/ClickHouse/pull/5599) ([Winter Zhang](https://github.com/zhang2014)) +- `bitmapContains` implementation. It’s 2x faster than `bitmapHasAny` if the second bitmap contains one element. [#5535](https://github.com/ClickHouse/ClickHouse/pull/5535) ([Zhichang Yu](https://github.com/yuzhichang)) +- Support for `crc32` function (with behaviour exactly as in MySQL or PHP). Do not use it if you need a hash function. [#5661](https://github.com/ClickHouse/ClickHouse/pull/5661) ([Remen Ivan](https://github.com/BHYCHIK)) +- Implemented `SYSTEM START/STOP DISTRIBUTED SENDS` queries to control asynchronous inserts into `Distributed` tables. [#4935](https://github.com/ClickHouse/ClickHouse/pull/4935) ([Winter Zhang](https://github.com/zhang2014)) #### Bug Fix {#bug-fix-22} -- Ignore query execution limits and max parts size for merge limits while executing mutations. [\#5659](https://github.com/ClickHouse/ClickHouse/pull/5659) ([Anton Popov](https://github.com/CurtizJ)) -- Fix bug which may lead to deduplication of normal blocks (extremely rare) and insertion of duplicate blocks (more often). [\#5549](https://github.com/ClickHouse/ClickHouse/pull/5549) ([alesapin](https://github.com/alesapin)) -- Fix of function `arrayEnumerateUniqRanked` for arguments with empty arrays [\#5559](https://github.com/ClickHouse/ClickHouse/pull/5559) ([proller](https://github.com/proller)) -- Don’t subscribe to Kafka topics without intent to poll any messages. [\#5698](https://github.com/ClickHouse/ClickHouse/pull/5698) ([Ivan](https://github.com/abyss7)) -- Make setting `join_use_nulls` get no effect for types that cannot be inside Nullable [\#5700](https://github.com/ClickHouse/ClickHouse/pull/5700) ([Olga Khvostikova](https://github.com/stavrolia)) -- Fixed `Incorrect size of index granularity` errors [\#5720](https://github.com/ClickHouse/ClickHouse/pull/5720) ([coraxster](https://github.com/coraxster)) -- Fix Float to Decimal convert overflow [\#5607](https://github.com/ClickHouse/ClickHouse/pull/5607) ([coraxster](https://github.com/coraxster)) -- Flush buffer when `WriteBufferFromHDFS`’s destructor is called. This fixes writing into `HDFS`. [\#5684](https://github.com/ClickHouse/ClickHouse/pull/5684) ([Xindong Peng](https://github.com/eejoin)) +- Ignore query execution limits and max parts size for merge limits while executing mutations. [#5659](https://github.com/ClickHouse/ClickHouse/pull/5659) ([Anton Popov](https://github.com/CurtizJ)) +- Fix bug which may lead to deduplication of normal blocks (extremely rare) and insertion of duplicate blocks (more often). [#5549](https://github.com/ClickHouse/ClickHouse/pull/5549) ([alesapin](https://github.com/alesapin)) +- Fix of function `arrayEnumerateUniqRanked` for arguments with empty arrays [#5559](https://github.com/ClickHouse/ClickHouse/pull/5559) ([proller](https://github.com/proller)) +- Don’t subscribe to Kafka topics without intent to poll any messages. [#5698](https://github.com/ClickHouse/ClickHouse/pull/5698) ([Ivan](https://github.com/abyss7)) +- Make setting `join_use_nulls` get no effect for types that cannot be inside Nullable [#5700](https://github.com/ClickHouse/ClickHouse/pull/5700) ([Olga Khvostikova](https://github.com/stavrolia)) +- Fixed `Incorrect size of index granularity` errors [#5720](https://github.com/ClickHouse/ClickHouse/pull/5720) ([coraxster](https://github.com/coraxster)) +- Fix Float to Decimal convert overflow [#5607](https://github.com/ClickHouse/ClickHouse/pull/5607) ([coraxster](https://github.com/coraxster)) +- Flush buffer when `WriteBufferFromHDFS`’s destructor is called. This fixes writing into `HDFS`. [#5684](https://github.com/ClickHouse/ClickHouse/pull/5684) ([Xindong Peng](https://github.com/eejoin)) #### Improvement {#improvement-7} -- Treat empty cells in `CSV` as default values when the setting `input_format_defaults_for_omitted_fields` is enabled. [\#5625](https://github.com/ClickHouse/ClickHouse/pull/5625) ([akuzm](https://github.com/akuzm)) -- Non-blocking loading of external dictionaries. [\#5567](https://github.com/ClickHouse/ClickHouse/pull/5567) ([Vitaly Baranov](https://github.com/vitlibar)) -- Network timeouts can be dynamically changed for already established connections according to the settings. [\#4558](https://github.com/ClickHouse/ClickHouse/pull/4558) ([Konstantin Podshumok](https://github.com/podshumok)) -- Using “public\_suffix\_list” for functions `firstSignificantSubdomain`, `cutToFirstSignificantSubdomain`. It’s using a perfect hash table generated by `gperf` with a list generated from the file: https://publicsuffix.org/list/public\_suffix\_list.dat. (for example, now we recognize the domain `ac.uk` as non-significant). [\#5030](https://github.com/ClickHouse/ClickHouse/pull/5030) ([Guillaume Tassery](https://github.com/YiuRULE)) -- Adopted `IPv6` data type in system tables; unified client info columns in `system.processes` and `system.query_log` [\#5640](https://github.com/ClickHouse/ClickHouse/pull/5640) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Using sessions for connections with MySQL compatibility protocol. \#5476 [\#5646](https://github.com/ClickHouse/ClickHouse/pull/5646) ([Yuriy Baranov](https://github.com/yurriy)) -- Support more `ALTER` queries `ON CLUSTER`. [\#5593](https://github.com/ClickHouse/ClickHouse/pull/5593) [\#5613](https://github.com/ClickHouse/ClickHouse/pull/5613) ([sundyli](https://github.com/sundy-li)) -- Support `` section in `clickhouse-local` config file. [\#5540](https://github.com/ClickHouse/ClickHouse/pull/5540) ([proller](https://github.com/proller)) -- Allow run query with `remote` table function in `clickhouse-local` [\#5627](https://github.com/ClickHouse/ClickHouse/pull/5627) ([proller](https://github.com/proller)) +- Treat empty cells in `CSV` as default values when the setting `input_format_defaults_for_omitted_fields` is enabled. [#5625](https://github.com/ClickHouse/ClickHouse/pull/5625) ([akuzm](https://github.com/akuzm)) +- Non-blocking loading of external dictionaries. [#5567](https://github.com/ClickHouse/ClickHouse/pull/5567) ([Vitaly Baranov](https://github.com/vitlibar)) +- Network timeouts can be dynamically changed for already established connections according to the settings. [#4558](https://github.com/ClickHouse/ClickHouse/pull/4558) ([Konstantin Podshumok](https://github.com/podshumok)) +- Using “public_suffix_list” for functions `firstSignificantSubdomain`, `cutToFirstSignificantSubdomain`. It’s using a perfect hash table generated by `gperf` with a list generated from the file: https://publicsuffix.org/list/public_suffix_list.dat. (for example, now we recognize the domain `ac.uk` as non-significant). [#5030](https://github.com/ClickHouse/ClickHouse/pull/5030) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Adopted `IPv6` data type in system tables; unified client info columns in `system.processes` and `system.query_log` [#5640](https://github.com/ClickHouse/ClickHouse/pull/5640) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Using sessions for connections with MySQL compatibility protocol. #5476 [#5646](https://github.com/ClickHouse/ClickHouse/pull/5646) ([Yuriy Baranov](https://github.com/yurriy)) +- Support more `ALTER` queries `ON CLUSTER`. [#5593](https://github.com/ClickHouse/ClickHouse/pull/5593) [#5613](https://github.com/ClickHouse/ClickHouse/pull/5613) ([sundyli](https://github.com/sundy-li)) +- Support `` section in `clickhouse-local` config file. [#5540](https://github.com/ClickHouse/ClickHouse/pull/5540) ([proller](https://github.com/proller)) +- Allow run query with `remote` table function in `clickhouse-local` [#5627](https://github.com/ClickHouse/ClickHouse/pull/5627) ([proller](https://github.com/proller)) #### Performance Improvement {#performance-improvement-5} -- Add the possibility to write the final mark at the end of MergeTree columns. It allows to avoid useless reads for keys that are out of table data range. It is enabled only if adaptive index granularity is in use. [\#5624](https://github.com/ClickHouse/ClickHouse/pull/5624) ([alesapin](https://github.com/alesapin)) -- Improved performance of MergeTree tables on very slow filesystems by reducing number of `stat` syscalls. [\#5648](https://github.com/ClickHouse/ClickHouse/pull/5648) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed performance degradation in reading from MergeTree tables that was introduced in version 19.6. Fixes \#5631. [\#5633](https://github.com/ClickHouse/ClickHouse/pull/5633) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Add the possibility to write the final mark at the end of MergeTree columns. It allows to avoid useless reads for keys that are out of table data range. It is enabled only if adaptive index granularity is in use. [#5624](https://github.com/ClickHouse/ClickHouse/pull/5624) ([alesapin](https://github.com/alesapin)) +- Improved performance of MergeTree tables on very slow filesystems by reducing number of `stat` syscalls. [#5648](https://github.com/ClickHouse/ClickHouse/pull/5648) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed performance degradation in reading from MergeTree tables that was introduced in version 19.6. Fixes #5631. [#5633](https://github.com/ClickHouse/ClickHouse/pull/5633) ([alexey-milovidov](https://github.com/alexey-milovidov)) #### Build/Testing/Packaging Improvement {#buildtestingpackaging-improvement-8} -- Implemented `TestKeeper` as an implementation of ZooKeeper interface used for testing [\#5643](https://github.com/ClickHouse/ClickHouse/pull/5643) ([alexey-milovidov](https://github.com/alexey-milovidov)) ([levushkin aleksej](https://github.com/alexey-milovidov)) -- From now on `.sql` tests can be run isolated by server, in parallel, with random database. It allows to run them faster, add new tests with custom server configurations, and be sure that different tests doesn’t affect each other. [\#5554](https://github.com/ClickHouse/ClickHouse/pull/5554) ([Ivan](https://github.com/abyss7)) -- Remove `` and `` from performance tests [\#5672](https://github.com/ClickHouse/ClickHouse/pull/5672) ([Olga Khvostikova](https://github.com/stavrolia)) -- Fixed “select\_format” performance test for `Pretty` formats [\#5642](https://github.com/ClickHouse/ClickHouse/pull/5642) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Implemented `TestKeeper` as an implementation of ZooKeeper interface used for testing [#5643](https://github.com/ClickHouse/ClickHouse/pull/5643) ([alexey-milovidov](https://github.com/alexey-milovidov)) ([levushkin aleksej](https://github.com/alexey-milovidov)) +- From now on `.sql` tests can be run isolated by server, in parallel, with random database. It allows to run them faster, add new tests with custom server configurations, and be sure that different tests doesn’t affect each other. [#5554](https://github.com/ClickHouse/ClickHouse/pull/5554) ([Ivan](https://github.com/abyss7)) +- Remove `` and `` from performance tests [#5672](https://github.com/ClickHouse/ClickHouse/pull/5672) ([Olga Khvostikova](https://github.com/stavrolia)) +- Fixed “select_format” performance test for `Pretty` formats [#5642](https://github.com/ClickHouse/ClickHouse/pull/5642) ([alexey-milovidov](https://github.com/alexey-milovidov)) ## ClickHouse Release 19.9 {#clickhouse-release-19-9} @@ -1169,67 +1169,67 @@ toc_title: '2019' #### Bug Fix {#bug-fix-23} -- Fix segfault in Delta codec which affects columns with values less than 32 bits size. The bug led to random memory corruption. [\#5786](https://github.com/ClickHouse/ClickHouse/pull/5786) ([alesapin](https://github.com/alesapin)) -- Fix rare bug in checking of part with LowCardinality column. [\#5832](https://github.com/ClickHouse/ClickHouse/pull/5832) ([alesapin](https://github.com/alesapin)) -- Fix segfault in TTL merge with non-physical columns in block. [\#5819](https://github.com/ClickHouse/ClickHouse/pull/5819) ([Anton Popov](https://github.com/CurtizJ)) -- Fix potential infinite sleeping of low-priority queries. [\#5842](https://github.com/ClickHouse/ClickHouse/pull/5842) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix how ClickHouse determines default time zone as UCT instead of UTC. [\#5828](https://github.com/ClickHouse/ClickHouse/pull/5828) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix bug about executing distributed DROP/ALTER/TRUNCATE/OPTIMIZE ON CLUSTER queries on follower replica before leader replica. Now they will be executed directly on leader replica. [\#5757](https://github.com/ClickHouse/ClickHouse/pull/5757) ([alesapin](https://github.com/alesapin)) -- Fix race condition, which cause that some queries may not appear in query\_log instantly after SYSTEM FLUSH LOGS query. [\#5685](https://github.com/ClickHouse/ClickHouse/pull/5685) ([Anton Popov](https://github.com/CurtizJ)) -- Added missing support for constant arguments to `evalMLModel` function. [\#5820](https://github.com/ClickHouse/ClickHouse/pull/5820) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix segfault in Delta codec which affects columns with values less than 32 bits size. The bug led to random memory corruption. [#5786](https://github.com/ClickHouse/ClickHouse/pull/5786) ([alesapin](https://github.com/alesapin)) +- Fix rare bug in checking of part with LowCardinality column. [#5832](https://github.com/ClickHouse/ClickHouse/pull/5832) ([alesapin](https://github.com/alesapin)) +- Fix segfault in TTL merge with non-physical columns in block. [#5819](https://github.com/ClickHouse/ClickHouse/pull/5819) ([Anton Popov](https://github.com/CurtizJ)) +- Fix potential infinite sleeping of low-priority queries. [#5842](https://github.com/ClickHouse/ClickHouse/pull/5842) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix how ClickHouse determines default time zone as UCT instead of UTC. [#5828](https://github.com/ClickHouse/ClickHouse/pull/5828) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix bug about executing distributed DROP/ALTER/TRUNCATE/OPTIMIZE ON CLUSTER queries on follower replica before leader replica. Now they will be executed directly on leader replica. [#5757](https://github.com/ClickHouse/ClickHouse/pull/5757) ([alesapin](https://github.com/alesapin)) +- Fix race condition, which cause that some queries may not appear in query_log instantly after SYSTEM FLUSH LOGS query. [#5685](https://github.com/ClickHouse/ClickHouse/pull/5685) ([Anton Popov](https://github.com/CurtizJ)) +- Added missing support for constant arguments to `evalMLModel` function. [#5820](https://github.com/ClickHouse/ClickHouse/pull/5820) ([alexey-milovidov](https://github.com/alexey-milovidov)) ### ClickHouse Release 19.9.2.4, 2019-06-24 {#clickhouse-release-19-9-2-4-2019-06-24} #### New Feature {#new-feature-8} -- Print information about frozen parts in `system.parts` table. [\#5471](https://github.com/ClickHouse/ClickHouse/pull/5471) ([proller](https://github.com/proller)) -- Ask client password on clickhouse-client start on tty if not set in arguments [\#5092](https://github.com/ClickHouse/ClickHouse/pull/5092) ([proller](https://github.com/proller)) -- Implement `dictGet` and `dictGetOrDefault` functions for Decimal types. [\#5394](https://github.com/ClickHouse/ClickHouse/pull/5394) ([Artem Zuikov](https://github.com/4ertus2)) +- Print information about frozen parts in `system.parts` table. [#5471](https://github.com/ClickHouse/ClickHouse/pull/5471) ([proller](https://github.com/proller)) +- Ask client password on clickhouse-client start on tty if not set in arguments [#5092](https://github.com/ClickHouse/ClickHouse/pull/5092) ([proller](https://github.com/proller)) +- Implement `dictGet` and `dictGetOrDefault` functions for Decimal types. [#5394](https://github.com/ClickHouse/ClickHouse/pull/5394) ([Artem Zuikov](https://github.com/4ertus2)) #### Improvement {#improvement-8} -- Debian init: Add service stop timeout [\#5522](https://github.com/ClickHouse/ClickHouse/pull/5522) ([proller](https://github.com/proller)) -- Add setting forbidden by default to create table with suspicious types for LowCardinality [\#5448](https://github.com/ClickHouse/ClickHouse/pull/5448) ([Olga Khvostikova](https://github.com/stavrolia)) -- Regression functions return model weights when not used as State in function `evalMLMethod`. [\#5411](https://github.com/ClickHouse/ClickHouse/pull/5411) ([Quid37](https://github.com/Quid37)) -- Rename and improve regression methods. [\#5492](https://github.com/ClickHouse/ClickHouse/pull/5492) ([Quid37](https://github.com/Quid37)) -- Clearer interfaces of string searchers. [\#5586](https://github.com/ClickHouse/ClickHouse/pull/5586) ([Danila Kutenin](https://github.com/danlark1)) +- Debian init: Add service stop timeout [#5522](https://github.com/ClickHouse/ClickHouse/pull/5522) ([proller](https://github.com/proller)) +- Add setting forbidden by default to create table with suspicious types for LowCardinality [#5448](https://github.com/ClickHouse/ClickHouse/pull/5448) ([Olga Khvostikova](https://github.com/stavrolia)) +- Regression functions return model weights when not used as State in function `evalMLMethod`. [#5411](https://github.com/ClickHouse/ClickHouse/pull/5411) ([Quid37](https://github.com/Quid37)) +- Rename and improve regression methods. [#5492](https://github.com/ClickHouse/ClickHouse/pull/5492) ([Quid37](https://github.com/Quid37)) +- Clearer interfaces of string searchers. [#5586](https://github.com/ClickHouse/ClickHouse/pull/5586) ([Danila Kutenin](https://github.com/danlark1)) #### Bug Fix {#bug-fix-24} -- Fix potential data loss in Kafka [\#5445](https://github.com/ClickHouse/ClickHouse/pull/5445) ([Ivan](https://github.com/abyss7)) -- Fix potential infinite loop in `PrettySpace` format when called with zero columns [\#5560](https://github.com/ClickHouse/ClickHouse/pull/5560) ([Olga Khvostikova](https://github.com/stavrolia)) -- Fixed UInt32 overflow bug in linear models. Allow eval ML model for non-const model argument. [\#5516](https://github.com/ClickHouse/ClickHouse/pull/5516) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- `ALTER TABLE ... DROP INDEX IF EXISTS ...` should not raise an exception if provided index does not exist [\#5524](https://github.com/ClickHouse/ClickHouse/pull/5524) ([Gleb Novikov](https://github.com/NanoBjorn)) -- Fix segfault with `bitmapHasAny` in scalar subquery [\#5528](https://github.com/ClickHouse/ClickHouse/pull/5528) ([Zhichang Yu](https://github.com/yuzhichang)) -- Fixed error when replication connection pool doesn’t retry to resolve host, even when DNS cache was dropped. [\#5534](https://github.com/ClickHouse/ClickHouse/pull/5534) ([alesapin](https://github.com/alesapin)) -- Fixed `ALTER ... MODIFY TTL` on ReplicatedMergeTree. [\#5539](https://github.com/ClickHouse/ClickHouse/pull/5539) ([Anton Popov](https://github.com/CurtizJ)) -- Fix INSERT into Distributed table with MATERIALIZED column [\#5429](https://github.com/ClickHouse/ClickHouse/pull/5429) ([Azat Khuzhin](https://github.com/azat)) -- Fix bad alloc when truncate Join storage [\#5437](https://github.com/ClickHouse/ClickHouse/pull/5437) ([TCeason](https://github.com/TCeason)) -- In recent versions of package tzdata some of files are symlinks now. The current mechanism for detecting default timezone gets broken and gives wrong names for some timezones. Now at least we force the timezone name to the contents of TZ if provided. [\#5443](https://github.com/ClickHouse/ClickHouse/pull/5443) ([Ivan](https://github.com/abyss7)) -- Fix some extremely rare cases with MultiVolnitsky searcher when the constant needles in sum are at least 16KB long. The algorithm missed or overwrote the previous results which can lead to the incorrect result of `multiSearchAny`. [\#5588](https://github.com/ClickHouse/ClickHouse/pull/5588) ([Danila Kutenin](https://github.com/danlark1)) -- Fix the issue when settings for ExternalData requests couldn’t use ClickHouse settings. Also, for now, settings `date_time_input_format` and `low_cardinality_allow_in_native_format` cannot be used because of the ambiguity of names (in external data it can be interpreted as table format and in the query it can be a setting). [\#5455](https://github.com/ClickHouse/ClickHouse/pull/5455) ([Danila Kutenin](https://github.com/danlark1)) -- Fix bug when parts were removed only from FS without dropping them from Zookeeper. [\#5520](https://github.com/ClickHouse/ClickHouse/pull/5520) ([alesapin](https://github.com/alesapin)) -- Remove debug logging from MySQL protocol [\#5478](https://github.com/ClickHouse/ClickHouse/pull/5478) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Skip ZNONODE during DDL query processing [\#5489](https://github.com/ClickHouse/ClickHouse/pull/5489) ([Azat Khuzhin](https://github.com/azat)) -- Fix mix `UNION ALL` result column type. There were cases with inconsistent data and column types of resulting columns. [\#5503](https://github.com/ClickHouse/ClickHouse/pull/5503) ([Artem Zuikov](https://github.com/4ertus2)) -- Throw an exception on wrong integers in `dictGetT` functions instead of crash. [\#5446](https://github.com/ClickHouse/ClickHouse/pull/5446) ([Artem Zuikov](https://github.com/4ertus2)) -- Fix wrong element\_count and load\_factor for hashed dictionary in `system.dictionaries` table. [\#5440](https://github.com/ClickHouse/ClickHouse/pull/5440) ([Azat Khuzhin](https://github.com/azat)) +- Fix potential data loss in Kafka [#5445](https://github.com/ClickHouse/ClickHouse/pull/5445) ([Ivan](https://github.com/abyss7)) +- Fix potential infinite loop in `PrettySpace` format when called with zero columns [#5560](https://github.com/ClickHouse/ClickHouse/pull/5560) ([Olga Khvostikova](https://github.com/stavrolia)) +- Fixed UInt32 overflow bug in linear models. Allow eval ML model for non-const model argument. [#5516](https://github.com/ClickHouse/ClickHouse/pull/5516) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- `ALTER TABLE ... DROP INDEX IF EXISTS ...` should not raise an exception if provided index does not exist [#5524](https://github.com/ClickHouse/ClickHouse/pull/5524) ([Gleb Novikov](https://github.com/NanoBjorn)) +- Fix segfault with `bitmapHasAny` in scalar subquery [#5528](https://github.com/ClickHouse/ClickHouse/pull/5528) ([Zhichang Yu](https://github.com/yuzhichang)) +- Fixed error when replication connection pool doesn’t retry to resolve host, even when DNS cache was dropped. [#5534](https://github.com/ClickHouse/ClickHouse/pull/5534) ([alesapin](https://github.com/alesapin)) +- Fixed `ALTER ... MODIFY TTL` on ReplicatedMergeTree. [#5539](https://github.com/ClickHouse/ClickHouse/pull/5539) ([Anton Popov](https://github.com/CurtizJ)) +- Fix INSERT into Distributed table with MATERIALIZED column [#5429](https://github.com/ClickHouse/ClickHouse/pull/5429) ([Azat Khuzhin](https://github.com/azat)) +- Fix bad alloc when truncate Join storage [#5437](https://github.com/ClickHouse/ClickHouse/pull/5437) ([TCeason](https://github.com/TCeason)) +- In recent versions of package tzdata some of files are symlinks now. The current mechanism for detecting default timezone gets broken and gives wrong names for some timezones. Now at least we force the timezone name to the contents of TZ if provided. [#5443](https://github.com/ClickHouse/ClickHouse/pull/5443) ([Ivan](https://github.com/abyss7)) +- Fix some extremely rare cases with MultiVolnitsky searcher when the constant needles in sum are at least 16KB long. The algorithm missed or overwrote the previous results which can lead to the incorrect result of `multiSearchAny`. [#5588](https://github.com/ClickHouse/ClickHouse/pull/5588) ([Danila Kutenin](https://github.com/danlark1)) +- Fix the issue when settings for ExternalData requests couldn’t use ClickHouse settings. Also, for now, settings `date_time_input_format` and `low_cardinality_allow_in_native_format` cannot be used because of the ambiguity of names (in external data it can be interpreted as table format and in the query it can be a setting). [#5455](https://github.com/ClickHouse/ClickHouse/pull/5455) ([Danila Kutenin](https://github.com/danlark1)) +- Fix bug when parts were removed only from FS without dropping them from Zookeeper. [#5520](https://github.com/ClickHouse/ClickHouse/pull/5520) ([alesapin](https://github.com/alesapin)) +- Remove debug logging from MySQL protocol [#5478](https://github.com/ClickHouse/ClickHouse/pull/5478) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Skip ZNONODE during DDL query processing [#5489](https://github.com/ClickHouse/ClickHouse/pull/5489) ([Azat Khuzhin](https://github.com/azat)) +- Fix mix `UNION ALL` result column type. There were cases with inconsistent data and column types of resulting columns. [#5503](https://github.com/ClickHouse/ClickHouse/pull/5503) ([Artem Zuikov](https://github.com/4ertus2)) +- Throw an exception on wrong integers in `dictGetT` functions instead of crash. [#5446](https://github.com/ClickHouse/ClickHouse/pull/5446) ([Artem Zuikov](https://github.com/4ertus2)) +- Fix wrong element_count and load_factor for hashed dictionary in `system.dictionaries` table. [#5440](https://github.com/ClickHouse/ClickHouse/pull/5440) ([Azat Khuzhin](https://github.com/azat)) #### Build/Testing/Packaging Improvement {#buildtestingpackaging-improvement-9} -- Fixed build without `Brotli` HTTP compression support (`ENABLE_BROTLI=OFF` cmake variable). [\#5521](https://github.com/ClickHouse/ClickHouse/pull/5521) ([Anton Yuzhaninov](https://github.com/citrin)) -- Include roaring.h as roaring/roaring.h [\#5523](https://github.com/ClickHouse/ClickHouse/pull/5523) ([Orivej Desh](https://github.com/orivej)) -- Fix gcc9 warnings in hyperscan (\#line directive is evil!) [\#5546](https://github.com/ClickHouse/ClickHouse/pull/5546) ([Danila Kutenin](https://github.com/danlark1)) -- Fix all warnings when compiling with gcc-9. Fix some contrib issues. Fix gcc9 ICE and submit it to bugzilla. [\#5498](https://github.com/ClickHouse/ClickHouse/pull/5498) ([Danila Kutenin](https://github.com/danlark1)) -- Fixed linking with lld [\#5477](https://github.com/ClickHouse/ClickHouse/pull/5477) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Remove unused specializations in dictionaries [\#5452](https://github.com/ClickHouse/ClickHouse/pull/5452) ([Artem Zuikov](https://github.com/4ertus2)) -- Improvement performance tests for formatting and parsing tables for different types of files [\#5497](https://github.com/ClickHouse/ClickHouse/pull/5497) ([Olga Khvostikova](https://github.com/stavrolia)) -- Fixes for parallel test run [\#5506](https://github.com/ClickHouse/ClickHouse/pull/5506) ([proller](https://github.com/proller)) -- Docker: use configs from clickhouse-test [\#5531](https://github.com/ClickHouse/ClickHouse/pull/5531) ([proller](https://github.com/proller)) -- Fix compile for FreeBSD [\#5447](https://github.com/ClickHouse/ClickHouse/pull/5447) ([proller](https://github.com/proller)) -- Upgrade boost to 1.70 [\#5570](https://github.com/ClickHouse/ClickHouse/pull/5570) ([proller](https://github.com/proller)) -- Fix build clickhouse as submodule [\#5574](https://github.com/ClickHouse/ClickHouse/pull/5574) ([proller](https://github.com/proller)) -- Improve JSONExtract performance tests [\#5444](https://github.com/ClickHouse/ClickHouse/pull/5444) ([Vitaly Baranov](https://github.com/vitlibar)) +- Fixed build without `Brotli` HTTP compression support (`ENABLE_BROTLI=OFF` cmake variable). [#5521](https://github.com/ClickHouse/ClickHouse/pull/5521) ([Anton Yuzhaninov](https://github.com/citrin)) +- Include roaring.h as roaring/roaring.h [#5523](https://github.com/ClickHouse/ClickHouse/pull/5523) ([Orivej Desh](https://github.com/orivej)) +- Fix gcc9 warnings in hyperscan (#line directive is evil!) [#5546](https://github.com/ClickHouse/ClickHouse/pull/5546) ([Danila Kutenin](https://github.com/danlark1)) +- Fix all warnings when compiling with gcc-9. Fix some contrib issues. Fix gcc9 ICE and submit it to bugzilla. [#5498](https://github.com/ClickHouse/ClickHouse/pull/5498) ([Danila Kutenin](https://github.com/danlark1)) +- Fixed linking with lld [#5477](https://github.com/ClickHouse/ClickHouse/pull/5477) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Remove unused specializations in dictionaries [#5452](https://github.com/ClickHouse/ClickHouse/pull/5452) ([Artem Zuikov](https://github.com/4ertus2)) +- Improvement performance tests for formatting and parsing tables for different types of files [#5497](https://github.com/ClickHouse/ClickHouse/pull/5497) ([Olga Khvostikova](https://github.com/stavrolia)) +- Fixes for parallel test run [#5506](https://github.com/ClickHouse/ClickHouse/pull/5506) ([proller](https://github.com/proller)) +- Docker: use configs from clickhouse-test [#5531](https://github.com/ClickHouse/ClickHouse/pull/5531) ([proller](https://github.com/proller)) +- Fix compile for FreeBSD [#5447](https://github.com/ClickHouse/ClickHouse/pull/5447) ([proller](https://github.com/proller)) +- Upgrade boost to 1.70 [#5570](https://github.com/ClickHouse/ClickHouse/pull/5570) ([proller](https://github.com/proller)) +- Fix build clickhouse as submodule [#5574](https://github.com/ClickHouse/ClickHouse/pull/5574) ([proller](https://github.com/proller)) +- Improve JSONExtract performance tests [#5444](https://github.com/ClickHouse/ClickHouse/pull/5444) ([Vitaly Baranov](https://github.com/vitlibar)) ## ClickHouse Release 19.8 {#clickhouse-release-19-8} @@ -1237,91 +1237,91 @@ toc_title: '2019' #### New Features {#new-features} -- Added functions to work with 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)) -- Add a function basename, with a similar behaviour to a basename function, which exists in a lot of languages (`os.path.basename` in 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)) -- Added `LIMIT n, m BY` or `LIMIT m OFFSET n BY` syntax to set offset of n for LIMIT BY clause. [\#5138](https://github.com/ClickHouse/ClickHouse/pull/5138) ([Anton Popov](https://github.com/CurtizJ)) -- Added new data type `SimpleAggregateFunction`, which allows to have columns with light aggregation in an `AggregatingMergeTree`. This can only be used with simple functions like `any`, `anyLast`, `sum`, `min`, `max`. [\#4629](https://github.com/ClickHouse/ClickHouse/pull/4629) ([Boris Granveaud](https://github.com/bgranvea)) -- Added support for non-constant arguments in function `ngramDistance` [\#5198](https://github.com/ClickHouse/ClickHouse/pull/5198) ([Danila Kutenin](https://github.com/danlark1)) -- Added functions `skewPop`, `skewSamp`, `kurtPop` and `kurtSamp` to compute for sequence skewness, sample skewness, kurtosis and sample kurtosis respectively. [\#5200](https://github.com/ClickHouse/ClickHouse/pull/5200) ([hcz](https://github.com/hczhcz)) -- Support rename operation for `MaterializeView` storage. [\#5209](https://github.com/ClickHouse/ClickHouse/pull/5209) ([Guillaume Tassery](https://github.com/YiuRULE)) -- Added server which allows connecting to ClickHouse using MySQL client. [\#4715](https://github.com/ClickHouse/ClickHouse/pull/4715) ([Yuriy Baranov](https://github.com/yurriy)) -- Add `toDecimal*OrZero` and `toDecimal*OrNull` functions. [\#5291](https://github.com/ClickHouse/ClickHouse/pull/5291) ([Artem Zuikov](https://github.com/4ertus2)) -- Support Decimal types in functions: `quantile`, `quantiles`, `median`, `quantileExactWeighted`, `quantilesExactWeighted`, medianExactWeighted. [\#5304](https://github.com/ClickHouse/ClickHouse/pull/5304) ([Artem Zuikov](https://github.com/4ertus2)) -- Added `toValidUTF8` function, which replaces all invalid UTF-8 characters by replacement character � (U+FFFD). [\#5322](https://github.com/ClickHouse/ClickHouse/pull/5322) ([Danila Kutenin](https://github.com/danlark1)) -- Added `format` function. Formatting constant pattern (simplified Python format pattern) with the strings listed in the arguments. [\#5330](https://github.com/ClickHouse/ClickHouse/pull/5330) ([Danila Kutenin](https://github.com/danlark1)) -- Added `system.detached_parts` table containing information about detached parts of `MergeTree` tables. [\#5353](https://github.com/ClickHouse/ClickHouse/pull/5353) ([akuzm](https://github.com/akuzm)) -- Added `ngramSearch` function to calculate the non-symmetric difference between needle and haystack. [\#5418](https://github.com/ClickHouse/ClickHouse/pull/5418)[\#5422](https://github.com/ClickHouse/ClickHouse/pull/5422) ([Danila Kutenin](https://github.com/danlark1)) -- Implementation of basic machine learning methods (stochastic linear regression and logistic regression) using aggregate functions interface. Has different strategies for updating model weights (simple gradient descent, momentum method, Nesterov method). Also supports mini-batches of custom size. [\#4943](https://github.com/ClickHouse/ClickHouse/pull/4943) ([Quid37](https://github.com/Quid37)) -- Implementation of `geohashEncode` and `geohashDecode` functions. [\#5003](https://github.com/ClickHouse/ClickHouse/pull/5003) ([Vasily Nemkov](https://github.com/Enmk)) -- Added aggregate function `timeSeriesGroupSum`, which can aggregate different time series that sample timestamp not alignment. It will use linear interpolation between two sample timestamp and then sum time-series together. Added aggregate function `timeSeriesGroupRateSum`, which calculates the rate of time-series and then sum rates together. [\#4542](https://github.com/ClickHouse/ClickHouse/pull/4542) ([Yangkuan Liu](https://github.com/LiuYangkuan)) -- Added functions `IPv4CIDRtoIPv4Range` and `IPv6CIDRtoIPv6Range` to calculate the lower and higher bounds for an IP in the subnet using a CIDR. [\#5095](https://github.com/ClickHouse/ClickHouse/pull/5095) ([Guillaume Tassery](https://github.com/YiuRULE)) -- Add a X-ClickHouse-Summary header when we send a query using HTTP with enabled setting `send_progress_in_http_headers`. Return the usual information of X-ClickHouse-Progress, with additional information like how many rows and bytes were inserted in the query. [\#5116](https://github.com/ClickHouse/ClickHouse/pull/5116) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Added functions to work with 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)) +- Add a function basename, with a similar behaviour to a basename function, which exists in a lot of languages (`os.path.basename` in 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)) +- Added `LIMIT n, m BY` or `LIMIT m OFFSET n BY` syntax to set offset of n for LIMIT BY clause. [#5138](https://github.com/ClickHouse/ClickHouse/pull/5138) ([Anton Popov](https://github.com/CurtizJ)) +- Added new data type `SimpleAggregateFunction`, which allows to have columns with light aggregation in an `AggregatingMergeTree`. This can only be used with simple functions like `any`, `anyLast`, `sum`, `min`, `max`. [#4629](https://github.com/ClickHouse/ClickHouse/pull/4629) ([Boris Granveaud](https://github.com/bgranvea)) +- Added support for non-constant arguments in function `ngramDistance` [#5198](https://github.com/ClickHouse/ClickHouse/pull/5198) ([Danila Kutenin](https://github.com/danlark1)) +- Added functions `skewPop`, `skewSamp`, `kurtPop` and `kurtSamp` to compute for sequence skewness, sample skewness, kurtosis and sample kurtosis respectively. [#5200](https://github.com/ClickHouse/ClickHouse/pull/5200) ([hcz](https://github.com/hczhcz)) +- Support rename operation for `MaterializeView` storage. [#5209](https://github.com/ClickHouse/ClickHouse/pull/5209) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Added server which allows connecting to ClickHouse using MySQL client. [#4715](https://github.com/ClickHouse/ClickHouse/pull/4715) ([Yuriy Baranov](https://github.com/yurriy)) +- Add `toDecimal*OrZero` and `toDecimal*OrNull` functions. [#5291](https://github.com/ClickHouse/ClickHouse/pull/5291) ([Artem Zuikov](https://github.com/4ertus2)) +- Support Decimal types in functions: `quantile`, `quantiles`, `median`, `quantileExactWeighted`, `quantilesExactWeighted`, medianExactWeighted. [#5304](https://github.com/ClickHouse/ClickHouse/pull/5304) ([Artem Zuikov](https://github.com/4ertus2)) +- Added `toValidUTF8` function, which replaces all invalid UTF-8 characters by replacement character � (U+FFFD). [#5322](https://github.com/ClickHouse/ClickHouse/pull/5322) ([Danila Kutenin](https://github.com/danlark1)) +- Added `format` function. Formatting constant pattern (simplified Python format pattern) with the strings listed in the arguments. [#5330](https://github.com/ClickHouse/ClickHouse/pull/5330) ([Danila Kutenin](https://github.com/danlark1)) +- Added `system.detached_parts` table containing information about detached parts of `MergeTree` tables. [#5353](https://github.com/ClickHouse/ClickHouse/pull/5353) ([akuzm](https://github.com/akuzm)) +- Added `ngramSearch` function to calculate the non-symmetric difference between needle and haystack. [#5418](https://github.com/ClickHouse/ClickHouse/pull/5418)[#5422](https://github.com/ClickHouse/ClickHouse/pull/5422) ([Danila Kutenin](https://github.com/danlark1)) +- Implementation of basic machine learning methods (stochastic linear regression and logistic regression) using aggregate functions interface. Has different strategies for updating model weights (simple gradient descent, momentum method, Nesterov method). Also supports mini-batches of custom size. [#4943](https://github.com/ClickHouse/ClickHouse/pull/4943) ([Quid37](https://github.com/Quid37)) +- Implementation of `geohashEncode` and `geohashDecode` functions. [#5003](https://github.com/ClickHouse/ClickHouse/pull/5003) ([Vasily Nemkov](https://github.com/Enmk)) +- Added aggregate function `timeSeriesGroupSum`, which can aggregate different time series that sample timestamp not alignment. It will use linear interpolation between two sample timestamp and then sum time-series together. Added aggregate function `timeSeriesGroupRateSum`, which calculates the rate of time-series and then sum rates together. [#4542](https://github.com/ClickHouse/ClickHouse/pull/4542) ([Yangkuan Liu](https://github.com/LiuYangkuan)) +- Added functions `IPv4CIDRtoIPv4Range` and `IPv6CIDRtoIPv6Range` to calculate the lower and higher bounds for an IP in the subnet using a CIDR. [#5095](https://github.com/ClickHouse/ClickHouse/pull/5095) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Add a X-ClickHouse-Summary header when we send a query using HTTP with enabled setting `send_progress_in_http_headers`. Return the usual information of X-ClickHouse-Progress, with additional information like how many rows and bytes were inserted in the query. [#5116](https://github.com/ClickHouse/ClickHouse/pull/5116) ([Guillaume Tassery](https://github.com/YiuRULE)) #### Improvements {#improvements} -- Added `max_parts_in_total` setting for MergeTree family of tables (default: 100 000) that prevents unsafe specification of partition key \#5166. [\#5171](https://github.com/ClickHouse/ClickHouse/pull/5171) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- `clickhouse-obfuscator`: derive seed for individual columns by combining initial seed with column name, not column position. This is intended to transform datasets with multiple related tables, so that tables will remain JOINable after transformation. [\#5178](https://github.com/ClickHouse/ClickHouse/pull/5178) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Added functions `JSONExtractRaw`, `JSONExtractKeyAndValues`. Renamed functions `jsonExtract` to `JSONExtract`. When something goes wrong these functions return the correspondent values, not `NULL`. Modified function `JSONExtract`, now it gets the return type from its last parameter and doesn’t inject nullables. Implemented fallback to RapidJSON in case AVX2 instructions are not available. Simdjson library updated to a new version. [\#5235](https://github.com/ClickHouse/ClickHouse/pull/5235) ([Vitaly Baranov](https://github.com/vitlibar)) -- Now `if` and `multiIf` functions don’t rely on the condition’s `Nullable`, but rely on the branches for sql compatibility. [\#5238](https://github.com/ClickHouse/ClickHouse/pull/5238) ([Jian Wu](https://github.com/janplus)) -- `In` predicate now generates `Null` result from `Null` input like the `Equal` function. [\#5152](https://github.com/ClickHouse/ClickHouse/pull/5152) ([Jian Wu](https://github.com/janplus)) -- Check the time limit every (flush\_interval / poll\_timeout) number of rows from Kafka. This allows to break the reading from Kafka consumer more frequently and to check the time limits for the top-level streams [\#5249](https://github.com/ClickHouse/ClickHouse/pull/5249) ([Ivan](https://github.com/abyss7)) -- Link rdkafka with bundled SASL. It should allow to use SASL SCRAM authentication [\#5253](https://github.com/ClickHouse/ClickHouse/pull/5253) ([Ivan](https://github.com/abyss7)) -- Batched version of RowRefList for ALL JOINS. [\#5267](https://github.com/ClickHouse/ClickHouse/pull/5267) ([Artem Zuikov](https://github.com/4ertus2)) -- clickhouse-server: more informative listen error messages. [\#5268](https://github.com/ClickHouse/ClickHouse/pull/5268) ([proller](https://github.com/proller)) -- Support dictionaries in clickhouse-copier for functions in `` [\#5270](https://github.com/ClickHouse/ClickHouse/pull/5270) ([proller](https://github.com/proller)) +- Added `max_parts_in_total` setting for MergeTree family of tables (default: 100 000) that prevents unsafe specification of partition key #5166. [#5171](https://github.com/ClickHouse/ClickHouse/pull/5171) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- `clickhouse-obfuscator`: derive seed for individual columns by combining initial seed with column name, not column position. This is intended to transform datasets with multiple related tables, so that tables will remain JOINable after transformation. [#5178](https://github.com/ClickHouse/ClickHouse/pull/5178) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Added functions `JSONExtractRaw`, `JSONExtractKeyAndValues`. Renamed functions `jsonExtract` to `JSONExtract`. When something goes wrong these functions return the correspondent values, not `NULL`. Modified function `JSONExtract`, now it gets the return type from its last parameter and doesn’t inject nullables. Implemented fallback to RapidJSON in case AVX2 instructions are not available. Simdjson library updated to a new version. [#5235](https://github.com/ClickHouse/ClickHouse/pull/5235) ([Vitaly Baranov](https://github.com/vitlibar)) +- Now `if` and `multiIf` functions don’t rely on the condition’s `Nullable`, but rely on the branches for sql compatibility. [#5238](https://github.com/ClickHouse/ClickHouse/pull/5238) ([Jian Wu](https://github.com/janplus)) +- `In` predicate now generates `Null` result from `Null` input like the `Equal` function. [#5152](https://github.com/ClickHouse/ClickHouse/pull/5152) ([Jian Wu](https://github.com/janplus)) +- Check the time limit every (flush_interval / poll_timeout) number of rows from Kafka. This allows to break the reading from Kafka consumer more frequently and to check the time limits for the top-level streams [#5249](https://github.com/ClickHouse/ClickHouse/pull/5249) ([Ivan](https://github.com/abyss7)) +- Link rdkafka with bundled SASL. It should allow to use SASL SCRAM authentication [#5253](https://github.com/ClickHouse/ClickHouse/pull/5253) ([Ivan](https://github.com/abyss7)) +- Batched version of RowRefList for ALL JOINS. [#5267](https://github.com/ClickHouse/ClickHouse/pull/5267) ([Artem Zuikov](https://github.com/4ertus2)) +- clickhouse-server: more informative listen error messages. [#5268](https://github.com/ClickHouse/ClickHouse/pull/5268) ([proller](https://github.com/proller)) +- Support dictionaries in clickhouse-copier for functions in `` [#5270](https://github.com/ClickHouse/ClickHouse/pull/5270) ([proller](https://github.com/proller)) - Add new setting `kafka_commit_every_batch` to regulate Kafka committing policy. - It allows to set commit mode: after every batch of messages is handled, or after the whole block is written to the storage. It’s a trade-off between losing some messages or reading them twice in some extreme situations. [\#5308](https://github.com/ClickHouse/ClickHouse/pull/5308) ([Ivan](https://github.com/abyss7)) -- Make `windowFunnel` support other Unsigned Integer Types. [\#5320](https://github.com/ClickHouse/ClickHouse/pull/5320) ([sundyli](https://github.com/sundy-li)) -- Allow to shadow virtual column `_table` in Merge engine. [\#5325](https://github.com/ClickHouse/ClickHouse/pull/5325) ([Ivan](https://github.com/abyss7)) -- Make `sequenceMatch` aggregate functions support other unsigned Integer types [\#5339](https://github.com/ClickHouse/ClickHouse/pull/5339) ([sundyli](https://github.com/sundy-li)) -- Better error messages if checksum mismatch is most likely caused by hardware failures. [\#5355](https://github.com/ClickHouse/ClickHouse/pull/5355) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Check that underlying tables support sampling for `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)) -- Improvements of MySQL Wire Protocol. Changed name of format to MySQLWire. Using RAII for calling RSA\_free. Disabling SSL if context cannot be created. [\#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) ([proller](https://github.com/proller)) -- Respect query settings in asynchronous INSERTs into Distributed tables. [\#4936](https://github.com/ClickHouse/ClickHouse/pull/4936) ([TCeason](https://github.com/TCeason)) -- Renamed functions `leastSqr` to `simpleLinearRegression`, `LinearRegression` to `linearRegression`, `LogisticRegression` to `logisticRegression`. [\#5391](https://github.com/ClickHouse/ClickHouse/pull/5391) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) + It allows to set commit mode: after every batch of messages is handled, or after the whole block is written to the storage. It’s a trade-off between losing some messages or reading them twice in some extreme situations. [#5308](https://github.com/ClickHouse/ClickHouse/pull/5308) ([Ivan](https://github.com/abyss7)) +- Make `windowFunnel` support other Unsigned Integer Types. [#5320](https://github.com/ClickHouse/ClickHouse/pull/5320) ([sundyli](https://github.com/sundy-li)) +- Allow to shadow virtual column `_table` in Merge engine. [#5325](https://github.com/ClickHouse/ClickHouse/pull/5325) ([Ivan](https://github.com/abyss7)) +- Make `sequenceMatch` aggregate functions support other unsigned Integer types [#5339](https://github.com/ClickHouse/ClickHouse/pull/5339) ([sundyli](https://github.com/sundy-li)) +- Better error messages if checksum mismatch is most likely caused by hardware failures. [#5355](https://github.com/ClickHouse/ClickHouse/pull/5355) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Check that underlying tables support sampling for `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)) +- Improvements of MySQL Wire Protocol. Changed name of format to MySQLWire. Using RAII for calling RSA_free. Disabling SSL if context cannot be created. [#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) ([proller](https://github.com/proller)) +- Respect query settings in asynchronous INSERTs into Distributed tables. [#4936](https://github.com/ClickHouse/ClickHouse/pull/4936) ([TCeason](https://github.com/TCeason)) +- Renamed functions `leastSqr` to `simpleLinearRegression`, `LinearRegression` to `linearRegression`, `LogisticRegression` to `logisticRegression`. [#5391](https://github.com/ClickHouse/ClickHouse/pull/5391) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) #### Performance Improvements {#performance-improvements} -- Parallelize processing of parts of non-replicated MergeTree tables in ALTER MODIFY query. [\#4639](https://github.com/ClickHouse/ClickHouse/pull/4639) ([Ivan Kush](https://github.com/IvanKush)) -- Optimizations in regular expressions extraction. [\#5193](https://github.com/ClickHouse/ClickHouse/pull/5193) [\#5191](https://github.com/ClickHouse/ClickHouse/pull/5191) ([Danila Kutenin](https://github.com/danlark1)) -- Do not add right join key column to join result if it’s used only in join on section. [\#5260](https://github.com/ClickHouse/ClickHouse/pull/5260) ([Artem Zuikov](https://github.com/4ertus2)) -- Freeze the Kafka buffer after first empty response. It avoids multiple invokations of `ReadBuffer::next()` for empty result in some row-parsing streams. [\#5283](https://github.com/ClickHouse/ClickHouse/pull/5283) ([Ivan](https://github.com/abyss7)) -- `concat` function optimization for multiple arguments. [\#5357](https://github.com/ClickHouse/ClickHouse/pull/5357) ([Danila Kutenin](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)) -- Upgrade our LZ4 implementation with reference one to have faster decompression. [\#5070](https://github.com/ClickHouse/ClickHouse/pull/5070) ([Danila Kutenin](https://github.com/danlark1)) -- Implemented MSD radix sort (based on kxsort), and partial sorting. [\#5129](https://github.com/ClickHouse/ClickHouse/pull/5129) ([Evgenii Pravda](https://github.com/kvinty)) +- Parallelize processing of parts of non-replicated MergeTree tables in ALTER MODIFY query. [#4639](https://github.com/ClickHouse/ClickHouse/pull/4639) ([Ivan Kush](https://github.com/IvanKush)) +- Optimizations in regular expressions extraction. [#5193](https://github.com/ClickHouse/ClickHouse/pull/5193) [#5191](https://github.com/ClickHouse/ClickHouse/pull/5191) ([Danila Kutenin](https://github.com/danlark1)) +- Do not add right join key column to join result if it’s used only in join on section. [#5260](https://github.com/ClickHouse/ClickHouse/pull/5260) ([Artem Zuikov](https://github.com/4ertus2)) +- Freeze the Kafka buffer after first empty response. It avoids multiple invokations of `ReadBuffer::next()` for empty result in some row-parsing streams. [#5283](https://github.com/ClickHouse/ClickHouse/pull/5283) ([Ivan](https://github.com/abyss7)) +- `concat` function optimization for multiple arguments. [#5357](https://github.com/ClickHouse/ClickHouse/pull/5357) ([Danila Kutenin](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)) +- Upgrade our LZ4 implementation with reference one to have faster decompression. [#5070](https://github.com/ClickHouse/ClickHouse/pull/5070) ([Danila Kutenin](https://github.com/danlark1)) +- Implemented MSD radix sort (based on kxsort), and partial sorting. [#5129](https://github.com/ClickHouse/ClickHouse/pull/5129) ([Evgenii Pravda](https://github.com/kvinty)) #### Bug Fixes {#bug-fixes} -- Fix push require columns with join [\#5192](https://github.com/ClickHouse/ClickHouse/pull/5192) ([Winter Zhang](https://github.com/zhang2014)) -- Fixed bug, when ClickHouse is run by systemd, the command `sudo service clickhouse-server forcerestart` was not working as expected. [\#5204](https://github.com/ClickHouse/ClickHouse/pull/5204) ([proller](https://github.com/proller)) -- Fix http error codes in DataPartsExchange (interserver http server on 9009 port always returned code 200, even on errors). [\#5216](https://github.com/ClickHouse/ClickHouse/pull/5216) ([proller](https://github.com/proller)) -- Fix SimpleAggregateFunction for String longer than MAX\_SMALL\_STRING\_SIZE [\#5311](https://github.com/ClickHouse/ClickHouse/pull/5311) ([Azat Khuzhin](https://github.com/azat)) -- Fix error for `Decimal` to `Nullable(Decimal)` conversion in IN. Support other Decimal to Decimal conversions (including different scales). [\#5350](https://github.com/ClickHouse/ClickHouse/pull/5350) ([Artem Zuikov](https://github.com/4ertus2)) -- Fixed FPU clobbering in simdjson library that lead to wrong calculation of `uniqHLL` and `uniqCombined` aggregate function and math functions such as `log`. [\#5354](https://github.com/ClickHouse/ClickHouse/pull/5354) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed handling mixed const/nonconst cases in JSON functions. [\#5435](https://github.com/ClickHouse/ClickHouse/pull/5435) ([Vitaly Baranov](https://github.com/vitlibar)) -- Fix `retention` function. Now all conditions that satisfy in a row of data are added to the data state. [\#5119](https://github.com/ClickHouse/ClickHouse/pull/5119) ([小路](https://github.com/nicelulu)) -- Fix result type for `quantileExact` with Decimals. [\#5304](https://github.com/ClickHouse/ClickHouse/pull/5304) ([Artem Zuikov](https://github.com/4ertus2)) +- Fix push require columns with join [#5192](https://github.com/ClickHouse/ClickHouse/pull/5192) ([Winter Zhang](https://github.com/zhang2014)) +- Fixed bug, when ClickHouse is run by systemd, the command `sudo service clickhouse-server forcerestart` was not working as expected. [#5204](https://github.com/ClickHouse/ClickHouse/pull/5204) ([proller](https://github.com/proller)) +- Fix http error codes in DataPartsExchange (interserver http server on 9009 port always returned code 200, even on errors). [#5216](https://github.com/ClickHouse/ClickHouse/pull/5216) ([proller](https://github.com/proller)) +- Fix SimpleAggregateFunction for String longer than MAX_SMALL_STRING_SIZE [#5311](https://github.com/ClickHouse/ClickHouse/pull/5311) ([Azat Khuzhin](https://github.com/azat)) +- Fix error for `Decimal` to `Nullable(Decimal)` conversion in IN. Support other Decimal to Decimal conversions (including different scales). [#5350](https://github.com/ClickHouse/ClickHouse/pull/5350) ([Artem Zuikov](https://github.com/4ertus2)) +- Fixed FPU clobbering in simdjson library that lead to wrong calculation of `uniqHLL` and `uniqCombined` aggregate function and math functions such as `log`. [#5354](https://github.com/ClickHouse/ClickHouse/pull/5354) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed handling mixed const/nonconst cases in JSON functions. [#5435](https://github.com/ClickHouse/ClickHouse/pull/5435) ([Vitaly Baranov](https://github.com/vitlibar)) +- Fix `retention` function. Now all conditions that satisfy in a row of data are added to the data state. [#5119](https://github.com/ClickHouse/ClickHouse/pull/5119) ([小路](https://github.com/nicelulu)) +- Fix result type for `quantileExact` with Decimals. [#5304](https://github.com/ClickHouse/ClickHouse/pull/5304) ([Artem Zuikov](https://github.com/4ertus2)) #### Documentation {#documentation} -- Translate documentation for `CollapsingMergeTree` to chinese. [\#5168](https://github.com/ClickHouse/ClickHouse/pull/5168) ([张风啸](https://github.com/AlexZFX)) +- Translate documentation for `CollapsingMergeTree` to chinese. [#5168](https://github.com/ClickHouse/ClickHouse/pull/5168) ([张风啸](https://github.com/AlexZFX)) - Translate some documentation about table engines to chinese. - [\#5134](https://github.com/ClickHouse/ClickHouse/pull/5134) - [\#5328](https://github.com/ClickHouse/ClickHouse/pull/5328) + [#5134](https://github.com/ClickHouse/ClickHouse/pull/5134) + [#5328](https://github.com/ClickHouse/ClickHouse/pull/5328) ([never lee](https://github.com/neverlee)) #### Build/Testing/Packaging Improvements {#buildtestingpackaging-improvements} -- Fix some sanitizer reports that show probable use-after-free.[\#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)) -- Move performance tests out of separate directories for convenience. [\#5158](https://github.com/ClickHouse/ClickHouse/pull/5158) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix incorrect performance tests. [\#5255](https://github.com/ClickHouse/ClickHouse/pull/5255) ([alesapin](https://github.com/alesapin)) -- Added a tool to calculate checksums caused by bit flips to debug hardware issues. [\#5334](https://github.com/ClickHouse/ClickHouse/pull/5334) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Make runner script more usable. [\#5340](https://github.com/ClickHouse/ClickHouse/pull/5340)[\#5360](https://github.com/ClickHouse/ClickHouse/pull/5360) ([filimonov](https://github.com/filimonov)) -- Add small instruction how to write performance tests. [\#5408](https://github.com/ClickHouse/ClickHouse/pull/5408) ([alesapin](https://github.com/alesapin)) -- Add ability to make substitutions in create, fill and drop query in performance tests [\#5367](https://github.com/ClickHouse/ClickHouse/pull/5367) ([Olga Khvostikova](https://github.com/stavrolia)) +- Fix some sanitizer reports that show probable use-after-free.[#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)) +- Move performance tests out of separate directories for convenience. [#5158](https://github.com/ClickHouse/ClickHouse/pull/5158) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix incorrect performance tests. [#5255](https://github.com/ClickHouse/ClickHouse/pull/5255) ([alesapin](https://github.com/alesapin)) +- Added a tool to calculate checksums caused by bit flips to debug hardware issues. [#5334](https://github.com/ClickHouse/ClickHouse/pull/5334) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Make runner script more usable. [#5340](https://github.com/ClickHouse/ClickHouse/pull/5340)[#5360](https://github.com/ClickHouse/ClickHouse/pull/5360) ([filimonov](https://github.com/filimonov)) +- Add small instruction how to write performance tests. [#5408](https://github.com/ClickHouse/ClickHouse/pull/5408) ([alesapin](https://github.com/alesapin)) +- Add ability to make substitutions in create, fill and drop query in performance tests [#5367](https://github.com/ClickHouse/ClickHouse/pull/5367) ([Olga Khvostikova](https://github.com/stavrolia)) ## ClickHouse Release 19.7 {#clickhouse-release-19-7} @@ -1329,35 +1329,35 @@ toc_title: '2019' #### Bug Fix {#bug-fix-25} -- Fix performance regression in some queries with JOIN. [\#5192](https://github.com/ClickHouse/ClickHouse/pull/5192) ([Winter Zhang](https://github.com/zhang2014)) +- Fix performance regression in some queries with JOIN. [#5192](https://github.com/ClickHouse/ClickHouse/pull/5192) ([Winter Zhang](https://github.com/zhang2014)) ### ClickHouse Release 19.7.5.27, 2019-06-09 {#clickhouse-release-19-7-5-27-2019-06-09} #### New Features {#new-features-1} -- Added bitmap related functions `bitmapHasAny` and `bitmapHasAll` analogous to `hasAny` and `hasAll` functions for arrays. [\#5279](https://github.com/ClickHouse/ClickHouse/pull/5279) ([Sergi Vladykin](https://github.com/svladykin)) +- Added bitmap related functions `bitmapHasAny` and `bitmapHasAll` analogous to `hasAny` and `hasAll` functions for arrays. [#5279](https://github.com/ClickHouse/ClickHouse/pull/5279) ([Sergi Vladykin](https://github.com/svladykin)) #### Bug Fixes {#bug-fixes-1} -- Fix segfault on `minmax` INDEX with Null value. [\#5246](https://github.com/ClickHouse/ClickHouse/pull/5246) ([Nikita Vasilev](https://github.com/nikvas0)) -- Mark all input columns in LIMIT BY as required output. It fixes ‘Not found column’ error in some distributed queries. [\#5407](https://github.com/ClickHouse/ClickHouse/pull/5407) ([Constantin S. Pan](https://github.com/kvap)) -- Fix “Column ‘0’ already exists” error in `SELECT .. PREWHERE` on column with DEFAULT [\#5397](https://github.com/ClickHouse/ClickHouse/pull/5397) ([proller](https://github.com/proller)) -- Fix `ALTER MODIFY TTL` query on `ReplicatedMergeTree`. [\#5539](https://github.com/ClickHouse/ClickHouse/pull/5539/commits) ([Anton Popov](https://github.com/CurtizJ)) -- Don’t crash the server when Kafka consumers have failed to start. [\#5285](https://github.com/ClickHouse/ClickHouse/pull/5285) ([Ivan](https://github.com/abyss7)) -- Fixed bitmap functions produce wrong result. [\#5359](https://github.com/ClickHouse/ClickHouse/pull/5359) ([Andy Yang](https://github.com/andyyzh)) -- Fix element\_count for hashed dictionary (do not include duplicates) [\#5440](https://github.com/ClickHouse/ClickHouse/pull/5440) ([Azat Khuzhin](https://github.com/azat)) -- Use contents of environment variable TZ as the name for timezone. It helps to correctly detect default timezone in some cases.[\#5443](https://github.com/ClickHouse/ClickHouse/pull/5443) ([Ivan](https://github.com/abyss7)) -- Do not try to convert integers in `dictGetT` functions, because it doesn’t work correctly. Throw an exception instead. [\#5446](https://github.com/ClickHouse/ClickHouse/pull/5446) ([Artem Zuikov](https://github.com/4ertus2)) -- Fix settings in ExternalData HTTP request. [\#5455](https://github.com/ClickHouse/ClickHouse/pull/5455) ([Danila +- Fix segfault on `minmax` INDEX with Null value. [#5246](https://github.com/ClickHouse/ClickHouse/pull/5246) ([Nikita Vasilev](https://github.com/nikvas0)) +- Mark all input columns in LIMIT BY as required output. It fixes ‘Not found column’ error in some distributed queries. [#5407](https://github.com/ClickHouse/ClickHouse/pull/5407) ([Constantin S. Pan](https://github.com/kvap)) +- Fix “Column ‘0’ already exists” error in `SELECT .. PREWHERE` on column with DEFAULT [#5397](https://github.com/ClickHouse/ClickHouse/pull/5397) ([proller](https://github.com/proller)) +- Fix `ALTER MODIFY TTL` query on `ReplicatedMergeTree`. [#5539](https://github.com/ClickHouse/ClickHouse/pull/5539/commits) ([Anton Popov](https://github.com/CurtizJ)) +- Don’t crash the server when Kafka consumers have failed to start. [#5285](https://github.com/ClickHouse/ClickHouse/pull/5285) ([Ivan](https://github.com/abyss7)) +- Fixed bitmap functions produce wrong result. [#5359](https://github.com/ClickHouse/ClickHouse/pull/5359) ([Andy Yang](https://github.com/andyyzh)) +- Fix element_count for hashed dictionary (do not include duplicates) [#5440](https://github.com/ClickHouse/ClickHouse/pull/5440) ([Azat Khuzhin](https://github.com/azat)) +- Use contents of environment variable TZ as the name for timezone. It helps to correctly detect default timezone in some cases.[#5443](https://github.com/ClickHouse/ClickHouse/pull/5443) ([Ivan](https://github.com/abyss7)) +- Do not try to convert integers in `dictGetT` functions, because it doesn’t work correctly. Throw an exception instead. [#5446](https://github.com/ClickHouse/ClickHouse/pull/5446) ([Artem Zuikov](https://github.com/4ertus2)) +- Fix settings in ExternalData HTTP request. [#5455](https://github.com/ClickHouse/ClickHouse/pull/5455) ([Danila Kutenin](https://github.com/danlark1)) -- Fix bug when parts were removed only from FS without dropping them from Zookeeper. [\#5520](https://github.com/ClickHouse/ClickHouse/pull/5520) ([alesapin](https://github.com/alesapin)) -- Fix segmentation fault in `bitmapHasAny` function. [\#5528](https://github.com/ClickHouse/ClickHouse/pull/5528) ([Zhichang Yu](https://github.com/yuzhichang)) -- Fixed error when replication connection pool doesn’t retry to resolve host, even when DNS cache was dropped. [\#5534](https://github.com/ClickHouse/ClickHouse/pull/5534) ([alesapin](https://github.com/alesapin)) -- Fixed `DROP INDEX IF EXISTS` query. Now `ALTER TABLE ... DROP INDEX IF EXISTS ...` query doesn’t raise an exception if provided index does not exist. [\#5524](https://github.com/ClickHouse/ClickHouse/pull/5524) ([Gleb Novikov](https://github.com/NanoBjorn)) -- Fix union all supertype column. There were cases with inconsistent data and column types of resulting columns. [\#5503](https://github.com/ClickHouse/ClickHouse/pull/5503) ([Artem Zuikov](https://github.com/4ertus2)) +- Fix bug when parts were removed only from FS without dropping them from Zookeeper. [#5520](https://github.com/ClickHouse/ClickHouse/pull/5520) ([alesapin](https://github.com/alesapin)) +- Fix segmentation fault in `bitmapHasAny` function. [#5528](https://github.com/ClickHouse/ClickHouse/pull/5528) ([Zhichang Yu](https://github.com/yuzhichang)) +- Fixed error when replication connection pool doesn’t retry to resolve host, even when DNS cache was dropped. [#5534](https://github.com/ClickHouse/ClickHouse/pull/5534) ([alesapin](https://github.com/alesapin)) +- Fixed `DROP INDEX IF EXISTS` query. Now `ALTER TABLE ... DROP INDEX IF EXISTS ...` query doesn’t raise an exception if provided index does not exist. [#5524](https://github.com/ClickHouse/ClickHouse/pull/5524) ([Gleb Novikov](https://github.com/NanoBjorn)) +- Fix union all supertype column. There were cases with inconsistent data and column types of resulting columns. [#5503](https://github.com/ClickHouse/ClickHouse/pull/5503) ([Artem Zuikov](https://github.com/4ertus2)) - Skip ZNONODE during DDL query processing. Before if another node removes the znode in task queue, the one that - did not process it, but already get list of children, will terminate the DDLWorker thread. [\#5489](https://github.com/ClickHouse/ClickHouse/pull/5489) ([Azat Khuzhin](https://github.com/azat)) -- Fix INSERT into Distributed() table with MATERIALIZED column. [\#5429](https://github.com/ClickHouse/ClickHouse/pull/5429) ([Azat Khuzhin](https://github.com/azat)) + did not process it, but already get list of children, will terminate the DDLWorker thread. [#5489](https://github.com/ClickHouse/ClickHouse/pull/5489) ([Azat Khuzhin](https://github.com/azat)) +- Fix INSERT into Distributed() table with MATERIALIZED column. [#5429](https://github.com/ClickHouse/ClickHouse/pull/5429) ([Azat Khuzhin](https://github.com/azat)) ### ClickHouse Release 19.7.3.9, 2019-05-30 {#clickhouse-release-19-7-3-9-2019-05-30} @@ -1365,65 +1365,65 @@ toc_title: '2019' - Allow to limit the range of a setting that can be specified by user. These constraints can be set up in user settings profile. - [\#4931](https://github.com/ClickHouse/ClickHouse/pull/4931) ([Vitaly + [#4931](https://github.com/ClickHouse/ClickHouse/pull/4931) ([Vitaly Baranov](https://github.com/vitlibar)) - Add a second version of the function `groupUniqArray` with an optional `max_size` parameter that limits the size of the resulting array. This behavior is similar to `groupArray(max_size)(x)` function. - [\#5026](https://github.com/ClickHouse/ClickHouse/pull/5026) ([Guillaume + [#5026](https://github.com/ClickHouse/ClickHouse/pull/5026) ([Guillaume Tassery](https://github.com/YiuRULE)) - For TSVWithNames/CSVWithNames input file formats, column order can now be determined from file header. This is controlled by `input_format_with_names_use_header` parameter. - [\#5081](https://github.com/ClickHouse/ClickHouse/pull/5081) + [#5081](https://github.com/ClickHouse/ClickHouse/pull/5081) ([Alexander](https://github.com/Akazz)) #### Bug Fixes {#bug-fixes-2} -- Crash with uncompressed\_cache + JOIN during merge (\#5197) - [\#5133](https://github.com/ClickHouse/ClickHouse/pull/5133) ([Danila +- Crash with uncompressed_cache + JOIN during merge (#5197) + [#5133](https://github.com/ClickHouse/ClickHouse/pull/5133) ([Danila Kutenin](https://github.com/danlark1)) -- Segmentation fault on a clickhouse-client query to system tables. \#5066 - [\#5127](https://github.com/ClickHouse/ClickHouse/pull/5127) +- Segmentation fault on a clickhouse-client query to system tables. #5066 + [#5127](https://github.com/ClickHouse/ClickHouse/pull/5127) ([Ivan](https://github.com/abyss7)) -- Data loss on heavy load via KafkaEngine (\#4736) - [\#5080](https://github.com/ClickHouse/ClickHouse/pull/5080) +- Data loss on heavy load via KafkaEngine (#4736) + [#5080](https://github.com/ClickHouse/ClickHouse/pull/5080) ([Ivan](https://github.com/abyss7)) -- Fixed very rare data race condition that could happen when executing a query with UNION ALL involving at least two SELECTs from system.columns, system.tables, system.parts, system.parts\_tables or tables of Merge family and performing ALTER of columns of the related tables concurrently. [\#5189](https://github.com/ClickHouse/ClickHouse/pull/5189) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed very rare data race condition that could happen when executing a query with UNION ALL involving at least two SELECTs from system.columns, system.tables, system.parts, system.parts_tables or tables of Merge family and performing ALTER of columns of the related tables concurrently. [#5189](https://github.com/ClickHouse/ClickHouse/pull/5189) ([alexey-milovidov](https://github.com/alexey-milovidov)) #### Performance Improvements {#performance-improvements-1} - Use radix sort for sorting by single numeric column in `ORDER BY` without - `LIMIT`. [\#5106](https://github.com/ClickHouse/ClickHouse/pull/5106), - [\#4439](https://github.com/ClickHouse/ClickHouse/pull/4439) + `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)) #### Documentation {#documentation-1} - Translate documentation for some table engines to Chinese. - [\#5107](https://github.com/ClickHouse/ClickHouse/pull/5107), - [\#5094](https://github.com/ClickHouse/ClickHouse/pull/5094), - [\#5087](https://github.com/ClickHouse/ClickHouse/pull/5087) + [#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) ([never + [#5068](https://github.com/ClickHouse/ClickHouse/pull/5068) ([never lee](https://github.com/neverlee)) #### Build/Testing/Packaging Improvements {#buildtestingpackaging-improvements-1} - Print UTF-8 characters properly in `clickhouse-test`. - [\#5084](https://github.com/ClickHouse/ClickHouse/pull/5084) + [#5084](https://github.com/ClickHouse/ClickHouse/pull/5084) ([alexey-milovidov](https://github.com/alexey-milovidov)) - Add command line parameter for clickhouse-client to always load suggestion - data. [\#5102](https://github.com/ClickHouse/ClickHouse/pull/5102) + data. [#5102](https://github.com/ClickHouse/ClickHouse/pull/5102) ([alexey-milovidov](https://github.com/alexey-milovidov)) - Resolve some of PVS-Studio warnings. - [\#5082](https://github.com/ClickHouse/ClickHouse/pull/5082) + [#5082](https://github.com/ClickHouse/ClickHouse/pull/5082) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Update LZ4 [\#5040](https://github.com/ClickHouse/ClickHouse/pull/5040) ([Danila +- Update LZ4 [#5040](https://github.com/ClickHouse/ClickHouse/pull/5040) ([Danila Kutenin](https://github.com/danlark1)) -- Add gperf to build requirements for upcoming pull request \#5030. - [\#5110](https://github.com/ClickHouse/ClickHouse/pull/5110) +- Add gperf to build requirements for upcoming pull request #5030. + [#5110](https://github.com/ClickHouse/ClickHouse/pull/5110) ([proller](https://github.com/proller)) ## ClickHouse Release 19.6 {#clickhouse-release-19-6} @@ -1432,60 +1432,60 @@ toc_title: '2019' #### Bug Fixes {#bug-fixes-3} -- Fixed IN condition pushdown for queries from table functions `mysql` and `odbc` and corresponding table engines. This fixes \#3540 and \#2384. [\#5313](https://github.com/ClickHouse/ClickHouse/pull/5313) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix deadlock in Zookeeper. [\#5297](https://github.com/ClickHouse/ClickHouse/pull/5297) ([github1youlc](https://github.com/github1youlc)) -- Allow quoted decimals in CSV. [\#5284](https://github.com/ClickHouse/ClickHouse/pull/5284) ([Artem Zuikov](https://github.com/4ertus2) -- Disallow conversion from float Inf/NaN into Decimals (throw exception). [\#5282](https://github.com/ClickHouse/ClickHouse/pull/5282) ([Artem Zuikov](https://github.com/4ertus2)) -- Fix data race in rename query. [\#5247](https://github.com/ClickHouse/ClickHouse/pull/5247) ([Winter Zhang](https://github.com/zhang2014)) -- Temporarily disable LFAlloc. Usage of LFAlloc might lead to a lot of MAP\_FAILED in allocating UncompressedCache and in a result to crashes of queries at high loaded servers. [cfdba93](https://github.com/ClickHouse/ClickHouse/commit/cfdba938ce22f16efeec504f7f90206a515b1280)([Danila Kutenin](https://github.com/danlark1)) +- Fixed IN condition pushdown for queries from table functions `mysql` and `odbc` and corresponding table engines. This fixes #3540 and #2384. [#5313](https://github.com/ClickHouse/ClickHouse/pull/5313) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix deadlock in Zookeeper. [#5297](https://github.com/ClickHouse/ClickHouse/pull/5297) ([github1youlc](https://github.com/github1youlc)) +- Allow quoted decimals in CSV. [#5284](https://github.com/ClickHouse/ClickHouse/pull/5284) ([Artem Zuikov](https://github.com/4ertus2) +- Disallow conversion from float Inf/NaN into Decimals (throw exception). [#5282](https://github.com/ClickHouse/ClickHouse/pull/5282) ([Artem Zuikov](https://github.com/4ertus2)) +- Fix data race in rename query. [#5247](https://github.com/ClickHouse/ClickHouse/pull/5247) ([Winter Zhang](https://github.com/zhang2014)) +- Temporarily disable LFAlloc. Usage of LFAlloc might lead to a lot of MAP_FAILED in allocating UncompressedCache and in a result to crashes of queries at high loaded servers. [cfdba93](https://github.com/ClickHouse/ClickHouse/commit/cfdba938ce22f16efeec504f7f90206a515b1280)([Danila Kutenin](https://github.com/danlark1)) ### ClickHouse Release 19.6.2.11, 2019-05-13 {#clickhouse-release-19-6-2-11-2019-05-13} #### New Features {#new-features-3} -- TTL expressions for columns and tables. [\#4212](https://github.com/ClickHouse/ClickHouse/pull/4212) ([Anton Popov](https://github.com/CurtizJ)) -- Added support for `brotli` compression for HTTP responses (Accept-Encoding: br) [\#4388](https://github.com/ClickHouse/ClickHouse/pull/4388) ([Mikhail](https://github.com/fandyushin)) -- Added new function `isValidUTF8` for checking whether a set of bytes is correctly utf-8 encoded. [\#4934](https://github.com/ClickHouse/ClickHouse/pull/4934) ([Danila Kutenin](https://github.com/danlark1)) -- Add new load balancing policy `first_or_random` which sends queries to the first specified host and if it’s inaccessible send queries to random hosts of shard. Useful for cross-replication topology setups. [\#5012](https://github.com/ClickHouse/ClickHouse/pull/5012) ([nvartolomei](https://github.com/nvartolomei)) +- TTL expressions for columns and tables. [#4212](https://github.com/ClickHouse/ClickHouse/pull/4212) ([Anton Popov](https://github.com/CurtizJ)) +- Added support for `brotli` compression for HTTP responses (Accept-Encoding: br) [#4388](https://github.com/ClickHouse/ClickHouse/pull/4388) ([Mikhail](https://github.com/fandyushin)) +- Added new function `isValidUTF8` for checking whether a set of bytes is correctly utf-8 encoded. [#4934](https://github.com/ClickHouse/ClickHouse/pull/4934) ([Danila Kutenin](https://github.com/danlark1)) +- Add new load balancing policy `first_or_random` which sends queries to the first specified host and if it’s inaccessible send queries to random hosts of shard. Useful for cross-replication topology setups. [#5012](https://github.com/ClickHouse/ClickHouse/pull/5012) ([nvartolomei](https://github.com/nvartolomei)) #### Experimental Features {#experimental-features-1} -- Add setting `index_granularity_bytes` (adaptive index granularity) for MergeTree\* tables family. [\#4826](https://github.com/ClickHouse/ClickHouse/pull/4826) ([alesapin](https://github.com/alesapin)) +- Add setting `index_granularity_bytes` (adaptive index granularity) for MergeTree\* tables family. [#4826](https://github.com/ClickHouse/ClickHouse/pull/4826) ([alesapin](https://github.com/alesapin)) #### Improvements {#improvements-1} -- Added support for non-constant and negative size and length arguments for function `substringUTF8`. [\#4989](https://github.com/ClickHouse/ClickHouse/pull/4989) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Disable push-down to right table in left join, left table in right join, and both tables in full join. This fixes wrong JOIN results in some cases. [\#4846](https://github.com/ClickHouse/ClickHouse/pull/4846) ([Ivan](https://github.com/abyss7)) -- `clickhouse-copier`: auto upload task configuration from `--task-file` option [\#4876](https://github.com/ClickHouse/ClickHouse/pull/4876) ([proller](https://github.com/proller)) -- Added typos handler for storage factory and table functions factory. [\#4891](https://github.com/ClickHouse/ClickHouse/pull/4891) ([Danila Kutenin](https://github.com/danlark1)) -- Support asterisks and qualified asterisks for multiple joins without subqueries [\#4898](https://github.com/ClickHouse/ClickHouse/pull/4898) ([Artem Zuikov](https://github.com/4ertus2)) -- Make missing column error message more user friendly. [\#4915](https://github.com/ClickHouse/ClickHouse/pull/4915) ([Artem Zuikov](https://github.com/4ertus2)) +- Added support for non-constant and negative size and length arguments for function `substringUTF8`. [#4989](https://github.com/ClickHouse/ClickHouse/pull/4989) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Disable push-down to right table in left join, left table in right join, and both tables in full join. This fixes wrong JOIN results in some cases. [#4846](https://github.com/ClickHouse/ClickHouse/pull/4846) ([Ivan](https://github.com/abyss7)) +- `clickhouse-copier`: auto upload task configuration from `--task-file` option [#4876](https://github.com/ClickHouse/ClickHouse/pull/4876) ([proller](https://github.com/proller)) +- Added typos handler for storage factory and table functions factory. [#4891](https://github.com/ClickHouse/ClickHouse/pull/4891) ([Danila Kutenin](https://github.com/danlark1)) +- Support asterisks and qualified asterisks for multiple joins without subqueries [#4898](https://github.com/ClickHouse/ClickHouse/pull/4898) ([Artem Zuikov](https://github.com/4ertus2)) +- Make missing column error message more user friendly. [#4915](https://github.com/ClickHouse/ClickHouse/pull/4915) ([Artem Zuikov](https://github.com/4ertus2)) #### Performance Improvements {#performance-improvements-2} -- Significant speedup of ASOF JOIN [\#4924](https://github.com/ClickHouse/ClickHouse/pull/4924) ([Martijn Bakker](https://github.com/Gladdy)) +- Significant speedup of ASOF JOIN [#4924](https://github.com/ClickHouse/ClickHouse/pull/4924) ([Martijn Bakker](https://github.com/Gladdy)) #### Backward Incompatible Changes {#backward-incompatible-changes} -- HTTP header `Query-Id` was renamed to `X-ClickHouse-Query-Id` for consistency. [\#4972](https://github.com/ClickHouse/ClickHouse/pull/4972) ([Mikhail](https://github.com/fandyushin)) +- HTTP header `Query-Id` was renamed to `X-ClickHouse-Query-Id` for consistency. [#4972](https://github.com/ClickHouse/ClickHouse/pull/4972) ([Mikhail](https://github.com/fandyushin)) #### Bug Fixes {#bug-fixes-4} -- Fixed potential null pointer dereference in `clickhouse-copier`. [\#4900](https://github.com/ClickHouse/ClickHouse/pull/4900) ([proller](https://github.com/proller)) -- Fixed error on query with JOIN + ARRAY JOIN [\#4938](https://github.com/ClickHouse/ClickHouse/pull/4938) ([Artem Zuikov](https://github.com/4ertus2)) -- Fixed hanging on start of the server when a dictionary depends on another dictionary via a database with engine=Dictionary. [\#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)) -- Fix potentially wrong result for `SELECT DISTINCT` with `JOIN` [\#5001](https://github.com/ClickHouse/ClickHouse/pull/5001) ([Artem Zuikov](https://github.com/4ertus2)) -- Fixed very rare data race condition that could happen when executing a query with UNION ALL involving at least two SELECTs from system.columns, system.tables, system.parts, system.parts\_tables or tables of Merge family and performing ALTER of columns of the related tables concurrently. [\#5189](https://github.com/ClickHouse/ClickHouse/pull/5189) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed potential null pointer dereference in `clickhouse-copier`. [#4900](https://github.com/ClickHouse/ClickHouse/pull/4900) ([proller](https://github.com/proller)) +- Fixed error on query with JOIN + ARRAY JOIN [#4938](https://github.com/ClickHouse/ClickHouse/pull/4938) ([Artem Zuikov](https://github.com/4ertus2)) +- Fixed hanging on start of the server when a dictionary depends on another dictionary via a database with engine=Dictionary. [#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)) +- Fix potentially wrong result for `SELECT DISTINCT` with `JOIN` [#5001](https://github.com/ClickHouse/ClickHouse/pull/5001) ([Artem Zuikov](https://github.com/4ertus2)) +- Fixed very rare data race condition that could happen when executing a query with UNION ALL involving at least two SELECTs from system.columns, system.tables, system.parts, system.parts_tables or tables of Merge family and performing ALTER of columns of the related tables concurrently. [#5189](https://github.com/ClickHouse/ClickHouse/pull/5189) ([alexey-milovidov](https://github.com/alexey-milovidov)) #### Build/Testing/Packaging Improvements {#buildtestingpackaging-improvements-2} -- Fixed test failures when running clickhouse-server on different host [\#4713](https://github.com/ClickHouse/ClickHouse/pull/4713) ([Vasily Nemkov](https://github.com/Enmk)) -- clickhouse-test: Disable color control sequences in non tty environment. [\#4937](https://github.com/ClickHouse/ClickHouse/pull/4937) ([alesapin](https://github.com/alesapin)) -- clickhouse-test: Allow use any test database (remove `test.` qualification where it possible) [\#5008](https://github.com/ClickHouse/ClickHouse/pull/5008) ([proller](https://github.com/proller)) -- Fix ubsan errors [\#5037](https://github.com/ClickHouse/ClickHouse/pull/5037) ([Vitaly Baranov](https://github.com/vitlibar)) -- Yandex LFAlloc was added to ClickHouse to allocate MarkCache and UncompressedCache data in different ways to catch segfaults more reliable [\#4995](https://github.com/ClickHouse/ClickHouse/pull/4995) ([Danila Kutenin](https://github.com/danlark1)) -- Python util to help with backports and changelogs. [\#4949](https://github.com/ClickHouse/ClickHouse/pull/4949) ([Ivan](https://github.com/abyss7)) +- Fixed test failures when running clickhouse-server on different host [#4713](https://github.com/ClickHouse/ClickHouse/pull/4713) ([Vasily Nemkov](https://github.com/Enmk)) +- clickhouse-test: Disable color control sequences in non tty environment. [#4937](https://github.com/ClickHouse/ClickHouse/pull/4937) ([alesapin](https://github.com/alesapin)) +- clickhouse-test: Allow use any test database (remove `test.` qualification where it possible) [#5008](https://github.com/ClickHouse/ClickHouse/pull/5008) ([proller](https://github.com/proller)) +- Fix ubsan errors [#5037](https://github.com/ClickHouse/ClickHouse/pull/5037) ([Vitaly Baranov](https://github.com/vitlibar)) +- Yandex LFAlloc was added to ClickHouse to allocate MarkCache and UncompressedCache data in different ways to catch segfaults more reliable [#4995](https://github.com/ClickHouse/ClickHouse/pull/4995) ([Danila Kutenin](https://github.com/danlark1)) +- Python util to help with backports and changelogs. [#4949](https://github.com/ClickHouse/ClickHouse/pull/4949) ([Ivan](https://github.com/abyss7)) ## ClickHouse Release 19.5 {#clickhouse-release-19-5} @@ -1493,110 +1493,110 @@ toc_title: '2019' #### Bug Fixes {#bug-fixes-5} -- Fixed possible crash in bitmap\* functions [\#5220](https://github.com/ClickHouse/ClickHouse/pull/5220) [\#5228](https://github.com/ClickHouse/ClickHouse/pull/5228) ([Andy Yang](https://github.com/andyyzh)) -- Fixed very rare data race condition that could happen when executing a query with UNION ALL involving at least two SELECTs from system.columns, system.tables, system.parts, system.parts\_tables or tables of Merge family and performing ALTER of columns of the related tables concurrently. [\#5189](https://github.com/ClickHouse/ClickHouse/pull/5189) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed error `Set for IN is not created yet in case of using single LowCardinality column in the left part of IN`. This error happened if LowCardinality column was the part of primary key. \#5031 [\#5154](https://github.com/ClickHouse/ClickHouse/pull/5154) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Modification of retention function: If a row satisfies both the first and NTH condition, only the first satisfied condition is added to the data state. Now all conditions that satisfy in a row of data are added to the data state. [\#5119](https://github.com/ClickHouse/ClickHouse/pull/5119) ([小路](https://github.com/nicelulu)) +- Fixed possible crash in bitmap\* functions [#5220](https://github.com/ClickHouse/ClickHouse/pull/5220) [#5228](https://github.com/ClickHouse/ClickHouse/pull/5228) ([Andy Yang](https://github.com/andyyzh)) +- Fixed very rare data race condition that could happen when executing a query with UNION ALL involving at least two SELECTs from system.columns, system.tables, system.parts, system.parts_tables or tables of Merge family and performing ALTER of columns of the related tables concurrently. [#5189](https://github.com/ClickHouse/ClickHouse/pull/5189) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed error `Set for IN is not created yet in case of using single LowCardinality column in the left part of IN`. This error happened if LowCardinality column was the part of primary key. #5031 [#5154](https://github.com/ClickHouse/ClickHouse/pull/5154) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Modification of retention function: If a row satisfies both the first and NTH condition, only the first satisfied condition is added to the data state. Now all conditions that satisfy in a row of data are added to the data state. [#5119](https://github.com/ClickHouse/ClickHouse/pull/5119) ([小路](https://github.com/nicelulu)) ### ClickHouse Release 19.5.3.8, 2019-04-18 {#clickhouse-release-19-5-3-8-2019-04-18} #### Bug Fixes {#bug-fixes-6} -- Fixed type of setting `max_partitions_per_insert_block` from boolean to UInt64. [\#5028](https://github.com/ClickHouse/ClickHouse/pull/5028) ([Mohammad Hossein Sekhavat](https://github.com/mhsekhavat)) +- Fixed type of setting `max_partitions_per_insert_block` from boolean to UInt64. [#5028](https://github.com/ClickHouse/ClickHouse/pull/5028) ([Mohammad Hossein Sekhavat](https://github.com/mhsekhavat)) ### ClickHouse Release 19.5.2.6, 2019-04-15 {#clickhouse-release-19-5-2-6-2019-04-15} #### New Features {#new-features-4} -- [Hyperscan](https://github.com/intel/hyperscan) multiple regular expression matching was added (functions `multiMatchAny`, `multiMatchAnyIndex`, `multiFuzzyMatchAny`, `multiFuzzyMatchAnyIndex`). [\#4780](https://github.com/ClickHouse/ClickHouse/pull/4780), [\#4841](https://github.com/ClickHouse/ClickHouse/pull/4841) ([Danila Kutenin](https://github.com/danlark1)) -- `multiSearchFirstPosition` function was added. [\#4780](https://github.com/ClickHouse/ClickHouse/pull/4780) ([Danila Kutenin](https://github.com/danlark1)) -- Implement the predefined expression filter per row for tables. [\#4792](https://github.com/ClickHouse/ClickHouse/pull/4792) ([Ivan](https://github.com/abyss7)) -- A new type of data skipping indices based on bloom filters (can be used for `equal`, `in` and `like` functions). [\#4499](https://github.com/ClickHouse/ClickHouse/pull/4499) ([Nikita Vasilev](https://github.com/nikvas0)) -- Added `ASOF JOIN` which allows to run queries that join to the most recent value known. [\#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)) -- Rewrite multiple `COMMA JOIN` to `CROSS JOIN`. Then rewrite them to `INNER JOIN` if possible. [\#4661](https://github.com/ClickHouse/ClickHouse/pull/4661) ([Artem Zuikov](https://github.com/4ertus2)) +- [Hyperscan](https://github.com/intel/hyperscan) multiple regular expression matching was added (functions `multiMatchAny`, `multiMatchAnyIndex`, `multiFuzzyMatchAny`, `multiFuzzyMatchAnyIndex`). [#4780](https://github.com/ClickHouse/ClickHouse/pull/4780), [#4841](https://github.com/ClickHouse/ClickHouse/pull/4841) ([Danila Kutenin](https://github.com/danlark1)) +- `multiSearchFirstPosition` function was added. [#4780](https://github.com/ClickHouse/ClickHouse/pull/4780) ([Danila Kutenin](https://github.com/danlark1)) +- Implement the predefined expression filter per row for tables. [#4792](https://github.com/ClickHouse/ClickHouse/pull/4792) ([Ivan](https://github.com/abyss7)) +- A new type of data skipping indices based on bloom filters (can be used for `equal`, `in` and `like` functions). [#4499](https://github.com/ClickHouse/ClickHouse/pull/4499) ([Nikita Vasilev](https://github.com/nikvas0)) +- Added `ASOF JOIN` which allows to run queries that join to the most recent value known. [#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)) +- Rewrite multiple `COMMA JOIN` to `CROSS JOIN`. Then rewrite them to `INNER JOIN` if possible. [#4661](https://github.com/ClickHouse/ClickHouse/pull/4661) ([Artem Zuikov](https://github.com/4ertus2)) #### Improvement {#improvement-9} -- `topK` and `topKWeighted` now supports custom `loadFactor` (fixes issue [\#4252](https://github.com/ClickHouse/ClickHouse/issues/4252)). [\#4634](https://github.com/ClickHouse/ClickHouse/pull/4634) ([Kirill Danshin](https://github.com/kirillDanshin)) -- Allow to use `parallel_replicas_count > 1` even for tables without sampling (the setting is simply ignored for them). In previous versions it was lead to exception. [\#4637](https://github.com/ClickHouse/ClickHouse/pull/4637) ([Alexey Elymanov](https://github.com/digitalist)) -- Support for `CREATE OR REPLACE VIEW`. Allow to create a view or set a new definition in a single statement. [\#4654](https://github.com/ClickHouse/ClickHouse/pull/4654) ([Boris Granveaud](https://github.com/bgranvea)) -- `Buffer` table engine now supports `PREWHERE`. [\#4671](https://github.com/ClickHouse/ClickHouse/pull/4671) ([Yangkuan Liu](https://github.com/LiuYangkuan)) -- Add ability to start replicated table without metadata in zookeeper in `readonly` mode. [\#4691](https://github.com/ClickHouse/ClickHouse/pull/4691) ([alesapin](https://github.com/alesapin)) -- Fixed flicker of progress bar in clickhouse-client. The issue was most noticeable when using `FORMAT Null` with streaming queries. [\#4811](https://github.com/ClickHouse/ClickHouse/pull/4811) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Allow to disable functions with `hyperscan` library on per user basis to limit potentially excessive and uncontrolled resource usage. [\#4816](https://github.com/ClickHouse/ClickHouse/pull/4816) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Add version number logging in all errors. [\#4824](https://github.com/ClickHouse/ClickHouse/pull/4824) ([proller](https://github.com/proller)) -- Added restriction to the `multiMatch` functions which requires string size to fit into `unsigned int`. Also added the number of arguments limit to the `multiSearch` functions. [\#4834](https://github.com/ClickHouse/ClickHouse/pull/4834) ([Danila Kutenin](https://github.com/danlark1)) -- Improved usage of scratch space and error handling in Hyperscan. [\#4866](https://github.com/ClickHouse/ClickHouse/pull/4866) ([Danila Kutenin](https://github.com/danlark1)) -- Fill `system.graphite_detentions` from a table config of `*GraphiteMergeTree` engine tables. [\#4584](https://github.com/ClickHouse/ClickHouse/pull/4584) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) -- Rename `trigramDistance` function to `ngramDistance` and add more functions with `CaseInsensitive` and `UTF`. [\#4602](https://github.com/ClickHouse/ClickHouse/pull/4602) ([Danila Kutenin](https://github.com/danlark1)) -- Improved data skipping indices calculation. [\#4640](https://github.com/ClickHouse/ClickHouse/pull/4640) ([Nikita Vasilev](https://github.com/nikvas0)) -- Keep ordinary, `DEFAULT`, `MATERIALIZED` and `ALIAS` columns in a single list (fixes issue [\#2867](https://github.com/ClickHouse/ClickHouse/issues/2867)). [\#4707](https://github.com/ClickHouse/ClickHouse/pull/4707) ([Alex Zatelepin](https://github.com/ztlpn)) +- `topK` and `topKWeighted` now supports custom `loadFactor` (fixes issue [#4252](https://github.com/ClickHouse/ClickHouse/issues/4252)). [#4634](https://github.com/ClickHouse/ClickHouse/pull/4634) ([Kirill Danshin](https://github.com/kirillDanshin)) +- Allow to use `parallel_replicas_count > 1` even for tables without sampling (the setting is simply ignored for them). In previous versions it was lead to exception. [#4637](https://github.com/ClickHouse/ClickHouse/pull/4637) ([Alexey Elymanov](https://github.com/digitalist)) +- Support for `CREATE OR REPLACE VIEW`. Allow to create a view or set a new definition in a single statement. [#4654](https://github.com/ClickHouse/ClickHouse/pull/4654) ([Boris Granveaud](https://github.com/bgranvea)) +- `Buffer` table engine now supports `PREWHERE`. [#4671](https://github.com/ClickHouse/ClickHouse/pull/4671) ([Yangkuan Liu](https://github.com/LiuYangkuan)) +- Add ability to start replicated table without metadata in zookeeper in `readonly` mode. [#4691](https://github.com/ClickHouse/ClickHouse/pull/4691) ([alesapin](https://github.com/alesapin)) +- Fixed flicker of progress bar in clickhouse-client. The issue was most noticeable when using `FORMAT Null` with streaming queries. [#4811](https://github.com/ClickHouse/ClickHouse/pull/4811) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Allow to disable functions with `hyperscan` library on per user basis to limit potentially excessive and uncontrolled resource usage. [#4816](https://github.com/ClickHouse/ClickHouse/pull/4816) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Add version number logging in all errors. [#4824](https://github.com/ClickHouse/ClickHouse/pull/4824) ([proller](https://github.com/proller)) +- Added restriction to the `multiMatch` functions which requires string size to fit into `unsigned int`. Also added the number of arguments limit to the `multiSearch` functions. [#4834](https://github.com/ClickHouse/ClickHouse/pull/4834) ([Danila Kutenin](https://github.com/danlark1)) +- Improved usage of scratch space and error handling in Hyperscan. [#4866](https://github.com/ClickHouse/ClickHouse/pull/4866) ([Danila Kutenin](https://github.com/danlark1)) +- Fill `system.graphite_detentions` from a table config of `*GraphiteMergeTree` engine tables. [#4584](https://github.com/ClickHouse/ClickHouse/pull/4584) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) +- Rename `trigramDistance` function to `ngramDistance` and add more functions with `CaseInsensitive` and `UTF`. [#4602](https://github.com/ClickHouse/ClickHouse/pull/4602) ([Danila Kutenin](https://github.com/danlark1)) +- Improved data skipping indices calculation. [#4640](https://github.com/ClickHouse/ClickHouse/pull/4640) ([Nikita Vasilev](https://github.com/nikvas0)) +- Keep ordinary, `DEFAULT`, `MATERIALIZED` and `ALIAS` columns in a single list (fixes issue [#2867](https://github.com/ClickHouse/ClickHouse/issues/2867)). [#4707](https://github.com/ClickHouse/ClickHouse/pull/4707) ([Alex Zatelepin](https://github.com/ztlpn)) #### Bug Fix {#bug-fix-26} -- Avoid `std::terminate` in case of memory allocation failure. Now `std::bad_alloc` exception is thrown as expected. [\#4665](https://github.com/ClickHouse/ClickHouse/pull/4665) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixes capnproto reading from buffer. Sometimes files wasn’t loaded successfully by HTTP. [\#4674](https://github.com/ClickHouse/ClickHouse/pull/4674) ([Vladislav](https://github.com/smirnov-vs)) -- Fix error `Unknown log entry type: 0` after `OPTIMIZE TABLE FINAL` query. [\#4683](https://github.com/ClickHouse/ClickHouse/pull/4683) ([Amos Bird](https://github.com/amosbird)) -- Wrong arguments to `hasAny` or `hasAll` functions may lead to segfault. [\#4698](https://github.com/ClickHouse/ClickHouse/pull/4698) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Deadlock may happen while executing `DROP DATABASE dictionary` query. [\#4701](https://github.com/ClickHouse/ClickHouse/pull/4701) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix undefined behavior in `median` and `quantile` functions. [\#4702](https://github.com/ClickHouse/ClickHouse/pull/4702) ([hcz](https://github.com/hczhcz)) -- Fix compression level detection when `network_compression_method` in lowercase. Broken in v19.1. [\#4706](https://github.com/ClickHouse/ClickHouse/pull/4706) ([proller](https://github.com/proller)) -- Fixed ignorance of `UTC` setting (fixes issue [\#4658](https://github.com/ClickHouse/ClickHouse/issues/4658)). [\#4718](https://github.com/ClickHouse/ClickHouse/pull/4718) ([proller](https://github.com/proller)) -- Fix `histogram` function behaviour with `Distributed` tables. [\#4741](https://github.com/ClickHouse/ClickHouse/pull/4741) ([olegkv](https://github.com/olegkv)) -- Fixed tsan report `destroy of a locked mutex`. [\#4742](https://github.com/ClickHouse/ClickHouse/pull/4742) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed TSan report on shutdown due to race condition in system logs usage. Fixed potential use-after-free on shutdown when part\_log is enabled. [\#4758](https://github.com/ClickHouse/ClickHouse/pull/4758) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix recheck parts in `ReplicatedMergeTreeAlterThread` in case of error. [\#4772](https://github.com/ClickHouse/ClickHouse/pull/4772) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Arithmetic operations on intermediate aggregate function states were not working for constant arguments (such as subquery results). [\#4776](https://github.com/ClickHouse/ClickHouse/pull/4776) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Always backquote column names in metadata. Otherwise it’s impossible to create a table with column named `index` (server won’t restart due to malformed `ATTACH` query in metadata). [\#4782](https://github.com/ClickHouse/ClickHouse/pull/4782) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix crash in `ALTER ... MODIFY ORDER BY` on `Distributed` table. [\#4790](https://github.com/ClickHouse/ClickHouse/pull/4790) ([TCeason](https://github.com/TCeason)) -- Fix segfault in `JOIN ON` with enabled `enable_optimize_predicate_expression`. [\#4794](https://github.com/ClickHouse/ClickHouse/pull/4794) ([Winter Zhang](https://github.com/zhang2014)) -- Fix bug with adding an extraneous row after consuming a protobuf message from Kafka. [\#4808](https://github.com/ClickHouse/ClickHouse/pull/4808) ([Vitaly Baranov](https://github.com/vitlibar)) -- Fix crash of `JOIN` on not-nullable vs nullable column. Fix `NULLs` in right keys in `ANY JOIN` + `join_use_nulls`. [\#4815](https://github.com/ClickHouse/ClickHouse/pull/4815) ([Artem Zuikov](https://github.com/4ertus2)) -- Fix segmentation fault in `clickhouse-copier`. [\#4835](https://github.com/ClickHouse/ClickHouse/pull/4835) ([proller](https://github.com/proller)) -- Fixed race condition in `SELECT` from `system.tables` if the table is renamed or altered concurrently. [\#4836](https://github.com/ClickHouse/ClickHouse/pull/4836) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed data race when fetching data part that is already obsolete. [\#4839](https://github.com/ClickHouse/ClickHouse/pull/4839) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed rare data race that can happen during `RENAME` table of MergeTree family. [\#4844](https://github.com/ClickHouse/ClickHouse/pull/4844) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed segmentation fault in function `arrayIntersect`. Segmentation fault could happen if function was called with mixed constant and ordinary arguments. [\#4847](https://github.com/ClickHouse/ClickHouse/pull/4847) ([Lixiang Qian](https://github.com/fancyqlx)) -- Fixed reading from `Array(LowCardinality)` column in rare case when column contained a long sequence of empty arrays. [\#4850](https://github.com/ClickHouse/ClickHouse/pull/4850) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Fix crash in `FULL/RIGHT JOIN` when we joining on nullable vs not nullable. [\#4855](https://github.com/ClickHouse/ClickHouse/pull/4855) ([Artem Zuikov](https://github.com/4ertus2)) -- Fix `No message received` exception while fetching parts between replicas. [\#4856](https://github.com/ClickHouse/ClickHouse/pull/4856) ([alesapin](https://github.com/alesapin)) -- Fixed `arrayIntersect` function wrong result in case of several repeated values in single array. [\#4871](https://github.com/ClickHouse/ClickHouse/pull/4871) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Fix a race condition during concurrent `ALTER COLUMN` queries that could lead to a server crash (fixes issue [\#3421](https://github.com/ClickHouse/ClickHouse/issues/3421)). [\#4592](https://github.com/ClickHouse/ClickHouse/pull/4592) ([Alex Zatelepin](https://github.com/ztlpn)) -- Fix incorrect result in `FULL/RIGHT JOIN` with const column. [\#4723](https://github.com/ClickHouse/ClickHouse/pull/4723) ([Artem Zuikov](https://github.com/4ertus2)) -- Fix duplicates in `GLOBAL JOIN` with asterisk. [\#4705](https://github.com/ClickHouse/ClickHouse/pull/4705) ([Artem Zuikov](https://github.com/4ertus2)) -- Fix parameter deduction in `ALTER MODIFY` of column `CODEC` when column type is not specified. [\#4883](https://github.com/ClickHouse/ClickHouse/pull/4883) ([alesapin](https://github.com/alesapin)) -- Functions `cutQueryStringAndFragment()` and `queryStringAndFragment()` now works correctly when `URL` contains a fragment and no query. [\#4894](https://github.com/ClickHouse/ClickHouse/pull/4894) ([Vitaly Baranov](https://github.com/vitlibar)) -- Fix rare bug when setting `min_bytes_to_use_direct_io` is greater than zero, which occures when thread have to seek backward in column file. [\#4897](https://github.com/ClickHouse/ClickHouse/pull/4897) ([alesapin](https://github.com/alesapin)) -- Fix wrong argument types for aggregate functions with `LowCardinality` arguments (fixes issue [\#4919](https://github.com/ClickHouse/ClickHouse/issues/4919)). [\#4922](https://github.com/ClickHouse/ClickHouse/pull/4922) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Fix wrong name qualification in `GLOBAL JOIN`. [\#4969](https://github.com/ClickHouse/ClickHouse/pull/4969) ([Artem Zuikov](https://github.com/4ertus2)) -- Fix function `toISOWeek` result for year 1970. [\#4988](https://github.com/ClickHouse/ClickHouse/pull/4988) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix `DROP`, `TRUNCATE` and `OPTIMIZE` queries duplication, when executed on `ON CLUSTER` for `ReplicatedMergeTree*` tables family. [\#4991](https://github.com/ClickHouse/ClickHouse/pull/4991) ([alesapin](https://github.com/alesapin)) +- Avoid `std::terminate` in case of memory allocation failure. Now `std::bad_alloc` exception is thrown as expected. [#4665](https://github.com/ClickHouse/ClickHouse/pull/4665) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixes capnproto reading from buffer. Sometimes files wasn’t loaded successfully by HTTP. [#4674](https://github.com/ClickHouse/ClickHouse/pull/4674) ([Vladislav](https://github.com/smirnov-vs)) +- Fix error `Unknown log entry type: 0` after `OPTIMIZE TABLE FINAL` query. [#4683](https://github.com/ClickHouse/ClickHouse/pull/4683) ([Amos Bird](https://github.com/amosbird)) +- Wrong arguments to `hasAny` or `hasAll` functions may lead to segfault. [#4698](https://github.com/ClickHouse/ClickHouse/pull/4698) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Deadlock may happen while executing `DROP DATABASE dictionary` query. [#4701](https://github.com/ClickHouse/ClickHouse/pull/4701) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix undefined behavior in `median` and `quantile` functions. [#4702](https://github.com/ClickHouse/ClickHouse/pull/4702) ([hcz](https://github.com/hczhcz)) +- Fix compression level detection when `network_compression_method` in lowercase. Broken in v19.1. [#4706](https://github.com/ClickHouse/ClickHouse/pull/4706) ([proller](https://github.com/proller)) +- Fixed ignorance of `UTC` setting (fixes issue [#4658](https://github.com/ClickHouse/ClickHouse/issues/4658)). [#4718](https://github.com/ClickHouse/ClickHouse/pull/4718) ([proller](https://github.com/proller)) +- Fix `histogram` function behaviour with `Distributed` tables. [#4741](https://github.com/ClickHouse/ClickHouse/pull/4741) ([olegkv](https://github.com/olegkv)) +- Fixed tsan report `destroy of a locked mutex`. [#4742](https://github.com/ClickHouse/ClickHouse/pull/4742) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed TSan report on shutdown due to race condition in system logs usage. Fixed potential use-after-free on shutdown when part_log is enabled. [#4758](https://github.com/ClickHouse/ClickHouse/pull/4758) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix recheck parts in `ReplicatedMergeTreeAlterThread` in case of error. [#4772](https://github.com/ClickHouse/ClickHouse/pull/4772) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Arithmetic operations on intermediate aggregate function states were not working for constant arguments (such as subquery results). [#4776](https://github.com/ClickHouse/ClickHouse/pull/4776) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Always backquote column names in metadata. Otherwise it’s impossible to create a table with column named `index` (server won’t restart due to malformed `ATTACH` query in metadata). [#4782](https://github.com/ClickHouse/ClickHouse/pull/4782) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix crash in `ALTER ... MODIFY ORDER BY` on `Distributed` table. [#4790](https://github.com/ClickHouse/ClickHouse/pull/4790) ([TCeason](https://github.com/TCeason)) +- Fix segfault in `JOIN ON` with enabled `enable_optimize_predicate_expression`. [#4794](https://github.com/ClickHouse/ClickHouse/pull/4794) ([Winter Zhang](https://github.com/zhang2014)) +- Fix bug with adding an extraneous row after consuming a protobuf message from Kafka. [#4808](https://github.com/ClickHouse/ClickHouse/pull/4808) ([Vitaly Baranov](https://github.com/vitlibar)) +- Fix crash of `JOIN` on not-nullable vs nullable column. Fix `NULLs` in right keys in `ANY JOIN` + `join_use_nulls`. [#4815](https://github.com/ClickHouse/ClickHouse/pull/4815) ([Artem Zuikov](https://github.com/4ertus2)) +- Fix segmentation fault in `clickhouse-copier`. [#4835](https://github.com/ClickHouse/ClickHouse/pull/4835) ([proller](https://github.com/proller)) +- Fixed race condition in `SELECT` from `system.tables` if the table is renamed or altered concurrently. [#4836](https://github.com/ClickHouse/ClickHouse/pull/4836) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed data race when fetching data part that is already obsolete. [#4839](https://github.com/ClickHouse/ClickHouse/pull/4839) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed rare data race that can happen during `RENAME` table of MergeTree family. [#4844](https://github.com/ClickHouse/ClickHouse/pull/4844) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed segmentation fault in function `arrayIntersect`. Segmentation fault could happen if function was called with mixed constant and ordinary arguments. [#4847](https://github.com/ClickHouse/ClickHouse/pull/4847) ([Lixiang Qian](https://github.com/fancyqlx)) +- Fixed reading from `Array(LowCardinality)` column in rare case when column contained a long sequence of empty arrays. [#4850](https://github.com/ClickHouse/ClickHouse/pull/4850) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fix crash in `FULL/RIGHT JOIN` when we joining on nullable vs not nullable. [#4855](https://github.com/ClickHouse/ClickHouse/pull/4855) ([Artem Zuikov](https://github.com/4ertus2)) +- Fix `No message received` exception while fetching parts between replicas. [#4856](https://github.com/ClickHouse/ClickHouse/pull/4856) ([alesapin](https://github.com/alesapin)) +- Fixed `arrayIntersect` function wrong result in case of several repeated values in single array. [#4871](https://github.com/ClickHouse/ClickHouse/pull/4871) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fix a race condition during concurrent `ALTER COLUMN` queries that could lead to a server crash (fixes issue [#3421](https://github.com/ClickHouse/ClickHouse/issues/3421)). [#4592](https://github.com/ClickHouse/ClickHouse/pull/4592) ([Alex Zatelepin](https://github.com/ztlpn)) +- Fix incorrect result in `FULL/RIGHT JOIN` with const column. [#4723](https://github.com/ClickHouse/ClickHouse/pull/4723) ([Artem Zuikov](https://github.com/4ertus2)) +- Fix duplicates in `GLOBAL JOIN` with asterisk. [#4705](https://github.com/ClickHouse/ClickHouse/pull/4705) ([Artem Zuikov](https://github.com/4ertus2)) +- Fix parameter deduction in `ALTER MODIFY` of column `CODEC` when column type is not specified. [#4883](https://github.com/ClickHouse/ClickHouse/pull/4883) ([alesapin](https://github.com/alesapin)) +- Functions `cutQueryStringAndFragment()` and `queryStringAndFragment()` now works correctly when `URL` contains a fragment and no query. [#4894](https://github.com/ClickHouse/ClickHouse/pull/4894) ([Vitaly Baranov](https://github.com/vitlibar)) +- Fix rare bug when setting `min_bytes_to_use_direct_io` is greater than zero, which occures when thread have to seek backward in column file. [#4897](https://github.com/ClickHouse/ClickHouse/pull/4897) ([alesapin](https://github.com/alesapin)) +- Fix wrong argument types for aggregate functions with `LowCardinality` arguments (fixes issue [#4919](https://github.com/ClickHouse/ClickHouse/issues/4919)). [#4922](https://github.com/ClickHouse/ClickHouse/pull/4922) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fix wrong name qualification in `GLOBAL JOIN`. [#4969](https://github.com/ClickHouse/ClickHouse/pull/4969) ([Artem Zuikov](https://github.com/4ertus2)) +- Fix function `toISOWeek` result for year 1970. [#4988](https://github.com/ClickHouse/ClickHouse/pull/4988) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix `DROP`, `TRUNCATE` and `OPTIMIZE` queries duplication, when executed on `ON CLUSTER` for `ReplicatedMergeTree*` tables family. [#4991](https://github.com/ClickHouse/ClickHouse/pull/4991) ([alesapin](https://github.com/alesapin)) #### Backward Incompatible Change {#backward-incompatible-change-8} -- Rename setting `insert_sample_with_metadata` to setting `input_format_defaults_for_omitted_fields`. [\#4771](https://github.com/ClickHouse/ClickHouse/pull/4771) ([Artem Zuikov](https://github.com/4ertus2)) -- Added setting `max_partitions_per_insert_block` (with value 100 by default). If inserted block contains larger number of partitions, an exception is thrown. Set it to 0 if you want to remove the limit (not recommended). [\#4845](https://github.com/ClickHouse/ClickHouse/pull/4845) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Multi-search functions were renamed (`multiPosition` to `multiSearchAllPositions`, `multiSearch` to `multiSearchAny`, `firstMatch` to `multiSearchFirstIndex`). [\#4780](https://github.com/ClickHouse/ClickHouse/pull/4780) ([Danila Kutenin](https://github.com/danlark1)) +- Rename setting `insert_sample_with_metadata` to setting `input_format_defaults_for_omitted_fields`. [#4771](https://github.com/ClickHouse/ClickHouse/pull/4771) ([Artem Zuikov](https://github.com/4ertus2)) +- Added setting `max_partitions_per_insert_block` (with value 100 by default). If inserted block contains larger number of partitions, an exception is thrown. Set it to 0 if you want to remove the limit (not recommended). [#4845](https://github.com/ClickHouse/ClickHouse/pull/4845) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Multi-search functions were renamed (`multiPosition` to `multiSearchAllPositions`, `multiSearch` to `multiSearchAny`, `firstMatch` to `multiSearchFirstIndex`). [#4780](https://github.com/ClickHouse/ClickHouse/pull/4780) ([Danila Kutenin](https://github.com/danlark1)) #### Performance Improvement {#performance-improvement-6} -- Optimize Volnitsky searcher by inlining, giving about 5-10% search improvement for queries with many needles or many similar bigrams. [\#4862](https://github.com/ClickHouse/ClickHouse/pull/4862) ([Danila Kutenin](https://github.com/danlark1)) -- Fix performance issue when setting `use_uncompressed_cache` is greater than zero, which appeared when all read data contained in cache. [\#4913](https://github.com/ClickHouse/ClickHouse/pull/4913) ([alesapin](https://github.com/alesapin)) +- Optimize Volnitsky searcher by inlining, giving about 5-10% search improvement for queries with many needles or many similar bigrams. [#4862](https://github.com/ClickHouse/ClickHouse/pull/4862) ([Danila Kutenin](https://github.com/danlark1)) +- Fix performance issue when setting `use_uncompressed_cache` is greater than zero, which appeared when all read data contained in cache. [#4913](https://github.com/ClickHouse/ClickHouse/pull/4913) ([alesapin](https://github.com/alesapin)) #### Build/Testing/Packaging Improvement {#buildtestingpackaging-improvement-10} -- Hardening debug build: more granular memory mappings and ASLR; add memory protection for mark cache and index. This allows to find more memory stomping bugs in case when ASan and MSan cannot do it. [\#4632](https://github.com/ClickHouse/ClickHouse/pull/4632) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Add support for cmake variables `ENABLE_PROTOBUF`, `ENABLE_PARQUET` and `ENABLE_BROTLI` which allows to enable/disable the above features (same as we can do for librdkafka, mysql, etc). [\#4669](https://github.com/ClickHouse/ClickHouse/pull/4669) ([Silviu Caragea](https://github.com/silviucpp)) -- Add ability to print process list and stacktraces of all threads if some queries are hung after test run. [\#4675](https://github.com/ClickHouse/ClickHouse/pull/4675) ([alesapin](https://github.com/alesapin)) -- Add retries on `Connection loss` error in `clickhouse-test`. [\#4682](https://github.com/ClickHouse/ClickHouse/pull/4682) ([alesapin](https://github.com/alesapin)) -- Add freebsd build with vagrant and build with thread sanitizer to packager script. [\#4712](https://github.com/ClickHouse/ClickHouse/pull/4712) [\#4748](https://github.com/ClickHouse/ClickHouse/pull/4748) ([alesapin](https://github.com/alesapin)) -- Now user asked for password for user `'default'` during installation. [\#4725](https://github.com/ClickHouse/ClickHouse/pull/4725) ([proller](https://github.com/proller)) -- Suppress warning in `rdkafka` library. [\#4740](https://github.com/ClickHouse/ClickHouse/pull/4740) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Allow ability to build without ssl. [\#4750](https://github.com/ClickHouse/ClickHouse/pull/4750) ([proller](https://github.com/proller)) -- Add a way to launch clickhouse-server image from a custom user. [\#4753](https://github.com/ClickHouse/ClickHouse/pull/4753) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) -- Upgrade contrib boost to 1.69. [\#4793](https://github.com/ClickHouse/ClickHouse/pull/4793) ([proller](https://github.com/proller)) -- Disable usage of `mremap` when compiled with Thread Sanitizer. Surprisingly enough, TSan does not intercept `mremap` (though it does intercept `mmap`, `munmap`) that leads to false positives. Fixed TSan report in stateful tests. [\#4859](https://github.com/ClickHouse/ClickHouse/pull/4859) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Add test checking using format schema via HTTP interface. [\#4864](https://github.com/ClickHouse/ClickHouse/pull/4864) ([Vitaly Baranov](https://github.com/vitlibar)) +- Hardening debug build: more granular memory mappings and ASLR; add memory protection for mark cache and index. This allows to find more memory stomping bugs in case when ASan and MSan cannot do it. [#4632](https://github.com/ClickHouse/ClickHouse/pull/4632) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Add support for cmake variables `ENABLE_PROTOBUF`, `ENABLE_PARQUET` and `ENABLE_BROTLI` which allows to enable/disable the above features (same as we can do for librdkafka, mysql, etc). [#4669](https://github.com/ClickHouse/ClickHouse/pull/4669) ([Silviu Caragea](https://github.com/silviucpp)) +- Add ability to print process list and stacktraces of all threads if some queries are hung after test run. [#4675](https://github.com/ClickHouse/ClickHouse/pull/4675) ([alesapin](https://github.com/alesapin)) +- Add retries on `Connection loss` error in `clickhouse-test`. [#4682](https://github.com/ClickHouse/ClickHouse/pull/4682) ([alesapin](https://github.com/alesapin)) +- Add freebsd build with vagrant and build with thread sanitizer to packager script. [#4712](https://github.com/ClickHouse/ClickHouse/pull/4712) [#4748](https://github.com/ClickHouse/ClickHouse/pull/4748) ([alesapin](https://github.com/alesapin)) +- Now user asked for password for user `'default'` during installation. [#4725](https://github.com/ClickHouse/ClickHouse/pull/4725) ([proller](https://github.com/proller)) +- Suppress warning in `rdkafka` library. [#4740](https://github.com/ClickHouse/ClickHouse/pull/4740) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Allow ability to build without ssl. [#4750](https://github.com/ClickHouse/ClickHouse/pull/4750) ([proller](https://github.com/proller)) +- Add a way to launch clickhouse-server image from a custom user. [#4753](https://github.com/ClickHouse/ClickHouse/pull/4753) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) +- Upgrade contrib boost to 1.69. [#4793](https://github.com/ClickHouse/ClickHouse/pull/4793) ([proller](https://github.com/proller)) +- Disable usage of `mremap` when compiled with Thread Sanitizer. Surprisingly enough, TSan does not intercept `mremap` (though it does intercept `mmap`, `munmap`) that leads to false positives. Fixed TSan report in stateful tests. [#4859](https://github.com/ClickHouse/ClickHouse/pull/4859) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Add test checking using format schema via HTTP interface. [#4864](https://github.com/ClickHouse/ClickHouse/pull/4864) ([Vitaly Baranov](https://github.com/vitlibar)) ## ClickHouse Release 19.4 {#clickhouse-release-19-4} @@ -1604,129 +1604,129 @@ toc_title: '2019' #### Bug Fixes {#bug-fixes-7} -- Avoid `std::terminate` in case of memory allocation failure. Now `std::bad_alloc` exception is thrown as expected. [\#4665](https://github.com/ClickHouse/ClickHouse/pull/4665) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixes capnproto reading from buffer. Sometimes files wasn’t loaded successfully by HTTP. [\#4674](https://github.com/ClickHouse/ClickHouse/pull/4674) ([Vladislav](https://github.com/smirnov-vs)) -- Fix error `Unknown log entry type: 0` after `OPTIMIZE TABLE FINAL` query. [\#4683](https://github.com/ClickHouse/ClickHouse/pull/4683) ([Amos Bird](https://github.com/amosbird)) -- Wrong arguments to `hasAny` or `hasAll` functions may lead to segfault. [\#4698](https://github.com/ClickHouse/ClickHouse/pull/4698) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Deadlock may happen while executing `DROP DATABASE dictionary` query. [\#4701](https://github.com/ClickHouse/ClickHouse/pull/4701) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix undefined behavior in `median` and `quantile` functions. [\#4702](https://github.com/ClickHouse/ClickHouse/pull/4702) ([hcz](https://github.com/hczhcz)) -- Fix compression level detection when `network_compression_method` in lowercase. Broken in v19.1. [\#4706](https://github.com/ClickHouse/ClickHouse/pull/4706) ([proller](https://github.com/proller)) -- Fixed ignorance of `UTC` setting (fixes issue [\#4658](https://github.com/ClickHouse/ClickHouse/issues/4658)). [\#4718](https://github.com/ClickHouse/ClickHouse/pull/4718) ([proller](https://github.com/proller)) -- Fix `histogram` function behaviour with `Distributed` tables. [\#4741](https://github.com/ClickHouse/ClickHouse/pull/4741) ([olegkv](https://github.com/olegkv)) -- Fixed tsan report `destroy of a locked mutex`. [\#4742](https://github.com/ClickHouse/ClickHouse/pull/4742) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed TSan report on shutdown due to race condition in system logs usage. Fixed potential use-after-free on shutdown when part\_log is enabled. [\#4758](https://github.com/ClickHouse/ClickHouse/pull/4758) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix recheck parts in `ReplicatedMergeTreeAlterThread` in case of error. [\#4772](https://github.com/ClickHouse/ClickHouse/pull/4772) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Arithmetic operations on intermediate aggregate function states were not working for constant arguments (such as subquery results). [\#4776](https://github.com/ClickHouse/ClickHouse/pull/4776) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Always backquote column names in metadata. Otherwise it’s impossible to create a table with column named `index` (server won’t restart due to malformed `ATTACH` query in metadata). [\#4782](https://github.com/ClickHouse/ClickHouse/pull/4782) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix crash in `ALTER ... MODIFY ORDER BY` on `Distributed` table. [\#4790](https://github.com/ClickHouse/ClickHouse/pull/4790) ([TCeason](https://github.com/TCeason)) -- Fix segfault in `JOIN ON` with enabled `enable_optimize_predicate_expression`. [\#4794](https://github.com/ClickHouse/ClickHouse/pull/4794) ([Winter Zhang](https://github.com/zhang2014)) -- Fix bug with adding an extraneous row after consuming a protobuf message from Kafka. [\#4808](https://github.com/ClickHouse/ClickHouse/pull/4808) ([Vitaly Baranov](https://github.com/vitlibar)) -- Fix segmentation fault in `clickhouse-copier`. [\#4835](https://github.com/ClickHouse/ClickHouse/pull/4835) ([proller](https://github.com/proller)) -- Fixed race condition in `SELECT` from `system.tables` if the table is renamed or altered concurrently. [\#4836](https://github.com/ClickHouse/ClickHouse/pull/4836) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed data race when fetching data part that is already obsolete. [\#4839](https://github.com/ClickHouse/ClickHouse/pull/4839) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed rare data race that can happen during `RENAME` table of MergeTree family. [\#4844](https://github.com/ClickHouse/ClickHouse/pull/4844) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed segmentation fault in function `arrayIntersect`. Segmentation fault could happen if function was called with mixed constant and ordinary arguments. [\#4847](https://github.com/ClickHouse/ClickHouse/pull/4847) ([Lixiang Qian](https://github.com/fancyqlx)) -- Fixed reading from `Array(LowCardinality)` column in rare case when column contained a long sequence of empty arrays. [\#4850](https://github.com/ClickHouse/ClickHouse/pull/4850) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Fix `No message received` exception while fetching parts between replicas. [\#4856](https://github.com/ClickHouse/ClickHouse/pull/4856) ([alesapin](https://github.com/alesapin)) -- Fixed `arrayIntersect` function wrong result in case of several repeated values in single array. [\#4871](https://github.com/ClickHouse/ClickHouse/pull/4871) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Fix a race condition during concurrent `ALTER COLUMN` queries that could lead to a server crash (fixes issue [\#3421](https://github.com/ClickHouse/ClickHouse/issues/3421)). [\#4592](https://github.com/ClickHouse/ClickHouse/pull/4592) ([Alex Zatelepin](https://github.com/ztlpn)) -- Fix parameter deduction in `ALTER MODIFY` of column `CODEC` when column type is not specified. [\#4883](https://github.com/ClickHouse/ClickHouse/pull/4883) ([alesapin](https://github.com/alesapin)) -- Functions `cutQueryStringAndFragment()` and `queryStringAndFragment()` now works correctly when `URL` contains a fragment and no query. [\#4894](https://github.com/ClickHouse/ClickHouse/pull/4894) ([Vitaly Baranov](https://github.com/vitlibar)) -- Fix rare bug when setting `min_bytes_to_use_direct_io` is greater than zero, which occures when thread have to seek backward in column file. [\#4897](https://github.com/ClickHouse/ClickHouse/pull/4897) ([alesapin](https://github.com/alesapin)) -- Fix wrong argument types for aggregate functions with `LowCardinality` arguments (fixes issue [\#4919](https://github.com/ClickHouse/ClickHouse/issues/4919)). [\#4922](https://github.com/ClickHouse/ClickHouse/pull/4922) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Fix function `toISOWeek` result for year 1970. [\#4988](https://github.com/ClickHouse/ClickHouse/pull/4988) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix `DROP`, `TRUNCATE` and `OPTIMIZE` queries duplication, when executed on `ON CLUSTER` for `ReplicatedMergeTree*` tables family. [\#4991](https://github.com/ClickHouse/ClickHouse/pull/4991) ([alesapin](https://github.com/alesapin)) +- Avoid `std::terminate` in case of memory allocation failure. Now `std::bad_alloc` exception is thrown as expected. [#4665](https://github.com/ClickHouse/ClickHouse/pull/4665) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixes capnproto reading from buffer. Sometimes files wasn’t loaded successfully by HTTP. [#4674](https://github.com/ClickHouse/ClickHouse/pull/4674) ([Vladislav](https://github.com/smirnov-vs)) +- Fix error `Unknown log entry type: 0` after `OPTIMIZE TABLE FINAL` query. [#4683](https://github.com/ClickHouse/ClickHouse/pull/4683) ([Amos Bird](https://github.com/amosbird)) +- Wrong arguments to `hasAny` or `hasAll` functions may lead to segfault. [#4698](https://github.com/ClickHouse/ClickHouse/pull/4698) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Deadlock may happen while executing `DROP DATABASE dictionary` query. [#4701](https://github.com/ClickHouse/ClickHouse/pull/4701) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix undefined behavior in `median` and `quantile` functions. [#4702](https://github.com/ClickHouse/ClickHouse/pull/4702) ([hcz](https://github.com/hczhcz)) +- Fix compression level detection when `network_compression_method` in lowercase. Broken in v19.1. [#4706](https://github.com/ClickHouse/ClickHouse/pull/4706) ([proller](https://github.com/proller)) +- Fixed ignorance of `UTC` setting (fixes issue [#4658](https://github.com/ClickHouse/ClickHouse/issues/4658)). [#4718](https://github.com/ClickHouse/ClickHouse/pull/4718) ([proller](https://github.com/proller)) +- Fix `histogram` function behaviour with `Distributed` tables. [#4741](https://github.com/ClickHouse/ClickHouse/pull/4741) ([olegkv](https://github.com/olegkv)) +- Fixed tsan report `destroy of a locked mutex`. [#4742](https://github.com/ClickHouse/ClickHouse/pull/4742) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed TSan report on shutdown due to race condition in system logs usage. Fixed potential use-after-free on shutdown when part_log is enabled. [#4758](https://github.com/ClickHouse/ClickHouse/pull/4758) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix recheck parts in `ReplicatedMergeTreeAlterThread` in case of error. [#4772](https://github.com/ClickHouse/ClickHouse/pull/4772) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Arithmetic operations on intermediate aggregate function states were not working for constant arguments (such as subquery results). [#4776](https://github.com/ClickHouse/ClickHouse/pull/4776) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Always backquote column names in metadata. Otherwise it’s impossible to create a table with column named `index` (server won’t restart due to malformed `ATTACH` query in metadata). [#4782](https://github.com/ClickHouse/ClickHouse/pull/4782) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix crash in `ALTER ... MODIFY ORDER BY` on `Distributed` table. [#4790](https://github.com/ClickHouse/ClickHouse/pull/4790) ([TCeason](https://github.com/TCeason)) +- Fix segfault in `JOIN ON` with enabled `enable_optimize_predicate_expression`. [#4794](https://github.com/ClickHouse/ClickHouse/pull/4794) ([Winter Zhang](https://github.com/zhang2014)) +- Fix bug with adding an extraneous row after consuming a protobuf message from Kafka. [#4808](https://github.com/ClickHouse/ClickHouse/pull/4808) ([Vitaly Baranov](https://github.com/vitlibar)) +- Fix segmentation fault in `clickhouse-copier`. [#4835](https://github.com/ClickHouse/ClickHouse/pull/4835) ([proller](https://github.com/proller)) +- Fixed race condition in `SELECT` from `system.tables` if the table is renamed or altered concurrently. [#4836](https://github.com/ClickHouse/ClickHouse/pull/4836) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed data race when fetching data part that is already obsolete. [#4839](https://github.com/ClickHouse/ClickHouse/pull/4839) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed rare data race that can happen during `RENAME` table of MergeTree family. [#4844](https://github.com/ClickHouse/ClickHouse/pull/4844) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed segmentation fault in function `arrayIntersect`. Segmentation fault could happen if function was called with mixed constant and ordinary arguments. [#4847](https://github.com/ClickHouse/ClickHouse/pull/4847) ([Lixiang Qian](https://github.com/fancyqlx)) +- Fixed reading from `Array(LowCardinality)` column in rare case when column contained a long sequence of empty arrays. [#4850](https://github.com/ClickHouse/ClickHouse/pull/4850) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fix `No message received` exception while fetching parts between replicas. [#4856](https://github.com/ClickHouse/ClickHouse/pull/4856) ([alesapin](https://github.com/alesapin)) +- Fixed `arrayIntersect` function wrong result in case of several repeated values in single array. [#4871](https://github.com/ClickHouse/ClickHouse/pull/4871) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fix a race condition during concurrent `ALTER COLUMN` queries that could lead to a server crash (fixes issue [#3421](https://github.com/ClickHouse/ClickHouse/issues/3421)). [#4592](https://github.com/ClickHouse/ClickHouse/pull/4592) ([Alex Zatelepin](https://github.com/ztlpn)) +- Fix parameter deduction in `ALTER MODIFY` of column `CODEC` when column type is not specified. [#4883](https://github.com/ClickHouse/ClickHouse/pull/4883) ([alesapin](https://github.com/alesapin)) +- Functions `cutQueryStringAndFragment()` and `queryStringAndFragment()` now works correctly when `URL` contains a fragment and no query. [#4894](https://github.com/ClickHouse/ClickHouse/pull/4894) ([Vitaly Baranov](https://github.com/vitlibar)) +- Fix rare bug when setting `min_bytes_to_use_direct_io` is greater than zero, which occures when thread have to seek backward in column file. [#4897](https://github.com/ClickHouse/ClickHouse/pull/4897) ([alesapin](https://github.com/alesapin)) +- Fix wrong argument types for aggregate functions with `LowCardinality` arguments (fixes issue [#4919](https://github.com/ClickHouse/ClickHouse/issues/4919)). [#4922](https://github.com/ClickHouse/ClickHouse/pull/4922) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fix function `toISOWeek` result for year 1970. [#4988](https://github.com/ClickHouse/ClickHouse/pull/4988) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix `DROP`, `TRUNCATE` and `OPTIMIZE` queries duplication, when executed on `ON CLUSTER` for `ReplicatedMergeTree*` tables family. [#4991](https://github.com/ClickHouse/ClickHouse/pull/4991) ([alesapin](https://github.com/alesapin)) #### Improvements {#improvements-2} -- Keep ordinary, `DEFAULT`, `MATERIALIZED` and `ALIAS` columns in a single list (fixes issue [\#2867](https://github.com/ClickHouse/ClickHouse/issues/2867)). [\#4707](https://github.com/ClickHouse/ClickHouse/pull/4707) ([Alex Zatelepin](https://github.com/ztlpn)) +- Keep ordinary, `DEFAULT`, `MATERIALIZED` and `ALIAS` columns in a single list (fixes issue [#2867](https://github.com/ClickHouse/ClickHouse/issues/2867)). [#4707](https://github.com/ClickHouse/ClickHouse/pull/4707) ([Alex Zatelepin](https://github.com/ztlpn)) ### ClickHouse Release 19.4.3.11, 2019-04-02 {#clickhouse-release-19-4-3-11-2019-04-02} #### Bug Fixes {#bug-fixes-8} -- Fix crash in `FULL/RIGHT JOIN` when we joining on nullable vs not nullable. [\#4855](https://github.com/ClickHouse/ClickHouse/pull/4855) ([Artem Zuikov](https://github.com/4ertus2)) -- Fix segmentation fault in `clickhouse-copier`. [\#4835](https://github.com/ClickHouse/ClickHouse/pull/4835) ([proller](https://github.com/proller)) +- Fix crash in `FULL/RIGHT JOIN` when we joining on nullable vs not nullable. [#4855](https://github.com/ClickHouse/ClickHouse/pull/4855) ([Artem Zuikov](https://github.com/4ertus2)) +- Fix segmentation fault in `clickhouse-copier`. [#4835](https://github.com/ClickHouse/ClickHouse/pull/4835) ([proller](https://github.com/proller)) #### Build/Testing/Packaging Improvement {#buildtestingpackaging-improvement-11} -- Add a way to launch clickhouse-server image from a custom user. [\#4753](https://github.com/ClickHouse/ClickHouse/pull/4753) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) +- Add a way to launch clickhouse-server image from a custom user. [#4753](https://github.com/ClickHouse/ClickHouse/pull/4753) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) ### ClickHouse Release 19.4.2.7, 2019-03-30 {#clickhouse-release-19-4-2-7-2019-03-30} #### Bug Fixes {#bug-fixes-9} -- Fixed reading from `Array(LowCardinality)` column in rare case when column contained a long sequence of empty arrays. [\#4850](https://github.com/ClickHouse/ClickHouse/pull/4850) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fixed reading from `Array(LowCardinality)` column in rare case when column contained a long sequence of empty arrays. [#4850](https://github.com/ClickHouse/ClickHouse/pull/4850) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) ### ClickHouse Release 19.4.1.3, 2019-03-19 {#clickhouse-release-19-4-1-3-2019-03-19} #### Bug Fixes {#bug-fixes-10} -- Fixed remote queries which contain both `LIMIT BY` and `LIMIT`. Previously, if `LIMIT BY` and `LIMIT` were used for remote query, `LIMIT` could happen before `LIMIT BY`, which led to too filtered result. [\#4708](https://github.com/ClickHouse/ClickHouse/pull/4708) ([Constantin S. Pan](https://github.com/kvap)) +- Fixed remote queries which contain both `LIMIT BY` and `LIMIT`. Previously, if `LIMIT BY` and `LIMIT` were used for remote query, `LIMIT` could happen before `LIMIT BY`, which led to too filtered result. [#4708](https://github.com/ClickHouse/ClickHouse/pull/4708) ([Constantin S. Pan](https://github.com/kvap)) ### ClickHouse Release 19.4.0.49, 2019-03-09 {#clickhouse-release-19-4-0-49-2019-03-09} #### New Features {#new-features-5} -- Added full support for `Protobuf` format (input and output, nested data structures). [\#4174](https://github.com/ClickHouse/ClickHouse/pull/4174) [\#4493](https://github.com/ClickHouse/ClickHouse/pull/4493) ([Vitaly Baranov](https://github.com/vitlibar)) -- Added bitmap functions with Roaring Bitmaps. [\#4207](https://github.com/ClickHouse/ClickHouse/pull/4207) ([Andy Yang](https://github.com/andyyzh)) [\#4568](https://github.com/ClickHouse/ClickHouse/pull/4568) ([Vitaly Baranov](https://github.com/vitlibar)) -- Parquet format support. [\#4448](https://github.com/ClickHouse/ClickHouse/pull/4448) ([proller](https://github.com/proller)) -- N-gram distance was added for fuzzy string comparison. It is similar to q-gram metrics in R language. [\#4466](https://github.com/ClickHouse/ClickHouse/pull/4466) ([Danila Kutenin](https://github.com/danlark1)) -- Combine rules for graphite rollup from dedicated aggregation and retention patterns. [\#4426](https://github.com/ClickHouse/ClickHouse/pull/4426) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) -- Added `max_execution_speed` and `max_execution_speed_bytes` to limit resource usage. Added `min_execution_speed_bytes` setting to complement the `min_execution_speed`. [\#4430](https://github.com/ClickHouse/ClickHouse/pull/4430) ([Winter Zhang](https://github.com/zhang2014)) -- Implemented function `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)) -- Added functions `arrayEnumerateDenseRanked` and `arrayEnumerateUniqRanked` (it’s like `arrayEnumerateUniq` but allows to fine tune array depth to look inside multidimensional arrays). [\#4475](https://github.com/ClickHouse/ClickHouse/pull/4475) ([proller](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)) +- Added full support for `Protobuf` format (input and output, nested data structures). [#4174](https://github.com/ClickHouse/ClickHouse/pull/4174) [#4493](https://github.com/ClickHouse/ClickHouse/pull/4493) ([Vitaly Baranov](https://github.com/vitlibar)) +- Added bitmap functions with Roaring Bitmaps. [#4207](https://github.com/ClickHouse/ClickHouse/pull/4207) ([Andy Yang](https://github.com/andyyzh)) [#4568](https://github.com/ClickHouse/ClickHouse/pull/4568) ([Vitaly Baranov](https://github.com/vitlibar)) +- Parquet format support. [#4448](https://github.com/ClickHouse/ClickHouse/pull/4448) ([proller](https://github.com/proller)) +- N-gram distance was added for fuzzy string comparison. It is similar to q-gram metrics in R language. [#4466](https://github.com/ClickHouse/ClickHouse/pull/4466) ([Danila Kutenin](https://github.com/danlark1)) +- Combine rules for graphite rollup from dedicated aggregation and retention patterns. [#4426](https://github.com/ClickHouse/ClickHouse/pull/4426) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) +- Added `max_execution_speed` and `max_execution_speed_bytes` to limit resource usage. Added `min_execution_speed_bytes` setting to complement the `min_execution_speed`. [#4430](https://github.com/ClickHouse/ClickHouse/pull/4430) ([Winter Zhang](https://github.com/zhang2014)) +- Implemented function `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)) +- Added functions `arrayEnumerateDenseRanked` and `arrayEnumerateUniqRanked` (it’s like `arrayEnumerateUniq` but allows to fine tune array depth to look inside multidimensional arrays). [#4475](https://github.com/ClickHouse/ClickHouse/pull/4475) ([proller](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)) #### Bug Fixes {#bug-fixes-11} - This release also contains all bug fixes from 19.3 and 19.1. -- Fixed bug in data skipping indices: order of granules after INSERT was incorrect. [\#4407](https://github.com/ClickHouse/ClickHouse/pull/4407) ([Nikita Vasilev](https://github.com/nikvas0)) -- Fixed `set` index for `Nullable` and `LowCardinality` columns. Before it, `set` index with `Nullable` or `LowCardinality` column led to error `Data type must be deserialized with multiple streams` while selecting. [\#4594](https://github.com/ClickHouse/ClickHouse/pull/4594) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Correctly set update\_time on full `executable` dictionary update. [\#4551](https://github.com/ClickHouse/ClickHouse/pull/4551) ([Tema Novikov](https://github.com/temoon)) -- Fix broken progress bar in 19.3. [\#4627](https://github.com/ClickHouse/ClickHouse/pull/4627) ([filimonov](https://github.com/filimonov)) -- Fixed inconsistent values of MemoryTracker when memory region was shrinked, in certain cases. [\#4619](https://github.com/ClickHouse/ClickHouse/pull/4619) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed undefined behaviour in ThreadPool. [\#4612](https://github.com/ClickHouse/ClickHouse/pull/4612) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed a very rare crash with the message `mutex lock failed: Invalid argument` that could happen when a MergeTree table was dropped concurrently with a SELECT. [\#4608](https://github.com/ClickHouse/ClickHouse/pull/4608) ([Alex Zatelepin](https://github.com/ztlpn)) -- ODBC driver compatibility with `LowCardinality` data type. [\#4381](https://github.com/ClickHouse/ClickHouse/pull/4381) ([proller](https://github.com/proller)) -- FreeBSD: Fixup for `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` table was created regardless to configuration. [\#4483](https://github.com/ClickHouse/ClickHouse/pull/4483) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix undefined behaviour in `dictIsIn` function for cache dictionaries. [\#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) ([Alex Zatelepin](https://github.com/ztlpn)) -- Disable compile\_expressions by default until we get own `llvm` contrib and can test it with `clang` and `asan`. [\#4579](https://github.com/ClickHouse/ClickHouse/pull/4579) ([alesapin](https://github.com/alesapin)) -- Prevent `std::terminate` when `invalidate_query` for `clickhouse` external dictionary source has returned wrong resultset (empty or more than one row or more than one column). Fixed issue when the `invalidate_query` was performed every five seconds regardless to the `lifetime`. [\#4583](https://github.com/ClickHouse/ClickHouse/pull/4583) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Avoid deadlock when the `invalidate_query` for a dictionary with `clickhouse` source was involving `system.dictionaries` table or `Dictionaries` database (rare case). [\#4599](https://github.com/ClickHouse/ClickHouse/pull/4599) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixes for CROSS JOIN with empty WHERE. [\#4598](https://github.com/ClickHouse/ClickHouse/pull/4598) ([Artem Zuikov](https://github.com/4ertus2)) -- Fixed segfault in function “replicate” when constant argument is passed. [\#4603](https://github.com/ClickHouse/ClickHouse/pull/4603) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix lambda function with predicate optimizer. [\#4408](https://github.com/ClickHouse/ClickHouse/pull/4408) ([Winter Zhang](https://github.com/zhang2014)) -- Multiple JOINs multiple fixes. [\#4595](https://github.com/ClickHouse/ClickHouse/pull/4595) ([Artem Zuikov](https://github.com/4ertus2)) +- Fixed bug in data skipping indices: order of granules after INSERT was incorrect. [#4407](https://github.com/ClickHouse/ClickHouse/pull/4407) ([Nikita Vasilev](https://github.com/nikvas0)) +- Fixed `set` index for `Nullable` and `LowCardinality` columns. Before it, `set` index with `Nullable` or `LowCardinality` column led to error `Data type must be deserialized with multiple streams` while selecting. [#4594](https://github.com/ClickHouse/ClickHouse/pull/4594) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Correctly set update_time on full `executable` dictionary update. [#4551](https://github.com/ClickHouse/ClickHouse/pull/4551) ([Tema Novikov](https://github.com/temoon)) +- Fix broken progress bar in 19.3. [#4627](https://github.com/ClickHouse/ClickHouse/pull/4627) ([filimonov](https://github.com/filimonov)) +- Fixed inconsistent values of MemoryTracker when memory region was shrinked, in certain cases. [#4619](https://github.com/ClickHouse/ClickHouse/pull/4619) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed undefined behaviour in ThreadPool. [#4612](https://github.com/ClickHouse/ClickHouse/pull/4612) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed a very rare crash with the message `mutex lock failed: Invalid argument` that could happen when a MergeTree table was dropped concurrently with a SELECT. [#4608](https://github.com/ClickHouse/ClickHouse/pull/4608) ([Alex Zatelepin](https://github.com/ztlpn)) +- ODBC driver compatibility with `LowCardinality` data type. [#4381](https://github.com/ClickHouse/ClickHouse/pull/4381) ([proller](https://github.com/proller)) +- FreeBSD: Fixup for `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` table was created regardless to configuration. [#4483](https://github.com/ClickHouse/ClickHouse/pull/4483) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix undefined behaviour in `dictIsIn` function for cache dictionaries. [#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) ([Alex Zatelepin](https://github.com/ztlpn)) +- Disable compile_expressions by default until we get own `llvm` contrib and can test it with `clang` and `asan`. [#4579](https://github.com/ClickHouse/ClickHouse/pull/4579) ([alesapin](https://github.com/alesapin)) +- Prevent `std::terminate` when `invalidate_query` for `clickhouse` external dictionary source has returned wrong resultset (empty or more than one row or more than one column). Fixed issue when the `invalidate_query` was performed every five seconds regardless to the `lifetime`. [#4583](https://github.com/ClickHouse/ClickHouse/pull/4583) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Avoid deadlock when the `invalidate_query` for a dictionary with `clickhouse` source was involving `system.dictionaries` table or `Dictionaries` database (rare case). [#4599](https://github.com/ClickHouse/ClickHouse/pull/4599) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixes for CROSS JOIN with empty WHERE. [#4598](https://github.com/ClickHouse/ClickHouse/pull/4598) ([Artem Zuikov](https://github.com/4ertus2)) +- Fixed segfault in function “replicate” when constant argument is passed. [#4603](https://github.com/ClickHouse/ClickHouse/pull/4603) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix lambda function with predicate optimizer. [#4408](https://github.com/ClickHouse/ClickHouse/pull/4408) ([Winter Zhang](https://github.com/zhang2014)) +- Multiple JOINs multiple fixes. [#4595](https://github.com/ClickHouse/ClickHouse/pull/4595) ([Artem Zuikov](https://github.com/4ertus2)) #### Improvements {#improvements-3} -- Support aliases in JOIN ON section for right table columns. [\#4412](https://github.com/ClickHouse/ClickHouse/pull/4412) ([Artem Zuikov](https://github.com/4ertus2)) -- Result of multiple JOINs need correct result names to be used in subselects. Replace flat aliases with source names in result. [\#4474](https://github.com/ClickHouse/ClickHouse/pull/4474) ([Artem Zuikov](https://github.com/4ertus2)) -- Improve push-down logic for joined statements. [\#4387](https://github.com/ClickHouse/ClickHouse/pull/4387) ([Ivan](https://github.com/abyss7)) +- Support aliases in JOIN ON section for right table columns. [#4412](https://github.com/ClickHouse/ClickHouse/pull/4412) ([Artem Zuikov](https://github.com/4ertus2)) +- Result of multiple JOINs need correct result names to be used in subselects. Replace flat aliases with source names in result. [#4474](https://github.com/ClickHouse/ClickHouse/pull/4474) ([Artem Zuikov](https://github.com/4ertus2)) +- Improve push-down logic for joined statements. [#4387](https://github.com/ClickHouse/ClickHouse/pull/4387) ([Ivan](https://github.com/abyss7)) #### Performance Improvements {#performance-improvements-3} -- Improved heuristics of “move to PREWHERE” optimization. [\#4405](https://github.com/ClickHouse/ClickHouse/pull/4405) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Use proper lookup tables that uses HashTable’s API for 8-bit and 16-bit keys. [\#4536](https://github.com/ClickHouse/ClickHouse/pull/4536) ([Amos Bird](https://github.com/amosbird)) -- Improved performance of string comparison. [\#4564](https://github.com/ClickHouse/ClickHouse/pull/4564) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Cleanup distributed DDL queue in a separate thread so that it doesn’t slow down the main loop that processes distributed DDL tasks. [\#4502](https://github.com/ClickHouse/ClickHouse/pull/4502) ([Alex Zatelepin](https://github.com/ztlpn)) -- When `min_bytes_to_use_direct_io` is set to 1, not every file was opened with O\_DIRECT mode because the data size to read was sometimes underestimated by the size of one compressed block. [\#4526](https://github.com/ClickHouse/ClickHouse/pull/4526) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Improved heuristics of “move to PREWHERE” optimization. [#4405](https://github.com/ClickHouse/ClickHouse/pull/4405) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Use proper lookup tables that uses HashTable’s API for 8-bit and 16-bit keys. [#4536](https://github.com/ClickHouse/ClickHouse/pull/4536) ([Amos Bird](https://github.com/amosbird)) +- Improved performance of string comparison. [#4564](https://github.com/ClickHouse/ClickHouse/pull/4564) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Cleanup distributed DDL queue in a separate thread so that it doesn’t slow down the main loop that processes distributed DDL tasks. [#4502](https://github.com/ClickHouse/ClickHouse/pull/4502) ([Alex Zatelepin](https://github.com/ztlpn)) +- When `min_bytes_to_use_direct_io` is set to 1, not every file was opened with O_DIRECT mode because the data size to read was sometimes underestimated by the size of one compressed block. [#4526](https://github.com/ClickHouse/ClickHouse/pull/4526) ([alexey-milovidov](https://github.com/alexey-milovidov)) #### Build/Testing/Packaging Improvement {#buildtestingpackaging-improvement-12} -- Added support for clang-9 [\#4604](https://github.com/ClickHouse/ClickHouse/pull/4604) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix wrong `__asm__` instructions (again) [\#4621](https://github.com/ClickHouse/ClickHouse/pull/4621) ([Konstantin Podshumok](https://github.com/podshumok)) -- Add ability to specify settings for `clickhouse-performance-test` from command line. [\#4437](https://github.com/ClickHouse/ClickHouse/pull/4437) ([alesapin](https://github.com/alesapin)) -- Add dictionaries tests to integration tests. [\#4477](https://github.com/ClickHouse/ClickHouse/pull/4477) ([alesapin](https://github.com/alesapin)) -- Added queries from the benchmark on the website to automated performance tests. [\#4496](https://github.com/ClickHouse/ClickHouse/pull/4496) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- `xxhash.h` does not exist in external lz4 because it is an implementation detail and its symbols are namespaced with `XXH_NAMESPACE` macro. When lz4 is external, xxHash has to be external too, and the dependents have to link to it. [\#4495](https://github.com/ClickHouse/ClickHouse/pull/4495) ([Orivej Desh](https://github.com/orivej)) -- Fixed a case when `quantileTiming` aggregate function can be called with negative or floating point argument (this fixes fuzz test with undefined behaviour sanitizer). [\#4506](https://github.com/ClickHouse/ClickHouse/pull/4506) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Spelling error correction. [\#4531](https://github.com/ClickHouse/ClickHouse/pull/4531) ([sdk2](https://github.com/sdk2)) -- Fix compilation on Mac. [\#4371](https://github.com/ClickHouse/ClickHouse/pull/4371) ([Vitaly Baranov](https://github.com/vitlibar)) -- Build fixes for FreeBSD and various unusual build configurations. [\#4444](https://github.com/ClickHouse/ClickHouse/pull/4444) ([proller](https://github.com/proller)) +- Added support for clang-9 [#4604](https://github.com/ClickHouse/ClickHouse/pull/4604) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix wrong `__asm__` instructions (again) [#4621](https://github.com/ClickHouse/ClickHouse/pull/4621) ([Konstantin Podshumok](https://github.com/podshumok)) +- Add ability to specify settings for `clickhouse-performance-test` from command line. [#4437](https://github.com/ClickHouse/ClickHouse/pull/4437) ([alesapin](https://github.com/alesapin)) +- Add dictionaries tests to integration tests. [#4477](https://github.com/ClickHouse/ClickHouse/pull/4477) ([alesapin](https://github.com/alesapin)) +- Added queries from the benchmark on the website to automated performance tests. [#4496](https://github.com/ClickHouse/ClickHouse/pull/4496) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- `xxhash.h` does not exist in external lz4 because it is an implementation detail and its symbols are namespaced with `XXH_NAMESPACE` macro. When lz4 is external, xxHash has to be external too, and the dependents have to link to it. [#4495](https://github.com/ClickHouse/ClickHouse/pull/4495) ([Orivej Desh](https://github.com/orivej)) +- Fixed a case when `quantileTiming` aggregate function can be called with negative or floating point argument (this fixes fuzz test with undefined behaviour sanitizer). [#4506](https://github.com/ClickHouse/ClickHouse/pull/4506) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Spelling error correction. [#4531](https://github.com/ClickHouse/ClickHouse/pull/4531) ([sdk2](https://github.com/sdk2)) +- Fix compilation on Mac. [#4371](https://github.com/ClickHouse/ClickHouse/pull/4371) ([Vitaly Baranov](https://github.com/vitlibar)) +- Build fixes for FreeBSD and various unusual build configurations. [#4444](https://github.com/ClickHouse/ClickHouse/pull/4444) ([proller](https://github.com/proller)) ## ClickHouse Release 19.3 {#clickhouse-release-19-3} @@ -1734,172 +1734,172 @@ toc_title: '2019' #### Bug Fixes {#bug-fixes-12} -- Fix crash in `FULL/RIGHT JOIN` when we joining on nullable vs not nullable. [\#4855](https://github.com/ClickHouse/ClickHouse/pull/4855) ([Artem Zuikov](https://github.com/4ertus2)) -- Fix segmentation fault in `clickhouse-copier`. [\#4835](https://github.com/ClickHouse/ClickHouse/pull/4835) ([proller](https://github.com/proller)) -- Fixed reading from `Array(LowCardinality)` column in rare case when column contained a long sequence of empty arrays. [\#4850](https://github.com/ClickHouse/ClickHouse/pull/4850) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fix crash in `FULL/RIGHT JOIN` when we joining on nullable vs not nullable. [#4855](https://github.com/ClickHouse/ClickHouse/pull/4855) ([Artem Zuikov](https://github.com/4ertus2)) +- Fix segmentation fault in `clickhouse-copier`. [#4835](https://github.com/ClickHouse/ClickHouse/pull/4835) ([proller](https://github.com/proller)) +- Fixed reading from `Array(LowCardinality)` column in rare case when column contained a long sequence of empty arrays. [#4850](https://github.com/ClickHouse/ClickHouse/pull/4850) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) #### Build/Testing/Packaging Improvement {#buildtestingpackaging-improvement-13} -- Add a way to launch clickhouse-server image from a custom user [\#4753](https://github.com/ClickHouse/ClickHouse/pull/4753) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) +- Add a way to launch clickhouse-server image from a custom user [#4753](https://github.com/ClickHouse/ClickHouse/pull/4753) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) ### ClickHouse Release 19.3.7, 2019-03-12 {#clickhouse-release-19-3-7-2019-03-12} #### Bug Fixes {#bug-fixes-13} -- Fixed error in \#3920. This error manifests itself as random cache corruption (messages `Unknown codec family code`, `Cannot seek through file`) and segfaults. This bug first appeared in version 19.1 and is present in versions up to 19.1.10 and 19.3.6. [\#4623](https://github.com/ClickHouse/ClickHouse/pull/4623) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed error in #3920. This error manifests itself as random cache corruption (messages `Unknown codec family code`, `Cannot seek through file`) and segfaults. This bug first appeared in version 19.1 and is present in versions up to 19.1.10 and 19.3.6. [#4623](https://github.com/ClickHouse/ClickHouse/pull/4623) ([alexey-milovidov](https://github.com/alexey-milovidov)) ### ClickHouse Release 19.3.6, 2019-03-02 {#clickhouse-release-19-3-6-2019-03-02} #### Bug Fixes {#bug-fixes-14} -- When there are more than 1000 threads in a thread pool, `std::terminate` may happen on thread exit. [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)) -- Now it’s possible to create `ReplicatedMergeTree*` tables with comments on columns without defaults and tables with columns codecs without comments and defaults. Also fix comparison of codecs. [\#4523](https://github.com/ClickHouse/ClickHouse/pull/4523) ([alesapin](https://github.com/alesapin)) -- Fixed crash on JOIN with array or tuple. [\#4552](https://github.com/ClickHouse/ClickHouse/pull/4552) ([Artem Zuikov](https://github.com/4ertus2)) -- Fixed crash in clickhouse-copier with the message `ThreadStatus not created`. [\#4540](https://github.com/ClickHouse/ClickHouse/pull/4540) ([Artem Zuikov](https://github.com/4ertus2)) -- Fixed hangup on server shutdown if distributed DDLs were used. [\#4472](https://github.com/ClickHouse/ClickHouse/pull/4472) ([Alex Zatelepin](https://github.com/ztlpn)) -- Incorrect column numbers were printed in error message about text format parsing for columns with number greater than 10. [\#4484](https://github.com/ClickHouse/ClickHouse/pull/4484) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- When there are more than 1000 threads in a thread pool, `std::terminate` may happen on thread exit. [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)) +- Now it’s possible to create `ReplicatedMergeTree*` tables with comments on columns without defaults and tables with columns codecs without comments and defaults. Also fix comparison of codecs. [#4523](https://github.com/ClickHouse/ClickHouse/pull/4523) ([alesapin](https://github.com/alesapin)) +- Fixed crash on JOIN with array or tuple. [#4552](https://github.com/ClickHouse/ClickHouse/pull/4552) ([Artem Zuikov](https://github.com/4ertus2)) +- Fixed crash in clickhouse-copier with the message `ThreadStatus not created`. [#4540](https://github.com/ClickHouse/ClickHouse/pull/4540) ([Artem Zuikov](https://github.com/4ertus2)) +- Fixed hangup on server shutdown if distributed DDLs were used. [#4472](https://github.com/ClickHouse/ClickHouse/pull/4472) ([Alex Zatelepin](https://github.com/ztlpn)) +- Incorrect column numbers were printed in error message about text format parsing for columns with number greater than 10. [#4484](https://github.com/ClickHouse/ClickHouse/pull/4484) ([alexey-milovidov](https://github.com/alexey-milovidov)) #### Build/Testing/Packaging Improvements {#buildtestingpackaging-improvements-3} -- Fixed build with AVX enabled. [\#4527](https://github.com/ClickHouse/ClickHouse/pull/4527) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Enable extended accounting and IO accounting based on good known version instead of kernel under which it is compiled. [\#4541](https://github.com/ClickHouse/ClickHouse/pull/4541) ([nvartolomei](https://github.com/nvartolomei)) -- Allow to skip setting of core\_dump.size\_limit, warning instead of throw if limit set fail. [\#4473](https://github.com/ClickHouse/ClickHouse/pull/4473) ([proller](https://github.com/proller)) -- Removed the `inline` tags of `void readBinary(...)` in `Field.cpp`. Also merged redundant `namespace DB` blocks. [\#4530](https://github.com/ClickHouse/ClickHouse/pull/4530) ([hcz](https://github.com/hczhcz)) +- Fixed build with AVX enabled. [#4527](https://github.com/ClickHouse/ClickHouse/pull/4527) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Enable extended accounting and IO accounting based on good known version instead of kernel under which it is compiled. [#4541](https://github.com/ClickHouse/ClickHouse/pull/4541) ([nvartolomei](https://github.com/nvartolomei)) +- Allow to skip setting of core_dump.size_limit, warning instead of throw if limit set fail. [#4473](https://github.com/ClickHouse/ClickHouse/pull/4473) ([proller](https://github.com/proller)) +- Removed the `inline` tags of `void readBinary(...)` in `Field.cpp`. Also merged redundant `namespace DB` blocks. [#4530](https://github.com/ClickHouse/ClickHouse/pull/4530) ([hcz](https://github.com/hczhcz)) ### ClickHouse Release 19.3.5, 2019-02-21 {#clickhouse-release-19-3-5-2019-02-21} #### Bug Fixes {#bug-fixes-15} -- Fixed bug with large http insert queries processing. [\#4454](https://github.com/ClickHouse/ClickHouse/pull/4454) ([alesapin](https://github.com/alesapin)) -- Fixed backward incompatibility with old versions due to wrong implementation of `send_logs_level` setting. [\#4445](https://github.com/ClickHouse/ClickHouse/pull/4445) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed backward incompatibility of table function `remote` introduced with column comments. [\#4446](https://github.com/ClickHouse/ClickHouse/pull/4446) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed bug with large http insert queries processing. [#4454](https://github.com/ClickHouse/ClickHouse/pull/4454) ([alesapin](https://github.com/alesapin)) +- Fixed backward incompatibility with old versions due to wrong implementation of `send_logs_level` setting. [#4445](https://github.com/ClickHouse/ClickHouse/pull/4445) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed backward incompatibility of table function `remote` introduced with column comments. [#4446](https://github.com/ClickHouse/ClickHouse/pull/4446) ([alexey-milovidov](https://github.com/alexey-milovidov)) ### ClickHouse Release 19.3.4, 2019-02-16 {#clickhouse-release-19-3-4-2019-02-16} #### Improvements {#improvements-4} -- Table index size is not accounted for memory limits when doing `ATTACH TABLE` query. Avoided the possibility that a table cannot be attached after being detached. [\#4396](https://github.com/ClickHouse/ClickHouse/pull/4396) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Slightly raised up the limit on max string and array size received from ZooKeeper. It allows to continue to work with increased size of `CLIENT_JVMFLAGS=-Djute.maxbuffer=...` on ZooKeeper. [\#4398](https://github.com/ClickHouse/ClickHouse/pull/4398) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Allow to repair abandoned replica even if it already has huge number of nodes in its queue. [\#4399](https://github.com/ClickHouse/ClickHouse/pull/4399) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Add one required argument to `SET` index (max stored rows number). [\#4386](https://github.com/ClickHouse/ClickHouse/pull/4386) ([Nikita Vasilev](https://github.com/nikvas0)) +- Table index size is not accounted for memory limits when doing `ATTACH TABLE` query. Avoided the possibility that a table cannot be attached after being detached. [#4396](https://github.com/ClickHouse/ClickHouse/pull/4396) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Slightly raised up the limit on max string and array size received from ZooKeeper. It allows to continue to work with increased size of `CLIENT_JVMFLAGS=-Djute.maxbuffer=...` on ZooKeeper. [#4398](https://github.com/ClickHouse/ClickHouse/pull/4398) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Allow to repair abandoned replica even if it already has huge number of nodes in its queue. [#4399](https://github.com/ClickHouse/ClickHouse/pull/4399) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Add one required argument to `SET` index (max stored rows number). [#4386](https://github.com/ClickHouse/ClickHouse/pull/4386) ([Nikita Vasilev](https://github.com/nikvas0)) #### Bug Fixes {#bug-fixes-16} -- Fixed `WITH ROLLUP` result for group by single `LowCardinality` key. [\#4384](https://github.com/ClickHouse/ClickHouse/pull/4384) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Fixed bug in the set index (dropping a granule if it contains more than `max_rows` rows). [\#4386](https://github.com/ClickHouse/ClickHouse/pull/4386) ([Nikita Vasilev](https://github.com/nikvas0)) -- A lot of FreeBSD build fixes. [\#4397](https://github.com/ClickHouse/ClickHouse/pull/4397) ([proller](https://github.com/proller)) -- Fixed aliases substitution in queries with subquery containing same alias (issue [\#4110](https://github.com/ClickHouse/ClickHouse/issues/4110)). [\#4351](https://github.com/ClickHouse/ClickHouse/pull/4351) ([Artem Zuikov](https://github.com/4ertus2)) +- Fixed `WITH ROLLUP` result for group by single `LowCardinality` key. [#4384](https://github.com/ClickHouse/ClickHouse/pull/4384) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fixed bug in the set index (dropping a granule if it contains more than `max_rows` rows). [#4386](https://github.com/ClickHouse/ClickHouse/pull/4386) ([Nikita Vasilev](https://github.com/nikvas0)) +- A lot of FreeBSD build fixes. [#4397](https://github.com/ClickHouse/ClickHouse/pull/4397) ([proller](https://github.com/proller)) +- Fixed aliases substitution in queries with subquery containing same alias (issue [#4110](https://github.com/ClickHouse/ClickHouse/issues/4110)). [#4351](https://github.com/ClickHouse/ClickHouse/pull/4351) ([Artem Zuikov](https://github.com/4ertus2)) #### Build/Testing/Packaging Improvements {#buildtestingpackaging-improvements-4} -- Add ability to run `clickhouse-server` for stateless tests in docker image. [\#4347](https://github.com/ClickHouse/ClickHouse/pull/4347) ([Vasily Nemkov](https://github.com/Enmk)) +- Add ability to run `clickhouse-server` for stateless tests in docker image. [#4347](https://github.com/ClickHouse/ClickHouse/pull/4347) ([Vasily Nemkov](https://github.com/Enmk)) ### ClickHouse Release 19.3.3, 2019-02-13 {#clickhouse-release-19-3-3-2019-02-13} #### New Features {#new-features-6} -- Added the `KILL MUTATION` statement that allows removing mutations that are for some reasons stuck. Added `latest_failed_part`, `latest_fail_time`, `latest_fail_reason` fields to the `system.mutations` table for easier troubleshooting. [\#4287](https://github.com/ClickHouse/ClickHouse/pull/4287) ([Alex Zatelepin](https://github.com/ztlpn)) -- Added aggregate function `entropy` which computes Shannon entropy. [\#4238](https://github.com/ClickHouse/ClickHouse/pull/4238) ([Quid37](https://github.com/Quid37)) -- Added ability to send queries `INSERT INTO tbl VALUES (....` to server without splitting on `query` and `data` parts. [\#4301](https://github.com/ClickHouse/ClickHouse/pull/4301) ([alesapin](https://github.com/alesapin)) -- Generic implementation of `arrayWithConstant` function was added. [\#4322](https://github.com/ClickHouse/ClickHouse/pull/4322) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Implemented `NOT BETWEEN` comparison operator. [\#4228](https://github.com/ClickHouse/ClickHouse/pull/4228) ([Dmitry Naumov](https://github.com/nezed)) -- Implement `sumMapFiltered` in order to be able to limit the number of keys for which values will be summed by `sumMap`. [\#4129](https://github.com/ClickHouse/ClickHouse/pull/4129) ([Léo Ercolanelli](https://github.com/ercolanelli-leo)) -- Added support of `Nullable` types in `mysql` table function. [\#4198](https://github.com/ClickHouse/ClickHouse/pull/4198) ([Emmanuel Donin de Rosière](https://github.com/edonin)) -- Support for arbitrary constant expressions in `LIMIT` clause. [\#4246](https://github.com/ClickHouse/ClickHouse/pull/4246) ([k3box](https://github.com/k3box)) -- Added `topKWeighted` aggregate function that takes additional argument with (unsigned integer) weight. [\#4245](https://github.com/ClickHouse/ClickHouse/pull/4245) ([Andrew Golman](https://github.com/andrewgolman)) -- `StorageJoin` now supports `join_any_take_last_row` setting that allows overwriting existing values of the same key. [\#3973](https://github.com/ClickHouse/ClickHouse/pull/3973) ([Amos Bird](https://github.com/amosbird) -- Added function `toStartOfInterval`. [\#4304](https://github.com/ClickHouse/ClickHouse/pull/4304) ([Vitaly Baranov](https://github.com/vitlibar)) -- Added `RowBinaryWithNamesAndTypes` format. [\#4200](https://github.com/ClickHouse/ClickHouse/pull/4200) ([Oleg V. Kozlyuk](https://github.com/DarkWanderer)) -- Added `IPv4` and `IPv6` data types. More effective implementations of `IPv*` functions. [\#3669](https://github.com/ClickHouse/ClickHouse/pull/3669) ([Vasily Nemkov](https://github.com/Enmk)) -- Added function `toStartOfTenMinutes()`. [\#4298](https://github.com/ClickHouse/ClickHouse/pull/4298) ([Vitaly Baranov](https://github.com/vitlibar)) -- Added `Protobuf` output format. [\#4005](https://github.com/ClickHouse/ClickHouse/pull/4005) [\#4158](https://github.com/ClickHouse/ClickHouse/pull/4158) ([Vitaly Baranov](https://github.com/vitlibar)) -- Added brotli support for HTTP interface for data import (INSERTs). [\#4235](https://github.com/ClickHouse/ClickHouse/pull/4235) ([Mikhail](https://github.com/fandyushin)) -- Added hints while user make typo in function name or type in command line client. [\#4239](https://github.com/ClickHouse/ClickHouse/pull/4239) ([Danila Kutenin](https://github.com/danlark1)) -- Added `Query-Id` to Server’s HTTP Response header. [\#4231](https://github.com/ClickHouse/ClickHouse/pull/4231) ([Mikhail](https://github.com/fandyushin)) +- Added the `KILL MUTATION` statement that allows removing mutations that are for some reasons stuck. Added `latest_failed_part`, `latest_fail_time`, `latest_fail_reason` fields to the `system.mutations` table for easier troubleshooting. [#4287](https://github.com/ClickHouse/ClickHouse/pull/4287) ([Alex Zatelepin](https://github.com/ztlpn)) +- Added aggregate function `entropy` which computes Shannon entropy. [#4238](https://github.com/ClickHouse/ClickHouse/pull/4238) ([Quid37](https://github.com/Quid37)) +- Added ability to send queries `INSERT INTO tbl VALUES (....` to server without splitting on `query` and `data` parts. [#4301](https://github.com/ClickHouse/ClickHouse/pull/4301) ([alesapin](https://github.com/alesapin)) +- Generic implementation of `arrayWithConstant` function was added. [#4322](https://github.com/ClickHouse/ClickHouse/pull/4322) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Implemented `NOT BETWEEN` comparison operator. [#4228](https://github.com/ClickHouse/ClickHouse/pull/4228) ([Dmitry Naumov](https://github.com/nezed)) +- Implement `sumMapFiltered` in order to be able to limit the number of keys for which values will be summed by `sumMap`. [#4129](https://github.com/ClickHouse/ClickHouse/pull/4129) ([Léo Ercolanelli](https://github.com/ercolanelli-leo)) +- Added support of `Nullable` types in `mysql` table function. [#4198](https://github.com/ClickHouse/ClickHouse/pull/4198) ([Emmanuel Donin de Rosière](https://github.com/edonin)) +- Support for arbitrary constant expressions in `LIMIT` clause. [#4246](https://github.com/ClickHouse/ClickHouse/pull/4246) ([k3box](https://github.com/k3box)) +- Added `topKWeighted` aggregate function that takes additional argument with (unsigned integer) weight. [#4245](https://github.com/ClickHouse/ClickHouse/pull/4245) ([Andrew Golman](https://github.com/andrewgolman)) +- `StorageJoin` now supports `join_any_take_last_row` setting that allows overwriting existing values of the same key. [#3973](https://github.com/ClickHouse/ClickHouse/pull/3973) ([Amos Bird](https://github.com/amosbird) +- Added function `toStartOfInterval`. [#4304](https://github.com/ClickHouse/ClickHouse/pull/4304) ([Vitaly Baranov](https://github.com/vitlibar)) +- Added `RowBinaryWithNamesAndTypes` format. [#4200](https://github.com/ClickHouse/ClickHouse/pull/4200) ([Oleg V. Kozlyuk](https://github.com/DarkWanderer)) +- Added `IPv4` and `IPv6` data types. More effective implementations of `IPv*` functions. [#3669](https://github.com/ClickHouse/ClickHouse/pull/3669) ([Vasily Nemkov](https://github.com/Enmk)) +- Added function `toStartOfTenMinutes()`. [#4298](https://github.com/ClickHouse/ClickHouse/pull/4298) ([Vitaly Baranov](https://github.com/vitlibar)) +- Added `Protobuf` output format. [#4005](https://github.com/ClickHouse/ClickHouse/pull/4005) [#4158](https://github.com/ClickHouse/ClickHouse/pull/4158) ([Vitaly Baranov](https://github.com/vitlibar)) +- Added brotli support for HTTP interface for data import (INSERTs). [#4235](https://github.com/ClickHouse/ClickHouse/pull/4235) ([Mikhail](https://github.com/fandyushin)) +- Added hints while user make typo in function name or type in command line client. [#4239](https://github.com/ClickHouse/ClickHouse/pull/4239) ([Danila Kutenin](https://github.com/danlark1)) +- Added `Query-Id` to Server’s HTTP Response header. [#4231](https://github.com/ClickHouse/ClickHouse/pull/4231) ([Mikhail](https://github.com/fandyushin)) #### Experimental Features {#experimental-features-2} -- Added `minmax` and `set` data skipping indices for MergeTree table engines family. [\#4143](https://github.com/ClickHouse/ClickHouse/pull/4143) ([Nikita Vasilev](https://github.com/nikvas0)) -- Added conversion of `CROSS JOIN` to `INNER JOIN` if possible. [\#4221](https://github.com/ClickHouse/ClickHouse/pull/4221) [\#4266](https://github.com/ClickHouse/ClickHouse/pull/4266) ([Artem Zuikov](https://github.com/4ertus2)) +- Added `minmax` and `set` data skipping indices for MergeTree table engines family. [#4143](https://github.com/ClickHouse/ClickHouse/pull/4143) ([Nikita Vasilev](https://github.com/nikvas0)) +- Added conversion of `CROSS JOIN` to `INNER JOIN` if possible. [#4221](https://github.com/ClickHouse/ClickHouse/pull/4221) [#4266](https://github.com/ClickHouse/ClickHouse/pull/4266) ([Artem Zuikov](https://github.com/4ertus2)) #### Bug Fixes {#bug-fixes-17} -- Fixed `Not found column` for duplicate columns in `JOIN ON` section. [\#4279](https://github.com/ClickHouse/ClickHouse/pull/4279) ([Artem Zuikov](https://github.com/4ertus2)) -- Make `START REPLICATED SENDS` command start replicated sends. [\#4229](https://github.com/ClickHouse/ClickHouse/pull/4229) ([nvartolomei](https://github.com/nvartolomei)) -- Fixed aggregate functions execution with `Array(LowCardinality)` arguments. [\#4055](https://github.com/ClickHouse/ClickHouse/pull/4055) ([KochetovNicolai](https://github.com/KochetovNicolai)) -- Fixed wrong behaviour when doing `INSERT ... SELECT ... FROM file(...)` query and file has `CSVWithNames` or `TSVWIthNames` format and the first data row is missing. [\#4297](https://github.com/ClickHouse/ClickHouse/pull/4297) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed crash on dictionary reload if dictionary not available. This bug was appeared in 19.1.6. [\#4188](https://github.com/ClickHouse/ClickHouse/pull/4188) ([proller](https://github.com/proller)) -- Fixed `ALL JOIN` with duplicates in right table. [\#4184](https://github.com/ClickHouse/ClickHouse/pull/4184) ([Artem Zuikov](https://github.com/4ertus2)) -- Fixed segmentation fault with `use_uncompressed_cache=1` and exception with wrong uncompressed size. This bug was appeared in 19.1.6. [\#4186](https://github.com/ClickHouse/ClickHouse/pull/4186) ([alesapin](https://github.com/alesapin)) -- Fixed `compile_expressions` bug with comparison of big (more than int16) dates. [\#4341](https://github.com/ClickHouse/ClickHouse/pull/4341) ([alesapin](https://github.com/alesapin)) -- Fixed infinite loop when selecting from table function `numbers(0)`. [\#4280](https://github.com/ClickHouse/ClickHouse/pull/4280) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Temporarily disable predicate optimization for `ORDER BY`. [\#3890](https://github.com/ClickHouse/ClickHouse/pull/3890) ([Winter Zhang](https://github.com/zhang2014)) -- Fixed `Illegal instruction` error when using base64 functions on old CPUs. This error has been reproduced only when ClickHouse was compiled with gcc-8. [\#4275](https://github.com/ClickHouse/ClickHouse/pull/4275) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed `No message received` error when interacting with PostgreSQL ODBC Driver through TLS connection. Also fixes segfault when using MySQL ODBC Driver. [\#4170](https://github.com/ClickHouse/ClickHouse/pull/4170) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed incorrect result when `Date` and `DateTime` arguments are used in branches of conditional operator (function `if`). Added generic case for function `if`. [\#4243](https://github.com/ClickHouse/ClickHouse/pull/4243) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- ClickHouse dictionaries now load within `clickhouse` process. [\#4166](https://github.com/ClickHouse/ClickHouse/pull/4166) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed deadlock when `SELECT` from a table with `File` engine was retried after `No such file or directory` error. [\#4161](https://github.com/ClickHouse/ClickHouse/pull/4161) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed race condition when selecting from `system.tables` may give `table doesn't exist` error. [\#4313](https://github.com/ClickHouse/ClickHouse/pull/4313) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- `clickhouse-client` can segfault on exit while loading data for command line suggestions if it was run in interactive mode. [\#4317](https://github.com/ClickHouse/ClickHouse/pull/4317) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed a bug when the execution of mutations containing `IN` operators was producing incorrect results. [\#4099](https://github.com/ClickHouse/ClickHouse/pull/4099) ([Alex Zatelepin](https://github.com/ztlpn)) -- Fixed error: if there is a database with `Dictionary` engine, all dictionaries forced to load at server startup, and if there is a dictionary with ClickHouse source from localhost, the dictionary cannot load. [\#4255](https://github.com/ClickHouse/ClickHouse/pull/4255) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed error when system logs are tried to create again at server shutdown. [\#4254](https://github.com/ClickHouse/ClickHouse/pull/4254) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Correctly return the right type and properly handle locks in `joinGet` function. [\#4153](https://github.com/ClickHouse/ClickHouse/pull/4153) ([Amos Bird](https://github.com/amosbird)) -- Added `sumMapWithOverflow` function. [\#4151](https://github.com/ClickHouse/ClickHouse/pull/4151) ([Léo Ercolanelli](https://github.com/ercolanelli-leo)) +- Fixed `Not found column` for duplicate columns in `JOIN ON` section. [#4279](https://github.com/ClickHouse/ClickHouse/pull/4279) ([Artem Zuikov](https://github.com/4ertus2)) +- Make `START REPLICATED SENDS` command start replicated sends. [#4229](https://github.com/ClickHouse/ClickHouse/pull/4229) ([nvartolomei](https://github.com/nvartolomei)) +- Fixed aggregate functions execution with `Array(LowCardinality)` arguments. [#4055](https://github.com/ClickHouse/ClickHouse/pull/4055) ([KochetovNicolai](https://github.com/KochetovNicolai)) +- Fixed wrong behaviour when doing `INSERT ... SELECT ... FROM file(...)` query and file has `CSVWithNames` or `TSVWIthNames` format and the first data row is missing. [#4297](https://github.com/ClickHouse/ClickHouse/pull/4297) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed crash on dictionary reload if dictionary not available. This bug was appeared in 19.1.6. [#4188](https://github.com/ClickHouse/ClickHouse/pull/4188) ([proller](https://github.com/proller)) +- Fixed `ALL JOIN` with duplicates in right table. [#4184](https://github.com/ClickHouse/ClickHouse/pull/4184) ([Artem Zuikov](https://github.com/4ertus2)) +- Fixed segmentation fault with `use_uncompressed_cache=1` and exception with wrong uncompressed size. This bug was appeared in 19.1.6. [#4186](https://github.com/ClickHouse/ClickHouse/pull/4186) ([alesapin](https://github.com/alesapin)) +- Fixed `compile_expressions` bug with comparison of big (more than int16) dates. [#4341](https://github.com/ClickHouse/ClickHouse/pull/4341) ([alesapin](https://github.com/alesapin)) +- Fixed infinite loop when selecting from table function `numbers(0)`. [#4280](https://github.com/ClickHouse/ClickHouse/pull/4280) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Temporarily disable predicate optimization for `ORDER BY`. [#3890](https://github.com/ClickHouse/ClickHouse/pull/3890) ([Winter Zhang](https://github.com/zhang2014)) +- Fixed `Illegal instruction` error when using base64 functions on old CPUs. This error has been reproduced only when ClickHouse was compiled with gcc-8. [#4275](https://github.com/ClickHouse/ClickHouse/pull/4275) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed `No message received` error when interacting with PostgreSQL ODBC Driver through TLS connection. Also fixes segfault when using MySQL ODBC Driver. [#4170](https://github.com/ClickHouse/ClickHouse/pull/4170) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed incorrect result when `Date` and `DateTime` arguments are used in branches of conditional operator (function `if`). Added generic case for function `if`. [#4243](https://github.com/ClickHouse/ClickHouse/pull/4243) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- ClickHouse dictionaries now load within `clickhouse` process. [#4166](https://github.com/ClickHouse/ClickHouse/pull/4166) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed deadlock when `SELECT` from a table with `File` engine was retried after `No such file or directory` error. [#4161](https://github.com/ClickHouse/ClickHouse/pull/4161) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed race condition when selecting from `system.tables` may give `table doesn't exist` error. [#4313](https://github.com/ClickHouse/ClickHouse/pull/4313) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- `clickhouse-client` can segfault on exit while loading data for command line suggestions if it was run in interactive mode. [#4317](https://github.com/ClickHouse/ClickHouse/pull/4317) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed a bug when the execution of mutations containing `IN` operators was producing incorrect results. [#4099](https://github.com/ClickHouse/ClickHouse/pull/4099) ([Alex Zatelepin](https://github.com/ztlpn)) +- Fixed error: if there is a database with `Dictionary` engine, all dictionaries forced to load at server startup, and if there is a dictionary with ClickHouse source from localhost, the dictionary cannot load. [#4255](https://github.com/ClickHouse/ClickHouse/pull/4255) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed error when system logs are tried to create again at server shutdown. [#4254](https://github.com/ClickHouse/ClickHouse/pull/4254) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Correctly return the right type and properly handle locks in `joinGet` function. [#4153](https://github.com/ClickHouse/ClickHouse/pull/4153) ([Amos Bird](https://github.com/amosbird)) +- Added `sumMapWithOverflow` function. [#4151](https://github.com/ClickHouse/ClickHouse/pull/4151) ([Léo Ercolanelli](https://github.com/ercolanelli-leo)) - Fixed segfault with `allow_experimental_multiple_joins_emulation`. [52de2c](https://github.com/ClickHouse/ClickHouse/commit/52de2cd927f7b5257dd67e175f0a5560a48840d0) ([Artem Zuikov](https://github.com/4ertus2)) -- Fixed bug with incorrect `Date` and `DateTime` comparison. [\#4237](https://github.com/ClickHouse/ClickHouse/pull/4237) ([valexey](https://github.com/valexey)) -- Fixed fuzz test under undefined behavior sanitizer: added parameter type check for `quantile*Weighted` family of functions. [\#4145](https://github.com/ClickHouse/ClickHouse/pull/4145) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed rare race condition when removing of old data parts can fail with `File not found` error. [\#4378](https://github.com/ClickHouse/ClickHouse/pull/4378) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix install package with missing /etc/clickhouse-server/config.xml. [\#4343](https://github.com/ClickHouse/ClickHouse/pull/4343) ([proller](https://github.com/proller)) +- Fixed bug with incorrect `Date` and `DateTime` comparison. [#4237](https://github.com/ClickHouse/ClickHouse/pull/4237) ([valexey](https://github.com/valexey)) +- Fixed fuzz test under undefined behavior sanitizer: added parameter type check for `quantile*Weighted` family of functions. [#4145](https://github.com/ClickHouse/ClickHouse/pull/4145) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed rare race condition when removing of old data parts can fail with `File not found` error. [#4378](https://github.com/ClickHouse/ClickHouse/pull/4378) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix install package with missing /etc/clickhouse-server/config.xml. [#4343](https://github.com/ClickHouse/ClickHouse/pull/4343) ([proller](https://github.com/proller)) #### Build/Testing/Packaging Improvements {#buildtestingpackaging-improvements-5} -- Debian package: correct /etc/clickhouse-server/preprocessed link according to config. [\#4205](https://github.com/ClickHouse/ClickHouse/pull/4205) ([proller](https://github.com/proller)) -- Various build fixes for FreeBSD. [\#4225](https://github.com/ClickHouse/ClickHouse/pull/4225) ([proller](https://github.com/proller)) -- Added ability to create, fill and drop tables in perftest. [\#4220](https://github.com/ClickHouse/ClickHouse/pull/4220) ([alesapin](https://github.com/alesapin)) -- Added a script to check for duplicate includes. [\#4326](https://github.com/ClickHouse/ClickHouse/pull/4326) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Added ability to run queries by index in performance test. [\#4264](https://github.com/ClickHouse/ClickHouse/pull/4264) ([alesapin](https://github.com/alesapin)) -- Package with debug symbols is suggested to be installed. [\#4274](https://github.com/ClickHouse/ClickHouse/pull/4274) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Refactoring of performance-test. Better logging and signals handling. [\#4171](https://github.com/ClickHouse/ClickHouse/pull/4171) ([alesapin](https://github.com/alesapin)) -- Added docs to anonymized Yandex.Metrika datasets. [\#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) ([Alex Zatelepin](https://github.com/ztlpn)) -- Added docs about two datasets in s3. [\#4144](https://github.com/ClickHouse/ClickHouse/pull/4144) ([alesapin](https://github.com/alesapin)) -- Added script which creates changelog from pull requests description. [\#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)) -- Added puppet module for ClickHouse. [\#4182](https://github.com/ClickHouse/ClickHouse/pull/4182) ([Maxim Fedotov](https://github.com/MaxFedotov)) -- Added docs for a group of undocumented functions. [\#4168](https://github.com/ClickHouse/ClickHouse/pull/4168) ([Winter Zhang](https://github.com/zhang2014)) -- ARM build fixes. [\#4210](https://github.com/ClickHouse/ClickHouse/pull/4210)[\#4306](https://github.com/ClickHouse/ClickHouse/pull/4306) [\#4291](https://github.com/ClickHouse/ClickHouse/pull/4291) ([proller](https://github.com/proller)) ([proller](https://github.com/proller)) -- Dictionary tests now able to run from `ctest`. [\#4189](https://github.com/ClickHouse/ClickHouse/pull/4189) ([proller](https://github.com/proller)) -- Now `/etc/ssl` is used as default directory with SSL certificates. [\#4167](https://github.com/ClickHouse/ClickHouse/pull/4167) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Added checking SSE and AVX instruction at start. [\#4234](https://github.com/ClickHouse/ClickHouse/pull/4234) ([Igr](https://github.com/igron99)) -- Init script will wait server until start. [\#4281](https://github.com/ClickHouse/ClickHouse/pull/4281) ([proller](https://github.com/proller)) +- Debian package: correct /etc/clickhouse-server/preprocessed link according to config. [#4205](https://github.com/ClickHouse/ClickHouse/pull/4205) ([proller](https://github.com/proller)) +- Various build fixes for FreeBSD. [#4225](https://github.com/ClickHouse/ClickHouse/pull/4225) ([proller](https://github.com/proller)) +- Added ability to create, fill and drop tables in perftest. [#4220](https://github.com/ClickHouse/ClickHouse/pull/4220) ([alesapin](https://github.com/alesapin)) +- Added a script to check for duplicate includes. [#4326](https://github.com/ClickHouse/ClickHouse/pull/4326) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Added ability to run queries by index in performance test. [#4264](https://github.com/ClickHouse/ClickHouse/pull/4264) ([alesapin](https://github.com/alesapin)) +- Package with debug symbols is suggested to be installed. [#4274](https://github.com/ClickHouse/ClickHouse/pull/4274) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Refactoring of performance-test. Better logging and signals handling. [#4171](https://github.com/ClickHouse/ClickHouse/pull/4171) ([alesapin](https://github.com/alesapin)) +- Added docs to anonymized Yandex.Metrika datasets. [#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) ([Alex Zatelepin](https://github.com/ztlpn)) +- Added docs about two datasets in s3. [#4144](https://github.com/ClickHouse/ClickHouse/pull/4144) ([alesapin](https://github.com/alesapin)) +- Added script which creates changelog from pull requests description. [#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)) +- Added puppet module for ClickHouse. [#4182](https://github.com/ClickHouse/ClickHouse/pull/4182) ([Maxim Fedotov](https://github.com/MaxFedotov)) +- Added docs for a group of undocumented functions. [#4168](https://github.com/ClickHouse/ClickHouse/pull/4168) ([Winter Zhang](https://github.com/zhang2014)) +- ARM build fixes. [#4210](https://github.com/ClickHouse/ClickHouse/pull/4210)[#4306](https://github.com/ClickHouse/ClickHouse/pull/4306) [#4291](https://github.com/ClickHouse/ClickHouse/pull/4291) ([proller](https://github.com/proller)) ([proller](https://github.com/proller)) +- Dictionary tests now able to run from `ctest`. [#4189](https://github.com/ClickHouse/ClickHouse/pull/4189) ([proller](https://github.com/proller)) +- Now `/etc/ssl` is used as default directory with SSL certificates. [#4167](https://github.com/ClickHouse/ClickHouse/pull/4167) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Added checking SSE and AVX instruction at start. [#4234](https://github.com/ClickHouse/ClickHouse/pull/4234) ([Igr](https://github.com/igron99)) +- Init script will wait server until start. [#4281](https://github.com/ClickHouse/ClickHouse/pull/4281) ([proller](https://github.com/proller)) #### Backward Incompatible Changes {#backward-incompatible-changes-1} -- Removed `allow_experimental_low_cardinality_type` setting. `LowCardinality` data types are production ready. [\#4323](https://github.com/ClickHouse/ClickHouse/pull/4323) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Reduce mark cache size and uncompressed cache size accordingly to available memory amount. [\#4240](https://github.com/ClickHouse/ClickHouse/pull/4240) ([Lopatin Konstantin](https://github.com/k-lopatin) -- Added keyword `INDEX` in `CREATE TABLE` query. A column with name `index` must be quoted with backticks or double quotes: `` `index` ``. [\#4143](https://github.com/ClickHouse/ClickHouse/pull/4143) ([Nikita Vasilev](https://github.com/nikvas0)) -- `sumMap` now promote result type instead of overflow. The old `sumMap` behavior can be obtained by using `sumMapWithOverflow` function. [\#4151](https://github.com/ClickHouse/ClickHouse/pull/4151) ([Léo Ercolanelli](https://github.com/ercolanelli-leo)) +- Removed `allow_experimental_low_cardinality_type` setting. `LowCardinality` data types are production ready. [#4323](https://github.com/ClickHouse/ClickHouse/pull/4323) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Reduce mark cache size and uncompressed cache size accordingly to available memory amount. [#4240](https://github.com/ClickHouse/ClickHouse/pull/4240) ([Lopatin Konstantin](https://github.com/k-lopatin) +- Added keyword `INDEX` in `CREATE TABLE` query. A column with name `index` must be quoted with backticks or double quotes: `` `index` ``. [#4143](https://github.com/ClickHouse/ClickHouse/pull/4143) ([Nikita Vasilev](https://github.com/nikvas0)) +- `sumMap` now promote result type instead of overflow. The old `sumMap` behavior can be obtained by using `sumMapWithOverflow` function. [#4151](https://github.com/ClickHouse/ClickHouse/pull/4151) ([Léo Ercolanelli](https://github.com/ercolanelli-leo)) #### Performance Improvements {#performance-improvements-4} -- `std::sort` replaced by `pdqsort` for queries without `LIMIT`. [\#4236](https://github.com/ClickHouse/ClickHouse/pull/4236) ([Evgenii Pravda](https://github.com/kvinty)) -- Now server reuse threads from global thread pool. This affects performance in some corner cases. [\#4150](https://github.com/ClickHouse/ClickHouse/pull/4150) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- `std::sort` replaced by `pdqsort` for queries without `LIMIT`. [#4236](https://github.com/ClickHouse/ClickHouse/pull/4236) ([Evgenii Pravda](https://github.com/kvinty)) +- Now server reuse threads from global thread pool. This affects performance in some corner cases. [#4150](https://github.com/ClickHouse/ClickHouse/pull/4150) ([alexey-milovidov](https://github.com/alexey-milovidov)) #### Improvements {#improvements-5} -- Implemented AIO support for FreeBSD. [\#4305](https://github.com/ClickHouse/ClickHouse/pull/4305) ([urgordeadbeef](https://github.com/urgordeadbeef)) -- `SELECT * FROM a JOIN b USING a, b` now return `a` and `b` columns only from the left table. [\#4141](https://github.com/ClickHouse/ClickHouse/pull/4141) ([Artem Zuikov](https://github.com/4ertus2)) -- Allow `-C` option of client to work as `-c` option. [\#4232](https://github.com/ClickHouse/ClickHouse/pull/4232) ([syominsergey](https://github.com/syominsergey)) -- Now option `--password` used without value requires password from stdin. [\#4230](https://github.com/ClickHouse/ClickHouse/pull/4230) ([BSD\_Conqueror](https://github.com/bsd-conqueror)) -- Added highlighting of unescaped metacharacters in string literals that contain `LIKE` expressions or regexps. [\#4327](https://github.com/ClickHouse/ClickHouse/pull/4327) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Added cancelling of HTTP read only queries if client socket goes away. [\#4213](https://github.com/ClickHouse/ClickHouse/pull/4213) ([nvartolomei](https://github.com/nvartolomei)) -- Now server reports progress to keep client connections alive. [\#4215](https://github.com/ClickHouse/ClickHouse/pull/4215) ([Ivan](https://github.com/abyss7)) -- Slightly better message with reason for OPTIMIZE query with `optimize_throw_if_noop` setting enabled. [\#4294](https://github.com/ClickHouse/ClickHouse/pull/4294) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Added support of `--version` option for clickhouse server. [\#4251](https://github.com/ClickHouse/ClickHouse/pull/4251) ([Lopatin Konstantin](https://github.com/k-lopatin)) -- Added `--help/-h` option to `clickhouse-server`. [\#4233](https://github.com/ClickHouse/ClickHouse/pull/4233) ([Yuriy Baranov](https://github.com/yurriy)) -- Added support for scalar subqueries with aggregate function state result. [\#4348](https://github.com/ClickHouse/ClickHouse/pull/4348) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- Improved server shutdown time and ALTERs waiting time. [\#4372](https://github.com/ClickHouse/ClickHouse/pull/4372) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Added info about the replicated\_can\_become\_leader setting to system.replicas and add logging if the replica won’t try to become leader. [\#4379](https://github.com/ClickHouse/ClickHouse/pull/4379) ([Alex Zatelepin](https://github.com/ztlpn)) +- Implemented AIO support for FreeBSD. [#4305](https://github.com/ClickHouse/ClickHouse/pull/4305) ([urgordeadbeef](https://github.com/urgordeadbeef)) +- `SELECT * FROM a JOIN b USING a, b` now return `a` and `b` columns only from the left table. [#4141](https://github.com/ClickHouse/ClickHouse/pull/4141) ([Artem Zuikov](https://github.com/4ertus2)) +- Allow `-C` option of client to work as `-c` option. [#4232](https://github.com/ClickHouse/ClickHouse/pull/4232) ([syominsergey](https://github.com/syominsergey)) +- Now option `--password` used without value requires password from stdin. [#4230](https://github.com/ClickHouse/ClickHouse/pull/4230) ([BSD_Conqueror](https://github.com/bsd-conqueror)) +- Added highlighting of unescaped metacharacters in string literals that contain `LIKE` expressions or regexps. [#4327](https://github.com/ClickHouse/ClickHouse/pull/4327) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Added cancelling of HTTP read only queries if client socket goes away. [#4213](https://github.com/ClickHouse/ClickHouse/pull/4213) ([nvartolomei](https://github.com/nvartolomei)) +- Now server reports progress to keep client connections alive. [#4215](https://github.com/ClickHouse/ClickHouse/pull/4215) ([Ivan](https://github.com/abyss7)) +- Slightly better message with reason for OPTIMIZE query with `optimize_throw_if_noop` setting enabled. [#4294](https://github.com/ClickHouse/ClickHouse/pull/4294) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Added support of `--version` option for clickhouse server. [#4251](https://github.com/ClickHouse/ClickHouse/pull/4251) ([Lopatin Konstantin](https://github.com/k-lopatin)) +- Added `--help/-h` option to `clickhouse-server`. [#4233](https://github.com/ClickHouse/ClickHouse/pull/4233) ([Yuriy Baranov](https://github.com/yurriy)) +- Added support for scalar subqueries with aggregate function state result. [#4348](https://github.com/ClickHouse/ClickHouse/pull/4348) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Improved server shutdown time and ALTERs waiting time. [#4372](https://github.com/ClickHouse/ClickHouse/pull/4372) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Added info about the replicated_can_become_leader setting to system.replicas and add logging if the replica won’t try to become leader. [#4379](https://github.com/ClickHouse/ClickHouse/pull/4379) ([Alex Zatelepin](https://github.com/ztlpn)) ## ClickHouse Release 19.1 {#clickhouse-release-19-1} @@ -1921,14 +1921,14 @@ This release contains exactly the same set of patches as 19.3.6. #### Bug Fixes {#bug-fixes-18} -- Fixed backward incompatibility with old versions due to wrong implementation of `send_logs_level` setting. [\#4445](https://github.com/ClickHouse/ClickHouse/pull/4445) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed backward incompatibility of table function `remote` introduced with column comments. [\#4446](https://github.com/ClickHouse/ClickHouse/pull/4446) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed backward incompatibility with old versions due to wrong implementation of `send_logs_level` setting. [#4445](https://github.com/ClickHouse/ClickHouse/pull/4445) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed backward incompatibility of table function `remote` introduced with column comments. [#4446](https://github.com/ClickHouse/ClickHouse/pull/4446) ([alexey-milovidov](https://github.com/alexey-milovidov)) ### ClickHouse Release 19.1.8, 2019-02-16 {#clickhouse-release-19-1-8-2019-02-16} #### Bug Fixes {#bug-fixes-19} -- Fix install package with missing /etc/clickhouse-server/config.xml. [\#4343](https://github.com/ClickHouse/ClickHouse/pull/4343) ([proller](https://github.com/proller)) +- Fix install package with missing /etc/clickhouse-server/config.xml. [#4343](https://github.com/ClickHouse/ClickHouse/pull/4343) ([proller](https://github.com/proller)) ## ClickHouse Release 19.1 {#clickhouse-release-19-1-2} @@ -1936,137 +1936,137 @@ This release contains exactly the same set of patches as 19.3.6. #### Bug Fixes {#bug-fixes-20} -- Correctly return the right type and properly handle locks in `joinGet` function. [\#4153](https://github.com/ClickHouse/ClickHouse/pull/4153) ([Amos Bird](https://github.com/amosbird)) -- Fixed error when system logs are tried to create again at server shutdown. [\#4254](https://github.com/ClickHouse/ClickHouse/pull/4254) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed error: if there is a database with `Dictionary` engine, all dictionaries forced to load at server startup, and if there is a dictionary with ClickHouse source from localhost, the dictionary cannot load. [\#4255](https://github.com/ClickHouse/ClickHouse/pull/4255) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed a bug when the execution of mutations containing `IN` operators was producing incorrect results. [\#4099](https://github.com/ClickHouse/ClickHouse/pull/4099) ([Alex Zatelepin](https://github.com/ztlpn)) -- `clickhouse-client` can segfault on exit while loading data for command line suggestions if it was run in interactive mode. [\#4317](https://github.com/ClickHouse/ClickHouse/pull/4317) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed race condition when selecting from `system.tables` may give `table doesn't exist` error. [\#4313](https://github.com/ClickHouse/ClickHouse/pull/4313) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed deadlock when `SELECT` from a table with `File` engine was retried after `No such file or directory` error. [\#4161](https://github.com/ClickHouse/ClickHouse/pull/4161) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed an issue: local ClickHouse dictionaries are loaded via TCP, but should load within process. [\#4166](https://github.com/ClickHouse/ClickHouse/pull/4166) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed `No message received` error when interacting with PostgreSQL ODBC Driver through TLS connection. Also fixes segfault when using MySQL ODBC Driver. [\#4170](https://github.com/ClickHouse/ClickHouse/pull/4170) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Temporarily disable predicate optimization for `ORDER BY`. [\#3890](https://github.com/ClickHouse/ClickHouse/pull/3890) ([Winter Zhang](https://github.com/zhang2014)) -- Fixed infinite loop when selecting from table function `numbers(0)`. [\#4280](https://github.com/ClickHouse/ClickHouse/pull/4280) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed `compile_expressions` bug with comparison of big (more than int16) dates. [\#4341](https://github.com/ClickHouse/ClickHouse/pull/4341) ([alesapin](https://github.com/alesapin)) -- Fixed segmentation fault with `uncompressed_cache=1` and exception with wrong uncompressed size. [\#4186](https://github.com/ClickHouse/ClickHouse/pull/4186) ([alesapin](https://github.com/alesapin)) -- Fixed `ALL JOIN` with duplicates in right table. [\#4184](https://github.com/ClickHouse/ClickHouse/pull/4184) ([Artem Zuikov](https://github.com/4ertus2)) -- Fixed wrong behaviour when doing `INSERT ... SELECT ... FROM file(...)` query and file has `CSVWithNames` or `TSVWIthNames` format and the first data row is missing. [\#4297](https://github.com/ClickHouse/ClickHouse/pull/4297) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed aggregate functions execution with `Array(LowCardinality)` arguments. [\#4055](https://github.com/ClickHouse/ClickHouse/pull/4055) ([KochetovNicolai](https://github.com/KochetovNicolai)) -- Debian package: correct /etc/clickhouse-server/preprocessed link according to config. [\#4205](https://github.com/ClickHouse/ClickHouse/pull/4205) ([proller](https://github.com/proller)) -- Fixed fuzz test under undefined behavior sanitizer: added parameter type check for `quantile*Weighted` family of functions. [\#4145](https://github.com/ClickHouse/ClickHouse/pull/4145) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Make `START REPLICATED SENDS` command start replicated sends. [\#4229](https://github.com/ClickHouse/ClickHouse/pull/4229) ([nvartolomei](https://github.com/nvartolomei)) -- Fixed `Not found column` for duplicate columns in JOIN ON section. [\#4279](https://github.com/ClickHouse/ClickHouse/pull/4279) ([Artem Zuikov](https://github.com/4ertus2)) -- Now `/etc/ssl` is used as default directory with SSL certificates. [\#4167](https://github.com/ClickHouse/ClickHouse/pull/4167) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed crash on dictionary reload if dictionary not available. [\#4188](https://github.com/ClickHouse/ClickHouse/pull/4188) ([proller](https://github.com/proller)) -- Fixed bug with incorrect `Date` and `DateTime` comparison. [\#4237](https://github.com/ClickHouse/ClickHouse/pull/4237) ([valexey](https://github.com/valexey)) -- Fixed incorrect result when `Date` and `DateTime` arguments are used in branches of conditional operator (function `if`). Added generic case for function `if`. [\#4243](https://github.com/ClickHouse/ClickHouse/pull/4243) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Correctly return the right type and properly handle locks in `joinGet` function. [#4153](https://github.com/ClickHouse/ClickHouse/pull/4153) ([Amos Bird](https://github.com/amosbird)) +- Fixed error when system logs are tried to create again at server shutdown. [#4254](https://github.com/ClickHouse/ClickHouse/pull/4254) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed error: if there is a database with `Dictionary` engine, all dictionaries forced to load at server startup, and if there is a dictionary with ClickHouse source from localhost, the dictionary cannot load. [#4255](https://github.com/ClickHouse/ClickHouse/pull/4255) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed a bug when the execution of mutations containing `IN` operators was producing incorrect results. [#4099](https://github.com/ClickHouse/ClickHouse/pull/4099) ([Alex Zatelepin](https://github.com/ztlpn)) +- `clickhouse-client` can segfault on exit while loading data for command line suggestions if it was run in interactive mode. [#4317](https://github.com/ClickHouse/ClickHouse/pull/4317) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed race condition when selecting from `system.tables` may give `table doesn't exist` error. [#4313](https://github.com/ClickHouse/ClickHouse/pull/4313) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed deadlock when `SELECT` from a table with `File` engine was retried after `No such file or directory` error. [#4161](https://github.com/ClickHouse/ClickHouse/pull/4161) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed an issue: local ClickHouse dictionaries are loaded via TCP, but should load within process. [#4166](https://github.com/ClickHouse/ClickHouse/pull/4166) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed `No message received` error when interacting with PostgreSQL ODBC Driver through TLS connection. Also fixes segfault when using MySQL ODBC Driver. [#4170](https://github.com/ClickHouse/ClickHouse/pull/4170) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Temporarily disable predicate optimization for `ORDER BY`. [#3890](https://github.com/ClickHouse/ClickHouse/pull/3890) ([Winter Zhang](https://github.com/zhang2014)) +- Fixed infinite loop when selecting from table function `numbers(0)`. [#4280](https://github.com/ClickHouse/ClickHouse/pull/4280) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed `compile_expressions` bug with comparison of big (more than int16) dates. [#4341](https://github.com/ClickHouse/ClickHouse/pull/4341) ([alesapin](https://github.com/alesapin)) +- Fixed segmentation fault with `uncompressed_cache=1` and exception with wrong uncompressed size. [#4186](https://github.com/ClickHouse/ClickHouse/pull/4186) ([alesapin](https://github.com/alesapin)) +- Fixed `ALL JOIN` with duplicates in right table. [#4184](https://github.com/ClickHouse/ClickHouse/pull/4184) ([Artem Zuikov](https://github.com/4ertus2)) +- Fixed wrong behaviour when doing `INSERT ... SELECT ... FROM file(...)` query and file has `CSVWithNames` or `TSVWIthNames` format and the first data row is missing. [#4297](https://github.com/ClickHouse/ClickHouse/pull/4297) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed aggregate functions execution with `Array(LowCardinality)` arguments. [#4055](https://github.com/ClickHouse/ClickHouse/pull/4055) ([KochetovNicolai](https://github.com/KochetovNicolai)) +- Debian package: correct /etc/clickhouse-server/preprocessed link according to config. [#4205](https://github.com/ClickHouse/ClickHouse/pull/4205) ([proller](https://github.com/proller)) +- Fixed fuzz test under undefined behavior sanitizer: added parameter type check for `quantile*Weighted` family of functions. [#4145](https://github.com/ClickHouse/ClickHouse/pull/4145) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Make `START REPLICATED SENDS` command start replicated sends. [#4229](https://github.com/ClickHouse/ClickHouse/pull/4229) ([nvartolomei](https://github.com/nvartolomei)) +- Fixed `Not found column` for duplicate columns in JOIN ON section. [#4279](https://github.com/ClickHouse/ClickHouse/pull/4279) ([Artem Zuikov](https://github.com/4ertus2)) +- Now `/etc/ssl` is used as default directory with SSL certificates. [#4167](https://github.com/ClickHouse/ClickHouse/pull/4167) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed crash on dictionary reload if dictionary not available. [#4188](https://github.com/ClickHouse/ClickHouse/pull/4188) ([proller](https://github.com/proller)) +- Fixed bug with incorrect `Date` and `DateTime` comparison. [#4237](https://github.com/ClickHouse/ClickHouse/pull/4237) ([valexey](https://github.com/valexey)) +- Fixed incorrect result when `Date` and `DateTime` arguments are used in branches of conditional operator (function `if`). Added generic case for function `if`. [#4243](https://github.com/ClickHouse/ClickHouse/pull/4243) ([alexey-milovidov](https://github.com/alexey-milovidov)) ### ClickHouse Release 19.1.6, 2019-01-24 {#clickhouse-release-19-1-6-2019-01-24} #### New Features {#new-features-7} -- Custom per column compression codecs for tables. [\#3899](https://github.com/ClickHouse/ClickHouse/pull/3899) [\#4111](https://github.com/ClickHouse/ClickHouse/pull/4111) ([alesapin](https://github.com/alesapin), [Winter Zhang](https://github.com/zhang2014), [Anatoly](https://github.com/Sindbag)) -- Added compression codec `Delta`. [\#4052](https://github.com/ClickHouse/ClickHouse/pull/4052) ([alesapin](https://github.com/alesapin)) -- Allow to `ALTER` compression codecs. [\#4054](https://github.com/ClickHouse/ClickHouse/pull/4054) ([alesapin](https://github.com/alesapin)) -- Added functions `left`, `right`, `trim`, `ltrim`, `rtrim`, `timestampadd`, `timestampsub` for SQL standard compatibility. [\#3826](https://github.com/ClickHouse/ClickHouse/pull/3826) ([Ivan Blinkov](https://github.com/blinkov)) -- Support for write in `HDFS` tables and `hdfs` table function. [\#4084](https://github.com/ClickHouse/ClickHouse/pull/4084) ([alesapin](https://github.com/alesapin)) -- Added functions to search for multiple constant strings from big haystack: `multiPosition`, `multiSearch` ,`firstMatch` also with `-UTF8`, `-CaseInsensitive`, and `-CaseInsensitiveUTF8` variants. [\#4053](https://github.com/ClickHouse/ClickHouse/pull/4053) ([Danila Kutenin](https://github.com/danlark1)) -- Pruning of unused shards if `SELECT` query filters by sharding key (setting `optimize_skip_unused_shards`). [\#3851](https://github.com/ClickHouse/ClickHouse/pull/3851) ([Gleb Kanterov](https://github.com/kanterov), [Ivan](https://github.com/abyss7)) -- Allow `Kafka` engine to ignore some number of parsing errors per block. [\#4094](https://github.com/ClickHouse/ClickHouse/pull/4094) ([Ivan](https://github.com/abyss7)) -- Added support for `CatBoost` multiclass models evaluation. Function `modelEvaluate` returns tuple with per-class raw predictions for multiclass models. `libcatboostmodel.so` should be built with [\#607](https://github.com/catboost/catboost/pull/607). [\#3959](https://github.com/ClickHouse/ClickHouse/pull/3959) ([KochetovNicolai](https://github.com/KochetovNicolai)) -- Added functions `filesystemAvailable`, `filesystemFree`, `filesystemCapacity`. [\#4097](https://github.com/ClickHouse/ClickHouse/pull/4097) ([Boris Granveaud](https://github.com/bgranvea)) -- Added hashing functions `xxHash64` and `xxHash32`. [\#3905](https://github.com/ClickHouse/ClickHouse/pull/3905) ([filimonov](https://github.com/filimonov)) -- Added `gccMurmurHash` hashing function (GCC flavoured Murmur hash) which uses the same hash seed as [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)) -- Added hashing functions `javaHash`, `hiveHash`. [\#3811](https://github.com/ClickHouse/ClickHouse/pull/3811) ([shangshujie365](https://github.com/shangshujie365)) -- Added table function `remoteSecure`. Function works as `remote`, but uses secure connection. [\#4088](https://github.com/ClickHouse/ClickHouse/pull/4088) ([proller](https://github.com/proller)) +- Custom per column compression codecs for tables. [#3899](https://github.com/ClickHouse/ClickHouse/pull/3899) [#4111](https://github.com/ClickHouse/ClickHouse/pull/4111) ([alesapin](https://github.com/alesapin), [Winter Zhang](https://github.com/zhang2014), [Anatoly](https://github.com/Sindbag)) +- Added compression codec `Delta`. [#4052](https://github.com/ClickHouse/ClickHouse/pull/4052) ([alesapin](https://github.com/alesapin)) +- Allow to `ALTER` compression codecs. [#4054](https://github.com/ClickHouse/ClickHouse/pull/4054) ([alesapin](https://github.com/alesapin)) +- Added functions `left`, `right`, `trim`, `ltrim`, `rtrim`, `timestampadd`, `timestampsub` for SQL standard compatibility. [#3826](https://github.com/ClickHouse/ClickHouse/pull/3826) ([Ivan Blinkov](https://github.com/blinkov)) +- Support for write in `HDFS` tables and `hdfs` table function. [#4084](https://github.com/ClickHouse/ClickHouse/pull/4084) ([alesapin](https://github.com/alesapin)) +- Added functions to search for multiple constant strings from big haystack: `multiPosition`, `multiSearch` ,`firstMatch` also with `-UTF8`, `-CaseInsensitive`, and `-CaseInsensitiveUTF8` variants. [#4053](https://github.com/ClickHouse/ClickHouse/pull/4053) ([Danila Kutenin](https://github.com/danlark1)) +- Pruning of unused shards if `SELECT` query filters by sharding key (setting `optimize_skip_unused_shards`). [#3851](https://github.com/ClickHouse/ClickHouse/pull/3851) ([Gleb Kanterov](https://github.com/kanterov), [Ivan](https://github.com/abyss7)) +- Allow `Kafka` engine to ignore some number of parsing errors per block. [#4094](https://github.com/ClickHouse/ClickHouse/pull/4094) ([Ivan](https://github.com/abyss7)) +- Added support for `CatBoost` multiclass models evaluation. Function `modelEvaluate` returns tuple with per-class raw predictions for multiclass models. `libcatboostmodel.so` should be built with [#607](https://github.com/catboost/catboost/pull/607). [#3959](https://github.com/ClickHouse/ClickHouse/pull/3959) ([KochetovNicolai](https://github.com/KochetovNicolai)) +- Added functions `filesystemAvailable`, `filesystemFree`, `filesystemCapacity`. [#4097](https://github.com/ClickHouse/ClickHouse/pull/4097) ([Boris Granveaud](https://github.com/bgranvea)) +- Added hashing functions `xxHash64` and `xxHash32`. [#3905](https://github.com/ClickHouse/ClickHouse/pull/3905) ([filimonov](https://github.com/filimonov)) +- Added `gccMurmurHash` hashing function (GCC flavoured Murmur hash) which uses the same hash seed as [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)) +- Added hashing functions `javaHash`, `hiveHash`. [#3811](https://github.com/ClickHouse/ClickHouse/pull/3811) ([shangshujie365](https://github.com/shangshujie365)) +- Added table function `remoteSecure`. Function works as `remote`, but uses secure connection. [#4088](https://github.com/ClickHouse/ClickHouse/pull/4088) ([proller](https://github.com/proller)) #### Experimental Features {#experimental-features-3} -- Added multiple JOINs emulation (`allow_experimental_multiple_joins_emulation` setting). [\#3946](https://github.com/ClickHouse/ClickHouse/pull/3946) ([Artem Zuikov](https://github.com/4ertus2)) +- Added multiple JOINs emulation (`allow_experimental_multiple_joins_emulation` setting). [#3946](https://github.com/ClickHouse/ClickHouse/pull/3946) ([Artem Zuikov](https://github.com/4ertus2)) #### Bug Fixes {#bug-fixes-21} -- Make `compiled_expression_cache_size` setting limited by default to lower memory consumption. [\#4041](https://github.com/ClickHouse/ClickHouse/pull/4041) ([alesapin](https://github.com/alesapin)) -- Fix a bug that led to hangups in threads that perform ALTERs of Replicated tables and in the thread that updates configuration from 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) ([Alex Zatelepin](https://github.com/ztlpn)) -- Fixed a race condition when executing a distributed ALTER task. The race condition led to more than one replica trying to execute the task and all replicas except one failing with a ZooKeeper error. [\#3904](https://github.com/ClickHouse/ClickHouse/pull/3904) ([Alex Zatelepin](https://github.com/ztlpn)) -- Fix a bug when `from_zk` config elements weren’t refreshed after a request to ZooKeeper timed out. [\#2947](https://github.com/ClickHouse/ClickHouse/issues/2947) [\#3947](https://github.com/ClickHouse/ClickHouse/pull/3947) ([Alex Zatelepin](https://github.com/ztlpn)) -- Fix bug with wrong prefix for IPv4 subnet masks. [\#3945](https://github.com/ClickHouse/ClickHouse/pull/3945) ([alesapin](https://github.com/alesapin)) -- Fixed crash (`std::terminate`) in rare cases when a new thread cannot be created due to exhausted resources. [\#3956](https://github.com/ClickHouse/ClickHouse/pull/3956) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix bug when in `remote` table function execution when wrong restrictions were used for in `getStructureOfRemoteTable`. [\#4009](https://github.com/ClickHouse/ClickHouse/pull/4009) ([alesapin](https://github.com/alesapin)) -- Fix a leak of netlink sockets. They were placed in a pool where they were never deleted and new sockets were created at the start of a new thread when all current sockets were in use. [\#4017](https://github.com/ClickHouse/ClickHouse/pull/4017) ([Alex Zatelepin](https://github.com/ztlpn)) -- Fix bug with closing `/proc/self/fd` directory earlier than all fds were read from `/proc` after forking `odbc-bridge` subprocess. [\#4120](https://github.com/ClickHouse/ClickHouse/pull/4120) ([alesapin](https://github.com/alesapin)) -- Fixed String to UInt monotonic conversion in case of usage String in primary key. [\#3870](https://github.com/ClickHouse/ClickHouse/pull/3870) ([Winter Zhang](https://github.com/zhang2014)) -- Fixed error in calculation of integer conversion function monotonicity. [\#3921](https://github.com/ClickHouse/ClickHouse/pull/3921) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed segfault in `arrayEnumerateUniq`, `arrayEnumerateDense` functions in case of some invalid arguments. [\#3909](https://github.com/ClickHouse/ClickHouse/pull/3909) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fix UB in StorageMerge. [\#3910](https://github.com/ClickHouse/ClickHouse/pull/3910) ([Amos Bird](https://github.com/amosbird)) -- Fixed segfault in functions `addDays`, `subtractDays`. [\#3913](https://github.com/ClickHouse/ClickHouse/pull/3913) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed error: functions `round`, `floor`, `trunc`, `ceil` may return bogus result when executed on integer argument and large negative scale. [\#3914](https://github.com/ClickHouse/ClickHouse/pull/3914) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed a bug induced by ‘kill query sync’ which leads to a core dump. [\#3916](https://github.com/ClickHouse/ClickHouse/pull/3916) ([muVulDeePecker](https://github.com/fancyqlx)) -- Fix bug with long delay after empty replication queue. [\#3928](https://github.com/ClickHouse/ClickHouse/pull/3928) [\#3932](https://github.com/ClickHouse/ClickHouse/pull/3932) ([alesapin](https://github.com/alesapin)) -- Fixed excessive memory usage in case of inserting into table with `LowCardinality` primary key. [\#3955](https://github.com/ClickHouse/ClickHouse/pull/3955) ([KochetovNicolai](https://github.com/KochetovNicolai)) -- Fixed `LowCardinality` serialization for `Native` format in case of empty arrays. [\#3907](https://github.com/ClickHouse/ClickHouse/issues/3907) [\#4011](https://github.com/ClickHouse/ClickHouse/pull/4011) ([KochetovNicolai](https://github.com/KochetovNicolai)) -- Fixed incorrect result while using distinct by single LowCardinality numeric column. [\#3895](https://github.com/ClickHouse/ClickHouse/issues/3895) [\#4012](https://github.com/ClickHouse/ClickHouse/pull/4012) ([KochetovNicolai](https://github.com/KochetovNicolai)) -- Fixed specialized aggregation with LowCardinality key (in case when `compile` setting is enabled). [\#3886](https://github.com/ClickHouse/ClickHouse/pull/3886) ([KochetovNicolai](https://github.com/KochetovNicolai)) -- Fix user and password forwarding for replicated tables queries. [\#3957](https://github.com/ClickHouse/ClickHouse/pull/3957) ([alesapin](https://github.com/alesapin)) ([小路](https://github.com/nicelulu)) -- Fixed very rare race condition that can happen when listing tables in Dictionary database while reloading dictionaries. [\#3970](https://github.com/ClickHouse/ClickHouse/pull/3970) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed incorrect result when HAVING was used with ROLLUP or CUBE. [\#3756](https://github.com/ClickHouse/ClickHouse/issues/3756) [\#3837](https://github.com/ClickHouse/ClickHouse/pull/3837) ([Sam Chou](https://github.com/reflection)) -- Fixed column aliases for query with `JOIN ON` syntax and distributed tables. [\#3980](https://github.com/ClickHouse/ClickHouse/pull/3980) ([Winter Zhang](https://github.com/zhang2014)) -- Fixed error in internal implementation of `quantileTDigest` (found by Artem Vakhrushev). This error never happens in ClickHouse and was relevant only for those who use ClickHouse codebase as a library directly. [\#3935](https://github.com/ClickHouse/ClickHouse/pull/3935) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Make `compiled_expression_cache_size` setting limited by default to lower memory consumption. [#4041](https://github.com/ClickHouse/ClickHouse/pull/4041) ([alesapin](https://github.com/alesapin)) +- Fix a bug that led to hangups in threads that perform ALTERs of Replicated tables and in the thread that updates configuration from 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) ([Alex Zatelepin](https://github.com/ztlpn)) +- Fixed a race condition when executing a distributed ALTER task. The race condition led to more than one replica trying to execute the task and all replicas except one failing with a ZooKeeper error. [#3904](https://github.com/ClickHouse/ClickHouse/pull/3904) ([Alex Zatelepin](https://github.com/ztlpn)) +- Fix a bug when `from_zk` config elements weren’t refreshed after a request to ZooKeeper timed out. [#2947](https://github.com/ClickHouse/ClickHouse/issues/2947) [#3947](https://github.com/ClickHouse/ClickHouse/pull/3947) ([Alex Zatelepin](https://github.com/ztlpn)) +- Fix bug with wrong prefix for IPv4 subnet masks. [#3945](https://github.com/ClickHouse/ClickHouse/pull/3945) ([alesapin](https://github.com/alesapin)) +- Fixed crash (`std::terminate`) in rare cases when a new thread cannot be created due to exhausted resources. [#3956](https://github.com/ClickHouse/ClickHouse/pull/3956) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix bug when in `remote` table function execution when wrong restrictions were used for in `getStructureOfRemoteTable`. [#4009](https://github.com/ClickHouse/ClickHouse/pull/4009) ([alesapin](https://github.com/alesapin)) +- Fix a leak of netlink sockets. They were placed in a pool where they were never deleted and new sockets were created at the start of a new thread when all current sockets were in use. [#4017](https://github.com/ClickHouse/ClickHouse/pull/4017) ([Alex Zatelepin](https://github.com/ztlpn)) +- Fix bug with closing `/proc/self/fd` directory earlier than all fds were read from `/proc` after forking `odbc-bridge` subprocess. [#4120](https://github.com/ClickHouse/ClickHouse/pull/4120) ([alesapin](https://github.com/alesapin)) +- Fixed String to UInt monotonic conversion in case of usage String in primary key. [#3870](https://github.com/ClickHouse/ClickHouse/pull/3870) ([Winter Zhang](https://github.com/zhang2014)) +- Fixed error in calculation of integer conversion function monotonicity. [#3921](https://github.com/ClickHouse/ClickHouse/pull/3921) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed segfault in `arrayEnumerateUniq`, `arrayEnumerateDense` functions in case of some invalid arguments. [#3909](https://github.com/ClickHouse/ClickHouse/pull/3909) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix UB in StorageMerge. [#3910](https://github.com/ClickHouse/ClickHouse/pull/3910) ([Amos Bird](https://github.com/amosbird)) +- Fixed segfault in functions `addDays`, `subtractDays`. [#3913](https://github.com/ClickHouse/ClickHouse/pull/3913) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed error: functions `round`, `floor`, `trunc`, `ceil` may return bogus result when executed on integer argument and large negative scale. [#3914](https://github.com/ClickHouse/ClickHouse/pull/3914) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed a bug induced by ‘kill query sync’ which leads to a core dump. [#3916](https://github.com/ClickHouse/ClickHouse/pull/3916) ([muVulDeePecker](https://github.com/fancyqlx)) +- Fix bug with long delay after empty replication queue. [#3928](https://github.com/ClickHouse/ClickHouse/pull/3928) [#3932](https://github.com/ClickHouse/ClickHouse/pull/3932) ([alesapin](https://github.com/alesapin)) +- Fixed excessive memory usage in case of inserting into table with `LowCardinality` primary key. [#3955](https://github.com/ClickHouse/ClickHouse/pull/3955) ([KochetovNicolai](https://github.com/KochetovNicolai)) +- Fixed `LowCardinality` serialization for `Native` format in case of empty arrays. [#3907](https://github.com/ClickHouse/ClickHouse/issues/3907) [#4011](https://github.com/ClickHouse/ClickHouse/pull/4011) ([KochetovNicolai](https://github.com/KochetovNicolai)) +- Fixed incorrect result while using distinct by single LowCardinality numeric column. [#3895](https://github.com/ClickHouse/ClickHouse/issues/3895) [#4012](https://github.com/ClickHouse/ClickHouse/pull/4012) ([KochetovNicolai](https://github.com/KochetovNicolai)) +- Fixed specialized aggregation with LowCardinality key (in case when `compile` setting is enabled). [#3886](https://github.com/ClickHouse/ClickHouse/pull/3886) ([KochetovNicolai](https://github.com/KochetovNicolai)) +- Fix user and password forwarding for replicated tables queries. [#3957](https://github.com/ClickHouse/ClickHouse/pull/3957) ([alesapin](https://github.com/alesapin)) ([小路](https://github.com/nicelulu)) +- Fixed very rare race condition that can happen when listing tables in Dictionary database while reloading dictionaries. [#3970](https://github.com/ClickHouse/ClickHouse/pull/3970) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed incorrect result when HAVING was used with ROLLUP or CUBE. [#3756](https://github.com/ClickHouse/ClickHouse/issues/3756) [#3837](https://github.com/ClickHouse/ClickHouse/pull/3837) ([Sam Chou](https://github.com/reflection)) +- Fixed column aliases for query with `JOIN ON` syntax and distributed tables. [#3980](https://github.com/ClickHouse/ClickHouse/pull/3980) ([Winter Zhang](https://github.com/zhang2014)) +- Fixed error in internal implementation of `quantileTDigest` (found by Artem Vakhrushev). This error never happens in ClickHouse and was relevant only for those who use ClickHouse codebase as a library directly. [#3935](https://github.com/ClickHouse/ClickHouse/pull/3935) ([alexey-milovidov](https://github.com/alexey-milovidov)) #### Improvements {#improvements-6} -- Support for `IF NOT EXISTS` in `ALTER TABLE ADD COLUMN` statements along with `IF EXISTS` in `DROP/MODIFY/CLEAR/COMMENT COLUMN`. [\#3900](https://github.com/ClickHouse/ClickHouse/pull/3900) ([Boris Granveaud](https://github.com/bgranvea)) -- Function `parseDateTimeBestEffort`: support for formats `DD.MM.YYYY`, `DD.MM.YY`, `DD-MM-YYYY`, `DD-Mon-YYYY`, `DD/Month/YYYY` and similar. [\#3922](https://github.com/ClickHouse/ClickHouse/pull/3922) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- `CapnProtoInputStream` now support jagged structures. [\#4063](https://github.com/ClickHouse/ClickHouse/pull/4063) ([Odin Hultgren Van Der Horst](https://github.com/Miniwoffer)) -- Usability improvement: added a check that server process is started from the data directory’s owner. Do not allow to start server from root if the data belongs to non-root user. [\#3785](https://github.com/ClickHouse/ClickHouse/pull/3785) ([sergey-v-galtsev](https://github.com/sergey-v-galtsev)) -- Better logic of checking required columns during analysis of queries with JOINs. [\#3930](https://github.com/ClickHouse/ClickHouse/pull/3930) ([Artem Zuikov](https://github.com/4ertus2)) -- Decreased the number of connections in case of large number of Distributed tables in a single server. [\#3726](https://github.com/ClickHouse/ClickHouse/pull/3726) ([Winter Zhang](https://github.com/zhang2014)) -- Supported totals row for `WITH TOTALS` query for ODBC driver. [\#3836](https://github.com/ClickHouse/ClickHouse/pull/3836) ([Maksim Koritckiy](https://github.com/nightweb)) -- Allowed to use `Enum`s as integers inside if function. [\#3875](https://github.com/ClickHouse/ClickHouse/pull/3875) ([Ivan](https://github.com/abyss7)) -- Added `low_cardinality_allow_in_native_format` setting. If disabled, do not use `LowCadrinality` type in `Native` format. [\#3879](https://github.com/ClickHouse/ClickHouse/pull/3879) ([KochetovNicolai](https://github.com/KochetovNicolai)) -- Removed some redundant objects from compiled expressions cache to lower memory usage. [\#4042](https://github.com/ClickHouse/ClickHouse/pull/4042) ([alesapin](https://github.com/alesapin)) -- Add check that `SET send_logs_level = 'value'` query accept appropriate value. [\#3873](https://github.com/ClickHouse/ClickHouse/pull/3873) ([Sabyanin Maxim](https://github.com/s-mx)) -- Fixed data type check in type conversion functions. [\#3896](https://github.com/ClickHouse/ClickHouse/pull/3896) ([Winter Zhang](https://github.com/zhang2014)) +- Support for `IF NOT EXISTS` in `ALTER TABLE ADD COLUMN` statements along with `IF EXISTS` in `DROP/MODIFY/CLEAR/COMMENT COLUMN`. [#3900](https://github.com/ClickHouse/ClickHouse/pull/3900) ([Boris Granveaud](https://github.com/bgranvea)) +- Function `parseDateTimeBestEffort`: support for formats `DD.MM.YYYY`, `DD.MM.YY`, `DD-MM-YYYY`, `DD-Mon-YYYY`, `DD/Month/YYYY` and similar. [#3922](https://github.com/ClickHouse/ClickHouse/pull/3922) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- `CapnProtoInputStream` now support jagged structures. [#4063](https://github.com/ClickHouse/ClickHouse/pull/4063) ([Odin Hultgren Van Der Horst](https://github.com/Miniwoffer)) +- Usability improvement: added a check that server process is started from the data directory’s owner. Do not allow to start server from root if the data belongs to non-root user. [#3785](https://github.com/ClickHouse/ClickHouse/pull/3785) ([sergey-v-galtsev](https://github.com/sergey-v-galtsev)) +- Better logic of checking required columns during analysis of queries with JOINs. [#3930](https://github.com/ClickHouse/ClickHouse/pull/3930) ([Artem Zuikov](https://github.com/4ertus2)) +- Decreased the number of connections in case of large number of Distributed tables in a single server. [#3726](https://github.com/ClickHouse/ClickHouse/pull/3726) ([Winter Zhang](https://github.com/zhang2014)) +- Supported totals row for `WITH TOTALS` query for ODBC driver. [#3836](https://github.com/ClickHouse/ClickHouse/pull/3836) ([Maksim Koritckiy](https://github.com/nightweb)) +- Allowed to use `Enum`s as integers inside if function. [#3875](https://github.com/ClickHouse/ClickHouse/pull/3875) ([Ivan](https://github.com/abyss7)) +- Added `low_cardinality_allow_in_native_format` setting. If disabled, do not use `LowCadrinality` type in `Native` format. [#3879](https://github.com/ClickHouse/ClickHouse/pull/3879) ([KochetovNicolai](https://github.com/KochetovNicolai)) +- Removed some redundant objects from compiled expressions cache to lower memory usage. [#4042](https://github.com/ClickHouse/ClickHouse/pull/4042) ([alesapin](https://github.com/alesapin)) +- Add check that `SET send_logs_level = 'value'` query accept appropriate value. [#3873](https://github.com/ClickHouse/ClickHouse/pull/3873) ([Sabyanin Maxim](https://github.com/s-mx)) +- Fixed data type check in type conversion functions. [#3896](https://github.com/ClickHouse/ClickHouse/pull/3896) ([Winter Zhang](https://github.com/zhang2014)) #### Performance Improvements {#performance-improvements-5} -- Add a MergeTree setting `use_minimalistic_part_header_in_zookeeper`. If enabled, Replicated tables will store compact part metadata in a single part znode. This can dramatically reduce ZooKeeper snapshot size (especially if the tables have a lot of columns). Note that after enabling this setting you will not be able to downgrade to a version that doesn’t support it. [\#3960](https://github.com/ClickHouse/ClickHouse/pull/3960) ([Alex Zatelepin](https://github.com/ztlpn)) -- Add an DFA-based implementation for functions `sequenceMatch` and `sequenceCount` in case pattern doesn’t contain time. [\#4004](https://github.com/ClickHouse/ClickHouse/pull/4004) ([Léo Ercolanelli](https://github.com/ercolanelli-leo)) -- Performance improvement for integer numbers serialization. [\#3968](https://github.com/ClickHouse/ClickHouse/pull/3968) ([Amos Bird](https://github.com/amosbird)) -- Zero left padding PODArray so that -1 element is always valid and zeroed. It’s used for branchless calculation of offsets. [\#3920](https://github.com/ClickHouse/ClickHouse/pull/3920) ([Amos Bird](https://github.com/amosbird)) -- Reverted `jemalloc` version which lead to performance degradation. [\#4018](https://github.com/ClickHouse/ClickHouse/pull/4018) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Add a MergeTree setting `use_minimalistic_part_header_in_zookeeper`. If enabled, Replicated tables will store compact part metadata in a single part znode. This can dramatically reduce ZooKeeper snapshot size (especially if the tables have a lot of columns). Note that after enabling this setting you will not be able to downgrade to a version that doesn’t support it. [#3960](https://github.com/ClickHouse/ClickHouse/pull/3960) ([Alex Zatelepin](https://github.com/ztlpn)) +- Add an DFA-based implementation for functions `sequenceMatch` and `sequenceCount` in case pattern doesn’t contain time. [#4004](https://github.com/ClickHouse/ClickHouse/pull/4004) ([Léo Ercolanelli](https://github.com/ercolanelli-leo)) +- Performance improvement for integer numbers serialization. [#3968](https://github.com/ClickHouse/ClickHouse/pull/3968) ([Amos Bird](https://github.com/amosbird)) +- Zero left padding PODArray so that -1 element is always valid and zeroed. It’s used for branchless calculation of offsets. [#3920](https://github.com/ClickHouse/ClickHouse/pull/3920) ([Amos Bird](https://github.com/amosbird)) +- Reverted `jemalloc` version which lead to performance degradation. [#4018](https://github.com/ClickHouse/ClickHouse/pull/4018) ([alexey-milovidov](https://github.com/alexey-milovidov)) #### Backward Incompatible Changes {#backward-incompatible-changes-2} -- Removed undocumented feature `ALTER MODIFY PRIMARY KEY` because it was superseded by the `ALTER MODIFY ORDER BY` command. [\#3887](https://github.com/ClickHouse/ClickHouse/pull/3887) ([Alex Zatelepin](https://github.com/ztlpn)) -- Removed function `shardByHash`. [\#3833](https://github.com/ClickHouse/ClickHouse/pull/3833) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Forbid using scalar subqueries with result of type `AggregateFunction`. [\#3865](https://github.com/ClickHouse/ClickHouse/pull/3865) ([Ivan](https://github.com/abyss7)) +- Removed undocumented feature `ALTER MODIFY PRIMARY KEY` because it was superseded by the `ALTER MODIFY ORDER BY` command. [#3887](https://github.com/ClickHouse/ClickHouse/pull/3887) ([Alex Zatelepin](https://github.com/ztlpn)) +- Removed function `shardByHash`. [#3833](https://github.com/ClickHouse/ClickHouse/pull/3833) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Forbid using scalar subqueries with result of type `AggregateFunction`. [#3865](https://github.com/ClickHouse/ClickHouse/pull/3865) ([Ivan](https://github.com/abyss7)) #### Build/Testing/Packaging Improvements {#buildtestingpackaging-improvements-6} -- Added support for PowerPC (`ppc64le`) build. [\#4132](https://github.com/ClickHouse/ClickHouse/pull/4132) ([Danila Kutenin](https://github.com/danlark1)) -- Stateful functional tests are run on public available dataset. [\#3969](https://github.com/ClickHouse/ClickHouse/pull/3969) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed error when the server cannot start with the `bash: /usr/bin/clickhouse-extract-from-config: Operation not permitted` message within Docker or systemd-nspawn. [\#4136](https://github.com/ClickHouse/ClickHouse/pull/4136) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Updated `rdkafka` library to v1.0.0-RC5. Used cppkafka instead of raw C interface. [\#4025](https://github.com/ClickHouse/ClickHouse/pull/4025) ([Ivan](https://github.com/abyss7)) -- Updated `mariadb-client` library. Fixed one of issues found by UBSan. [\#3924](https://github.com/ClickHouse/ClickHouse/pull/3924) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Some fixes for UBSan builds. [\#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)) +- Added support for PowerPC (`ppc64le`) build. [#4132](https://github.com/ClickHouse/ClickHouse/pull/4132) ([Danila Kutenin](https://github.com/danlark1)) +- Stateful functional tests are run on public available dataset. [#3969](https://github.com/ClickHouse/ClickHouse/pull/3969) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed error when the server cannot start with the `bash: /usr/bin/clickhouse-extract-from-config: Operation not permitted` message within Docker or systemd-nspawn. [#4136](https://github.com/ClickHouse/ClickHouse/pull/4136) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Updated `rdkafka` library to v1.0.0-RC5. Used cppkafka instead of raw C interface. [#4025](https://github.com/ClickHouse/ClickHouse/pull/4025) ([Ivan](https://github.com/abyss7)) +- Updated `mariadb-client` library. Fixed one of issues found by UBSan. [#3924](https://github.com/ClickHouse/ClickHouse/pull/3924) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Some fixes for UBSan builds. [#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)) - Added per-commit runs of tests with UBSan build. - Added per-commit runs of PVS-Studio static analyzer. -- Fixed bugs found by PVS-Studio. [\#4013](https://github.com/ClickHouse/ClickHouse/pull/4013) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed glibc compatibility issues. [\#4100](https://github.com/ClickHouse/ClickHouse/pull/4100) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Move Docker images to 18.10 and add compatibility file for glibc \>= 2.28 [\#3965](https://github.com/ClickHouse/ClickHouse/pull/3965) ([alesapin](https://github.com/alesapin)) -- Add env variable if user don’t want to chown directories in server Docker image. [\#3967](https://github.com/ClickHouse/ClickHouse/pull/3967) ([alesapin](https://github.com/alesapin)) -- Enabled most of the warnings from `-Weverything` in clang. Enabled `-Wpedantic`. [\#3986](https://github.com/ClickHouse/ClickHouse/pull/3986) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Added a few more warnings that are available only in clang 8. [\#3993](https://github.com/ClickHouse/ClickHouse/pull/3993) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Link to `libLLVM` rather than to individual LLVM libs when using shared linking. [\#3989](https://github.com/ClickHouse/ClickHouse/pull/3989) ([Orivej Desh](https://github.com/orivej)) -- Added sanitizer variables for test images. [\#4072](https://github.com/ClickHouse/ClickHouse/pull/4072) ([alesapin](https://github.com/alesapin)) -- `clickhouse-server` debian package will recommend `libcap2-bin` package to use `setcap` tool for setting capabilities. This is optional. [\#4093](https://github.com/ClickHouse/ClickHouse/pull/4093) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Improved compilation time, fixed includes. [\#3898](https://github.com/ClickHouse/ClickHouse/pull/3898) ([proller](https://github.com/proller)) -- Added performance tests for hash functions. [\#3918](https://github.com/ClickHouse/ClickHouse/pull/3918) ([filimonov](https://github.com/filimonov)) -- Fixed cyclic library dependences. [\#3958](https://github.com/ClickHouse/ClickHouse/pull/3958) ([proller](https://github.com/proller)) -- Improved compilation with low available memory. [\#4030](https://github.com/ClickHouse/ClickHouse/pull/4030) ([proller](https://github.com/proller)) -- Added test script to reproduce performance degradation in `jemalloc`. [\#4036](https://github.com/ClickHouse/ClickHouse/pull/4036) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Fixed misspells in comments and string literals under `dbms`. [\#4122](https://github.com/ClickHouse/ClickHouse/pull/4122) ([maiha](https://github.com/maiha)) -- Fixed typos in comments. [\#4089](https://github.com/ClickHouse/ClickHouse/pull/4089) ([Evgenii Pravda](https://github.com/kvinty)) +- Fixed bugs found by PVS-Studio. [#4013](https://github.com/ClickHouse/ClickHouse/pull/4013) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed glibc compatibility issues. [#4100](https://github.com/ClickHouse/ClickHouse/pull/4100) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Move Docker images to 18.10 and add compatibility file for glibc \>= 2.28 [#3965](https://github.com/ClickHouse/ClickHouse/pull/3965) ([alesapin](https://github.com/alesapin)) +- Add env variable if user don’t want to chown directories in server Docker image. [#3967](https://github.com/ClickHouse/ClickHouse/pull/3967) ([alesapin](https://github.com/alesapin)) +- Enabled most of the warnings from `-Weverything` in clang. Enabled `-Wpedantic`. [#3986](https://github.com/ClickHouse/ClickHouse/pull/3986) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Added a few more warnings that are available only in clang 8. [#3993](https://github.com/ClickHouse/ClickHouse/pull/3993) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Link to `libLLVM` rather than to individual LLVM libs when using shared linking. [#3989](https://github.com/ClickHouse/ClickHouse/pull/3989) ([Orivej Desh](https://github.com/orivej)) +- Added sanitizer variables for test images. [#4072](https://github.com/ClickHouse/ClickHouse/pull/4072) ([alesapin](https://github.com/alesapin)) +- `clickhouse-server` debian package will recommend `libcap2-bin` package to use `setcap` tool for setting capabilities. This is optional. [#4093](https://github.com/ClickHouse/ClickHouse/pull/4093) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Improved compilation time, fixed includes. [#3898](https://github.com/ClickHouse/ClickHouse/pull/3898) ([proller](https://github.com/proller)) +- Added performance tests for hash functions. [#3918](https://github.com/ClickHouse/ClickHouse/pull/3918) ([filimonov](https://github.com/filimonov)) +- Fixed cyclic library dependences. [#3958](https://github.com/ClickHouse/ClickHouse/pull/3958) ([proller](https://github.com/proller)) +- Improved compilation with low available memory. [#4030](https://github.com/ClickHouse/ClickHouse/pull/4030) ([proller](https://github.com/proller)) +- Added test script to reproduce performance degradation in `jemalloc`. [#4036](https://github.com/ClickHouse/ClickHouse/pull/4036) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fixed misspells in comments and string literals under `dbms`. [#4122](https://github.com/ClickHouse/ClickHouse/pull/4122) ([maiha](https://github.com/maiha)) +- Fixed typos in comments. [#4089](https://github.com/ClickHouse/ClickHouse/pull/4089) ([Evgenii Pravda](https://github.com/kvinty)) ## [Changelog for 2018](../../whats-new/changelog/2018.md#clickhouse-release-18-16) {#changelog-for-2018} diff --git a/docs/en/whats-new/security-changelog.md b/docs/en/whats-new/security-changelog.md index 80f5d2a421a..aecd7a26508 100644 --- a/docs/en/whats-new/security-changelog.md +++ b/docs/en/whats-new/security-changelog.md @@ -51,7 +51,7 @@ Credits: Andrey Krasichkov and Evgeny Sidorov of Yandex Information Security Tea ### CVE-2018-14668 {#cve-2018-14668} -“remote” table function allowed arbitrary symbols in “user”, “password” and “default\_database” fields which led to Cross Protocol Request Forgery Attacks. +“remote” table function allowed arbitrary symbols in “user”, “password” and “default_database” fields which led to Cross Protocol Request Forgery Attacks. Credits: Andrey Krasichkov of Yandex Information Security Team diff --git a/docs/es/development/architecture.md b/docs/es/development/architecture.md index 90f028541f3..1620a58a3a0 100644 --- a/docs/es/development/architecture.md +++ b/docs/es/development/architecture.md @@ -47,7 +47,7 @@ A `Block` es un contenedor que representa un subconjunto (porción) de una tabla Cuando calculamos alguna función sobre columnas en un bloque, agregamos otra columna con su resultado al bloque, y no tocamos columnas para argumentos de la función porque las operaciones son inmutables. Más tarde, las columnas innecesarias se pueden eliminar del bloque, pero no se pueden modificar. Es conveniente para la eliminación de subexpresiones comunes. -Se crean bloques para cada fragmento de datos procesado. Tenga en cuenta que para el mismo tipo de cálculo, los nombres y tipos de columna siguen siendo los mismos para diferentes bloques y solo cambian los datos de columna. Es mejor dividir los datos del bloque desde el encabezado del bloque porque los tamaños de bloque pequeños tienen una gran sobrecarga de cadenas temporales para copiar shared\_ptrs y nombres de columna. +Se crean bloques para cada fragmento de datos procesado. Tenga en cuenta que para el mismo tipo de cálculo, los nombres y tipos de columna siguen siendo los mismos para diferentes bloques y solo cambian los datos de columna. Es mejor dividir los datos del bloque desde el encabezado del bloque porque los tamaños de bloque pequeños tienen una gran sobrecarga de cadenas temporales para copiar shared_ptrs y nombres de columna. ## Bloquear flujos {#block-streams} diff --git a/docs/es/development/build.md b/docs/es/development/build.md index 0b190242f8c..7a1479874eb 100644 --- a/docs/es/development/build.md +++ b/docs/es/development/build.md @@ -9,7 +9,7 @@ toc_title: "C\xF3mo crear ClickHouse en Linux" El siguiente tutorial se basa en el sistema Ubuntu Linux. Con los cambios apropiados, también debería funcionar en cualquier otra distribución de Linux. -Plataformas compatibles: x86\_64 y AArch64. El soporte para Power9 es experimental. +Plataformas compatibles: x86_64 y AArch64. El soporte para Power9 es experimental. ## Instalar Git, CMake, Python y Ninja {#install-git-cmake-python-and-ninja} diff --git a/docs/es/development/contrib.md b/docs/es/development/contrib.md index a510c9e80a5..9018c19cc92 100644 --- a/docs/es/development/contrib.md +++ b/docs/es/development/contrib.md @@ -28,7 +28,7 @@ toc_title: Bibliotecas de terceros utilizadas | libpcg-al azar | [Licencia Apache 2.0](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libpcg-random/LICENSE-APACHE.txt) | | Libressl | [Licencia OpenSSL](https://github.com/ClickHouse-Extras/ssl/blob/master/COPYING) | | Librdkafka | [Licencia BSD de 2 cláusulas](https://github.com/edenhill/librdkafka/blob/363dcad5a23dc29381cc626620e68ae418b3af19/LICENSE) | -| libwidechar\_width | [CC0 1.0 Universal](https://github.com/ClickHouse/ClickHouse/blob/master/libs/libwidechar_width/LICENSE) | +| libwidechar_width | [CC0 1.0 Universal](https://github.com/ClickHouse/ClickHouse/blob/master/libs/libwidechar_width/LICENSE) | | llvm | [Licencia de 3 cláusulas BSD](https://github.com/ClickHouse-Extras/llvm/blob/163def217817c90fb982a6daf384744d8472b92b/llvm/LICENSE.TXT) | | lz4 | [Licencia BSD de 2 cláusulas](https://github.com/lz4/lz4/blob/c10863b98e1503af90616ae99725ecd120265dfb/LICENSE) | | mariadb-conector-c | [Información adicional](https://github.com/ClickHouse-Extras/mariadb-connector-c/blob/3.1/COPYING.LIB) | diff --git a/docs/es/development/developer-instruction.md b/docs/es/development/developer-instruction.md index 8e076a60ac4..8570fc1924c 100644 --- a/docs/es/development/developer-instruction.md +++ b/docs/es/development/developer-instruction.md @@ -44,7 +44,7 @@ En el terminal de línea de comandos, ejecute: git clone --recursive git@github.com:your_github_username/ClickHouse.git cd ClickHouse -Nota: por favor, sustituye *your\_github\_username* con lo que es apropiado! +Nota: por favor, sustituye *your_github_username* con lo que es apropiado! Este comando creará un directorio `ClickHouse` que contiene la copia de trabajo del proyecto. @@ -154,7 +154,7 @@ Ahora que está listo para construir ClickHouse, le recomendamos que cree un dir mkdir build cd build -Puede tener varios directorios diferentes (build\_release, build\_debug, etc.) para diferentes tipos de construcción. +Puede tener varios directorios diferentes (build_release, build_debug, etc.) para diferentes tipos de construcción. Mientras que dentro de la `build` directorio, configure su compilación ejecutando CMake. Antes de la primera ejecución, debe definir variables de entorno que especifiquen el compilador (compilador gcc versión 9 en este ejemplo). diff --git a/docs/es/development/style.md b/docs/es/development/style.md index cca56a3bd22..c358d613fca 100644 --- a/docs/es/development/style.md +++ b/docs/es/development/style.md @@ -356,7 +356,7 @@ En todos los demás casos, use un nombre que describa el significado. bool info_successfully_loaded = false; ``` -**9.** Nombres de `define`s y las constantes globales usan ALL\_CAPS con guiones bajos. +**9.** Nombres de `define`s y las constantes globales usan ALL_CAPS con guiones bajos. ``` cpp #define MAX_SRC_TABLE_NAMES_TO_STORE 1000 @@ -396,7 +396,7 @@ El sufijo de subrayado se puede omitir si el argumento no se usa en el cuerpo de timer (not m_timer) ``` -**14.** Para las constantes en un `enum`, usar CamelCase con una letra mayúscula. ALL\_CAPS también es aceptable. Si el `enum` no es local, utilice un `enum class`. +**14.** Para las constantes en un `enum`, usar CamelCase con una letra mayúscula. ALL_CAPS también es aceptable. Si el `enum` no es local, utilice un `enum class`. ``` cpp enum class CompressionMethod @@ -709,7 +709,7 @@ Se utiliza la biblioteca estándar (`libc++`). **4.**OS: Linux Ubuntu, no más viejo que Precise. -**5.**El código está escrito para la arquitectura de CPU x86\_64. +**5.**El código está escrito para la arquitectura de CPU x86_64. El conjunto de instrucciones de CPU es el conjunto mínimo admitido entre nuestros servidores. Actualmente, es SSE 4.2. diff --git a/docs/es/development/tests.md b/docs/es/development/tests.md index b6c225894a5..ae2b61b6ba7 100644 --- a/docs/es/development/tests.md +++ b/docs/es/development/tests.md @@ -202,7 +202,7 @@ Versión de depuración de `jemalloc` se utiliza para la compilación de depurac ClickHouse fuzzing se implementa tanto usando [LibFuzzer](https://llvm.org/docs/LibFuzzer.html) y consultas SQL aleatorias. Todas las pruebas de fuzz deben realizarse con desinfectantes (Dirección y Undefined). -LibFuzzer se usa para pruebas de fuzz aisladas del código de la biblioteca. Fuzzers se implementan como parte del código de prueba y tienen “\_fuzzer” nombre postfixes. +LibFuzzer se usa para pruebas de fuzz aisladas del código de la biblioteca. Fuzzers se implementan como parte del código de prueba y tienen “_fuzzer” nombre postfixes. El ejemplo de Fuzzer se puede encontrar en `src/Parsers/tests/lexer_fuzzer.cpp`. Las configuraciones, diccionarios y corpus específicos de LibFuzzer se almacenan en `tests/fuzz`. Le recomendamos que escriba pruebas fuzz para cada funcionalidad que maneje la entrada del usuario. diff --git a/docs/es/engines/table-engines/index.md b/docs/es/engines/table-engines/index.md index cbc7cc5aaa8..7be315e3ee3 100644 --- a/docs/es/engines/table-engines/index.md +++ b/docs/es/engines/table-engines/index.md @@ -62,7 +62,7 @@ Motores en la familia: - [Distribuido](special/distributed.md#distributed) - [Método de codificación de datos:](special/materializedview.md#materializedview) - [Diccionario](special/dictionary.md#dictionary) -- \[Fusión\](special/merge.md\#merge +- \[Fusión\](special/merge.md#merge - [File](special/file.md#file) - [Nulo](special/null.md#null) - [Establecer](special/set.md#set) diff --git a/docs/es/engines/table-engines/integrations/hdfs.md b/docs/es/engines/table-engines/integrations/hdfs.md index 500d6e91d90..5e0211660f5 100644 --- a/docs/es/engines/table-engines/integrations/hdfs.md +++ b/docs/es/engines/table-engines/integrations/hdfs.md @@ -73,12 +73,12 @@ Construcciones con `{}` son similares a la [remoto](../../../sql-reference/table 1. Supongamos que tenemos varios archivos en formato TSV con los siguientes URI en HDFS: -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_3’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_3’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_3’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_3’ 1. Hay varias maneras de hacer una tabla que consta de los seis archivos: diff --git a/docs/es/engines/table-engines/integrations/kafka.md b/docs/es/engines/table-engines/integrations/kafka.md index 508ac72b8e7..54250aae82a 100644 --- a/docs/es/engines/table-engines/integrations/kafka.md +++ b/docs/es/engines/table-engines/integrations/kafka.md @@ -134,7 +134,7 @@ Ejemplo: SELECT level, sum(total) FROM daily GROUP BY level; ``` -Para mejorar el rendimiento, los mensajes recibidos se agrupan en bloques del tamaño de [Max\_insert\_block\_size](../../../operations/server-configuration-parameters/settings.md#settings-max_insert_block_size). Si el bloque no se formó dentro de [Nombre de la red inalámbrica (SSID):](../../../operations/server-configuration-parameters/settings.md) milisegundos, los datos se vaciarán a la tabla independientemente de la integridad del bloque. +Para mejorar el rendimiento, los mensajes recibidos se agrupan en bloques del tamaño de [Max_insert_block_size](../../../operations/server-configuration-parameters/settings.md#settings-max_insert_block_size). Si el bloque no se formó dentro de [Nombre de la red inalámbrica (SSID):](../../../operations/server-configuration-parameters/settings.md) milisegundos, los datos se vaciarán a la tabla independientemente de la integridad del bloque. Para detener la recepción de datos de tema o cambiar la lógica de conversión, desconecte la vista materializada: diff --git a/docs/es/engines/table-engines/mergetree-family/custom-partitioning-key.md b/docs/es/engines/table-engines/mergetree-family/custom-partitioning-key.md index 49829ee864f..6cbc0a9192e 100644 --- a/docs/es/engines/table-engines/mergetree-family/custom-partitioning-key.md +++ b/docs/es/engines/table-engines/mergetree-family/custom-partitioning-key.md @@ -116,7 +116,7 @@ drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 5 12:09 201902_4_6_1 drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 1 16:48 detached ``` -Carpeta ‘201901\_1\_1\_0’, ‘201901\_1\_7\_1’ y así sucesivamente son los directorios de las partes. Cada parte se relaciona con una partición correspondiente y contiene datos solo para un mes determinado (la tabla de este ejemplo tiene particiones por mes). +Carpeta ‘201901_1_1_0’, ‘201901_1_7_1’ y así sucesivamente son los directorios de las partes. Cada parte se relaciona con una partición correspondiente y contiene datos solo para un mes determinado (la tabla de este ejemplo tiene particiones por mes). El `detached` el directorio contiene partes que se separaron de la tabla utilizando el [DETACH](../../../sql-reference/statements/alter.md#alter_detach-partition) consulta. Las partes dañadas también se mueven a este directorio, en lugar de eliminarse. El servidor no utiliza las piezas del `detached` directory. You can add, delete, or modify the data in this directory at any time – the server will not know about this until you run the [ATTACH](../../../sql-reference/statements/alter.md#alter_attach-partition) consulta. diff --git a/docs/es/engines/table-engines/mergetree-family/graphitemergetree.md b/docs/es/engines/table-engines/mergetree-family/graphitemergetree.md index 66438fc4bf9..d33ddcebac2 100644 --- a/docs/es/engines/table-engines/mergetree-family/graphitemergetree.md +++ b/docs/es/engines/table-engines/mergetree-family/graphitemergetree.md @@ -81,7 +81,7 @@ Todos los parámetros excepto `config_section` el mismo significado que en `Merg ## Configuración acumulativa {#rollup-configuration} -La configuración del paquete acumulativo está definida por [graphite\_rollup](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) parámetro en la configuración del servidor. El nombre del parámetro podría ser cualquiera. Puede crear varias configuraciones y usarlas para diferentes tablas. +La configuración del paquete acumulativo está definida por [graphite_rollup](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) parámetro en la configuración del servidor. El nombre del parámetro podría ser cualquiera. Puede crear varias configuraciones y usarlas para diferentes tablas. Estructura de configuración Rollup: diff --git a/docs/es/engines/table-engines/mergetree-family/mergetree.md b/docs/es/engines/table-engines/mergetree-family/mergetree.md index 9f7dc5a63f8..a4bab840b52 100644 --- a/docs/es/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/es/engines/table-engines/mergetree-family/mergetree.md @@ -239,7 +239,7 @@ En el siguiente ejemplo, el índice no se puede usar. SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' ``` -Para comprobar si ClickHouse puede usar el índice al ejecutar una consulta, use la configuración [Fecha de nacimiento](../../../operations/settings/settings.md#settings-force_index_by_date) y [force\_primary\_key](../../../operations/settings/settings.md). +Para comprobar si ClickHouse puede usar el índice al ejecutar una consulta, use la configuración [Fecha de nacimiento](../../../operations/settings/settings.md#settings-force_index_by_date) y [force_primary_key](../../../operations/settings/settings.md). La clave para particionar por mes permite leer solo aquellos bloques de datos que contienen fechas del rango adecuado. En este caso, el bloque de datos puede contener datos para muchas fechas (hasta un mes). Dentro de un bloque, los datos se ordenan por clave principal, que puede no contener la fecha como la primera columna. Debido a esto, el uso de una consulta con solo una condición de fecha que no especifica el prefijo de clave principal hará que se lean más datos que para una sola fecha. @@ -330,7 +330,7 @@ Condiciones en el `WHERE` cláusula contiene llamadas de las funciones que opera El `set` index se puede utilizar con todas las funciones. Subconjuntos de funciones para otros índices se muestran en la siguiente tabla. -| Función (operador) / Índice | clave primaria | minmax | Descripción | Sistema abierto. | bloom\_filter | +| Función (operador) / Índice | clave primaria | minmax | Descripción | Sistema abierto. | bloom_filter | |----------------------------------------------------------------------------------------------------------|----------------|--------|-------------|------------------|---------------| | [igual (=, ==)](../../../sql-reference/functions/comparison-functions.md#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | | [notEquals(!=, \<\>)](../../../sql-reference/functions/comparison-functions.md#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | @@ -495,7 +495,7 @@ La parte de datos es la unidad móvil mínima para `MergeTree`-mesas de motor. L - Volume — Ordered set of equal disks (similar to [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures)). - Storage policy — Set of volumes and the rules for moving data between them. -Los nombres dados a las entidades descritas se pueden encontrar en las tablas del sistema, [sistema.almacenamiento\_policies](../../../operations/system-tables.md#system_tables-storage_policies) y [sistema.disco](../../../operations/system-tables.md#system_tables-disks). Para aplicar una de las directivas de almacenamiento configuradas para una tabla, `storage_policy` establecimiento de `MergeTree`-mesas de la familia del motor. +Los nombres dados a las entidades descritas se pueden encontrar en las tablas del sistema, [sistema.almacenamiento_policies](../../../operations/system-tables.md#system_tables-storage_policies) y [sistema.disco](../../../operations/system-tables.md#system_tables-disks). Para aplicar una de las directivas de almacenamiento configuradas para una tabla, `storage_policy` establecimiento de `MergeTree`-mesas de la familia del motor. ### Configuración {#table_engine-mergetree-multiple-volumes_configure} @@ -642,7 +642,7 @@ En todos estos casos, excepto las mutaciones y la congelación de particiones, u Bajo el capó, las mutaciones y la congelación de particiones hacen uso de [enlaces duros](https://en.wikipedia.org/wiki/Hard_link). Los enlaces duros entre diferentes discos no son compatibles, por lo tanto, en tales casos las partes resultantes se almacenan en los mismos discos que los iniciales. En el fondo, las partes se mueven entre volúmenes en función de la cantidad de espacio libre (`move_factor` parámetro) según el orden en que se declaran los volúmenes en el archivo de configuración. -Los datos nunca se transfieren desde el último y al primero. Uno puede usar tablas del sistema [sistema.part\_log](../../../operations/system-tables.md#system_tables-part-log) (campo `type = MOVE_PART`) y [sistema.parte](../../../operations/system-tables.md#system_tables-parts) (campo `path` y `disk`) para monitorear movimientos de fondo. Además, la información detallada se puede encontrar en los registros del servidor. +Los datos nunca se transfieren desde el último y al primero. Uno puede usar tablas del sistema [sistema.part_log](../../../operations/system-tables.md#system_tables-part-log) (campo `type = MOVE_PART`) y [sistema.parte](../../../operations/system-tables.md#system_tables-parts) (campo `path` y `disk`) para monitorear movimientos de fondo. Además, la información detallada se puede encontrar en los registros del servidor. El usuario puede forzar el movimiento de una pieza o una partición de un volumen a otro mediante la consulta [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](../../../sql-reference/statements/alter.md#alter_move-partition), todas las restricciones para las operaciones en segundo plano se tienen en cuenta. La consulta inicia un movimiento por sí misma y no espera a que se completen las operaciones en segundo plano. El usuario recibirá un mensaje de error si no hay suficiente espacio libre disponible o si no se cumple alguna de las condiciones requeridas. diff --git a/docs/es/engines/table-engines/mergetree-family/replication.md b/docs/es/engines/table-engines/mergetree-family/replication.md index 52b4029fd82..505f5223800 100644 --- a/docs/es/engines/table-engines/mergetree-family/replication.md +++ b/docs/es/engines/table-engines/mergetree-family/replication.md @@ -59,7 +59,7 @@ Puede especificar cualquier clúster ZooKeeper existente y el sistema utilizará Si ZooKeeper no está establecido en el archivo de configuración, no puede crear tablas replicadas y las tablas replicadas existentes serán de solo lectura. -ZooKeeper no se utiliza en `SELECT` consultas porque la replicación no afecta al rendimiento de `SELECT` y las consultas se ejecutan tan rápido como lo hacen para las tablas no replicadas. Al consultar tablas replicadas distribuidas, el comportamiento de ClickHouse se controla mediante la configuración [max\_replica\_delay\_for\_distributed\_queries](../../../operations/settings/settings.md#settings-max_replica_delay_for_distributed_queries) y [fallback\_to\_stale\_replicas\_for\_distributed\_queries](../../../operations/settings/settings.md#settings-fallback_to_stale_replicas_for_distributed_queries). +ZooKeeper no se utiliza en `SELECT` consultas porque la replicación no afecta al rendimiento de `SELECT` y las consultas se ejecutan tan rápido como lo hacen para las tablas no replicadas. Al consultar tablas replicadas distribuidas, el comportamiento de ClickHouse se controla mediante la configuración [max_replica_delay_for_distributed_queries](../../../operations/settings/settings.md#settings-max_replica_delay_for_distributed_queries) y [fallback_to_stale_replicas_for_distributed_queries](../../../operations/settings/settings.md#settings-fallback_to_stale_replicas_for_distributed_queries). Para cada `INSERT` consulta, aproximadamente diez entradas se agregan a ZooKeeper a través de varias transacciones. (Para ser más precisos, esto es para cada bloque de datos insertado; una consulta INSERT contiene un bloque o un bloque por `max_insert_block_size = 1048576` filas.) Esto conduce a latencias ligeramente más largas para `INSERT` en comparación con las tablas no replicadas. Pero si sigue las recomendaciones para insertar datos en lotes de no más de uno `INSERT` por segundo, no crea ningún problema. Todo el clúster ClickHouse utilizado para coordinar un clúster ZooKeeper tiene un total de varios cientos `INSERTs` por segundo. El rendimiento en las inserciones de datos (el número de filas por segundo) es tan alto como para los datos no replicados. @@ -71,7 +71,7 @@ De forma predeterminada, una consulta INSERT espera la confirmación de la escri Cada bloque de datos se escribe atómicamente. La consulta INSERT se divide en bloques hasta `max_insert_block_size = 1048576` filas. En otras palabras, si el `INSERT` consulta tiene menos de 1048576 filas, se hace atómicamente. -Los bloques de datos se deduplican. Para varias escrituras del mismo bloque de datos (bloques de datos del mismo tamaño que contienen las mismas filas en el mismo orden), el bloque solo se escribe una vez. La razón de esto es en caso de fallas de red cuando la aplicación cliente no sabe si los datos se escribieron en la base de datos, por lo que `INSERT` consulta simplemente se puede repetir. No importa a qué réplica se enviaron los INSERT con datos idénticos. `INSERTs` son idempotentes. Los parámetros de desduplicación son controlados por [merge\_tree](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-merge_tree) configuración del servidor. +Los bloques de datos se deduplican. Para varias escrituras del mismo bloque de datos (bloques de datos del mismo tamaño que contienen las mismas filas en el mismo orden), el bloque solo se escribe una vez. La razón de esto es en caso de fallas de red cuando la aplicación cliente no sabe si los datos se escribieron en la base de datos, por lo que `INSERT` consulta simplemente se puede repetir. No importa a qué réplica se enviaron los INSERT con datos idénticos. `INSERTs` son idempotentes. Los parámetros de desduplicación son controlados por [merge_tree](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-merge_tree) configuración del servidor. Durante la replicación, sólo los datos de origen que se van a insertar se transfieren a través de la red. La transformación de datos adicional (fusión) se coordina y se realiza en todas las réplicas de la misma manera. Esto minimiza el uso de la red, lo que significa que la replicación funciona bien cuando las réplicas residen en centros de datos diferentes. (Tenga en cuenta que la duplicación de datos en diferentes centros de datos es el objetivo principal de la replicación.) diff --git a/docs/es/engines/table-engines/special/buffer.md b/docs/es/engines/table-engines/special/buffer.md index 3c8a20beaf3..b3a26ff356a 100644 --- a/docs/es/engines/table-engines/special/buffer.md +++ b/docs/es/engines/table-engines/special/buffer.md @@ -36,7 +36,7 @@ Ejemplo: CREATE TABLE merge.hits_buffer AS merge.hits ENGINE = Buffer(merge, hits, 16, 10, 100, 10000, 1000000, 10000000, 100000000) ``` -Creación de un ‘merge.hits\_buffer’ mesa con la misma estructura que ‘merge.hits’ y usando el motor Buffer. Al escribir en esta tabla, los datos se almacenan en la memoria RAM y ‘merge.hits’ tabla. Se crean 16 búferes. Los datos de cada uno de ellos se vacían si han pasado 100 segundos o se han escrito un millón de filas o se han escrito 100 MB de datos; o si simultáneamente han pasado 10 segundos y se han escrito 10.000 filas y 10 MB de datos. Por ejemplo, si solo se ha escrito una fila, después de 100 segundos se vaciará, pase lo que pase. Pero si se han escrito muchas filas, los datos se vaciarán antes. +Creación de un ‘merge.hits_buffer’ mesa con la misma estructura que ‘merge.hits’ y usando el motor Buffer. Al escribir en esta tabla, los datos se almacenan en la memoria RAM y ‘merge.hits’ tabla. Se crean 16 búferes. Los datos de cada uno de ellos se vacían si han pasado 100 segundos o se han escrito un millón de filas o se han escrito 100 MB de datos; o si simultáneamente han pasado 10 segundos y se han escrito 10.000 filas y 10 MB de datos. Por ejemplo, si solo se ha escrito una fila, después de 100 segundos se vaciará, pase lo que pase. Pero si se han escrito muchas filas, los datos se vaciarán antes. Cuando se detiene el servidor, con DROP TABLE o DETACH TABLE, los datos del búfer también se vacían a la tabla de destino. @@ -58,7 +58,7 @@ FINAL y SAMPLE no funcionan correctamente para las tablas Buffer. Estas condicio Al agregar datos a un búfer, uno de los búferes está bloqueado. Esto provoca retrasos si se realiza una operación de lectura simultáneamente desde la tabla. -Los datos que se insertan en una tabla de búfer pueden terminar en la tabla subordinada en un orden diferente y en bloques diferentes. Debido a esto, una tabla Buffer es difícil de usar para escribir en un CollapsingMergeTree correctamente. Para evitar problemas, puede establecer ‘num\_layers’ a 1. +Los datos que se insertan en una tabla de búfer pueden terminar en la tabla subordinada en un orden diferente y en bloques diferentes. Debido a esto, una tabla Buffer es difícil de usar para escribir en un CollapsingMergeTree correctamente. Para evitar problemas, puede establecer ‘num_layers’ a 1. Si se replica la tabla de destino, se pierden algunas características esperadas de las tablas replicadas al escribir en una tabla de búfer. Los cambios aleatorios en el orden de las filas y los tamaños de las partes de datos hacen que la desduplicación de datos deje de funcionar, lo que significa que no es posible tener un ‘exactly once’ escribir en tablas replicadas. diff --git a/docs/es/engines/table-engines/special/distributed.md b/docs/es/engines/table-engines/special/distributed.md index df773c7aec4..bac407a651a 100644 --- a/docs/es/engines/table-engines/special/distributed.md +++ b/docs/es/engines/table-engines/special/distributed.md @@ -85,13 +85,13 @@ Los nombres de clúster no deben contener puntos. Los parámetros `host`, `port`, y opcionalmente `user`, `password`, `secure`, `compression` se especifican para cada servidor: - `host` – The address of the remote server. You can use either the domain or the IPv4 or IPv6 address. If you specify the domain, the server makes a DNS request when it starts, and the result is stored as long as the server is running. If the DNS request fails, the server doesn't start. If you change the DNS record, restart the server. -- `port` – The TCP port for messenger activity (‘tcp\_port’ en la configuración, generalmente establecido en 9000). No lo confundas con http\_port. +- `port` – The TCP port for messenger activity (‘tcp_port’ en la configuración, generalmente establecido en 9000). No lo confundas con http_port. - `user` – Name of the user for connecting to a remote server. Default value: default. This user must have access to connect to the specified server. Access is configured in the users.xml file. For more information, see the section [Derechos de acceso](../../../operations/access-rights.md). - `password` – The password for connecting to a remote server (not masked). Default value: empty string. - `secure` - Use ssl para la conexión, por lo general también debe definir `port` = 9440. El servidor debe escuchar en `9440` y tener certificados correctos. - `compression` - Utilice la compresión de datos. Valor predeterminado: true. -When specifying replicas, one of the available replicas will be selected for each of the shards when reading. You can configure the algorithm for load balancing (the preference for which replica to access) – see the [load\_balancing](../../../operations/settings/settings.md#settings-load_balancing) configuración. +When specifying replicas, one of the available replicas will be selected for each of the shards when reading. You can configure the algorithm for load balancing (the preference for which replica to access) – see the [load_balancing](../../../operations/settings/settings.md#settings-load_balancing) configuración. Si no se establece la conexión con el servidor, habrá un intento de conectarse con un breve tiempo de espera. Si la conexión falla, se seleccionará la siguiente réplica, y así sucesivamente para todas las réplicas. Si el intento de conexión falló para todas las réplicas, el intento se repetirá de la misma manera, varias veces. Esto funciona a favor de la resiliencia, pero no proporciona una tolerancia completa a errores: un servidor remoto podría aceptar la conexión, pero podría no funcionar o funcionar mal. @@ -113,13 +113,13 @@ En segundo lugar, puede realizar INSERT en una tabla distribuida. En este caso, Cada fragmento puede tener un peso definido en el archivo de configuración. Por defecto, el peso es igual a uno. Los datos se distribuyen entre fragmentos en la cantidad proporcional al peso del fragmento. Por ejemplo, si hay dos fragmentos y el primero tiene un peso de 9 mientras que el segundo tiene un peso de 10, el primero se enviará 9 / 19 partes de las filas, y el segundo se enviará 10 / 19. -Cada fragmento puede tener el ‘internal\_replication’ parámetro definido en el archivo de configuración. +Cada fragmento puede tener el ‘internal_replication’ parámetro definido en el archivo de configuración. Si este parámetro se establece en ‘true’, la operación de escritura selecciona la primera réplica en buen estado y escribe datos en ella. Utilice esta alternativa si la tabla Distribuida “looks at” tablas replicadas. En otras palabras, si la tabla donde se escribirán los datos los replicará por sí misma. Si se establece en ‘false’ (el valor predeterminado), los datos se escriben en todas las réplicas. En esencia, esto significa que la tabla distribuida replica los datos en sí. Esto es peor que usar tablas replicadas, porque no se verifica la consistencia de las réplicas y, con el tiempo, contendrán datos ligeramente diferentes. -Para seleccionar el fragmento al que se envía una fila de datos, se analiza la expresión de fragmentación y su resto se toma de dividirlo por el peso total de los fragmentos. La fila se envía al fragmento que corresponde al medio intervalo de los restos de ‘prev\_weight’ a ‘prev\_weights + weight’, donde ‘prev\_weights’ es el peso total de los fragmentos con el número más pequeño, y ‘weight’ es el peso de este fragmento. Por ejemplo, si hay dos fragmentos, y el primero tiene un peso de 9 mientras que el segundo tiene un peso de 10, la fila se enviará al primer fragmento para los restos del rango \[0, 9), y al segundo para los restos del rango \[9, 19). +Para seleccionar el fragmento al que se envía una fila de datos, se analiza la expresión de fragmentación y su resto se toma de dividirlo por el peso total de los fragmentos. La fila se envía al fragmento que corresponde al medio intervalo de los restos de ‘prev_weight’ a ‘prev_weights + weight’, donde ‘prev_weights’ es el peso total de los fragmentos con el número más pequeño, y ‘weight’ es el peso de este fragmento. Por ejemplo, si hay dos fragmentos, y el primero tiene un peso de 9 mientras que el segundo tiene un peso de 10, la fila se enviará al primer fragmento para los restos del rango \[0, 9), y al segundo para los restos del rango \[9, 19). La expresión de fragmentación puede ser cualquier expresión de constantes y columnas de tabla que devuelva un entero. Por ejemplo, puede usar la expresión ‘rand()’ para la distribución aleatoria de datos, o ‘UserID’ para la distribución por el resto de dividir la ID del usuario (entonces los datos de un solo usuario residirán en un solo fragmento, lo que simplifica la ejecución de IN y JOIN por los usuarios). Si una de las columnas no se distribuye lo suficientemente uniformemente, puede envolverla en una función hash: intHash64(UserID) . @@ -132,11 +132,11 @@ Debería preocuparse por el esquema de fragmentación en los siguientes casos: - Se utilizan consultas que requieren unir datos (IN o JOIN) mediante una clave específica. Si esta clave fragmenta datos, puede usar IN local o JOIN en lugar de GLOBAL IN o GLOBAL JOIN, que es mucho más eficiente. - Se usa una gran cantidad de servidores (cientos o más) con una gran cantidad de consultas pequeñas (consultas de clientes individuales: sitios web, anunciantes o socios). Para que las pequeñas consultas no afecten a todo el clúster, tiene sentido ubicar datos para un solo cliente en un solo fragmento. Alternativamente, como lo hemos hecho en Yandex.Metrica, puede configurar sharding de dos niveles: divida todo el clúster en “layers”, donde una capa puede consistir en varios fragmentos. Los datos de un único cliente se encuentran en una sola capa, pero los fragmentos se pueden agregar a una capa según sea necesario y los datos se distribuyen aleatoriamente dentro de ellos. Las tablas distribuidas se crean para cada capa y se crea una única tabla distribuida compartida para consultas globales. -Los datos se escriben de forma asíncrona. Cuando se inserta en la tabla, el bloque de datos se acaba de escribir en el sistema de archivos local. Los datos se envían a los servidores remotos en segundo plano tan pronto como sea posible. El período de envío de datos está gestionado por el [Distributed\_directory\_monitor\_sleep\_time\_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms) y [Distributed\_directory\_monitor\_max\_sleep\_time\_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms) configuración. El `Distributed` el motor envía cada archivo con datos insertados por separado, pero puede habilitar el envío por lotes de archivos [distributed\_directory\_monitor\_batch\_inserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts) configuración. Esta configuración mejora el rendimiento del clúster al utilizar mejor los recursos de red y servidor local. Debe comprobar si los datos se envían correctamente comprobando la lista de archivos (datos en espera de ser enviados) en el directorio de la tabla: `/var/lib/clickhouse/data/database/table/`. +Los datos se escriben de forma asíncrona. Cuando se inserta en la tabla, el bloque de datos se acaba de escribir en el sistema de archivos local. Los datos se envían a los servidores remotos en segundo plano tan pronto como sea posible. El período de envío de datos está gestionado por el [Distributed_directory_monitor_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms) y [Distributed_directory_monitor_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms) configuración. El `Distributed` el motor envía cada archivo con datos insertados por separado, pero puede habilitar el envío por lotes de archivos [distributed_directory_monitor_batch_inserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts) configuración. Esta configuración mejora el rendimiento del clúster al utilizar mejor los recursos de red y servidor local. Debe comprobar si los datos se envían correctamente comprobando la lista de archivos (datos en espera de ser enviados) en el directorio de la tabla: `/var/lib/clickhouse/data/database/table/`. Si el servidor dejó de existir o tuvo un reinicio aproximado (por ejemplo, después de un error de dispositivo) después de un INSERT en una tabla distribuida, es posible que se pierdan los datos insertados. Si se detecta un elemento de datos dañado en el directorio de la tabla, se transfiere al ‘broken’ subdirectorio y ya no se utiliza. -Cuando la opción max\_parallel\_replicas está habilitada, el procesamiento de consultas se paralela en todas las réplicas dentro de un solo fragmento. Para obtener más información, consulte la sección [max\_parallel\_replicas](../../../operations/settings/settings.md#settings-max_parallel_replicas). +Cuando la opción max_parallel_replicas está habilitada, el procesamiento de consultas se paralela en todas las réplicas dentro de un solo fragmento. Para obtener más información, consulte la sección [max_parallel_replicas](../../../operations/settings/settings.md#settings-max_parallel_replicas). ## Virtual Columnas {#virtual-columns} diff --git a/docs/es/engines/table-engines/special/external-data.md b/docs/es/engines/table-engines/special/external-data.md index ce64b375ba2..f2ce4abbb0f 100644 --- a/docs/es/engines/table-engines/special/external-data.md +++ b/docs/es/engines/table-engines/special/external-data.md @@ -27,10 +27,10 @@ Puede tener varias secciones como esta, para el número de tablas que se transmi **–file** – Path to the file with the table dump, or -, which refers to stdin. Solo se puede recuperar una sola tabla de stdin. -Los siguientes parámetros son opcionales: **–name**– Name of the table. If omitted, \_data is used. +Los siguientes parámetros son opcionales: **–name**– Name of the table. If omitted, _data is used. **–format** – Data format in the file. If omitted, TabSeparated is used. -Se requiere uno de los siguientes parámetros:**–types** – A list of comma-separated column types. For example: `UInt64,String`. The columns will be named \_1, \_2, … +Se requiere uno de los siguientes parámetros:**–types** – A list of comma-separated column types. For example: `UInt64,String`. The columns will be named _1, _2, … **–structure**– The table structure in the format`UserID UInt64`, `URL String`. Define los nombres y tipos de columna. Los archivos especificados en ‘file’ se analizará mediante el formato especificado en ‘format’ utilizando los tipos de datos especificados en ‘types’ o ‘structure’. La mesa será cargado en el servidor y accesibles, como una tabla temporal con el nombre de ‘name’. @@ -48,7 +48,7 @@ $ cat /etc/passwd | sed 's/:/\t/g' | clickhouse-client --query="SELECT shell, co /bin/sync 1 ``` -Cuando se utiliza la interfaz HTTP, los datos externos se pasan en el formato multipart/form-data. Cada tabla se transmite como un archivo separado. El nombre de la tabla se toma del nombre del archivo. El ‘query\_string’ se pasa los parámetros ‘name\_format’, ‘name\_types’, y ‘name\_structure’, donde ‘name’ es el nombre de la tabla a la que corresponden estos parámetros. El significado de los parámetros es el mismo que cuando se usa el cliente de línea de comandos. +Cuando se utiliza la interfaz HTTP, los datos externos se pasan en el formato multipart/form-data. Cada tabla se transmite como un archivo separado. El nombre de la tabla se toma del nombre del archivo. El ‘query_string’ se pasa los parámetros ‘name_format’, ‘name_types’, y ‘name_structure’, donde ‘name’ es el nombre de la tabla a la que corresponden estos parámetros. El significado de los parámetros es el mismo que cuando se usa el cliente de línea de comandos. Ejemplo: diff --git a/docs/es/engines/table-engines/special/join.md b/docs/es/engines/table-engines/special/join.md index a553a0d24ef..83e21b7c8cc 100644 --- a/docs/es/engines/table-engines/special/join.md +++ b/docs/es/engines/table-engines/special/join.md @@ -95,8 +95,8 @@ Al crear una tabla, se aplican los siguientes valores: - [Sistema abierto.](../../../operations/settings/settings.md#join_use_nulls) - [Método de codificación de datos:](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) - [Método de codificación de datos:](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) -- [join\_overflow\_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) -- [join\_any\_take\_last\_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) +- [join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) +- [join_any_take_last_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) El `Join`-las tablas del motor no se pueden usar en `GLOBAL JOIN` operación. diff --git a/docs/es/engines/table-engines/special/merge.md b/docs/es/engines/table-engines/special/merge.md index 0c19fd7e2b0..6ed2c272914 100644 --- a/docs/es/engines/table-engines/special/merge.md +++ b/docs/es/engines/table-engines/special/merge.md @@ -31,7 +31,7 @@ La forma típica de usar el `Merge` para trabajar con un gran número de `TinyLo Ejemplo 2: -Digamos que tiene una tabla antigua (WatchLog\_old) y decidió cambiar la partición sin mover datos a una nueva tabla (WatchLog\_new) y necesita ver datos de ambas tablas. +Digamos que tiene una tabla antigua (WatchLog_old) y decidió cambiar la partición sin mover datos a una nueva tabla (WatchLog_new) y necesita ver datos de ambas tablas. ``` sql CREATE TABLE WatchLog_old(date Date, UserId Int64, EventType String, Cnt UInt64) diff --git a/docs/es/faq/general.md b/docs/es/faq/general.md index dc6ba5b5915..f8446e99152 100644 --- a/docs/es/faq/general.md +++ b/docs/es/faq/general.md @@ -17,7 +17,7 @@ La mayoría de las implementaciones de MapReduce le permiten ejecutar código ar ## ¿Qué sucede si tengo un problema con las codificaciones al usar Oracle a través de ODBC? {#oracle-odbc-encodings} -Si utiliza Oracle a través del controlador ODBC como fuente de diccionarios externos, debe establecer el valor `NLS_LANG` variable de entorno en `/etc/default/clickhouse`. Para obtener más información, consulte [Oracle NLS\_LANG Preguntas frecuentes](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html). +Si utiliza Oracle a través del controlador ODBC como fuente de diccionarios externos, debe establecer el valor `NLS_LANG` variable de entorno en `/etc/default/clickhouse`. Para obtener más información, consulte [Oracle NLS_LANG Preguntas frecuentes](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html). **Ejemplo** diff --git a/docs/es/getting-started/example-datasets/amplab-benchmark.md b/docs/es/getting-started/example-datasets/amplab-benchmark.md index c8fa77953cb..066bf036266 100644 --- a/docs/es/getting-started/example-datasets/amplab-benchmark.md +++ b/docs/es/getting-started/example-datasets/amplab-benchmark.md @@ -9,7 +9,7 @@ toc_title: Referencia de Big Data de AMPLab Ver https://amplab.cs.berkeley.edu/benchmark/ -Regístrese para obtener una cuenta gratuita en https://aws.amazon.com. Requiere una tarjeta de crédito, correo electrónico y número de teléfono. Obtenga una nueva clave de acceso en https://console.aws.amazon.com/iam/home?nc2=h\_m\_sc\#security\_credential +Regístrese para obtener una cuenta gratuita en https://aws.amazon.com. Requiere una tarjeta de crédito, correo electrónico y número de teléfono. Obtenga una nueva clave de acceso en https://console.aws.amazon.com/iam/home?nc2=h_m_sc#security_credential Ejecute lo siguiente en la consola: diff --git a/docs/es/getting-started/example-datasets/metrica.md b/docs/es/getting-started/example-datasets/metrica.md index e1c6a2a361e..b99346eda29 100644 --- a/docs/es/getting-started/example-datasets/metrica.md +++ b/docs/es/getting-started/example-datasets/metrica.md @@ -9,7 +9,7 @@ toc_title: El Yandex.Metrica Datos El conjunto de datos consta de dos tablas que contienen datos anónimos sobre los hits (`hits_v1`) y visitas (`visits_v1`) el Yandex.Métrica. Puedes leer más sobre Yandex.Metrica en [Historial de ClickHouse](../../introduction/history.md) apartado. -El conjunto de datos consta de dos tablas, cualquiera de ellas se puede descargar como `tsv.xz` o como particiones preparadas. Además, una versión extendida de la `hits` La tabla que contiene 100 millones de filas está disponible como TSV en https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits\_100m\_obfuscated\_v1.tsv.xz y como particiones preparadas en https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits\_100m\_obfuscated\_v1.tar.xz. +El conjunto de datos consta de dos tablas, cualquiera de ellas se puede descargar como `tsv.xz` o como particiones preparadas. Además, una versión extendida de la `hits` La tabla que contiene 100 millones de filas está disponible como TSV en https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits_100m_obfuscated_v1.tsv.xz y como particiones preparadas en https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits_100m_obfuscated_v1.tar.xz. ## Obtención de tablas a partir de particiones preparadas {#obtaining-tables-from-prepared-partitions} diff --git a/docs/es/getting-started/example-datasets/nyc-taxi.md b/docs/es/getting-started/example-datasets/nyc-taxi.md index e7dfe0898fe..4a2bae83a0a 100644 --- a/docs/es/getting-started/example-datasets/nyc-taxi.md +++ b/docs/es/getting-started/example-datasets/nyc-taxi.md @@ -197,7 +197,7 @@ Los datos de esta tabla utilizan 142 GB. (Importar datos directamente desde Postgres también es posible usando `COPY ... TO PROGRAM`.) -Unfortunately, all the fields associated with the weather (precipitation…average\_wind\_speed) were filled with NULL. Because of this, we will remove them from the final data set. +Unfortunately, all the fields associated with the weather (precipitation…average_wind_speed) were filled with NULL. Because of this, we will remove them from the final data set. Para empezar, crearemos una tabla en un único servidor. Posteriormente haremos la mesa distribuida. diff --git a/docs/es/getting-started/install.md b/docs/es/getting-started/install.md index 89b5735f192..092ef47b2f7 100644 --- a/docs/es/getting-started/install.md +++ b/docs/es/getting-started/install.md @@ -9,9 +9,9 @@ toc_title: "Instalaci\xF3n" ## Requisitos del sistema {#system-requirements} -ClickHouse puede ejecutarse en cualquier Linux, FreeBSD o Mac OS X con arquitectura de CPU x86\_64, AArch64 o PowerPC64LE. +ClickHouse puede ejecutarse en cualquier Linux, FreeBSD o Mac OS X con arquitectura de CPU x86_64, AArch64 o PowerPC64LE. -Los binarios oficiales preconstruidos generalmente se compilan para x86\_64 y aprovechan el conjunto de instrucciones SSE 4.2, por lo que, a menos que se indique lo contrario, el uso de la CPU que lo admite se convierte en un requisito adicional del sistema. Aquí está el comando para verificar si la CPU actual tiene soporte para SSE 4.2: +Los binarios oficiales preconstruidos generalmente se compilan para x86_64 y aprovechan el conjunto de instrucciones SSE 4.2, por lo que, a menos que se indique lo contrario, el uso de la CPU que lo admite se convierte en un requisito adicional del sistema. Aquí está el comando para verificar si la CPU actual tiene soporte para SSE 4.2: ``` bash $ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not supported" diff --git a/docs/es/index.md b/docs/es/index.md index 45c41a2c7f9..c76fe32e33b 100644 --- a/docs/es/index.md +++ b/docs/es/index.md @@ -13,10 +13,10 @@ En un DBMS “normal”, orientado a filas, los datos se almacenan en este orden | Fila | Argumento | JavaEnable | Titular | GoodEvent | EventTime | |------|-------------|------------|---------------------------|-----------|---------------------| -| \#0 | 89354350662 | 1 | Relaciones con inversores | 1 | 2016-05-18 05:19:20 | -| \#1 | 90329509958 | 0 | Contáctenos | 1 | 2016-05-18 08:10:20 | -| \#2 | 89953706054 | 1 | Mision | 1 | 2016-05-18 07:38:00 | -| \#N | … | … | … | … | … | +| #0 | 89354350662 | 1 | Relaciones con inversores | 1 | 2016-05-18 05:19:20 | +| #1 | 90329509958 | 0 | Contáctenos | 1 | 2016-05-18 08:10:20 | +| #2 | 89953706054 | 1 | Mision | 1 | 2016-05-18 07:38:00 | +| #N | … | … | … | … | … | En otras palabras, todos los valores relacionados con una fila se almacenan físicamente uno junto al otro. @@ -24,7 +24,7 @@ Ejemplos de un DBMS orientado a filas son MySQL, Postgres y MS SQL Server. En un DBMS orientado a columnas, los datos se almacenan así: -| Fila: | \#0 | \#1 | \#2 | \#N | +| Fila: | #0 | #1 | #2 | #N | |-------------|---------------------------|---------------------|---------------------|-----| | Argumento: | 89354350662 | 90329509958 | 89953706054 | … | | JavaEnable: | 1 | 0 | 1 | … | diff --git a/docs/es/interfaces/formats.md b/docs/es/interfaces/formats.md index 1af53a1ab3a..03c1873d306 100644 --- a/docs/es/interfaces/formats.md +++ b/docs/es/interfaces/formats.md @@ -201,7 +201,7 @@ Configuración `format_template_resultset` especifica la ruta al archivo, que co - `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 . +- `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 @@ -352,21 +352,21 @@ El análisis permite la presencia del campo adicional `tskv` sin el signo igual 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). +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. +\*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) +[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)). +`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`. @@ -451,12 +451,12 @@ SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase WITH TOTA } ``` -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. +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 . +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). @@ -543,7 +543,7 @@ ClickHouse ignora los espacios entre los elementos y las comas después de los o ClickHouse sustituye los valores omitidos por los valores predeterminados para el [tipos de datos](../sql-reference/data-types/index.md). -Si `DEFAULT expr` se especifica, ClickHouse utiliza diferentes reglas de sustitución dependiendo de la [Entrada\_format\_defaults\_for\_omitted\_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) configuración. +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: @@ -586,7 +586,7 @@ A diferencia de la [JSON](#json) formato, no hay sustitución de secuencias UTF- ### Uso de estructuras anidadas {#jsoneachrow-nested} -Si tienes una mesa con [Anidar](../sql-reference/data-types/nested-data-structures/nested.md) columnas de tipo de datos, puede insertar datos JSON con la misma estructura. Habilite esta función con el [Entrada\_format\_import\_nested\_json](../operations/settings/settings.md#settings-input_format_import_nested_json) configuración. +Si tienes una mesa con [Anidar](../sql-reference/data-types/nested-data-structures/nested.md) columnas de tipo de datos, puede insertar datos JSON con la misma estructura. Habilite esta función con el [Entrada_format_import_nested_json](../operations/settings/settings.md#settings-input_format_import_nested_json) configuración. Por ejemplo, considere la siguiente tabla: @@ -600,7 +600,7 @@ Como se puede ver en el `Nested` descripción del tipo de datos, ClickHouse trat 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). +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 { @@ -782,7 +782,7 @@ The minimum set of characters that you need to escape when passing data in Value 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. +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} @@ -1198,13 +1198,13 @@ puede contener una ruta absoluta o una ruta relativa al directorio actual en el Si utiliza el cliente en el [modo por lotes](../interfaces/cli.md#cli_usage), la ruta de acceso al esquema debe ser relativa por razones de seguridad. Si introduce o emite datos a través del [Interfaz HTTP](../interfaces/http.md) el nombre de archivo especificado en el esquema de formato -debe estar ubicado en el directorio especificado en [format\_schema\_path](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-format_schema_path) +debe estar ubicado en el directorio especificado en [format_schema_path](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-format_schema_path) en la configuración del servidor. ## Salto de errores {#skippingerrors} -Algunos formatos como `CSV`, `TabSeparated`, `TSKV`, `JSONEachRow`, `Template`, `CustomSeparated` y `Protobuf` puede omitir la fila rota si se produjo un error de análisis y continuar el análisis desde el comienzo de la siguiente fila. Ver [Entrada\_format\_allow\_errors\_num](../operations/settings/settings.md#settings-input_format_allow_errors_num) y -[Entrada\_format\_allow\_errors\_ratio](../operations/settings/settings.md#settings-input_format_allow_errors_ratio) configuración. +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. diff --git a/docs/es/interfaces/http.md b/docs/es/interfaces/http.md index ebce0ec7a51..ab510a268e3 100644 --- a/docs/es/interfaces/http.md +++ b/docs/es/interfaces/http.md @@ -11,7 +11,7 @@ La interfaz HTTP le permite usar ClickHouse en cualquier plataforma desde cualqu De forma predeterminada, clickhouse-server escucha HTTP en el puerto 8123 (esto se puede cambiar en la configuración). -Si realiza una solicitud GET / sin parámetros, devuelve 200 códigos de respuesta y la cadena que definió en [http\_server\_default\_response](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-http_server_default_response) valor predeterminado “Ok.” (con un avance de línea al final) +Si realiza una solicitud GET / sin parámetros, devuelve 200 códigos de respuesta y la cadena que definió en [http_server_default_response](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-http_server_default_response) valor predeterminado “Ok.” (con un avance de línea al final) ``` bash $ curl 'http://localhost:8123/' @@ -147,12 +147,12 @@ $ echo 'DROP TABLE t' | curl 'http://localhost:8123/' --data-binary @- Para las solicitudes correctas que no devuelven una tabla de datos, se devuelve un cuerpo de respuesta vacío. -Puede utilizar el formato interno de compresión ClickHouse al transmitir datos. Los datos comprimidos tienen un formato no estándar, y deberá usar el `clickhouse-compressor` programa para trabajar con él (se instala con el `clickhouse-client` paquete). Para aumentar la eficiencia de la inserción de datos, puede deshabilitar la verificación de suma de comprobación [http\_native\_compression\_disable\_checksumming\_on\_decompress](../operations/settings/settings.md#settings-http_native_compression_disable_checksumming_on_decompress) configuración. +Puede utilizar el formato interno de compresión ClickHouse al transmitir datos. Los datos comprimidos tienen un formato no estándar, y deberá usar el `clickhouse-compressor` programa para trabajar con él (se instala con el `clickhouse-client` paquete). Para aumentar la eficiencia de la inserción de datos, puede deshabilitar la verificación de suma de comprobación [http_native_compression_disable_checksumming_on_decompress](../operations/settings/settings.md#settings-http_native_compression_disable_checksumming_on_decompress) configuración. Si ha especificado `compress=1` en la URL, el servidor comprime los datos que le envía. Si ha especificado `decompress=1` en la dirección URL, el servidor descomprime los mismos datos que `POST` método. -También puede optar por utilizar [Compresión HTTP](https://en.wikipedia.org/wiki/HTTP_compression). Para enviar un `POST` solicitud, agregue el encabezado de solicitud `Content-Encoding: compression_method`. Para que ClickHouse comprima la respuesta, debe agregar `Accept-Encoding: compression_method`. Soporta ClickHouse `gzip`, `br`, y `deflate` [métodos de compresión](https://en.wikipedia.org/wiki/HTTP_compression#Content-Encoding_tokens). Para habilitar la compresión HTTP, debe usar ClickHouse [enable\_http\_compression](../operations/settings/settings.md#settings-enable_http_compression) configuración. Puede configurar el nivel de compresión de datos [http\_zlib\_compression\_level](#settings-http_zlib_compression_level) para todos los métodos de compresión. +También puede optar por utilizar [Compresión HTTP](https://en.wikipedia.org/wiki/HTTP_compression). Para enviar un `POST` solicitud, agregue el encabezado de solicitud `Content-Encoding: compression_method`. Para que ClickHouse comprima la respuesta, debe agregar `Accept-Encoding: compression_method`. Soporta ClickHouse `gzip`, `br`, y `deflate` [métodos de compresión](https://en.wikipedia.org/wiki/HTTP_compression#Content-Encoding_tokens). Para habilitar la compresión HTTP, debe usar ClickHouse [enable_http_compression](../operations/settings/settings.md#settings-enable_http_compression) configuración. Puede configurar el nivel de compresión de datos [http_zlib_compression_level](#settings-http_zlib_compression_level) para todos los métodos de compresión. Puede usar esto para reducir el tráfico de red al transmitir una gran cantidad de datos o para crear volcados que se comprimen inmediatamente. @@ -214,7 +214,7 @@ $ echo 'SELECT 1' | curl -H 'X-ClickHouse-User: user' -H 'X-ClickHouse-Key: pass ``` Si no se especifica el nombre de usuario, `default` se utiliza el nombre. Si no se especifica la contraseña, se utiliza la contraseña vacía. -También puede utilizar los parámetros de URL para especificar cualquier configuración para procesar una sola consulta o perfiles completos de configuración. Ejemplo:http://localhost:8123/?perfil=web&max\_rows\_to\_read=1000000000&consulta=SELECCIONA+1 +También puede utilizar los parámetros de URL para especificar cualquier configuración para procesar una sola consulta o perfiles completos de configuración. Ejemplo:http://localhost:8123/?perfil=web&max_rows_to_read=1000000000&consulta=SELECCIONA+1 Para obtener más información, consulte [Configuración](../operations/settings/index.md) apartado. @@ -236,7 +236,7 @@ Para obtener información sobre otros parámetros, consulte la sección “SET Del mismo modo, puede utilizar sesiones ClickHouse en el protocolo HTTP. Para hacer esto, debe agregar el `session_id` GET parámetro a la solicitud. Puede usar cualquier cadena como ID de sesión. De forma predeterminada, la sesión finaliza después de 60 segundos de inactividad. Para cambiar este tiempo de espera, modifique `default_session_timeout` configuración en la configuración del servidor, o `session_timeout` GET parámetro a la solicitud. Para comprobar el estado de la sesión, `session_check=1` parámetro. Solo se puede ejecutar una consulta a la vez en una sola sesión. -Puede recibir información sobre el progreso de una consulta en `X-ClickHouse-Progress` encabezados de respuesta. Para hacer esto, habilite [send\_progress\_in\_http\_headers](../operations/settings/settings.md#settings-send_progress_in_http_headers). Ejemplo de la secuencia de encabezado: +Puede recibir información sobre el progreso de una consulta en `X-ClickHouse-Progress` encabezados de respuesta. Para hacer esto, habilite [send_progress_in_http_headers](../operations/settings/settings.md#settings-send_progress_in_http_headers). Ejemplo de la secuencia de encabezado: ``` text X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128"} @@ -253,9 +253,9 @@ Posibles campos de encabezado: - `written_bytes` — Volume of data written in bytes. Las solicitudes en ejecución no se detienen automáticamente si se pierde la conexión HTTP. El análisis y el formato de datos se realizan en el lado del servidor, y el uso de la red puede ser ineficaz. -Opcional ‘query\_id’ parámetro se puede pasar como el ID de consulta (cualquier cadena). Para obtener más información, consulte la sección “Settings, replace\_running\_query”. +Opcional ‘query_id’ parámetro se puede pasar como el ID de consulta (cualquier cadena). Para obtener más información, consulte la sección “Settings, replace_running_query”. -Opcional ‘quota\_key’ parámetro se puede pasar como la clave de cuota (cualquier cadena). Para obtener más información, consulte la sección “Quotas”. +Opcional ‘quota_key’ parámetro se puede pasar como la clave de cuota (cualquier cadena). Para obtener más información, consulte la sección “Quotas”. La interfaz HTTP permite pasar datos externos (tablas temporales externas) para consultar. Para obtener más información, consulte la sección “External data for query processing”. @@ -379,9 +379,9 @@ Como puede ver en el ejemplo, si `` está configurado en la confi > `` contiene la parte de procesamiento principal. Ahora `` puede configurar ``, ``, ``, ``, ``, ``. > \> `` Actualmente soporta tres tipos: **Dirección de correo electrónico**, **Nombre de la red inalámbrica (SSID):**, **estática**. > \> -> \> `` - utilizar con el tipo predefined\_query\_handler, ejecuta la consulta cuando se llama al controlador. +> \> `` - utilizar con el tipo predefined_query_handler, ejecuta la consulta cuando se llama al controlador. > \> -> \> `` - utilizar con el tipo dynamic\_query\_handler, extrae y ejecuta el valor correspondiente al `` valor en parámetros de solicitud HTTP. +> \> `` - utilizar con el tipo dynamic_query_handler, extrae y ejecuta el valor correspondiente al `` valor en parámetros de solicitud HTTP. > \> > \> `` - uso con tipo estático, código de estado de respuesta. > \> @@ -393,7 +393,7 @@ A continuación están los métodos de configuración para los diferentes `` admite la configuración de valores Settings y query\_params. Puede configurar `` en el tipo de ``. +`` admite la configuración de valores Settings y query_params. Puede configurar `` en el tipo de ``. `` valor es una consulta predefinida de ``, que es ejecutado por ClickHouse cuando se hace coincidir una solicitud HTTP y se devuelve el resultado de la consulta. Es una configuración imprescindible. @@ -434,7 +434,7 @@ En ``, consulta se escribe en forma de param de la solici ClickHouse extrae y ejecuta el valor correspondiente al `` valor en la url de la solicitud HTTP. El valor predeterminado de `` ser `/query` . Es una configuración opcional. Si no hay una definición en el archivo de configuración, el parámetro no se pasa. -Para experimentar con esta funcionalidad, el ejemplo define los valores de max\_threads y max\_alter\_threads y consulta si la configuración se estableció correctamente. +Para experimentar con esta funcionalidad, el ejemplo define los valores de max_threads y max_alter_threads y consulta si la configuración se estableció correctamente. Ejemplo: @@ -459,7 +459,7 @@ max_alter_threads 2 ## estática {#static} -`` puede volver [Content\_type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type), [estatus](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status) y response\_content. response\_content puede devolver el contenido especificado +`` puede volver [Content_type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type), [estatus](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status) y response_content. response_content puede devolver el contenido especificado Ejemplo: diff --git a/docs/es/interfaces/mysql.md b/docs/es/interfaces/mysql.md index 796bfbcf1a4..a5124c61dd5 100644 --- a/docs/es/interfaces/mysql.md +++ b/docs/es/interfaces/mysql.md @@ -7,7 +7,7 @@ toc_title: Interfaz MySQL # Interfaz MySQL {#mysql-interface} -ClickHouse soporta el protocolo de cable MySQL. Puede ser habilitado por [mysql\_port](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-mysql_port) configuración en el archivo de configuración: +ClickHouse soporta el protocolo de cable MySQL. Puede ser habilitado por [mysql_port](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-mysql_port) configuración en el archivo de configuración: ``` xml 9004 diff --git a/docs/es/interfaces/third-party/client-libraries.md b/docs/es/interfaces/third-party/client-libraries.md index 9dbbe0a0022..818bdbbc6f0 100644 --- a/docs/es/interfaces/third-party/client-libraries.md +++ b/docs/es/interfaces/third-party/client-libraries.md @@ -9,7 +9,7 @@ toc_title: Client Libraries Yandex does **not** maintain the libraries listed below and haven’t done any extensive testing to ensure their quality. - Python - - [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm) + - [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm) - [clickhouse-driver](https://github.com/mymarilyn/clickhouse-driver) - [clickhouse-client](https://github.com/yurial/clickhouse-client) - [aiochclient](https://github.com/maximdanilchenko/aiochclient) @@ -45,7 +45,7 @@ toc_title: Client Libraries - [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client) - Kotlin - [AORM](https://github.com/TanVD/AORM) -- C\# +- C# - [Octonica.ClickHouseClient](https://github.com/Octonica/ClickHouseClient) - [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net) - [ClickHouse.Client](https://github.com/DarkWanderer/ClickHouse.Client) diff --git a/docs/es/interfaces/third-party/integrations.md b/docs/es/interfaces/third-party/integrations.md index e752725d727..7588bef0230 100644 --- a/docs/es/interfaces/third-party/integrations.md +++ b/docs/es/interfaces/third-party/integrations.md @@ -17,15 +17,15 @@ toc_title: Integrations - [clickhouse-mysql-data-reader](https://github.com/Altinity/clickhouse-mysql-data-reader) - [horgh-replicator](https://github.com/larsnovikov/horgh-replicator) - [PostgreSQL](https://www.postgresql.org) - - [clickhousedb\_fdw](https://github.com/Percona-Lab/clickhousedb_fdw) - - [infi.clickhouse\_fdw](https://github.com/Infinidat/infi.clickhouse_fdw) (uses [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) + - [clickhousedb_fdw](https://github.com/Percona-Lab/clickhousedb_fdw) + - [infi.clickhouse_fdw](https://github.com/Infinidat/infi.clickhouse_fdw) (uses [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - [pg2ch](https://github.com/mkabilov/pg2ch) - - [clickhouse\_fdw](https://github.com/adjust/clickhouse_fdw) + - [clickhouse_fdw](https://github.com/adjust/clickhouse_fdw) - [MSSQL](https://en.wikipedia.org/wiki/Microsoft_SQL_Server) - [ClickHouseMigrator](https://github.com/zlzforever/ClickHouseMigrator) - Message queues - [Kafka](https://kafka.apache.org) - - [clickhouse\_sinker](https://github.com/housepower/clickhouse_sinker) (uses [Go client](https://github.com/ClickHouse/clickhouse-go/)) + - [clickhouse_sinker](https://github.com/housepower/clickhouse_sinker) (uses [Go client](https://github.com/ClickHouse/clickhouse-go/)) - [stream-loader-clickhouse](https://github.com/adform/stream-loader) - Stream processing - [Flink](https://flink.apache.org) @@ -49,12 +49,12 @@ toc_title: Integrations - [Grafana](https://grafana.com/) - [clickhouse-grafana](https://github.com/Vertamedia/clickhouse-grafana) - [Prometheus](https://prometheus.io/) - - [clickhouse\_exporter](https://github.com/f1yegor/clickhouse_exporter) + - [clickhouse_exporter](https://github.com/f1yegor/clickhouse_exporter) - [PromHouse](https://github.com/Percona-Lab/PromHouse) - - [clickhouse\_exporter](https://github.com/hot-wifi/clickhouse_exporter) (uses [Go client](https://github.com/kshvakov/clickhouse/)) + - [clickhouse_exporter](https://github.com/hot-wifi/clickhouse_exporter) (uses [Go client](https://github.com/kshvakov/clickhouse/)) - [Nagios](https://www.nagios.org/) - - [check\_clickhouse](https://github.com/exogroup/check_clickhouse/) - - [check\_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py) + - [check_clickhouse](https://github.com/exogroup/check_clickhouse/) + - [check_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py) - [Zabbix](https://www.zabbix.com) - [clickhouse-zabbix-template](https://github.com/Altinity/clickhouse-zabbix-template) - [Sematext](https://sematext.com/) @@ -74,7 +74,7 @@ toc_title: Integrations - Python - [SQLAlchemy](https://www.sqlalchemy.org) - - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (uses [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) + - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (uses [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - [pandas](https://pandas.pydata.org) - [pandahouse](https://github.com/kszucs/pandahouse) - PHP @@ -89,7 +89,7 @@ toc_title: Integrations - Scala - [Akka](https://akka.io) - [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client) -- C\# +- C# - [ADO.NET](https://docs.microsoft.com/en-us/dotnet/framework/data/adonet/ado-net-overview) - [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net) - [ClickHouse.Client](https://github.com/DarkWanderer/ClickHouse.Client) @@ -97,7 +97,7 @@ toc_title: Integrations - [ClickHouse.Net.Migrations](https://github.com/ilyabreev/ClickHouse.Net.Migrations) - Elixir - [Ecto](https://github.com/elixir-ecto/ecto) - - [clickhouse\_ecto](https://github.com/appodeal/clickhouse_ecto) + - [clickhouse_ecto](https://github.com/appodeal/clickhouse_ecto) - Ruby - [Ruby on Rails](https://rubyonrails.org/) - [activecube](https://github.com/bitquery/activecube) diff --git a/docs/es/operations/access-rights.md b/docs/es/operations/access-rights.md index 5f00dc901d8..6c777d9f081 100644 --- a/docs/es/operations/access-rights.md +++ b/docs/es/operations/access-rights.md @@ -134,10 +134,10 @@ Consultas de gestión: - Configure un directorio para el almacenamiento de configuraciones. - ClickHouse almacena las configuraciones de entidades de acceso en la carpeta [access\_control\_path](server-configuration-parameters/settings.md#access_control_path) parámetro de configuración del servidor. + ClickHouse almacena las configuraciones de entidades de acceso en la carpeta [access_control_path](server-configuration-parameters/settings.md#access_control_path) parámetro de configuración del servidor. - Habilite el control de acceso controlado por SQL y la administración de cuentas para al menos una cuenta de usuario. - De forma predeterminada, el control de acceso controlado por SQL y la administración de cuentas se activan para todos los usuarios. Debe configurar al menos un usuario en el `users.xml` archivo de configuración y asigne 1 al [access\_management](settings/settings-users.md#access_management-user-setting) configuración. + De forma predeterminada, el control de acceso controlado por SQL y la administración de cuentas se activan para todos los usuarios. Debe configurar al menos un usuario en el `users.xml` archivo de configuración y asigne 1 al [access_management](settings/settings-users.md#access_management-user-setting) configuración. [Artículo Original](https://clickhouse.tech/docs/en/operations/access_rights/) diff --git a/docs/es/operations/configuration-files.md b/docs/es/operations/configuration-files.md index 578d7b3f9ff..d9aa8567868 100644 --- a/docs/es/operations/configuration-files.md +++ b/docs/es/operations/configuration-files.md @@ -20,7 +20,7 @@ Si `replace` se especifica, reemplaza todo el elemento por el especificado. Si `remove` se especifica, elimina el elemento. -La configuración también puede definir “substitutions”. Si un elemento tiene el `incl` atributo, la sustitución correspondiente del archivo se utilizará como el valor. De forma predeterminada, la ruta al archivo con sustituciones es `/etc/metrika.xml`. Esto se puede cambiar en el [include\_from](server-configuration-parameters/settings.md#server_configuration_parameters-include_from) elemento en la configuración del servidor. Los valores de sustitución se especifican en `/yandex/substitution_name` elementos en este archivo. Si una sustitución especificada en `incl` no existe, se registra en el registro. Para evitar que ClickHouse registre las sustituciones que faltan, especifique `optional="true"` atributo (por ejemplo, ajustes para [macro](server-configuration-parameters/settings.md)). +La configuración también puede definir “substitutions”. Si un elemento tiene el `incl` atributo, la sustitución correspondiente del archivo se utilizará como el valor. De forma predeterminada, la ruta al archivo con sustituciones es `/etc/metrika.xml`. Esto se puede cambiar en el [include_from](server-configuration-parameters/settings.md#server_configuration_parameters-include_from) elemento en la configuración del servidor. Los valores de sustitución se especifican en `/yandex/substitution_name` elementos en este archivo. Si una sustitución especificada en `incl` no existe, se registra en el registro. Para evitar que ClickHouse registre las sustituciones que faltan, especifique `optional="true"` atributo (por ejemplo, ajustes para [macro](server-configuration-parameters/settings.md)). Las sustituciones también se pueden realizar desde ZooKeeper. Para hacer esto, especifique el atributo `from_zk = "/path/to/node"`. El valor del elemento se sustituye por el contenido del nodo en `/path/to/node` en ZooKeeper. También puede colocar un subárbol XML completo en el nodo ZooKeeper y se insertará completamente en el elemento de origen. diff --git a/docs/es/operations/monitoring.md b/docs/es/operations/monitoring.md index 173d23aa041..19912d23f3b 100644 --- a/docs/es/operations/monitoring.md +++ b/docs/es/operations/monitoring.md @@ -35,7 +35,7 @@ ClickHouse recoge: - Diferentes métricas de cómo el servidor utiliza recursos computacionales. - Estadísticas comunes sobre el procesamiento de consultas. -Puede encontrar métricas en el [sistema.métricas](../operations/system-tables.md#system_tables-metrics), [sistema.evento](../operations/system-tables.md#system_tables-events), y [sistema.asynchronous\_metrics](../operations/system-tables.md#system_tables-asynchronous_metrics) tabla. +Puede encontrar métricas en el [sistema.métricas](../operations/system-tables.md#system_tables-metrics), [sistema.evento](../operations/system-tables.md#system_tables-events), y [sistema.asynchronous_metrics](../operations/system-tables.md#system_tables-asynchronous_metrics) tabla. Puede configurar ClickHouse para exportar métricas a [Grafito](https://github.com/graphite-project). Ver el [Sección de grafito](server-configuration-parameters/settings.md#server_configuration_parameters-graphite) en el archivo de configuración del servidor ClickHouse. Antes de configurar la exportación de métricas, debe configurar Graphite siguiendo sus [guiar](https://graphite.readthedocs.io/en/latest/install.html). @@ -43,4 +43,4 @@ Puede configurar ClickHouse para exportar métricas a [Prometeo](https://prometh Además, puede supervisar la disponibilidad del servidor a través de la API HTTP. Enviar el `HTTP GET` solicitud de `/ping`. Si el servidor está disponible, responde con `200 OK`. -Para supervisar servidores en una configuración de clúster, debe establecer [max\_replica\_delay\_for\_distributed\_queries](settings/settings.md#settings-max_replica_delay_for_distributed_queries) parámetro y utilizar el recurso HTTP `/replicas_status`. Una solicitud para `/replicas_status` devoluciones `200 OK` si la réplica está disponible y no se retrasa detrás de las otras réplicas. Si una réplica se retrasa, devuelve `503 HTTP_SERVICE_UNAVAILABLE` con información sobre la brecha. +Para supervisar servidores en una configuración de clúster, debe establecer [max_replica_delay_for_distributed_queries](settings/settings.md#settings-max_replica_delay_for_distributed_queries) parámetro y utilizar el recurso HTTP `/replicas_status`. Una solicitud para `/replicas_status` devoluciones `200 OK` si la réplica está disponible y no se retrasa detrás de las otras réplicas. Si una réplica se retrasa, devuelve `503 HTTP_SERVICE_UNAVAILABLE` con información sobre la brecha. diff --git a/docs/es/operations/optimizing-performance/sampling-query-profiler.md b/docs/es/operations/optimizing-performance/sampling-query-profiler.md index 00ea86b6e8c..a474dde6af2 100644 --- a/docs/es/operations/optimizing-performance/sampling-query-profiler.md +++ b/docs/es/operations/optimizing-performance/sampling-query-profiler.md @@ -11,11 +11,11 @@ ClickHouse ejecuta el generador de perfiles de muestreo que permite analizar la Para usar el generador de perfiles: -- Configurar el [trace\_log](../server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) sección de la configuración del servidor. +- Configurar el [trace_log](../server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) sección de la configuración del servidor. - Esta sección configura la [trace\_log](../../operations/system-tables.md#system_tables-trace_log) tabla del sistema que contiene los resultados del funcionamiento del generador de perfiles. Está configurado de forma predeterminada. Recuerde que los datos de esta tabla solo son válidos para un servidor en ejecución. Después de reiniciar el servidor, ClickHouse no limpia la tabla y toda la dirección de memoria virtual almacenada puede dejar de ser válida. + Esta sección configura la [trace_log](../../operations/system-tables.md#system_tables-trace_log) tabla del sistema que contiene los resultados del funcionamiento del generador de perfiles. Está configurado de forma predeterminada. Recuerde que los datos de esta tabla solo son válidos para un servidor en ejecución. Después de reiniciar el servidor, ClickHouse no limpia la tabla y toda la dirección de memoria virtual almacenada puede dejar de ser válida. -- Configurar el [Los resultados de la prueba](../settings/settings.md#query_profiler_cpu_time_period_ns) o [query\_profiler\_real\_time\_period\_ns](../settings/settings.md#query_profiler_real_time_period_ns) configuración. Ambos ajustes se pueden utilizar simultáneamente. +- Configurar el [Los resultados de la prueba](../settings/settings.md#query_profiler_cpu_time_period_ns) o [query_profiler_real_time_period_ns](../settings/settings.md#query_profiler_real_time_period_ns) configuración. Ambos ajustes se pueden utilizar simultáneamente. Estas opciones le permiten configurar temporizadores del generador de perfiles. Como estos son los ajustes de sesión, puede obtener diferentes frecuencias de muestreo para todo el servidor, usuarios individuales o perfiles de usuario, para su sesión interactiva y para cada consulta individual. @@ -25,7 +25,7 @@ Para analizar el `trace_log` tabla del sistema: - Instale el `clickhouse-common-static-dbg` paquete. Ver [Instalar desde paquetes DEB](../../getting-started/install.md#install-from-deb-packages). -- Permitir funciones de introspección [allow\_introspection\_functions](../settings/settings.md#settings-allow_introspection_functions) configuración. +- Permitir funciones de introspección [allow_introspection_functions](../settings/settings.md#settings-allow_introspection_functions) configuración. Por razones de seguridad, las funciones de introspección están deshabilitadas de forma predeterminada. diff --git a/docs/es/operations/requirements.md b/docs/es/operations/requirements.md index 29ed7a655a7..d6f0f25cf21 100644 --- a/docs/es/operations/requirements.md +++ b/docs/es/operations/requirements.md @@ -9,7 +9,7 @@ toc_title: Requisito ## CPU {#cpu} -Para la instalación desde paquetes deb precompilados, utilice una CPU con arquitectura x86\_64 y soporte para las instrucciones de SSE 4.2. Para ejecutar ClickHouse con procesadores que no admiten SSE 4.2 o tienen arquitectura AArch64 o PowerPC64LE, debe compilar ClickHouse a partir de fuentes. +Para la instalación desde paquetes deb precompilados, utilice una CPU con arquitectura x86_64 y soporte para las instrucciones de SSE 4.2. Para ejecutar ClickHouse con procesadores que no admiten SSE 4.2 o tienen arquitectura AArch64 o PowerPC64LE, debe compilar ClickHouse a partir de fuentes. ClickHouse implementa el procesamiento de datos paralelo y utiliza todos los recursos de hardware disponibles. Al elegir un procesador, tenga en cuenta que ClickHouse funciona de manera más eficiente en configuraciones con un gran número de núcleos pero con una velocidad de reloj más baja que en configuraciones con menos núcleos y una velocidad de reloj más alta. Por ejemplo, 16 núcleos con 2600 MHz es preferible a 8 núcleos con 3600 MHz. diff --git a/docs/es/operations/server-configuration-parameters/settings.md b/docs/es/operations/server-configuration-parameters/settings.md index 29f726ee635..86264ed0440 100644 --- a/docs/es/operations/server-configuration-parameters/settings.md +++ b/docs/es/operations/server-configuration-parameters/settings.md @@ -7,7 +7,7 @@ toc_title: "Configuraci\xF3n del servidor" # Configuración del servidor {#server-settings} -## builtin\_dictionaries\_reload\_interval {#builtin-dictionaries-reload-interval} +## builtin_dictionaries_reload_interval {#builtin-dictionaries-reload-interval} El intervalo en segundos antes de volver a cargar los diccionarios integrados. @@ -68,7 +68,7 @@ Si no se cumplen condiciones para un elemento de datos, ClickHouse utiliza el `l ``` -## default\_database {#default-database} +## default_database {#default-database} La base de datos predeterminada. @@ -80,7 +80,7 @@ Para obtener una lista de bases de datos, [SHOW DATABASES](../../sql-reference/s default ``` -## default\_profile {#default-profile} +## default_profile {#default-profile} Perfil de configuración predeterminado. @@ -92,7 +92,7 @@ Los perfiles de configuración se encuentran en el archivo especificado en el pa default ``` -## Diccionarios\_config {#server_configuration_parameters-dictionaries_config} +## Diccionarios_config {#server_configuration_parameters-dictionaries_config} La ruta de acceso al archivo de configuración para diccionarios externos. @@ -109,7 +109,7 @@ Ver también “[Diccionarios externos](../../sql-reference/dictionaries/externa *_dictionary.xml ``` -## Diccionarios\_lazy\_load {#server_configuration_parameters-dictionaries_lazy_load} +## Diccionarios_lazy_load {#server_configuration_parameters-dictionaries_lazy_load} La carga perezosa de los diccionarios. @@ -125,7 +125,7 @@ El valor predeterminado es `true`. true ``` -## format\_schema\_path {#server_configuration_parameters-format_schema_path} +## format_schema_path {#server_configuration_parameters-format_schema_path} La ruta de acceso al directorio con los esquemas para los datos de entrada, como los esquemas [CapnProto](../../interfaces/formats.md#capnproto) formato. @@ -146,11 +146,11 @@ Configuración: - port – The port on the Graphite server. - interval – The interval for sending, in seconds. - timeout – The timeout for sending data, in seconds. -- root\_path – Prefix for keys. +- root_path – Prefix for keys. - metrics – Sending data from the [sistema.métricas](../../operations/system-tables.md#system_tables-metrics) tabla. - events – Sending deltas data accumulated for the time period from the [sistema.evento](../../operations/system-tables.md#system_tables-events) tabla. -- events\_cumulative – Sending cumulative data from the [sistema.evento](../../operations/system-tables.md#system_tables-events) tabla. -- asynchronous\_metrics – Sending data from the [sistema.asynchronous\_metrics](../../operations/system-tables.md#system_tables-asynchronous_metrics) tabla. +- events_cumulative – Sending cumulative data from the [sistema.evento](../../operations/system-tables.md#system_tables-events) tabla. +- asynchronous_metrics – Sending data from the [sistema.asynchronous_metrics](../../operations/system-tables.md#system_tables-asynchronous_metrics) tabla. Puede configurar múltiples `` clausula. Por ejemplo, puede usar esto para enviar datos diferentes a intervalos diferentes. @@ -170,7 +170,7 @@ Puede configurar múltiples `` clausula. Por ejemplo, puede usar esto ``` -## graphite\_rollup {#server_configuration_parameters-graphite-rollup} +## graphite_rollup {#server_configuration_parameters-graphite-rollup} Ajustes para reducir los datos de grafito. @@ -198,7 +198,7 @@ Para obtener más información, consulte [GraphiteMergeTree](../../engines/table ``` -## http\_port/https\_port {#http-porthttps-port} +## http_port/https_port {#http-porthttps-port} El puerto para conectarse al servidor a través de HTTP(s). @@ -212,7 +212,7 @@ Si `http_port` se especifica, la configuración de OpenSSL se ignora incluso si 9999 ``` -## http\_server\_default\_response {#server_configuration_parameters-http_server_default_response} +## http_server_default_response {#server_configuration_parameters-http_server_default_response} La página que se muestra de forma predeterminada al acceder al servidor HTTP de ClickHouse. El valor predeterminado es “Ok.” (con un avance de línea al final) @@ -227,7 +227,7 @@ Abrir `https://tabix.io/` al acceder `http://localhost: http_port`. ``` -## include\_from {#server_configuration_parameters-include_from} +## include_from {#server_configuration_parameters-include_from} La ruta al archivo con sustituciones. @@ -263,7 +263,7 @@ Si se omite, se define de la misma manera que el `hostname-f` comando. example.yandex.ru ``` -## interserver\_http\_credentials {#server-settings-interserver-http-credentials} +## interserver_http_credentials {#server-settings-interserver-http-credentials} El nombre de usuario y la contraseña utilizados para [replicación](../../engines/table-engines/mergetree-family/replication.md) con los motores Replicated\*. Estas credenciales sólo se utilizan para la comunicación entre réplicas y no están relacionadas con las credenciales de los clientes de ClickHouse. El servidor está comprobando estas credenciales para conectar réplicas y utiliza las mismas credenciales cuando se conecta a otras réplicas. Por lo tanto, estas credenciales deben establecerse igual para todas las réplicas de un clúster. De forma predeterminada, la autenticación no se utiliza. @@ -282,7 +282,7 @@ Esta sección contiene los siguientes parámetros: ``` -## keep\_alive\_timeout {#keep-alive-timeout} +## keep_alive_timeout {#keep-alive-timeout} El número de segundos que ClickHouse espera las solicitudes entrantes antes de cerrar la conexión. El valor predeterminado es de 3 segundos. @@ -292,7 +292,7 @@ El número de segundos que ClickHouse espera las solicitudes entrantes antes de 3 ``` -## listen\_host {#server_configuration_parameters-listen_host} +## listen_host {#server_configuration_parameters-listen_host} Restricción en hosts de los que pueden provenir las solicitudes. Si desea que el servidor responda a todos ellos, especifique `::`. @@ -343,10 +343,10 @@ También se admite la escritura en el syslog. Config ejemplo: Claves: -- use\_syslog — Required setting if you want to write to the syslog. +- use_syslog — Required setting if you want to write to the syslog. - address — The host\[:port\] of syslogd. If omitted, the local daemon is used. - hostname — Optional. The name of the host that logs are sent from. -- facility — [La palabra clave syslog facility](https://en.wikipedia.org/wiki/Syslog#Facility) en letras mayúsculas con el “LOG\_” prefijo: (`LOG_USER`, `LOG_DAEMON`, `LOG_LOCAL3` y así sucesivamente). +- facility — [La palabra clave syslog facility](https://en.wikipedia.org/wiki/Syslog#Facility) en letras mayúsculas con el “LOG_” prefijo: (`LOG_USER`, `LOG_DAEMON`, `LOG_LOCAL3` y así sucesivamente). Valor predeterminado: `LOG_USER` si `address` se especifica, `LOG_DAEMON otherwise.` - format – Message format. Possible values: `bsd` y `syslog.` @@ -376,7 +376,7 @@ La memoria caché se comparte para el servidor y la memoria se asigna según sea 5368709120 ``` -## max\_concurrent\_queries {#max-concurrent-queries} +## max_concurrent_queries {#max-concurrent-queries} El número máximo de solicitudes procesadas simultáneamente. @@ -386,7 +386,7 @@ El número máximo de solicitudes procesadas simultáneamente. 100 ``` -## max\_connections {#max-connections} +## max_connections {#max-connections} El número máximo de conexiones entrantes. @@ -396,7 +396,7 @@ El número máximo de conexiones entrantes. 4096 ``` -## max\_open\_files {#max-open-files} +## max_open_files {#max-open-files} El número máximo de archivos abiertos. @@ -410,7 +410,7 @@ Recomendamos usar esta opción en Mac OS X desde el `getrlimit()` función devue 262144 ``` -## max\_table\_size\_to\_drop {#max-table-size-to-drop} +## max_table_size_to_drop {#max-table-size-to-drop} Restricción en la eliminación de tablas. @@ -428,7 +428,7 @@ El valor 0 significa que puede eliminar todas las tablas sin restricciones. 0 ``` -## merge\_tree {#server_configuration_parameters-merge_tree} +## merge_tree {#server_configuration_parameters-merge_tree} Ajuste fino para tablas en el [Método de codificación de datos:](../../engines/table-engines/mergetree-family/mergetree.md). @@ -463,7 +463,7 @@ Claves para la configuración del servidor/cliente: - sessionTimeout – Time for caching the session on the server. - extendedVerification – Automatically extended verification of certificates after the session ends. Acceptable values: `true`, `false`. - requireTLSv1 – Require a TLSv1 connection. Acceptable values: `true`, `false`. -- requireTLSv1\_1 – Require a TLSv1.1 connection. Acceptable values: `true`, `false`. +- requireTLSv1_1 – Require a TLSv1.1 connection. Acceptable values: `true`, `false`. - requireTLSv1 – Require a TLSv1.2 connection. Acceptable values: `true`, `false`. - fips – Activates OpenSSL FIPS mode. Supported if the library's OpenSSL version supports FIPS. - privateKeyPassphraseHandler – Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: ``, `KeyFileHandler`, `test`, ``. @@ -501,11 +501,11 @@ Claves para la configuración del servidor/cliente: ``` -## part\_log {#server_configuration_parameters-part-log} +## part_log {#server_configuration_parameters-part-log} Registro de eventos asociados con [Método de codificación de datos:](../../engines/table-engines/mergetree-family/mergetree.md). Por ejemplo, agregar o fusionar datos. Puede utilizar el registro para simular algoritmos de combinación y comparar sus características. Puede visualizar el proceso de fusión. -Las consultas se registran en el [sistema.part\_log](../../operations/system-tables.md#system_tables-part-log) tabla, no en un archivo separado. Puede configurar el nombre de esta tabla en el `table` parámetro (ver más abajo). +Las consultas se registran en el [sistema.part_log](../../operations/system-tables.md#system_tables-part-log) tabla, no en un archivo separado. Puede configurar el nombre de esta tabla en el `table` parámetro (ver más abajo). Utilice los siguientes parámetros para configurar el registro: @@ -548,7 +548,7 @@ Configuración: - `port` – Port for `endpoint`. - `metrics` – Flag that sets to expose metrics from the [sistema.métricas](../system-tables.md#system_tables-metrics) tabla. - `events` – Flag that sets to expose metrics from the [sistema.evento](../system-tables.md#system_tables-events) tabla. -- `asynchronous_metrics` – Flag that sets to expose current metrics values from the [sistema.asynchronous\_metrics](../system-tables.md#system_tables-asynchronous_metrics) tabla. +- `asynchronous_metrics` – Flag that sets to expose current metrics values from the [sistema.asynchronous_metrics](../system-tables.md#system_tables-asynchronous_metrics) tabla. **Ejemplo** @@ -562,11 +562,11 @@ Configuración: ``` -## query\_log {#server_configuration_parameters-query-log} +## query_log {#server_configuration_parameters-query-log} -Configuración de las consultas de registro recibidas con [log\_queries=1](../settings/settings.md) configuración. +Configuración de las consultas de registro recibidas con [log_queries=1](../settings/settings.md) configuración. -Las consultas se registran en el [sistema.query\_log](../../operations/system-tables.md#system_tables-query_log) tabla, no en un archivo separado. Puede cambiar el nombre de la tabla en el `table` parámetro (ver más abajo). +Las consultas se registran en el [sistema.query_log](../../operations/system-tables.md#system_tables-query_log) tabla, no en un archivo separado. Puede cambiar el nombre de la tabla en el `table` parámetro (ver más abajo). Utilice los siguientes parámetros para configurar el registro: @@ -590,7 +590,7 @@ Si la tabla no existe, ClickHouse la creará. Si la estructura del registro de c ## Sistema abierto {#server_configuration_parameters-query-thread-log} -Configuración de subprocesos de registro de consultas recibidas con [Log\_query\_threads = 1](../settings/settings.md#settings-log-query-threads) configuración. +Configuración de subprocesos de registro de consultas recibidas con [Log_query_threads = 1](../settings/settings.md#settings-log-query-threads) configuración. Las consultas se registran en el [sistema.Sistema abierto.](../../operations/system-tables.md#system_tables-query-thread-log) tabla, no en un archivo separado. Puede cambiar el nombre de la tabla en el `table` parámetro (ver más abajo). @@ -614,9 +614,9 @@ Si la tabla no existe, ClickHouse la creará. Si la estructura del registro de s ``` -## trace\_log {#server_configuration_parameters-trace_log} +## trace_log {#server_configuration_parameters-trace_log} -Ajustes para el [trace\_log](../../operations/system-tables.md#system_tables-trace_log) operación de la tabla del sistema. +Ajustes para el [trace_log](../../operations/system-tables.md#system_tables-trace_log) operación de la tabla del sistema. Parámetros: @@ -636,7 +636,7 @@ El archivo de configuración del servidor predeterminado `config.xml` contiene l ``` -## query\_masking\_rules {#query-masking-rules} +## query_masking_rules {#query-masking-rules} Reglas basadas en Regexp, que se aplicarán a las consultas, así como a todos los mensajes de registro antes de almacenarlos en los registros del servidor, `system.query_log`, `system.text_log`, `system.processes` tabla, y en los registros enviados al cliente. Eso permite prevenir @@ -667,7 +667,7 @@ Las reglas de enmascaramiento se aplican a toda la consulta (para evitar fugas d Para consultas distribuidas, cada servidor debe configurarse por separado; de lo contrario, las subconsultas pasan a otros los nodos se almacenarán sin enmascarar. -## remote\_servers {#server-settings-remote-servers} +## remote_servers {#server-settings-remote-servers} Configuración de los clústeres utilizados por [Distribuido](../../engines/table-engines/special/distributed.md) motor de mesa y por el `cluster` función de la tabla. @@ -681,7 +681,7 @@ Para el valor de la `incl` atributo, consulte la sección “[Archivos de config **Ver también** -- [skip\_unavailable\_shards](../settings/settings.md#settings-skip_unavailable_shards) +- [skip_unavailable_shards](../settings/settings.md#settings-skip_unavailable_shards) ## Zona horaria {#server_configuration_parameters-timezone} @@ -697,7 +697,7 @@ La zona horaria es necesaria para las conversiones entre los formatos String y D Europe/Moscow ``` -## Tcp\_port {#server_configuration_parameters-tcp_port} +## Tcp_port {#server_configuration_parameters-tcp_port} Puerto para comunicarse con clientes a través del protocolo TCP. @@ -707,7 +707,7 @@ Puerto para comunicarse con clientes a través del protocolo TCP. 9000 ``` -## Tcp\_port\_secure {#server_configuration_parameters-tcp_port_secure} +## Tcp_port_secure {#server_configuration_parameters-tcp_port_secure} Puerto TCP para una comunicación segura con los clientes. Úselo con [OpenSSL](#server_configuration_parameters-openssl) configuración. @@ -721,7 +721,7 @@ Entero positivo. 9440 ``` -## mysql\_port {#server_configuration_parameters-mysql_port} +## mysql_port {#server_configuration_parameters-mysql_port} Puerto para comunicarse con clientes a través del protocolo MySQL. @@ -735,7 +735,7 @@ Ejemplo 9004 ``` -## tmp\_path {#server-settings-tmp_path} +## tmp_path {#server-settings-tmp_path} Ruta de acceso a datos temporales para procesar consultas grandes. @@ -748,7 +748,7 @@ Ruta de acceso a datos temporales para procesar consultas grandes. /var/lib/clickhouse/tmp/ ``` -## tmp\_policy {#server-settings-tmp-policy} +## tmp_policy {#server-settings-tmp-policy} Política de [`storage_configuration`](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) para almacenar archivos temporales. Si no se establece [`tmp_path`](#server-settings-tmp_path) se utiliza, de lo contrario se ignora. @@ -759,11 +759,11 @@ Si no se establece [`tmp_path`](#server-settings-tmp_path) se utiliza, de lo con - `max_data_part_size_bytes` se ignora - debe tener exactamente un volumen en esa política -## Uncompressed\_cache\_size {#server-settings-uncompressed_cache_size} +## Uncompressed_cache_size {#server-settings-uncompressed_cache_size} Tamaño de la memoria caché (en bytes) para los datos sin comprimir utilizados por los motores de [Método de codificación de datos:](../../engines/table-engines/mergetree-family/mergetree.md). -Hay una caché compartida para el servidor. La memoria se asigna a pedido. La caché se usa si la opción [Use\_uncompressed\_cache](../settings/settings.md#setting-use_uncompressed_cache) está habilitado. +Hay una caché compartida para el servidor. La memoria se asigna a pedido. La caché se usa si la opción [Use_uncompressed_cache](../settings/settings.md#setting-use_uncompressed_cache) está habilitado. La caché sin comprimir es ventajosa para consultas muy cortas en casos individuales. @@ -773,7 +773,7 @@ La caché sin comprimir es ventajosa para consultas muy cortas en casos individu 8589934592 ``` -## user\_files\_path {#server_configuration_parameters-user_files_path} +## user_files_path {#server_configuration_parameters-user_files_path} El directorio con archivos de usuario. Utilizado en la función de tabla [file()](../../sql-reference/table-functions/file.md). @@ -783,7 +783,7 @@ El directorio con archivos de usuario. Utilizado en la función de tabla [file() /var/lib/clickhouse/user_files/ ``` -## users\_config {#users-config} +## users_config {#users-config} Ruta de acceso al archivo que contiene: @@ -851,13 +851,13 @@ Esta sección contiene los siguientes parámetros: - [Replicación](../../engines/table-engines/mergetree-family/replication.md) - [Guía del programador ZooKeeper](http://zookeeper.apache.org/doc/current/zookeeperProgrammers.html) -## use\_minimalistic\_part\_header\_in\_zookeeper {#server-settings-use_minimalistic_part_header_in_zookeeper} +## use_minimalistic_part_header_in_zookeeper {#server-settings-use_minimalistic_part_header_in_zookeeper} Método de almacenamiento para encabezados de parte de datos en ZooKeeper. Esta configuración sólo se aplica a `MergeTree` familia. Se puede especificar: -- A nivel mundial en el [merge\_tree](#server_configuration_parameters-merge_tree) sección de la `config.xml` file. +- A nivel mundial en el [merge_tree](#server_configuration_parameters-merge_tree) sección de la `config.xml` file. ClickHouse utiliza la configuración para todas las tablas del servidor. Puede cambiar la configuración en cualquier momento. Las tablas existentes cambian su comportamiento cuando cambia la configuración. @@ -879,21 +879,21 @@ Si `use_minimalistic_part_header_in_zookeeper = 1`, entonces [repetición](../.. **Valor predeterminado:** 0. -## disable\_internal\_dns\_cache {#server-settings-disable-internal-dns-cache} +## disable_internal_dns_cache {#server-settings-disable-internal-dns-cache} Deshabilita la memoria caché DNS interna. Recomendado para operar ClickHouse en sistemas con infraestructura que cambia frecuentemente como Kubernetes. **Valor predeterminado:** 0. -## dns\_cache\_update\_period {#server-settings-dns-cache-update-period} +## dns_cache_update_period {#server-settings-dns-cache-update-period} El período de actualización de las direcciones IP almacenadas en la caché DNS interna de ClickHouse (en segundos). La actualización se realiza de forma asíncrona, en un subproceso del sistema separado. **Valor predeterminado**: 15. -## access\_control\_path {#access_control_path} +## access_control_path {#access_control_path} Ruta de acceso a una carpeta donde un servidor ClickHouse almacena configuraciones de usuario y rol creadas por comandos SQL. diff --git a/docs/es/operations/settings/query-complexity.md b/docs/es/operations/settings/query-complexity.md index d65fb03ad13..82bc235c30d 100644 --- a/docs/es/operations/settings/query-complexity.md +++ b/docs/es/operations/settings/query-complexity.md @@ -14,8 +14,8 @@ Casi todas las restricciones solo se aplican a `SELECT`. Para el procesamiento d ClickHouse comprueba las restricciones para las partes de datos, no para cada fila. Significa que puede exceder el valor de restricción con el tamaño de la parte de datos. Restricciones en el “maximum amount of something” puede tomar el valor 0, lo que significa “unrestricted”. -La mayoría de las restricciones también tienen un ‘overflow\_mode’ establecer, lo que significa qué hacer cuando se excede el límite. -Puede tomar uno de dos valores: `throw` o `break`. Las restricciones en la agregación (group\_by\_overflow\_mode) también tienen el valor `any`. +La mayoría de las restricciones también tienen un ‘overflow_mode’ establecer, lo que significa qué hacer cuando se excede el límite. +Puede tomar uno de dos valores: `throw` o `break`. Las restricciones en la agregación (group_by_overflow_mode) también tienen el valor `any`. `throw` – Throw an exception (default). @@ -40,7 +40,7 @@ El uso de memoria no se realiza un seguimiento completo de los estados de las fu El consumo de memoria también está restringido por los parámetros `max_memory_usage_for_user` y `max_memory_usage_for_all_queries`. -## Max\_memory\_usage\_for\_user {#max-memory-usage-for-user} +## Max_memory_usage_for_user {#max-memory-usage-for-user} La cantidad máxima de RAM que se utilizará para ejecutar las consultas de un usuario en un único servidor. @@ -74,12 +74,12 @@ Qué hacer cuando el volumen de datos leídos excede uno de los límites: ‘thr Un número máximo de claves únicas recibidas de la agregación. Esta configuración le permite limitar el consumo de memoria al agregar. -## Grupo\_by\_overflow\_mode {#group-by-overflow-mode} +## Grupo_by_overflow_mode {#group-by-overflow-mode} Qué hacer cuando el número de claves únicas para la agregación excede el límite: ‘throw’, ‘break’, o ‘any’. Por defecto, throw. Uso de la ‘any’ valor le permite ejecutar una aproximación de GROUP BY. La calidad de esta aproximación depende de la naturaleza estadística de los datos. -## max\_bytes\_before\_external\_group\_by {#settings-max_bytes_before_external_group_by} +## max_bytes_before_external_group_by {#settings-max_bytes_before_external_group_by} Habilita o deshabilita la ejecución de `GROUP BY` en la memoria externa. Ver [GROUP BY en memoria externa](../../sql-reference/statements/select/group-by.md#select-group-by-in-external-memory). @@ -98,23 +98,23 @@ Un número máximo de filas antes de ordenar. Esto le permite limitar el consumo Un número máximo de bytes antes de ordenar. -## sort\_overflow\_mode {#sort-overflow-mode} +## sort_overflow_mode {#sort-overflow-mode} Qué hacer si el número de filas recibidas antes de ordenar excede uno de los límites: ‘throw’ o ‘break’. Por defecto, throw. -## max\_result\_rows {#setting-max_result_rows} +## max_result_rows {#setting-max_result_rows} Límite en el número de filas en el resultado. También se comprueba si hay subconsultas y en servidores remotos cuando se ejecutan partes de una consulta distribuida. -## max\_result\_bytes {#max-result-bytes} +## max_result_bytes {#max-result-bytes} Límite en el número de bytes en el resultado. Lo mismo que el ajuste anterior. -## result\_overflow\_mode {#result-overflow-mode} +## result_overflow_mode {#result-overflow-mode} Qué hacer si el volumen del resultado excede uno de los límites: ‘throw’ o ‘break’. Por defecto, throw. -Utilizar ‘break’ es similar a usar LIMIT. `Break` interrumpe la ejecución sólo en el nivel de bloque. Esto significa que la cantidad de filas devueltas es mayor que [max\_result\_rows](#setting-max_result_rows), múltiplo de [max\_block\_size](settings.md#setting-max_block_size) y depende de [max\_threads](settings.md#settings-max_threads). +Utilizar ‘break’ es similar a usar LIMIT. `Break` interrumpe la ejecución sólo en el nivel de bloque. Esto significa que la cantidad de filas devueltas es mayor que [max_result_rows](#setting-max_result_rows), múltiplo de [max_block_size](settings.md#setting-max_block_size) y depende de [max_threads](settings.md#settings-max_threads). Ejemplo: @@ -133,62 +133,62 @@ Resultado: 6666 rows in set. ... ``` -## max\_execution\_time {#max-execution-time} +## max_execution_time {#max-execution-time} Tiempo máximo de ejecución de la consulta en segundos. En este momento, no se comprueba una de las etapas de clasificación, o al fusionar y finalizar funciones agregadas. -## timeout\_overflow\_mode {#timeout-overflow-mode} +## timeout_overflow_mode {#timeout-overflow-mode} -Qué hacer si la consulta se ejecuta más de ‘max\_execution\_time’: ‘throw’ o ‘break’. Por defecto, throw. +Qué hacer si la consulta se ejecuta más de ‘max_execution_time’: ‘throw’ o ‘break’. Por defecto, throw. ## Método de codificación de datos: {#min-execution-speed} -Velocidad de ejecución mínima en filas por segundo. Comprobado en cada bloque de datos cuando ‘timeout\_before\_checking\_execution\_speed’ expirar. Si la velocidad de ejecución es menor, se produce una excepción. +Velocidad de ejecución mínima en filas por segundo. Comprobado en cada bloque de datos cuando ‘timeout_before_checking_execution_speed’ expirar. Si la velocidad de ejecución es menor, se produce una excepción. ## Todos los derechos reservados {#min-execution-speed-bytes} -Un número mínimo de bytes de ejecución por segundo. Comprobado en cada bloque de datos cuando ‘timeout\_before\_checking\_execution\_speed’ expirar. Si la velocidad de ejecución es menor, se produce una excepción. +Un número mínimo de bytes de ejecución por segundo. Comprobado en cada bloque de datos cuando ‘timeout_before_checking_execution_speed’ expirar. Si la velocidad de ejecución es menor, se produce una excepción. -## Max\_execution\_speed {#max-execution-speed} +## Max_execution_speed {#max-execution-speed} -Un número máximo de filas de ejecución por segundo. Comprobado en cada bloque de datos cuando ‘timeout\_before\_checking\_execution\_speed’ expirar. Si la velocidad de ejecución es alta, la velocidad de ejecución se reducirá. +Un número máximo de filas de ejecución por segundo. Comprobado en cada bloque de datos cuando ‘timeout_before_checking_execution_speed’ expirar. Si la velocidad de ejecución es alta, la velocidad de ejecución se reducirá. -## Max\_execution\_speed\_bytes {#max-execution-speed-bytes} +## Max_execution_speed_bytes {#max-execution-speed-bytes} -Un número máximo de bytes de ejecución por segundo. Comprobado en cada bloque de datos cuando ‘timeout\_before\_checking\_execution\_speed’ expirar. Si la velocidad de ejecución es alta, la velocidad de ejecución se reducirá. +Un número máximo de bytes de ejecución por segundo. Comprobado en cada bloque de datos cuando ‘timeout_before_checking_execution_speed’ expirar. Si la velocidad de ejecución es alta, la velocidad de ejecución se reducirá. -## Tiempo de espera antes de comprobar\_ejecución\_velocidad {#timeout-before-checking-execution-speed} +## Tiempo de espera antes de comprobar_ejecución_velocidad {#timeout-before-checking-execution-speed} -Comprueba que la velocidad de ejecución no sea demasiado lenta (no menos de ‘min\_execution\_speed’), después de que el tiempo especificado en segundos haya expirado. +Comprueba que la velocidad de ejecución no sea demasiado lenta (no menos de ‘min_execution_speed’), después de que el tiempo especificado en segundos haya expirado. -## Max\_columns\_to\_read {#max-columns-to-read} +## Max_columns_to_read {#max-columns-to-read} Un número máximo de columnas que se pueden leer de una tabla en una sola consulta. Si una consulta requiere leer un mayor número de columnas, produce una excepción. -## max\_temporary\_columns {#max-temporary-columns} +## max_temporary_columns {#max-temporary-columns} Un número máximo de columnas temporales que se deben mantener en la memoria RAM al mismo tiempo cuando se ejecuta una consulta, incluidas las columnas constantes. Si hay más columnas temporales que esto, arroja una excepción. -## max\_temporary\_non\_const\_columns {#max-temporary-non-const-columns} +## max_temporary_non_const_columns {#max-temporary-non-const-columns} -Lo mismo que ‘max\_temporary\_columns’, pero sin contar columnas constantes. +Lo mismo que ‘max_temporary_columns’, pero sin contar columnas constantes. Tenga en cuenta que las columnas constantes se forman con bastante frecuencia cuando se ejecuta una consulta, pero requieren aproximadamente cero recursos informáticos. -## max\_subquery\_depth {#max-subquery-depth} +## max_subquery_depth {#max-subquery-depth} Profundidad máxima de anidamiento de subconsultas. Si las subconsultas son más profundas, se produce una excepción. De forma predeterminada, 100. -## max\_pipeline\_depth {#max-pipeline-depth} +## max_pipeline_depth {#max-pipeline-depth} Profundidad máxima de la tubería. Corresponde al número de transformaciones que realiza cada bloque de datos durante el procesamiento de consultas. Contado dentro de los límites de un único servidor. Si la profundidad de la canalización es mayor, se produce una excepción. Por defecto, 1000. -## max\_ast\_depth {#max-ast-depth} +## max_ast_depth {#max-ast-depth} Profundidad máxima de anidamiento de un árbol sintáctico de consulta. Si se supera, se produce una excepción. En este momento, no se verifica durante el análisis, sino solo después de analizar la consulta. Es decir, se puede crear un árbol sintáctico demasiado profundo durante el análisis, pero la consulta fallará. Por defecto, 1000. -## max\_ast\_elements {#max-ast-elements} +## max_ast_elements {#max-ast-elements} Un número máximo de elementos en un árbol sintáctico de consulta. Si se supera, se produce una excepción. De la misma manera que la configuración anterior, se verifica solo después de analizar la consulta. De forma predeterminada, 50.000. @@ -201,7 +201,7 @@ Un número máximo de filas para un conjunto de datos en la cláusula IN creada Número máximo de bytes (datos sin comprimir) utilizados por un conjunto en la cláusula IN creada a partir de una subconsulta. -## set\_overflow\_mode {#set-overflow-mode} +## set_overflow_mode {#set-overflow-mode} Qué hacer cuando la cantidad de datos excede uno de los límites: ‘throw’ o ‘break’. Por defecto, throw. @@ -213,19 +213,19 @@ Un número máximo de filas diferentes al usar DISTINCT. Un número máximo de bytes utilizados por una tabla hash cuando se utiliza DISTINCT. -## distinct\_overflow\_mode {#distinct-overflow-mode} +## distinct_overflow_mode {#distinct-overflow-mode} Qué hacer cuando la cantidad de datos excede uno de los límites: ‘throw’ o ‘break’. Por defecto, throw. -## max\_rows\_to\_transfer {#max-rows-to-transfer} +## max_rows_to_transfer {#max-rows-to-transfer} Un número máximo de filas que se pueden pasar a un servidor remoto o guardar en una tabla temporal cuando se utiliza GLOBAL IN. -## max\_bytes\_to\_transfer {#max-bytes-to-transfer} +## max_bytes_to_transfer {#max-bytes-to-transfer} Un número máximo de bytes (datos sin comprimir) que se pueden pasar a un servidor remoto o guardar en una tabla temporal cuando se utiliza GLOBAL IN. -## transfer\_overflow\_mode {#transfer-overflow-mode} +## transfer_overflow_mode {#transfer-overflow-mode} Qué hacer cuando la cantidad de datos excede uno de los límites: ‘throw’ o ‘break’. Por defecto, throw. @@ -237,7 +237,7 @@ Esta configuración se aplica a [SELECT … JOIN](../../sql-reference/statements Si una consulta contiene varias combinaciones, ClickHouse comprueba esta configuración para cada resultado intermedio. -ClickHouse puede proceder con diferentes acciones cuando se alcanza el límite. Utilice el [join\_overflow\_mode](#settings-join_overflow_mode) configuración para elegir la acción. +ClickHouse puede proceder con diferentes acciones cuando se alcanza el límite. Utilice el [join_overflow_mode](#settings-join_overflow_mode) configuración para elegir la acción. Valores posibles: @@ -254,7 +254,7 @@ Esta configuración se aplica a [SELECT … JOIN](../../sql-reference/statements Si la consulta contiene combinaciones, ClickHouse comprueba esta configuración para cada resultado intermedio. -ClickHouse puede proceder con diferentes acciones cuando se alcanza el límite. Utilizar [join\_overflow\_mode](#settings-join_overflow_mode) para elegir la acción. +ClickHouse puede proceder con diferentes acciones cuando se alcanza el límite. Utilizar [join_overflow_mode](#settings-join_overflow_mode) para elegir la acción. Valores posibles: @@ -263,7 +263,7 @@ Valores posibles: Valor predeterminado: 0. -## join\_overflow\_mode {#settings-join_overflow_mode} +## join_overflow_mode {#settings-join_overflow_mode} Define qué acción realiza ClickHouse cuando se alcanza cualquiera de los siguientes límites de combinación: @@ -282,7 +282,7 @@ Valor predeterminado: `THROW`. - [Cláusula JOIN](../../sql-reference/statements/select/join.md#select-join) - [Unirse al motor de tabla](../../engines/table-engines/special/join.md) -## max\_partitions\_per\_insert\_block {#max-partitions-per-insert-block} +## max_partitions_per_insert_block {#max-partitions-per-insert-block} Limita el número máximo de particiones en un único bloque insertado. @@ -295,6 +295,6 @@ Valor predeterminado: 100. Al insertar datos, ClickHouse calcula el número de particiones en el bloque insertado. Si el número de particiones es mayor que `max_partitions_per_insert_block`, ClickHouse lanza una excepción con el siguiente texto: -> “Too many partitions for single INSERT block (more than” ¿Cómo puedo hacerlo? “). The limit is controlled by ‘max\_partitions\_per\_insert\_block’ setting. A large number of partitions is a common misconception. It will lead to severe negative performance impact, including slow server startup, slow INSERT queries and slow SELECT queries. Recommended total number of partitions for a table is under 1000..10000. Please note, that partitioning is not intended to speed up SELECT queries (ORDER BY key is sufficient to make range queries fast). Partitions are intended for data manipulation (DROP PARTITION, etc).” +> “Too many partitions for single INSERT block (more than” ¿Cómo puedo hacerlo? “). The limit is controlled by ‘max_partitions_per_insert_block’ setting. A large number of partitions is a common misconception. It will lead to severe negative performance impact, including slow server startup, slow INSERT queries and slow SELECT queries. Recommended total number of partitions for a table is under 1000..10000. Please note, that partitioning is not intended to speed up SELECT queries (ORDER BY key is sufficient to make range queries fast). Partitions are intended for data manipulation (DROP PARTITION, etc).” [Artículo Original](https://clickhouse.tech/docs/en/operations/settings/query_complexity/) diff --git a/docs/es/operations/settings/settings-users.md b/docs/es/operations/settings/settings-users.md index 2f767c959ae..1c1ac7914f0 100644 --- a/docs/es/operations/settings/settings-users.md +++ b/docs/es/operations/settings/settings-users.md @@ -43,7 +43,7 @@ Estructura del `users` apartado: ``` -### user\_name/contraseña {#user-namepassword} +### user_name/contraseña {#user-namepassword} La contraseña se puede especificar en texto sin formato o en SHA256 (formato hexagonal). @@ -75,7 +75,7 @@ La contraseña se puede especificar en texto sin formato o en SHA256 (formato he La primera línea del resultado es la contraseña. La segunda línea es el hash SHA1 doble correspondiente. -### access\_management {#access_management-user-setting} +### access_management {#access_management-user-setting} Esta configuración habilita deshabilita el uso de [control de acceso y gestión de cuentas](../access-rights.md#access-control) para el usuario. @@ -86,7 +86,7 @@ Valores posibles: Valor predeterminado: 0. -### user\_name/redes {#user-namenetworks} +### user_name/redes {#user-namenetworks} Lista de redes desde las que el usuario puede conectarse al servidor ClickHouse. @@ -128,18 +128,18 @@ Para abrir el acceso solo desde localhost, especifique: 127.0.0.1 ``` -### user\_name/perfil {#user-nameprofile} +### user_name/perfil {#user-nameprofile} Puede asignar un perfil de configuración para el usuario. Los perfiles de configuración se configuran en una sección separada del `users.xml` file. Para obtener más información, consulte [Perfiles de configuración](settings-profiles.md). -### user\_name/cuota {#user-namequota} +### user_name/cuota {#user-namequota} Las cuotas le permiten realizar un seguimiento o limitar el uso de recursos durante un período de tiempo. Las cuotas se configuran en el `quotas` sección de la `users.xml` archivo de configuración. Puede asignar un conjunto de cuotas para el usuario. Para obtener una descripción detallada de la configuración de las cuotas, consulte [Cuota](../quotas.md#quotas). -### nombre\_usuario/bases de datos {#user-namedatabases} +### nombre_usuario/bases de datos {#user-namedatabases} En esta sección, puede limitar las filas devueltas por ClickHouse para `SELECT` consultas realizadas por el usuario actual, implementando así la seguridad básica a nivel de fila. diff --git a/docs/es/operations/settings/settings.md b/docs/es/operations/settings/settings.md index d709bb69bc8..62511dd9fc0 100644 --- a/docs/es/operations/settings/settings.md +++ b/docs/es/operations/settings/settings.md @@ -5,7 +5,7 @@ machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd # Configuración {#settings} -## distributed\_product\_mode {#distributed-product-mode} +## distributed_product_mode {#distributed-product-mode} Cambia el comportamiento de [subconsultas distribuidas](../../sql-reference/operators/in.md). @@ -25,7 +25,7 @@ Valores posibles: - `global` — Replaces the `IN`/`JOIN` consulta con `GLOBAL IN`/`GLOBAL JOIN.` - `allow` — Allows the use of these types of subqueries. -## enable\_optimize\_predicate\_expression {#enable-optimize-predicate-expression} +## enable_optimize_predicate_expression {#enable-optimize-predicate-expression} Activa el pushdown de predicado en `SELECT` consulta. @@ -49,7 +49,7 @@ Si `enable_optimize_predicate_expression = 1`, entonces el tiempo de ejecución Si `enable_optimize_predicate_expression = 0`, entonces el tiempo de ejecución de la segunda consulta es mucho más largo, porque el `WHERE` cláusula se aplica a todos los datos después de que finalice la subconsulta. -## fallback\_to\_stale\_replicas\_for\_distributed\_queries {#settings-fallback_to_stale_replicas_for_distributed_queries} +## fallback_to_stale_replicas_for_distributed_queries {#settings-fallback_to_stale_replicas_for_distributed_queries} Fuerza una consulta a una réplica obsoleta si los datos actualizados no están disponibles. Ver [Replicación](../../engines/table-engines/mergetree-family/replication.md). @@ -67,7 +67,7 @@ Funciona con tablas de la familia MergeTree. Si `force_index_by_date=1`, ClickHouse comprueba si la consulta tiene una condición de clave de fecha que se puede usar para restringir intervalos de datos. Si no hay una condición adecuada, arroja una excepción. Sin embargo, no comprueba si la condición reduce la cantidad de datos a leer. Por ejemplo, la condición `Date != ' 2000-01-01 '` es aceptable incluso cuando coincide con todos los datos de la tabla (es decir, ejecutar la consulta requiere un escaneo completo). Para obtener más información acerca de los intervalos de datos en las tablas MergeTree, vea [Método de codificación de datos:](../../engines/table-engines/mergetree-family/mergetree.md). -## force\_primary\_key {#force-primary-key} +## force_primary_key {#force-primary-key} Deshabilita la ejecución de consultas si no es posible la indexación mediante la clave principal. @@ -75,17 +75,17 @@ Funciona con tablas de la familia MergeTree. Si `force_primary_key=1`, ClickHouse comprueba si la consulta tiene una condición de clave principal que se puede usar para restringir rangos de datos. Si no hay una condición adecuada, arroja una excepción. Sin embargo, no comprueba si la condición reduce la cantidad de datos a leer. Para obtener más información acerca de los intervalos de datos en las tablas MergeTree, consulte [Método de codificación de datos:](../../engines/table-engines/mergetree-family/mergetree.md). -## Formato\_esquema {#format-schema} +## Formato_esquema {#format-schema} Este parámetro es útil cuando se utilizan formatos que requieren una definición de esquema, como [Cap'n Proto](https://capnproto.org/) o [Protobuf](https://developers.google.com/protocol-buffers/). El valor depende del formato. -## fsync\_metadata {#fsync-metadata} +## fsync_metadata {#fsync-metadata} Habilita o deshabilita [fsync](http://pubs.opengroup.org/onlinepubs/9699919799/functions/fsync.html) al escribir `.sql` file. Habilitado de forma predeterminada. Tiene sentido desactivarlo si el servidor tiene millones de pequeñas tablas que se crean y destruyen constantemente. -## enable\_http\_compression {#settings-enable_http_compression} +## enable_http_compression {#settings-enable_http_compression} Habilita o deshabilita la compresión de datos en la respuesta a una solicitud HTTP. @@ -98,15 +98,15 @@ Valores posibles: Valor predeterminado: 0. -## http\_zlib\_compression\_level {#settings-http_zlib_compression_level} +## http_zlib_compression_level {#settings-http_zlib_compression_level} -Establece el nivel de compresión de datos en la respuesta a una solicitud HTTP si [enable\_http\_compression = 1](#settings-enable_http_compression). +Establece el nivel de compresión de datos en la respuesta a una solicitud HTTP si [enable_http_compression = 1](#settings-enable_http_compression). Valores posibles: Números del 1 al 9. Valor predeterminado: 3. -## http\_native\_compression\_disable\_checksumming\_on\_decompress {#settings-http_native_compression_disable_checksumming_on_decompress} +## http_native_compression_disable_checksumming_on_decompress {#settings-http_native_compression_disable_checksumming_on_decompress} Habilita o deshabilita la verificación de suma de comprobación al descomprimir los datos HTTP POST del cliente. Se usa solo para el formato de compresión nativa ClickHouse (no se usa con `gzip` o `deflate`). @@ -119,7 +119,7 @@ Valores posibles: Valor predeterminado: 0. -## send\_progress\_in\_http\_headers {#settings-send_progress_in_http_headers} +## send_progress_in_http_headers {#settings-send_progress_in_http_headers} Habilita o deshabilita `X-ClickHouse-Progress` Encabezados de respuesta HTTP en `clickhouse-server` respuesta. @@ -143,7 +143,7 @@ Valores posibles: Valor predeterminado: 0. -## Entrada\_format\_allow\_errors\_num {#settings-input_format_allow_errors_num} +## Entrada_format_allow_errors_num {#settings-input_format_allow_errors_num} Establece el número máximo de errores aceptables al leer desde formatos de texto (CSV, TSV, etc.). @@ -155,7 +155,7 @@ Si se produjo un error al leer filas, pero el contador de errores sigue siendo m Si ambos `input_format_allow_errors_num` y `input_format_allow_errors_ratio` se exceden, ClickHouse lanza una excepción. -## Entrada\_format\_allow\_errors\_ratio {#settings-input_format_allow_errors_ratio} +## Entrada_format_allow_errors_ratio {#settings-input_format_allow_errors_ratio} Establece el porcentaje máximo de errores permitidos al leer desde formatos de texto (CSV, TSV, etc.). El porcentaje de errores se establece como un número de punto flotante entre 0 y 1. @@ -168,7 +168,7 @@ Si se produjo un error al leer filas, pero el contador de errores sigue siendo m Si ambos `input_format_allow_errors_num` y `input_format_allow_errors_ratio` se exceden, ClickHouse lanza una excepción. -## input\_format\_values\_interpret\_expressions {#settings-input_format_values_interpret_expressions} +## input_format_values_interpret_expressions {#settings-input_format_values_interpret_expressions} Habilita o deshabilita el analizador SQL completo si el analizador de secuencias rápidas no puede analizar los datos. Esta configuración sólo se utiliza para [Valor](../../interfaces/formats.md#data-format-values) formato en la inserción de datos. Para obtener más información sobre el análisis de sintaxis, consulte [Sintaxis](../../sql-reference/syntax.md) apartado. @@ -218,7 +218,7 @@ INSERT INTO datetime_t SELECT now() Ok. ``` -## input\_format\_values\_deduce\_templates\_of\_expressions {#settings-input_format_values_deduce_templates_of_expressions} +## input_format_values_deduce_templates_of_expressions {#settings-input_format_values_deduce_templates_of_expressions} Habilita o deshabilita la deducción de plantilla para expresiones SQL en [Valor](../../interfaces/formats.md#data-format-values) formato. Permite analizar e interpretar expresiones en `Values` mucho más rápido si las expresiones en filas consecutivas tienen la misma estructura. ClickHouse intenta deducir la plantilla de una expresión, analizar las siguientes filas utilizando esta plantilla y evaluar la expresión en un lote de filas analizadas correctamente. @@ -239,7 +239,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( - Si `input_format_values_interpret_expressions=0` y `format_values_deduce_templates_of_expressions=1`, las expresiones en la primera, segunda y tercera filas se analizan usando la plantilla `lower(String)` e interpretados juntos, la expresión en la cuarta fila se analiza con otra plantilla (`upper(String)`). - Si `input_format_values_interpret_expressions=1` y `format_values_deduce_templates_of_expressions=1`, lo mismo que en el caso anterior, pero también permite la alternativa a la interpretación de expresiones por separado si no es posible deducir la plantilla. -## Entrada\_format\_values\_accurate\_types\_of\_literals {#settings-input-format-values-accurate-types-of-literals} +## Entrada_format_values_accurate_types_of_literals {#settings-input-format-values-accurate-types-of-literals} Esta configuración sólo se utiliza cuando `input_format_values_deduce_templates_of_expressions = 1`. Puede suceder que las expresiones para alguna columna tengan la misma estructura, pero contengan literales numéricos de diferentes tipos, por ejemplo @@ -261,7 +261,7 @@ Valores posibles: Valor predeterminado: 1. -## Entrada\_format\_defaults\_for\_omitted\_fields {#session_settings-input_format_defaults_for_omitted_fields} +## Entrada_format_defaults_for_omitted_fields {#session_settings-input_format_defaults_for_omitted_fields} Al realizar `INSERT` consultas, reemplace los valores de columna de entrada omitidos con valores predeterminados de las columnas respectivas. Esta opción sólo se aplica a [JSONEachRow](../../interfaces/formats.md#jsoneachrow), [CSV](../../interfaces/formats.md#csv) y [TabSeparated](../../interfaces/formats.md#tabseparated) formato. @@ -275,17 +275,17 @@ Valores posibles: Valor predeterminado: 1. -## input\_format\_tsv\_empty\_as\_default {#settings-input-format-tsv-empty-as-default} +## input_format_tsv_empty_as_default {#settings-input-format-tsv-empty-as-default} Cuando esté habilitado, reemplace los campos de entrada vacíos en TSV con valores predeterminados. Para expresiones predeterminadas complejas `input_format_defaults_for_omitted_fields` debe estar habilitado también. Deshabilitado de forma predeterminada. -## input\_format\_null\_as\_default {#settings-input-format-null-as-default} +## input_format_null_as_default {#settings-input-format-null-as-default} Habilita o deshabilita el uso de valores predeterminados si los datos de entrada `NULL`, pero el tipo de datos de la columna correspondiente en no `Nullable(T)` (para formatos de entrada de texto). -## input\_format\_skip\_unknown\_fields {#settings-input-format-skip-unknown-fields} +## input_format_skip_unknown_fields {#settings-input-format-skip-unknown-fields} Habilita o deshabilita omitir la inserción de datos adicionales. @@ -305,7 +305,7 @@ Valores posibles: Valor predeterminado: 0. -## Entrada\_format\_import\_nested\_json {#settings-input_format_import_nested_json} +## Entrada_format_import_nested_json {#settings-input_format_import_nested_json} Habilita o deshabilita la inserción de datos JSON con objetos anidados. @@ -324,7 +324,7 @@ Ver también: - [Uso de estructuras anidadas](../../interfaces/formats.md#jsoneachrow-nested) con el `JSONEachRow` formato. -## Entrada\_format\_with\_names\_use\_header {#settings-input-format-with-names-use-header} +## Entrada_format_with_names_use_header {#settings-input-format-with-names-use-header} Habilita o deshabilita la comprobación del orden de las columnas al insertar datos. @@ -342,7 +342,7 @@ Valores posibles: Valor predeterminado: 1. -## Date\_time\_input\_format {#settings-date_time_input_format} +## Date_time_input_format {#settings-date_time_input_format} Permite elegir un analizador de la representación de texto de fecha y hora. @@ -378,7 +378,7 @@ Valores posibles: Valor predeterminado: `ALL`. -## join\_any\_take\_last\_row {#settings-join_any_take_last_row} +## join_any_take_last_row {#settings-join_any_take_last_row} Cambia el comportamiento de las operaciones de unión con `ANY` rigor. @@ -409,7 +409,7 @@ Valores posibles: Valor predeterminado: 0. -## max\_block\_size {#setting-max_block_size} +## max_block_size {#setting-max_block_size} En ClickHouse, los datos se procesan mediante bloques (conjuntos de partes de columna). Los ciclos de procesamiento interno para un solo bloque son lo suficientemente eficientes, pero hay gastos notables en cada bloque. El `max_block_size` set es una recomendación para el tamaño del bloque (en un recuento de filas) para cargar desde las tablas. El tamaño del bloque no debe ser demasiado pequeño, por lo que los gastos en cada bloque aún se notan, pero no demasiado grande para que la consulta con LIMIT que se complete después del primer bloque se procese rápidamente. El objetivo es evitar consumir demasiada memoria al extraer un gran número de columnas en múltiples subprocesos y preservar al menos alguna localidad de caché. @@ -417,13 +417,13 @@ Valor predeterminado: 65,536. Bloquea el tamaño de `max_block_size` no siempre se cargan desde la tabla. Si es obvio que se deben recuperar menos datos, se procesa un bloque más pequeño. -## preferred\_block\_size\_bytes {#preferred-block-size-bytes} +## preferred_block_size_bytes {#preferred-block-size-bytes} Utilizado para el mismo propósito que `max_block_size`, pero establece el tamaño de bloque recomendado en bytes adaptándolo al número de filas en el bloque. Sin embargo, el tamaño del bloque no puede ser más que `max_block_size` filas. Por defecto: 1,000,000. Solo funciona cuando se lee desde los motores MergeTree. -## merge\_tree\_min\_rows\_for\_concurrent\_read {#setting-merge-tree-min-rows-for-concurrent-read} +## merge_tree_min_rows_for_concurrent_read {#setting-merge-tree-min-rows-for-concurrent-read} Si el número de filas que se leerán de un fichero [Método de codificación de datos:](../../engines/table-engines/mergetree-family/mergetree.md) mesa excede `merge_tree_min_rows_for_concurrent_read` luego ClickHouse intenta realizar una lectura simultánea de este archivo en varios hilos. @@ -433,7 +433,7 @@ Valores posibles: Valor predeterminado: 163840. -## merge\_tree\_min\_bytes\_for\_concurrent\_read {#setting-merge-tree-min-bytes-for-concurrent-read} +## merge_tree_min_bytes_for_concurrent_read {#setting-merge-tree-min-bytes-for-concurrent-read} Si el número de bytes a leer de un archivo de un [Método de codificación de datos:](../../engines/table-engines/mergetree-family/mergetree.md)-La tabla del motor excede `merge_tree_min_bytes_for_concurrent_read`, entonces ClickHouse intenta leer simultáneamente este archivo en varios subprocesos. @@ -453,7 +453,7 @@ Valores posibles: Valor predeterminado: 0. -## merge\_tree\_min\_bytes\_for\_seek {#setting-merge-tree-min-bytes-for-seek} +## merge_tree_min_bytes_for_seek {#setting-merge-tree-min-bytes-for-seek} Si la distancia entre dos bloques de datos que se leen en un archivo es menor que `merge_tree_min_bytes_for_seek` bytes, luego ClickHouse lee secuencialmente un rango de archivos que contiene ambos bloques, evitando así la búsqueda adicional. @@ -463,7 +463,7 @@ Valores posibles: Valor predeterminado: 0. -## merge\_tree\_coarse\_index\_granularity {#setting-merge-tree-coarse-index-granularity} +## merge_tree_coarse_index_granularity {#setting-merge-tree-coarse-index-granularity} Al buscar datos, ClickHouse comprueba las marcas de datos en el archivo de índice. Si ClickHouse encuentra que las claves requeridas están en algún rango, divide este rango en `merge_tree_coarse_index_granularity` subintervalos y busca las claves necesarias allí de forma recursiva. @@ -473,11 +473,11 @@ Valores posibles: Valor predeterminado: 8. -## merge\_tree\_max\_rows\_to\_use\_cache {#setting-merge-tree-max-rows-to-use-cache} +## merge_tree_max_rows_to_use_cache {#setting-merge-tree-max-rows-to-use-cache} Si ClickHouse debería leer más de `merge_tree_max_rows_to_use_cache` en una consulta, no usa la memoria caché de bloques sin comprimir. -La memoria caché de bloques sin comprimir almacena datos extraídos para consultas. ClickHouse utiliza esta memoria caché para acelerar las respuestas a pequeñas consultas repetidas. Esta configuración protege la memoria caché del deterioro de las consultas que leen una gran cantidad de datos. El [Uncompressed\_cache\_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuración del servidor define el tamaño de la memoria caché de bloques sin comprimir. +La memoria caché de bloques sin comprimir almacena datos extraídos para consultas. ClickHouse utiliza esta memoria caché para acelerar las respuestas a pequeñas consultas repetidas. Esta configuración protege la memoria caché del deterioro de las consultas que leen una gran cantidad de datos. El [Uncompressed_cache_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuración del servidor define el tamaño de la memoria caché de bloques sin comprimir. Valores posibles: @@ -485,11 +485,11 @@ Valores posibles: Default value: 128 ✕ 8192. -## merge\_tree\_max\_bytes\_to\_use\_cache {#setting-merge-tree-max-bytes-to-use-cache} +## merge_tree_max_bytes_to_use_cache {#setting-merge-tree-max-bytes-to-use-cache} Si ClickHouse debería leer más de `merge_tree_max_bytes_to_use_cache` bytes en una consulta, no usa el caché de bloques sin comprimir. -La memoria caché de bloques sin comprimir almacena datos extraídos para consultas. ClickHouse utiliza esta memoria caché para acelerar las respuestas a pequeñas consultas repetidas. Esta configuración protege la memoria caché del deterioro de las consultas que leen una gran cantidad de datos. El [Uncompressed\_cache\_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuración del servidor define el tamaño de la memoria caché de bloques sin comprimir. +La memoria caché de bloques sin comprimir almacena datos extraídos para consultas. ClickHouse utiliza esta memoria caché para acelerar las respuestas a pequeñas consultas repetidas. Esta configuración protege la memoria caché del deterioro de las consultas que leen una gran cantidad de datos. El [Uncompressed_cache_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuración del servidor define el tamaño de la memoria caché de bloques sin comprimir. Valor posible: @@ -510,11 +510,11 @@ Valores posibles: Valor predeterminado: 0. -## Log\_queries {#settings-log-queries} +## Log_queries {#settings-log-queries} Configuración del registro de consultas. -Las consultas enviadas a ClickHouse con esta configuración se registran de acuerdo con las reglas [query\_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query-log) parámetro de configuración del servidor. +Las consultas enviadas a ClickHouse con esta configuración se registran de acuerdo con las reglas [query_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query-log) parámetro de configuración del servidor. Ejemplo: @@ -540,7 +540,7 @@ Se puede usar para limitar a qué entiries va `query_log`, digamos que eres inte log_queries_min_type='EXCEPTION_WHILE_PROCESSING' ``` -## Log\_query\_threads {#settings-log-query-threads} +## Log_query_threads {#settings-log-query-threads} Configuración del registro de subprocesos de consulta. @@ -552,12 +552,12 @@ Ejemplo: log_query_threads=1 ``` -## Max\_insert\_block\_size {#settings-max_insert_block_size} +## Max_insert_block_size {#settings-max_insert_block_size} El tamaño de los bloques a formar para su inserción en una tabla. Esta configuración solo se aplica en los casos en que el servidor forma los bloques. Por ejemplo, para un INSERT a través de la interfaz HTTP, el servidor analiza el formato de datos y forma bloques del tamaño especificado. -Pero al usar clickhouse-client, el cliente analiza los datos en sí, y el ‘max\_insert\_block\_size’ configuración en el servidor no afecta el tamaño de los bloques insertados. +Pero al usar clickhouse-client, el cliente analiza los datos en sí, y el ‘max_insert_block_size’ configuración en el servidor no afecta el tamaño de los bloques insertados. La configuración tampoco tiene un propósito cuando se usa INSERT SELECT , ya que los datos se insertan usando los mismos bloques que se forman después de SELECT . Valor predeterminado: 1.048.576. @@ -586,7 +586,7 @@ Valores posibles: Valor predeterminado: 268435456. -## max\_replica\_delay\_for\_distributed\_queries {#settings-max_replica_delay_for_distributed_queries} +## max_replica_delay_for_distributed_queries {#settings-max_replica_delay_for_distributed_queries} Deshabilita las réplicas rezagadas para consultas distribuidas. Ver [Replicación](../../engines/table-engines/mergetree-family/replication.md). @@ -596,18 +596,18 @@ Valor predeterminado: 300. Se utiliza al realizar `SELECT` desde una tabla distribuida que apunta a tablas replicadas. -## max\_threads {#settings-max_threads} +## max_threads {#settings-max_threads} -El número máximo de subprocesos de procesamiento de consultas, excluyendo subprocesos para recuperar datos de servidores ‘max\_distributed\_connections’ parámetro). +El número máximo de subprocesos de procesamiento de consultas, excluyendo subprocesos para recuperar datos de servidores ‘max_distributed_connections’ parámetro). Este parámetro se aplica a los subprocesos que realizan las mismas etapas de la canalización de procesamiento de consultas en paralelo. -Por ejemplo, al leer desde una tabla, si es posible evaluar expresiones con funciones, filtre con WHERE y preagregue para GROUP BY en paralelo usando al menos ‘max\_threads’ número de hilos, entonces ‘max\_threads’ se utilizan. +Por ejemplo, al leer desde una tabla, si es posible evaluar expresiones con funciones, filtre con WHERE y preagregue para GROUP BY en paralelo usando al menos ‘max_threads’ número de hilos, entonces ‘max_threads’ se utilizan. Valor predeterminado: el número de núcleos de CPU físicos. Si normalmente se ejecuta menos de una consulta SELECT en un servidor a la vez, establezca este parámetro en un valor ligeramente inferior al número real de núcleos de procesador. -Para las consultas que se completan rápidamente debido a un LIMIT, puede establecer un ‘max\_threads’. Por ejemplo, si el número necesario de entradas se encuentra en cada bloque y max\_threads = 8, entonces se recuperan 8 bloques, aunque hubiera sido suficiente leer solo uno. +Para las consultas que se completan rápidamente debido a un LIMIT, puede establecer un ‘max_threads’. Por ejemplo, si el número necesario de entradas se encuentra en cada bloque y max_threads = 8, entonces se recuperan 8 bloques, aunque hubiera sido suficiente leer solo uno. Cuanto menor sea el `max_threads` valor, menos memoria se consume. @@ -622,10 +622,10 @@ Valores posibles: Valor predeterminado: 0. -Paralelo `INSERT SELECT` sólo tiene efecto si el `SELECT` parte se ejecuta en paralelo, ver [max\_threads](#settings-max_threads) configuración. +Paralelo `INSERT SELECT` sólo tiene efecto si el `SELECT` parte se ejecuta en paralelo, ver [max_threads](#settings-max_threads) configuración. Los valores más altos conducirán a un mayor uso de memoria. -## max\_compress\_block\_size {#max-compress-block-size} +## max_compress_block_size {#max-compress-block-size} El tamaño máximo de bloques de datos sin comprimir antes de comprimir para escribir en una tabla. De forma predeterminada, 1.048.576 (1 MiB). Si se reduce el tamaño, la tasa de compresión se reduce significativamente, la velocidad de compresión y descompresión aumenta ligeramente debido a la localidad de la memoria caché, y se reduce el consumo de memoria. Por lo general, no hay ninguna razón para cambiar esta configuración. @@ -633,26 +633,26 @@ No confunda bloques para la compresión (un fragmento de memoria que consta de b ## Descripción del producto {#min-compress-block-size} -Para [Método de codificación de datos:](../../engines/table-engines/mergetree-family/mergetree.md)" tabla. Para reducir la latencia al procesar consultas, un bloque se comprime al escribir la siguiente marca si su tamaño es al menos ‘min\_compress\_block\_size’. De forma predeterminada, 65.536. +Para [Método de codificación de datos:](../../engines/table-engines/mergetree-family/mergetree.md)" tabla. Para reducir la latencia al procesar consultas, un bloque se comprime al escribir la siguiente marca si su tamaño es al menos ‘min_compress_block_size’. De forma predeterminada, 65.536. -El tamaño real del bloque, si los datos sin comprimir son menores que ‘max\_compress\_block\_size’, no es menor que este valor y no menor que el volumen de datos para una marca. +El tamaño real del bloque, si los datos sin comprimir son menores que ‘max_compress_block_size’, no es menor que este valor y no menor que el volumen de datos para una marca. -Veamos un ejemplo. Supongamos que ‘index\_granularity’ se estableció en 8192 durante la creación de la tabla. +Veamos un ejemplo. Supongamos que ‘index_granularity’ se estableció en 8192 durante la creación de la tabla. -Estamos escribiendo una columna de tipo UInt32 (4 bytes por valor). Al escribir 8192 filas, el total será de 32 KB de datos. Como min\_compress\_block\_size = 65,536, se formará un bloque comprimido por cada dos marcas. +Estamos escribiendo una columna de tipo UInt32 (4 bytes por valor). Al escribir 8192 filas, el total será de 32 KB de datos. Como min_compress_block_size = 65,536, se formará un bloque comprimido por cada dos marcas. Estamos escribiendo una columna URL con el tipo String (tamaño promedio de 60 bytes por valor). Al escribir 8192 filas, el promedio será ligeramente inferior a 500 KB de datos. Como esto es más de 65,536, se formará un bloque comprimido para cada marca. En este caso, al leer datos del disco en el rango de una sola marca, los datos adicionales no se descomprimirán. Por lo general, no hay ninguna razón para cambiar esta configuración. -## max\_query\_size {#settings-max_query_size} +## max_query_size {#settings-max_query_size} La parte máxima de una consulta que se puede llevar a la RAM para analizar con el analizador SQL. La consulta INSERT también contiene datos para INSERT que es procesado por un analizador de secuencias independiente (que consume O(1) RAM), que no está incluido en esta restricción. Valor predeterminado: 256 KiB. -## interactive\_delay {#interactive-delay} +## interactive_delay {#interactive-delay} El intervalo en microsegundos para comprobar si la ejecución de la solicitud se ha cancelado y enviar el progreso. @@ -664,19 +664,19 @@ Tiempos de espera en segundos en el socket utilizado para comunicarse con el cli Valor predeterminado: 10, 300, 300. -## Cancel\_http\_readonly\_queries\_on\_client\_close {#cancel-http-readonly-queries-on-client-close} +## Cancel_http_readonly_queries_on_client_close {#cancel-http-readonly-queries-on-client-close} Cancels HTTP read-only queries (e.g. SELECT) when a client closes the connection without waiting for the response. Valor predeterminado: 0 -## poll\_interval {#poll-interval} +## poll_interval {#poll-interval} Bloquear en un bucle de espera durante el número especificado de segundos. Valor predeterminado: 10. -## max\_distributed\_connections {#max-distributed-connections} +## max_distributed_connections {#max-distributed-connections} El número máximo de conexiones simultáneas con servidores remotos para el procesamiento distribuido de una única consulta a una única tabla distribuida. Se recomienda establecer un valor no menor que el número de servidores en el clúster. @@ -684,20 +684,20 @@ Valor predeterminado: 1024. Los siguientes parámetros solo se usan al crear tablas distribuidas (y al iniciar un servidor), por lo que no hay ninguna razón para cambiarlas en tiempo de ejecución. -## Distributed\_connections\_pool\_size {#distributed-connections-pool-size} +## Distributed_connections_pool_size {#distributed-connections-pool-size} El número máximo de conexiones simultáneas con servidores remotos para el procesamiento distribuido de todas las consultas a una única tabla distribuida. Se recomienda establecer un valor no menor que el número de servidores en el clúster. Valor predeterminado: 1024. -## Conecte\_timeout\_with\_failover\_ms {#connect-timeout-with-failover-ms} +## Conecte_timeout_with_failover_ms {#connect-timeout-with-failover-ms} El tiempo de espera en milisegundos para conectarse a un servidor remoto para un motor de tablas distribuidas ‘shard’ y ‘replica’ secciones se utilizan en la definición de clúster. Si no tiene éxito, se realizan varios intentos para conectarse a varias réplicas. Valor predeterminado: 50. -## connections\_with\_failover\_max\_tries {#connections-with-failover-max-tries} +## connections_with_failover_max_tries {#connections-with-failover-max-tries} El número máximo de intentos de conexión con cada réplica para el motor de tablas distribuidas. @@ -708,19 +708,19 @@ Valor predeterminado: 3. Ya sea para contar valores extremos (los mínimos y máximos en columnas de un resultado de consulta). Acepta 0 o 1. De forma predeterminada, 0 (deshabilitado). Para obtener más información, consulte la sección “Extreme values”. -## Use\_uncompressed\_cache {#setting-use_uncompressed_cache} +## Use_uncompressed_cache {#setting-use_uncompressed_cache} Si se debe usar una memoria caché de bloques sin comprimir. Acepta 0 o 1. De forma predeterminada, 0 (deshabilitado). -El uso de la memoria caché sin comprimir (solo para tablas de la familia MergeTree) puede reducir significativamente la latencia y aumentar el rendimiento cuando se trabaja con un gran número de consultas cortas. Habilite esta configuración para los usuarios que envían solicitudes cortas frecuentes. También preste atención al [Uncompressed\_cache\_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuration parameter (only set in the config file) – the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed and the least-used data is automatically deleted. +El uso de la memoria caché sin comprimir (solo para tablas de la familia MergeTree) puede reducir significativamente la latencia y aumentar el rendimiento cuando se trabaja con un gran número de consultas cortas. Habilite esta configuración para los usuarios que envían solicitudes cortas frecuentes. También preste atención al [Uncompressed_cache_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuration parameter (only set in the config file) – the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed and the least-used data is automatically deleted. -Para consultas que leen al menos un volumen algo grande de datos (un millón de filas o más), la memoria caché sin comprimir se desactiva automáticamente para ahorrar espacio para consultas realmente pequeñas. Esto significa que puede mantener el ‘use\_uncompressed\_cache’ ajuste siempre establecido en 1. +Para consultas que leen al menos un volumen algo grande de datos (un millón de filas o más), la memoria caché sin comprimir se desactiva automáticamente para ahorrar espacio para consultas realmente pequeñas. Esto significa que puede mantener el ‘use_uncompressed_cache’ ajuste siempre establecido en 1. -## Reemplazar\_running\_query {#replace-running-query} +## Reemplazar_running_query {#replace-running-query} -Cuando se utiliza la interfaz HTTP, el ‘query\_id’ parámetro puede ser pasado. Se trata de cualquier cadena que sirva como identificador de consulta. -Si una consulta del mismo usuario ‘query\_id’ que ya existe en este momento, el comportamiento depende de la ‘replace\_running\_query’ parámetro. +Cuando se utiliza la interfaz HTTP, el ‘query_id’ parámetro puede ser pasado. Se trata de cualquier cadena que sirva como identificador de consulta. +Si una consulta del mismo usuario ‘query_id’ que ya existe en este momento, el comportamiento depende de la ‘replace_running_query’ parámetro. -`0` (default) – Throw an exception (don't allow the query to run if a query with the same ‘query\_id’ ya se está ejecutando). +`0` (default) – Throw an exception (don't allow the query to run if a query with the same ‘query_id’ ya se está ejecutando). `1` – Cancel the old query and start running the new one. @@ -728,13 +728,13 @@ El Yandex.Metrica utiliza este parámetro establecido en 1 para implementar suge ## Nombre de la red inalámbrica (SSID): {#stream-flush-interval-ms} -Funciona para tablas con streaming en el caso de un tiempo de espera, o cuando un subproceso genera [Max\_insert\_block\_size](#settings-max_insert_block_size) filas. +Funciona para tablas con streaming en el caso de un tiempo de espera, o cuando un subproceso genera [Max_insert_block_size](#settings-max_insert_block_size) filas. El valor predeterminado es 7500. Cuanto menor sea el valor, más a menudo los datos se vacían en la tabla. Establecer el valor demasiado bajo conduce a un rendimiento deficiente. -## load\_balancing {#settings-load_balancing} +## load_balancing {#settings-load_balancing} Especifica el algoritmo de selección de réplicas que se utiliza para el procesamiento de consultas distribuidas. @@ -787,31 +787,31 @@ Este algoritmo elige la primera réplica del conjunto o una réplica aleatoria s El `first_or_random` resuelve el problema del algoritmo `in_order` algoritmo. Con `in_order`, si una réplica se cae, la siguiente obtiene una carga doble mientras que las réplicas restantes manejan la cantidad habitual de tráfico. Cuando se utiliza el `first_or_random` algoritmo, la carga se distribuye uniformemente entre las réplicas que todavía están disponibles. -## prefer\_localhost\_replica {#settings-prefer-localhost-replica} +## prefer_localhost_replica {#settings-prefer-localhost-replica} Habilita/deshabilita el uso preferible de la réplica localhost al procesar consultas distribuidas. Valores posibles: - 1 — ClickHouse always sends a query to the localhost replica if it exists. -- 0 — ClickHouse uses the balancing strategy specified by the [load\_balancing](#settings-load_balancing) configuración. +- 0 — ClickHouse uses the balancing strategy specified by the [load_balancing](#settings-load_balancing) configuración. Valor predeterminado: 1. !!! warning "Advertencia" - Deshabilite esta configuración si usa [max\_parallel\_replicas](#settings-max_parallel_replicas). + Deshabilite esta configuración si usa [max_parallel_replicas](#settings-max_parallel_replicas). -## totals\_mode {#totals-mode} +## totals_mode {#totals-mode} -Cómo calcular TOTALS cuando HAVING está presente, así como cuando max\_rows\_to\_group\_by y group\_by\_overflow\_mode = ‘any’ están presentes. +Cómo calcular TOTALS cuando HAVING está presente, así como cuando max_rows_to_group_by y group_by_overflow_mode = ‘any’ están presentes. Vea la sección “WITH TOTALS modifier”. -## totals\_auto\_threshold {#totals-auto-threshold} +## totals_auto_threshold {#totals-auto-threshold} El umbral para `totals_mode = 'auto'`. Vea la sección “WITH TOTALS modifier”. -## max\_parallel\_replicas {#settings-max_parallel_replicas} +## max_parallel_replicas {#settings-max_parallel_replicas} El número máximo de réplicas para cada fragmento al ejecutar una consulta. Para obtener coherencia (para obtener diferentes partes de la misma división de datos), esta opción solo funciona cuando se establece la clave de muestreo. @@ -824,7 +824,7 @@ Habilitar la compilación de consultas. De forma predeterminada, 0 (deshabilitad La compilación solo se usa para parte de la canalización de procesamiento de consultas: para la primera etapa de agregación (GROUP BY). Si se compiló esta parte de la canalización, la consulta puede ejecutarse más rápido debido a la implementación de ciclos cortos y a las llamadas de función agregadas en línea. La mejora del rendimiento máximo (hasta cuatro veces más rápido en casos excepcionales) se ve para consultas con múltiples funciones agregadas simples. Por lo general, la ganancia de rendimiento es insignificante. En casos muy raros, puede ralentizar la ejecución de la consulta. -## min\_count\_to\_compile {#min-count-to-compile} +## min_count_to_compile {#min-count-to-compile} ¿Cuántas veces usar potencialmente un fragmento de código compilado antes de ejecutar la compilación? Por defecto, 3. For testing, the value can be set to 0: compilation runs synchronously and the query waits for the end of the compilation process before continuing execution. For all other cases, use values ​​starting with 1. Compilation normally takes about 5-10 seconds. @@ -833,27 +833,27 @@ Si el valor es 1 o más, la compilación se produce de forma asíncrona en un su Se requiere código compilado para cada combinación diferente de funciones agregadas utilizadas en la consulta y el tipo de claves en la cláusula GROUP BY. The results of the compilation are saved in the build directory in the form of .so files. There is no restriction on the number of compilation results since they don't use very much space. Old results will be used after server restarts, except in the case of a server upgrade – in this case, the old results are deleted. -## output\_format\_json\_quote\_64bit\_integers {#session_settings-output_format_json_quote_64bit_integers} +## output_format_json_quote_64bit_integers {#session_settings-output_format_json_quote_64bit_integers} Si el valor es true, los enteros aparecen entre comillas cuando se usan los formatos JSON\* Int64 y UInt64 (por compatibilidad con la mayoría de las implementaciones de JavaScript); de lo contrario, los enteros se generan sin las comillas. -## Formato\_csv\_delimiter {#settings-format_csv_delimiter} +## Formato_csv_delimiter {#settings-format_csv_delimiter} El carácter interpretado como un delimitador en los datos CSV. De forma predeterminada, el delimitador es `,`. -## input\_format\_csv\_unquoted\_null\_literal\_as\_null {#settings-input_format_csv_unquoted_null_literal_as_null} +## input_format_csv_unquoted_null_literal_as_null {#settings-input_format_csv_unquoted_null_literal_as_null} Para el formato de entrada CSV, habilita o deshabilita el análisis de `NULL` como literal (sinónimo de `\N`). -## output\_format\_csv\_crlf\_end\_of\_line {#settings-output-format-csv-crlf-end-of-line} +## output_format_csv_crlf_end_of_line {#settings-output-format-csv-crlf-end-of-line} Utilice el separador de línea de estilo DOS / Windows (CRLF) en CSV en lugar de estilo Unix (LF). -## output\_format\_tsv\_crlf\_end\_of\_line {#settings-output-format-tsv-crlf-end-of-line} +## output_format_tsv_crlf_end_of_line {#settings-output-format-tsv-crlf-end-of-line} Utilice el separador de línea de estilo DOC / Windows (CRLF) en TSV en lugar del estilo Unix (LF). -## insert\_quorum {#settings-insert_quorum} +## insert_quorum {#settings-insert_quorum} Habilita las escrituras de quórum. @@ -868,7 +868,7 @@ Quorum escribe Todas las réplicas del quórum son consistentes, es decir, contienen datos de todas las réplicas anteriores `INSERT` consulta. El `INSERT` la secuencia está linealizada. -Al leer los datos escritos desde el `insert_quorum` usted puede utilizar el [select\_sequential\_consistency](#settings-select_sequential_consistency) opcion. +Al leer los datos escritos desde el `insert_quorum` usted puede utilizar el [select_sequential_consistency](#settings-select_sequential_consistency) opcion. ClickHouse genera una excepción @@ -877,10 +877,10 @@ ClickHouse genera una excepción Ver también: -- [insert\_quorum\_timeout](#settings-insert_quorum_timeout) -- [select\_sequential\_consistency](#settings-select_sequential_consistency) +- [insert_quorum_timeout](#settings-insert_quorum_timeout) +- [select_sequential_consistency](#settings-select_sequential_consistency) -## insert\_quorum\_timeout {#settings-insert_quorum_timeout} +## insert_quorum_timeout {#settings-insert_quorum_timeout} Escribir en tiempo de espera de quórum en segundos. Si el tiempo de espera ha pasado y aún no se ha realizado ninguna escritura, ClickHouse generará una excepción y el cliente debe repetir la consulta para escribir el mismo bloque en la misma réplica o en cualquier otra réplica. @@ -888,10 +888,10 @@ Valor predeterminado: 60 segundos. Ver también: -- [insert\_quorum](#settings-insert_quorum) -- [select\_sequential\_consistency](#settings-select_sequential_consistency) +- [insert_quorum](#settings-insert_quorum) +- [select_sequential_consistency](#settings-select_sequential_consistency) -## select\_sequential\_consistency {#settings-select_sequential_consistency} +## select_sequential_consistency {#settings-select_sequential_consistency} Habilita o deshabilita la coherencia secuencial para `SELECT` consulta: @@ -908,10 +908,10 @@ Cuando se habilita la coherencia secuencial, ClickHouse permite al cliente ejecu Ver también: -- [insert\_quorum](#settings-insert_quorum) -- [insert\_quorum\_timeout](#settings-insert_quorum_timeout) +- [insert_quorum](#settings-insert_quorum) +- [insert_quorum_timeout](#settings-insert_quorum_timeout) -## insert\_deduplicate {#settings-insert-deduplicate} +## insert_deduplicate {#settings-insert-deduplicate} Habilita o deshabilita la desduplicación de bloques `INSERT` (para tablas replicadas\* @@ -924,7 +924,7 @@ Valor predeterminado: 1. De forma predeterminada, los bloques insertados en tablas replicadas `INSERT` declaración se deduplican (ver [Replicación de datos](../../engines/table-engines/mergetree-family/replication.md)). -## deduplicate\_blocks\_in\_dependent\_materialized\_views {#settings-deduplicate-blocks-in-dependent-materialized-views} +## deduplicate_blocks_in_dependent_materialized_views {#settings-deduplicate-blocks-in-dependent-materialized-views} Habilita o deshabilita la comprobación de desduplicación para las vistas materializadas que reciben datos de tablas replicadas\*. @@ -986,7 +986,7 @@ Valores posibles: Valor predeterminado: 0. -## count\_distinct\_implementation {#settings-count_distinct_implementation} +## count_distinct_implementation {#settings-count_distinct_implementation} Especifica cuál de las `uniq*` se deben utilizar para realizar el [COUNT(DISTINCT …)](../../sql-reference/aggregate-functions/reference.md#agg_function-count) construcción. @@ -1000,7 +1000,7 @@ Valores posibles: Valor predeterminado: `uniqExact`. -## skip\_unavailable\_shards {#settings-skip_unavailable_shards} +## skip_unavailable_shards {#settings-skip_unavailable_shards} Habilita o deshabilita la omisión silenciosa de fragmentos no disponibles. @@ -1030,13 +1030,13 @@ Valores posibles: Valor predeterminado: 0. -## Optize\_skip\_unused\_shards {#settings-optimize_skip_unused_shards} +## Optize_skip_unused_shards {#settings-optimize_skip_unused_shards} Habilita o deshabilita la omisión de fragmentos no utilizados para las consultas SELECT que tienen la condición de clave de fragmentación en PREWHERE / WHERE (supone que los datos se distribuyen mediante clave de fragmentación, de lo contrario no hacer nada). Valor predeterminado: 0 -## Fuerza\_optimize\_skip\_unused\_shards {#settings-force_optimize_skip_unused_shards} +## Fuerza_optimize_skip_unused_shards {#settings-force_optimize_skip_unused_shards} Habilita o deshabilita la ejecución de consultas si [`optimize_skip_unused_shards`](#settings-optimize_skip_unused_shards) no es posible omitir fragmentos no utilizados. Si la omisión no es posible y la configuración está habilitada, se lanzará una excepción. @@ -1048,7 +1048,7 @@ Valores posibles: Valor predeterminado: 0 -## Optize\_throw\_if\_noop {#setting-optimize_throw_if_noop} +## Optize_throw_if_noop {#setting-optimize_throw_if_noop} Habilita o deshabilita el lanzamiento de una excepción [OPTIMIZE](../../sql-reference/statements/misc.md#misc_operations-optimize) la consulta no realizó una fusión. @@ -1061,19 +1061,19 @@ Valores posibles: Valor predeterminado: 0. -## distributed\_replica\_error\_half\_life {#settings-distributed_replica_error_half_life} +## distributed_replica_error_half_life {#settings-distributed_replica_error_half_life} - Tipo: segundos - Valor predeterminado: 60 segundos -Controla la rapidez con la que se ponen a cero los errores en las tablas distribuidas. Si una réplica no está disponible durante algún tiempo, acumula 5 errores y distribut\_replica\_error\_half\_life se establece en 1 segundo, la réplica se considera normal 3 segundos después del último error. +Controla la rapidez con la que se ponen a cero los errores en las tablas distribuidas. Si una réplica no está disponible durante algún tiempo, acumula 5 errores y distribut_replica_error_half_life se establece en 1 segundo, la réplica se considera normal 3 segundos después del último error. Ver también: - [Motor de tabla distribuido](../../engines/table-engines/special/distributed.md) -- [distributed\_replica\_error\_cap](#settings-distributed_replica_error_cap) +- [distributed_replica_error_cap](#settings-distributed_replica_error_cap) -## distributed\_replica\_error\_cap {#settings-distributed_replica_error_cap} +## distributed_replica_error_cap {#settings-distributed_replica_error_cap} - Tipo: unsigned int - Valor predeterminado: 1000 @@ -1083,9 +1083,9 @@ El recuento de errores de cada réplica está limitado a este valor, lo que impi Ver también: - [Motor de tabla distribuido](../../engines/table-engines/special/distributed.md) -- [distributed\_replica\_error\_half\_life](#settings-distributed_replica_error_half_life) +- [distributed_replica_error_half_life](#settings-distributed_replica_error_half_life) -## Distributed\_directory\_monitor\_sleep\_time\_ms {#distributed_directory_monitor_sleep_time_ms} +## Distributed_directory_monitor_sleep_time_ms {#distributed_directory_monitor_sleep_time_ms} Intervalo base para el [Distribuido](../../engines/table-engines/special/distributed.md) motor de tabla para enviar datos. El intervalo real crece exponencialmente en caso de errores. @@ -1095,9 +1095,9 @@ Valores posibles: Valor predeterminado: 100 milisegundos. -## Distributed\_directory\_monitor\_max\_sleep\_time\_ms {#distributed_directory_monitor_max_sleep_time_ms} +## Distributed_directory_monitor_max_sleep_time_ms {#distributed_directory_monitor_max_sleep_time_ms} -Intervalo máximo para el [Distribuido](../../engines/table-engines/special/distributed.md) motor de tabla para enviar datos. Limita el crecimiento exponencial del intervalo establecido en el [Distributed\_directory\_monitor\_sleep\_time\_ms](#distributed_directory_monitor_sleep_time_ms) configuración. +Intervalo máximo para el [Distribuido](../../engines/table-engines/special/distributed.md) motor de tabla para enviar datos. Limita el crecimiento exponencial del intervalo establecido en el [Distributed_directory_monitor_sleep_time_ms](#distributed_directory_monitor_sleep_time_ms) configuración. Valores posibles: @@ -1105,7 +1105,7 @@ Valores posibles: Valor predeterminado: 30000 milisegundos (30 segundos). -## distributed\_directory\_monitor\_batch\_inserts {#distributed_directory_monitor_batch_inserts} +## distributed_directory_monitor_batch_inserts {#distributed_directory_monitor_batch_inserts} Habilita/deshabilita el envío de datos insertados en lotes. @@ -1118,7 +1118,7 @@ Valores posibles: Valor predeterminado: 0. -## os\_thread\_priority {#setting-os-thread-priority} +## os_thread_priority {#setting-os-thread-priority} Establece la prioridad ([agradable](https://en.wikipedia.org/wiki/Nice_(Unix))) para subprocesos que ejecutan consultas. El programador del sistema operativo considera esta prioridad al elegir el siguiente hilo para ejecutar en cada núcleo de CPU disponible. @@ -1133,7 +1133,7 @@ Los valores más bajos significan mayor prioridad. Hilos con bajo `nice` Los val Valor predeterminado: 0. -## query\_profiler\_real\_time\_period\_ns {#query_profiler_real_time_period_ns} +## query_profiler_real_time_period_ns {#query_profiler_real_time_period_ns} Establece el período para un temporizador de reloj real del [perfilador de consultas](../../operations/optimizing-performance/sampling-query-profiler.md). El temporizador de reloj real cuenta el tiempo del reloj de pared. @@ -1154,7 +1154,7 @@ Valor predeterminado: 1000000000 nanosegundos (una vez por segundo). Ver también: -- Tabla del sistema [trace\_log](../../operations/system-tables.md#system_tables-trace_log) +- Tabla del sistema [trace_log](../../operations/system-tables.md#system_tables-trace_log) ## Los resultados de la prueba {#query_profiler_cpu_time_period_ns} @@ -1177,9 +1177,9 @@ Valor predeterminado: 1000000000 nanosegundos. Ver también: -- Tabla del sistema [trace\_log](../../operations/system-tables.md#system_tables-trace_log) +- Tabla del sistema [trace_log](../../operations/system-tables.md#system_tables-trace_log) -## allow\_introspection\_functions {#settings-allow_introspection_functions} +## allow_introspection_functions {#settings-allow_introspection_functions} Habilita deshabilita [funciones de introspecciones](../../sql-reference/functions/introspection.md) para la creación de perfiles de consultas. @@ -1193,9 +1193,9 @@ Valor predeterminado: 0. **Ver también** - [Analizador de consultas de muestreo](../optimizing-performance/sampling-query-profiler.md) -- Tabla del sistema [trace\_log](../../operations/system-tables.md#system_tables-trace_log) +- Tabla del sistema [trace_log](../../operations/system-tables.md#system_tables-trace_log) -## input\_format\_parallel\_parsing {#input-format-parallel-parsing} +## input_format_parallel_parsing {#input-format-parallel-parsing} - Tipo: bool - Valor predeterminado: True @@ -1241,7 +1241,7 @@ Tipo: URL Valor predeterminado: Vacío -## background\_pool\_size {#background_pool_size} +## background_pool_size {#background_pool_size} Establece el número de subprocesos que realizan operaciones en segundo plano en motores de tabla (por ejemplo, fusiona [Motor MergeTree](../../engines/table-engines/mergetree-family/index.md) tabla). Esta configuración se aplica al inicio del servidor ClickHouse y no se puede cambiar en una sesión de usuario. Al ajustar esta configuración, puede administrar la carga de la CPU y el disco. Un tamaño de grupo más pequeño utiliza menos recursos de CPU y disco, pero los procesos en segundo plano avanzan más lentamente, lo que eventualmente podría afectar el rendimiento de la consulta. diff --git a/docs/es/operations/system-tables.md b/docs/es/operations/system-tables.md index 6f487ce66e3..18e7f7227da 100644 --- a/docs/es/operations/system-tables.md +++ b/docs/es/operations/system-tables.md @@ -13,7 +13,7 @@ Las tablas del sistema no tienen archivos con datos en el disco o archivos con m Las tablas del sistema son de solo lectura. Están ubicados en el ‘system’ base. -## sistema.asynchronous\_metrics {#system_tables-asynchronous_metrics} +## sistema.asynchronous_metrics {#system_tables-asynchronous_metrics} Contiene métricas que se calculan periódicamente en segundo plano. Por ejemplo, la cantidad de RAM en uso. @@ -48,7 +48,7 @@ SELECT * FROM system.asynchronous_metrics LIMIT 10 - [Monitoreo](monitoring.md) — Base concepts of ClickHouse monitoring. - [sistema.métricas](#system_tables-metrics) — Contains instantly calculated metrics. - [sistema.evento](#system_tables-events) — Contains a number of events that have occurred. -- [sistema.metric\_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [sistema.metric_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. ## sistema.Cluster {#system-clusters} @@ -72,8 +72,8 @@ Tenga en cuenta que `errors_count` se actualiza una vez por consulta al clúster **Ver también** - [Motor de tabla distribuido](../engines/table-engines/special/distributed.md) -- [distributed\_replica\_error\_cap configuración](settings/settings.md#settings-distributed_replica_error_cap) -- [distributed\_replica\_error\_half\_life configuración](settings/settings.md#settings-distributed_replica_error_half_life) +- [distributed_replica_error_cap configuración](settings/settings.md#settings-distributed_replica_error_cap) +- [distributed_replica_error_half_life configuración](settings/settings.md#settings-distributed_replica_error_half_life) ## sistema.columna {#system-columns} @@ -145,7 +145,7 @@ Esta tabla contiene una sola columna String llamada ‘name’ – the name of a Cada base de datos que el servidor conoce tiene una entrada correspondiente en la tabla. Esta tabla del sistema se utiliza para implementar el `SHOW DATABASES` consulta. -## sistema.detached\_parts {#system_tables-detached_parts} +## sistema.detached_parts {#system_tables-detached_parts} Contiene información sobre piezas separadas de [Método de codificación de datos:](../engines/table-engines/mergetree-family/mergetree.md) tabla. El `reason` columna especifica por qué se separó la pieza. Para las piezas separadas por el usuario, el motivo está vacío. Tales partes se pueden unir con [ALTER TABLE ATTACH PARTITION\|PART](../sql-reference/statements/alter.md#alter_attach-partition) comando. Para obtener la descripción de otras columnas, consulte [sistema.parte](#system_tables-parts). Si el nombre de la pieza no es válido, los valores de algunas columnas pueden ser `NULL`. Tales partes se pueden eliminar con [ALTER TABLE DROP DETACHED PART](../sql-reference/statements/alter.md#alter_drop-detached). @@ -239,9 +239,9 @@ SELECT * FROM system.events LIMIT 5 **Ver también** -- [sistema.asynchronous\_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [sistema.asynchronous_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. - [sistema.métricas](#system_tables-metrics) — Contains instantly calculated metrics. -- [sistema.metric\_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [sistema.metric_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. - [Monitoreo](monitoring.md) — Base concepts of ClickHouse monitoring. ## sistema.función {#system-functions} @@ -253,9 +253,9 @@ Columna: - `name`(`String`) – The name of the function. - `is_aggregate`(`UInt8`) — Whether the function is aggregate. -## sistema.graphite\_retentions {#system-graphite-retentions} +## sistema.graphite_retentions {#system-graphite-retentions} -Contiene información sobre los parámetros [graphite\_rollup](server-configuration-parameters/settings.md#server_configuration_parameters-graphite) que se utilizan en tablas con [\*GraphiteMergeTree](../engines/table-engines/mergetree-family/graphitemergetree.md) motor. +Contiene información sobre los parámetros [graphite_rollup](server-configuration-parameters/settings.md#server_configuration_parameters-graphite) que se utilizan en tablas con [\*GraphiteMergeTree](../engines/table-engines/mergetree-family/graphitemergetree.md) motor. Columna: @@ -324,12 +324,12 @@ SELECT * FROM system.metrics LIMIT 10 **Ver también** -- [sistema.asynchronous\_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [sistema.asynchronous_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. - [sistema.evento](#system_tables-events) — Contains a number of events that occurred. -- [sistema.metric\_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [sistema.metric_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. - [Monitoreo](monitoring.md) — Base concepts of ClickHouse monitoring. -## sistema.metric\_log {#system_tables-metric_log} +## sistema.metric_log {#system_tables-metric_log} Contiene el historial de valores de métricas de tablas `system.metrics` y `system.events`, periódicamente enjuagado al disco. Para activar la recopilación de historial de métricas en `system.metric_log`, crear `/etc/clickhouse-server/config.d/metric_log.xml` con el siguiente contenido: @@ -380,7 +380,7 @@ CurrentMetric_ReplicatedChecks: 0 **Ver también** -- [sistema.asynchronous\_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [sistema.asynchronous_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. - [sistema.evento](#system_tables-events) — Contains a number of events that occurred. - [sistema.métricas](#system_tables-metrics) — Contains instantly calculated metrics. - [Monitoreo](monitoring.md) — Base concepts of ClickHouse monitoring. @@ -391,7 +391,7 @@ Esta tabla contiene una única columna UInt64 llamada ‘number’ que contiene Puede usar esta tabla para pruebas, o si necesita hacer una búsqueda de fuerza bruta. Las lecturas de esta tabla no están paralelizadas. -## sistema.Números\_mt {#system-numbers-mt} +## sistema.Números_mt {#system-numbers-mt} Lo mismo que ‘system.numbers’ pero las lecturas están paralelizadas. Los números se pueden devolver en cualquier orden. Se utiliza para pruebas. @@ -483,9 +483,9 @@ Columna: - `marks_size` (`UInt64`) – Alias for `marks_bytes`. -## sistema.part\_log {#system_tables-part-log} +## sistema.part_log {#system_tables-part-log} -El `system.part_log` se crea sólo si el [part\_log](server-configuration-parameters/settings.md#server_configuration_parameters-part-log) se especifica la configuración del servidor. +El `system.part_log` se crea sólo si el [part_log](server-configuration-parameters/settings.md#server_configuration_parameters-part-log) se especifica la configuración del servidor. Esta tabla contiene información sobre eventos que ocurrieron con [partes de datos](../engines/table-engines/mergetree-family/custom-partitioning-key.md) en el [Método de codificación de datos:](../engines/table-engines/mergetree-family/mergetree.md) tablas familiares, como agregar o fusionar datos. @@ -532,7 +532,7 @@ Columna: - `query` (String) – The query text. For `INSERT`, no incluye los datos para insertar. - `query_id` (String) – Query ID, if defined. -## sistema.text\_log {#system_tables-text_log} +## sistema.text_log {#system_tables-text_log} Contiene entradas de registro. El nivel de registro que va a esta tabla se puede limitar con `text_log.level` configuración del servidor. @@ -559,16 +559,16 @@ Columna: - `source_file` (`LowCardinality(String)`) - Archivo de origen desde el que se realizó el registro. - `source_line` (`UInt64`) - Línea de origen desde la que se realizó el registro. -## sistema.query\_log {#system_tables-query_log} +## sistema.query_log {#system_tables-query_log} Contiene información sobre la ejecución de consultas. Para cada consulta, puede ver la hora de inicio del procesamiento, la duración del procesamiento, los mensajes de error y otra información. !!! note "Nota" La tabla no contiene datos de entrada para `INSERT` consulta. -ClickHouse crea esta tabla sólo si el [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) se especifica el parámetro server. Este parámetro establece las reglas de registro, como el intervalo de registro o el nombre de la tabla en la que se registrarán las consultas. +ClickHouse crea esta tabla sólo si el [query_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) se especifica el parámetro server. Este parámetro establece las reglas de registro, como el intervalo de registro o el nombre de la tabla en la que se registrarán las consultas. -Para habilitar el registro de consultas, [Log\_queries](settings/settings.md#settings-log-queries) parámetro a 1. Para obtener más información, consulte el [Configuración](settings/settings.md) apartado. +Para habilitar el registro de consultas, [Log_queries](settings/settings.md#settings-log-queries) parámetro a 1. Para obtener más información, consulte el [Configuración](settings/settings.md) apartado. El `system.query_log` tabla registra dos tipos de consultas: @@ -636,14 +636,14 @@ Cada consulta crea una o dos filas en el `query_log` tabla, dependiendo del esta 2. Si se produjo un error durante el procesamiento de la consulta, se crean dos eventos con los tipos 1 y 4. 3. Si se produjo un error antes de iniciar la consulta, se crea un solo evento con el tipo 3. -De forma predeterminada, los registros se agregan a la tabla a intervalos de 7,5 segundos. Puede establecer este intervalo en el [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) configuración del servidor (consulte el `flush_interval_milliseconds` parámetro). Para vaciar los registros a la fuerza desde el búfer de memoria a la tabla, utilice `SYSTEM FLUSH LOGS` consulta. +De forma predeterminada, los registros se agregan a la tabla a intervalos de 7,5 segundos. Puede establecer este intervalo en el [query_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) configuración del servidor (consulte el `flush_interval_milliseconds` parámetro). Para vaciar los registros a la fuerza desde el búfer de memoria a la tabla, utilice `SYSTEM FLUSH LOGS` consulta. Cuando la tabla se elimina manualmente, se creará automáticamente sobre la marcha. Tenga en cuenta que se eliminarán todos los registros anteriores. !!! note "Nota" El período de almacenamiento para los registros es ilimitado. Los registros no se eliminan automáticamente de la tabla. Debe organizar la eliminación de registros obsoletos usted mismo. -Puede especificar una clave de partición arbitraria `system.query_log` mesa en el [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) configuración del servidor (consulte el `partition_by` parámetro). +Puede especificar una clave de partición arbitraria `system.query_log` mesa en el [query_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) configuración del servidor (consulte el `partition_by` parámetro). ## sistema.Sistema abierto {#system_tables-query-thread-log} @@ -651,7 +651,7 @@ La tabla contiene información sobre cada subproceso de ejecución de consultas. ClickHouse crea esta tabla sólo si el [Sistema abierto.](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) se especifica el parámetro server. Este parámetro establece las reglas de registro, como el intervalo de registro o el nombre de la tabla en la que se registrarán las consultas. -Para habilitar el registro de consultas, [Log\_query\_threads](settings/settings.md#settings-log-query-threads) parámetro a 1. Para obtener más información, consulte el [Configuración](settings/settings.md) apartado. +Para habilitar el registro de consultas, [Log_query_threads](settings/settings.md#settings-log-query-threads) parámetro a 1. Para obtener más información, consulte el [Configuración](settings/settings.md) apartado. Columna: @@ -710,11 +710,11 @@ Cuando la tabla se elimina manualmente, se creará automáticamente sobre la mar Puede especificar una clave de partición arbitraria `system.query_thread_log` mesa en el [Sistema abierto.](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) configuración del servidor (consulte el `partition_by` parámetro). -## sistema.trace\_log {#system_tables-trace_log} +## sistema.trace_log {#system_tables-trace_log} Contiene seguimientos de pila recopilados por el generador de perfiles de consultas de muestreo. -ClickHouse crea esta tabla cuando el [trace\_log](server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) se establece la sección de configuración del servidor. También el [query\_profiler\_real\_time\_period\_ns](settings/settings.md#query_profiler_real_time_period_ns) y [Los resultados de la prueba](settings/settings.md#query_profiler_cpu_time_period_ns) los ajustes deben establecerse. +ClickHouse crea esta tabla cuando el [trace_log](server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) se establece la sección de configuración del servidor. También el [query_profiler_real_time_period_ns](settings/settings.md#query_profiler_real_time_period_ns) y [Los resultados de la prueba](settings/settings.md#query_profiler_cpu_time_period_ns) los ajustes deben establecerse. Para analizar los registros, utilice el `addressToLine`, `addressToSymbol` y `demangle` funciones de inspección. @@ -737,7 +737,7 @@ Columna: - `thread_number` ([UInt32](../sql-reference/data-types/int-uint.md)) — Thread identifier. -- `query_id` ([Cadena](../sql-reference/data-types/string.md)) — Query identifier that can be used to get details about a query that was running from the [query\_log](#system_tables-query_log) tabla del sistema. +- `query_id` ([Cadena](../sql-reference/data-types/string.md)) — Query identifier that can be used to get details about a query that was running from the [query_log](#system_tables-query_log) tabla del sistema. - `trace` ([Matriz (UInt64)](../sql-reference/data-types/array.md)) — Stack trace at the moment of sampling. Each element is a virtual memory address inside ClickHouse server process. @@ -824,7 +824,7 @@ Columna: - `parts_to_check` (`UInt32`) - El número de partes de datos en la cola para la verificación. Una pieza se coloca en la cola de verificación si existe la sospecha de que podría estar dañada. - `zookeeper_path` (`String`) - Ruta de acceso a los datos de la tabla en ZooKeeper. - `replica_name` (`String`) - Nombre de réplica en ZooKeeper. Diferentes réplicas de la misma tabla tienen diferentes nombres. -- `replica_path` (`String`) - Ruta de acceso a los datos de réplica en ZooKeeper. Lo mismo que concatenar ‘zookeeper\_path/replicas/replica\_path’. +- `replica_path` (`String`) - Ruta de acceso a los datos de réplica en ZooKeeper. Lo mismo que concatenar ‘zookeeper_path/replicas/replica_path’. - `columns_version` (`Int32`) - Número de versión de la estructura de la tabla. Indica cuántas veces se realizó ALTER. Si las réplicas tienen versiones diferentes, significa que algunas réplicas aún no han hecho todas las ALTER. - `queue_size` (`UInt32`) - Tamaño de la cola para las operaciones en espera de ser realizadas. Las operaciones incluyen insertar bloques de datos, fusiones y otras acciones. Por lo general, coincide con `future_parts`. - `inserts_in_queue` (`UInt32`) - Número de inserciones de bloques de datos que deben realizarse. Las inserciones generalmente se replican con bastante rapidez. Si este número es grande, significa que algo anda mal. @@ -845,7 +845,7 @@ Las siguientes 4 columnas tienen un valor distinto de cero solo cuando hay una s - `active_replicas` (`UInt8`) - El número de réplicas de esta tabla que tienen una sesión en ZooKeeper (es decir, el número de réplicas en funcionamiento). Si solicita todas las columnas, la tabla puede funcionar un poco lentamente, ya que se realizan varias lecturas de ZooKeeper para cada fila. -Si no solicita las últimas 4 columnas (log\_max\_index, log\_pointer, total\_replicas, active\_replicas), la tabla funciona rápidamente. +Si no solicita las últimas 4 columnas (log_max_index, log_pointer, total_replicas, active_replicas), la tabla funciona rápidamente. Por ejemplo, puede verificar que todo funcione correctamente de esta manera: @@ -932,7 +932,7 @@ SELECT * FROM system.settings WHERE changed AND name='load_balancing' - [Permisos para consultas](settings/permissions-for-queries.md#settings_readonly) - [Restricciones en la configuración](settings/constraints-on-settings.md) -## sistema.table\_engines {#system.table_engines} +## sistema.table_engines {#system.table_engines} ``` text ┌─name───────────────────┬─value───────┐ @@ -943,7 +943,7 @@ SELECT * FROM system.settings WHERE changed AND name='load_balancing' └────────────────────────┴─────────────┘ ``` -## sistema.merge\_tree\_settings {#system-merge_tree_settings} +## sistema.merge_tree_settings {#system-merge_tree_settings} Contiene información sobre la configuración `MergeTree` tabla. @@ -955,7 +955,7 @@ Columna: - `type` (String) — Setting type (implementation specific string value). - `changed` (UInt8) — Whether the setting was explicitly defined in the config or explicitly changed. -## sistema.table\_engines {#system-table-engines} +## sistema.table_engines {#system-table-engines} Contiene la descripción de los motores de tablas admitidos por el servidor y su información de soporte de características. @@ -1118,17 +1118,17 @@ La tabla contiene información sobre [mutación](../sql-reference/statements/alt **base**, **tabla** - El nombre de la base de datos y la tabla a la que se aplicó la mutación. -**mutation\_id** - La identificación de la mutación. Para las tablas replicadas, estos identificadores corresponden a los nombres de znode `/mutations/` directorio en ZooKeeper. Para las tablas no duplicadas, los ID corresponden a los nombres de archivo en el directorio de datos de la tabla. +**mutation_id** - La identificación de la mutación. Para las tablas replicadas, estos identificadores corresponden a los nombres de znode `/mutations/` directorio en ZooKeeper. Para las tablas no duplicadas, los ID corresponden a los nombres de archivo en el directorio de datos de la tabla. **comando** - La cadena de comandos de mutación (la parte de la consulta después de `ALTER TABLE [db.]table`). -**create\_time** - Cuando este comando de mutación fue enviado para su ejecución. +**create_time** - Cuando este comando de mutación fue enviado para su ejecución. -**block\_numbers.partition\_id**, **block\_numbers.numero** - Una columna anidada. Para las mutaciones de tablas replicadas, contiene un registro para cada partición: el ID de partición y el número de bloque que fue adquirido por la mutación (en cada partición, solo se mutarán las partes que contienen bloques con números menores que el número de bloque adquirido por la mutación en esa partición). En tablas no replicadas, los números de bloque en todas las particiones forman una sola secuencia. Esto significa que para las mutaciones de tablas no replicadas, la columna contendrá un registro con un solo número de bloque adquirido por la mutación. +**block_numbers.partition_id**, **block_numbers.numero** - Una columna anidada. Para las mutaciones de tablas replicadas, contiene un registro para cada partición: el ID de partición y el número de bloque que fue adquirido por la mutación (en cada partición, solo se mutarán las partes que contienen bloques con números menores que el número de bloque adquirido por la mutación en esa partición). En tablas no replicadas, los números de bloque en todas las particiones forman una sola secuencia. Esto significa que para las mutaciones de tablas no replicadas, la columna contendrá un registro con un solo número de bloque adquirido por la mutación. -**partes\_a\_do** - El número de partes de datos que deben mutarse para que finalice la mutación. +**partes_a_do** - El número de partes de datos que deben mutarse para que finalice la mutación. -**is\_done** - Es la mutación hecho? Tenga en cuenta que incluso si `parts_to_do = 0` es posible que aún no se haya realizado una mutación de una tabla replicada debido a un INSERT de larga ejecución que creará una nueva parte de datos que deberá mutarse. +**is_done** - Es la mutación hecho? Tenga en cuenta que incluso si `parts_to_do = 0` es posible que aún no se haya realizado una mutación de una tabla replicada debido a un INSERT de larga ejecución que creará una nueva parte de datos que deberá mutarse. Si hubo problemas con la mutación de algunas partes, las siguientes columnas contienen información adicional: @@ -1150,7 +1150,7 @@ Columna: - `total_space` ([UInt64](../sql-reference/data-types/int-uint.md)) — Disk volume in bytes. - `keep_free_space` ([UInt64](../sql-reference/data-types/int-uint.md)) — Amount of disk space that should stay free on disk in bytes. Defined in the `keep_free_space_bytes` parámetro de configuración del disco. -## sistema.almacenamiento\_policies {#system_tables-storage_policies} +## sistema.almacenamiento_policies {#system_tables-storage_policies} Contiene información sobre las directivas de almacenamiento y los volúmenes [configuración del servidor](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes_configure). diff --git a/docs/es/operations/tips.md b/docs/es/operations/tips.md index 4d3e76c7d62..deb226450aa 100644 --- a/docs/es/operations/tips.md +++ b/docs/es/operations/tips.md @@ -59,7 +59,7 @@ Al crear RAID-10, seleccione el `far` diseño. Si su presupuesto lo permite, elija RAID-10. Si tiene más de 4 discos, utilice RAID-6 (preferido) o RAID-50, en lugar de RAID-5. -Cuando use RAID-5, RAID-6 o RAID-50, siempre aumente stripe\_cache\_size, ya que el valor predeterminado generalmente no es la mejor opción. +Cuando use RAID-5, RAID-6 o RAID-50, siempre aumente stripe_cache_size, ya que el valor predeterminado generalmente no es la mejor opción. ``` bash $ echo 4096 | sudo tee /sys/block/md2/md/stripe_cache_size diff --git a/docs/es/operations/troubleshooting.md b/docs/es/operations/troubleshooting.md index 04b00e4a5cd..9e8d2caca59 100644 --- a/docs/es/operations/troubleshooting.md +++ b/docs/es/operations/troubleshooting.md @@ -105,7 +105,7 @@ Comprobar: - Configuración del punto final. - Comprobar [listen\_host](server-configuration-parameters/settings.md#server_configuration_parameters-listen_host) y [Tcp\_port](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) configuración. + Comprobar [listen_host](server-configuration-parameters/settings.md#server_configuration_parameters-listen_host) y [Tcp_port](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) configuración. El servidor ClickHouse acepta conexiones localhost solo de forma predeterminada. @@ -117,7 +117,7 @@ Comprobar: Comprobar: - - El [Tcp\_port\_secure](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) configuración. + - El [Tcp_port_secure](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) configuración. - Ajustes para [Sertificados SSL](server-configuration-parameters/settings.md#server_configuration_parameters-openssl). Utilice los parámetros adecuados mientras se conecta. Por ejemplo, utilice el `port_secure` parámetro con `clickhouse_client`. diff --git a/docs/es/sql-reference/aggregate-functions/parametric-functions.md b/docs/es/sql-reference/aggregate-functions/parametric-functions.md index 9aac66ae47e..ea32920401b 100644 --- a/docs/es/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/es/sql-reference/aggregate-functions/parametric-functions.md @@ -494,6 +494,6 @@ Solution: Write in the GROUP BY query SearchPhrase HAVING uniqUpTo(4)(UserID) >= [Artículo Original](https://clickhouse.tech/docs/en/query_language/agg_functions/parametric_functions/) -## sumMapFiltered(keys\_to\_keep)(claves, valores) {#summapfilteredkeys-to-keepkeys-values} +## sumMapFiltered(keys_to_keep)(claves, valores) {#summapfilteredkeys-to-keepkeys-values} El mismo comportamiento que [sumMap](reference.md#agg_functions-summap) excepto que una matriz de claves se pasa como un parámetro. Esto puede ser especialmente útil cuando se trabaja con una alta cardinalidad de claves. diff --git a/docs/es/sql-reference/aggregate-functions/reference.md b/docs/es/sql-reference/aggregate-functions/reference.md index b7722ecd221..927d16bc748 100644 --- a/docs/es/sql-reference/aggregate-functions/reference.md +++ b/docs/es/sql-reference/aggregate-functions/reference.md @@ -31,7 +31,7 @@ En ambos casos el tipo del valor devuelto es [UInt64](../../sql-reference/data-t **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. +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. diff --git a/docs/es/sql-reference/ansi.md b/docs/es/sql-reference/ansi.md index 3bf06d4419b..a16a4cb5798 100644 --- a/docs/es/sql-reference/ansi.md +++ b/docs/es/sql-reference/ansi.md @@ -36,8 +36,8 @@ En la tabla siguiente se enumeran los casos en que la característica de consult | E021-01 | Tipo de datos CHARACTER | No{.text-danger} | | | E021-02 | Tipo de datos CHARACTER VARYING | No{.text-danger} | `String` se comporta de manera similar, pero sin límite de longitud entre paréntesis | | E021-03 | Literales de caracteres | Parcial{.text-warning} | Sin concatenación automática de literales consecutivos y compatibilidad con el conjunto de caracteres | -| E021-04 | Función CHARACTER\_LENGTH | Parcial{.text-warning} | No `USING` clausula | -| E021-05 | Función OCTET\_LENGTH | No{.text-danger} | `LENGTH` se comporta de manera similar | +| E021-04 | Función CHARACTER_LENGTH | Parcial{.text-warning} | No `USING` clausula | +| E021-05 | Función OCTET_LENGTH | No{.text-danger} | `LENGTH` se comporta de manera similar | | E021-06 | SUBSTRING | Parcial{.text-warning} | No hay soporte para `SIMILAR` y `ESCAPE` cláusulas, no `SUBSTRING_REGEX` variante | | E021-07 | Concatenación de caracteres | Parcial{.text-warning} | No `COLLATE` clausula | | E021-08 | Funciones SUPERIOR e INFERIOR | Sí{.text-success} | | @@ -144,7 +144,7 @@ En la tabla siguiente se enumeran los casos en que la característica de consult | F051-03 | Tipo de datos TIMESTAMP (incluido el soporte del literal TIMESTAMP) con una precisión de segundos fraccionarios de al menos 0 y 6 | No{.text-danger} | `DateTime64` tiempo proporciona una funcionalidad similar | | F051-04 | Predicado de comparación en los tipos de datos DATE, TIME y TIMESTAMP | Parcial{.text-warning} | Sólo un tipo de datos disponible | | F051-05 | CAST explícito entre tipos de fecha y hora y tipos de cadena de caracteres | Sí{.text-success} | | -| F051-06 | CURRENT\_DATE | No{.text-danger} | `today()` es similar | +| F051-06 | CURRENT_DATE | No{.text-danger} | `today()` es similar | | F051-07 | LOCALTIME | No{.text-danger} | `now()` es similar | | F051-08 | LOCALTIMESTAMP | No{.text-danger} | | | **F081** | **UNIÓN y EXCEPTO en vistas** | **Parcial**{.text-warning} | | diff --git a/docs/es/sql-reference/data-types/aggregatefunction.md b/docs/es/sql-reference/data-types/aggregatefunction.md index 39153c9eba4..c7eab28c6ce 100644 --- a/docs/es/sql-reference/data-types/aggregatefunction.md +++ b/docs/es/sql-reference/data-types/aggregatefunction.md @@ -5,7 +5,7 @@ toc_priority: 52 toc_title: "Agregar funci\xF3n (nombre, types_of_arguments)...)" --- -# AggregateFunction(name, types\_of\_arguments…) {#data-type-aggregatefunction} +# AggregateFunction(name, types_of_arguments…) {#data-type-aggregatefunction} Aggregate functions can have an implementation-defined intermediate state that can be serialized to an AggregateFunction(…) data type and stored in a table, usually, by means of [una vista materializada](../../sql-reference/statements/create.md#create-view). La forma común de producir un estado de función agregada es llamando a la función agregada con el `-State` sufijo. Para obtener el resultado final de la agregación en el futuro, debe utilizar la misma función de agregado con el `-Merge`sufijo. diff --git a/docs/es/sql-reference/data-types/datetime.md b/docs/es/sql-reference/data-types/datetime.md index 0fa214fddf3..0163da2002a 100644 --- a/docs/es/sql-reference/data-types/datetime.md +++ b/docs/es/sql-reference/data-types/datetime.md @@ -31,7 +31,7 @@ El [Casa de clics-cliente](../../interfaces/cli.md) aplica la zona horaria del s ClickHouse genera valores en `YYYY-MM-DD hh:mm:ss` formato de texto por defecto. Puede cambiar la salida con el [formatDateTime](../../sql-reference/functions/date-time-functions.md#formatdatetime) función. -Al insertar datos en ClickHouse, puede usar diferentes formatos de cadenas de fecha y hora, dependiendo del valor de la [Date\_time\_input\_format](../../operations/settings/settings.md#settings-date_time_input_format) configuración. +Al insertar datos en ClickHouse, puede usar diferentes formatos de cadenas de fecha y hora, dependiendo del valor de la [Date_time_input_format](../../operations/settings/settings.md#settings-date_time_input_format) configuración. ## Ejemplos {#examples} diff --git a/docs/es/sql-reference/data-types/fixedstring.md b/docs/es/sql-reference/data-types/fixedstring.md index 23d17bac1a5..f12a308b7de 100644 --- a/docs/es/sql-reference/data-types/fixedstring.md +++ b/docs/es/sql-reference/data-types/fixedstring.md @@ -22,7 +22,7 @@ El `FixedString` tipo es eficiente cuando los datos tienen la longitud de `N` by Ejemplos de los valores que se pueden almacenar eficientemente en `FixedString`escrito columnas: - La representación binaria de direcciones IP (`FixedString(16)` para IPv6). -- Language codes (ru\_RU, en\_US … ). +- Language codes (ru_RU, en_US … ). - Currency codes (USD, RUB … ). - Representación binaria de hashes (`FixedString(16)` para MD5, `FixedString(32)` para SHA256). diff --git a/docs/es/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md b/docs/es/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md index 4b67a843835..68f18367872 100644 --- a/docs/es/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md +++ b/docs/es/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md @@ -27,7 +27,7 @@ Mira la siguiente estructura jerárquica: Esta jerarquía se puede expresar como la siguiente tabla de diccionario. -| region\_id | parent\_region | nombre\_región | +| region_id | parent_region | nombre_región | |------------|----------------|----------------| | 1 | 0 | Rusia | | 2 | 1 | Moscu | diff --git a/docs/es/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/es/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 02a1de9b0bb..85001c181eb 100644 --- a/docs/es/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/es/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -57,9 +57,9 @@ LAYOUT(LAYOUT_TYPE(param value)) -- layout settings - [Sistema abierto.](#dicts-external_dicts_dict_layout-sparse_hashed) - [cache](#cache) - [directo](#direct) -- [range\_hashed](#range-hashed) +- [range_hashed](#range-hashed) - [Método de codificación de datos:](#complex-key-hashed) -- [complejo\_key\_cache](#complex-key-cache) +- [complejo_key_cache](#complex-key-cache) - [Método de codificación de datos:](#ip-trie) ### plano {#flat} @@ -138,7 +138,7 @@ Ejemplo de configuración: LAYOUT(COMPLEX_KEY_HASHED()) ``` -### range\_hashed {#range-hashed} +### range_hashed {#range-hashed} El diccionario se almacena en la memoria en forma de una tabla hash con una matriz ordenada de rangos y sus valores correspondientes. @@ -293,7 +293,7 @@ Establezca un tamaño de caché lo suficientemente grande. Necesitas experimenta !!! warning "Advertencia" No use ClickHouse como fuente, ya que es lento procesar consultas con lecturas aleatorias. -### complejo\_key\_cache {#complex-key-cache} +### complejo_key_cache {#complex-key-cache} Este tipo de almacenamiento es para su uso con material compuesto [claves](external-dicts-dict-structure.md). Similar a `cache`. diff --git a/docs/es/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/es/sql-reference/dictionaries/external-dictionaries/external-dicts.md index e408c63e257..ecb4fa9fc63 100644 --- a/docs/es/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ b/docs/es/sql-reference/dictionaries/external-dictionaries/external-dicts.md @@ -15,9 +15,9 @@ Haga clic en Casa: - Actualiza periódicamente los diccionarios y carga dinámicamente los valores que faltan. En otras palabras, los diccionarios se pueden cargar dinámicamente. - Permite crear diccionarios externos con archivos xml o [Consultas DDL](../../statements/create.md#create-dictionary-query). -La configuración de diccionarios externos se puede ubicar en uno o más archivos xml. La ruta de acceso a la configuración se especifica en el [Diccionarios\_config](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config) parámetro. +La configuración de diccionarios externos se puede ubicar en uno o más archivos xml. La ruta de acceso a la configuración se especifica en el [Diccionarios_config](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config) parámetro. -Los diccionarios se pueden cargar en el inicio del servidor o en el primer uso, dependiendo de la [Diccionarios\_lazy\_load](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load) configuración. +Los diccionarios se pueden cargar en el inicio del servidor o en el primer uso, dependiendo de la [Diccionarios_lazy_load](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load) configuración. El [diccionario](../../../operations/system-tables.md#system_tables-dictionaries) La tabla del sistema contiene información sobre los diccionarios configurados en el servidor. Para cada diccionario se puede encontrar allí: diff --git a/docs/es/sql-reference/functions/array-functions.md b/docs/es/sql-reference/functions/array-functions.md index 677996efabd..460e4775b44 100644 --- a/docs/es/sql-reference/functions/array-functions.md +++ b/docs/es/sql-reference/functions/array-functions.md @@ -1028,7 +1028,7 @@ Resultado: ## arrayAUC {#arrayauc} -Calcule AUC (Área bajo la curva, que es un concepto en el aprendizaje automático, vea más detalles: https://en.wikipedia.org/wiki/Receiver\_operating\_characteristic\#Area\_under\_the\_curve ). +Calcule AUC (Área bajo la curva, que es un concepto en el aprendizaje automático, vea más detalles: https://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve ). **Sintaxis** diff --git a/docs/es/sql-reference/functions/bitmap-functions.md b/docs/es/sql-reference/functions/bitmap-functions.md index 68e956c9819..1643801c77b 100644 --- a/docs/es/sql-reference/functions/bitmap-functions.md +++ b/docs/es/sql-reference/functions/bitmap-functions.md @@ -65,7 +65,7 @@ SELECT bitmapToArray(bitmapBuild([1, 2, 3, 4, 5])) AS res ## bitmapSubsetInRange {#bitmap-functions-bitmapsubsetinrange} -Devuelve el subconjunto en el rango especificado (no incluye range\_end). +Devuelve el subconjunto en el rango especificado (no incluye range_end). ``` sql bitmapSubsetInRange(bitmap, range_start, range_end) @@ -241,7 +241,7 @@ SELECT bitmapCardinality(bitmapBuild([1, 2, 3, 4, 5])) AS res ## bitmapMin {#bitmapmin} -Vuelva a ejecutar el valor más pequeño de tipo UInt64 en el conjunto, UINT32\_MAX si el conjunto está vacío. +Vuelva a ejecutar el valor más pequeño de tipo UInt64 en el conjunto, UINT32_MAX si el conjunto está vacío. bitmapMin(bitmap) @@ -288,8 +288,8 @@ Transformar una matriz de valores en un mapa de bits a otra matriz de valores, e **Parámetros** - `bitmap` – bitmap object. -- `from_array` – UInt32 array. For idx in range \[0, from\_array.size()), if bitmap contains from\_array\[idx\], then replace it with to\_array\[idx\]. Note that the result depends on array ordering if there are common elements between from\_array and to\_array. -- `to_array` – UInt32 array, its size shall be the same to from\_array. +- `from_array` – UInt32 array. For idx in range \[0, from_array.size()), if bitmap contains from_array\[idx\], then replace it with to_array\[idx\]. Note that the result depends on array ordering if there are common elements between from_array and to_array. +- `to_array` – UInt32 array, its size shall be the same to from_array. **Ejemplo** diff --git a/docs/es/sql-reference/functions/geo.md b/docs/es/sql-reference/functions/geo.md index 3d55480c150..b8e3a3b5ffd 100644 --- a/docs/es/sql-reference/functions/geo.md +++ b/docs/es/sql-reference/functions/geo.md @@ -209,10 +209,10 @@ Devuelve una matriz de cadenas codificadas por geohash de precisión dada que ca **Valores de entrada** -- longitude\_min - longitud mínima, valor flotante en el rango `[-180°, 180°]` -- latitude\_min - latitud mínima, valor flotante en el rango `[-90°, 90°]` -- longitude\_max - longitud máxima, valor flotante en el rango `[-180°, 180°]` -- latitude\_max - latitud máxima, valor flotante en el rango `[-90°, 90°]` +- longitude_min - longitud mínima, valor flotante en el rango `[-180°, 180°]` +- latitude_min - latitud mínima, valor flotante en el rango `[-90°, 90°]` +- longitude_max - longitud máxima, valor flotante en el rango `[-180°, 180°]` +- latitude_max - latitud máxima, valor flotante en el rango `[-90°, 90°]` - precisión - precisión del geohash, `UInt8` en el rango `[1, 12]` Tenga en cuenta que todos los parámetros de coordenadas deben ser del mismo tipo: `Float32` o `Float64`. diff --git a/docs/es/sql-reference/functions/hash-functions.md b/docs/es/sql-reference/functions/hash-functions.md index 36b44dfc793..165ca2be3d7 100644 --- a/docs/es/sql-reference/functions/hash-functions.md +++ b/docs/es/sql-reference/functions/hash-functions.md @@ -317,7 +317,7 @@ Calcula JumpConsistentHash forma un UInt64. Acepta dos argumentos: una clave de tipo UInt64 y el número de cubos. Devuelve Int32. Para obtener más información, consulte el enlace: [SaltarConsistentHash](https://arxiv.org/pdf/1406.2294.pdf) -## murmurHash2\_32, murmurHash2\_64 {#murmurhash2-32-murmurhash2-64} +## murmurHash2_32, murmurHash2_64 {#murmurhash2-32-murmurhash2-64} Produce un [Método de codificación de datos:](https://github.com/aappleby/smhasher) valor hash. @@ -385,7 +385,7 @@ Resultado: └──────────────────────┴─────────────────────┘ ``` -## murmurHash3\_32, murmurHash3\_64 {#murmurhash3-32-murmurhash3-64} +## murmurHash3_32, murmurHash3_64 {#murmurhash3-32-murmurhash3-64} Produce un [Método de codificación de datos:](https://github.com/aappleby/smhasher) valor hash. @@ -415,7 +415,7 @@ SELECT murmurHash3_32(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23: └─────────────┴────────┘ ``` -## murmurHash3\_128 {#murmurhash3-128} +## murmurHash3_128 {#murmurhash3-128} Produce un [Método de codificación de datos:](https://github.com/aappleby/smhasher) valor hash. diff --git a/docs/es/sql-reference/functions/introspection.md b/docs/es/sql-reference/functions/introspection.md index 27990af0318..1482bbc3190 100644 --- a/docs/es/sql-reference/functions/introspection.md +++ b/docs/es/sql-reference/functions/introspection.md @@ -16,11 +16,11 @@ Para el correcto funcionamiento de las funciones de introspección: - Instale el `clickhouse-common-static-dbg` paquete. -- Establezca el [allow\_introspection\_functions](../../operations/settings/settings.md#settings-allow_introspection_functions) a 1. +- Establezca el [allow_introspection_functions](../../operations/settings/settings.md#settings-allow_introspection_functions) a 1. For security reasons introspection functions are disabled by default. -ClickHouse guarda los informes del generador de perfiles [trace\_log](../../operations/system-tables.md#system_tables-trace_log) tabla del sistema. Asegúrese de que la tabla y el generador de perfiles estén configurados correctamente. +ClickHouse guarda los informes del generador de perfiles [trace_log](../../operations/system-tables.md#system_tables-trace_log) tabla del sistema. Asegúrese de que la tabla y el generador de perfiles estén configurados correctamente. ## addressToLine {#addresstoline} diff --git a/docs/es/sql-reference/functions/json-functions.md b/docs/es/sql-reference/functions/json-functions.md index b004c98bfc8..09b66abb2b8 100644 --- a/docs/es/sql-reference/functions/json-functions.md +++ b/docs/es/sql-reference/functions/json-functions.md @@ -75,7 +75,7 @@ SELECT isValidJSON('{"a": "hello", "b": [-100, 200.0, 300]}') = 1 SELECT isValidJSON('not a json') = 0 ``` -## JSONHas(json\[, indices\_or\_keys\]…) {#jsonhasjson-indices-or-keys} +## JSONHas(json\[, indices_or_keys\]…) {#jsonhasjson-indices-or-keys} Si el valor existe en el documento JSON, `1` serán devueltos. @@ -108,7 +108,7 @@ SELECT JSONExtractKey('{"a": "hello", "b": [-100, 200.0, 300]}', -2) = 'a' SELECT JSONExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 1) = 'hello' ``` -## JSONLength(json\[, indices\_or\_keys\]…) {#jsonlengthjson-indices-or-keys} +## JSONLength(json\[, indices_or_keys\]…) {#jsonlengthjson-indices-or-keys} Devuelve la longitud de una matriz JSON o un objeto JSON. @@ -121,7 +121,7 @@ SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 3 SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}') = 2 ``` -## JSONType(json\[, indices\_or\_keys\]…) {#jsontypejson-indices-or-keys} +## JSONType(json\[, indices_or_keys\]…) {#jsontypejson-indices-or-keys} Devuelve el tipo de un valor JSON. @@ -135,13 +135,13 @@ SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'a') = 'String' SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 'Array' ``` -## JSONExtractUInt(json\[, indices\_or\_keys\]…) {#jsonextractuintjson-indices-or-keys} +## JSONExtractUInt(json\[, indices_or_keys\]…) {#jsonextractuintjson-indices-or-keys} -## JSONExtractInt(json\[, indices\_or\_keys\]…) {#jsonextractintjson-indices-or-keys} +## JSONExtractInt(json\[, indices_or_keys\]…) {#jsonextractintjson-indices-or-keys} -## JSONExtractFloat(json\[, indices\_or\_keys\]…) {#jsonextractfloatjson-indices-or-keys} +## JSONExtractFloat(json\[, indices_or_keys\]…) {#jsonextractfloatjson-indices-or-keys} -## JSONExtractBool(json\[, indices\_or\_keys\]…) {#jsonextractbooljson-indices-or-keys} +## JSONExtractBool(json\[, indices_or_keys\]…) {#jsonextractbooljson-indices-or-keys} Analiza un JSON y extrae un valor. Estas funciones son similares a `visitParam` función. @@ -155,7 +155,7 @@ SELECT JSONExtractFloat('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 2) = 200 SELECT JSONExtractUInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', -1) = 300 ``` -## JSONExtractString(json\[, indices\_or\_keys\]…) {#jsonextractstringjson-indices-or-keys} +## JSONExtractString(json\[, indices_or_keys\]…) {#jsonextractstringjson-indices-or-keys} Analiza un JSON y extrae una cadena. Esta función es similar a `visitParamExtractString` función. @@ -173,7 +173,7 @@ SELECT JSONExtractString('{"abc":"\\u263"}', 'abc') = '' SELECT JSONExtractString('{"abc":"hello}', 'abc') = '' ``` -## JSONExtract(json\[, indices\_or\_keys…\], Return\_type) {#jsonextractjson-indices-or-keys-return-type} +## JSONExtract(json\[, indices_or_keys…\], Return_type) {#jsonextractjson-indices-or-keys-return-type} Analiza un JSON y extrae un valor del tipo de datos ClickHouse dado. @@ -194,7 +194,7 @@ SELECT JSONExtract('{"day": "Thursday"}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday SELECT JSONExtract('{"day": 5}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \'Tuesday\' = 2, \'Wednesday\' = 3, \'Thursday\' = 4, \'Friday\' = 5, \'Saturday\' = 6)') = 'Friday' ``` -## JSONExtractKeysAndValues(json\[, indices\_or\_keys…\], Value\_type) {#jsonextractkeysandvaluesjson-indices-or-keys-value-type} +## JSONExtractKeysAndValues(json\[, indices_or_keys…\], Value_type) {#jsonextractkeysandvaluesjson-indices-or-keys-value-type} Analiza los pares clave-valor de un JSON donde los valores son del tipo de datos ClickHouse especificado. @@ -204,7 +204,7 @@ Ejemplo: SELECT JSONExtractKeysAndValues('{"x": {"a": 5, "b": 7, "c": 11}}', 'x', 'Int8') = [('a',5),('b',7),('c',11)] ``` -## JSONExtractRaw(json\[, indices\_or\_keys\]…) {#jsonextractrawjson-indices-or-keys} +## JSONExtractRaw(json\[, indices_or_keys\]…) {#jsonextractrawjson-indices-or-keys} Devuelve una parte de JSON como cadena sin analizar. @@ -216,7 +216,7 @@ Ejemplo: SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = '[-100, 200.0, 300]' ``` -## JSONExtractArrayRaw(json\[, indices\_or\_keys…\]) {#jsonextractarrayrawjson-indices-or-keys} +## JSONExtractArrayRaw(json\[, indices_or_keys…\]) {#jsonextractarrayrawjson-indices-or-keys} Devuelve una matriz con elementos de matriz JSON, cada uno representado como cadena sin analizar. diff --git a/docs/es/sql-reference/functions/other-functions.md b/docs/es/sql-reference/functions/other-functions.md index b9b180f6c61..3704355167a 100644 --- a/docs/es/sql-reference/functions/other-functions.md +++ b/docs/es/sql-reference/functions/other-functions.md @@ -439,7 +439,7 @@ Tipo: Cuando se indica la misma letra (T o U), para los tipos numéricos pueden no ser tipos coincidentes, sino tipos que tienen un tipo común. Por ejemplo, el primer argumento puede tener el tipo Int64, mientras que el segundo tiene el tipo Array(UInt16). -Si el ‘x’ valor es igual a uno de los elementos en el ‘array\_from’ matriz, devuelve el elemento existente (que está numerado igual) de la ‘array\_to’ matriz. De lo contrario, devuelve ‘default’. Si hay varios elementos coincidentes en ‘array\_from’, devuelve una de las coincidencias. +Si el ‘x’ valor es igual a uno de los elementos en el ‘array_from’ matriz, devuelve el elemento existente (que está numerado igual) de la ‘array_to’ matriz. De lo contrario, devuelve ‘default’. Si hay varios elementos coincidentes en ‘array_from’, devuelve una de las coincidencias. Ejemplo: @@ -464,7 +464,7 @@ ORDER BY c DESC ### ¿Cómo puedo hacerlo?) {#transformx-array-from-array-to} Difiere de la primera variación en que el ‘default’ se omite el argumento. -Si el ‘x’ valor es igual a uno de los elementos en el ‘array\_from’ matriz, devuelve el elemento coincidente (que está numerado igual) de la ‘array\_to’ matriz. De lo contrario, devuelve ‘x’. +Si el ‘x’ valor es igual a uno de los elementos en el ‘array_from’ matriz, devuelve el elemento coincidente (que está numerado igual) de la ‘array_to’ matriz. De lo contrario, devuelve ‘x’. Tipo: @@ -1121,7 +1121,7 @@ Resultado: └──────────────────────────────────────────────────┘ ``` -## modelEvaluate(model\_name, …) {#function-modelevaluate} +## modelEvaluate(model_name, …) {#function-modelevaluate} Evaluar modelo externo. Acepta un nombre de modelo y argumentos de modelo. Devuelve Float64. @@ -1129,7 +1129,7 @@ Acepta un nombre de modelo y argumentos de modelo. Devuelve Float64. ## ¿Cómo puedo hacerlo?\]) {#throwifx-custom-message} Lance una excepción si el argumento no es cero. -custom\_message - es un parámetro opcional: una cadena constante, proporciona un mensaje de error +custom_message - es un parámetro opcional: una cadena constante, proporciona un mensaje de error ``` sql SELECT throwIf(number = 3, 'Too many') FROM numbers(10); diff --git a/docs/es/sql-reference/functions/string-functions.md b/docs/es/sql-reference/functions/string-functions.md index 11d75dd6775..43025465b31 100644 --- a/docs/es/sql-reference/functions/string-functions.md +++ b/docs/es/sql-reference/functions/string-functions.md @@ -31,12 +31,12 @@ La función también funciona para matrices. Devuelve la longitud de una cadena en puntos de código Unicode (no en caracteres), suponiendo que la cadena contiene un conjunto de bytes que componen texto codificado en UTF-8. Si no se cumple esta suposición, devuelve algún resultado (no arroja una excepción). El tipo de resultado es UInt64. -## char\_length, CHAR\_LENGTH {#char-length} +## char_length, CHAR_LENGTH {#char-length} Devuelve la longitud de una cadena en puntos de código Unicode (no en caracteres), suponiendo que la cadena contiene un conjunto de bytes que componen texto codificado en UTF-8. Si no se cumple esta suposición, devuelve algún resultado (no arroja una excepción). El tipo de resultado es UInt64. -## character\_length, CHARACTER\_LENGTH {#character-length} +## character_length, CHARACTER_LENGTH {#character-length} Devuelve la longitud de una cadena en puntos de código Unicode (no en caracteres), suponiendo que la cadena contiene un conjunto de bytes que componen texto codificado en UTF-8. Si no se cumple esta suposición, devuelve algún resultado (no arroja una excepción). El tipo de resultado es UInt64. @@ -77,7 +77,7 @@ toValidUTF8( input_string ) Parámetros: -- input\_string — Any set of bytes represented as the [Cadena](../../sql-reference/data-types/string.md) objeto de tipo de datos. +- input_string — Any set of bytes represented as the [Cadena](../../sql-reference/data-types/string.md) objeto de tipo de datos. Valor devuelto: cadena UTF-8 válida. diff --git a/docs/es/sql-reference/functions/type-conversion-functions.md b/docs/es/sql-reference/functions/type-conversion-functions.md index 3c2c629fc1d..d5e41ee0e52 100644 --- a/docs/es/sql-reference/functions/type-conversion-functions.md +++ b/docs/es/sql-reference/functions/type-conversion-functions.md @@ -246,7 +246,7 @@ YYYY-MM-DD YYYY-MM-DD hh:mm:ss ``` -Como excepción, si convierte de tipos numéricos UInt32, Int32, UInt64 o Int64 a Date, y si el número es mayor o igual que 65536, el número se interpreta como una marca de tiempo Unix (y no como el número de días) y se redondea a la fecha. Esto permite soporte para la ocurrencia común de la escritura ‘toDate(unix\_timestamp)’, que de otra manera sería un error y requeriría escribir el más engorroso ‘toDate(toDateTime(unix\_timestamp))’. +Como excepción, si convierte de tipos numéricos UInt32, Int32, UInt64 o Int64 a Date, y si el número es mayor o igual que 65536, el número se interpreta como una marca de tiempo Unix (y no como el número de días) y se redondea a la fecha. Esto permite soporte para la ocurrencia común de la escritura ‘toDate(unix_timestamp)’, que de otra manera sería un error y requeriría escribir el más engorroso ‘toDate(toDateTime(unix_timestamp))’. La conversión entre una fecha y una fecha con la hora se realiza de la manera natural: agregando un tiempo nulo o eliminando el tiempo. diff --git a/docs/es/sql-reference/functions/url-functions.md b/docs/es/sql-reference/functions/url-functions.md index a46af64ca0c..de4c32ead2b 100644 --- a/docs/es/sql-reference/functions/url-functions.md +++ b/docs/es/sql-reference/functions/url-functions.md @@ -125,11 +125,11 @@ Devuelve la ruta de acceso. Ejemplo: `/top/news.html` La ruta de acceso no inclu ### pathFull {#pathfull} -Lo mismo que el anterior, pero incluyendo cadena de consulta y fragmento. Ejemplo: /top/news.html?Página = 2 \# comentarios +Lo mismo que el anterior, pero incluyendo cadena de consulta y fragmento. Ejemplo: /top/news.html?Página = 2 # comentarios ### queryString {#querystring} -Devuelve la cadena de consulta. Ejemplo: page=1&lr=213. query-string no incluye el signo de interrogación inicial, así como \# y todo después de \#. +Devuelve la cadena de consulta. Ejemplo: page=1&lr=213. query-string no incluye el signo de interrogación inicial, así como # y todo después de #. ### fragmento {#fragment} @@ -137,7 +137,7 @@ Devuelve el identificador de fragmento. el fragmento no incluye el símbolo hash ### queryStringAndFragment {#querystringandfragment} -Devuelve la cadena de consulta y el identificador de fragmento. Ejemplo: page=1\#29390. +Devuelve la cadena de consulta y el identificador de fragmento. Ejemplo: page=1#29390. ### extractURLParameter(URL, nombre) {#extracturlparameterurl-name} diff --git a/docs/es/sql-reference/functions/ym-dict-functions.md b/docs/es/sql-reference/functions/ym-dict-functions.md index 486d27af786..311bbe9f5e8 100644 --- a/docs/es/sql-reference/functions/ym-dict-functions.md +++ b/docs/es/sql-reference/functions/ym-dict-functions.md @@ -17,12 +17,12 @@ ClickHouse admite trabajar con múltiples geobases alternativas (jerarquías reg El ‘clickhouse-server’ config especifica el archivo con la jerarquía regional::`/opt/geo/regions_hierarchy.txt` -Además de este archivo, también busca archivos cercanos que tengan el símbolo \_ y cualquier sufijo anexado al nombre (antes de la extensión del archivo). +Además de este archivo, también busca archivos cercanos que tengan el símbolo _ y cualquier sufijo anexado al nombre (antes de la extensión del archivo). Por ejemplo, también encontrará el archivo `/opt/geo/regions_hierarchy_ua.txt` si está presente. `ua` se llama la clave del diccionario. Para un diccionario sin un sufijo, la clave es una cadena vacía. -Todos los diccionarios se vuelven a cargar en tiempo de ejecución (una vez cada cierto número de segundos, como se define en el parámetro de configuración builtin\_dictionaries\_reload\_interval , o una vez por hora por defecto). Sin embargo, la lista de diccionarios disponibles se define una vez, cuando se inicia el servidor. +Todos los diccionarios se vuelven a cargar en tiempo de ejecución (una vez cada cierto número de segundos, como se define en el parámetro de configuración builtin_dictionaries_reload_interval , o una vez por hora por defecto). Sin embargo, la lista de diccionarios disponibles se define una vez, cuando se inicia el servidor. All functions for working with regions have an optional argument at the end – the dictionary key. It is referred to as the geobase. Ejemplo: diff --git a/docs/es/sql-reference/operators/in.md b/docs/es/sql-reference/operators/in.md index f5540497eba..988aafb25cb 100644 --- a/docs/es/sql-reference/operators/in.md +++ b/docs/es/sql-reference/operators/in.md @@ -117,9 +117,9 @@ Para una consulta no distribuida, utilice el `IN` / `JOIN`. Tenga cuidado al usar subconsultas en el `IN` / `JOIN` para el procesamiento de consultas distribuidas. -Veamos algunos ejemplos. Supongamos que cada servidor del clúster tiene un **local\_table**. Cada servidor también tiene un **distributed\_table** mesa con el **Distribuido** tipo, que mira todos los servidores del clúster. +Veamos algunos ejemplos. Supongamos que cada servidor del clúster tiene un **local_table**. Cada servidor también tiene un **distributed_table** mesa con el **Distribuido** tipo, que mira todos los servidores del clúster. -Para una consulta al **distributed\_table**, la consulta se enviará a todos los servidores remotos y se ejecutará en ellos usando el **local\_table**. +Para una consulta al **distributed_table**, la consulta se enviará a todos los servidores remotos y se ejecutará en ellos usando el **local_table**. Por ejemplo, la consulta @@ -153,7 +153,7 @@ En otras palabras, los datos establecidos en la cláusula IN se recopilarán en Esto funcionará correctamente y de manera óptima si está preparado para este caso y ha distribuido datos en los servidores de clúster de modo que los datos de un único ID de usuario residen completamente en un único servidor. En este caso, todos los datos necesarios estarán disponibles localmente en cada servidor. De lo contrario, el resultado será inexacto. Nos referimos a esta variación de la consulta como “local IN”. -Para corregir cómo funciona la consulta cuando los datos se distribuyen aleatoriamente entre los servidores de clúster, puede especificar **distributed\_table** dentro de una subconsulta. La consulta se vería así: +Para corregir cómo funciona la consulta cuando los datos se distribuyen aleatoriamente entre los servidores de clúster, puede especificar **distributed_table** dentro de una subconsulta. La consulta se vería así: ``` sql SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM distributed_table WHERE CounterID = 34) diff --git a/docs/es/sql-reference/statements/alter.md b/docs/es/sql-reference/statements/alter.md index 7ee45c89fff..194b0cea00f 100644 --- a/docs/es/sql-reference/statements/alter.md +++ b/docs/es/sql-reference/statements/alter.md @@ -208,7 +208,7 @@ Las siguientes operaciones con [partición](../../engines/table-engines/mergetre - [ATTACH PART\|PARTITION](#alter_attach-partition) – Adds a part or partition from the `detached` directorio a la tabla. - [ATTACH PARTITION FROM](#alter_attach-partition-from) – Copies the data partition from one table to another and adds. - [REPLACE PARTITION](#alter_replace-partition) - Copia la partición de datos de una tabla a otra y reemplaza. -- [MOVE PARTITION TO TABLE](#alter_move_to_table-partition)(\#alter\_move\_to\_table-partition) - Mover la partición de datos de una tabla a otra. +- [MOVE PARTITION TO TABLE](#alter_move_to_table-partition)(#alter_move_to_table-partition) - Mover la partición de datos de una tabla a otra. - [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) - Restablece el valor de una columna especificada en una partición. - [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) - Restablece el índice secundario especificado en una partición. - [FREEZE PARTITION](#alter_freeze-partition) – Creates a backup of a partition. @@ -436,7 +436,7 @@ Puede especificar la expresión de partición en `ALTER ... PARTITION` de difere - Como valor de la `partition` columna de la `system.parts` tabla. Por ejemplo, `ALTER TABLE visits DETACH PARTITION 201901`. - Como la expresión de la columna de la tabla. Se admiten constantes y expresiones constantes. Por ejemplo, `ALTER TABLE visits DETACH PARTITION toYYYYMM(toDate('2019-01-25'))`. - Usando el ID de partición. El ID de partición es un identificador de cadena de la partición (legible por humanos, si es posible) que se usa como nombres de particiones en el sistema de archivos y en ZooKeeper. El ID de partición debe especificarse en el `PARTITION ID` cláusula, entre comillas simples. Por ejemplo, `ALTER TABLE visits DETACH PARTITION ID '201901'`. -- En el [ALTER ATTACH PART](#alter_attach-partition) y [DROP DETACHED PART](#alter_drop-detached) consulta, para especificar el nombre de una parte, utilice un literal de cadena con un valor `name` columna de la [sistema.detached\_parts](../../operations/system-tables.md#system_tables-detached_parts) tabla. Por ejemplo, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. +- En el [ALTER ATTACH PART](#alter_attach-partition) y [DROP DETACHED PART](#alter_drop-detached) consulta, para especificar el nombre de una parte, utilice un literal de cadena con un valor `name` columna de la [sistema.detached_parts](../../operations/system-tables.md#system_tables-detached_parts) tabla. Por ejemplo, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. El uso de comillas al especificar la partición depende del tipo de expresión de partición. Por ejemplo, para el `String` tipo, debe especificar su nombre entre comillas (`'`). Para el `Date` y `Int*` tipos no se necesitan comillas. diff --git a/docs/es/sql-reference/statements/insert-into.md b/docs/es/sql-reference/statements/insert-into.md index abff9bf9cc2..4f64af830e7 100644 --- a/docs/es/sql-reference/statements/insert-into.md +++ b/docs/es/sql-reference/statements/insert-into.md @@ -20,7 +20,7 @@ La consulta puede especificar una lista de columnas para insertar `[(c1, c2, c3) - Los valores calculados a partir del `DEFAULT` expresiones especificadas en la definición de la tabla. - Ceros y cadenas vacías, si `DEFAULT` expresiones no están definidas. -Si [strict\_insert\_defaults=1](../../operations/settings/settings.md), columnas que no tienen `DEFAULT` definido debe figurar en la consulta. +Si [strict_insert_defaults=1](../../operations/settings/settings.md), columnas que no tienen `DEFAULT` definido debe figurar en la consulta. Los datos se pueden pasar al INSERT en cualquier [formato](../../interfaces/formats.md#formats) con el apoyo de ClickHouse. El formato debe especificarse explícitamente en la consulta: diff --git a/docs/es/sql-reference/statements/misc.md b/docs/es/sql-reference/statements/misc.md index 542d147676a..49cbc4fe558 100644 --- a/docs/es/sql-reference/statements/misc.md +++ b/docs/es/sql-reference/statements/misc.md @@ -57,7 +57,7 @@ Para `MergeTree` motores familiares, el `CHECK TABLE` query muestra un estado de Si la tabla está dañada, puede copiar los datos no dañados a otra tabla. Para hacer esto: 1. Cree una nueva tabla con la misma estructura que la tabla dañada. Para ello, ejecute la consulta `CREATE TABLE AS `. -2. Establezca el [max\_threads](../../operations/settings/settings.md#settings-max_threads) valor a 1 para procesar la siguiente consulta en un único subproceso. Para ello, ejecute la consulta `SET max_threads = 1`. +2. Establezca el [max_threads](../../operations/settings/settings.md#settings-max_threads) valor a 1 para procesar la siguiente consulta en un único subproceso. Para ello, ejecute la consulta `SET max_threads = 1`. 3. Ejecutar la consulta `INSERT INTO SELECT * FROM `. Esta solicitud copia los datos no dañados de la tabla dañada a otra tabla. Solo se copiarán los datos anteriores a la parte dañada. 4. Reinicie el `clickhouse-client` para restablecer el `max_threads` valor. @@ -253,7 +253,7 @@ El `OPTMIZE` consulta también es compatible con el [Método de codificación de Cuando `OPTIMIZE` se utiliza con el [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/replication.md) la familia de motores de tablas, ClickHouse crea una tarea para fusionar y espera la ejecución en todos los nodos (si `replication_alter_partitions_sync` está habilitada la configuración). -- Si `OPTIMIZE` no realiza una fusión por ningún motivo, no notifica al cliente. Para habilitar las notificaciones, [Optize\_throw\_if\_noop](../../operations/settings/settings.md#setting-optimize_throw_if_noop) configuración. +- Si `OPTIMIZE` no realiza una fusión por ningún motivo, no notifica al cliente. Para habilitar las notificaciones, [Optize_throw_if_noop](../../operations/settings/settings.md#setting-optimize_throw_if_noop) configuración. - Si especifica un `PARTITION`, sólo la partición especificada está optimizada. [Cómo establecer la expresión de partición](alter.md#alter-how-to-specify-part-expr). - Si especifica `FINAL`, la optimización se realiza incluso cuando todos los datos ya están en una parte. - Si especifica `DEDUPLICATE`, luego se deduplicarán filas completamente idénticas (se comparan todas las columnas), tiene sentido solo para el motor MergeTree. diff --git a/docs/es/sql-reference/statements/select/group-by.md b/docs/es/sql-reference/statements/select/group-by.md index fa5a5bd8c15..d6be9dc06e3 100644 --- a/docs/es/sql-reference/statements/select/group-by.md +++ b/docs/es/sql-reference/statements/select/group-by.md @@ -57,7 +57,7 @@ Esta fila adicional solo se produce en `JSON*`, `TabSeparated*`, y `Pretty*` for - En `Pretty*` formatea, la fila se muestra como una tabla separada después del resultado principal. - En los otros formatos no está disponible. -`WITH TOTALS` se puede ejecutar de diferentes maneras cuando HAVING está presente. El comportamiento depende de la ‘totals\_mode’ configuración. +`WITH TOTALS` se puede ejecutar de diferentes maneras cuando HAVING está presente. El comportamiento depende de la ‘totals_mode’ configuración. ### Configuración del procesamiento de totales {#configuring-totals-processing} @@ -67,9 +67,9 @@ Las otras alternativas incluyen solo las filas que pasan por HAVING en ‘totals `after_having_exclusive` – Don't include rows that didn't pass through `max_rows_to_group_by`. En otras palabras, ‘totals’ tendrá menos o el mismo número de filas que si `max_rows_to_group_by` se omitieron. -`after_having_inclusive` – Include all the rows that didn't pass through ‘max\_rows\_to\_group\_by’ en ‘totals’. En otras palabras, ‘totals’ tendrá más o el mismo número de filas como lo haría si `max_rows_to_group_by` se omitieron. +`after_having_inclusive` – Include all the rows that didn't pass through ‘max_rows_to_group_by’ en ‘totals’. En otras palabras, ‘totals’ tendrá más o el mismo número de filas como lo haría si `max_rows_to_group_by` se omitieron. -`after_having_auto` – Count the number of rows that passed through HAVING. If it is more than a certain amount (by default, 50%), include all the rows that didn't pass through ‘max\_rows\_to\_group\_by’ en ‘totals’. De lo contrario, no los incluya. +`after_having_auto` – Count the number of rows that passed through HAVING. If it is more than a certain amount (by default, 50%), include all the rows that didn't pass through ‘max_rows_to_group_by’ en ‘totals’. De lo contrario, no los incluya. `totals_auto_threshold` – By default, 0.5. The coefficient for `after_having_auto`. @@ -117,7 +117,7 @@ La agregación es una de las características más importantes de un DBMS orient ### GROUP BY en memoria externa {#select-group-by-in-external-memory} Puede habilitar el volcado de datos temporales en el disco para restringir el uso de memoria durante `GROUP BY`. -El [max\_bytes\_before\_external\_group\_by](../../../operations/settings/settings.md#settings-max_bytes_before_external_group_by) determina el umbral de consumo de RAM para el dumping `GROUP BY` datos temporales al sistema de archivos. Si se establece en 0 (el valor predeterminado), está deshabilitado. +El [max_bytes_before_external_group_by](../../../operations/settings/settings.md#settings-max_bytes_before_external_group_by) determina el umbral de consumo de RAM para el dumping `GROUP BY` datos temporales al sistema de archivos. Si se establece en 0 (el valor predeterminado), está deshabilitado. Cuando se utiliza `max_bytes_before_external_group_by`, le recomendamos que establezca `max_memory_usage` aproximadamente el doble de alto. Esto es necesario porque hay dos etapas para la agregación: leer los datos y formar datos intermedios (1) y fusionar los datos intermedios (2). El volcado de datos al sistema de archivos solo puede ocurrir durante la etapa 1. Si los datos temporales no se volcaron, entonces la etapa 2 puede requerir hasta la misma cantidad de memoria que en la etapa 1. diff --git a/docs/es/sql-reference/statements/select/join.md b/docs/es/sql-reference/statements/select/join.md index 158731d679c..4f0c5095008 100644 --- a/docs/es/sql-reference/statements/select/join.md +++ b/docs/es/sql-reference/statements/select/join.md @@ -141,7 +141,7 @@ Si necesita restringir el consumo de memoria de la operación de unión, use la - [Método de codificación de datos:](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) — Limits number of rows in the hash table. - [Método de codificación de datos:](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) — Limits size of the hash table. -Cuando se alcanza cualquiera de estos límites, ClickHouse actúa como el [join\_overflow\_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) configuración instruye. +Cuando se alcanza cualquiera de estos límites, ClickHouse actúa como el [join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) configuración instruye. ## Ejemplos {#examples} diff --git a/docs/es/sql-reference/statements/system.md b/docs/es/sql-reference/statements/system.md index 4189cf6d2c5..1443dd9a740 100644 --- a/docs/es/sql-reference/statements/system.md +++ b/docs/es/sql-reference/statements/system.md @@ -24,12 +24,12 @@ toc_title: SYSTEM ## RELOAD DICTIONARIES {#query_language-system-reload-dictionaries} Vuelve a cargar todos los diccionarios que se han cargado correctamente antes. -De forma predeterminada, los diccionarios se cargan perezosamente (ver [Diccionarios\_lazy\_load](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load)), por lo que en lugar de cargarse automáticamente al inicio, se inicializan en el primer acceso a través de la función dictGet o SELECT desde tablas con ENGINE = Dictionary . El `SYSTEM RELOAD DICTIONARIES` consulta vuelve a cargar dichos diccionarios (LOADED). +De forma predeterminada, los diccionarios se cargan perezosamente (ver [Diccionarios_lazy_load](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load)), por lo que en lugar de cargarse automáticamente al inicio, se inicializan en el primer acceso a través de la función dictGet o SELECT desde tablas con ENGINE = Dictionary . El `SYSTEM RELOAD DICTIONARIES` consulta vuelve a cargar dichos diccionarios (LOADED). Siempre vuelve `Ok.` independientemente del resultado de la actualización del diccionario. -## RELOAD DICTIONARY Dictionary\_name {#query_language-system-reload-dictionary} +## RELOAD DICTIONARY Dictionary_name {#query_language-system-reload-dictionary} -Recarga completamente un diccionario `dictionary_name`, independientemente del estado del diccionario (LOADED / NOT\_LOADED / FAILED). +Recarga completamente un diccionario `dictionary_name`, independientemente del estado del diccionario (LOADED / NOT_LOADED / FAILED). Siempre vuelve `Ok.` independientemente del resultado de la actualización del diccionario. El estado del diccionario se puede comprobar consultando el `system.dictionaries` tabla. @@ -41,7 +41,7 @@ SELECT name, status FROM system.dictionaries; Restablece la caché DNS interna de ClickHouse. A veces (para versiones anteriores de ClickHouse) es necesario usar este comando al cambiar la infraestructura (cambiar la dirección IP de otro servidor de ClickHouse o el servidor utilizado por los diccionarios). -Para obtener una administración de caché más conveniente (automática), consulte disable\_internal\_dns\_cache, dns\_cache\_update\_period parameters. +Para obtener una administración de caché más conveniente (automática), consulte disable_internal_dns_cache, dns_cache_update_period parameters. ## DROP MARK CACHE {#query_language-system-drop-mark-cache} @@ -49,7 +49,7 @@ Restablece la caché de marcas. Utilizado en el desarrollo de ClickHouse y prueb ## FLUSH LOGS {#query_language-system-flush_logs} -Flushes buffers of log messages to system tables (e.g. system.query\_log). Allows you to not wait 7.5 seconds when debugging. +Flushes buffers of log messages to system tables (e.g. system.query_log). Allows you to not wait 7.5 seconds when debugging. ## RELOAD CONFIG {#query_language-system-reload-config} diff --git a/docs/es/sql-reference/syntax.md b/docs/es/sql-reference/syntax.md index c3e6134d8dd..93cd44c2b91 100644 --- a/docs/es/sql-reference/syntax.md +++ b/docs/es/sql-reference/syntax.md @@ -15,9 +15,9 @@ El `INSERT` consulta utiliza ambos analizadores: INSERT INTO t VALUES (1, 'Hello, world'), (2, 'abc'), (3, 'def') ``` -El `INSERT INTO t VALUES` fragmento es analizado por el analizador completo, y los datos `(1, 'Hello, world'), (2, 'abc'), (3, 'def')` es analizado por el analizador de flujo rápido. También puede activar el analizador completo de los datos mediante el [input\_format\_values\_interpret\_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions) configuración. Cuando `input_format_values_interpret_expressions = 1`, ClickHouse primero intenta analizar valores con el analizador de flujo rápido. Si falla, ClickHouse intenta usar el analizador completo para los datos, tratándolo como un SQL [expresion](#syntax-expressions). +El `INSERT INTO t VALUES` fragmento es analizado por el analizador completo, y los datos `(1, 'Hello, world'), (2, 'abc'), (3, 'def')` es analizado por el analizador de flujo rápido. También puede activar el analizador completo de los datos mediante el [input_format_values_interpret_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions) configuración. Cuando `input_format_values_interpret_expressions = 1`, ClickHouse primero intenta analizar valores con el analizador de flujo rápido. Si falla, ClickHouse intenta usar el analizador completo para los datos, tratándolo como un SQL [expresion](#syntax-expressions). -Los datos pueden tener cualquier formato. Cuando se recibe una consulta, el servidor no calcula más de [max\_query\_size](../operations/settings/settings.md#settings-max_query_size) bytes de la solicitud en RAM (por defecto, 1 MB), y el resto se analiza la secuencia. +Los datos pueden tener cualquier formato. Cuando se recibe una consulta, el servidor no calcula más de [max_query_size](../operations/settings/settings.md#settings-max_query_size) bytes de la solicitud en RAM (por defecto, 1 MB), y el resto se analiza la secuencia. Permite evitar problemas con grandes `INSERT` consulta. Cuando se utiliza el `Values` formato en un `INSERT` consulta, puede parecer que los datos se analizan igual que las expresiones en un `SELECT` consulta, pero esto no es cierto. El `Values` formato es mucho más limitado. diff --git a/docs/es/sql-reference/table-functions/file.md b/docs/es/sql-reference/table-functions/file.md index 3880cbffe43..0d1c9b66016 100644 --- a/docs/es/sql-reference/table-functions/file.md +++ b/docs/es/sql-reference/table-functions/file.md @@ -15,7 +15,7 @@ file(path, format, structure) **Parámetros de entrada** -- `path` — The relative path to the file from [user\_files\_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Soporte de ruta a archivo siguiendo globs en modo de solo lectura: `*`, `?`, `{abc,def}` y `{N..M}` donde `N`, `M` — numbers, \``'abc', 'def'` — strings. +- `path` — The relative path to the file from [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Soporte de ruta a archivo siguiendo globs en modo de solo lectura: `*`, `?`, `{abc,def}` y `{N..M}` donde `N`, `M` — numbers, \``'abc', 'def'` — strings. - `format` — The [formato](../../interfaces/formats.md#formats) del archivo. - `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. @@ -72,12 +72,12 @@ Construcciones con `{}` son similares a la [función de tabla remota](../../sql- 1. Supongamos que tenemos varios archivos con las siguientes rutas relativas: -- ‘some\_dir/some\_file\_1’ -- ‘some\_dir/some\_file\_2’ -- ‘some\_dir/some\_file\_3’ -- ‘another\_dir/some\_file\_1’ -- ‘another\_dir/some\_file\_2’ -- ‘another\_dir/some\_file\_3’ +- ‘some_dir/some_file_1’ +- ‘some_dir/some_file_2’ +- ‘some_dir/some_file_3’ +- ‘another_dir/some_file_1’ +- ‘another_dir/some_file_2’ +- ‘another_dir/some_file_3’ 1. Consulta la cantidad de filas en estos archivos: diff --git a/docs/es/sql-reference/table-functions/hdfs.md b/docs/es/sql-reference/table-functions/hdfs.md index 706336b36c6..8c90f1d49ca 100644 --- a/docs/es/sql-reference/table-functions/hdfs.md +++ b/docs/es/sql-reference/table-functions/hdfs.md @@ -55,12 +55,12 @@ Construcciones con `{}` son similares a la [función de tabla remota](../../sql- 1. Supongamos que tenemos varios archivos con los siguientes URI en HDFS: -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_3’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_3’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_3’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_3’ 1. Consulta la cantidad de filas en estos archivos: diff --git a/docs/es/sql-reference/table-functions/index.md b/docs/es/sql-reference/table-functions/index.md index 41bd04f237e..60882327e6a 100644 --- a/docs/es/sql-reference/table-functions/index.md +++ b/docs/es/sql-reference/table-functions/index.md @@ -16,7 +16,7 @@ Puede usar funciones de tabla en: The method for creating a temporary table that is available only in the current query. The table is deleted when the query finishes. -- [CREAR TABLA COMO \](../statements/create.md#create-table-query) consulta. +- [CREAR TABLA COMO \](../statements/create.md#create-table-query) consulta. It's one of the methods of creating a table. diff --git a/docs/es/sql-reference/table-functions/remote.md b/docs/es/sql-reference/table-functions/remote.md index d614c7b3c0e..e9784cb4cd1 100644 --- a/docs/es/sql-reference/table-functions/remote.md +++ b/docs/es/sql-reference/table-functions/remote.md @@ -56,7 +56,7 @@ example01-{01..02}-1 Si tiene varios pares de llaves, genera el producto directo de los conjuntos correspondientes. -Las direcciones y partes de las direcciones entre llaves se pueden separar mediante el símbolo de tubería (\|). En este caso, los conjuntos de direcciones correspondientes se interpretan como réplicas y la consulta se enviará a la primera réplica en buen estado. Sin embargo, las réplicas se iteran en el orden establecido actualmente en el [load\_balancing](../../operations/settings/settings.md) configuración. +Las direcciones y partes de las direcciones entre llaves se pueden separar mediante el símbolo de tubería (\|). En este caso, los conjuntos de direcciones correspondientes se interpretan como réplicas y la consulta se enviará a la primera réplica en buen estado. Sin embargo, las réplicas se iteran en el orden establecido actualmente en el [load_balancing](../../operations/settings/settings.md) configuración. Ejemplo: @@ -80,6 +80,6 @@ El `remote` puede ser útil en los siguientes casos: Si el usuario no está especificado, `default` se utiliza. Si no se especifica la contraseña, se utiliza una contraseña vacía. -`remoteSecure` - igual que `remote` but with secured connection. Default port — [Tcp\_port\_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) de config o 9440. +`remoteSecure` - igual que `remote` but with secured connection. Default port — [Tcp_port_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) de config o 9440. [Artículo Original](https://clickhouse.tech/docs/en/query_language/table_functions/remote/) diff --git a/docs/es/whats-new/security-changelog.md b/docs/es/whats-new/security-changelog.md index ddba9cae073..86d2e7df2f7 100644 --- a/docs/es/whats-new/security-changelog.md +++ b/docs/es/whats-new/security-changelog.md @@ -53,7 +53,7 @@ Créditos: Andrey Krasichkov y Evgeny Sidorov del equipo de seguridad de la info ### CVE-2018-14668 {#cve-2018-14668} -“remote” función de tabla permitió símbolos arbitrarios en “user”, “password” y “default\_database” campos que llevaron a ataques de falsificación de solicitudes de protocolo cruzado. +“remote” función de tabla permitió símbolos arbitrarios en “user”, “password” y “default_database” campos que llevaron a ataques de falsificación de solicitudes de protocolo cruzado. Créditos: Andrey Krasichkov del equipo de seguridad de la información de Yandex diff --git a/docs/fa/development/build.md b/docs/fa/development/build.md index 7ba21c0fa93..db3aa96285f 100644 --- a/docs/fa/development/build.md +++ b/docs/fa/development/build.md @@ -10,7 +10,7 @@ toc_title: "\u0646\u062D\u0648\u0647 \u0633\u0627\u062E\u062A \u06A9\u0644\u06CC راهنمای زیر بر اساس سیستم لینوکس اوبونتو است. با تغییرات مناسب, همچنین باید بر روی هر توزیع لینوکس دیگر کار. -سیستم عامل های پشتیبانی شده: ایکس86\_64 و عاشق64. پشتیبانی از قدرت9 تجربی است. +سیستم عامل های پشتیبانی شده: ایکس86_64 و عاشق64. پشتیبانی از قدرت9 تجربی است. ## شرح متنی (توضیحات سایت) در صفحات شما دیده نمی شود {#install-git-cmake-python-and-ninja} diff --git a/docs/fa/development/contrib.md b/docs/fa/development/contrib.md index 3ee172e3104..25573c28125 100644 --- a/docs/fa/development/contrib.md +++ b/docs/fa/development/contrib.md @@ -30,7 +30,7 @@ toc_title: "\u06A9\u062A\u0627\u0628\u062E\u0627\u0646\u0647 \u0647\u0627\u06CC | سوالات متداول | [نمایی مجوز 2.0](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libpcg-random/LICENSE-APACHE.txt) | | libressl | [OpenSSL مجوز](https://github.com/ClickHouse-Extras/ssl/blob/master/COPYING) | | کتابدار | [لیسانس 2 بند](https://github.com/edenhill/librdkafka/blob/363dcad5a23dc29381cc626620e68ae418b3af19/LICENSE) | -| \_عرض | [CC0 1.0 جهانی](https://github.com/ClickHouse/ClickHouse/blob/master/libs/libwidechar_width/LICENSE) | +| _عرض | [CC0 1.0 جهانی](https://github.com/ClickHouse/ClickHouse/blob/master/libs/libwidechar_width/LICENSE) | | llvm | [لیسانس 3 بند](https://github.com/ClickHouse-Extras/llvm/blob/163def217817c90fb982a6daf384744d8472b92b/llvm/LICENSE.TXT) | | lz4 | [لیسانس 2 بند](https://github.com/lz4/lz4/blob/c10863b98e1503af90616ae99725ecd120265dfb/LICENSE) | | ماریادب-اتصال-ج | [الجی پی ال2.1](https://github.com/ClickHouse-Extras/mariadb-connector-c/blob/3.1/COPYING.LIB) | diff --git a/docs/fa/development/developer-instruction.md b/docs/fa/development/developer-instruction.md index 94e5535d67f..c9461748098 100644 --- a/docs/fa/development/developer-instruction.md +++ b/docs/fa/development/developer-instruction.md @@ -46,7 +46,7 @@ toc_title: "\u062F\u0633\u062A\u0648\u0631\u0627\u0644\u0639\u0645\u0644 \u062A\ git clone --recursive git@github.com:your_github_username/ClickHouse.git cd ClickHouse -توجه: لطفا جایگزین کنید *تغییر \_نامهی تو* با چه مناسب است! +توجه: لطفا جایگزین کنید *تغییر _نامهی تو* با چه مناسب است! این دستور یک دایرکتوری ایجاد خواهد کرد `ClickHouse` حاوی کپی کار از پروژه. @@ -156,7 +156,7 @@ toc_title: "\u062F\u0633\u062A\u0648\u0631\u0627\u0644\u0639\u0645\u0644 \u062A\ mkdir build cd build -شما می توانید چندین دایرکتوری های مختلف (build\_release, build\_debug ، ) برای انواع مختلف ساخت. +شما می توانید چندین دایرکتوری های مختلف (build_release, build_debug ، ) برای انواع مختلف ساخت. در حالی که در داخل `build` فهرست, پیکربندی ساخت خود را با در حال اجرا کیک. قبل از اولین اجرا, شما نیاز به تعریف متغیرهای محیطی که کامپایلر را مشخص (نسخه 9 کامپایلر شورای همکاری خلیج فارس در این مثال). diff --git a/docs/fa/development/style.md b/docs/fa/development/style.md index 7bfe43c49fe..67af6fa1e76 100644 --- a/docs/fa/development/style.md +++ b/docs/fa/development/style.md @@ -357,7 +357,7 @@ class IBlockInputStream bool info_successfully_loaded = false; ``` -**9.** اسامی `define`بازدید کنندگان و ثابت جهانی استفاده از همه\_کاپ با زیرخط. +**9.** اسامی `define`بازدید کنندگان و ثابت جهانی استفاده از همه_کاپ با زیرخط. ``` cpp #define MAX_SRC_TABLE_NAMES_TO_STORE 1000 @@ -397,7 +397,7 @@ FileQueueProcessor( timer (not m_timer) ``` -**14.** برای ثابت در یک `enum` استفاده از CamelCase با حرف بزرگ. ت\_کاپها نیز قابل قبول است. اگر `enum` غیر محلی است, استفاده از یک `enum class`. +**14.** برای ثابت در یک `enum` استفاده از CamelCase با حرف بزرگ. ت_کاپها نیز قابل قبول است. اگر `enum` غیر محلی است, استفاده از یک `enum class`. ``` cpp enum class CompressionMethod @@ -710,7 +710,7 @@ auto s = std::string{"Hello"}; **4.**سیستم عامل: لینوکس اوبونتو, مسن تر از دقیق نیست. -**5.**کد برای معماری پردازنده ایکس86\_64 نوشته شده است. +**5.**کد برای معماری پردازنده ایکس86_64 نوشته شده است. مجموعه دستورالعمل پردازنده حداقل مجموعه پشتیبانی در میان سرورهای ما است. در حال حاضر, این سوس است 4.2. diff --git a/docs/fa/development/tests.md b/docs/fa/development/tests.md index cd219f0968c..abab3230e2f 100644 --- a/docs/fa/development/tests.md +++ b/docs/fa/development/tests.md @@ -203,7 +203,7 @@ $ clickhouse benchmark --concurrency 16 < queries.tsv ریش ریش شدن کلیک هر دو با استفاده از اجرا شده است [هرزه](https://llvm.org/docs/LibFuzzer.html) و تصادفی گذاشتن نمایش داده شد. تمام تست ریش شدن باید با ضدعفونی کننده انجام شود (نشانی و تعریف نشده). -پازل برای تست ریش ریش شدن جدا شده از کد کتابخانه استفاده می شود. طبع به عنوان بخشی از کد تست اجرا و “\_fuzzer” نام پسوند. +پازل برای تست ریش ریش شدن جدا شده از کد کتابخانه استفاده می شود. طبع به عنوان بخشی از کد تست اجرا و “_fuzzer” نام پسوند. به عنوان مثال ریش ریش شدن را می توان در یافت `src/Parsers/tests/lexer_fuzzer.cpp`. تنظیمات-پازل خاص, لغت نامه ها و جسم در ذخیره می شود `tests/fuzz`. ما شما را تشویق به نوشتن تست ریش ریش شدن برای هر قابلیت که دسته ورودی کاربر. diff --git a/docs/fa/engines/table-engines/integrations/hdfs.md b/docs/fa/engines/table-engines/integrations/hdfs.md index c5689c2d852..36837362e07 100644 --- a/docs/fa/engines/table-engines/integrations/hdfs.md +++ b/docs/fa/engines/table-engines/integrations/hdfs.md @@ -73,12 +73,12 @@ SELECT * FROM hdfs_engine_table LIMIT 2 1. فرض کنید ما چندین فایل را در قالب فیلم با اوریس زیر در اچ دی ها داریم: -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_3’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_3’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_3’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_3’ 1. راه های مختلفی برای ایجاد یک جدول متشکل از تمام شش فایل وجود دارد: diff --git a/docs/fa/engines/table-engines/integrations/kafka.md b/docs/fa/engines/table-engines/integrations/kafka.md index c4ea0afad64..bdadd8b8381 100644 --- a/docs/fa/engines/table-engines/integrations/kafka.md +++ b/docs/fa/engines/table-engines/integrations/kafka.md @@ -134,7 +134,7 @@ Kafka(kafka_broker_list, kafka_topic_list, kafka_group_name, kafka_format SELECT level, sum(total) FROM daily GROUP BY level; ``` -برای بهبود عملکرد, پیام های دریافت شده را به بلوک های اندازه گروه بندی می شوند [ا\_فزونهها](../../../operations/server-configuration-parameters/settings.md#settings-max_insert_block_size). اگر بلوک در داخل تشکیل نشده است [\_خاله جریان](../../../operations/server-configuration-parameters/settings.md) میلی ثانیه, داده خواهد شد به جدول بدون در نظر گرفتن کامل از بلوک سرخ. +برای بهبود عملکرد, پیام های دریافت شده را به بلوک های اندازه گروه بندی می شوند [ا_فزونهها](../../../operations/server-configuration-parameters/settings.md#settings-max_insert_block_size). اگر بلوک در داخل تشکیل نشده است [_خاله جریان](../../../operations/server-configuration-parameters/settings.md) میلی ثانیه, داده خواهد شد به جدول بدون در نظر گرفتن کامل از بلوک سرخ. برای جلوگیری از دریافت داده های موضوع و یا تغییر منطق تبدیل جدا مشاهده محقق: diff --git a/docs/fa/engines/table-engines/mergetree-family/custom-partitioning-key.md b/docs/fa/engines/table-engines/mergetree-family/custom-partitioning-key.md index f2c010e9661..825535c9c95 100644 --- a/docs/fa/engines/table-engines/mergetree-family/custom-partitioning-key.md +++ b/docs/fa/engines/table-engines/mergetree-family/custom-partitioning-key.md @@ -117,7 +117,7 @@ drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 5 12:09 201902_4_6_1 drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 1 16:48 detached ``` -پوشهها ‘201901\_1\_1\_0’, ‘201901\_1\_7\_1’ و به همین ترتیب دایرکتوری از قطعات هستند. هر بخش مربوط به پارتیشن مربوطه و شامل داده ها فقط برای یک ماه خاص (جدول در این مثال پارتیشن بندی توسط ماه). +پوشهها ‘201901_1_1_0’, ‘201901_1_7_1’ و به همین ترتیب دایرکتوری از قطعات هستند. هر بخش مربوط به پارتیشن مربوطه و شامل داده ها فقط برای یک ماه خاص (جدول در این مثال پارتیشن بندی توسط ماه). این `detached` دایرکتوری شامل قطعات است که از جدول با استفاده از جدا شد [DETACH](../../../sql-reference/statements/alter.md#alter_detach-partition) پرس و جو. قطعات خراب نیز به این دایرکتوری منتقل, به جای اینکه حذف. سرور از قطعات از `detached` directory. You can add, delete, or modify the data in this directory at any time – the server will not know about this until you run the [ATTACH](../../../sql-reference/statements/alter.md#alter_attach-partition) پرس و جو. diff --git a/docs/fa/engines/table-engines/mergetree-family/graphitemergetree.md b/docs/fa/engines/table-engines/mergetree-family/graphitemergetree.md index 81aafd19058..2b2ed45152d 100644 --- a/docs/fa/engines/table-engines/mergetree-family/graphitemergetree.md +++ b/docs/fa/engines/table-engines/mergetree-family/graphitemergetree.md @@ -81,7 +81,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ## پیکربندی رولپ {#rollup-configuration} -تنظیمات برای خلاصه توسط تعریف [لغزش \_ نمودار](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) پارامتر در پیکربندی سرور. نام پارامتر می تواند هر. شما می توانید تنظیمات متعددی ایجاد کنید و برای جداول مختلف استفاده کنید. +تنظیمات برای خلاصه توسط تعریف [لغزش _ نمودار](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) پارامتر در پیکربندی سرور. نام پارامتر می تواند هر. شما می توانید تنظیمات متعددی ایجاد کنید و برای جداول مختلف استفاده کنید. ساختار پیکربندی رولپ: diff --git a/docs/fa/engines/table-engines/mergetree-family/mergetree.md b/docs/fa/engines/table-engines/mergetree-family/mergetree.md index be4f8454ef1..264c7379466 100644 --- a/docs/fa/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/fa/engines/table-engines/mergetree-family/mergetree.md @@ -330,7 +330,7 @@ INDEX sample_index3 (lower(str), str) TYPE ngrambf_v1(3, 256, 2, 0) GRANULARITY این `set` شاخص را می توان با تمام توابع استفاده می شود. زیر مجموعه های تابع برای شاخص های دیگر در جدول زیر نشان داده شده است. -| تابع (اپراتور) / شاخص | کلید اصلی | مینمکس | نمرمبف1 | توکنبف1 | ت\_ضعیت | +| تابع (اپراتور) / شاخص | کلید اصلی | مینمکس | نمرمبف1 | توکنبف1 | ت_ضعیت | |----------------------------------------------------------------------------------------------------------------------|-----------|--------|---------|---------|---------| | [اطلاعات دقیق)](../../../sql-reference/functions/comparison-functions.md#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | | [نقلقولهای جدید از این نویسنده=, \<\>)](../../../sql-reference/functions/comparison-functions.md#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | @@ -495,7 +495,7 @@ ALTER TABLE example_table - Volume — Ordered set of equal disks (similar to [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures)). - Storage policy — Set of volumes and the rules for moving data between them. -اسامی داده شده به اشخاص توصیف شده را می توان در جداول سیستم یافت می شود, [سیستم.داستان\_یابی](../../../operations/system-tables.md#system_tables-storage_policies) و [سیستم.دیسکها](../../../operations/system-tables.md#system_tables-disks). برای اعمال یکی از سیاست های ذخیره سازی پیکربندی شده برای یک جدول از `storage_policy` تنظیم از `MergeTree`- جداول خانواده موتور . +اسامی داده شده به اشخاص توصیف شده را می توان در جداول سیستم یافت می شود, [سیستم.داستان_یابی](../../../operations/system-tables.md#system_tables-storage_policies) و [سیستم.دیسکها](../../../operations/system-tables.md#system_tables-disks). برای اعمال یکی از سیاست های ذخیره سازی پیکربندی شده برای یک جدول از `storage_policy` تنظیم از `MergeTree`- جداول خانواده موتور . ### پیکربندی {#table_engine-mergetree-multiple-volumes_configure} @@ -642,7 +642,7 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' تحت هود جهش و پارتیشن انجماد استفاده از [لینک های سخت](https://en.wikipedia.org/wiki/Hard_link). لینک های سخت بین دیسک های مختلف پشتیبانی نمی شوند بنابراین در چنین مواردی قطعات حاصل شده بر روی دیسک های مشابه به عنوان اولیه ذخیره می شوند. در پس زمینه, قطعات بین حجم بر اساس مقدار فضای رایگان نقل مکان کرد (`move_factor` پارامتر) با توجه به سفارش حجم در فایل پیکربندی اعلام کرد. -داده ها هرگز از گذشته و به یکی از اولین منتقل شده است. ممکن است از جداول سیستم استفاده کنید [سیستم.\_خروج](../../../operations/system-tables.md#system_tables-part-log) (زمینه `type = MOVE_PART`) و [سیستم.قطعات](../../../operations/system-tables.md#system_tables-parts) (فیلدها `path` و `disk`) برای نظارت بر حرکت پس زمینه . همچنین, اطلاعات دقیق را می توان در سیاهههای مربوط به سرور پیدا شده است. +داده ها هرگز از گذشته و به یکی از اولین منتقل شده است. ممکن است از جداول سیستم استفاده کنید [سیستم._خروج](../../../operations/system-tables.md#system_tables-part-log) (زمینه `type = MOVE_PART`) و [سیستم.قطعات](../../../operations/system-tables.md#system_tables-parts) (فیلدها `path` و `disk`) برای نظارت بر حرکت پس زمینه . همچنین, اطلاعات دقیق را می توان در سیاهههای مربوط به سرور پیدا شده است. کاربر می تواند نیروی حرکت بخشی یا پارتیشن از یک حجم به دیگری با استفاده از پرس و جو [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](../../../sql-reference/statements/alter.md#alter_move-partition), تمام محدودیت برای عملیات پس زمینه در نظر گرفته شود. پرس و جو شروع یک حرکت به خودی خود و منتظر نیست برای عملیات پس زمینه به پایان خواهد رسید. کاربر یک پیام خطا اگر فضای رایگان به اندازه کافی در دسترس است و یا اگر هر یک از شرایط مورد نیاز را ملاقات کرد. diff --git a/docs/fa/engines/table-engines/mergetree-family/replication.md b/docs/fa/engines/table-engines/mergetree-family/replication.md index 9687b890c0e..be37a588968 100644 --- a/docs/fa/engines/table-engines/mergetree-family/replication.md +++ b/docs/fa/engines/table-engines/mergetree-family/replication.md @@ -59,7 +59,7 @@ toc_title: "\u062A\u06A9\u0631\u0627\u0631 \u062F\u0627\u062F\u0647 \u0647\u0627 اگر باغ وحش در فایل پیکربندی تنظیم نشده, شما می توانید جداول تکرار ایجاد کنید, و هر جداول تکرار موجود خواهد شد فقط به عنوان خوانده شده. -باغ وحش در استفاده نمی شود `SELECT` نمایش داده شد به دلیل تکرار می کند عملکرد تاثیر نمی گذارد `SELECT` و نمایش داده شد اجرا فقط به همان سرعتی که برای جداول غیر تکرار انجام می دهند. هنگامی که پرس و جو جداول تکرار توزیع, رفتار کلیک است که توسط تنظیمات کنترل [\_شروع مجدد \_شروع مجدد \_شروع مجدد \_کاربری](../../../operations/settings/settings.md#settings-max_replica_delay_for_distributed_queries) و [شناسه بسته:](../../../operations/settings/settings.md#settings-fallback_to_stale_replicas_for_distributed_queries). +باغ وحش در استفاده نمی شود `SELECT` نمایش داده شد به دلیل تکرار می کند عملکرد تاثیر نمی گذارد `SELECT` و نمایش داده شد اجرا فقط به همان سرعتی که برای جداول غیر تکرار انجام می دهند. هنگامی که پرس و جو جداول تکرار توزیع, رفتار کلیک است که توسط تنظیمات کنترل [_شروع مجدد _شروع مجدد _شروع مجدد _کاربری](../../../operations/settings/settings.md#settings-max_replica_delay_for_distributed_queries) و [شناسه بسته:](../../../operations/settings/settings.md#settings-fallback_to_stale_replicas_for_distributed_queries). برای هر `INSERT` پرس و جو, حدود ده ورودی از طریق معاملات چند به باغ وحش دار اضافه. (به عبارت دقیق تر, این است که برای هر بلوک قرار داده شده از داده; پرس و جو درج شامل یک بلوک و یا یک بلوک در هر `max_insert_block_size = 1048576` ردیف) این منجر به زمان شروع کمی طولانی تر برای `INSERT` در مقایسه با جداول غیر تکرار. اما اگر شما به دنبال توصیه برای وارد کردن داده ها در دسته بیش از یک `INSERT` در هر ثانیه هیچ مشکلی ایجاد نمی کند. کل خوشه محل کلیک مورد استفاده برای هماهنگی یک خوشه باغ وحش در مجموع چند صد است `INSERTs` در هر ثانیه. توان در درج داده (تعداد ردیف در ثانیه) فقط به عنوان بالا به عنوان داده های غیر تکرار شده است. diff --git a/docs/fa/engines/table-engines/special/buffer.md b/docs/fa/engines/table-engines/special/buffer.md index 12d21a899e0..e96a963af43 100644 --- a/docs/fa/engines/table-engines/special/buffer.md +++ b/docs/fa/engines/table-engines/special/buffer.md @@ -36,7 +36,7 @@ Buffer(database, table, num_layers, min_time, max_time, min_rows, max_rows, min_ CREATE TABLE merge.hits_buffer AS merge.hits ENGINE = Buffer(merge, hits, 16, 10, 100, 10000, 1000000, 10000000, 100000000) ``` -ایجاد یک ‘merge.hits\_buffer’ جدول با ساختار مشابه ‘merge.hits’ و با استفاده از موتور بافر. هنگام نوشتن به این جدول, داده ها در رم بافر و بعد به نوشته ‘merge.hits’ جدول 16 بافر ایجاد می کند. اگر 100 ثانیه گذشت یا یک میلیون ردیف نوشته شده یا 100 مگابایت از داده ها نوشته شده است داده ها در هر یک از فوران است; یا اگر به طور همزمان 10 ثانیه گذشت و 10000 ردیف و 10 مگابایت داده ها نوشته شده است. مثلا, اگر فقط یک ردیف نوشته شده است, بعد از 100 ثانیه سرخ خواهد شد, مهم نیست که چه. اما اگر بسیاری از ردیف نوشته شده است, داده خواهد شد هر چه زودتر سرخ. +ایجاد یک ‘merge.hits_buffer’ جدول با ساختار مشابه ‘merge.hits’ و با استفاده از موتور بافر. هنگام نوشتن به این جدول, داده ها در رم بافر و بعد به نوشته ‘merge.hits’ جدول 16 بافر ایجاد می کند. اگر 100 ثانیه گذشت یا یک میلیون ردیف نوشته شده یا 100 مگابایت از داده ها نوشته شده است داده ها در هر یک از فوران است; یا اگر به طور همزمان 10 ثانیه گذشت و 10000 ردیف و 10 مگابایت داده ها نوشته شده است. مثلا, اگر فقط یک ردیف نوشته شده است, بعد از 100 ثانیه سرخ خواهد شد, مهم نیست که چه. اما اگر بسیاری از ردیف نوشته شده است, داده خواهد شد هر چه زودتر سرخ. هنگامی که سرور متوقف شده است, با جدول قطره و یا جدا جدول, داده های بافر نیز به جدول مقصد سرخ. @@ -58,7 +58,7 @@ CREATE TABLE merge.hits_buffer AS merge.hits ENGINE = Buffer(merge, hits, 16, 10 هنگام اضافه کردن داده ها به یک بافر, یکی از بافر قفل شده است. این باعث تاخیر اگر یک عملیات به عنوان خوانده شده است به طور همزمان از جدول انجام. -داده هایی که به یک جدول بافر قرار داده شده ممکن است در نهایت در جدول تابع در جهت های مختلف و در بلوک های مختلف. به خاطر همین, یک جدول بافر دشوار است به استفاده از برای نوشتن به یک سقوط به درستی. برای جلوگیری از مشکلات, شما می توانید مجموعه ‘num\_layers’ به 1. +داده هایی که به یک جدول بافر قرار داده شده ممکن است در نهایت در جدول تابع در جهت های مختلف و در بلوک های مختلف. به خاطر همین, یک جدول بافر دشوار است به استفاده از برای نوشتن به یک سقوط به درستی. برای جلوگیری از مشکلات, شما می توانید مجموعه ‘num_layers’ به 1. اگر جدول مقصد تکرار شده است, برخی از ویژگی های مورد انتظار از جداول تکرار از دست داده در هنگام نوشتن به یک جدول بافر. تغییرات تصادفی به منظور از سطر و اندازه قطعات داده باعث تقسیم بندی داده ها به ترک کار, به این معنی که ممکن است به یک قابل اعتماد ‘exactly once’ ارسال به جداول تکرار. diff --git a/docs/fa/engines/table-engines/special/distributed.md b/docs/fa/engines/table-engines/special/distributed.md index d9c3f85d185..628f2fdd4f6 100644 --- a/docs/fa/engines/table-engines/special/distributed.md +++ b/docs/fa/engines/table-engines/special/distributed.md @@ -85,13 +85,13 @@ logs – The cluster name in the server's config file. پارامترها `host`, `port` و در صورت تمایل `user`, `password`, `secure`, `compression` برای هر سرور مشخص شده است: - `host` – The address of the remote server. You can use either the domain or the IPv4 or IPv6 address. If you specify the domain, the server makes a DNS request when it starts, and the result is stored as long as the server is running. If the DNS request fails, the server doesn't start. If you change the DNS record, restart the server. -- `port` – The TCP port for messenger activity (‘tcp\_port’ در پیکربندی, معمولا به مجموعه 9000). نه اشتباه آن را با http\_port. +- `port` – The TCP port for messenger activity (‘tcp_port’ در پیکربندی, معمولا به مجموعه 9000). نه اشتباه آن را با http_port. - `user` – Name of the user for connecting to a remote server. Default value: default. This user must have access to connect to the specified server. Access is configured in the users.xml file. For more information, see the section [حقوق دسترسی](../../../operations/access-rights.md). - `password` – The password for connecting to a remote server (not masked). Default value: empty string. - `secure` - استفاده از اس اس ال برای اتصال, معمولا شما همچنین باید تعریف `port` = 9440. سرور باید گوش کند `9440` و گواهی صحیح. - `compression` - استفاده از فشرده سازی داده ها. مقدار پیش فرض: درست. -When specifying replicas, one of the available replicas will be selected for each of the shards when reading. You can configure the algorithm for load balancing (the preference for which replica to access) – see the [\_تبالسازی](../../../operations/settings/settings.md#settings-load_balancing) تنظیمات. +When specifying replicas, one of the available replicas will be selected for each of the shards when reading. You can configure the algorithm for load balancing (the preference for which replica to access) – see the [_تبالسازی](../../../operations/settings/settings.md#settings-load_balancing) تنظیمات. اگر ارتباط با سرور ایجاد نشده است, وجود خواهد داشت تلاش برای ارتباط با یک ایست کوتاه. اگر اتصال شکست خورده, ماکت بعدی انتخاب خواهد شد, و به همین ترتیب برای همه کپی. اگر تلاش اتصال برای تمام کپی شکست خورده, تلاش تکرار خواهد شد به همان شیوه, چندین بار. این کار به نفع حالت ارتجاعی, اما تحمل گسل کامل را فراهم نمی کند: یک سرور از راه دور ممکن است اتصال قبول, اما ممکن است کار نمی کند, و یا کار ضعیف. @@ -113,13 +113,13 @@ The Distributed engine requires writing clusters to the config file. Clusters fr هر سفال می تواند وزن تعریف شده در فایل پیکربندی داشته باشد. به طور پیش فرض, وزن به یک برابر است. داده ها در سراسر خرده ریز در مقدار متناسب با وزن سفال توزیع. مثلا, اگر دو خرده ریز وجود دارد و برای اولین بار دارای وزن 9 در حالی که دوم دارای وزن 10, برای اولین بار ارسال خواهد شد 9 / 19 بخش هایی از ردیف, و دوم ارسال خواهد شد 10 / 19. -هر سفال می تواند داشته باشد ‘internal\_replication’ پارامتر تعریف شده در فایل پیکربندی. +هر سفال می تواند داشته باشد ‘internal_replication’ پارامتر تعریف شده در فایل پیکربندی. اگر این پارامتر قرار است به ‘true’ عملیات نوشتن اولین ماکت سالم را انتخاب می کند و داده ها را می نویسد. با استفاده از این جایگزین اگر جدول توزیع شده “looks at” جداول تکرار. به عبارت دیگر اگر جدول ای که داده ها نوشته می شود خود را تکرار می کند. اگر قرار است ‘false’ (به طور پیش فرض), داده ها به تمام کپی نوشته شده. در اصل این بدان معنی است که توزیع جدول تکرار داده های خود را. این بدتر از استفاده از جداول تکرار شده است زیرا سازگاری کپی ها بررسی نشده است و در طول زمان حاوی اطلاعات کمی متفاوت خواهد بود. -برای انتخاب سفال که یک ردیف از داده های فرستاده شده به sharding بیان تجزيه و تحليل است و آن باقی مانده است از تقسیم آن با وزن کلی خرده ریز. ردیف به سفال که مربوط به نیمه فاصله از باقی مانده از ارسال ‘prev\_weight’ به ‘prev\_weights + weight’ کجا ‘prev\_weights’ وزن کل خرده ریز با کمترین تعداد است, و ‘weight’ وزن این سفال است. مثلا, اگر دو خرده ریز وجود دارد, و برای اولین بار دارای یک وزن 9 در حالی که دوم دارای وزن 10, ردیف خواهد شد به سفال اول برای باقی مانده از محدوده ارسال \[0, 9), و دوم برای باقی مانده از محدوده \[9, 19). +برای انتخاب سفال که یک ردیف از داده های فرستاده شده به sharding بیان تجزيه و تحليل است و آن باقی مانده است از تقسیم آن با وزن کلی خرده ریز. ردیف به سفال که مربوط به نیمه فاصله از باقی مانده از ارسال ‘prev_weight’ به ‘prev_weights + weight’ کجا ‘prev_weights’ وزن کل خرده ریز با کمترین تعداد است, و ‘weight’ وزن این سفال است. مثلا, اگر دو خرده ریز وجود دارد, و برای اولین بار دارای یک وزن 9 در حالی که دوم دارای وزن 10, ردیف خواهد شد به سفال اول برای باقی مانده از محدوده ارسال \[0, 9), و دوم برای باقی مانده از محدوده \[9, 19). بیان شاردینگ می تواند هر عبارت از ثابت ها و ستون های جدول که یک عدد صحیح را برمی گرداند. برای مثال شما می توانید با استفاده از بیان ‘rand()’ برای توزیع تصادفی داده ها یا ‘UserID’ برای توزیع توسط باقی مانده از تقسیم شناسه کاربر (سپس داده ها از یک کاربر تنها بر روی یک سفال تنها اقامت, که ساده در حال اجرا در و پیوستن به کاربران). اگر یکی از ستون ها به طور مساوی توزیع نشده باشد می توانید در یک تابع هش قرار دهید: اینتاش64 (شناسه). @@ -136,7 +136,7 @@ SELECT queries are sent to all the shards and work regardless of how data is dis اگر سرور متوقف به وجود داشته باشد و یا راه اندازی مجدد خشن بود (مثلا, پس از یک شکست دستگاه) پس از قرار دادن به یک جدول توزیع, داده های درج شده ممکن است از دست داده. اگر بخشی از داده های خراب شده در دایرکتوری جدول شناسایی شود به ‘broken’ دایرکتوری فرعی و دیگر استفاده می شود. -پردازش پرس و جو در سراسر تمام کپی در یک سفال واحد موازی است زمانی که گزینه حداکثر\_پرورالهراپیلاس فعال است. برای کسب اطلاعات بیشتر به بخش مراجعه کنید [بیشینه\_راپرال\_راپیکال](../../../operations/settings/settings.md#settings-max_parallel_replicas). +پردازش پرس و جو در سراسر تمام کپی در یک سفال واحد موازی است زمانی که گزینه حداکثر_پرورالهراپیلاس فعال است. برای کسب اطلاعات بیشتر به بخش مراجعه کنید [بیشینه_راپرال_راپیکال](../../../operations/settings/settings.md#settings-max_parallel_replicas). ## ستونهای مجازی {#virtual-columns} diff --git a/docs/fa/engines/table-engines/special/external-data.md b/docs/fa/engines/table-engines/special/external-data.md index c55592faed5..3c6a1bd69b5 100644 --- a/docs/fa/engines/table-engines/special/external-data.md +++ b/docs/fa/engines/table-engines/special/external-data.md @@ -27,10 +27,10 @@ toc_title: "\u062F\u0627\u062F\u0647\u0647\u0627\u06CC \u062E\u0627\u0631\u062C\ **–file** – Path to the file with the table dump, or -, which refers to stdin. فقط یک جدول را می توان از استدین بازیابی. -پارامترهای زیر اختیاری هستند: **–name**– Name of the table. If omitted, \_data is used. +پارامترهای زیر اختیاری هستند: **–name**– Name of the table. If omitted, _data is used. **–format** – Data format in the file. If omitted, TabSeparated is used. -یکی از پارامترهای زیر مورد نیاز است:**–types** – A list of comma-separated column types. For example: `UInt64,String`. The columns will be named \_1, \_2, … +یکی از پارامترهای زیر مورد نیاز است:**–types** – A list of comma-separated column types. For example: `UInt64,String`. The columns will be named _1, _2, … **–structure**– The table structure in the format`UserID UInt64`, `URL String`. تعریف نام ستون و انواع. فایل های مشخص شده در ‘file’ خواهد شد با فرمت مشخص شده در تجزیه ‘format’ با استفاده از انواع داده های مشخص شده در ‘types’ یا ‘structure’. جدول خواهد شد به سرور ارسال شده و در دسترس وجود دارد به عنوان یک جدول موقت با نام در ‘name’. @@ -48,7 +48,7 @@ $ cat /etc/passwd | sed 's/:/\t/g' | clickhouse-client --query="SELECT shell, co /bin/sync 1 ``` -هنگام استفاده از رابط اچ تی پی, داده های خارجی در قالب چند/فرم داده به تصویب رسید. هر جدول به عنوان یک فایل جداگانه منتقل می شود. نام جدول از نام فایل گرفته شده است. این ‘query\_string’ پارامترهای منتقل می شود ‘name\_format’, ‘name\_types’ و ‘name\_structure’ کجا ‘name’ نام جدول که این پارامترها به مطابقت است. معنای پارامترهای همان است که در هنگام استفاده از مشتری خط فرمان است. +هنگام استفاده از رابط اچ تی پی, داده های خارجی در قالب چند/فرم داده به تصویب رسید. هر جدول به عنوان یک فایل جداگانه منتقل می شود. نام جدول از نام فایل گرفته شده است. این ‘query_string’ پارامترهای منتقل می شود ‘name_format’, ‘name_types’ و ‘name_structure’ کجا ‘name’ نام جدول که این پارامترها به مطابقت است. معنای پارامترهای همان است که در هنگام استفاده از مشتری خط فرمان است. مثال: diff --git a/docs/fa/engines/table-engines/special/join.md b/docs/fa/engines/table-engines/special/join.md index 07518b5b897..b527767eb90 100644 --- a/docs/fa/engines/table-engines/special/join.md +++ b/docs/fa/engines/table-engines/special/join.md @@ -93,9 +93,9 @@ SELECT joinGet('id_val_join', 'val', toUInt32(1)) هنگام ایجاد یک جدول تنظیمات زیر اعمال می شود: - [ارزشهای خبری عبارتند از:](../../../operations/settings/settings.md#join_use_nulls) -- [\_پاک کردن \_روشن گرافیک](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) -- [\_پویش همیشگی](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) -- [\_شروع مجدد](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) +- [_پاک کردن _روشن گرافیک](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) +- [_پویش همیشگی](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) +- [_شروع مجدد](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) - [نمایش سایت](../../../operations/settings/settings.md#settings-join_any_take_last_row) این `Join`- جداول موتور نمی تواند مورد استفاده قرار گیرد `GLOBAL JOIN` عملیات. diff --git a/docs/fa/engines/table-engines/special/merge.md b/docs/fa/engines/table-engines/special/merge.md index ecc5e62fbf9..e66fdbc2013 100644 --- a/docs/fa/engines/table-engines/special/merge.md +++ b/docs/fa/engines/table-engines/special/merge.md @@ -31,7 +31,7 @@ Regular expressions — [شماره 2](https://github.com/google/re2) (پشتی مثال 2: -بیایید می گویند شما باید یک جدول (WatchLog\_old) و تصمیم به تغییر پارتیشن بندی بدون حرکت داده ها به یک جدول جدید (WatchLog\_new) و شما نیاز به مراجعه به داده ها از هر دو جدول. +بیایید می گویند شما باید یک جدول (WatchLog_old) و تصمیم به تغییر پارتیشن بندی بدون حرکت داده ها به یک جدول جدید (WatchLog_new) و شما نیاز به مراجعه به داده ها از هر دو جدول. ``` sql CREATE TABLE WatchLog_old(date Date, UserId Int64, EventType String, Cnt UInt64) diff --git a/docs/fa/faq/general.md b/docs/fa/faq/general.md index 05065cb66cd..91870614d3f 100644 --- a/docs/fa/faq/general.md +++ b/docs/fa/faq/general.md @@ -17,7 +17,7 @@ toc_title: "\u0633\u0648\u0627\u0644\u0627\u062A \u0639\u0645\u0648\u0645\u06CC" ## اگر من یک مشکل با کدگذاریها در هنگام استفاده از اوراکل از طریق ان بی سی دارند? {#oracle-odbc-encodings} -اگر شما استفاده از اوراکل از طریق راننده او بی سی به عنوان یک منبع از لغت نامه های خارجی, شما نیاز به تنظیم مقدار صحیح برای `NLS_LANG` متغیر محیطی در `/etc/default/clickhouse`. برای کسب اطلاعات بیشتر, دیدن [اوراکل NLS\_LANG پرسش و پاسخ](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html). +اگر شما استفاده از اوراکل از طریق راننده او بی سی به عنوان یک منبع از لغت نامه های خارجی, شما نیاز به تنظیم مقدار صحیح برای `NLS_LANG` متغیر محیطی در `/etc/default/clickhouse`. برای کسب اطلاعات بیشتر, دیدن [اوراکل NLS_LANG پرسش و پاسخ](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html). **مثال** diff --git a/docs/fa/getting-started/example-datasets/amplab-benchmark.md b/docs/fa/getting-started/example-datasets/amplab-benchmark.md index 9c76980e6d2..593a3a2d669 100644 --- a/docs/fa/getting-started/example-datasets/amplab-benchmark.md +++ b/docs/fa/getting-started/example-datasets/amplab-benchmark.md @@ -11,7 +11,7 @@ toc_title: "\u0645\u0639\u06CC\u0627\u0631 \u0628\u0632\u0631\u06AF \u062F\u0627 ببینید https://amplab.cs.berkeley.edu/benchmark/ -ثبت نام برای یک حساب کاربری رایگان در https://aws.amazon.com. این نیاز به یک کارت اعتباری, پست الکترونیک, و شماره تلفن. یک کلید دسترسی جدید در https://console.aws.amazon.com/iam/home?nc2=h\_m\_sc\#security\_credential +ثبت نام برای یک حساب کاربری رایگان در https://aws.amazon.com. این نیاز به یک کارت اعتباری, پست الکترونیک, و شماره تلفن. یک کلید دسترسی جدید در https://console.aws.amazon.com/iam/home?nc2=h_m_sc#security_credential زیر را در کنسول اجرا کنید: diff --git a/docs/fa/getting-started/example-datasets/metrica.md b/docs/fa/getting-started/example-datasets/metrica.md index fc753b826c2..b1bdf3fd131 100644 --- a/docs/fa/getting-started/example-datasets/metrica.md +++ b/docs/fa/getting-started/example-datasets/metrica.md @@ -10,7 +10,7 @@ toc_title: "\u06CC\u0627\u0646\u062F\u06A9\u0633\u0627\u0637\u0644\u0627\u0639\u مجموعه داده شامل دو جدول حاوی داده های ناشناس در مورد بازدید (`hits_v1`) و بازدیدکننده داشته است (`visits_v1`) یاندکس . متریکا شما می توانید اطلاعات بیشتر در مورد یاندکس به عنوان خوانده شده.متریکا در [تاریخچه کلیک](../../introduction/history.md) بخش. -مجموعه داده ها شامل دو جدول است که هر کدام می توانند به عنوان یک فشرده دانلود شوند `tsv.xz` فایل و یا به عنوان پارتیشن تهیه شده است. علاوه بر این, یک نسخه طولانی از `hits` جدول حاوی 100 میلیون ردیف به عنوان تسو در دسترس است https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits\_100m\_obfuscated\_v1.tsv.xz و به عنوان پارتیشن تهیه شده در https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits\_100m\_obfuscated\_v1.tar.xz. +مجموعه داده ها شامل دو جدول است که هر کدام می توانند به عنوان یک فشرده دانلود شوند `tsv.xz` فایل و یا به عنوان پارتیشن تهیه شده است. علاوه بر این, یک نسخه طولانی از `hits` جدول حاوی 100 میلیون ردیف به عنوان تسو در دسترس است https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits_100m_obfuscated_v1.tsv.xz و به عنوان پارتیشن تهیه شده در https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits_100m_obfuscated_v1.tar.xz. ## اخذ جداول از پارتیشن های تهیه شده {#obtaining-tables-from-prepared-partitions} diff --git a/docs/fa/getting-started/example-datasets/nyc-taxi.md b/docs/fa/getting-started/example-datasets/nyc-taxi.md index 1fc135f001e..56255e1e09b 100644 --- a/docs/fa/getting-started/example-datasets/nyc-taxi.md +++ b/docs/fa/getting-started/example-datasets/nyc-taxi.md @@ -198,7 +198,7 @@ real 75m56.214s (وارد کردن داده ها به طور مستقیم از پستگرس نیز ممکن است با استفاده از `COPY ... TO PROGRAM`.) -Unfortunately, all the fields associated with the weather (precipitation…average\_wind\_speed) were filled with NULL. Because of this, we will remove them from the final data set. +Unfortunately, all the fields associated with the weather (precipitation…average_wind_speed) were filled with NULL. Because of this, we will remove them from the final data set. برای شروع, ما یک جدول بر روی یک سرور ایجاد. بعد ما را به جدول توزیع. diff --git a/docs/fa/getting-started/install.md b/docs/fa/getting-started/install.md index d3e39e97c80..50a9d6230b4 100644 --- a/docs/fa/getting-started/install.md +++ b/docs/fa/getting-started/install.md @@ -10,9 +10,9 @@ toc_title: "\u0646\u0635\u0628 \u0648 \u0631\u0627\u0647 \u0627\u0646\u062F\u062 ## سیستم مورد نیاز {#system-requirements} -ClickHouse می تواند اجرا بر روی هر Linux, FreeBSD یا سیستم عامل Mac OS X با x86\_64, AArch64 یا PowerPC64LE معماری CPU. +ClickHouse می تواند اجرا بر روی هر Linux, FreeBSD یا سیستم عامل Mac OS X با x86_64, AArch64 یا PowerPC64LE معماری CPU. -رسمی از پیش ساخته شده باینری به طور معمول وارد شده برای ایکس86\_64 و اهرم بورس تحصیلی 4.2 مجموعه دستورالعمل, بنابراین مگر اینکه در غیر این صورت اعلام کرد استفاده از پردازنده است که پشتیبانی می شود یک سیستم اضافی مورد نیاز. در اینجا دستور برای بررسی اگر پردازنده فعلی دارای پشتیبانی برای اس اس 4.2: +رسمی از پیش ساخته شده باینری به طور معمول وارد شده برای ایکس86_64 و اهرم بورس تحصیلی 4.2 مجموعه دستورالعمل, بنابراین مگر اینکه در غیر این صورت اعلام کرد استفاده از پردازنده است که پشتیبانی می شود یک سیستم اضافی مورد نیاز. در اینجا دستور برای بررسی اگر پردازنده فعلی دارای پشتیبانی برای اس اس 4.2: ``` bash $ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not supported" diff --git a/docs/fa/index.md b/docs/fa/index.md index 4e32b4b489c..2637d22638c 100644 --- a/docs/fa/index.md +++ b/docs/fa/index.md @@ -8,10 +8,10 @@ ClickHouse یک مدیریت دیتابیس (DBMS) ستون گرا برای پر | Row | WatchID | JavaEnable | Title | GoodEvent | EventTime | |-----|---------------------|------------|--------------------|-----------|---------------------| -| \#0 | 5385521489354350662 | 1 | Investor Relations | 1 | 2016-05-18 05:19:20 | -| \#1 | 5385521490329509958 | 0 | Contact us | 1 | 2016-05-18 08:10:20 | -| \#2 | 5385521489953706054 | 1 | Mission | 1 | 2016-05-18 07:38:00 | -| \#N | … | … | … | … | … | +| #0 | 5385521489354350662 | 1 | Investor Relations | 1 | 2016-05-18 05:19:20 | +| #1 | 5385521490329509958 | 0 | Contact us | 1 | 2016-05-18 08:10:20 | +| #2 | 5385521489953706054 | 1 | Mission | 1 | 2016-05-18 07:38:00 | +| #N | … | … | … | … | … | به این صورت، تمام مقادیر مربوط به یک سطر (رکورد) به صورت فیزیکی و در کنار یکدگیر ذخیره سازی می شوند. @@ -20,7 +20,7 @@ ClickHouse یک مدیریت دیتابیس (DBMS) ستون گرا برای پر در یک دیتابیس ستون گرا، داده ها به شکل زیر ذخیره سازی می شوند: -| Row: | \#0 | \#1 | \#2 | \#N | +| Row: | #0 | #1 | #2 | #N | |-------------|---------------------|---------------------|---------------------|-----| | WatchID: | 5385521489354350662 | 5385521490329509958 | 5385521489953706054 | … | | JavaEnable: | 1 | 0 | 1 | … | diff --git a/docs/fa/interfaces/formats.md b/docs/fa/interfaces/formats.md index 6a70badfbd4..b21328fe07f 100644 --- a/docs/fa/interfaces/formats.md +++ b/docs/fa/interfaces/formats.md @@ -202,7 +202,7 @@ SELECT * FROM nestedt FORMAT TSV - `min` ردیف با حداقل مقدار در `format_template_row` فرمت (هنگامی که افراط و به مجموعه 1) - `max` ردیف با حداکثر ارزش در است `format_template_row` فرمت (هنگامی که افراط و به مجموعه 1) - `rows` تعداد کل ردیف خروجی است -- `rows_before_limit` است حداقل تعداد ردیف وجود دارد که بدون محدودیت بوده است. خروجی تنها در صورتی که پرس و جو شامل حد. اگر پرس و جو شامل گروه های, ردیف ها\_افور\_لیمیت\_تلاست تعداد دقیق ردیف وجود دارد که بدون محدودیت بوده است. +- `rows_before_limit` است حداقل تعداد ردیف وجود دارد که بدون محدودیت بوده است. خروجی تنها در صورتی که پرس و جو شامل حد. اگر پرس و جو شامل گروه های, ردیف ها_افور_لیمیت_تلاست تعداد دقیق ردیف وجود دارد که بدون محدودیت بوده است. - `time` زمان اجرای درخواست در ثانیه است - `rows_read` است تعداد ردیف خوانده شده است - `bytes_read` تعداد بایت (غیر فشرده) خوانده شده است @@ -353,21 +353,21 @@ Both data output and parsing are supported in this format. For parsing, any orde با کاما از هم جدا فرمت ارزش ([RFC](https://tools.ietf.org/html/rfc4180)). -هنگام قالب بندی, ردیف در دو نقل قول محصور. نقل قول دو در داخل یک رشته خروجی به عنوان دو نقل قول دو در یک ردیف است. هیچ قانون دیگری برای فرار از شخصیت وجود دارد. تاریخ و تاریخ زمان در دو نقل قول محصور شده است. اعداد خروجی بدون نقل قول. ارزش ها توسط یک شخصیت جداساز از هم جدا, که `,` به طور پیش فرض. شخصیت جداساز در تنظیمات تعریف شده است [\_مخفی کردن \_قابلیت \_جدید](../operations/settings/settings.md#settings-format_csv_delimiter). ردیف ها با استفاده از خوراک خط یونیکس جدا می شوند. ارریس در سی سی اس وی به شرح زیر مرتب شده است: ابتدا مجموعه ای به یک رشته به عنوان در قالب تبسپار شده مرتب شده است و سپس رشته حاصل خروجی به سی سی اس وی در دو نقل قول است. دسته بندی ها در قالب سی اس وی به صورت ستون های جداگانه مرتب می شوند (به این معنا که لانه خود را در تاپل از دست داده است). +هنگام قالب بندی, ردیف در دو نقل قول محصور. نقل قول دو در داخل یک رشته خروجی به عنوان دو نقل قول دو در یک ردیف است. هیچ قانون دیگری برای فرار از شخصیت وجود دارد. تاریخ و تاریخ زمان در دو نقل قول محصور شده است. اعداد خروجی بدون نقل قول. ارزش ها توسط یک شخصیت جداساز از هم جدا, که `,` به طور پیش فرض. شخصیت جداساز در تنظیمات تعریف شده است [_مخفی کردن _قابلیت _جدید](../operations/settings/settings.md#settings-format_csv_delimiter). ردیف ها با استفاده از خوراک خط یونیکس جدا می شوند. ارریس در سی سی اس وی به شرح زیر مرتب شده است: ابتدا مجموعه ای به یک رشته به عنوان در قالب تبسپار شده مرتب شده است و سپس رشته حاصل خروجی به سی سی اس وی در دو نقل قول است. دسته بندی ها در قالب سی اس وی به صورت ستون های جداگانه مرتب می شوند (به این معنا که لانه خود را در تاپل از دست داده است). ``` bash $ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV" < data.csv ``` -\* به طور پیش فرض, جداساز است `,`. دیدن [\_مخفی کردن \_قابلیت \_جدید](../operations/settings/settings.md#settings-format_csv_delimiter) تنظیم برای اطلاعات بیشتر. +\* به طور پیش فرض, جداساز است `,`. دیدن [_مخفی کردن _قابلیت _جدید](../operations/settings/settings.md#settings-format_csv_delimiter) تنظیم برای اطلاعات بیشتر. هنگامی که تجزیه, تمام مقادیر را می توان یا با یا بدون نقل قول تجزیه. هر دو نقل قول دو و تک پشتیبانی می شوند. ردیف همچنین می توانید بدون نقل قول مرتب شود. در این مورد, به شخصیت جداساز و یا خوراک خط تجزیه (کروم و یا ال اف). در هنگام تجزیه ردیف بدون نقل قول فضاهای پیشرو و انتهایی و زبانه ها نادیده گرفته می شوند. برای اشتراک خط, یونیکس (کلیک کنید), پنجره ها (کروم ال اف) و سیستم عامل مک کلاسیک (کروم ال اف) انواع پشتیبانی می شوند. مقادیر ورودی بدون علامت خالی با مقادیر پیش فرض برای ستون های مربوطه جایگزین می شوند -[\_پوشه های ورودی و خروجی](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) +[_پوشه های ورودی و خروجی](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) فعال است. -`NULL` به عنوان فرمت `\N` یا `NULL` یا یک رشته بدون علامت خالی (تنظیمات را ببینید [\_فرستادن به \_کوچکنمایی](../operations/settings/settings.md#settings-input_format_csv_unquoted_null_literal_as_null) و [\_پوشه های ورودی و خروجی](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields)). +`NULL` به عنوان فرمت `\N` یا `NULL` یا یک رشته بدون علامت خالی (تنظیمات را ببینید [_فرستادن به _کوچکنمایی](../operations/settings/settings.md#settings-input_format_csv_unquoted_null_literal_as_null) و [_پوشه های ورودی و خروجی](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields)). پشتیبانی از خروجی بالغ و افراط به همان شیوه به عنوان `TabSeparated`. @@ -452,12 +452,12 @@ SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase WITH TOTA } ``` -جانسون سازگار با جاوا اسکریپت است. برای اطمینان از این, برخی از شخصیت ها علاوه بر فرار: بریده بریده `/` فرار به عنوان `\/`; معافیت خط جایگزین `U+2028` و `U+2029`, که شکستن برخی از مرورگرهای, به عنوان فرار `\uXXXX`. شخصیت های کنترل اسکی فرار: برگشت به عقب, خوراک فرم, خوراک خط, بازگشت حمل, و تب افقی با جایگزین `\b`, `\f`, `\n`, `\r`, `\t` , و همچنین بایت باقی مانده در محدوده 00-1ف با استفاده از `\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 [خروجی \_فرمان\_جسون\_کوات\_64بیت\_تنظیمی](../operations/settings/settings.md#session_settings-output_format_json_quote_64bit_integers) به 0. +جانسون سازگار با جاوا اسکریپت است. برای اطمینان از این, برخی از شخصیت ها علاوه بر فرار: بریده بریده `/` فرار به عنوان `\/`; معافیت خط جایگزین `U+2028` و `U+2029`, که شکستن برخی از مرورگرهای, به عنوان فرار `\uXXXX`. شخصیت های کنترل اسکی فرار: برگشت به عقب, خوراک فرم, خوراک خط, بازگشت حمل, و تب افقی با جایگزین `\b`, `\f`, `\n`, `\r`, `\t` , و همچنین بایت باقی مانده در محدوده 00-1ف با استفاده از `\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 [خروجی _فرمان_جسون_کوات_64بیت_تنظیمی](../operations/settings/settings.md#session_settings-output_format_json_quote_64bit_integers) به 0. `rows` – The total number of output rows. `rows_before_limit_at_least` حداقل تعداد ردیف وجود دارد که بدون محدودیت بوده است. خروجی تنها در صورتی که پرس و جو شامل حد. -اگر پرس و جو شامل گروه های, ردیف ها\_افور\_لیمیت\_تلاست تعداد دقیق ردیف وجود دارد که بدون محدودیت بوده است. +اگر پرس و جو شامل گروه های, ردیف ها_افور_لیمیت_تلاست تعداد دقیق ردیف وجود دارد که بدون محدودیت بوده است. `totals` – Total values (when using WITH TOTALS). @@ -544,7 +544,7 @@ INSERT INTO UserActivity FORMAT JSONEachRow {"PageViews":5, "UserID":"4324182021 را کلیک کنید جایگزین مقادیر حذف شده با مقادیر پیش فرض برای مربوطه [انواع داده ها](../sql-reference/data-types/index.md). -اگر `DEFAULT expr` مشخص شده است, تاتر با استفاده از قوانین تعویض مختلف بسته به [\_پوشه های ورودی و خروجی](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) تنظیمات. +اگر `DEFAULT expr` مشخص شده است, تاتر با استفاده از قوانین تعویض مختلف بسته به [_پوشه های ورودی و خروجی](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) تنظیمات. جدول زیر را در نظر بگیرید: @@ -587,7 +587,7 @@ CREATE TABLE IF NOT EXISTS example_table ### استفاده از ساختارهای تو در تو {#jsoneachrow-nested} -اگر شما یک جدول با [تو در تو](../sql-reference/data-types/nested-data-structures/nested.md) ستون نوع داده, شما می توانید داده های جانسون با همان ساختار وارد. فعال کردن این ویژگی با [تغییر \_کم\_تر\_تنظیم مجدد \_جنسان](../operations/settings/settings.md#settings-input_format_import_nested_json) تنظیمات. +اگر شما یک جدول با [تو در تو](../sql-reference/data-types/nested-data-structures/nested.md) ستون نوع داده, شما می توانید داده های جانسون با همان ساختار وارد. فعال کردن این ویژگی با [تغییر _کم_تر_تنظیم مجدد _جنسان](../operations/settings/settings.md#settings-input_format_import_nested_json) تنظیمات. برای مثال جدول زیر را در نظر بگیرید: @@ -601,7 +601,7 @@ CREATE TABLE json_each_row_nested (n Nested (s String, i Int32) ) ENGINE = Memor INSERT INTO json_each_row_nested FORMAT JSONEachRow {"n.s": ["abc", "def"], "n.i": [1, 23]} ``` -برای قرار دادن داده ها به عنوان یک شی جوسون سلسله مراتبی, تنظیم [وارد کردن \_ترمپ\_م\_تر\_اس\_جسون ثبت شده=1](../operations/settings/settings.md#settings-input_format_import_nested_json). +برای قرار دادن داده ها به عنوان یک شی جوسون سلسله مراتبی, تنظیم [وارد کردن _ترمپ_م_تر_اس_جسون ثبت شده=1](../operations/settings/settings.md#settings-input_format_import_nested_json). ``` json { @@ -1046,7 +1046,7 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Avro" > file.avro - شروع با `[A-Za-z_]` - متعاقبا تنها حاوی `[A-Za-z0-9_]` -خروجی فشرده سازی فایل و فاصله همگام سازی را می توان با پیکربندی [\_فرماندگی لبه بام](../operations/settings/settings.md#settings-output_format_avro_codec) و [\_فرماندگی لبه چشم](../operations/settings/settings.md#settings-output_format_avro_sync_interval) به ترتیب. +خروجی فشرده سازی فایل و فاصله همگام سازی را می توان با پیکربندی [_فرماندگی لبه بام](../operations/settings/settings.md#settings-output_format_avro_codec) و [_فرماندگی لبه چشم](../operations/settings/settings.md#settings-output_format_avro_sync_interval) به ترتیب. ## هشدار داده می شود {#data-format-avro-confluent} @@ -1056,7 +1056,7 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Avro" > file.avro طرحواره ذخیره سازی یک بار حل شود. -نشانی وب رجیستری طرحواره با پیکربندی [باز کردن \_نمایش مجدد](../operations/settings/settings.md#settings-format_avro_schema_registry_url) +نشانی وب رجیستری طرحواره با پیکربندی [باز کردن _نمایش مجدد](../operations/settings/settings.md#settings-format_avro_schema_registry_url) ### تطبیق انواع داده ها {#data_types-matching-1} @@ -1199,12 +1199,12 @@ e.g. `schemafile.proto:MessageType`. اگر شما با استفاده از مشتری در [حالت دسته ای](../interfaces/cli.md#cli_usage) مسیر طرح باید به دلایل امنیتی نسبی باشد. اگر داده های ورودی یا خروجی را از طریق [رابط قام](../interfaces/http.md) نام پرونده مشخص شده در شمای قالب -باید در دایرکتوری مشخص شده در واقع [قالب\_شکلمات شیمی](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-format_schema_path) +باید در دایرکتوری مشخص شده در واقع [قالب_شکلمات شیمی](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-format_schema_path) در پیکربندی سرور. ## پرش خطاها {#skippingerrors} -برخی از فرمت های مانند `CSV`, `TabSeparated`, `TSKV`, `JSONEachRow`, `Template`, `CustomSeparated` و `Protobuf` می توانید ردیف شکسته جست و خیز اگر خطای تجزیه رخ داده است و ادامه تجزیه از ابتدای ردیف بعدی. ببینید [وارد کردن \_فرست\_مرزیابی \_نمایش مجدد](../operations/settings/settings.md#settings-input_format_allow_errors_num) و +برخی از فرمت های مانند `CSV`, `TabSeparated`, `TSKV`, `JSONEachRow`, `Template`, `CustomSeparated` و `Protobuf` می توانید ردیف شکسته جست و خیز اگر خطای تجزیه رخ داده است و ادامه تجزیه از ابتدای ردیف بعدی. ببینید [وارد کردن _فرست_مرزیابی _نمایش مجدد](../operations/settings/settings.md#settings-input_format_allow_errors_num) و [ثبت نام](../operations/settings/settings.md#settings-input_format_allow_errors_ratio) تنظیمات. محدودیت ها: - در صورت خطای تجزیه `JSONEachRow` پرش تمام داده ها تا زمانی که خط جدید (یا بخش ویژه), بنابراین ردیف باید توسط حد و مرز مشخصی `\n` برای شمارش خطاها به درستی. diff --git a/docs/fa/interfaces/http.md b/docs/fa/interfaces/http.md index 9ce40c17e6f..16d7aa390dc 100644 --- a/docs/fa/interfaces/http.md +++ b/docs/fa/interfaces/http.md @@ -152,7 +152,7 @@ $ echo 'DROP TABLE t' | curl 'http://localhost:8123/' --data-binary @- اگر شما مشخص `compress=1` در نشانی وب سرور دادههای ارسالی شما را فشرده میکند. اگر شما مشخص `decompress=1` در نشانی اینترنتی کارگزار دادههای مشابهی را که در `POST` روش. -شما همچنین می توانید استفاده کنید را انتخاب کنید [فشردهسازی قام](https://en.wikipedia.org/wiki/HTTP_compression). برای ارسال یک فشرده `POST` درخواست, اضافه هدر درخواست `Content-Encoding: compression_method`. به منظور کلیک برای فشرده سازی پاسخ, شما باید اضافه `Accept-Encoding: compression_method`. پشتیبانی از کلیک `gzip`, `br` و `deflate` [روش های فشرده سازی](https://en.wikipedia.org/wiki/HTTP_compression#Content-Encoding_tokens). برای فعال کردن فشرده سازی قام, شما باید از خانه کلیک استفاده [نصب و راه اندازی](../operations/settings/settings.md#settings-enable_http_compression) تنظیمات. شما می توانید سطح فشرده سازی داده ها در پیکربندی [\_تنظیم مجدد به حالت اولیه](#settings-http_zlib_compression_level) تنظیم برای تمام روش های فشرده سازی. +شما همچنین می توانید استفاده کنید را انتخاب کنید [فشردهسازی قام](https://en.wikipedia.org/wiki/HTTP_compression). برای ارسال یک فشرده `POST` درخواست, اضافه هدر درخواست `Content-Encoding: compression_method`. به منظور کلیک برای فشرده سازی پاسخ, شما باید اضافه `Accept-Encoding: compression_method`. پشتیبانی از کلیک `gzip`, `br` و `deflate` [روش های فشرده سازی](https://en.wikipedia.org/wiki/HTTP_compression#Content-Encoding_tokens). برای فعال کردن فشرده سازی قام, شما باید از خانه کلیک استفاده [نصب و راه اندازی](../operations/settings/settings.md#settings-enable_http_compression) تنظیمات. شما می توانید سطح فشرده سازی داده ها در پیکربندی [_تنظیم مجدد به حالت اولیه](#settings-http_zlib_compression_level) تنظیم برای تمام روش های فشرده سازی. شما می توانید این برای کاهش ترافیک شبکه در هنگام انتقال مقدار زیادی از داده ها و یا برای ایجاد افسردگی است که بلافاصله فشرده استفاده کنید. @@ -214,7 +214,7 @@ $ echo 'SELECT 1' | curl -H 'X-ClickHouse-User: user' -H 'X-ClickHouse-Key: pass ``` اگر نام کاربر مشخص نشده است `default` نام استفاده شده است. اگر رمز عبور مشخص نشده است, رمز عبور خالی استفاده شده است. -شما همچنین می توانید از پارامترهای نشانی وب برای مشخص کردن هر گونه تنظیمات برای پردازش یک پرس و جو یا کل پروفایل های تنظیمات استفاده کنید. هشدار داده می شودمشخصات=وب و حداکثر\_نظیم = 1000000000 & پرس و جو = انتخاب+1 +شما همچنین می توانید از پارامترهای نشانی وب برای مشخص کردن هر گونه تنظیمات برای پردازش یک پرس و جو یا کل پروفایل های تنظیمات استفاده کنید. هشدار داده می شودمشخصات=وب و حداکثر_نظیم = 1000000000 & پرس و جو = انتخاب+1 برای کسب اطلاعات بیشتر, دیدن [تنظیمات](../operations/settings/index.md) بخش. @@ -253,9 +253,9 @@ X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_ro - `written_bytes` — Volume of data written in bytes. درخواست های در حال اجرا به طور خودکار متوقف نمی شود اگر اتصال قام از دست داده است. تجزیه و قالب بندی داده ها در سمت سرور انجام, و با استفاده از شبکه ممکن است بی اثر. -اختیاری ‘query\_id’ پارامتر را می توان به عنوان شناسه پرس و جو (هر رشته) منتقل می شود. برای کسب اطلاعات بیشتر به بخش مراجعه کنید “Settings, replace\_running\_query”. +اختیاری ‘query_id’ پارامتر را می توان به عنوان شناسه پرس و جو (هر رشته) منتقل می شود. برای کسب اطلاعات بیشتر به بخش مراجعه کنید “Settings, replace_running_query”. -اختیاری ‘quota\_key’ پارامتر را می توان به عنوان کلید سهمیه (هر رشته) منتقل می شود. برای کسب اطلاعات بیشتر به بخش مراجعه کنید “Quotas”. +اختیاری ‘quota_key’ پارامتر را می توان به عنوان کلید سهمیه (هر رشته) منتقل می شود. برای کسب اطلاعات بیشتر به بخش مراجعه کنید “Quotas”. رابط اچ تی پی اجازه می دهد تا عبور داده های خارجی (جداول موقت خارجی) برای پرس و جو. برای کسب اطلاعات بیشتر به بخش مراجعه کنید “External data for query processing”. @@ -379,9 +379,9 @@ $ curl -v 'http://localhost:8123/predefined_query' > `` شامل بخش پردازش اصلی. حالا `` می توانید پیکربندی کنید ``, ``, ``, ``, ``, ``. > \> `` در حال حاضر پشتیبانی از سه نوع: **باز تعریف**, **هشدار داده می شود**, **ایستا**. > \> -> \> `` - استفاده از با نوع بازتعریف\_کرکی\_ هندلر, اجرا پرس و جو زمانی که کنترل نامیده می شود. +> \> `` - استفاده از با نوع بازتعریف_کرکی_ هندلر, اجرا پرس و جو زمانی که کنترل نامیده می شود. > \> -> \> `` - استفاده با نوع داینامیک\_کرکی\_خندلر عصارهها و اجرا مقدار مربوط به `` ارزش در پارامترهای درخواست قام. +> \> `` - استفاده با نوع داینامیک_کرکی_خندلر عصارهها و اجرا مقدار مربوط به `` ارزش در پارامترهای درخواست قام. > \> > \> `` - استفاده با نوع استاتیک, پاسخ کد وضعیت. > \> @@ -393,7 +393,7 @@ $ curl -v 'http://localhost:8123/predefined_query' ## باز تعریف {#predefined_query_handler} -`` پشتیبانی از تنظیمات و مقادیر قوری\_پرم. شما می توانید پیکربندی کنید `` در نوع ``. +`` پشتیبانی از تنظیمات و مقادیر قوری_پرم. شما می توانید پیکربندی کنید `` در نوع ``. `` مقدار پرس و جو از پیش تعریف شده است ``, است که توسط کلیکهاوس اجرا زمانی که یک درخواست قام همسان است و در نتیجه از پرس و جو بازگشته است. این پیکربندی باید است. @@ -434,7 +434,7 @@ max_alter_threads 2 عصاره کلیک و اجرا ارزش مربوط به `` مقدار در نشانی وب درخواست قام. مقدار پیش فرض `` هست `/query` . این پیکربندی اختیاری است. در صورتی که هیچ تعریف در فایل پیکربندی وجود دارد, پرم در تصویب نشده است. -برای آزمایش این قابلیت به عنوان مثال تعریف ارزش از max\_threads و max\_alter\_threads و پرس و جو که آیا تنظیمات راه اندازی شد با موفقیت. +برای آزمایش این قابلیت به عنوان مثال تعریف ارزش از max_threads و max_alter_threads و پرس و جو که آیا تنظیمات راه اندازی شد با موفقیت. مثال: @@ -459,7 +459,7 @@ max_alter_threads 2 ## ایستا {#static} -`` می توانید بازگشت [\_نوع تماس](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type), [وضعیت](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status) و پاسخ دهنده. پاسخ \_حرکتکننده می تواند محتوای مشخص شده را بازگرداند +`` می توانید بازگشت [_نوع تماس](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type), [وضعیت](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status) و پاسخ دهنده. پاسخ _حرکتکننده می تواند محتوای مشخص شده را بازگرداند مثال: diff --git a/docs/fa/interfaces/mysql.md b/docs/fa/interfaces/mysql.md index 4634fb2a2b5..cb0e478d1a4 100644 --- a/docs/fa/interfaces/mysql.md +++ b/docs/fa/interfaces/mysql.md @@ -7,7 +7,7 @@ toc_title: "\u0631\u0627\u0628\u0637 MySQL" # رابط MySQL {#mysql-interface} -کلیک پروتکل سیم خروجی زیر را پشتیبانی می کند. این را می توان با فعال [\_وارد کردن](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-mysql_port) تنظیم در پرونده پیکربندی: +کلیک پروتکل سیم خروجی زیر را پشتیبانی می کند. این را می توان با فعال [_وارد کردن](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-mysql_port) تنظیم در پرونده پیکربندی: ``` xml 9004 diff --git a/docs/fa/interfaces/third-party/client-libraries.md b/docs/fa/interfaces/third-party/client-libraries.md index 4d35089e1d3..ad2d3e86aae 100644 --- a/docs/fa/interfaces/third-party/client-libraries.md +++ b/docs/fa/interfaces/third-party/client-libraries.md @@ -12,7 +12,7 @@ toc_title: "\u06A9\u062A\u0627\u0628\u062E\u0627\u0646\u0647 \u0647\u0627\u06CC یاندکس می کند **نه** حفظ کتابخانه های ذکر شده در زیر و هر تست گسترده برای اطمینان از کیفیت خود را انجام نداده اند. - پایتون - - [اطالعات.کلیک \_شورم](https://github.com/Infinidat/infi.clickhouse_orm) + - [اطالعات.کلیک _شورم](https://github.com/Infinidat/infi.clickhouse_orm) - [کلیک راننده](https://github.com/mymarilyn/clickhouse-driver) - [کلیک مشتری](https://github.com/yurial/clickhouse-client) - [اطلاعات دقیق](https://github.com/maximdanilchenko/aiochclient) @@ -48,7 +48,7 @@ toc_title: "\u06A9\u062A\u0627\u0628\u062E\u0627\u0646\u0647 \u0647\u0627\u06CC - [تاتر-اسکالا-کارفرما](https://github.com/crobox/clickhouse-scala-client) - کوتلین - [AORM](https://github.com/TanVD/AORM) -- C\# +- C# - [Octonica.ClickHouseClient](https://github.com/Octonica/ClickHouseClient) - [فاحشه خانه.ادو](https://github.com/killwort/ClickHouse-Net) - [فاحشه خانه.کارگیر](https://github.com/DarkWanderer/ClickHouse.Client) diff --git a/docs/fa/interfaces/third-party/integrations.md b/docs/fa/interfaces/third-party/integrations.md index d198909095d..80cf94c53e0 100644 --- a/docs/fa/interfaces/third-party/integrations.md +++ b/docs/fa/interfaces/third-party/integrations.md @@ -19,10 +19,10 @@ toc_title: "\u06CC\u06A9\u067E\u0627\u0631\u0686\u06AF\u06CC" - [تاتر-خروجی زیر-داده خوان](https://github.com/Altinity/clickhouse-mysql-data-reader) - [horgh-replicator](https://github.com/larsnovikov/horgh-replicator) - [PostgreSQL](https://www.postgresql.org) - - [\_پاک کردن تصویر](https://github.com/Percona-Lab/clickhousedb_fdw) - - [اطالعات.\_پاک کردن](https://github.com/Infinidat/infi.clickhouse_fdw) (استفاده [اطالعات.کلیک \_شورم](https://github.com/Infinidat/infi.clickhouse_orm)) + - [_پاک کردن تصویر](https://github.com/Percona-Lab/clickhousedb_fdw) + - [اطالعات._پاک کردن](https://github.com/Infinidat/infi.clickhouse_fdw) (استفاده [اطالعات.کلیک _شورم](https://github.com/Infinidat/infi.clickhouse_orm)) - [پی جی 2چ](https://github.com/mkabilov/pg2ch) - - [\_پاک کردن](https://github.com/adjust/clickhouse_fdw) + - [_پاک کردن](https://github.com/adjust/clickhouse_fdw) - [MSSQL](https://en.wikipedia.org/wiki/Microsoft_SQL_Server) - [کلیک کنیدهاجر](https://github.com/zlzforever/ClickHouseMigrator) - صف پیام @@ -51,12 +51,12 @@ toc_title: "\u06CC\u06A9\u067E\u0627\u0631\u0686\u06AF\u06CC" - [گرافانا](https://grafana.com/) - [فاحشه خانه-گرافانا](https://github.com/Vertamedia/clickhouse-grafana) - [پرومتیوس](https://prometheus.io/) - - [کلیک \_گزارشسپر](https://github.com/f1yegor/clickhouse_exporter) + - [کلیک _گزارشسپر](https://github.com/f1yegor/clickhouse_exporter) - [مجلس رقص رسمی دبیرستان](https://github.com/Percona-Lab/PromHouse) - - [کلیک \_گزارشسپر](https://github.com/hot-wifi/clickhouse_exporter) (استفاده [برو کارگیر](https://github.com/kshvakov/clickhouse/)) + - [کلیک _گزارشسپر](https://github.com/hot-wifi/clickhouse_exporter) (استفاده [برو کارگیر](https://github.com/kshvakov/clickhouse/)) - [Nagios](https://www.nagios.org/) - - [\_تخچه نشانزنی](https://github.com/exogroup/check_clickhouse/) - - [check\_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py) + - [_تخچه نشانزنی](https://github.com/exogroup/check_clickhouse/) + - [check_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py) - [زاببیکس](https://www.zabbix.com) - [هوکاوس-زاببیکس-قالب](https://github.com/Altinity/clickhouse-zabbix-template) - [کلیپ برد چند منظوره](https://sematext.com/) @@ -76,7 +76,7 @@ toc_title: "\u06CC\u06A9\u067E\u0627\u0631\u0686\u06AF\u06CC" - پایتون - [SQLAlchemy](https://www.sqlalchemy.org) - - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (استفاده [اطالعات.کلیک \_شورم](https://github.com/Infinidat/infi.clickhouse_orm)) + - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (استفاده [اطالعات.کلیک _شورم](https://github.com/Infinidat/infi.clickhouse_orm)) - [پانداها](https://pandas.pydata.org) - [پانداهاوس](https://github.com/kszucs/pandahouse) - PHP @@ -91,7 +91,7 @@ toc_title: "\u06CC\u06A9\u067E\u0627\u0631\u0686\u06AF\u06CC" - اسکالا - [اککا](https://akka.io) - [تاتر-اسکالا-کارفرما](https://github.com/crobox/clickhouse-scala-client) -- C\# +- C# - [ADO.NET](https://docs.microsoft.com/en-us/dotnet/framework/data/adonet/ado-net-overview) - [فاحشه خانه.ادو](https://github.com/killwort/ClickHouse-Net) - [فاحشه خانه.کارگیر](https://github.com/DarkWanderer/ClickHouse.Client) diff --git a/docs/fa/operations/access-rights.md b/docs/fa/operations/access-rights.md index 7e2c7c3027f..6746a661424 100644 --- a/docs/fa/operations/access-rights.md +++ b/docs/fa/operations/access-rights.md @@ -135,7 +135,7 @@ toc_title: "\u06A9\u0646\u062A\u0631\u0644 \u062F\u0633\u062A\u0631\u0633\u06CC - راه اندازی یک دایرکتوری برای ذخیره سازی تنظیمات. - فروشگاه های کلیک دسترسی به تنظیمات نهاد در مجموعه پوشه در [\_پوشه دستیابی](server-configuration-parameters/settings.md#access_control_path) پارامتر پیکربندی سرور. + فروشگاه های کلیک دسترسی به تنظیمات نهاد در مجموعه پوشه در [_پوشه دستیابی](server-configuration-parameters/settings.md#access_control_path) پارامتر پیکربندی سرور. - فعال کردن گذاشتن محور کنترل دسترسی و مدیریت حساب برای حداقل یک حساب کاربری. diff --git a/docs/fa/operations/configuration-files.md b/docs/fa/operations/configuration-files.md index 3e3542e454e..c1416100c6d 100644 --- a/docs/fa/operations/configuration-files.md +++ b/docs/fa/operations/configuration-files.md @@ -21,7 +21,7 @@ toc_title: "\u067E\u0631\u0648\u0646\u062F\u0647\u0647\u0627\u06CC \u067E\u06CC\ اگر `remove` مشخص شده است, حذف عنصر. -پیکربندی همچنین می توانید تعریف “substitutions”. اگر یک عنصر است `incl` ویژگی, جایگزینی مربوطه را از فایل خواهد شد به عنوان ارزش استفاده. به طور پیش فرض, مسیر به فایل با تعویض است `/etc/metrika.xml`. این را می توان در تغییر [شامل \_فروم](server-configuration-parameters/settings.md#server_configuration_parameters-include_from) عنصر در پیکربندی سرور. مقادیر جایگزینی در مشخص `/yandex/substitution_name` عناصر در این فایل. اگر جایگزینی مشخص شده در `incl` وجود ندارد, این است که در ورود به سیستم ثبت. برای جلوگیری از جایگزینی ورود به سیستم کلیک کنید `optional="true"` ویژگی (مثلا, تنظیمات برای [& کلاندارها](server-configuration-parameters/settings.md)). +پیکربندی همچنین می توانید تعریف “substitutions”. اگر یک عنصر است `incl` ویژگی, جایگزینی مربوطه را از فایل خواهد شد به عنوان ارزش استفاده. به طور پیش فرض, مسیر به فایل با تعویض است `/etc/metrika.xml`. این را می توان در تغییر [شامل _فروم](server-configuration-parameters/settings.md#server_configuration_parameters-include_from) عنصر در پیکربندی سرور. مقادیر جایگزینی در مشخص `/yandex/substitution_name` عناصر در این فایل. اگر جایگزینی مشخص شده در `incl` وجود ندارد, این است که در ورود به سیستم ثبت. برای جلوگیری از جایگزینی ورود به سیستم کلیک کنید `optional="true"` ویژگی (مثلا, تنظیمات برای [& کلاندارها](server-configuration-parameters/settings.md)). تعویض همچنین می توانید از باغ وحش انجام شود. برای انجام این کار ویژگی را مشخص کنید `from_zk = "/path/to/node"`. مقدار عنصر با محتویات گره در جایگزین `/path/to/node` در باغ وحش. شما همچنین می توانید یک زیر درخت کل در گره باغ وحش قرار داده و به طور کامل به عنصر منبع وارد می شود. diff --git a/docs/fa/operations/monitoring.md b/docs/fa/operations/monitoring.md index 91b5d4613da..48baa568952 100644 --- a/docs/fa/operations/monitoring.md +++ b/docs/fa/operations/monitoring.md @@ -35,7 +35,7 @@ toc_title: "\u0646\u0638\u0627\u0631\u062A" - معیارهای مختلف چگونه سرور با استفاده از منابع محاسباتی. - ارقام مشترک در پردازش پرس و جو. -شما می توانید معیارهای موجود در [سیستم.متریک](../operations/system-tables.md#system_tables-metrics), [سیستم.رویدادها](../operations/system-tables.md#system_tables-events) و [سیستم.\_نامهنویسی ناهمزمان](../operations/system-tables.md#system_tables-asynchronous_metrics) میز +شما می توانید معیارهای موجود در [سیستم.متریک](../operations/system-tables.md#system_tables-metrics), [سیستم.رویدادها](../operations/system-tables.md#system_tables-events) و [سیستم._نامهنویسی ناهمزمان](../operations/system-tables.md#system_tables-asynchronous_metrics) میز شما می توانید کلیک کنید هاوس به صادرات معیارهای به پیکربندی کنید [گرافیت](https://github.com/graphite-project). دیدن [بخش گرافیت](server-configuration-parameters/settings.md#server_configuration_parameters-graphite) در فایل پیکربندی سرور کلیک. قبل از پیکربندی صادرات معیارهای, شما باید راه اندازی گرافیت با پیروی از رسمی خود را [راهنما](https://graphite.readthedocs.io/en/latest/install.html). @@ -43,4 +43,4 @@ toc_title: "\u0646\u0638\u0627\u0631\u062A" علاوه بر این, شما می توانید در دسترس بودن سرور از طریق صفحه اصلی نظارت. ارسال `HTTP GET` درخواست برای `/ping`. اگر سرور در دسترس است, با پاسخ `200 OK`. -برای نظارت بر سرور در یک پیکربندی خوشه, شما باید مجموعه ای از [\_شروع مجدد \_شروع مجدد \_شروع مجدد \_کاربری](settings/settings.md#settings-max_replica_delay_for_distributed_queries) پارامتر و استفاده از منبع قام `/replicas_status`. یک درخواست برای `/replicas_status` بازگشت `200 OK` اگر ماکت در دسترس است و در پشت کپی دیگر به تعویق افتاد. اگر یک ماکت به تاخیر افتاد, باز می گردد `503 HTTP_SERVICE_UNAVAILABLE` با اطلاعات در مورد شکاف. +برای نظارت بر سرور در یک پیکربندی خوشه, شما باید مجموعه ای از [_شروع مجدد _شروع مجدد _شروع مجدد _کاربری](settings/settings.md#settings-max_replica_delay_for_distributed_queries) پارامتر و استفاده از منبع قام `/replicas_status`. یک درخواست برای `/replicas_status` بازگشت `200 OK` اگر ماکت در دسترس است و در پشت کپی دیگر به تعویق افتاد. اگر یک ماکت به تاخیر افتاد, باز می گردد `503 HTTP_SERVICE_UNAVAILABLE` با اطلاعات در مورد شکاف. diff --git a/docs/fa/operations/optimizing-performance/sampling-query-profiler.md b/docs/fa/operations/optimizing-performance/sampling-query-profiler.md index a97db8c16b8..3573c5e0806 100644 --- a/docs/fa/operations/optimizing-performance/sampling-query-profiler.md +++ b/docs/fa/operations/optimizing-performance/sampling-query-profiler.md @@ -12,9 +12,9 @@ toc_title: "\u067E\u0631\u0648\u0641\u0627\u06CC\u0644 \u067E\u0631\u0633 \u0648 برای استفاده از پروفیل: -- برپایی [\_قطع](../server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) بخش پیکربندی سرور. +- برپایی [_قطع](../server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) بخش پیکربندی سرور. - در این بخش پیکربندی [\_قطع](../../operations/system-tables.md#system_tables-trace_log) جدول سیستم حاوی نتایج حاصل از عملکرد پیشفیلتر. این است که به طور پیش فرض پیکربندی شده است. به یاد داشته باشید که داده ها در این جدول تنها برای یک سرور در حال اجرا معتبر است. پس از راه اندازی مجدد سرور, تاتر تمیز نمی کند تا جدول و تمام نشانی حافظه مجازی ذخیره شده ممکن است نامعتبر. + در این بخش پیکربندی [_قطع](../../operations/system-tables.md#system_tables-trace_log) جدول سیستم حاوی نتایج حاصل از عملکرد پیشفیلتر. این است که به طور پیش فرض پیکربندی شده است. به یاد داشته باشید که داده ها در این جدول تنها برای یک سرور در حال اجرا معتبر است. پس از راه اندازی مجدد سرور, تاتر تمیز نمی کند تا جدول و تمام نشانی حافظه مجازی ذخیره شده ممکن است نامعتبر. - برپایی [ایران در تهران](../settings/settings.md#query_profiler_cpu_time_period_ns) یا [جستجو](../settings/settings.md#query_profiler_real_time_period_ns) تنظیمات. هر دو تنظیمات را می توان به طور همزمان استفاده کرد. @@ -26,7 +26,7 @@ toc_title: "\u067E\u0631\u0648\u0641\u0627\u06CC\u0644 \u067E\u0631\u0633 \u0648 - نصب `clickhouse-common-static-dbg` بسته ببینید [نصب از بسته های دب](../../getting-started/install.md#install-from-deb-packages). -- اجازه توابع درون گرایی توسط [اجازه دادن به \_فعال کردن اختلال در عملکرد](../settings/settings.md#settings-allow_introspection_functions) تنظیمات. +- اجازه توابع درون گرایی توسط [اجازه دادن به _فعال کردن اختلال در عملکرد](../settings/settings.md#settings-allow_introspection_functions) تنظیمات. به دلایل امنیتی, توابع درون گرایی به طور پیش فرض غیر فعال. diff --git a/docs/fa/operations/requirements.md b/docs/fa/operations/requirements.md index 9f345905497..f116f59208b 100644 --- a/docs/fa/operations/requirements.md +++ b/docs/fa/operations/requirements.md @@ -9,7 +9,7 @@ toc_title: "\u0627\u0644\u0632\u0627\u0645\u0627\u062A" ## CPU {#cpu} -برای نصب و راه اندازی از بسته های پیش ساخته دب, استفاده از یک پردازنده با معماری ایکس86\_64 و پشتیبانی برای سوس 4.2 دستورالعمل. برای اجرای ClickHouse با پردازنده های که پشتیبانی نمی کند SSE 4.2 یا AArch64 یا PowerPC64LE معماری شما باید ساخت ClickHouse از منابع. +برای نصب و راه اندازی از بسته های پیش ساخته دب, استفاده از یک پردازنده با معماری ایکس86_64 و پشتیبانی برای سوس 4.2 دستورالعمل. برای اجرای ClickHouse با پردازنده های که پشتیبانی نمی کند SSE 4.2 یا AArch64 یا PowerPC64LE معماری شما باید ساخت ClickHouse از منابع. تاتر پیاده سازی پردازش داده های موازی و با استفاده از تمام منابع سخت افزاری در دسترس. در هنگام انتخاب یک پردازنده, را به حساب که فاحشه خانه کار می کند موثر تر در تنظیمات با تعداد زیادی از هسته اما نرخ ساعت پایین تر از در تنظیمات با هسته کمتر و نرخ ساعت بالاتر. مثلا, 16 هسته با 2600 مگاهرتز بهتر از است 8 هسته با 3600 مگاهرتز. diff --git a/docs/fa/operations/server-configuration-parameters/settings.md b/docs/fa/operations/server-configuration-parameters/settings.md index 1459c20dd5c..b57a9d1bfa8 100644 --- a/docs/fa/operations/server-configuration-parameters/settings.md +++ b/docs/fa/operations/server-configuration-parameters/settings.md @@ -110,7 +110,7 @@ toc_title: "\u062A\u0646\u0638\u06CC\u0645\u0627\u062A \u06A9\u0627\u0631\u06AF\ *_dictionary.xml ``` -## \_بارگیری کامل {#server_configuration_parameters-dictionaries_lazy_load} +## _بارگیری کامل {#server_configuration_parameters-dictionaries_lazy_load} بارگذاری تنبل از لغت نامه. @@ -126,7 +126,7 @@ toc_title: "\u062A\u0646\u0638\u06CC\u0645\u0627\u062A \u06A9\u0627\u0631\u06AF\ true ``` -## قالب\_شکلمات شیمی {#server_configuration_parameters-format_schema_path} +## قالب_شکلمات شیمی {#server_configuration_parameters-format_schema_path} مسیر به دایرکتوری با طرح برای داده های ورودی, مانند طرحواره برای [کاپپروتو](../../interfaces/formats.md#capnproto) قالب. @@ -147,11 +147,11 @@ toc_title: "\u062A\u0646\u0638\u06CC\u0645\u0627\u062A \u06A9\u0627\u0631\u06AF\ - port – The port on the Graphite server. - interval – The interval for sending, in seconds. - timeout – The timeout for sending data, in seconds. -- root\_path – Prefix for keys. +- root_path – Prefix for keys. - metrics – Sending data from the [سیستم.متریک](../../operations/system-tables.md#system_tables-metrics) جدول - events – Sending deltas data accumulated for the time period from the [سیستم.رویدادها](../../operations/system-tables.md#system_tables-events) جدول -- events\_cumulative – Sending cumulative data from the [سیستم.رویدادها](../../operations/system-tables.md#system_tables-events) جدول -- asynchronous\_metrics – Sending data from the [سیستم.\_نامهنویسی ناهمزمان](../../operations/system-tables.md#system_tables-asynchronous_metrics) جدول +- events_cumulative – Sending cumulative data from the [سیستم.رویدادها](../../operations/system-tables.md#system_tables-events) جدول +- asynchronous_metrics – Sending data from the [سیستم._نامهنویسی ناهمزمان](../../operations/system-tables.md#system_tables-asynchronous_metrics) جدول شما می توانید چند پیکربندی کنید `` بند. برای مثال شما می توانید از این برای ارسال داده های مختلف در فواصل مختلف استفاده کنید. @@ -171,7 +171,7 @@ toc_title: "\u062A\u0646\u0638\u06CC\u0645\u0627\u062A \u06A9\u0627\u0631\u06AF\ ``` -## لغزش \_ نمودار {#server_configuration_parameters-graphite-rollup} +## لغزش _ نمودار {#server_configuration_parameters-graphite-rollup} تنظیمات برای نازک شدن داده ها برای گرافیت. @@ -199,7 +199,7 @@ toc_title: "\u062A\u0646\u0638\u06CC\u0645\u0627\u062A \u06A9\u0627\u0631\u06AF\ ``` -## \_وارد کردن/پشتیبانی {#http-porthttps-port} +## _وارد کردن/پشتیبانی {#http-porthttps-port} درگاه برای اتصال به کارساز بالای صفحه) ها (. @@ -228,7 +228,7 @@ toc_title: "\u062A\u0646\u0638\u06CC\u0645\u0627\u062A \u06A9\u0627\u0631\u06AF\ ``` -## شامل \_فروم {#server_configuration_parameters-include_from} +## شامل _فروم {#server_configuration_parameters-include_from} مسیر به فایل با تعویض. @@ -240,7 +240,7 @@ toc_title: "\u062A\u0646\u0638\u06CC\u0645\u0627\u062A \u06A9\u0627\u0631\u06AF\ /etc/metrica.xml ``` -## \_صادر کردن {#interserver-http-port} +## _صادر کردن {#interserver-http-port} پورت برای تبادل اطلاعات بین سرور های فاحشه خانه. @@ -293,7 +293,7 @@ toc_title: "\u062A\u0646\u0638\u06CC\u0645\u0627\u062A \u06A9\u0627\u0631\u06AF\ 3 ``` -## \_نوست فهرست {#server_configuration_parameters-listen_host} +## _نوست فهرست {#server_configuration_parameters-listen_host} محدودیت در میزبان که درخواست می توانید از. اگر می خواهید سرور برای پاسخ به همه انها مشخص شود `::`. @@ -344,10 +344,10 @@ toc_title: "\u062A\u0646\u0638\u06CC\u0645\u0627\u062A \u06A9\u0627\u0631\u06AF\ کلید: -- use\_syslog — Required setting if you want to write to the syslog. +- use_syslog — Required setting if you want to write to the syslog. - address — The host\[:port\] of syslogd. If omitted, the local daemon is used. - hostname — Optional. The name of the host that logs are sent from. -- facility — [کلمه کلیدی تسهیلات سیسلوگ](https://en.wikipedia.org/wiki/Syslog#Facility) در حروف بزرگ با “LOG\_” پیشوند: (`LOG_USER`, `LOG_DAEMON`, `LOG_LOCAL3`, و به همین ترتیب). +- facility — [کلمه کلیدی تسهیلات سیسلوگ](https://en.wikipedia.org/wiki/Syslog#Facility) در حروف بزرگ با “LOG_” پیشوند: (`LOG_USER`, `LOG_DAEMON`, `LOG_LOCAL3`, و به همین ترتیب). مقدار پیشفرض: `LOG_USER` اگر `address` مشخص شده است, `LOG_DAEMON otherwise.` - format – Message format. Possible values: `bsd` و `syslog.` @@ -365,7 +365,7 @@ toc_title: "\u062A\u0646\u0638\u06CC\u0645\u0627\u062A \u06A9\u0627\u0631\u06AF\ ``` -## نشاندار کردن \_چ\_سیز {#server-mark-cache-size} +## نشاندار کردن _چ_سیز {#server-mark-cache-size} اندازه تقریبی (به بایت) کش علامت های استفاده شده توسط موتورهای جدول [ادغام](../../engines/table-engines/mergetree-family/mergetree.md) خانواده @@ -377,7 +377,7 @@ toc_title: "\u062A\u0646\u0638\u06CC\u0645\u0627\u062A \u06A9\u0627\u0631\u06AF\ 5368709120 ``` -## م\_قیاس تصویر {#max-concurrent-queries} +## م_قیاس تصویر {#max-concurrent-queries} حداکثر تعداد درخواست به طور همزمان پردازش. @@ -387,7 +387,7 @@ toc_title: "\u062A\u0646\u0638\u06CC\u0645\u0627\u062A \u06A9\u0627\u0631\u06AF\ 100 ``` -## \_تنامههای بیشینه {#max-connections} +## _تنامههای بیشینه {#max-connections} حداکثر تعداد اتصالات ورودی. @@ -397,7 +397,7 @@ toc_title: "\u062A\u0646\u0638\u06CC\u0645\u0627\u062A \u06A9\u0627\u0631\u06AF\ 4096 ``` -## \_موضوعات بیشینه {#max-open-files} +## _موضوعات بیشینه {#max-open-files} حداکثر تعداد فایل های باز. @@ -411,7 +411,7 @@ toc_title: "\u062A\u0646\u0638\u06CC\u0645\u0627\u062A \u06A9\u0627\u0631\u06AF\ 262144 ``` -## حداکثر\_طب\_ضز\_توقف {#max-table-size-to-drop} +## حداکثر_طب_ضز_توقف {#max-table-size-to-drop} محدودیت در حذف جداول. @@ -464,7 +464,7 @@ SSL client/server configuration. - sessionTimeout – Time for caching the session on the server. - extendedVerification – Automatically extended verification of certificates after the session ends. Acceptable values: `true`, `false`. - requireTLSv1 – Require a TLSv1 connection. Acceptable values: `true`, `false`. -- requireTLSv1\_1 – Require a TLSv1.1 connection. Acceptable values: `true`, `false`. +- requireTLSv1_1 – Require a TLSv1.1 connection. Acceptable values: `true`, `false`. - requireTLSv1 – Require a TLSv1.2 connection. Acceptable values: `true`, `false`. - fips – Activates OpenSSL FIPS mode. Supported if the library's OpenSSL version supports FIPS. - privateKeyPassphraseHandler – Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: ``, `KeyFileHandler`, `test`, ``. @@ -502,11 +502,11 @@ SSL client/server configuration. ``` -## \_خروج {#server_configuration_parameters-part-log} +## _خروج {#server_configuration_parameters-part-log} وقایع ورود به سیستم که با مرتبط [ادغام](../../engines/table-engines/mergetree-family/mergetree.md). برای مثال, اضافه کردن یا ادغام داده ها. شما می توانید ورود به سیستم برای شبیه سازی الگوریتم های ادغام و مقایسه ویژگی های خود استفاده کنید. شما می توانید روند ادغام تجسم. -نمایش داده شد در سیستم وارد [سیستم.\_خروج](../../operations/system-tables.md#system_tables-part-log) جدول, نه در یک فایل جداگانه. شما می توانید نام این جدول را در پیکربندی `table` پارامتر (پایین را ببینید). +نمایش داده شد در سیستم وارد [سیستم._خروج](../../operations/system-tables.md#system_tables-part-log) جدول, نه در یک فایل جداگانه. شما می توانید نام این جدول را در پیکربندی `table` پارامتر (پایین را ببینید). از پارامترهای زیر برای پیکربندی ورود استفاده کنید: @@ -549,7 +549,7 @@ SSL client/server configuration. - `port` – Port for `endpoint`. - `metrics` – Flag that sets to expose metrics from the [سیستم.متریک](../system-tables.md#system_tables-metrics) جدول - `events` – Flag that sets to expose metrics from the [سیستم.رویدادها](../system-tables.md#system_tables-events) جدول -- `asynchronous_metrics` – Flag that sets to expose current metrics values from the [سیستم.\_نامهنویسی ناهمزمان](../system-tables.md#system_tables-asynchronous_metrics) جدول +- `asynchronous_metrics` – Flag that sets to expose current metrics values from the [سیستم._نامهنویسی ناهمزمان](../system-tables.md#system_tables-asynchronous_metrics) جدول **مثال** @@ -563,11 +563,11 @@ SSL client/server configuration. ``` -## \_خروج {#server_configuration_parameters-query-log} +## _خروج {#server_configuration_parameters-query-log} -تنظیم برای ورود به سیستم نمایش داده شد با دریافت [\_ترکیب = 1](../settings/settings.md) تنظیمات. +تنظیم برای ورود به سیستم نمایش داده شد با دریافت [_ترکیب = 1](../settings/settings.md) تنظیمات. -نمایش داده شد در سیستم وارد [سیستم.\_خروج](../../operations/system-tables.md#system_tables-query_log) جدول, نه در یک فایل جداگانه. شما می توانید نام جدول را در `table` پارامتر (پایین را ببینید). +نمایش داده شد در سیستم وارد [سیستم._خروج](../../operations/system-tables.md#system_tables-query_log) جدول, نه در یک فایل جداگانه. شما می توانید نام جدول را در `table` پارامتر (پایین را ببینید). از پارامترهای زیر برای پیکربندی ورود استفاده کنید: @@ -589,11 +589,11 @@ SSL client/server configuration. ``` -## \_ر\_خروج {#server_configuration_parameters-query-thread-log} +## _ر_خروج {#server_configuration_parameters-query-thread-log} تنظیم برای ورود به سیستم موضوعات نمایش داده شد دریافت شده با [& پایین: 1](../settings/settings.md#settings-log-query-threads) تنظیمات. -نمایش داده شد در سیستم وارد [سیستم.\_ر\_خروج](../../operations/system-tables.md#system_tables-query-thread-log) جدول, نه در یک فایل جداگانه. شما می توانید نام جدول را در `table` پارامتر (پایین را ببینید). +نمایش داده شد در سیستم وارد [سیستم._ر_خروج](../../operations/system-tables.md#system_tables-query-thread-log) جدول, نه در یک فایل جداگانه. شما می توانید نام جدول را در `table` پارامتر (پایین را ببینید). از پارامترهای زیر برای پیکربندی ورود استفاده کنید: @@ -615,9 +615,9 @@ SSL client/server configuration. ``` -## \_قطع {#server_configuration_parameters-trace_log} +## _قطع {#server_configuration_parameters-trace_log} -تنظیمات برای [\_قطع](../../operations/system-tables.md#system_tables-trace_log) عملیات جدول سیستم. +تنظیمات برای [_قطع](../../operations/system-tables.md#system_tables-trace_log) عملیات جدول سیستم. پارامترها: @@ -637,7 +637,7 @@ SSL client/server configuration. ``` -## \_منبع {#query-masking-rules} +## _منبع {#query-masking-rules} قوانین مبتنی بر عبارت منظم, خواهد شد که به نمایش داده شد و همچنین تمام پیام های ورود به سیستم قبل از ذخیره سازی در سیاهههای مربوط به سرور اعمال, `system.query_log`, `system.text_log`, `system.processes` جدول, و در سیاهههای مربوط به مشتری ارسال. که اجازه می دهد تا جلوگیری از @@ -698,7 +698,7 @@ SSL client/server configuration. Europe/Moscow ``` -## \_صادر کردن {#server_configuration_parameters-tcp_port} +## _صادر کردن {#server_configuration_parameters-tcp_port} پورت برای برقراری ارتباط با مشتریان بیش از پروتکل تی سی پی. @@ -708,7 +708,7 @@ SSL client/server configuration. 9000 ``` -## \_شروع مجدد {#server_configuration_parameters-tcp_port_secure} +## _شروع مجدد {#server_configuration_parameters-tcp_port_secure} پورت تی سی پی برای برقراری ارتباط امن با مشتریان. با استفاده از [OpenSSL](#server_configuration_parameters-openssl) تنظیمات. @@ -722,7 +722,7 @@ SSL client/server configuration. 9440 ``` -## \_وارد کردن {#server_configuration_parameters-mysql_port} +## _وارد کردن {#server_configuration_parameters-mysql_port} پورت برای برقراری ارتباط با مشتریان بیش از پروتکل خروجی زیر. @@ -736,7 +736,7 @@ SSL client/server configuration. 9004 ``` -## \_مخفی کردن {#server-settings-tmp_path} +## _مخفی کردن {#server-settings-tmp_path} مسیر به داده های موقت برای پردازش نمایش داده شد بزرگ است. @@ -760,7 +760,7 @@ SSL client/server configuration. - `max_data_part_size_bytes` نادیده گرفته شده است - شما باید دقیقا یک جلد در این سیاست داشته باشید -## \_بالا {#server-settings-uncompressed_cache_size} +## _بالا {#server-settings-uncompressed_cache_size} اندازه کش (به بایت) برای داده های غیر فشرده استفاده شده توسط موتورهای جدول از [ادغام](../../engines/table-engines/mergetree-family/mergetree.md). @@ -774,7 +774,7 @@ SSL client/server configuration. 8589934592 ``` -## \_مخفی کردن \_صفحه {#server_configuration_parameters-user_files_path} +## _مخفی کردن _صفحه {#server_configuration_parameters-user_files_path} دایرکتوری با فایل های کاربر. مورد استفاده در تابع جدول [پرونده()](../../sql-reference/table-functions/file.md). @@ -784,7 +784,7 @@ SSL client/server configuration. /var/lib/clickhouse/user_files/ ``` -## \_تنفورد {#users-config} +## _تنفورد {#users-config} مسیر پروندهی شامل: @@ -894,7 +894,7 @@ SSL client/server configuration. **مقدار پیشفرض**: 15. -## \_پوشه دستیابی {#access_control_path} +## _پوشه دستیابی {#access_control_path} مسیر را به یک پوشه که یک سرور کلیک ذخیره کاربر و نقش تنظیمات ایجاد شده توسط دستورات گذاشتن. diff --git a/docs/fa/operations/settings/permissions-for-queries.md b/docs/fa/operations/settings/permissions-for-queries.md index 9cad6fd5f5c..6b49c6ca334 100644 --- a/docs/fa/operations/settings/permissions-for-queries.md +++ b/docs/fa/operations/settings/permissions-for-queries.md @@ -19,7 +19,7 @@ toc_title: "\u0645\u062C\u0648\u0632 \u0628\u0631\u0627\u06CC \u0646\u0645\u0627 تنظیمات زیر تنظیم مجوز کاربر بر اساس نوع پرس و جو: - [فقط خواندنی](#settings_readonly) — Restricts permissions for all types of queries except DDL queries. -- [اجازه دادن به \_نشانی](#settings_allow_ddl) — Restricts permissions for DDL queries. +- [اجازه دادن به _نشانی](#settings_allow_ddl) — Restricts permissions for DDL queries. `KILL QUERY` را می توان با هر تنظیمات انجام می شود. @@ -44,7 +44,7 @@ toc_title: "\u0645\u062C\u0648\u0632 \u0628\u0631\u0627\u06CC \u0646\u0645\u0627 مقدار پیشفرض: 0 -## اجازه دادن به \_نشانی {#settings_allow_ddl} +## اجازه دادن به _نشانی {#settings_allow_ddl} اجازه می دهد یا رد می کند [DDL](https://en.wikipedia.org/wiki/Data_definition_language) نمایش داده شد. diff --git a/docs/fa/operations/settings/query-complexity.md b/docs/fa/operations/settings/query-complexity.md index bed318c63d3..e78193e671e 100644 --- a/docs/fa/operations/settings/query-complexity.md +++ b/docs/fa/operations/settings/query-complexity.md @@ -15,8 +15,8 @@ toc_title: "\u0645\u062D\u062F\u0648\u062F\u06CC\u062A \u062F\u0631 \u067E\u06CC خانه را کلیک کنید چک محدودیت برای قطعات داده, نه برای هر سطر. این بدان معنی است که شما می توانید ارزش محدودیت با اندازه بخش داده ها تجاوز. محدودیت در “maximum amount of something” می توانید مقدار را 0, که به معنی “unrestricted”. -اکثر محدودیت ها نیز دارند ‘overflow\_mode’ محیط, به این معنی چه باید بکنید هنگامی که از حد فراتر رفته است. -این می تواند یکی از دو مقدار را: `throw` یا `break`. محدودیت در تجمع (کد \_شورت\_فلو\_وشه گروه) نیز ارزش داشته باشد `any`. +اکثر محدودیت ها نیز دارند ‘overflow_mode’ محیط, به این معنی چه باید بکنید هنگامی که از حد فراتر رفته است. +این می تواند یکی از دو مقدار را: `throw` یا `break`. محدودیت در تجمع (کد _شورت_فلو_وشه گروه) نیز ارزش داشته باشد `any`. `throw` – Throw an exception (default). @@ -24,7 +24,7 @@ toc_title: "\u0645\u062D\u062F\u0648\u062F\u06CC\u062A \u062F\u0631 \u067E\u06CC `any (only for group_by_overflow_mode)` – Continuing aggregation for the keys that got into the set, but don't add new keys to the set. -## \_کاساژ بیشینه {#settings_max_memory_usage} +## _کاساژ بیشینه {#settings_max_memory_usage} حداکثر مقدار رم برای استفاده برای اجرای پرس و جو بر روی یک سرور واحد. @@ -41,47 +41,47 @@ toc_title: "\u0645\u062D\u062F\u0648\u062F\u06CC\u062A \u062F\u0631 \u067E\u06CC مصرف حافظه نیز توسط پارامترها محدود شده است `max_memory_usage_for_user` و `max_memory_usage_for_all_queries`. -## \_شمارهگیر بیشینه {#max-memory-usage-for-user} +## _شمارهگیر بیشینه {#max-memory-usage-for-user} حداکثر مقدار رم برای استفاده برای اجرای نمایش داده شد کاربر بر روی یک سرور واحد. مقادیر پیش فرض در تعریف [تنظیمات.ه](https://github.com/ClickHouse/ClickHouse/blob/master/src/Core/Settings.h#L288). به طور پیش فرض مقدار محدود نمی شود (`max_memory_usage_for_user = 0`). -همچنین نگاه کنید به شرح [\_کاساژ بیشینه](#settings_max_memory_usage). +همچنین نگاه کنید به شرح [_کاساژ بیشینه](#settings_max_memory_usage). -## \_شیشه بخاطر \_خروج {#max-memory-usage-for-all-queries} +## _شیشه بخاطر _خروج {#max-memory-usage-for-all-queries} حداکثر مقدار رم برای استفاده برای اجرای تمام نمایش داده شد بر روی یک سرور واحد. مقادیر پیش فرض در تعریف [تنظیمات.ه](https://github.com/ClickHouse/ClickHouse/blob/master/src/Core/Settings.h#L289). به طور پیش فرض مقدار محدود نمی شود (`max_memory_usage_for_all_queries = 0`). -همچنین نگاه کنید به شرح [\_کاساژ بیشینه](#settings_max_memory_usage). +همچنین نگاه کنید به شرح [_کاساژ بیشینه](#settings_max_memory_usage). -## \_گذرواژههای \_ورود {#max-rows-to-read} +## _گذرواژههای _ورود {#max-rows-to-read} محدودیت های زیر را می توان در هر بلوک بررسی (به جای در هر سطر). به این معنا که, محدودیت را می توان شکسته کمی. هنگامی که در حال اجرا یک پرس و جو در موضوعات مختلف, محدودیت های زیر به هر موضوع اعمال می شود به طور جداگانه. حداکثر تعداد ردیف است که می تواند از یک جدول زمانی که در حال اجرا یک پرس و جو به عنوان خوانده شده. -## \_مخفی کردن {#max-bytes-to-read} +## _مخفی کردن {#max-bytes-to-read} حداکثر تعداد بایت (داده های غیر فشرده) است که می تواند از یک جدول به عنوان خوانده شده در هنگام اجرای یک پرس و جو. -## \_ورود به سیستم {#read-overflow-mode} +## _ورود به سیستم {#read-overflow-mode} چه باید بکنید هنگامی که حجم داده ها به عنوان خوانده شده بیش از یکی از محدودیت های: ‘throw’ یا ‘break’. به طور پیش فرض, پرتاب. -## \_رو\_تو\_گروهها {#settings-max-rows-to-group-by} +## _رو_تو_گروهها {#settings-max-rows-to-group-by} حداکثر تعداد کلید منحصر به فرد دریافت شده از تجمع. این تنظیم به شما امکان مصرف حافظه محدود در هنگام جمع. -## \_شماره \_شماره گروه {#group-by-overflow-mode} +## _شماره _شماره گروه {#group-by-overflow-mode} چه باید بکنید هنگامی که تعدادی از کلید های منحصر به فرد برای تجمع بیش از حد: ‘throw’, ‘break’ یا ‘any’. به طور پیش فرض, پرتاب. با استفاده از ‘any’ ارزش شما اجازه می دهد یک تقریب از گروه های اجرا. کیفیت این تقریب بستگی به ماهیت استاتیک داده ها دارد. -## ا\_فزون\_بر\_گونهی\_گونهی زیر\_گروهها {#settings-max_bytes_before_external_group_by} +## ا_فزون_بر_گونهی_گونهی زیر_گروهها {#settings-max_bytes_before_external_group_by} فعالسازی یا غیرفعالسازی اعدام `GROUP BY` بند در حافظه خارجی. ببینید [گروه در حافظه خارجی](../../sql-reference/statements/select/group-by.md#select-group-by-in-external-memory). @@ -92,15 +92,15 @@ toc_title: "\u0645\u062D\u062F\u0648\u062F\u06CC\u062A \u062F\u0631 \u067E\u06CC مقدار پیش فرض: 0. -## \_شماره بیشینه {#max-rows-to-sort} +## _شماره بیشینه {#max-rows-to-sort} حداکثر تعداد ردیف قبل از مرتب سازی. این اجازه می دهد تا شما را به محدود کردن مصرف حافظه در هنگام مرتب سازی. -## ا\_سلایدی {#max-bytes-to-sort} +## ا_سلایدی {#max-bytes-to-sort} حداکثر تعداد بایت قبل از مرتب سازی. -## کد\_و\_وشهیابی {#sort-overflow-mode} +## کد_و_وشهیابی {#sort-overflow-mode} چه باید بکنید اگر تعداد ردیف قبل از مرتب سازی دریافت بیش از یکی از محدودیت: ‘throw’ یا ‘break’. به طور پیش فرض, پرتاب. @@ -108,15 +108,15 @@ toc_title: "\u0645\u062D\u062F\u0648\u062F\u06CC\u062A \u062F\u0631 \u067E\u06CC محدود در تعداد ردیف در نتیجه. همچنین برای زیرمجموعه بررسی, و بر روی سرور از راه دور در هنگام اجرای بخش هایی از یک پرس و جو توزیع. -## حداکثر\_زمین بایت {#max-result-bytes} +## حداکثر_زمین بایت {#max-result-bytes} محدود در تعداد بایت در نتیجه. همان تنظیمات قبلی. -## \_شماره حاصل {#result-overflow-mode} +## _شماره حاصل {#result-overflow-mode} چه باید بکنید اگر حجم نتیجه بیش از یکی از محدودیت های: ‘throw’ یا ‘break’. به طور پیش فرض, پرتاب. -با استفاده از ‘break’ شبیه به استفاده از حد است. `Break` قطع اعدام تنها در سطح بلوک. این به این معنی است که مقدار ردیف بازگشت بیشتر از [بارشهای بیشینه](#setting-max_result_rows) چندین [ت\_مایش بیشینه](settings.md#setting-max_block_size) و بستگی دارد [\_مخفی کردن](settings.md#settings-max_threads). +با استفاده از ‘break’ شبیه به استفاده از حد است. `Break` قطع اعدام تنها در سطح بلوک. این به این معنی است که مقدار ردیف بازگشت بیشتر از [بارشهای بیشینه](#setting-max_result_rows) چندین [ت_مایش بیشینه](settings.md#setting-max_block_size) و بستگی دارد [_مخفی کردن](settings.md#settings-max_threads). مثال: @@ -135,87 +135,87 @@ FORMAT Null; 6666 rows in set. ... ``` -## زمان \_شنامهی حداکثر {#max-execution-time} +## زمان _شنامهی حداکثر {#max-execution-time} حداکثر زمان اجرای پرس و جو در ثانیه. در این زمان برای یکی از مراحل مرتب سازی بررسی نمی شود و یا هنگام ادغام و نهایی کردن توابع کلی. -## \_شروع مجدد {#timeout-overflow-mode} +## _شروع مجدد {#timeout-overflow-mode} -چه باید بکنید اگر پرس و جو اجرا می شود بیش از ‘max\_execution\_time’: ‘throw’ یا ‘break’. به طور پیش فرض, پرتاب. +چه باید بکنید اگر پرس و جو اجرا می شود بیش از ‘max_execution_time’: ‘throw’ یا ‘break’. به طور پیش فرض, پرتاب. -## \_شروع مجدد {#min-execution-speed} +## _شروع مجدد {#min-execution-speed} -سرعت اجرای حداقل در ردیف در هر ثانیه. بررسی در هر بلوک داده زمانی که ‘timeout\_before\_checking\_execution\_speed’ انقضا مییابد. اگر سرعت اجرای پایین تر است, یک استثنا پرتاب می شود. +سرعت اجرای حداقل در ردیف در هر ثانیه. بررسی در هر بلوک داده زمانی که ‘timeout_before_checking_execution_speed’ انقضا مییابد. اگر سرعت اجرای پایین تر است, یک استثنا پرتاب می شود. -## ا\_فزونهها {#min-execution-speed-bytes} +## ا_فزونهها {#min-execution-speed-bytes} -حداقل تعداد بایت اعدام در هر ثانیه. بررسی در هر بلوک داده زمانی که ‘timeout\_before\_checking\_execution\_speed’ انقضا مییابد. اگر سرعت اجرای پایین تر است, یک استثنا پرتاب می شود. +حداقل تعداد بایت اعدام در هر ثانیه. بررسی در هر بلوک داده زمانی که ‘timeout_before_checking_execution_speed’ انقضا مییابد. اگر سرعت اجرای پایین تر است, یک استثنا پرتاب می شود. -## حداکثر\_حاقسازی سرعت {#max-execution-speed} +## حداکثر_حاقسازی سرعت {#max-execution-speed} -حداکثر تعداد ردیف اعدام در هر ثانیه. بررسی در هر بلوک داده زمانی که ‘timeout\_before\_checking\_execution\_speed’ انقضا مییابد. اگر سرعت اجرای بالا است, سرعت اجرای کاهش خواهد یافت. +حداکثر تعداد ردیف اعدام در هر ثانیه. بررسی در هر بلوک داده زمانی که ‘timeout_before_checking_execution_speed’ انقضا مییابد. اگر سرعت اجرای بالا است, سرعت اجرای کاهش خواهد یافت. -## حداکثر\_کشن\_پیمایههای سرعت {#max-execution-speed-bytes} +## حداکثر_کشن_پیمایههای سرعت {#max-execution-speed-bytes} -حداکثر تعداد بایت اعدام در هر ثانیه. بررسی در هر بلوک داده زمانی که ‘timeout\_before\_checking\_execution\_speed’ انقضا مییابد. اگر سرعت اجرای بالا است, سرعت اجرای کاهش خواهد یافت. +حداکثر تعداد بایت اعدام در هر ثانیه. بررسی در هر بلوک داده زمانی که ‘timeout_before_checking_execution_speed’ انقضا مییابد. اگر سرعت اجرای بالا است, سرعت اجرای کاهش خواهد یافت. ## جستجو {#timeout-before-checking-execution-speed} -چک که سرعت اجرای بیش از حد کند نیست (کمتر از ‘min\_execution\_speed’), پس از زمان مشخص شده در ثانیه تمام شده است. +چک که سرعت اجرای بیش از حد کند نیست (کمتر از ‘min_execution_speed’), پس از زمان مشخص شده در ثانیه تمام شده است. -## \_رنگ \_ورود {#max-columns-to-read} +## _رنگ _ورود {#max-columns-to-read} حداکثر تعداد ستون است که می تواند از یک جدول در یک پرس و جو به عنوان خوانده شده. اگر پرس و جو نیاز به خواندن تعداد بیشتری از ستون, این می اندازد یک استثنا. -## \_رنگ بیشینه {#max-temporary-columns} +## _رنگ بیشینه {#max-temporary-columns} حداکثر تعداد ستون موقت است که باید در رم در همان زمان نگه داشته شود که در حال اجرا یک پرس و جو, از جمله ستون ثابت. اگر ستون موقت بیش از این وجود دارد, این یک استثنا می اندازد. -## \_رنگ {#max-temporary-non-const-columns} +## _رنگ {#max-temporary-non-const-columns} -همان چیزی که به عنوان ‘max\_temporary\_columns’, اما بدون شمارش ستون ثابت. +همان چیزی که به عنوان ‘max_temporary_columns’, اما بدون شمارش ستون ثابت. توجه داشته باشید که ستون های ثابت در حال اجرا یک پرس و جو نسبتا اغلب تشکیل, اما نیاز به حدود صفر منابع محاسباتی. ## حداکثر {#max-subquery-depth} حداکثر عمق تودرتو از کارخانه های فرعی. اگر کارخانه های فرعی عمیق تر, یک استثنا پرتاب می شود. به طور پیش فرض, 100. -## حداکثر \_پیپیلین {#max-pipeline-depth} +## حداکثر _پیپیلین {#max-pipeline-depth} حداکثر عمق خط لوله. مربوط به تعدادی از تحولات که هر بلوک داده می رود از طریق در طول پردازش پرس و جو. شمارش در محدوده یک سرور واحد. اگر عمق خط لوله بیشتر است, یک استثنا پرتاب می شود. به طور پیش فرض 1000. -## \_ص\_خلاف {#max-ast-depth} +## _ص_خلاف {#max-ast-depth} حداکثر عمق تودرتو از یک درخت نحوی پرس و جو. اگر بیش از, یک استثنا پرتاب می شود. در این زمان در تجزیه بررسی نمی شود اما تنها پس از تجزیه پرس و جو. به این معنا که, یک درخت نحوی است که بیش از حد عمیق می تواند در طول تجزیه ایجاد, اما پرس و جو شکست مواجه خواهد شد. به طور پیش فرض 1000. -## \_محلولات حداکثر {#max-ast-elements} +## _محلولات حداکثر {#max-ast-elements} حداکثر تعداد عناصر در یک درخت نحوی پرس و جو. اگر بیش از, یک استثنا پرتاب می شود. در همان راه به عنوان تنظیمات قبلی تنها پس از تجزیه پرس و جو بررسی می شود. به طور پیش فرض 50000. -## \_رو\_ تنظیم {#max-rows-in-set} +## _رو_ تنظیم {#max-rows-in-set} حداکثر تعداد ردیف برای یک مجموعه داده ها در بند در ایجاد شده از یک خرده فروشی. -## تنظیم \_سریع {#max-bytes-in-set} +## تنظیم _سریع {#max-bytes-in-set} حداکثر تعداد بایت (داده های غیر فشرده) استفاده شده توسط یک مجموعه در بند در ایجاد شده از یک خرده فروشی. -## \_حالت تنظیم {#set-overflow-mode} +## _حالت تنظیم {#set-overflow-mode} چه باید بکنید هنگامی که مقدار داده ها بیش از یکی از محدودیت های: ‘throw’ یا ‘break’. به طور پیش فرض, پرتاب. -## حوزه \_کاربری مکس {#max-rows-in-distinct} +## حوزه _کاربری مکس {#max-rows-in-distinct} حداکثر تعداد ردیف های مختلف در هنگام استفاده از متمایز. -## مک\_بتس\_ حوزه {#max-bytes-in-distinct} +## مک_بتس_ حوزه {#max-bytes-in-distinct} حداکثر تعداد بایت استفاده شده توسط یک جدول هش در هنگام استفاده متمایز. -## \_شروع مجدد {#distinct-overflow-mode} +## _شروع مجدد {#distinct-overflow-mode} چه باید بکنید هنگامی که مقدار داده ها بیش از یکی از محدودیت های: ‘throw’ یا ‘break’. به طور پیش فرض, پرتاب. @@ -227,11 +227,11 @@ FORMAT Null; حداکثر تعداد بایت (داده های غیر فشرده) است که می تواند به یک سرور از راه دور منتقل می شود و یا ذخیره شده در یک جدول موقت در هنگام استفاده از جهانی در. -## \_شروع مجدد {#transfer-overflow-mode} +## _شروع مجدد {#transfer-overflow-mode} چه باید بکنید هنگامی که مقدار داده ها بیش از یکی از محدودیت های: ‘throw’ یا ‘break’. به طور پیش فرض, پرتاب. -## \_پاک کردن \_روشن گرافیک {#settings-max_rows_in_join} +## _پاک کردن _روشن گرافیک {#settings-max_rows_in_join} محدودیت تعداد ردیف در جدول هش استفاده شده است که در هنگام پیوستن به جداول. @@ -239,7 +239,7 @@ FORMAT Null; اگر یک پرس و جو شامل چند می پیوندد, خانه چک این تنظیم برای هر نتیجه متوسط. -تاتر می توانید با اقدامات مختلف ادامه دهید زمانی که از حد رسیده است. استفاده از [\_شروع مجدد](#settings-join_overflow_mode) تنظیم برای انتخاب عمل. +تاتر می توانید با اقدامات مختلف ادامه دهید زمانی که از حد رسیده است. استفاده از [_شروع مجدد](#settings-join_overflow_mode) تنظیم برای انتخاب عمل. مقادیر ممکن: @@ -248,7 +248,7 @@ FORMAT Null; مقدار پیش فرض: 0. -## \_پویش همیشگی {#settings-max_bytes_in_join} +## _پویش همیشگی {#settings-max_bytes_in_join} محدودیت اندازه در بایت از جدول هش استفاده می شود در هنگام پیوستن به جداول. @@ -256,7 +256,7 @@ FORMAT Null; اگر پرس و جو شامل می پیوندد, کلیک چک این تنظیمات برای هر نتیجه متوسط. -تاتر می توانید با اقدامات مختلف ادامه دهید زمانی که از حد رسیده است. استفاده [\_شروع مجدد](#settings-join_overflow_mode) تنظیمات برای انتخاب عمل. +تاتر می توانید با اقدامات مختلف ادامه دهید زمانی که از حد رسیده است. استفاده [_شروع مجدد](#settings-join_overflow_mode) تنظیمات برای انتخاب عمل. مقادیر ممکن: @@ -265,12 +265,12 @@ FORMAT Null; مقدار پیش فرض: 0. -## \_شروع مجدد {#settings-join_overflow_mode} +## _شروع مجدد {#settings-join_overflow_mode} تعریف می کند که چه عمل کلیک انجام زمانی که هر یک از محدودیت های زیر ملحق رسیده است: -- [\_پویش همیشگی](#settings-max_bytes_in_join) -- [\_پاک کردن \_روشن گرافیک](#settings-max_rows_in_join) +- [_پویش همیشگی](#settings-max_bytes_in_join) +- [_پاک کردن _روشن گرافیک](#settings-max_rows_in_join) مقادیر ممکن: @@ -284,7 +284,7 @@ FORMAT Null; - [پیوستن بند](../../sql-reference/statements/select/join.md#select-join) - [پیوستن به موتور جدول](../../engines/table-engines/special/join.md) -## \_مسدود کردن بیشینه {#max-partitions-per-insert-block} +## _مسدود کردن بیشینه {#max-partitions-per-insert-block} حداکثر تعداد پارتیشن در یک بلوک قرار داده شده را محدود می کند. @@ -297,6 +297,6 @@ FORMAT Null; هنگام قرار دادن داده ها, تاتر محاسبه تعداد پارتیشن در بلوک قرار داده. اگر تعداد پارتیشن ها بیش از `max_partitions_per_insert_block`, خانه را کلیک می اندازد یک استثنا با متن زیر: -> “Too many partitions for single INSERT block (more than” اطلاعات دقیق “). The limit is controlled by ‘max\_partitions\_per\_insert\_block’ setting. A large number of partitions is a common misconception. It will lead to severe negative performance impact, including slow server startup, slow INSERT queries and slow SELECT queries. Recommended total number of partitions for a table is under 1000..10000. Please note, that partitioning is not intended to speed up SELECT queries (ORDER BY key is sufficient to make range queries fast). Partitions are intended for data manipulation (DROP PARTITION, etc).” +> “Too many partitions for single INSERT block (more than” اطلاعات دقیق “). The limit is controlled by ‘max_partitions_per_insert_block’ setting. A large number of partitions is a common misconception. It will lead to severe negative performance impact, including slow server startup, slow INSERT queries and slow SELECT queries. Recommended total number of partitions for a table is under 1000..10000. Please note, that partitioning is not intended to speed up SELECT queries (ORDER BY key is sufficient to make range queries fast). Partitions are intended for data manipulation (DROP PARTITION, etc).” [مقاله اصلی](https://clickhouse.tech/docs/en/operations/settings/query_complexity/) diff --git a/docs/fa/operations/settings/settings.md b/docs/fa/operations/settings/settings.md index 6b820dcf5c2..61cb2a9793f 100644 --- a/docs/fa/operations/settings/settings.md +++ b/docs/fa/operations/settings/settings.md @@ -5,7 +5,7 @@ machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd # تنظیمات {#settings} -## \_شماره توزیع شده {#distributed-product-mode} +## _شماره توزیع شده {#distributed-product-mode} تغییر رفتار [توزیع subqueries](../../sql-reference/operators/in.md). @@ -75,7 +75,7 @@ ClickHouse applies this setting when the query contains the product of distribut اگر `force_primary_key=1` چک خانه را ببینید اگر پرس و جو شرایط کلیدی اولیه است که می تواند مورد استفاده قرار گیرد برای محدود کردن محدوده داده است. اگر هیچ شرایط مناسب وجود دارد, این یک استثنا می اندازد. با این حال, بررسی نمی کند که وضعیت مقدار داده ها به خواندن را کاهش می دهد. برای کسب اطلاعات بیشتر در مورد محدوده داده ها در جداول ادغام, دیدن [ادغام](../../engines/table-engines/mergetree-family/mergetree.md). -## قالب\_نما {#format-schema} +## قالب_نما {#format-schema} این پارامتر زمانی مفید است که شما با استفاده از فرمت های که نیاز به یک تعریف طرح, مانند [سروان نیا](https://capnproto.org/) یا [Protobuf](https://developers.google.com/protocol-buffers/). ارزش بستگی به فرمت. @@ -98,7 +98,7 @@ ClickHouse applies this setting when the query contains the product of distribut مقدار پیش فرض: 0. -## \_تنظیم مجدد به حالت اولیه {#settings-http_zlib_compression_level} +## _تنظیم مجدد به حالت اولیه {#settings-http_zlib_compression_level} سطح فشرده سازی داده ها را در پاسخ به درخواست قام تنظیم می کند [قابلیت تنظیم صدا = 1](#settings-enable_http_compression). @@ -143,7 +143,7 @@ ClickHouse applies this setting when the query contains the product of distribut مقدار پیش فرض: 0. -## وارد کردن \_فرست\_مرزیابی \_نمایش مجدد {#settings-input_format_allow_errors_num} +## وارد کردن _فرست_مرزیابی _نمایش مجدد {#settings-input_format_allow_errors_num} حداکثر تعداد خطاهای قابل قبول را در هنگام خواندن از فرمت های متنی تنظیم می کند). @@ -239,7 +239,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( - اگر `input_format_values_interpret_expressions=0` و `format_values_deduce_templates_of_expressions=1`, عبارات در اولین, ردیف دوم و سوم با استفاده از الگو تجزیه `lower(String)` و با هم تفسیر, بیان در ردیف جلو با قالب دیگری تجزیه (`upper(String)`). - اگر `input_format_values_interpret_expressions=1` و `format_values_deduce_templates_of_expressions=1`, همان است که در مورد قبلی, بلکه اجازه می دهد تا عقب نشینی به تفسیر عبارات به طور جداگانه اگر این امکان وجود ندارد به استنباط الگو. -## وارد کردن \_تماس\_عول\_ایجاد \_شکلتهای \_شخصی {#settings-input-format-values-accurate-types-of-literals} +## وارد کردن _تماس_عول_ایجاد _شکلتهای _شخصی {#settings-input-format-values-accurate-types-of-literals} این تنظیم تنها زمانی استفاده می شود `input_format_values_deduce_templates_of_expressions = 1`. این می تواند رخ دهد, که عبارت برای برخی از ستون دارای ساختار مشابه, اما حاوی لیتر عددی از انواع مختلف, به عنوان مثال @@ -261,7 +261,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( مقدار پیش فرض: 1. -## \_پوشه های ورودی و خروجی {#session_settings-input_format_defaults_for_omitted_fields} +## _پوشه های ورودی و خروجی {#session_settings-input_format_defaults_for_omitted_fields} هنگام انجام `INSERT` نمایش داده شد, جایگزین مقادیر ستون ورودی حذف شده با مقادیر پیش فرض از ستون مربوطه. این گزینه فقط برای اعمال [جیسانچرو](../../interfaces/formats.md#jsoneachrow), [CSV](../../interfaces/formats.md#csv) و [جدول دار](../../interfaces/formats.md#tabseparated) فرمتها. @@ -285,7 +285,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( را قادر می سازد و یا غیر فعال با استفاده از مقادیر پیش فرض اگر داده های ورودی شامل `NULL`, اما نوع داده از ستون مربوطه در نمی `Nullable(T)` (برای فرمت های ورودی متن). -## \_دفتر\_صرفههای شناسنامهی ورودی {#settings-input-format-skip-unknown-fields} +## _دفتر_صرفههای شناسنامهی ورودی {#settings-input-format-skip-unknown-fields} را قادر می سازد و یا غیر فعال پرش درج داده های اضافی. @@ -305,7 +305,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( مقدار پیش فرض: 0. -## تغییر \_کم\_تر\_تنظیم مجدد \_جنسان {#settings-input_format_import_nested_json} +## تغییر _کم_تر_تنظیم مجدد _جنسان {#settings-input_format_import_nested_json} درج دادههای جسون را با اشیای تو در تو فعال یا غیرفعال میکند. @@ -324,7 +324,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( - [استفاده از ساختارهای تو در تو](../../interfaces/formats.md#jsoneachrow-nested) با `JSONEachRow` قالب. -## \_فرست\_ام\_امنمایش گذرواژه {#settings-input-format-with-names-use-header} +## _فرست_ام_امنمایش گذرواژه {#settings-input-format-with-names-use-header} را قادر می سازد و یا غیر فعال چک کردن سفارش ستون در هنگام قرار دادن داده ها. @@ -342,7 +342,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( مقدار پیش فرض: 1. -## تغییر \_شماره {#settings-date_time_input_format} +## تغییر _شماره {#settings-date_time_input_format} اجازه می دهد تا انتخاب تجزیه کننده از نمایش متن از تاریخ و زمان. @@ -409,7 +409,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( مقدار پیش فرض: 0. -## ت\_مایش بیشینه {#setting-max_block_size} +## ت_مایش بیشینه {#setting-max_block_size} در خانه, داده ها توسط بلوک های پردازش (مجموعه ای از قطعات ستون). چرخه پردازش داخلی برای یک بلوک به اندازه کافی موثر هستند, اما هزینه های قابل توجه در هر بلوک وجود دارد. این `max_block_size` تنظیم یک توصیه برای چه اندازه بلوک (در تعداد ردیف) برای بارگذاری از جداول است. اندازه بلوک نباید بیش از حد کوچک, به طوری که هزینه در هر بلوک هنوز هم قابل توجه است, اما نه بیش از حد بزرگ به طوری که پرس و جو با محدودیت است که پس از اولین بلوک به سرعت پردازش تکمیل. هدف این است که برای جلوگیری از مصرف حافظه بیش از حد در هنگام استخراج تعداد زیادی از ستون ها در موضوعات مختلف و برای حفظ حداقل برخی از محل کش. @@ -417,13 +417,13 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( بلوک اندازه `max_block_size` همیشه از جدول لود نمی. اگر واضح است که داده های کمتر نیاز به بازیابی یک بلوک کوچکتر پردازش می شود. -## ترجی\_حات {#preferred-block-size-bytes} +## ترجی_حات {#preferred-block-size-bytes} مورد استفاده برای همان هدف به عنوان `max_block_size` اما با تطبیق تعداد سطرها در بلوک اندازه بلوک توصیه شده را در بایت تنظیم می کند. با این حال, اندازه بلوک نمی تواند بیش از `max_block_size` ردیف به طور پیش فرض: 1,000,000. تنها در هنگام خواندن از موتورهای ادغام کار می کند. -## ادغام \_تر\_م\_را\_م\_مایش مجدد {#setting-merge-tree-min-rows-for-concurrent-read} +## ادغام _تر_م_را_م_مایش مجدد {#setting-merge-tree-min-rows-for-concurrent-read} اگر تعداد ردیف از یک فایل از یک خوانده شود [ادغام](../../engines/table-engines/mergetree-family/mergetree.md) جدول بیش از `merge_tree_min_rows_for_concurrent_read` سپس کلیک کنیدهاوس تلاش می کند برای انجام خواندن همزمان از این فایل در موضوعات مختلف. @@ -433,7 +433,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( مقدار پیش فرض: 163840. -## \_انتقال به \_انتقال به \_شخصی {#setting-merge-tree-min-bytes-for-concurrent-read} +## _انتقال به _انتقال به _شخصی {#setting-merge-tree-min-bytes-for-concurrent-read} اگر تعداد بایت برای خواندن از یک فایل از یک [ادغام](../../engines/table-engines/mergetree-family/mergetree.md)- جدول موتور بیش از `merge_tree_min_bytes_for_concurrent_read` سپس کلیک کنیدهاوس تلاش می کند به صورت همزمان از این فایل در موضوعات مختلف به عنوان خوانده شده. @@ -443,7 +443,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( مقدار پیش فرض: 251658240. -## ادغام \_تر\_م\_را\_م\_را\_مایش مجدد {#setting-merge-tree-min-rows-for-seek} +## ادغام _تر_م_را_م_را_مایش مجدد {#setting-merge-tree-min-rows-for-seek} اگر فاصله بین دو بلوک داده در یک فایل خوانده شود کمتر از `merge_tree_min_rows_for_seek` ردیف, سپس کلیک می کند از طریق فایل به دنبال ندارد اما می خواند پی در پی داده ها. @@ -453,7 +453,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( مقدار پیش فرض: 0. -## ادغام \_تر\_حضربه \_ترکمال {#setting-merge-tree-min-bytes-for-seek} +## ادغام _تر_حضربه _ترکمال {#setting-merge-tree-min-bytes-for-seek} اگر فاصله بین دو بلوک داده در یک فایل خوانده شود کمتر از `merge_tree_min_bytes_for_seek` بایت, سپس پی در پی تاتر می خواند طیف وسیعی از فایل است که شامل هر دو بلوک, در نتیجه اجتناب اضافی به دنبال. @@ -463,7 +463,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( مقدار پیش فرض: 0. -## ادغام \_تر\_کوارسی\_یندگرمانی {#setting-merge-tree-coarse-index-granularity} +## ادغام _تر_کوارسی_یندگرمانی {#setting-merge-tree-coarse-index-granularity} هنگامی که جستجو برای داده ها, تاتر چک علامت داده ها در فایل شاخص. اگر فاحشه خانه می یابد که کلید های مورد نیاز در برخی از محدوده هستند, این تقسیم این محدوده به `merge_tree_coarse_index_granularity` موشک و جستجو کلید های مورد نیاز وجود دارد به صورت بازگشتی. @@ -473,11 +473,11 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( مقدار پیش فرض: 8. -## \_انتقال به \_انتقال {#setting-merge-tree-max-rows-to-use-cache} +## _انتقال به _انتقال {#setting-merge-tree-max-rows-to-use-cache} اگر کلیک خانه باید بیش از خواندن `merge_tree_max_rows_to_use_cache` ردیف ها در یک پرس و جو از کش بلوک های غیر فشرده استفاده نمی کنند. -ذخیره سازی داده های ذخیره شده بلوک های غیر فشرده برای نمایش داده شد. تاتر با استفاده از این کش برای سرعت بخشیدن به پاسخ به نمایش داده شد کوچک تکرار شده است. این تنظیم محافظت از کش از سطل زباله توسط نمایش داده شد که مقدار زیادی از داده ها به عنوان خوانده شده. این [\_بالا](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) تنظیم سرور اندازه کش از بلوک های غیر فشرده را تعریف می کند. +ذخیره سازی داده های ذخیره شده بلوک های غیر فشرده برای نمایش داده شد. تاتر با استفاده از این کش برای سرعت بخشیدن به پاسخ به نمایش داده شد کوچک تکرار شده است. این تنظیم محافظت از کش از سطل زباله توسط نمایش داده شد که مقدار زیادی از داده ها به عنوان خوانده شده. این [_بالا](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) تنظیم سرور اندازه کش از بلوک های غیر فشرده را تعریف می کند. مقادیر ممکن: @@ -485,11 +485,11 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( Default value: 128 ✕ 8192. -## \_انتقال به \_انتقال {#setting-merge-tree-max-bytes-to-use-cache} +## _انتقال به _انتقال {#setting-merge-tree-max-bytes-to-use-cache} اگر کلیک خانه باید بیش از خواندن `merge_tree_max_bytes_to_use_cache` بایت در یک پرس و جو, این کش از بلوک های غیر فشرده استفاده نمی. -ذخیره سازی داده های ذخیره شده بلوک های غیر فشرده برای نمایش داده شد. تاتر با استفاده از این کش برای سرعت بخشیدن به پاسخ به نمایش داده شد کوچک تکرار شده است. این تنظیم محافظت از کش از سطل زباله توسط نمایش داده شد که مقدار زیادی از داده ها به عنوان خوانده شده. این [\_بالا](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) تنظیم سرور اندازه کش از بلوک های غیر فشرده را تعریف می کند. +ذخیره سازی داده های ذخیره شده بلوک های غیر فشرده برای نمایش داده شد. تاتر با استفاده از این کش برای سرعت بخشیدن به پاسخ به نمایش داده شد کوچک تکرار شده است. این تنظیم محافظت از کش از سطل زباله توسط نمایش داده شد که مقدار زیادی از داده ها به عنوان خوانده شده. این [_بالا](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) تنظیم سرور اندازه کش از بلوک های غیر فشرده را تعریف می کند. مقدار ممکن: @@ -497,7 +497,7 @@ Default value: 128 ✕ 8192. مقدار پیش فرض: 2013265920. -## \_عنوان \_تو\_میشه {#settings-min-bytes-to-use-direct-io} +## _عنوان _تو_میشه {#settings-min-bytes-to-use-direct-io} حداقل حجم داده های مورد نیاز برای استفاده از دسترسی مستقیم به دیسک ذخیره سازی. @@ -510,11 +510,11 @@ Default value: 128 ✕ 8192. مقدار پیش فرض: 0. -## \_خروج {#settings-log-queries} +## _خروج {#settings-log-queries} راه اندازی ورود به سیستم پرس و جو. -نمایش داده شد با توجه به قوانین در به کلیک خانه فرستاده می شود [\_خروج](../server-configuration-parameters/settings.md#server_configuration_parameters-query-log) پارامتر پیکربندی سرور. +نمایش داده شد با توجه به قوانین در به کلیک خانه فرستاده می شود [_خروج](../server-configuration-parameters/settings.md#server_configuration_parameters-query-log) پارامتر پیکربندی سرور. مثال: @@ -522,7 +522,7 @@ Default value: 128 ✕ 8192. log_queries=1 ``` -## \_قاب کردن \_نوع {#settings-log-queries-min-type} +## _قاب کردن _نوع {#settings-log-queries-min-type} `query_log` حداقل نوع برای ورود به سیستم. @@ -544,7 +544,7 @@ log_queries_min_type='EXCEPTION_WHILE_PROCESSING' راه اندازی موضوعات پرس و جو ورود به سیستم. -نمایش داده شد' موضوعات runned توسط ClickHouse با راه اندازی این سیستم هستند با توجه به قوانین در [\_ر\_خروج](../server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) پارامتر پیکربندی سرور. +نمایش داده شد' موضوعات runned توسط ClickHouse با راه اندازی این سیستم هستند با توجه به قوانین در [_ر_خروج](../server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) پارامتر پیکربندی سرور. مثال: @@ -552,19 +552,19 @@ log_queries_min_type='EXCEPTION_WHILE_PROCESSING' log_query_threads=1 ``` -## ا\_فزونهها {#settings-max_insert_block_size} +## ا_فزونهها {#settings-max_insert_block_size} اندازه بلوک به شکل برای درج به یک جدول. این تنظیم فقط در مواردی که سرور بلوک را تشکیل می دهد اعمال می شود. برای مثال برای درج از طریق رابط اچ.تی. تی. پی سرور تجزیه فرمت داده ها و اشکال بلوک از اندازه مشخص شده است. -اما هنگامی که با استفاده از کلیک-مشتری تجزیه داده های خود را و ‘max\_insert\_block\_size’ تنظیم بر روی سرور به اندازه بلوک قرار داده تاثیر نمی گذارد. +اما هنگامی که با استفاده از کلیک-مشتری تجزیه داده های خود را و ‘max_insert_block_size’ تنظیم بر روی سرور به اندازه بلوک قرار داده تاثیر نمی گذارد. تنظیمات نیز یک هدف در هنگام استفاده از درج را انتخاب کنید ندارد, از داده ها با استفاده از بلوک های مشابه که پس از انتخاب تشکیل قرار داده. مقدار پیش فرض: 1,048,576. به طور پیش فرض کمی بیش از `max_block_size`. دلیل این کار این است زیرا موتورهای جدول خاص (`*MergeTree`) بخش داده ها بر روی دیسک برای هر بلوک قرار داده شده است که یک نهاد نسبتا بزرگ را تشکیل می دهند. به طور مشابه, `*MergeTree` جداول مرتب سازی بر داده ها در هنگام درج و اندازه بلوک به اندازه کافی بزرگ اجازه می دهد مرتب سازی داده های بیشتر در رم. -## \_معرض \_سبک\_ز\_وز {#min-insert-block-size-rows} +## _معرض _سبک_ز_وز {#min-insert-block-size-rows} مجموعه حداقل تعداد ردیف در بلوک است که می تواند به یک جدول توسط یک قرار داده `INSERT` پرس و جو. بلوک های کوچکتر به اندازه به موارد بزرگتر له می شوند. @@ -575,7 +575,7 @@ log_query_threads=1 مقدار پیش فرض: 1048576. -## ا\_فزونهها {#min-insert-block-size-bytes} +## ا_فزونهها {#min-insert-block-size-bytes} مجموعه حداقل تعداد بایت در بلوک است که می تواند به یک جدول توسط یک قرار داده `INSERT` پرس و جو. بلوک های کوچکتر به اندازه به موارد بزرگتر له می شوند. @@ -586,7 +586,7 @@ log_query_threads=1 مقدار پیش فرض: 268435456. -## \_شروع مجدد \_شروع مجدد \_شروع مجدد \_کاربری {#settings-max_replica_delay_for_distributed_queries} +## _شروع مجدد _شروع مجدد _شروع مجدد _کاربری {#settings-max_replica_delay_for_distributed_queries} غیرفعال تاخیر کپی برای نمایش داده شد توزیع شده است. ببینید [تکرار](../../engines/table-engines/mergetree-family/replication.md). @@ -596,22 +596,22 @@ log_query_threads=1 مورد استفاده در هنگام انجام `SELECT` از یک جدول توزیع شده است که اشاره به جداول تکرار. -## \_مخفی کردن {#settings-max_threads} +## _مخفی کردن {#settings-max_threads} -حداکثر تعداد موضوعات پردازش پرس و جو, به جز موضوعات برای بازیابی داده ها از سرور از راه دور (دیدن ‘max\_distributed\_connections’ پارامتر). +حداکثر تعداد موضوعات پردازش پرس و جو, به جز موضوعات برای بازیابی داده ها از سرور از راه دور (دیدن ‘max_distributed_connections’ پارامتر). این پارامتر شامل موضوعات است که انجام همان مراحل از خط لوله پردازش پرس و جو به صورت موازی. -مثلا, در هنگام خواندن از یک جدول, اگر ممکن است به ارزیابی عبارات با توابع, فیلتر با کجا و از پیش جمع شده برای گروه به صورت موازی با استفاده از حداقل ‘max\_threads’ تعداد موضوعات, سپس ‘max\_threads’ استفاده می شود. +مثلا, در هنگام خواندن از یک جدول, اگر ممکن است به ارزیابی عبارات با توابع, فیلتر با کجا و از پیش جمع شده برای گروه به صورت موازی با استفاده از حداقل ‘max_threads’ تعداد موضوعات, سپس ‘max_threads’ استفاده می شود. مقدار پیش فرض: تعداد هسته های پردازنده فیزیکی. اگر کمتر از یک پرس و جو را انتخاب کنید به طور معمول بر روی یک سرور در یک زمان اجرا, تنظیم این پارامتر به یک مقدار کمی کمتر از تعداد واقعی هسته پردازنده. -برای نمایش داده شد که به سرعت به دلیل محدودیت تکمیل, شما می توانید یک مجموعه پایین تر ‘max\_threads’. مثلا , اگر تعداد لازم از نوشته در هر بلوک و حداکثر \_سرخ واقع = 8, سپس 8 بلوک بازیابی می شوند, اگر چه این امر می توانست به اندازه کافی برای خواندن فقط یک بوده است. +برای نمایش داده شد که به سرعت به دلیل محدودیت تکمیل, شما می توانید یک مجموعه پایین تر ‘max_threads’. مثلا , اگر تعداد لازم از نوشته در هر بلوک و حداکثر _سرخ واقع = 8, سپس 8 بلوک بازیابی می شوند, اگر چه این امر می توانست به اندازه کافی برای خواندن فقط یک بوده است. کوچکتر `max_threads` ارزش حافظه کمتر مصرف می شود. -## ا\_فزونهها {#settings-max-insert-threads} +## ا_فزونهها {#settings-max-insert-threads} حداکثر تعداد موضوعات برای اجرای `INSERT SELECT` پرس و جو. @@ -622,61 +622,61 @@ log_query_threads=1 مقدار پیش فرض: 0. -موازی `INSERT SELECT` اثر تنها در صورتی که `SELECT` بخش به صورت موازی اجرا, دیدن [\_مخفی کردن](#settings-max_threads) تنظیمات. +موازی `INSERT SELECT` اثر تنها در صورتی که `SELECT` بخش به صورت موازی اجرا, دیدن [_مخفی کردن](#settings-max_threads) تنظیمات. مقادیر بالاتر به استفاده از حافظه بالاتر منجر شود. -## \_بزرگنمایی {#max-compress-block-size} +## _بزرگنمایی {#max-compress-block-size} حداکثر اندازه بلوک از داده های غیر فشرده قبل از فشرده سازی برای نوشتن به یک جدول. به طور پیش فرض, 1,048,576 (1 مگابایت). اگر اندازه کاهش می یابد, میزان فشرده سازی به طور قابل توجهی کاهش می یابد, سرعت فشرده سازی و رفع فشار کمی با توجه به محل کش را افزایش می دهد, و مصرف حافظه کاهش می یابد. معمولا وجود دارد هر دلیلی برای تغییر این تنظیم نیست. هنوز بلوک برای فشرده سازی اشتباه نیست (یک تکه از حافظه متشکل از بایت) با بلوک برای پردازش پرس و جو (مجموعه ای از ردیف از یک جدول). -## \_بزرگنمایی {#min-compress-block-size} +## _بزرگنمایی {#min-compress-block-size} -برای [ادغام](../../engines/table-engines/mergetree-family/mergetree.md)"جداول . به منظور کاهش زمان تاخیر در هنگام پردازش نمایش داده شد, یک بلوک فشرده شده است در هنگام نوشتن علامت بعدی اگر اندازه خود را حداقل ‘min\_compress\_block\_size’. به طور پیش فرض 65,536. +برای [ادغام](../../engines/table-engines/mergetree-family/mergetree.md)"جداول . به منظور کاهش زمان تاخیر در هنگام پردازش نمایش داده شد, یک بلوک فشرده شده است در هنگام نوشتن علامت بعدی اگر اندازه خود را حداقل ‘min_compress_block_size’. به طور پیش فرض 65,536. -اندازه واقعی بلوک, اگر داده غیر فشرده کمتر از است ‘max\_compress\_block\_size’, کمتر از این مقدار و کمتر از حجم داده ها برای یک علامت. +اندازه واقعی بلوک, اگر داده غیر فشرده کمتر از است ‘max_compress_block_size’, کمتر از این مقدار و کمتر از حجم داده ها برای یک علامت. -بیایید نگاهی به عنوان مثال. فرض کنیم که ‘index\_granularity’ در طول ایجاد جدول به 8192 تنظیم شد. +بیایید نگاهی به عنوان مثال. فرض کنیم که ‘index_granularity’ در طول ایجاد جدول به 8192 تنظیم شد. -ما در حال نوشتن یک ستون نوع 32 (4 بایت در هر مقدار). هنگام نوشتن 8192 ردیف, کل خواهد بود 32 کیلوبایت داده. پس min\_compress\_block\_size = 65,536 یک فشرده بلوک تشکیل خواهد شد برای هر دو نشانه است. +ما در حال نوشتن یک ستون نوع 32 (4 بایت در هر مقدار). هنگام نوشتن 8192 ردیف, کل خواهد بود 32 کیلوبایت داده. پس min_compress_block_size = 65,536 یک فشرده بلوک تشکیل خواهد شد برای هر دو نشانه است. ما در حال نوشتن یک ستون نشانی اینترنتی با نوع رشته (اندازه متوسط 60 بایت در هر مقدار). هنگام نوشتن 8192 ردیف, متوسط خواهد بود کمی کمتر از 500 کیلوبایت داده. پس از این بیش از است 65,536, یک بلوک فشرده خواهد شد برای هر علامت تشکیل. در این مورد, در هنگام خواندن داده ها از دیسک در طیف وسیعی از یک علامت, اطلاعات اضافی نمی خواهد از حالت فشرده خارج شود. معمولا وجود دارد هر دلیلی برای تغییر این تنظیم نیست. -## بیشینه\_کرکی\_سیز {#settings-max_query_size} +## بیشینه_کرکی_سیز {#settings-max_query_size} حداکثر بخشی از پرس و جو است که می تواند به رم برای تجزیه با تجزیه کننده گذاشتن گرفته شده است. پرس و جو درج همچنین شامل داده ها برای درج است که توسط تجزیه کننده جریان جداگانه پردازش (که مصرف درجه(1) رم), است که در این محدودیت شامل نمی شود. مقدار پیش فرض: 256 کیلوبایت. -## فعالسازی \_دلای {#interactive-delay} +## فعالسازی _دلای {#interactive-delay} فاصله در میکروثانیه برای بررسی اینکه اجرای درخواست لغو شده است و ارسال پیشرفت. مقدار پیش فرض: 100,000 (چک برای لغو و پیشرفت می فرستد ده بار در ثانیه). -## connect\_timeout, receive\_timeout, send\_timeout {#connect-timeout-receive-timeout-send-timeout} +## connect_timeout, receive_timeout, send_timeout {#connect-timeout-receive-timeout-send-timeout} وقفه در ثانیه بر روی سوکت مورد استفاده برای برقراری ارتباط با مشتری. مقدار پیش فرض: 10, 300, 300. -## \_انتقال به \_ار\_خروج {#cancel-http-readonly-queries-on-client-close} +## _انتقال به _ار_خروج {#cancel-http-readonly-queries-on-client-close} Cancels HTTP read-only queries (e.g. SELECT) when a client closes the connection without waiting for the response. مقدار پیشفرض: 0 -## پول\_نتروال {#poll-interval} +## پول_نتروال {#poll-interval} قفل در یک حلقه انتظار برای تعداد مشخصی از ثانیه. مقدار پیش فرض: 10. -## \_ادغام گیر {#max-distributed-connections} +## _ادغام گیر {#max-distributed-connections} حداکثر تعداد اتصالات همزمان با سرور از راه دور برای پردازش توزیع از یک پرس و جو تنها به یک جدول توزیع. ما توصیه می کنیم تنظیم یک مقدار کمتر از تعداد سرور در خوشه. @@ -690,7 +690,7 @@ Cancels HTTP read-only queries (e.g. SELECT) when a client closes the connectio مقدار پیش فرض: 1024. -## \_انتقال به \_مزاح\_اف\_کننده {#connect-timeout-with-failover-ms} +## _انتقال به _مزاح_اف_کننده {#connect-timeout-with-failover-ms} فاصله در میلی ثانیه برای اتصال به یک سرور از راه دور برای یک موتور جدول توزیع, اگر ‘shard’ و ‘replica’ بخش ها در تعریف خوشه استفاده می شود. اگر ناموفق, چندین تلاش برای اتصال به کپی های مختلف ساخته شده. @@ -711,30 +711,30 @@ Cancels HTTP read-only queries (e.g. SELECT) when a client closes the connectio ## همترازی پایین {#setting-use_uncompressed_cache} اینکه از یک کش از بلوکهای غیر فشرده استفاده شود یا خیر. می پذیرد 0 یا 1. به طور پیش فرض, 0 (غیر فعال). -با استفاده از کش غیر فشرده (فقط برای جداول در خانواده ادغام) می تواند به طور قابل توجهی کاهش زمان تاخیر و افزایش توان در هنگام کار با تعداد زیادی از نمایش داده شد کوتاه است. فعال کردن این تنظیم برای کاربرانی که ارسال درخواست کوتاه مکرر. همچنین با توجه به پرداخت [\_بالا](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuration parameter (only set in the config file) – the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed and the least-used data is automatically deleted. +با استفاده از کش غیر فشرده (فقط برای جداول در خانواده ادغام) می تواند به طور قابل توجهی کاهش زمان تاخیر و افزایش توان در هنگام کار با تعداد زیادی از نمایش داده شد کوتاه است. فعال کردن این تنظیم برای کاربرانی که ارسال درخواست کوتاه مکرر. همچنین با توجه به پرداخت [_بالا](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuration parameter (only set in the config file) – the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed and the least-used data is automatically deleted. -برای نمایش داده شد که خواندن حداقل حجم تا حدودی زیادی از داده ها (یک میلیون ردیف یا بیشتر) غیر فشرده کش غیر فعال است به طور خودکار به صرفه جویی در فضا برای واقعا کوچک نمایش داده شد. این به این معنی است که شما می توانید نگه دارید ‘use\_uncompressed\_cache’ تنظیم همیشه به مجموعه 1. +برای نمایش داده شد که خواندن حداقل حجم تا حدودی زیادی از داده ها (یک میلیون ردیف یا بیشتر) غیر فشرده کش غیر فعال است به طور خودکار به صرفه جویی در فضا برای واقعا کوچک نمایش داده شد. این به این معنی است که شما می توانید نگه دارید ‘use_uncompressed_cache’ تنظیم همیشه به مجموعه 1. -## جایگزینی \_خروج {#replace-running-query} +## جایگزینی _خروج {#replace-running-query} -هنگام استفاده از رابط قام ‘query\_id’ پارامتر را می توان گذشت. این هر رشته که به عنوان شناسه پرس و جو در خدمت است. -اگر پرس و جو از همان کاربر با همان ‘query\_id’ در حال حاضر در این زمان وجود دارد, رفتار بستگی به ‘replace\_running\_query’ پارامتر. +هنگام استفاده از رابط قام ‘query_id’ پارامتر را می توان گذشت. این هر رشته که به عنوان شناسه پرس و جو در خدمت است. +اگر پرس و جو از همان کاربر با همان ‘query_id’ در حال حاضر در این زمان وجود دارد, رفتار بستگی به ‘replace_running_query’ پارامتر. -`0` (default) – Throw an exception (don't allow the query to run if a query with the same ‘query\_id’ در حال حاضر در حال اجرا). +`0` (default) – Throw an exception (don't allow the query to run if a query with the same ‘query_id’ در حال حاضر در حال اجرا). `1` – Cancel the old query and start running the new one. یاندکسمتریکا با استفاده از این پارامتر را به 1 برای اجرای پیشنهادات خود را برای شرایط تقسیم بندی. پس از ورود به شخصیت بعدی, اگر پرس و جو قدیمی هنوز تمام نشده است, باید لغو شود. -## \_خاله جریان {#stream-flush-interval-ms} +## _خاله جریان {#stream-flush-interval-ms} -این نسخهها کار میکند برای جداول با جریان در مورد یک ایست, و یا زمانی که یک موضوع تولید [ا\_فزونهها](#settings-max_insert_block_size) ردیف +این نسخهها کار میکند برای جداول با جریان در مورد یک ایست, و یا زمانی که یک موضوع تولید [ا_فزونهها](#settings-max_insert_block_size) ردیف مقدار پیش فرض 7500 است. کوچکتر ارزش, اطلاعات بیشتر به جدول سرخ. تنظیم مقدار خیلی کم منجر به عملکرد ضعیف می شود. -## \_تبالسازی {#settings-load_balancing} +## _تبالسازی {#settings-load_balancing} تعیین الگوریتم انتخاب کپی است که برای پردازش پرس و جو توزیع استفاده. @@ -794,16 +794,16 @@ load_balancing = first_or_random مقادیر ممکن: - 1 — ClickHouse always sends a query to the localhost replica if it exists. -- 0 — ClickHouse uses the balancing strategy specified by the [\_تبالسازی](#settings-load_balancing) تنظیمات. +- 0 — ClickHouse uses the balancing strategy specified by the [_تبالسازی](#settings-load_balancing) تنظیمات. مقدار پیش فرض: 1. !!! warning "اخطار" - غیر فعال کردن این تنظیم در صورت استفاده [بیشینه\_راپرال\_راپیکال](#settings-max_parallel_replicas). + غیر فعال کردن این تنظیم در صورت استفاده [بیشینه_راپرال_راپیکال](#settings-max_parallel_replicas). -## کد \_ورود {#totals-mode} +## کد _ورود {#totals-mode} -چگونه برای محاسبه مجموع زمانی که نیاز است در حال حاضر به عنوان زمانی که max\_rows\_to\_group\_by و group\_by\_overflow\_mode = ‘any’ حضور دارند. +چگونه برای محاسبه مجموع زمانی که نیاز است در حال حاضر به عنوان زمانی که max_rows_to_group_by و group_by_overflow_mode = ‘any’ حضور دارند. بخش را ببینید “WITH TOTALS modifier”. ## در حال بارگذاری {#totals-auto-threshold} @@ -811,7 +811,7 @@ load_balancing = first_or_random نگهبان `totals_mode = 'auto'`. بخش را ببینید “WITH TOTALS modifier”. -## بیشینه\_راپرال\_راپیکال {#settings-max_parallel_replicas} +## بیشینه_راپرال_راپیکال {#settings-max_parallel_replicas} حداکثر تعداد کپی برای هر سفال در هنگام اجرای یک پرس و جو. برای سازگاری (برای دریافت بخش های مختلف از تقسیم داده های مشابه), این گزینه تنها کار می کند زمانی که کلید نمونه گیری قرار است. @@ -824,7 +824,7 @@ load_balancing = first_or_random تدوین تنها برای بخشی از خط لوله پرس و جو پردازش استفاده می شود: برای مرحله اول تجمع (گروه های). اگر این بخش از خط لوله وارد شده بود, پرس و جو ممکن است سریع تر با توجه به استقرار چرخه های کوتاه و محدود تماس تابع جمع اجرا. حداکثر بهبود عملکرد (تا چهار برابر سریعتر در موارد نادر) برای نمایش داده شد با توابع چند دانه ساده دیده می شود. معمولا افزایش عملکرد ناچیز است. در موارد بسیار نادر, ممکن است کم کردن سرعت اجرای پرس و جو. -## \_کوچکنمایی {#min-count-to-compile} +## _کوچکنمایی {#min-count-to-compile} چند بار به طور بالقوه استفاده از یک تکه وارد شده از کد قبل از در حال اجرا تلفیقی. به طور پیش فرض, 3. For testing, the value can be set to 0: compilation runs synchronously and the query waits for the end of the compilation process before continuing execution. For all other cases, use values ​​starting with 1. Compilation normally takes about 5-10 seconds. @@ -833,27 +833,27 @@ For testing, the value can be set to 0: compilation runs synchronously and the q کد کامپایل شده برای هر ترکیب های مختلف از توابع کل مورد استفاده در پرس و جو و نوع کلید در گروه بند مورد نیاز است. The results of the compilation are saved in the build directory in the form of .so files. There is no restriction on the number of compilation results since they don't use very much space. Old results will be used after server restarts, except in the case of a server upgrade – in this case, the old results are deleted. -## خروجی \_فرمان\_جسون\_کوات\_64بیت\_تنظیمی {#session_settings-output_format_json_quote_64bit_integers} +## خروجی _فرمان_جسون_کوات_64بیت_تنظیمی {#session_settings-output_format_json_quote_64bit_integers} اگر مقدار درست است صحیح به نظر می رسد در نقل قول ها در هنگام استفاده از JSON\* Int64 و UInt64 فرمت (برای سازگاری بیشتر با جاوا اسکریپت پیاده سازی); در غیر این صورت اعداد صحیح هستند و خروجی بدون نقل قول. -## \_مخفی کردن \_قابلیت \_جدید {#settings-format_csv_delimiter} +## _مخفی کردن _قابلیت _جدید {#settings-format_csv_delimiter} شخصیت به عنوان یک جداساز در داده های سی سی. وی تفسیر شده است. به طور پیش فرض, جداساز است `,`. -## \_فرستادن به \_کوچکنمایی {#settings-input_format_csv_unquoted_null_literal_as_null} +## _فرستادن به _کوچکنمایی {#settings-input_format_csv_unquoted_null_literal_as_null} برای فرمت ورودی سی اس وی را قادر می سازد و یا غیر فعال تجزیه بدون نقل `NULL` به عنوان تحت اللفظی (مترادف برای `\N`). -## \_انتقال به \_شروع مجدد {#settings-output-format-csv-crlf-end-of-line} +## _انتقال به _شروع مجدد {#settings-output-format-csv-crlf-end-of-line} استفاده از داس/ویندوز-سبک خط جدا کننده (CRLF) در CSV به جای یونیکس سبک (LF). -## \_فرستادن در\_م\_مایش از \_برخط {#settings-output-format-tsv-crlf-end-of-line} +## _فرستادن در_م_مایش از _برخط {#settings-output-format-tsv-crlf-end-of-line} استفاده از توضیحات / ویندوز به سبک خط جدا (سازمان تنظیم مقررات) در واحد پشتیبانی فنی فنی فنی مهندسی به جای سبک یونیکس. -## \_معامله {#settings-insert_quorum} +## _معامله {#settings-insert_quorum} را قادر می سازد حد نصاب می نویسد. @@ -877,10 +877,10 @@ The results of the compilation are saved in the build directory in the form of . همچنین نگاه کنید به: -- [\_بههنگامسازی](#settings-insert_quorum_timeout) +- [_بههنگامسازی](#settings-insert_quorum_timeout) - [مورد احترام](#settings-select_sequential_consistency) -## \_بههنگامسازی {#settings-insert_quorum_timeout} +## _بههنگامسازی {#settings-insert_quorum_timeout} ارسال به فاصله حد نصاب در ثانیه. اگر ایست را تصویب کرده است و بدون نوشتن صورت گرفته است و در عین حال, تاتر یک استثنا تولید و مشتری باید پرس و جو تکرار برای نوشتن همان بلوک به همان و یا هر ماکت دیگر. @@ -888,7 +888,7 @@ The results of the compilation are saved in the build directory in the form of . همچنین نگاه کنید به: -- [\_معامله](#settings-insert_quorum) +- [_معامله](#settings-insert_quorum) - [مورد احترام](#settings-select_sequential_consistency) ## مورد احترام {#settings-select_sequential_consistency} @@ -908,10 +908,10 @@ The results of the compilation are saved in the build directory in the form of . همچنین نگاه کنید به: -- [\_معامله](#settings-insert_quorum) -- [\_بههنگامسازی](#settings-insert_quorum_timeout) +- [_معامله](#settings-insert_quorum) +- [_بههنگامسازی](#settings-insert_quorum_timeout) -## \_تنظیم مجدد به حالت اولیه {#settings-insert-deduplicate} +## _تنظیم مجدد به حالت اولیه {#settings-insert-deduplicate} امکان حذف یا غیرفعال کردن مسدود کردن تقسیم بندی `INSERT` (برای تکرار\* جداول). @@ -942,7 +942,7 @@ The results of the compilation are saved in the build directory in the form of . همزمان, این رفتار “breaks” `INSERT` حق تقدم. اگر یک `INSERT` به جدول اصلی موفق بود و `INSERT` into a materialized view failed (e.g. because of communication failure with Zookeeper) a client will get an error and can retry the operation. However, the materialized view won't receive the second insert because it will be discarded by deduplication in the main (source) table. The setting `deduplicate_blocks_in_dependent_materialized_views` اجازه می دهد تا برای تغییر این رفتار. در تلاش مجدد, یک دیدگاه محقق درج تکرار دریافت خواهد کرد و بررسی تکرار به خودی خود انجام, نادیده گرفتن نتیجه چک برای جدول منبع, و ردیف به دلیل شکست اول از دست داده وارد. -## ویژ\_گیها {#settings-max-network-bytes} +## ویژ_گیها {#settings-max-network-bytes} محدودیت حجم داده ها (به بایت) است که دریافت و یا انتقال بر روی شبکه در هنگام اجرای یک پرس و جو. این تنظیم در مورد هر پرس و جو فردی. @@ -953,7 +953,7 @@ The results of the compilation are saved in the build directory in the form of . مقدار پیش فرض: 0. -## \_عرض {#settings-max-network-bandwidth} +## _عرض {#settings-max-network-bandwidth} محدودیت سرعت تبادل داده ها بر روی شبکه در بایت در هر ثانیه. این تنظیم در مورد هر پرس و جو. @@ -964,7 +964,7 @@ The results of the compilation are saved in the build directory in the form of . مقدار پیش فرض: 0. -## \_شمارهگیر بیشینه {#settings-max-network-bandwidth-for-user} +## _شمارهگیر بیشینه {#settings-max-network-bandwidth-for-user} محدودیت سرعت تبادل داده ها بر روی شبکه در بایت در هر ثانیه. این تنظیم به تمام نمایش داده شد همزمان در حال اجرا انجام شده توسط یک کاربر اعمال می شود. @@ -975,7 +975,7 @@ The results of the compilation are saved in the build directory in the form of . مقدار پیش فرض: 0. -## \_شمارهگیرها {#settings-max-network-bandwidth-for-all-users} +## _شمارهگیرها {#settings-max-network-bandwidth-for-all-users} محدودیت سرعت است که داده ها در بیش از شبکه در بایت در هر ثانیه رد و بدل. این تنظیم در مورد تمام نمایش داده شد به صورت همزمان در حال اجرا بر روی سرور. @@ -986,7 +986,7 @@ The results of the compilation are saved in the build directory in the form of . مقدار پیش فرض: 0. -## ا\_فزونهها {#settings-count_distinct_implementation} +## ا_فزونهها {#settings-count_distinct_implementation} مشخص می کند که کدام یک از `uniq*` توابع باید برای انجام [COUNT(DISTINCT …)](../../sql-reference/aggregate-functions/reference.md#agg_function-count) ساخت و ساز. @@ -1048,7 +1048,7 @@ The results of the compilation are saved in the build directory in the form of . مقدار پیشفرض: 0 -## ا\_فزون\_ف\_کوپ {#setting-optimize_throw_if_noop} +## ا_فزون_ف_کوپ {#setting-optimize_throw_if_noop} را قادر می سازد و یا غیر فعال پرتاب یک استثنا اگر یک [OPTIMIZE](../../sql-reference/statements/misc.md#misc_operations-optimize) پرس و جو یک ادغام انجام نمی. @@ -1066,7 +1066,7 @@ The results of the compilation are saved in the build directory in the form of . - نوع: ثانیه - مقدار پیشفرض: 60 ثانیه -کنترل خطاهای چگونه سریع در جداول توزیع صفر. اگر یک ماکت برای برخی از زمان در دسترس نیست, تجمع می یابد 5 اشتباهات, و توزیع \_راپیرار\_لفا\_لایف تنظیم شده است 1 دوم, سپس ماکت در نظر گرفته شده است طبیعی 3 ثانیه پس از خطا گذشته. +کنترل خطاهای چگونه سریع در جداول توزیع صفر. اگر یک ماکت برای برخی از زمان در دسترس نیست, تجمع می یابد 5 اشتباهات, و توزیع _راپیرار_لفا_لایف تنظیم شده است 1 دوم, سپس ماکت در نظر گرفته شده است طبیعی 3 ثانیه پس از خطا گذشته. همچنین نگاه کنید به: @@ -1118,7 +1118,7 @@ The results of the compilation are saved in the build directory in the form of . مقدار پیش فرض: 0. -## \_شخصیت {#setting-os-thread-priority} +## _شخصیت {#setting-os-thread-priority} اولویت را تنظیم می کند ([خوبه](https://en.wikipedia.org/wiki/Nice_(Unix))) برای موضوعات که نمایش داده شد را اجرا کند . زمانبندی سیستم عامل این اولویت در نظر هنگام انتخاب موضوع بعدی به اجرا در هر هسته پردازنده در دسترس است. @@ -1154,7 +1154,7 @@ The results of the compilation are saved in the build directory in the form of . همچنین نگاه کنید به: -- جدول سیستم [\_قطع](../../operations/system-tables.md#system_tables-trace_log) +- جدول سیستم [_قطع](../../operations/system-tables.md#system_tables-trace_log) ## ایران در تهران {#query_profiler_cpu_time_period_ns} @@ -1177,9 +1177,9 @@ The results of the compilation are saved in the build directory in the form of . همچنین نگاه کنید به: -- جدول سیستم [\_قطع](../../operations/system-tables.md#system_tables-trace_log) +- جدول سیستم [_قطع](../../operations/system-tables.md#system_tables-trace_log) -## اجازه دادن به \_فعال کردن اختلال در عملکرد {#settings-allow_introspection_functions} +## اجازه دادن به _فعال کردن اختلال در عملکرد {#settings-allow_introspection_functions} فعالسازی از کارانداختن [توابع درون گونه](../../sql-reference/functions/introspection.md) برای پروفایل پرس و جو. @@ -1193,23 +1193,23 @@ The results of the compilation are saved in the build directory in the form of . **همچنین نگاه کنید به** - [پروفایل پرس و جو نمونه برداری](../optimizing-performance/sampling-query-profiler.md) -- جدول سیستم [\_قطع](../../operations/system-tables.md#system_tables-trace_log) +- جدول سیستم [_قطع](../../operations/system-tables.md#system_tables-trace_log) -## وارد\_فرمت\_پارلل\_درپارس {#input-format-parallel-parsing} +## وارد_فرمت_پارلل_درپارس {#input-format-parallel-parsing} - نوع: بولی - مقدار پیشفرض: درست فعال کردن نظم حفظ تجزیه موازی از فرمت های داده. پشتیبانی تنها برای TSV TKSV CSV و JSONEachRow فرمت های. -## \_حداقل کردن \_بیتس\_برای\_پرال\_درپارس {#min-chunk-bytes-for-parallel-parsing} +## _حداقل کردن _بیتس_برای_پرال_درپارس {#min-chunk-bytes-for-parallel-parsing} - نوع: امضا نشده - مقدار پیشفرض: 1 مگابایت حداقل اندازه تکه در بایت, که هر موضوع به صورت موازی تجزیه خواهد شد. -## \_فرماندگی لبه بام {#settings-output_format_avro_codec} +## _فرماندگی لبه بام {#settings-output_format_avro_codec} مجموعه کدک فشرده سازی مورد استفاده برای خروجی فایل اورو. @@ -1223,7 +1223,7 @@ The results of the compilation are saved in the build directory in the form of . مقدار پیشفرض: `snappy` (در صورت موجود بودن) یا `deflate`. -## \_فرماندگی لبه چشم {#settings-output_format_avro_sync_interval} +## _فرماندگی لبه چشم {#settings-output_format_avro_sync_interval} مجموعه حداقل اندازه داده (در بایت) بین نشانگر هماهنگ سازی برای فایل خروجی هواپیما. @@ -1233,7 +1233,7 @@ The results of the compilation are saved in the build directory in the form of . مقدار پیش فرض: 32768 (32 کیلوبایت) -## باز کردن \_نمایش مجدد {#settings-format_avro_schema_registry_url} +## باز کردن _نمایش مجدد {#settings-format_avro_schema_registry_url} نشانی اینترنتی رجیستری طرحواره را برای استفاده تنظیم میکند [هشدار داده می شود](../../interfaces/formats.md#data-format-avro-confluent) قالب diff --git a/docs/fa/operations/system-tables.md b/docs/fa/operations/system-tables.md index 39e90e4130f..d634e6dc541 100644 --- a/docs/fa/operations/system-tables.md +++ b/docs/fa/operations/system-tables.md @@ -13,7 +13,7 @@ toc_title: "\u062C\u062F\u0627\u0648\u0644 \u0633\u06CC\u0633\u062A\u0645" جداول سیستم فقط خواندنی. این در واقع ‘system’ بانک اطلاعات. -## سیستم.\_نامهنویسی ناهمزمان {#system_tables-asynchronous_metrics} +## سیستم._نامهنویسی ناهمزمان {#system_tables-asynchronous_metrics} شامل معیارهای که به صورت دوره ای در پس زمینه محاسبه می شود. مثلا, مقدار رم در حال استفاده. @@ -48,7 +48,7 @@ SELECT * FROM system.asynchronous_metrics LIMIT 10 - [نظارت](monitoring.md) — Base concepts of ClickHouse monitoring. - [سیستم.متریک](#system_tables-metrics) — Contains instantly calculated metrics. - [سیستم.رویدادها](#system_tables-events) — Contains a number of events that have occurred. -- [سیستم.\_اشکالزدایی](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [سیستم._اشکالزدایی](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. ## سیستم.خوشه {#system-clusters} @@ -72,8 +72,8 @@ SELECT * FROM system.asynchronous_metrics LIMIT 10 **همچنین نگاه کنید به** - [موتور جدول توزیع شده است](../engines/table-engines/special/distributed.md) -- [تنظیمات \_فرهنگ توزیع میشود](settings/settings.md#settings-distributed_replica_error_cap) -- [پخش \_راپیشا\_را\_را\_را\_حالف\_لایف تنظیم](settings/settings.md#settings-distributed_replica_error_half_life) +- [تنظیمات _فرهنگ توزیع میشود](settings/settings.md#settings-distributed_replica_error_cap) +- [پخش _راپیشا_را_را_را_حالف_لایف تنظیم](settings/settings.md#settings-distributed_replica_error_half_life) ## سیستم.ستونها {#system-columns} @@ -239,9 +239,9 @@ SELECT * FROM system.events LIMIT 5 **همچنین نگاه کنید به** -- [سیستم.\_نامهنویسی ناهمزمان](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [سیستم._نامهنویسی ناهمزمان](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. - [سیستم.متریک](#system_tables-metrics) — Contains instantly calculated metrics. -- [سیستم.\_اشکالزدایی](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [سیستم._اشکالزدایی](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. - [نظارت](monitoring.md) — Base concepts of ClickHouse monitoring. ## سیستم.توابع {#system-functions} @@ -255,7 +255,7 @@ SELECT * FROM system.events LIMIT 5 ## سیستم.بازداشت گرافیت {#system-graphite-retentions} -حاوی اطلاعات در مورد پارامترها [لغزش \_ نمودار](server-configuration-parameters/settings.md#server_configuration_parameters-graphite) که در جداول با استفاده [اطلاعات دقیق](../engines/table-engines/mergetree-family/graphitemergetree.md) موتورها. +حاوی اطلاعات در مورد پارامترها [لغزش _ نمودار](server-configuration-parameters/settings.md#server_configuration_parameters-graphite) که در جداول با استفاده [اطلاعات دقیق](../engines/table-engines/mergetree-family/graphitemergetree.md) موتورها. ستونها: @@ -324,12 +324,12 @@ SELECT * FROM system.metrics LIMIT 10 **همچنین نگاه کنید به** -- [سیستم.\_نامهنویسی ناهمزمان](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [سیستم._نامهنویسی ناهمزمان](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. - [سیستم.رویدادها](#system_tables-events) — Contains a number of events that occurred. -- [سیستم.\_اشکالزدایی](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [سیستم._اشکالزدایی](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. - [نظارت](monitoring.md) — Base concepts of ClickHouse monitoring. -## سیستم.\_اشکالزدایی {#system_tables-metric_log} +## سیستم._اشکالزدایی {#system_tables-metric_log} دارای تاریخچه معیارهای ارزش از جداول `system.metrics` و `system.events`, دوره ای به دیسک سرخ. برای روشن کردن مجموعه تاریخچه معیارهای در `system.metric_log` ایجاد `/etc/clickhouse-server/config.d/metric_log.xml` با محتوای زیر: @@ -380,7 +380,7 @@ CurrentMetric_ReplicatedChecks: 0 **همچنین نگاه کنید به** -- [سیستم.\_نامهنویسی ناهمزمان](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [سیستم._نامهنویسی ناهمزمان](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. - [سیستم.رویدادها](#system_tables-events) — Contains a number of events that occurred. - [سیستم.متریک](#system_tables-metrics) — Contains instantly calculated metrics. - [نظارت](monitoring.md) — Base concepts of ClickHouse monitoring. @@ -391,7 +391,7 @@ CurrentMetric_ReplicatedChecks: 0 شما می توانید این جدول برای تست استفاده, و یا اگر شما نیاز به انجام یک جستجو نیروی بی رحم. بار خوانده شده از این جدول موازی نیست. -## سیستم.\_شماره حساب {#system-numbers-mt} +## سیستم._شماره حساب {#system-numbers-mt} همان ‘system.numbers’ اما بار خوانده شده موازی هستند. اعداد را می توان در هر سفارش بازگشت. مورد استفاده برای تست. @@ -483,9 +483,9 @@ CurrentMetric_ReplicatedChecks: 0 - `marks_size` (`UInt64`) – Alias for `marks_bytes`. -## سیستم.\_خروج {#system_tables-part-log} +## سیستم._خروج {#system_tables-part-log} -این `system.part_log` جدول تنها در صورتی ایجاد می شود [\_خروج](server-configuration-parameters/settings.md#server_configuration_parameters-part-log) تنظیم سرور مشخص شده است. +این `system.part_log` جدول تنها در صورتی ایجاد می شود [_خروج](server-configuration-parameters/settings.md#server_configuration_parameters-part-log) تنظیم سرور مشخص شده است. این جدول حاوی اطلاعات در مورد اتفاقاتی که با رخ داده است [قطعات داده](../engines/table-engines/mergetree-family/custom-partitioning-key.md) در [ادغام](../engines/table-engines/mergetree-family/mergetree.md) جداول خانواده, مانند اضافه کردن و یا ادغام داده ها. @@ -528,11 +528,11 @@ CurrentMetric_ReplicatedChecks: 0 - `rows_read` (UInt64) – The number of rows read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. - `bytes_read` (UInt64) – The number of uncompressed bytes read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. - `total_rows_approx` (UInt64) – The approximation of the total number of rows that should be read. For distributed processing, on the requestor server, this is the total for all remote servers. It can be updated during request processing, when new sources to process become known. -- `memory_usage` (UInt64) – Amount of RAM the request uses. It might not include some types of dedicated memory. See the [\_کاساژ بیشینه](../operations/settings/query-complexity.md#settings_max_memory_usage) تنظیمات. +- `memory_usage` (UInt64) – Amount of RAM the request uses. It might not include some types of dedicated memory. See the [_کاساژ بیشینه](../operations/settings/query-complexity.md#settings_max_memory_usage) تنظیمات. - `query` (String) – The query text. For `INSERT` این شامل داده ها برای وارد کردن نیست. - `query_id` (String) – Query ID, if defined. -## سیستم.\_خروج {#system_tables-text_log} +## سیستم._خروج {#system_tables-text_log} شامل ورودی ورود به سیستم. سطح ورود به سیستم که می رود به این جدول را می توان با محدود `text_log.level` تنظیم سرور. @@ -559,16 +559,16 @@ CurrentMetric_ReplicatedChecks: 0 - `source_file` (`LowCardinality(String)`)- فایل منبع که از ورود به سیستم انجام شد . - `source_line` (`UInt64`)- خط منبع که از ورود به سیستم انجام شد. -## سیستم.\_خروج {#system_tables-query_log} +## سیستم._خروج {#system_tables-query_log} حاوی اطلاعات در مورد اجرای نمایش داده شد. برای هر پرس و جو, شما می توانید زمان شروع پردازش را ببینید, مدت زمان پردازش, پیام های خطا و اطلاعات دیگر. !!! note "یادداشت" جدول حاوی اطلاعات ورودی برای `INSERT` نمایش داده شد. -تاتر این جدول را فقط در صورتی ایجاد می کند [\_خروج](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) پارامتر سرور مشخص شده است. این پارامتر مجموعه قوانین ورود به سیستم, مانند فاصله ورود به سیستم و یا نام جدول نمایش داده شد خواهد شد وارد سایت شوید. +تاتر این جدول را فقط در صورتی ایجاد می کند [_خروج](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) پارامتر سرور مشخص شده است. این پارامتر مجموعه قوانین ورود به سیستم, مانند فاصله ورود به سیستم و یا نام جدول نمایش داده شد خواهد شد وارد سایت شوید. -برای فعال کردن ورود به سیستم پرس و جو, تنظیم [\_خروج](settings/settings.md#settings-log-queries) پارامتر به 1. برای اطلاعات بیشتر [تنظیمات](settings/settings.md) بخش. +برای فعال کردن ورود به سیستم پرس و جو, تنظیم [_خروج](settings/settings.md#settings-log-queries) پارامتر به 1. برای اطلاعات بیشتر [تنظیمات](settings/settings.md) بخش. این `system.query_log` جدول ثبت دو نوع نمایش داده شد: @@ -636,20 +636,20 @@ CurrentMetric_ReplicatedChecks: 0 2. اگر یک خطا در طول پردازش پرس و جو رخ داده است, دو رویداد با انواع 1 و 4 ایجاد می شوند. 3. اگر یک خطا قبل از راه اندازی پرس و جو رخ داده است, یک رویداد واحد با نوع 3 ایجاد شده است. -به طور پیش فرض, سیاهههای مربوط به جدول در فواصل 7.5 ثانیه اضافه. شما می توانید این فاصله در مجموعه [\_خروج](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) تنظیم سرور (نگاه کنید به `flush_interval_milliseconds` پارامتر). به خیط و پیت کردن سیاهههای مربوط به زور از بافر حافظه را به جدول, استفاده از `SYSTEM FLUSH LOGS` پرس و جو. +به طور پیش فرض, سیاهههای مربوط به جدول در فواصل 7.5 ثانیه اضافه. شما می توانید این فاصله در مجموعه [_خروج](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) تنظیم سرور (نگاه کنید به `flush_interval_milliseconds` پارامتر). به خیط و پیت کردن سیاهههای مربوط به زور از بافر حافظه را به جدول, استفاده از `SYSTEM FLUSH LOGS` پرس و جو. هنگامی که جدول به صورت دستی حذف, به طور خودکار در پرواز ایجاد. توجه داشته باشید که تمام سیاهههای مربوط قبلی حذف خواهد شد. !!! note "یادداشت" دوره ذخیره سازی برای سیاهههای مربوط نامحدود است. سیاهههای مربوط به طور خودکار از جدول حذف نمی شود. شما نیاز به سازماندهی حذف سیاهههای مربوط منسوخ شده خود را. -شما می توانید یک کلید پارتیشن بندی دلخواه برای مشخص `system.query_log` جدول در [\_خروج](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) تنظیم سرور (نگاه کنید به `partition_by` پارامتر). +شما می توانید یک کلید پارتیشن بندی دلخواه برای مشخص `system.query_log` جدول در [_خروج](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) تنظیم سرور (نگاه کنید به `partition_by` پارامتر). -## سیستم.\_ر\_خروج {#system_tables-query-thread-log} +## سیستم._ر_خروج {#system_tables-query-thread-log} جدول شامل اطلاعات در مورد هر موضوع اجرای پرس و جو. -تاتر این جدول را فقط در صورتی ایجاد می کند [\_ر\_خروج](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) پارامتر سرور مشخص شده است. این پارامتر مجموعه قوانین ورود به سیستم, مانند فاصله ورود به سیستم و یا نام جدول نمایش داده شد خواهد شد وارد سایت شوید. +تاتر این جدول را فقط در صورتی ایجاد می کند [_ر_خروج](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) پارامتر سرور مشخص شده است. این پارامتر مجموعه قوانین ورود به سیستم, مانند فاصله ورود به سیستم و یا نام جدول نمایش داده شد خواهد شد وارد سایت شوید. برای فعال کردن ورود به سیستم پرس و جو, تنظیم [باز کردن](settings/settings.md#settings-log-query-threads) پارامتر به 1. برای اطلاعات بیشتر [تنظیمات](settings/settings.md) بخش. @@ -701,20 +701,20 @@ CurrentMetric_ReplicatedChecks: 0 - `ProfileEvents.Names` (Array(String)) — Counters that measure different metrics for this thread. The description of them could be found in the table [سیستم.رویدادها](#system_tables-events) - `ProfileEvents.Values` (Array(UInt64)) — Values of metrics for this thread that are listed in the `ProfileEvents.Names` ستون. -به طور پیش فرض, سیاهههای مربوط به جدول در فواصل 7.5 ثانیه اضافه. شما می توانید این فاصله در مجموعه [\_ر\_خروج](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) تنظیم سرور (نگاه کنید به `flush_interval_milliseconds` پارامتر). به خیط و پیت کردن سیاهههای مربوط به زور از بافر حافظه را به جدول, استفاده از `SYSTEM FLUSH LOGS` پرس و جو. +به طور پیش فرض, سیاهههای مربوط به جدول در فواصل 7.5 ثانیه اضافه. شما می توانید این فاصله در مجموعه [_ر_خروج](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) تنظیم سرور (نگاه کنید به `flush_interval_milliseconds` پارامتر). به خیط و پیت کردن سیاهههای مربوط به زور از بافر حافظه را به جدول, استفاده از `SYSTEM FLUSH LOGS` پرس و جو. هنگامی که جدول به صورت دستی حذف, به طور خودکار در پرواز ایجاد. توجه داشته باشید که تمام سیاهههای مربوط قبلی حذف خواهد شد. !!! note "یادداشت" دوره ذخیره سازی برای سیاهههای مربوط نامحدود است. سیاهههای مربوط به طور خودکار از جدول حذف نمی شود. شما نیاز به سازماندهی حذف سیاهههای مربوط منسوخ شده خود را. -شما می توانید یک کلید پارتیشن بندی دلخواه برای مشخص `system.query_thread_log` جدول در [\_ر\_خروج](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) تنظیم سرور (نگاه کنید به `partition_by` پارامتر). +شما می توانید یک کلید پارتیشن بندی دلخواه برای مشخص `system.query_thread_log` جدول در [_ر_خروج](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) تنظیم سرور (نگاه کنید به `partition_by` پارامتر). -## سیستم.\_قطع {#system_tables-trace_log} +## سیستم._قطع {#system_tables-trace_log} حاوی ردیاب های پشته ای است که توسط پروفایل پرس و جو نمونه گیری می شود. -تاتر این جدول زمانی ایجاد می کند [\_قطع](server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) بخش پیکربندی سرور تنظیم شده است. همچنین [جستجو](settings/settings.md#query_profiler_real_time_period_ns) و [ایران در تهران](settings/settings.md#query_profiler_cpu_time_period_ns) تنظیمات باید تنظیم شود. +تاتر این جدول زمانی ایجاد می کند [_قطع](server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) بخش پیکربندی سرور تنظیم شده است. همچنین [جستجو](settings/settings.md#query_profiler_real_time_period_ns) و [ایران در تهران](settings/settings.md#query_profiler_cpu_time_period_ns) تنظیمات باید تنظیم شود. برای تجزیه و تحلیل سیاهههای مربوط, استفاده از `addressToLine`, `addressToSymbol` و `demangle` توابع درون گرایی. @@ -737,7 +737,7 @@ CurrentMetric_ReplicatedChecks: 0 - `thread_number` ([UInt32](../sql-reference/data-types/int-uint.md)) — Thread identifier. -- `query_id` ([رشته](../sql-reference/data-types/string.md)) — Query identifier that can be used to get details about a query that was running from the [\_خروج](#system_tables-query_log) جدول سیستم. +- `query_id` ([رشته](../sql-reference/data-types/string.md)) — Query identifier that can be used to get details about a query that was running from the [_خروج](#system_tables-query_log) جدول سیستم. - `trace` ([Array(UInt64)](../sql-reference/data-types/array.md)) — Stack trace at the moment of sampling. Each element is a virtual memory address inside ClickHouse server process. @@ -824,7 +824,7 @@ active_replicas: 2 - `parts_to_check` (`UInt32`)- تعداد قطعات داده در صف برای تایید. اگر شک وجود دارد که ممکن است صدمه دیده است بخشی در صف تایید قرار داده است. - `zookeeper_path` (`String`)- مسیر به داده های جدول در باغ وحش. - `replica_name` (`String`)- نام ماکت در باغ وحش. کپی های مختلف از همان جدول نام های مختلف. -- `replica_path` (`String`)- مسیر به داده های ماکت در باغ وحش. همان الحاق ‘zookeeper\_path/replicas/replica\_path’. +- `replica_path` (`String`)- مسیر به داده های ماکت در باغ وحش. همان الحاق ‘zookeeper_path/replicas/replica_path’. - `columns_version` (`Int32`)- تعداد نسخه از ساختار جدول . نشان می دهد که چند بار تغییر انجام شد. اگر کپی نسخه های مختلف, به این معنی برخی از کپی ساخته شده است همه از تغییر نکرده است. - `queue_size` (`UInt32`)- اندازه صف برای عملیات در حال انتظار برای انجام شود . عملیات شامل قرار دادن بلوک های داده ادغام و برخی اقدامات دیگر. معمولا همزمان با `future_parts`. - `inserts_in_queue` (`UInt32`)- تعداد درج بلوک از داده ها که نیاز به ساخته شده است . درج معمولا نسبتا به سرعت تکرار. اگر این تعداد بزرگ است, به این معنی چیزی اشتباه است. @@ -845,7 +845,7 @@ active_replicas: 2 - `active_replicas` (`UInt8`)- تعداد کپی از این جدول که یک جلسه در باغ وحش (یعنی تعداد تکرار عملکرد). اگر شما درخواست تمام ستون, جدول ممکن است کمی کند کار, از چند بار خوانده شده از باغ وحش برای هر سطر ساخته شده. -اگر شما درخواست آخرین 4 ستون (log\_max\_index, log\_pointer, total\_replicas, active\_replicas) جدول با این نسخهها کار به سرعت. +اگر شما درخواست آخرین 4 ستون (log_max_index, log_pointer, total_replicas, active_replicas) جدول با این نسخهها کار به سرعت. مثلا, شما می توانید بررسی کنید که همه چیز به درستی کار مثل این: @@ -932,7 +932,7 @@ SELECT * FROM system.settings WHERE changed AND name='load_balancing' - [مجوز برای نمایش داده شد](settings/permissions-for-queries.md#settings_readonly) - [محدودیت در تنظیمات](settings/constraints-on-settings.md) -## سیستم.\_زبانهها {#system.table_engines} +## سیستم._زبانهها {#system.table_engines} ``` text ┌─name───────────────────┬─value───────┐ @@ -955,7 +955,7 @@ SELECT * FROM system.settings WHERE changed AND name='load_balancing' - `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} شامل شرح موتورهای جدول پشتیبانی شده توسط سرور و اطلاعات پشتیبانی از ویژگی های خود را. @@ -1122,21 +1122,21 @@ path: /clickhouse/tables/01-08/visits/replicas **فرمان** - رشته فرمان جهش (بخشی از پرس و جو پس از `ALTER TABLE [db.]table`). -**\_بروزرسانی** - هنگامی که این دستور جهش برای اجرای ارسال شد . +**_بروزرسانی** - هنگامی که این دستور جهش برای اجرای ارسال شد . -**\_شمارهی بلوک.ا\_ضافه کردن**, **\_شمارهی بلوک.شماره** - ستون تو در تو . برای جهش از جداول تکرار, این شامل یک رکورد برای هر پارتیشن: شناسه پارتیشن و شماره بلوک که توسط جهش خریداری شد (در هر پارتیشن, تنها بخش هایی که حاوی بلوک با اعداد کمتر از تعداد بلوک های خریداری شده توسط جهش در پارتیشن که جهش خواهد شد). در جداول غیر تکرار, تعداد بلوک در تمام پارتیشن به صورت یک توالی واحد. این به این معنی است که برای جهش از جداول غیر تکرار, ستون یک رکورد با یک عدد بلوک واحد خریداری شده توسط جهش شامل. +**_شمارهی بلوک.ا_ضافه کردن**, **_شمارهی بلوک.شماره** - ستون تو در تو . برای جهش از جداول تکرار, این شامل یک رکورد برای هر پارتیشن: شناسه پارتیشن و شماره بلوک که توسط جهش خریداری شد (در هر پارتیشن, تنها بخش هایی که حاوی بلوک با اعداد کمتر از تعداد بلوک های خریداری شده توسط جهش در پارتیشن که جهش خواهد شد). در جداول غیر تکرار, تعداد بلوک در تمام پارتیشن به صورت یک توالی واحد. این به این معنی است که برای جهش از جداول غیر تکرار, ستون یک رکورد با یک عدد بلوک واحد خریداری شده توسط جهش شامل. -**\_کوچکنمایی** - تعدادی از قطعات داده است که نیاز به جهش را به پایان برساند جهش یافته است . +**_کوچکنمایی** - تعدادی از قطعات داده است که نیاز به جهش را به پایان برساند جهش یافته است . -**\_مخفی کردن** - توجه داشته باشید که حتی اگر `parts_to_do = 0` ممکن است که جهش جدول تکرار هنوز به دلیل درج طولانی در حال اجرا است که ایجاد بخش داده های جدید است که نیاز به جهش انجام می شود است. +**_مخفی کردن** - توجه داشته باشید که حتی اگر `parts_to_do = 0` ممکن است که جهش جدول تکرار هنوز به دلیل درج طولانی در حال اجرا است که ایجاد بخش داده های جدید است که نیاز به جهش انجام می شود است. اگر مشکلی با جهش برخی از قطعات وجود دارد, ستون های زیر حاوی اطلاعات اضافی: -**\_شروع مجدد** - نام جدید ترین بخش است که نمی تواند جهش یافته است. +**_شروع مجدد** - نام جدید ترین بخش است که نمی تواند جهش یافته است. -**زمان \_رشته** - زمان جدید ترین شکست جهش بخشی . +**زمان _رشته** - زمان جدید ترین شکست جهش بخشی . -**\_شروع مجدد** - پیام استثنا که باعث شکست جهش بخشی اخیر. +**_شروع مجدد** - پیام استثنا که باعث شکست جهش بخشی اخیر. ## سیستم.دیسکها {#system_tables-disks} @@ -1150,7 +1150,7 @@ path: /clickhouse/tables/01-08/visits/replicas - `total_space` ([UInt64](../sql-reference/data-types/int-uint.md)) — Disk volume in bytes. - `keep_free_space` ([UInt64](../sql-reference/data-types/int-uint.md)) — Amount of disk space that should stay free on disk in bytes. Defined in the `keep_free_space_bytes` پارامتر پیکربندی دیسک. -## سیستم.داستان\_یابی {#system_tables-storage_policies} +## سیستم.داستان_یابی {#system_tables-storage_policies} حاوی اطلاعات در مورد سیاست های ذخیره سازی و حجم تعریف شده در [پیکربندی کارساز](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes_configure). diff --git a/docs/fa/operations/troubleshooting.md b/docs/fa/operations/troubleshooting.md index cc456bce674..f2dc276dc89 100644 --- a/docs/fa/operations/troubleshooting.md +++ b/docs/fa/operations/troubleshooting.md @@ -105,7 +105,7 @@ $ sudo -u clickhouse /usr/bin/clickhouse-server --config-file /etc/clickhouse-se - تنظیمات نقطه پایانی. - بررسی [\_نوست فهرست](server-configuration-parameters/settings.md#server_configuration_parameters-listen_host) و [\_صادر کردن](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) تنظیمات. + بررسی [_نوست فهرست](server-configuration-parameters/settings.md#server_configuration_parameters-listen_host) و [_صادر کردن](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) تنظیمات. سرور کلیک می پذیرد اتصالات مجنون تنها به طور پیش فرض. @@ -117,7 +117,7 @@ $ sudo -u clickhouse /usr/bin/clickhouse-server --config-file /etc/clickhouse-se بررسی: - - این [\_شروع مجدد](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) تنظیمات. + - این [_شروع مجدد](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) تنظیمات. - تنظیمات برای [SSL sertificates](server-configuration-parameters/settings.md#server_configuration_parameters-openssl). استفاده از پارامترهای مناسب در حالی که اتصال. برای مثال با استفاده از `port_secure` پارامتر با `clickhouse_client`. diff --git a/docs/fa/sql-reference/aggregate-functions/parametric-functions.md b/docs/fa/sql-reference/aggregate-functions/parametric-functions.md index 06d1a0bfcf7..c034720e485 100644 --- a/docs/fa/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/fa/sql-reference/aggregate-functions/parametric-functions.md @@ -494,6 +494,6 @@ Solution: Write in the GROUP BY query SearchPhrase HAVING uniqUpTo(4)(UserID) >= [مقاله اصلی](https://clickhouse.tech/docs/en/query_language/agg_functions/parametric_functions/) -## sumMapFiltered(keys\_to\_keep)(کلید ارزش ها) {#summapfilteredkeys-to-keepkeys-values} +## sumMapFiltered(keys_to_keep)(کلید ارزش ها) {#summapfilteredkeys-to-keepkeys-values} رفتار مشابه [& سواپ](reference.md#agg_functions-summap) جز این که مجموعه ای از کلید به عنوان یک پارامتر منتقل می شود. این می تواند مفید باشد به خصوص در هنگام کار با یک کارت از کلید های بالا. diff --git a/docs/fa/sql-reference/aggregate-functions/reference.md b/docs/fa/sql-reference/aggregate-functions/reference.md index acc8a270fed..0933bf50b20 100644 --- a/docs/fa/sql-reference/aggregate-functions/reference.md +++ b/docs/fa/sql-reference/aggregate-functions/reference.md @@ -31,7 +31,7 @@ ClickHouse زیر پشتیبانی می کند syntaxes برای `count`: **اطلاعات دقیق** -تاتر از `COUNT(DISTINCT ...)` نحو. رفتار این ساخت و ساز بستگی به [ا\_فزونهها](../../operations/settings/settings.md#settings-count_distinct_implementation) تنظیمات. این تعریف می کند که کدام یک از [دانشگاه\*](#agg_function-uniq) توابع برای انجام عملیات استفاده می شود. به طور پیش فرض است [قرارداد اتحادیه](#agg_function-uniqexact) تابع. +تاتر از `COUNT(DISTINCT ...)` نحو. رفتار این ساخت و ساز بستگی به [ا_فزونهها](../../operations/settings/settings.md#settings-count_distinct_implementation) تنظیمات. این تعریف می کند که کدام یک از [دانشگاه\*](#agg_function-uniq) توابع برای انجام عملیات استفاده می شود. به طور پیش فرض است [قرارداد اتحادیه](#agg_function-uniqexact) تابع. این `SELECT count() FROM table` پرس و جو بهینه سازی شده نیست, چرا که تعداد ورودی در جدول به طور جداگانه ذخیره نمی. این ستون کوچک را از جدول انتخاب می کند و تعداد مقادیر موجود را شمارش می کند. @@ -721,7 +721,7 @@ uniqExact(x[, ...]) - [مخلوط نشده](#agg_function-uniqcombined) - [یونقلل12](#agg_function-uniqhll12) -## groupArray(x) groupArray(max\_size)(x) {#agg_function-grouparray} +## groupArray(x) groupArray(max_size)(x) {#agg_function-grouparray} مجموعه ای از مقادیر استدلال را ایجاد می کند. مقادیر را می توان به ترتیب در هر (نامعین) اضافه کرد. @@ -967,7 +967,7 @@ FROM t └───────────┴──────────────────────────────────┴───────────────────────┘ ``` -## groupUniqArray(x) groupUniqArray(max\_size)(x) {#groupuniqarrayx-groupuniqarraymax-sizex} +## groupUniqArray(x) groupUniqArray(max_size)(x) {#groupuniqarrayx-groupuniqarraymax-sizex} مجموعه ای از مقادیر مختلف استدلال ایجاد می کند. مصرف حافظه همان است که برای `uniqExact` تابع. diff --git a/docs/fa/sql-reference/ansi.md b/docs/fa/sql-reference/ansi.md index 1f4eb677958..4e469891cd4 100644 --- a/docs/fa/sql-reference/ansi.md +++ b/docs/fa/sql-reference/ansi.md @@ -36,7 +36,7 @@ toc_title: "\u0633\u0627\u0632\u06AF\u0627\u0631\u06CC \u0627\u0646\u0633\u06CC" | E021-01 | نوع دادههای نویسه | نه{.text-danger} | | | E021-02 | شخصیت های مختلف نوع داده ها | نه{.text-danger} | `String` رفتار مشابه, اما بدون محدودیت طول در پرانتز | | E021-03 | شخصیت literals | نسبی{.text-warning} | بدون الحاق خودکار از لیتر متوالی و شخصیت پشتیبانی مجموعه | -| E021-04 | تابع \_شخصی | نسبی{.text-warning} | نه `USING` بند | +| E021-04 | تابع _شخصی | نسبی{.text-warning} | نه `USING` بند | | E021-05 | تابع اکتبر | نه{.text-danger} | `LENGTH` رفتار مشابه | | E021-06 | SUBSTRING | نسبی{.text-warning} | هیچ پشتیبانی برای `SIMILAR` و `ESCAPE` بند نه `SUBSTRING_REGEX` گزینه | | E021-07 | الحاق شخصیت | نسبی{.text-warning} | نه `COLLATE` بند | @@ -144,7 +144,7 @@ toc_title: "\u0633\u0627\u0632\u06AF\u0627\u0631\u06CC \u0627\u0646\u0633\u06CC" | F051-03 | نوع داده برچسب زمان (از جمله پشتیبانی از تحت اللفظی برچسب زمان) با دقت ثانیه کسری از حداقل 0 و 6 | نه{.text-danger} | `DateTime64` زمان فراهم می کند قابلیت های مشابه | | F051-04 | مقایسه گزاره در تاریخ, زمان, و انواع داده های برچسب زمان | نسبی{.text-warning} | فقط یک نوع داده موجود است | | F051-05 | بازیگران صریح و روشن بین انواع تاریخ ساعت و انواع رشته شخصیت | بله{.text-success} | | -| F051-06 | CURRENT\_DATE | نه{.text-danger} | `today()` مشابه است | +| F051-06 | CURRENT_DATE | نه{.text-danger} | `today()` مشابه است | | F051-07 | LOCALTIME | نه{.text-danger} | `now()` مشابه است | | F051-08 | LOCALTIMESTAMP | نه{.text-danger} | | | **F081** | **اتحادیه و به جز در دیدگاه** | **نسبی**{.text-warning} | | diff --git a/docs/fa/sql-reference/data-types/aggregatefunction.md b/docs/fa/sql-reference/data-types/aggregatefunction.md index 34a3634cd6e..85f80076192 100644 --- a/docs/fa/sql-reference/data-types/aggregatefunction.md +++ b/docs/fa/sql-reference/data-types/aggregatefunction.md @@ -6,7 +6,7 @@ toc_title: "\u06A9\u0627\u0631\u06A9\u0631\u062F(\u0646\u0627\u0645 \u0648 \u064 \u0645 \u062E\u0627\u0646\u0648\u0627\u062F\u06AF\u06CC..)" --- -# AggregateFunction(name, types\_of\_arguments…) {#data-type-aggregatefunction} +# AggregateFunction(name, types_of_arguments…) {#data-type-aggregatefunction} Aggregate functions can have an implementation-defined intermediate state that can be serialized to an AggregateFunction(…) data type and stored in a table, usually, by means of [مشاهده محقق](../../sql-reference/statements/create.md#create-view). راه معمول برای تولید یک دولت تابع جمع است با فراخوانی تابع جمع با `-State` پسوند. برای دریافت نتیجه نهایی از تجمع در اینده, شما باید همان تابع کل با استفاده از `-Merge`پسوند. diff --git a/docs/fa/sql-reference/data-types/datetime.md b/docs/fa/sql-reference/data-types/datetime.md index 9de82d07420..413b814f440 100644 --- a/docs/fa/sql-reference/data-types/datetime.md +++ b/docs/fa/sql-reference/data-types/datetime.md @@ -31,7 +31,7 @@ DateTime([timezone]) خروجی کلیک ارزش در `YYYY-MM-DD hh:mm:ss` قالب متن به طور پیش فرض. شما می توانید خروجی را با تغییر [formatDateTime](../../sql-reference/functions/date-time-functions.md#formatdatetime) تابع. -هنگام قرار دادن داده ها به تاتر, شما می توانید فرمت های مختلف تاریخ و زمان رشته استفاده, بسته به ارزش [تغییر \_شماره](../../operations/settings/settings.md#settings-date_time_input_format) تنظیمات. +هنگام قرار دادن داده ها به تاتر, شما می توانید فرمت های مختلف تاریخ و زمان رشته استفاده, بسته به ارزش [تغییر _شماره](../../operations/settings/settings.md#settings-date_time_input_format) تنظیمات. ## مثالها {#examples} diff --git a/docs/fa/sql-reference/data-types/fixedstring.md b/docs/fa/sql-reference/data-types/fixedstring.md index 9824acb0d2f..36d13975dbf 100644 --- a/docs/fa/sql-reference/data-types/fixedstring.md +++ b/docs/fa/sql-reference/data-types/fixedstring.md @@ -22,7 +22,7 @@ toc_title: "\u0631\u0634\u062A\u0647 \u062B\u0627\u0628\u062A)" نمونه هایی از مقادیر است که می تواند موثر در ذخیره می شود `FixedString`- ستون های تایپ شده: - نمایندگی دودویی نشانی های اینترنتی (`FixedString(16)` برای ایپو6). -- Language codes (ru\_RU, en\_US … ). +- Language codes (ru_RU, en_US … ). - Currency codes (USD, RUB … ). - نمایش دودویی رشته هش (`FixedString(16)` برای ام دی 5, `FixedString(32)` برای شی256). diff --git a/docs/fa/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md b/docs/fa/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md index 0cb27e5ec39..e36cd4cd110 100644 --- a/docs/fa/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md +++ b/docs/fa/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md @@ -28,7 +28,7 @@ toc_title: "\u0644\u063A\u062A\u0646\u0627\u0645\u0647\u0647\u0627 \u0633\u0644\ این سلسله مراتب را می توان به عنوان جدول فرهنگ لغت زیر بیان شده است. -| \_ورود | \_ نواحی | نام \_خانوادگی | +| _ورود | _ نواحی | نام _خانوادگی | |--------|----------|----------------| | 1 | 0 | روسیه | | 2 | 1 | مسکو | diff --git a/docs/fa/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/fa/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index eee74368127..dcef633c9e1 100644 --- a/docs/fa/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/fa/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -10,7 +10,7 @@ toc_title: "\u0630\u062E\u06CC\u0631\u0647 \u0648\u0627\u0698\u0647\u0646\u0627\ راه های مختلفی برای ذخیره لغت نامه ها در حافظه وجود دارد. -ما توصیه می کنیم [تخت](#flat), [درهم](#dicts-external_dicts_dict_layout-hashed) و [\_ساخت مجتمع](#complex-key-hashed). که سرعت پردازش بهینه را فراهم می کند. +ما توصیه می کنیم [تخت](#flat), [درهم](#dicts-external_dicts_dict_layout-hashed) و [_ساخت مجتمع](#complex-key-hashed). که سرعت پردازش بهینه را فراهم می کند. ذخیره سازی به دلیل عملکرد بالقوه ضعیف و مشکلات در انتخاب پارامترهای مطلوب توصیه نمی شود. ادامه مطلب در بخش “[نهانگاه](#cache)”. @@ -59,8 +59,8 @@ LAYOUT(LAYOUT_TYPE(param value)) -- layout settings - [نهانگاه](#cache) - [مستقیم](#direct) - [رنگها](#range-hashed) -- [\_ساخت مجتمع](#complex-key-hashed) -- [\_پیچید\_چهای پیچیده](#complex-key-cache) +- [_ساخت مجتمع](#complex-key-hashed) +- [_پیچید_چهای پیچیده](#complex-key-cache) - [شمال اروپا](#ip-trie) ### تخت {#flat} @@ -123,7 +123,7 @@ LAYOUT(HASHED()) LAYOUT(SPARSE_HASHED()) ``` -### \_ساخت مجتمع {#complex-key-hashed} +### _ساخت مجتمع {#complex-key-hashed} این نوع ذخیره سازی برای استفاده با کامپوزیت است [کلید](external-dicts-dict-structure.md). مشابه به `hashed`. @@ -294,7 +294,7 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000)) !!! warning "اخطار" هنوز تاتر به عنوان یک منبع استفاده نمی, چرا که کند است برای پردازش نمایش داده شد با تصادفی می خواند. -### \_پیچید\_چهای پیچیده {#complex-key-cache} +### _پیچید_چهای پیچیده {#complex-key-cache} این نوع ذخیره سازی برای استفاده با کامپوزیت است [کلید](external-dicts-dict-structure.md). مشابه به `cache`. diff --git a/docs/fa/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/fa/sql-reference/dictionaries/external-dictionaries/external-dicts.md index 3640c07a092..295913d0378 100644 --- a/docs/fa/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ b/docs/fa/sql-reference/dictionaries/external-dictionaries/external-dicts.md @@ -17,7 +17,7 @@ toc_title: "\u062A\u0648\u0636\u06CC\u062D\u0627\u062A \u06A9\u0644\u06CC" پیکربندی لغت نامه های خارجی را می توان در یک یا چند میلی لیتر فایل واقع شده است. مسیر پیکربندی در مشخص [دیکشنامهای](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config) پارامتر. -واژهنامهها را می توان در هنگام راه اندازی سرور و یا در اولین استفاده لود, بسته به [\_بارگیری کامل](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load) تنظیمات. +واژهنامهها را می توان در هنگام راه اندازی سرور و یا در اولین استفاده لود, بسته به [_بارگیری کامل](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load) تنظیمات. این [واژهنامهها](../../../operations/system-tables.md#system_tables-dictionaries) جدول سیستم شامل اطلاعات در مورد لغت نامه پیکربندی در سرور. برای هر فرهنگ لغت شما می توانید وجود دارد: diff --git a/docs/fa/sql-reference/functions/array-functions.md b/docs/fa/sql-reference/functions/array-functions.md index 6f4e8326557..f971ba7c0b8 100644 --- a/docs/fa/sql-reference/functions/array-functions.md +++ b/docs/fa/sql-reference/functions/array-functions.md @@ -1028,7 +1028,7 @@ SELECT arrayZip(['a', 'b', 'c'], [5, 2, 1]) ## ارریایکو {#arrayauc} -محاسبه حراج (منطقه تحت منحنی, که یک مفهوم در یادگیری ماشین است, مشاهده اطلاعات بیشتر: https://en.wikipedia.org/wiki/Receiver\_operating\_characteristic\#Area\_under\_the\_curve). +محاسبه حراج (منطقه تحت منحنی, که یک مفهوم در یادگیری ماشین است, مشاهده اطلاعات بیشتر: https://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve). **نحو** diff --git a/docs/fa/sql-reference/functions/bitmap-functions.md b/docs/fa/sql-reference/functions/bitmap-functions.md index f7740cf02e9..061489fd08c 100644 --- a/docs/fa/sql-reference/functions/bitmap-functions.md +++ b/docs/fa/sql-reference/functions/bitmap-functions.md @@ -241,7 +241,7 @@ SELECT bitmapCardinality(bitmapBuild([1, 2, 3, 4, 5])) AS res ## بیت مپمن {#bitmapmin} -Retrun کوچکترین مقدار از نوع UInt64 در مجموعه UINT32\_MAX اگر این مجموعه خالی است. +Retrun کوچکترین مقدار از نوع UInt64 در مجموعه UINT32_MAX اگر این مجموعه خالی است. bitmapMin(bitmap) @@ -288,8 +288,8 @@ SELECT bitmapMax(bitmapBuild([1, 2, 3, 4, 5])) AS res **پارامترها** - `bitmap` – bitmap object. -- `from_array` – UInt32 array. For idx in range \[0, from\_array.size()), if bitmap contains from\_array\[idx\], then replace it with to\_array\[idx\]. Note that the result depends on array ordering if there are common elements between from\_array and to\_array. -- `to_array` – UInt32 array, its size shall be the same to from\_array. +- `from_array` – UInt32 array. For idx in range \[0, from_array.size()), if bitmap contains from_array\[idx\], then replace it with to_array\[idx\]. Note that the result depends on array ordering if there are common elements between from_array and to_array. +- `to_array` – UInt32 array, its size shall be the same to from_array. **مثال** diff --git a/docs/fa/sql-reference/functions/date-time-functions.md b/docs/fa/sql-reference/functions/date-time-functions.md index e956b7b71fa..5a6ebc08d8f 100644 --- a/docs/fa/sql-reference/functions/date-time-functions.md +++ b/docs/fa/sql-reference/functions/date-time-functions.md @@ -164,7 +164,7 @@ SELECT toUnixTimestamp('2017-11-05 08:07:47', 'Asia/Tokyo') AS unix_timestamp دور پایین تاریخ با زمان به شروع فاصله پانزده دقیقه. -## toStartOfInterval(time\_or\_data فاصله x واحد \[, time\_zone\]) {#tostartofintervaltime-or-data-interval-x-unit-time-zone} +## toStartOfInterval(time_or_data فاصله x واحد \[, time_zone\]) {#tostartofintervaltime-or-data-interval-x-unit-time-zone} این یک تعمیم توابع دیگر به نام است `toStartOf*`. به عنوان مثال, `toStartOfInterval(t, INTERVAL 1 year)` همان را برمی گرداند `toStartOfYear(t)`, @@ -426,25 +426,25 @@ Function formats a Time according given Format string. N.B.: Format is a constan | تغییردهنده | توصیف | مثال | |------------|-----------------------------------------------------------------------------|------------| | %C | سال تقسیم بر 100 و کوتاه به عدد صحیح (00-99) | 20 | -| \# د | روز از ماه, صفر خالی (01-31) | 02 | +| # د | روز از ماه, صفر خالی (01-31) | 02 | | %D | کوتاه میلی متر/دی دی/یی تاریخ, معادل %متر/%د / %و | 01/02/18 | -| \# ا | روز از ماه, فضا خالی ( 1-31) | 2 | +| # ا | روز از ماه, فضا خالی ( 1-31) | 2 | | %F | کوتاه تاریخ یی-میلی متر-دی دی, معادل%و-%متر - % د | 2018-01-02 | | %H | ساعت در فرمت 24 ساعت (00-23) | 22 | | %I | ساعت در فرمت 12 ساعت (01-12) | 10 | -| \# ج | روز سال (001-366) | 002 | +| # ج | روز سال (001-366) | 002 | | % متر | ماه به عنوان یک عدد اعشاری (01-12) | 01 | | %M | دقیقه (00-59) | 33 | | % ن | شخصیت جدید خط (") | | -| \# پ | هستم یا بعد از ظهر تعیین | PM | +| # پ | هستم یا بعد از ظهر تعیین | PM | | %R | 24-ساعت ساعت ساعت: زمان میلی متر, معادل %ساعت: % متر | 22:33 | | %S | دوم (00-59) | 44 | | % تی | شخصیت افقی تب (') | | | %T | ایزو 8601 فرمت زمان (ساعت:میلی متر:اس اس), معادل %ساعت:%متر:%بازدید کنندگان | 22:33:44 | -| \# تو | ایزو 8601 روز هفته به عنوان شماره با دوشنبه به عنوان 1 (1-7) | 2 | +| # تو | ایزو 8601 روز هفته به عنوان شماره با دوشنبه به عنوان 1 (1-7) | 2 | | %V | ایزو 8601 هفته شماره (01-53) | 01 | -| \# وات | روز هفته به عنوان یک عدد اعشاری با یکشنبه به عنوان 0 (0-6) | 2 | -| \#... | سال گذشته دو رقم (00-99) | 18 | +| # وات | روز هفته به عنوان یک عدد اعشاری با یکشنبه به عنوان 0 (0-6) | 2 | +| #... | سال گذشته دو رقم (00-99) | 18 | | %Y | سال | 2018 | | %% | یک % نشانه | % | diff --git a/docs/fa/sql-reference/functions/hash-functions.md b/docs/fa/sql-reference/functions/hash-functions.md index fae02d22950..678655f8cf0 100644 --- a/docs/fa/sql-reference/functions/hash-functions.md +++ b/docs/fa/sql-reference/functions/hash-functions.md @@ -317,7 +317,7 @@ SELECT metroHash64(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00: می پذیرد دو استدلال: یک کلید بین 64 نوع و تعداد سطل. بازده Int32. برای کسب اطلاعات بیشتر به لینک مراجعه کنید: [مورد احترام](https://arxiv.org/pdf/1406.2294.pdf) -## سوفلش2\_32, سوفلشه2\_64 {#murmurhash2-32-murmurhash2-64} +## سوفلش2_32, سوفلشه2_64 {#murmurhash2-32-murmurhash2-64} تولید یک [زمزمه 2](https://github.com/aappleby/smhasher) مقدار هش. @@ -385,7 +385,7 @@ SELECT └──────────────────────┴─────────────────────┘ ``` -## سوفلش3\_32, سوفلشه3\_64 {#murmurhash3-32-murmurhash3-64} +## سوفلش3_32, سوفلشه3_64 {#murmurhash3-32-murmurhash3-64} تولید یک [سوفلهاش3](https://github.com/aappleby/smhasher) مقدار هش. @@ -415,7 +415,7 @@ SELECT murmurHash3_32(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23: └─────────────┴────────┘ ``` -## سوفلش3\_128 {#murmurhash3-128} +## سوفلش3_128 {#murmurhash3-128} تولید 128 بیتی [سوفلهاش3](https://github.com/aappleby/smhasher) مقدار هش. diff --git a/docs/fa/sql-reference/functions/introspection.md b/docs/fa/sql-reference/functions/introspection.md index 3f47fa77943..5885a8c8123 100644 --- a/docs/fa/sql-reference/functions/introspection.md +++ b/docs/fa/sql-reference/functions/introspection.md @@ -16,11 +16,11 @@ toc_title: "\u062F\u0631\u0648\u0646 \u0646\u06AF\u0631\u06CC" - نصب `clickhouse-common-static-dbg` بسته -- تنظیم [اجازه دادن به \_فعال کردن اختلال در عملکرد](../../operations/settings/settings.md#settings-allow_introspection_functions) تنظیم به 1. +- تنظیم [اجازه دادن به _فعال کردن اختلال در عملکرد](../../operations/settings/settings.md#settings-allow_introspection_functions) تنظیم به 1. For security reasons introspection functions are disabled by default. -تاتر موجب صرفه جویی در گزارش نیمرخ به [\_قطع](../../operations/system-tables.md#system_tables-trace_log) جدول سیستم. اطمینان حاصل کنید که جدول و پیشفیلتر به درستی پیکربندی شده است. +تاتر موجب صرفه جویی در گزارش نیمرخ به [_قطع](../../operations/system-tables.md#system_tables-trace_log) جدول سیستم. اطمینان حاصل کنید که جدول و پیشفیلتر به درستی پیکربندی شده است. ## افزودن مدخل جدید {#addresstoline} diff --git a/docs/fa/sql-reference/functions/json-functions.md b/docs/fa/sql-reference/functions/json-functions.md index 0f53a751989..494bb129817 100644 --- a/docs/fa/sql-reference/functions/json-functions.md +++ b/docs/fa/sql-reference/functions/json-functions.md @@ -75,7 +75,7 @@ SELECT isValidJSON('{"a": "hello", "b": [-100, 200.0, 300]}') = 1 SELECT isValidJSON('not a json') = 0 ``` -## JSONHas(json\[, indices\_or\_keys\]…) {#jsonhasjson-indices-or-keys} +## JSONHas(json\[, indices_or_keys\]…) {#jsonhasjson-indices-or-keys} اگر مقدار در سند جسون وجود داشته باشد, `1` برگردانده خواهد شد. @@ -108,7 +108,7 @@ SELECT JSONExtractKey('{"a": "hello", "b": [-100, 200.0, 300]}', -2) = 'a' SELECT JSONExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 1) = 'hello' ``` -## JSONLength(json\[, indices\_or\_keys\]…) {#jsonlengthjson-indices-or-keys} +## JSONLength(json\[, indices_or_keys\]…) {#jsonlengthjson-indices-or-keys} بازگشت طول یک مجموعه جانسون یا یک شی جانسون. @@ -121,7 +121,7 @@ SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 3 SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}') = 2 ``` -## JSONType(json\[, indices\_or\_keys\]…) {#jsontypejson-indices-or-keys} +## JSONType(json\[, indices_or_keys\]…) {#jsontypejson-indices-or-keys} بازگشت به نوع یک مقدار جانسون. @@ -135,13 +135,13 @@ SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'a') = 'String' SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 'Array' ``` -## JSONExtractUInt(json\[, indices\_or\_keys\]…) {#jsonextractuintjson-indices-or-keys} +## JSONExtractUInt(json\[, indices_or_keys\]…) {#jsonextractuintjson-indices-or-keys} -## JSONExtractInt(json\[, indices\_or\_keys\]…) {#jsonextractintjson-indices-or-keys} +## JSONExtractInt(json\[, indices_or_keys\]…) {#jsonextractintjson-indices-or-keys} -## JSONExtractFloat(json\[, indices\_or\_keys\]…) {#jsonextractfloatjson-indices-or-keys} +## JSONExtractFloat(json\[, indices_or_keys\]…) {#jsonextractfloatjson-indices-or-keys} -## JSONExtractBool(json\[, indices\_or\_keys\]…) {#jsonextractbooljson-indices-or-keys} +## JSONExtractBool(json\[, indices_or_keys\]…) {#jsonextractbooljson-indices-or-keys} تجزیه جانسون و استخراج ارزش. این توابع شبیه به `visitParam` توابع. @@ -155,7 +155,7 @@ SELECT JSONExtractFloat('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 2) = 200 SELECT JSONExtractUInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', -1) = 300 ``` -## JSONExtractString(json\[, indices\_or\_keys\]…) {#jsonextractstringjson-indices-or-keys} +## JSONExtractString(json\[, indices_or_keys\]…) {#jsonextractstringjson-indices-or-keys} تجزیه جانسون و استخراج یک رشته. این تابع شبیه به `visitParamExtractString` توابع. @@ -173,7 +173,7 @@ SELECT JSONExtractString('{"abc":"\\u263"}', 'abc') = '' SELECT JSONExtractString('{"abc":"hello}', 'abc') = '' ``` -## JSONExtract(json\[, indices\_or\_keys…\], Return\_type) {#jsonextractjson-indices-or-keys-return-type} +## JSONExtract(json\[, indices_or_keys…\], Return_type) {#jsonextractjson-indices-or-keys-return-type} تجزیه یک جسون و استخراج یک مقدار از نوع داده داده داده کلیک. @@ -194,7 +194,7 @@ SELECT JSONExtract('{"day": "Thursday"}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday SELECT JSONExtract('{"day": 5}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \'Tuesday\' = 2, \'Wednesday\' = 3, \'Thursday\' = 4, \'Friday\' = 5, \'Saturday\' = 6)') = 'Friday' ``` -## JSONExtractKeysAndValues(json\[, indices\_or\_keys…\], Value\_type) {#jsonextractkeysandvaluesjson-indices-or-keys-value-type} +## JSONExtractKeysAndValues(json\[, indices_or_keys…\], Value_type) {#jsonextractkeysandvaluesjson-indices-or-keys-value-type} تجزیه جفت کلید ارزش از یک جانسون که ارزش از نوع داده داده داده خانه عروسکی هستند. @@ -204,7 +204,7 @@ SELECT JSONExtract('{"day": 5}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \ SELECT JSONExtractKeysAndValues('{"x": {"a": 5, "b": 7, "c": 11}}', 'x', 'Int8') = [('a',5),('b',7),('c',11)] ``` -## JSONExtractRaw(json\[, indices\_or\_keys\]…) {#jsonextractrawjson-indices-or-keys} +## JSONExtractRaw(json\[, indices_or_keys\]…) {#jsonextractrawjson-indices-or-keys} بازگرداندن بخشی از جانسون به عنوان رشته نامحدود. @@ -216,7 +216,7 @@ SELECT JSONExtractKeysAndValues('{"x": {"a": 5, "b": 7, "c": 11}}', 'x', 'Int8') SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = '[-100, 200.0, 300]' ``` -## JSONExtractArrayRaw(json\[, indices\_or\_keys…\]) {#jsonextractarrayrawjson-indices-or-keys} +## JSONExtractArrayRaw(json\[, indices_or_keys…\]) {#jsonextractarrayrawjson-indices-or-keys} بازگرداندن مجموعه ای با عناصر از مجموعه جانسون,هر یک به عنوان رشته نامحدود نشان. diff --git a/docs/fa/sql-reference/functions/other-functions.md b/docs/fa/sql-reference/functions/other-functions.md index d18fac245c5..7ef56a0478b 100644 --- a/docs/fa/sql-reference/functions/other-functions.md +++ b/docs/fa/sql-reference/functions/other-functions.md @@ -419,7 +419,7 @@ ORDER BY h ASC تبدیل یک ارزش با توجه به نقشه برداری به صراحت تعریف شده از برخی از عناصر به دیگر. دو نوع از این تابع وجود دارد: -### تبدیل(x array\_from, array\_to به طور پیش فرض) {#transformx-array-from-array-to-default} +### تبدیل(x array_from, array_to به طور پیش فرض) {#transformx-array-from-array-to-default} `x` – What to transform. @@ -439,7 +439,7 @@ ORDER BY h ASC از کجا همان نامه نشان داده شده است (تی یا تو), برای انواع عددی این ممکن است تطبیق انواع, اما انواع که یک نوع رایج. برای مثال استدلال می توانید نوع Int64 در حالی که دوم آرایه(UInt16) نوع. -اگر ‘x’ ارزش به یکی از عناصر در برابر است ‘array\_from’ مجموعه, این بازگرداندن عنصر موجود (که شماره همان) از ‘array\_to’ صف کردن. در غیر این صورت, باز می گردد ‘default’. اگر عناصر تطبیق های متعدد در وجود دارد ‘array\_from’ این یکی از مسابقات را برمی گرداند. +اگر ‘x’ ارزش به یکی از عناصر در برابر است ‘array_from’ مجموعه, این بازگرداندن عنصر موجود (که شماره همان) از ‘array_to’ صف کردن. در غیر این صورت, باز می گردد ‘default’. اگر عناصر تطبیق های متعدد در وجود دارد ‘array_from’ این یکی از مسابقات را برمی گرداند. مثال: @@ -464,7 +464,7 @@ ORDER BY c DESC ### تبدیل) {#transformx-array-from-array-to} متفاوت از تنوع برای اولین بار در که ‘default’ استدلال حذف شده است. -اگر ‘x’ ارزش به یکی از عناصر در برابر است ‘array\_from’ مجموعه, این بازگرداندن عنصر تطبیق (که شماره همان) از ‘array\_to’ صف کردن. در غیر این صورت, باز می گردد ‘x’. +اگر ‘x’ ارزش به یکی از عناصر در برابر است ‘array_from’ مجموعه, این بازگرداندن عنصر تطبیق (که شماره همان) از ‘array_to’ صف کردن. در غیر این صورت, باز می گردد ‘x’. انواع: @@ -1121,15 +1121,15 @@ SELECT joinGet(db_test.id_val,'val',toUInt32(number)) from numbers(4) SETTINGS j └──────────────────────────────────────────────────┘ ``` -## modelEvaluate(model\_name, …) {#function-modelevaluate} +## modelEvaluate(model_name, …) {#function-modelevaluate} ارزیابی مدل خارجی. می پذیرد نام مدل و استدلال مدل. را برمی گرداند شناور64. -## throwIf(x\[, custom\_message\]) {#throwifx-custom-message} +## throwIf(x\[, custom_message\]) {#throwifx-custom-message} پرتاب یک استثنا اگر استدلال غیر صفر است. -\_پیغام سفارشی-پارامتر اختیاری است: یک رشته ثابت, فراهم می کند یک پیغام خطا +_پیغام سفارشی-پارامتر اختیاری است: یک رشته ثابت, فراهم می کند یک پیغام خطا ``` sql SELECT throwIf(number = 3, 'Too many') FROM numbers(10); diff --git a/docs/fa/sql-reference/functions/string-functions.md b/docs/fa/sql-reference/functions/string-functions.md index cc69b6fce32..ff4940044da 100644 --- a/docs/fa/sql-reference/functions/string-functions.md +++ b/docs/fa/sql-reference/functions/string-functions.md @@ -31,12 +31,12 @@ toc_title: "\u06A9\u0627\u0631 \u0628\u0627 \u0631\u0634\u062A\u0647 \u0647\u062 بازگرداندن طول یک رشته در نقاط کد یونیکد (نه در شخصیت), فرض کنید که رشته شامل مجموعه ای از بایت است که متن کد گذاری شده را تشکیل می دهند. اگر این فرض ملاقات نکرده است, این گرداند برخی از نتیجه (این یک استثنا پرتاب نمی کند). نتیجه این نوع UInt64. -## \_شروع مجدد {#char-length} +## _شروع مجدد {#char-length} بازگرداندن طول یک رشته در نقاط کد یونیکد (نه در شخصیت), فرض کنید که رشته شامل مجموعه ای از بایت است که متن کد گذاری شده را تشکیل می دهند. اگر این فرض ملاقات نکرده است, این گرداند برخی از نتیجه (این یک استثنا پرتاب نمی کند). نتیجه این نوع UInt64. -## \_شخصیت شناسی {#character-length} +## _شخصیت شناسی {#character-length} بازگرداندن طول یک رشته در نقاط کد یونیکد (نه در شخصیت), فرض کنید که رشته شامل مجموعه ای از بایت است که متن کد گذاری شده را تشکیل می دهند. اگر این فرض ملاقات نکرده است, این گرداند برخی از نتیجه (این یک استثنا پرتاب نمی کند). نتیجه این نوع UInt64. @@ -77,7 +77,7 @@ toValidUTF8( input_string ) پارامترها: -- input\_string — Any set of bytes represented as the [رشته](../../sql-reference/data-types/string.md) شی نوع داده. +- input_string — Any set of bytes represented as the [رشته](../../sql-reference/data-types/string.md) شی نوع داده. مقدار بازگشتی: معتبر یونایتد-8 رشته. diff --git a/docs/fa/sql-reference/functions/type-conversion-functions.md b/docs/fa/sql-reference/functions/type-conversion-functions.md index 353701818b8..61605475dad 100644 --- a/docs/fa/sql-reference/functions/type-conversion-functions.md +++ b/docs/fa/sql-reference/functions/type-conversion-functions.md @@ -246,7 +246,7 @@ YYYY-MM-DD YYYY-MM-DD hh:mm:ss ``` -به عنوان یک استثنا اگر تبدیل از UInt32, Int32, UInt64 یا Int64 عددی انواع, به, تاریخ, و اگر عدد بزرگتر یا مساوی به 65536 تعدادی را به عنوان تفسیر یک زمان یونیکس (و نه به عنوان تعداد روز) و گرد است به تاریخ. این اجازه می دهد تا پشتیبانی از وقوع مشترک نوشتن ‘toDate(unix\_timestamp)’ که در غیر این صورت یک خطا خواهد بود و نیاز به نوشتن بیشتر دست و پا گیر ‘toDate(toDateTime(unix\_timestamp))’. +به عنوان یک استثنا اگر تبدیل از UInt32, Int32, UInt64 یا Int64 عددی انواع, به, تاریخ, و اگر عدد بزرگتر یا مساوی به 65536 تعدادی را به عنوان تفسیر یک زمان یونیکس (و نه به عنوان تعداد روز) و گرد است به تاریخ. این اجازه می دهد تا پشتیبانی از وقوع مشترک نوشتن ‘toDate(unix_timestamp)’ که در غیر این صورت یک خطا خواهد بود و نیاز به نوشتن بیشتر دست و پا گیر ‘toDate(toDateTime(unix_timestamp))’. تبدیل بین تاریخ و تاریخ با زمان انجام شده است راه طبیعی: با اضافه کردن یک زمان خالی و یا حذف زمان. diff --git a/docs/fa/sql-reference/functions/url-functions.md b/docs/fa/sql-reference/functions/url-functions.md index 054caee4c4c..e270e9de424 100644 --- a/docs/fa/sql-reference/functions/url-functions.md +++ b/docs/fa/sql-reference/functions/url-functions.md @@ -126,11 +126,11 @@ SELECT topLevelDomain('svn+ssh://www.some.svn-hosting.com:80/repo/trunk') ### مسیر {#pathfull} -همان بالا, اما از جمله رشته پرس و جو و قطعه. مثال:/بالا / اخبار.زنگامصفحه = 2 \# نظرات +همان بالا, اما از جمله رشته پرس و جو و قطعه. مثال:/بالا / اخبار.زنگامصفحه = 2 # نظرات ### رشته {#querystring} -بازگرداندن رشته پرس و جو. مثال: صفحه=1&چاپی=213. پرس و جو رشته علامت سوال اولیه را شامل نمی شود, و همچنین \# و همه چیز بعد از \#. +بازگرداندن رشته پرس و جو. مثال: صفحه=1&چاپی=213. پرس و جو رشته علامت سوال اولیه را شامل نمی شود, و همچنین # و همه چیز بعد از #. ### قطعه {#fragment} @@ -138,7 +138,7 @@ SELECT topLevelDomain('svn+ssh://www.some.svn-hosting.com:80/repo/trunk') ### وضعیت زیستشناختی رکورد {#querystringandfragment} -بازگرداندن رشته پرس و جو و شناسه قطعه. مثال: صفحه=1\#29390. +بازگرداندن رشته پرس و جو و شناسه قطعه. مثال: صفحه=1#29390. ### نام) {#extracturlparameterurl-name} diff --git a/docs/fa/sql-reference/functions/ym-dict-functions.md b/docs/fa/sql-reference/functions/ym-dict-functions.md index 9e08ee16516..689d0171666 100644 --- a/docs/fa/sql-reference/functions/ym-dict-functions.md +++ b/docs/fa/sql-reference/functions/ym-dict-functions.md @@ -19,12 +19,12 @@ ClickHouse پشتیبانی از کار با چند جایگزین geobases (م این ‘clickhouse-server’ پیکربندی فایل را با سلسله مراتب منطقه ای مشخص می کند::`/opt/geo/regions_hierarchy.txt` -علاوه بر این فایل, همچنین برای فایل های جستجو در این نزدیکی هست که نماد \_ و هر پسوند اضافه به نام (قبل از پسوند فایل). +علاوه بر این فایل, همچنین برای فایل های جستجو در این نزدیکی هست که نماد _ و هر پسوند اضافه به نام (قبل از پسوند فایل). مثلا, همچنین فایل را پیدا خواهد کرد `/opt/geo/regions_hierarchy_ua.txt`, اگر در حال حاضر. `ua` کلید فرهنگ لغت نامیده می شود. برای یک فرهنگ لغت بدون پسوند, کلید یک رشته خالی است. -تمام واژهنامهها دوباره لود شده در زمان اجرا (یک بار در هر تعداد معینی از ثانیه, همانطور که در دستور داخلی تعریف شده \_فرهنگ\_ پارامتر پیکربندی, و یا یک بار در ساعت به طور پیش فرض). با این حال, لیست لغت نامه های موجود تعریف شده است یک بار, زمانی که سرور شروع می شود. +تمام واژهنامهها دوباره لود شده در زمان اجرا (یک بار در هر تعداد معینی از ثانیه, همانطور که در دستور داخلی تعریف شده _فرهنگ_ پارامتر پیکربندی, و یا یک بار در ساعت به طور پیش فرض). با این حال, لیست لغت نامه های موجود تعریف شده است یک بار, زمانی که سرور شروع می شود. All functions for working with regions have an optional argument at the end – the dictionary key. It is referred to as the geobase. مثال: diff --git a/docs/fa/sql-reference/operators/in.md b/docs/fa/sql-reference/operators/in.md index 779d92e0174..7ddffca0e2b 100644 --- a/docs/fa/sql-reference/operators/in.md +++ b/docs/fa/sql-reference/operators/in.md @@ -117,9 +117,9 @@ FROM t_null مراقب باشید در هنگام استفاده از کارخانه های فرعی در `IN` / `JOIN` بند برای پردازش پرس و جو توزیع. -بیایید نگاهی به برخی از نمونه. فرض کنید که هر سرور در خوشه طبیعی است **\_تمل**. هر سرور همچنین دارای یک **توزیع \_تماس** جدول با **توزیع شده** نوع, که به نظر می رسد در تمام سرور در خوشه. +بیایید نگاهی به برخی از نمونه. فرض کنید که هر سرور در خوشه طبیعی است **_تمل**. هر سرور همچنین دارای یک **توزیع _تماس** جدول با **توزیع شده** نوع, که به نظر می رسد در تمام سرور در خوشه. -برای پرس و جو به **توزیع \_تماس** پرس و جو به تمام سرورهای راه دور ارسال می شود و با استفاده از **\_تمل**. +برای پرس و جو به **توزیع _تماس** پرس و جو به تمام سرورهای راه دور ارسال می شود و با استفاده از **_تمل**. برای مثال پرس و جو @@ -153,7 +153,7 @@ SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID IN (SEL این به درستی و بهینه کار خواهد کرد اگر شما برای این مورد تهیه و داده ها در سراسر سرورهای خوشه گسترش یافته اند به طوری که داده ها را برای یک شناسه تنها ساکن به طور کامل بر روی یک سرور واحد. در این مورد, تمام اطلاعات لازم در دسترس خواهد بود به صورت محلی بر روی هر سرور. در غیر این صورت نتیجه نادرست خواهد بود. ما به این تنوع از پرس و جو به عنوان مراجعه کنید “local IN”. -برای اصلاح چگونه پرس و جو کار می کند زمانی که داده ها به طور تصادفی در سراسر سرور خوشه گسترش, شما می توانید مشخص **توزیع \_تماس** در داخل یک خرده فروشی. پرس و جو شبیه به این خواهد بود: +برای اصلاح چگونه پرس و جو کار می کند زمانی که داده ها به طور تصادفی در سراسر سرور خوشه گسترش, شما می توانید مشخص **توزیع _تماس** در داخل یک خرده فروشی. پرس و جو شبیه به این خواهد بود: ``` sql SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM distributed_table WHERE CounterID = 34) diff --git a/docs/fa/sql-reference/statements/alter.md b/docs/fa/sql-reference/statements/alter.md index 36711201bba..9a689546e84 100644 --- a/docs/fa/sql-reference/statements/alter.md +++ b/docs/fa/sql-reference/statements/alter.md @@ -208,7 +208,7 @@ ALTER TABLE [db].name DROP CONSTRAINT constraint_name; - [ATTACH PART\|PARTITION](#alter_attach-partition) – Adds a part or partition from the `detached` دایرکتوری به جدول. - [ATTACH PARTITION FROM](#alter_attach-partition-from) – Copies the data partition from one table to another and adds. - [REPLACE PARTITION](#alter_replace-partition) - پارتیشن داده ها را از یک جدول به دیگری کپی می کند و جایگزین می شود. -- [MOVE PARTITION TO TABLE](#alter_move_to_table-partition)(\#تغییر\_موف\_ قابل تنظیم-پارتیشن) - پارتیشن داده را از یک جدول به دیگری حرکت دهید. +- [MOVE PARTITION TO TABLE](#alter_move_to_table-partition)(#تغییر_موف_ قابل تنظیم-پارتیشن) - پارتیشن داده را از یک جدول به دیگری حرکت دهید. - [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) - بازنشانی ارزش یک ستون مشخص شده در یک پارتیشن. - [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) - بازنشانی شاخص ثانویه مشخص شده در یک پارتیشن. - [FREEZE PARTITION](#alter_freeze-partition) – Creates a backup of a partition. diff --git a/docs/fa/sql-reference/statements/insert-into.md b/docs/fa/sql-reference/statements/insert-into.md index 39950c99688..c8d2c76f31c 100644 --- a/docs/fa/sql-reference/statements/insert-into.md +++ b/docs/fa/sql-reference/statements/insert-into.md @@ -20,7 +20,7 @@ INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), . - مقادیر محاسبه شده از `DEFAULT` عبارات مشخص شده در تعریف جدول. - صفر و رشته خالی, اگر `DEFAULT` عبارات تعریف نشده. -اگر [\_مرحلهای دقیق = 1](../../operations/settings/settings.md) ستون هایی که ندارند `DEFAULT` تعریف شده باید در پرس و جو ذکر شده است. +اگر [_مرحلهای دقیق = 1](../../operations/settings/settings.md) ستون هایی که ندارند `DEFAULT` تعریف شده باید در پرس و جو ذکر شده است. داده ها را می توان به درج در هر گذشت [قالب](../../interfaces/formats.md#formats) پشتیبانی شده توسط فاحشه خانه. قالب باید به صراحت در پرس و جو مشخص شود: diff --git a/docs/fa/sql-reference/statements/misc.md b/docs/fa/sql-reference/statements/misc.md index d67983bd5ba..20491f9a668 100644 --- a/docs/fa/sql-reference/statements/misc.md +++ b/docs/fa/sql-reference/statements/misc.md @@ -57,7 +57,7 @@ CHECK TABLE [db.]name اگر جدول خراب شده است, شما می توانید داده های غیر خراب به جدول دیگر کپی کنید. برای انجام این کار: 1. ایجاد یک جدول جدید با ساختار همان جدول صدمه دیده است. برای انجام این کار پرس و جو را اجرا کنید `CREATE TABLE AS `. -2. تنظیم [\_مخفی کردن](../../operations/settings/settings.md#settings-max_threads) ارزش به 1 برای پردازش پرس و جو بعدی در یک موضوع واحد. برای انجام این کار پرس و جو را اجرا کنید `SET max_threads = 1`. +2. تنظیم [_مخفی کردن](../../operations/settings/settings.md#settings-max_threads) ارزش به 1 برای پردازش پرس و جو بعدی در یک موضوع واحد. برای انجام این کار پرس و جو را اجرا کنید `SET max_threads = 1`. 3. اجرای پرسوجو `INSERT INTO SELECT * FROM `. این درخواست داده های غیر خراب شده را از جدول خراب شده به جدول دیگر کپی می کند. فقط داده ها قبل از قسمت خراب کپی خواهد شد. 4. راه اندازی مجدد `clickhouse-client` برای تنظیم مجدد `max_threads` ارزش. @@ -253,7 +253,7 @@ OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition | PARTITION I چه زمانی `OPTIMIZE` با استفاده از [تکرار غذای اصلی](../../engines/table-engines/mergetree-family/replication.md) خانواده از موتورهای جدول, تاتر ایجاد یک کار برای ادغام و منتظر اعدام در تمام گره (در صورتی که `replication_alter_partitions_sync` تنظیم فعال است). -- اگر `OPTIMIZE` یک ادغام به هر دلیلی انجام نمی, این کار مشتری اطلاع نیست. برای فعال کردن اعلان ها از [ا\_فزون\_ف\_کوپ](../../operations/settings/settings.md#setting-optimize_throw_if_noop) تنظیمات. +- اگر `OPTIMIZE` یک ادغام به هر دلیلی انجام نمی, این کار مشتری اطلاع نیست. برای فعال کردن اعلان ها از [ا_فزون_ف_کوپ](../../operations/settings/settings.md#setting-optimize_throw_if_noop) تنظیمات. - اگر شما یک مشخص `PARTITION` فقط پارتیشن مشخص شده بهینه شده است. [نحوه تنظیم بیان پارتیشن](alter.md#alter-how-to-specify-part-expr). - اگر شما مشخص کنید `FINAL` حتی زمانی که تمام داده ها در حال حاضر در یک بخش بهینه سازی انجام شده است. - اگر شما مشخص کنید `DEDUPLICATE` و سپس به طور کامل یکسان ردیف خواهد بود deduplicated (تمام ستون ها در مقایسه با) آن را حس می کند تنها برای MergeTree موتور. diff --git a/docs/fa/sql-reference/statements/select/group-by.md b/docs/fa/sql-reference/statements/select/group-by.md index 9b543a4b606..7c8307796cd 100644 --- a/docs/fa/sql-reference/statements/select/group-by.md +++ b/docs/fa/sql-reference/statements/select/group-by.md @@ -57,7 +57,7 @@ machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd - داخل `Pretty*` فرمت, ردیف خروجی به عنوان یک جدول جداگانه پس از نتیجه اصلی است. - در فرمت های دیگر در دسترس نیست. -`WITH TOTALS` می توان در راه های مختلف اجرا زمانی که داشتن حاضر است. رفتار بستگی به ‘totals\_mode’ تنظیمات. +`WITH TOTALS` می توان در راه های مختلف اجرا زمانی که داشتن حاضر است. رفتار بستگی به ‘totals_mode’ تنظیمات. ### پیکربندی پردازش بالغ {#configuring-totals-processing} @@ -67,9 +67,9 @@ machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd `after_having_exclusive` – Don't include rows that didn't pass through `max_rows_to_group_by`. به عبارت دیگر, ‘totals’ کمتر از و یا به همان تعداد از ردیف به عنوان اگر داشته باشد `max_rows_to_group_by` حذف شد. -`after_having_inclusive` – Include all the rows that didn't pass through ‘max\_rows\_to\_group\_by’ داخل ‘totals’. به عبارت دیگر, ‘totals’ بیش از و یا به همان تعداد از ردیف به عنوان اگر داشته باشد `max_rows_to_group_by` حذف شد. +`after_having_inclusive` – Include all the rows that didn't pass through ‘max_rows_to_group_by’ داخل ‘totals’. به عبارت دیگر, ‘totals’ بیش از و یا به همان تعداد از ردیف به عنوان اگر داشته باشد `max_rows_to_group_by` حذف شد. -`after_having_auto` – Count the number of rows that passed through HAVING. If it is more than a certain amount (by default, 50%), include all the rows that didn't pass through ‘max\_rows\_to\_group\_by’ داخل ‘totals’. در غیر این صورت, را شامل نمی شود. +`after_having_auto` – Count the number of rows that passed through HAVING. If it is more than a certain amount (by default, 50%), include all the rows that didn't pass through ‘max_rows_to_group_by’ داخل ‘totals’. در غیر این صورت, را شامل نمی شود. `totals_auto_threshold` – By default, 0.5. The coefficient for `after_having_auto`. @@ -117,11 +117,11 @@ GROUP BY domain ### گروه در حافظه خارجی {#select-group-by-in-external-memory} شما می توانید اطلاعات موقت تخلیه به دیسک را قادر به محدود کردن استفاده از حافظه در طول `GROUP BY`. -این [ا\_فزون\_بر\_گونهی\_گونهی زیر\_گروهها](../../../operations/settings/settings.md#settings-max_bytes_before_external_group_by) تنظیم تعیین کننده مصرف رم را برای تخلیه می کند `GROUP BY` اطلاعات موقت به سیستم فایل. اگر به 0 (به طور پیش فرض), غیر فعال است. +این [ا_فزون_بر_گونهی_گونهی زیر_گروهها](../../../operations/settings/settings.md#settings-max_bytes_before_external_group_by) تنظیم تعیین کننده مصرف رم را برای تخلیه می کند `GROUP BY` اطلاعات موقت به سیستم فایل. اگر به 0 (به طور پیش فرض), غیر فعال است. هنگام استفاده از `max_bytes_before_external_group_by`, توصیه می کنیم که به شما در تنظیم `max_memory_usage` در مورد دو برابر بالا. این لازم است زیرا دو مرحله برای تجمع وجود دارد: خواندن داده ها و تشکیل داده های متوسط (1) و ادغام داده های متوسط (2). واژگون اطلاعات به سیستم فایل تنها می تواند در طول مرحله رخ می دهد 1. اگر داده های موقت ریخته نمی شد, سپس مرحله 2 ممکن است نیاز به همان مقدار از حافظه در مرحله 1. -برای مثال اگر [\_کاساژ بیشینه](../../../operations/settings/settings.md#settings_max_memory_usage) به 1000000000 تنظیم شد و شما می خواهید به استفاده از تجمع خارجی, این را حس می کند به مجموعه `max_bytes_before_external_group_by` به 10000000000 و `max_memory_usage` به 20000000000. هنگامی که تجمع خارجی باعث شده است (اگر حداقل یک روگرفت از داده های موقت وجود دارد), حداکثر مصرف رم تنها کمی بیشتر از `max_bytes_before_external_group_by`. +برای مثال اگر [_کاساژ بیشینه](../../../operations/settings/settings.md#settings_max_memory_usage) به 1000000000 تنظیم شد و شما می خواهید به استفاده از تجمع خارجی, این را حس می کند به مجموعه `max_bytes_before_external_group_by` به 10000000000 و `max_memory_usage` به 20000000000. هنگامی که تجمع خارجی باعث شده است (اگر حداقل یک روگرفت از داده های موقت وجود دارد), حداکثر مصرف رم تنها کمی بیشتر از `max_bytes_before_external_group_by`. با پردازش پرس و جو توزیع, تجمع خارجی بر روی سرور از راه دور انجام. به منظور سرور درخواست به استفاده از تنها مقدار کمی از رم, تنظیم `distributed_aggregation_memory_efficient` به 1. diff --git a/docs/fa/sql-reference/statements/select/join.md b/docs/fa/sql-reference/statements/select/join.md index c77049fb280..e8f516448f8 100644 --- a/docs/fa/sql-reference/statements/select/join.md +++ b/docs/fa/sql-reference/statements/select/join.md @@ -139,10 +139,10 @@ USING (equi_column1, ... equi_columnN, asof_column) اگر شما نیاز به محدود کردن پیوستن به مصرف حافظه عملیات استفاده از تنظیمات زیر: -- [\_پاک کردن \_روشن گرافیک](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) — Limits number of rows in the hash table. -- [\_پویش همیشگی](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) — Limits size of the hash table. +- [_پاک کردن _روشن گرافیک](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) — Limits number of rows in the hash table. +- [_پویش همیشگی](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) — Limits size of the hash table. -هنگامی که هر یک از این محدودیت رسیده است, کلیک به عنوان عمل می کند [\_شروع مجدد](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) تنظیم دستور. +هنگامی که هر یک از این محدودیت رسیده است, کلیک به عنوان عمل می کند [_شروع مجدد](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) تنظیم دستور. ## مثالها {#examples} diff --git a/docs/fa/sql-reference/statements/system.md b/docs/fa/sql-reference/statements/system.md index 386bf982235..892e9d44f5b 100644 --- a/docs/fa/sql-reference/statements/system.md +++ b/docs/fa/sql-reference/statements/system.md @@ -24,12 +24,12 @@ toc_title: SYSTEM ## RELOAD DICTIONARIES {#query_language-system-reload-dictionaries} بارگذاری مجدد تمام لغت نامه که با موفقیت قبل از لود شده است. -به طور پیش فرض, لغت نامه ها به صورت تنبلی لود (دیدن [\_بارگیری کامل](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load)), بنابراین به جای اینکه به طور خودکار در هنگام راه اندازی لود, در اولین دسترسی از طریق تابع دیکته مقداردهی اولیه و یا از جداول با موتور = فرهنگ لغت را انتخاب کنید. این `SYSTEM RELOAD DICTIONARIES` پرس و جو بارگذاری مجدد از جمله لغت نامه (لود شده). +به طور پیش فرض, لغت نامه ها به صورت تنبلی لود (دیدن [_بارگیری کامل](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load)), بنابراین به جای اینکه به طور خودکار در هنگام راه اندازی لود, در اولین دسترسی از طریق تابع دیکته مقداردهی اولیه و یا از جداول با موتور = فرهنگ لغت را انتخاب کنید. این `SYSTEM RELOAD DICTIONARIES` پرس و جو بارگذاری مجدد از جمله لغت نامه (لود شده). همیشه باز می گردد `Ok.` صرف نظر از نتیجه به روز رسانی فرهنگ لغت. ## بازخوانی لغتنامهها {#query_language-system-reload-dictionary} -به طور کامل یک فرهنگ لغت را دوباره بارگذاری کنید `dictionary_name` بدون در نظر گرفتن دولت از فرهنگ لغت (لود / NOT\_LOADED / شکست خورده). +به طور کامل یک فرهنگ لغت را دوباره بارگذاری کنید `dictionary_name` بدون در نظر گرفتن دولت از فرهنگ لغت (لود / NOT_LOADED / شکست خورده). همیشه باز می گردد `Ok.` صرف نظر از نتیجه به روز رسانی فرهنگ لغت. وضعیت فرهنگ لغت را می توان با پرس و جو بررسی کرد `system.dictionaries` جدول @@ -41,7 +41,7 @@ SELECT name, status FROM system.dictionaries; کش دی ان اس داخلی بازنشانی را کلیک کنید. گاهی اوقات (برای ClickHouse نسخه) لازم است برای استفاده از این دستور هنگامی که در حال تغییر زیرساخت ها (تغییر آدرس IP دیگر ClickHouse سرور یا سرور استفاده شده توسط لغت نامه). -برای راحت تر (اتوماتیک) مدیریت کش دیدن disable\_internal\_dns\_cache, dns\_cache\_update\_period پارامترهای. +برای راحت تر (اتوماتیک) مدیریت کش دیدن disable_internal_dns_cache, dns_cache_update_period پارامترهای. ## DROP MARK CACHE {#query_language-system-drop-mark-cache} @@ -49,7 +49,7 @@ SELECT name, status FROM system.dictionaries; ## FLUSH LOGS {#query_language-system-flush_logs} -Flushes buffers of log messages to system tables (e.g. system.query\_log). Allows you to not wait 7.5 seconds when debugging. +Flushes buffers of log messages to system tables (e.g. system.query_log). Allows you to not wait 7.5 seconds when debugging. ## RELOAD CONFIG {#query_language-system-reload-config} diff --git a/docs/fa/sql-reference/syntax.md b/docs/fa/sql-reference/syntax.md index 795f8302961..e5b7cdd4fdd 100644 --- a/docs/fa/sql-reference/syntax.md +++ b/docs/fa/sql-reference/syntax.md @@ -17,7 +17,7 @@ INSERT INTO t VALUES (1, 'Hello, world'), (2, 'abc'), (3, 'def') این `INSERT INTO t VALUES` قطعه توسط تجزیه کننده کامل و داده ها تجزیه می شود `(1, 'Hello, world'), (2, 'abc'), (3, 'def')` توسط تجزیه کننده جریان سریع تجزیه می شود. شما همچنین می توانید تجزیه کننده کامل برای داده ها با استفاده از [در حال خواندن:](../operations/settings/settings.md#settings-input_format_values_interpret_expressions) تنظیمات. چه زمانی `input_format_values_interpret_expressions = 1` کلیک هاوس اول سعی می کند به تجزیه ارزش با تجزیه کننده جریان سریع. در صورت عدم موفقیت کلیک هاوس تلاش می کند تا از تجزیه کننده کامل برای داده ها استفاده کند و مانند یک مربع درمان شود [عبارت](#syntax-expressions). -داده ها می توانند هر فرمت داشته باشند. هنگامی که یک پرس و جو را دریافت کرده است, سرور محاسبه بیش از [بیشینه\_کرکی\_سیز](../operations/settings/settings.md#settings-max_query_size) بایت از درخواست در رم (به طور پیش فرض, 1 مگابایت), و بقیه جریان تجزیه. +داده ها می توانند هر فرمت داشته باشند. هنگامی که یک پرس و جو را دریافت کرده است, سرور محاسبه بیش از [بیشینه_کرکی_سیز](../operations/settings/settings.md#settings-max_query_size) بایت از درخواست در رم (به طور پیش فرض, 1 مگابایت), و بقیه جریان تجزیه. این اجازه می دهد تا برای اجتناب از مشکلات بزرگ `INSERT` نمایش داده شد. هنگام استفاده از `Values` قالب در یک `INSERT` پرس و جو, ممکن است به نظر می رسد که داده ها همان عبارات در تجزیه `SELECT` پرس و جو, اما این درست نیست. این `Values` فرمت بسیار محدود تر است. diff --git a/docs/fa/sql-reference/table-functions/file.md b/docs/fa/sql-reference/table-functions/file.md index 7bb676670a4..8a11d4ce3e6 100644 --- a/docs/fa/sql-reference/table-functions/file.md +++ b/docs/fa/sql-reference/table-functions/file.md @@ -15,7 +15,7 @@ file(path, format, structure) **پارامترهای ورودی** -- `path` — The relative path to the file from [\_مخفی کردن \_صفحه](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). مسیر به فایل پشتیبانی پس از دل تنگی در حالت فقط خواندنی: `*`, `?`, `{abc,def}` و `{N..M}` کجا `N`, `M` — numbers, \``'abc', 'def'` — strings. +- `path` — The relative path to the file from [_مخفی کردن _صفحه](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). مسیر به فایل پشتیبانی پس از دل تنگی در حالت فقط خواندنی: `*`, `?`, `{abc,def}` و `{N..M}` کجا `N`, `M` — numbers, \``'abc', 'def'` — strings. - `format` — The [قالب](../../interfaces/formats.md#formats) پرونده - `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. @@ -72,12 +72,12 @@ SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 U 1. فرض کنید ما چندین فایل با مسیرهای نسبی زیر داریم: -- ‘some\_dir/some\_file\_1’ -- ‘some\_dir/some\_file\_2’ -- ‘some\_dir/some\_file\_3’ -- ‘another\_dir/some\_file\_1’ -- ‘another\_dir/some\_file\_2’ -- ‘another\_dir/some\_file\_3’ +- ‘some_dir/some_file_1’ +- ‘some_dir/some_file_2’ +- ‘some_dir/some_file_3’ +- ‘another_dir/some_file_1’ +- ‘another_dir/some_file_2’ +- ‘another_dir/some_file_3’ 1. پرس و جو مقدار ردیف در این فایل ها: diff --git a/docs/fa/sql-reference/table-functions/hdfs.md b/docs/fa/sql-reference/table-functions/hdfs.md index ae42eda5165..367b11110da 100644 --- a/docs/fa/sql-reference/table-functions/hdfs.md +++ b/docs/fa/sql-reference/table-functions/hdfs.md @@ -55,12 +55,12 @@ LIMIT 2 1. فرض کنید که ما چندین فایل با اوریس زیر در اچ دی ها داریم: -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_3’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_3’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_3’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_3’ 1. پرس و جو مقدار ردیف در این فایل ها: diff --git a/docs/fa/sql-reference/table-functions/index.md b/docs/fa/sql-reference/table-functions/index.md index 0ed98d139b2..e0df309737c 100644 --- a/docs/fa/sql-reference/table-functions/index.md +++ b/docs/fa/sql-reference/table-functions/index.md @@ -16,12 +16,12 @@ toc_title: "\u0645\u0639\u0631\u0641\u06CC \u0634\u0631\u06A9\u062A" The method for creating a temporary table that is available only in the current query. The table is deleted when the query finishes. -- [ایجاد جدول به عنوان \](../statements/create.md#create-table-query) پرس و جو. +- [ایجاد جدول به عنوان \](../statements/create.md#create-table-query) پرس و جو. It's one of the methods of creating a table. !!! warning "اخطار" - شما می توانید توابع جدول اگر استفاده نمی [اجازه دادن به \_نشانی](../../operations/settings/permissions-for-queries.md#settings_allow_ddl) تنظیم غیر فعال است. + شما می توانید توابع جدول اگر استفاده نمی [اجازه دادن به _نشانی](../../operations/settings/permissions-for-queries.md#settings_allow_ddl) تنظیم غیر فعال است. | تابع | توصیف | |-----------------------|------------------------------------------------------------------------------------------------------------------------------------------------| diff --git a/docs/fa/sql-reference/table-functions/remote.md b/docs/fa/sql-reference/table-functions/remote.md index dd6e6725be4..c28d05062eb 100644 --- a/docs/fa/sql-reference/table-functions/remote.md +++ b/docs/fa/sql-reference/table-functions/remote.md @@ -56,7 +56,7 @@ example01-{01..02}-1 اگر شما جفت های متعدد از براکت در اشکال مختلف, این تولید محصول مستقیم از مجموعه مربوطه. -نشانی ها و بخش هایی از نشانی در براکت فرفری را می توان با نماد لوله جدا (\|). در این مورد, مجموعه مربوطه را از نشانی ها به عنوان کپی تفسیر, و پرس و جو خواهد شد به اولین ماکت سالم ارسال. با این حال, کپی در نظم در حال حاضر در مجموعه تکرار [\_تبالسازی](../../operations/settings/settings.md) تنظیمات. +نشانی ها و بخش هایی از نشانی در براکت فرفری را می توان با نماد لوله جدا (\|). در این مورد, مجموعه مربوطه را از نشانی ها به عنوان کپی تفسیر, و پرس و جو خواهد شد به اولین ماکت سالم ارسال. با این حال, کپی در نظم در حال حاضر در مجموعه تکرار [_تبالسازی](../../operations/settings/settings.md) تنظیمات. مثال: @@ -80,6 +80,6 @@ example01-{01..02}-{1|2} اگر کاربر مشخص نشده است, `default` استفاده شده است. اگر رمز عبور مشخص نشده است, رمز عبور خالی استفاده شده است. -`remoteSecure` - مثل `remote` but with secured connection. Default port — [\_شروع مجدد](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) از پیکربندی و یا 9440. +`remoteSecure` - مثل `remote` but with secured connection. Default port — [_شروع مجدد](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) از پیکربندی و یا 9440. [مقاله اصلی](https://clickhouse.tech/docs/en/query_language/table_functions/remote/) diff --git a/docs/fa/whats-new/security-changelog.md b/docs/fa/whats-new/security-changelog.md index 2b0a0bcf846..97777e79e8e 100644 --- a/docs/fa/whats-new/security-changelog.md +++ b/docs/fa/whats-new/security-changelog.md @@ -54,7 +54,7 @@ unixODBC اجازه بارگذاری دلخواه اشیاء مشترک از ف ### CVE-2018-14668 {#cve-2018-14668} -“remote” تابع جدول اجازه نمادهای دلخواه در “user”, “password” و “default\_database” زمینه های که منجر به عبور از پروتکل درخواست حملات جعل. +“remote” تابع جدول اجازه نمادهای دلخواه در “user”, “password” و “default_database” زمینه های که منجر به عبور از پروتکل درخواست حملات جعل. اعتبار: اندری کراسیچکوف از تیم امنیت اطلاعات یاندکس diff --git a/docs/fr/development/architecture.md b/docs/fr/development/architecture.md index 97de3c8d1ce..7c105d430f6 100644 --- a/docs/fr/development/architecture.md +++ b/docs/fr/development/architecture.md @@ -47,7 +47,7 @@ A `Block` est un conteneur qui représente un sous-ensemble (morceau) d'une tabl Lorsque nous calculons une fonction sur des colonnes dans un bloc, nous ajoutons une autre colonne avec son résultat au bloc, et nous ne touchons pas les colonnes pour les arguments de la fonction car les opérations sont immuables. Plus tard, les colonnes inutiles peuvent être supprimées du bloc, mais pas modifiées. Il est pratique pour l'élimination des sous-expressions communes. -Des blocs sont créés pour chaque bloc de données traité. Notez que pour le même type de calcul, les noms et les types de colonnes restent les mêmes pour différents blocs, et seules les données de colonne changent. Il est préférable de diviser les données de bloc de l'en-tête de bloc car les petites tailles de Bloc ont une surcharge élevée de chaînes temporaires pour copier shared\_ptrs et les noms de colonnes. +Des blocs sont créés pour chaque bloc de données traité. Notez que pour le même type de calcul, les noms et les types de colonnes restent les mêmes pour différents blocs, et seules les données de colonne changent. Il est préférable de diviser les données de bloc de l'en-tête de bloc car les petites tailles de Bloc ont une surcharge élevée de chaînes temporaires pour copier shared_ptrs et les noms de colonnes. ## Bloquer Les Flux {#block-streams} diff --git a/docs/fr/development/build.md b/docs/fr/development/build.md index d05f0999720..0d2713aaaa5 100644 --- a/docs/fr/development/build.md +++ b/docs/fr/development/build.md @@ -9,7 +9,7 @@ toc_title: Comment Construire ClickHouse sur Linux Le tutoriel suivant est basé sur le système Linux Ubuntu. Avec les modifications appropriées, il devrait également fonctionner sur toute autre distribution Linux. -Plates-formes prises en charge: x86\_64 et AArch64. La prise en charge de Power9 est expérimentale. +Plates-formes prises en charge: x86_64 et AArch64. La prise en charge de Power9 est expérimentale. ## Installez Git, CMake, Python et Ninja {#install-git-cmake-python-and-ninja} diff --git a/docs/fr/development/contrib.md b/docs/fr/development/contrib.md index 52eac7a71ac..f4006d0a787 100644 --- a/docs/fr/development/contrib.md +++ b/docs/fr/development/contrib.md @@ -28,7 +28,7 @@ toc_title: "Biblioth\xE8ques Tierces Utilis\xE9es" | libpcg-aléatoire | [Licence Apache 2.0](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libpcg-random/LICENSE-APACHE.txt) | | libressl | [Licence OpenSSL](https://github.com/ClickHouse-Extras/ssl/blob/master/COPYING) | | librdkafka | [Licence BSD 2-Clause](https://github.com/edenhill/librdkafka/blob/363dcad5a23dc29381cc626620e68ae418b3af19/LICENSE) | -| libwidechar\_width | [CC0 1.0 universel](https://github.com/ClickHouse/ClickHouse/blob/master/libs/libwidechar_width/LICENSE) | +| libwidechar_width | [CC0 1.0 universel](https://github.com/ClickHouse/ClickHouse/blob/master/libs/libwidechar_width/LICENSE) | | llvm | [Licence BSD 3-Clause](https://github.com/ClickHouse-Extras/llvm/blob/163def217817c90fb982a6daf384744d8472b92b/llvm/LICENSE.TXT) | | lz4 | [Licence BSD 2-Clause](https://github.com/lz4/lz4/blob/c10863b98e1503af90616ae99725ecd120265dfb/LICENSE) | | mariadb-connecteur-c | [LGPL v2.1](https://github.com/ClickHouse-Extras/mariadb-connector-c/blob/3.1/COPYING.LIB) | diff --git a/docs/fr/development/developer-instruction.md b/docs/fr/development/developer-instruction.md index 8d549d3a670..576188941bd 100644 --- a/docs/fr/development/developer-instruction.md +++ b/docs/fr/development/developer-instruction.md @@ -44,7 +44,7 @@ Dans le terminal de ligne de commande exécuter: git clone --recursive git@github.com:your_github_username/ClickHouse.git cd ClickHouse -Remarque: Veuillez remplacer *your\_github\_username* avec ce qui est approprié! +Remarque: Veuillez remplacer *your_github_username* avec ce qui est approprié! Cette commande va créer un répertoire `ClickHouse` contenant la copie de travail du projet. @@ -154,7 +154,7 @@ Maintenant que vous êtes prêt à construire ClickHouse nous vous conseillons d mkdir build cd build -Vous pouvez avoir plusieurs répertoires différents (build\_release, build\_debug, etc.) pour les différents types de construction. +Vous pouvez avoir plusieurs répertoires différents (build_release, build_debug, etc.) pour les différents types de construction. Tandis qu'à l'intérieur de la `build` répertoire, configurez votre build en exécutant CMake. Avant la première exécution, vous devez définir des variables d'environnement qui spécifient le compilateur (compilateur gcc version 9 dans cet exemple). diff --git a/docs/fr/development/style.md b/docs/fr/development/style.md index 9a0035ce41e..1259962b577 100644 --- a/docs/fr/development/style.md +++ b/docs/fr/development/style.md @@ -356,7 +356,7 @@ Dans tous les autres cas, utilisez un nom qui décrit la signification. bool info_successfully_loaded = false; ``` -**9.** Les noms de `define`les constantes s et globales utilisent ALL\_CAPS avec des traits de soulignement. +**9.** Les noms de `define`les constantes s et globales utilisent ALL_CAPS avec des traits de soulignement. ``` cpp #define MAX_SRC_TABLE_NAMES_TO_STORE 1000 @@ -396,7 +396,7 @@ Le suffixe de soulignement peut être omis si l'argument n'est pas utilisé dans timer (not m_timer) ``` -**14.** Pour les constantes dans un `enum`, utilisez CamelCase avec une lettre majuscule. ALL\_CAPS est également acceptable. Si l' `enum` est non local, utilisez un `enum class`. +**14.** Pour les constantes dans un `enum`, utilisez CamelCase avec une lettre majuscule. ALL_CAPS est également acceptable. Si l' `enum` est non local, utilisez un `enum class`. ``` cpp enum class CompressionMethod @@ -709,7 +709,7 @@ La bibliothèque standard est utilisée (`libc++`). **4.**OS: Linux Ubuntu, pas plus vieux que précis. -**5.**Le Code est écrit pour l'architecture CPU x86\_64. +**5.**Le Code est écrit pour l'architecture CPU x86_64. Le jeu D'instructions CPU est l'ensemble minimum pris en charge parmi nos serveurs. Actuellement, il s'agit de SSE 4.2. diff --git a/docs/fr/development/tests.md b/docs/fr/development/tests.md index 4664f533c0a..5fd5cd36e2d 100644 --- a/docs/fr/development/tests.md +++ b/docs/fr/development/tests.md @@ -202,7 +202,7 @@ Version de débogage de `jemalloc` est utilisé pour la construction de débogag Clickhouse fuzzing est implémenté à la fois en utilisant [libFuzzer](https://llvm.org/docs/LibFuzzer.html) et des requêtes SQL aléatoires. Tous les tests de fuzz doivent être effectués avec des désinfectants (adresse et indéfini). -LibFuzzer est utilisé pour les tests de fuzz isolés du code de la bibliothèque. Les Fuzzers sont implémentés dans le cadre du code de test et ont “\_fuzzer” nom postfixes. +LibFuzzer est utilisé pour les tests de fuzz isolés du code de la bibliothèque. Les Fuzzers sont implémentés dans le cadre du code de test et ont “_fuzzer” nom postfixes. Exemple Fuzzer peut être trouvé à `src/Parsers/tests/lexer_fuzzer.cpp`. Les configs, dictionnaires et corpus spécifiques à LibFuzzer sont stockés à `tests/fuzz`. Nous vous encourageons à écrire des tests fuzz pour chaque fonctionnalité qui gère l'entrée de l'utilisateur. diff --git a/docs/fr/engines/table-engines/index.md b/docs/fr/engines/table-engines/index.md index a05ab19868c..51654fd6bb2 100644 --- a/docs/fr/engines/table-engines/index.md +++ b/docs/fr/engines/table-engines/index.md @@ -62,7 +62,7 @@ Moteurs dans la famille: - [Distribué](special/distributed.md#distributed) - [MaterializedView](special/materializedview.md#materializedview) - [Dictionnaire](special/dictionary.md#dictionary) -- \[Fusion\](spécial/de fusion.md\#fusion +- \[Fusion\](spécial/de fusion.md#fusion - [Fichier](special/file.md#file) - [NULL](special/null.md#null) - [Définir](special/set.md#set) diff --git a/docs/fr/engines/table-engines/integrations/hdfs.md b/docs/fr/engines/table-engines/integrations/hdfs.md index fd207793b02..7416ed7ba91 100644 --- a/docs/fr/engines/table-engines/integrations/hdfs.md +++ b/docs/fr/engines/table-engines/integrations/hdfs.md @@ -73,12 +73,12 @@ Les Constructions avec `{}` sont similaires à l' [distant](../../../sql-referen 1. Supposons que nous ayons plusieurs fichiers au format TSV avec les URI suivants sur HDFS: -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_3’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_3’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_3’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_3’ 1. Il y a plusieurs façons de faire une table composée des six fichiers: diff --git a/docs/fr/engines/table-engines/integrations/kafka.md b/docs/fr/engines/table-engines/integrations/kafka.md index d4a2de42f22..c8c16f2c3c4 100644 --- a/docs/fr/engines/table-engines/integrations/kafka.md +++ b/docs/fr/engines/table-engines/integrations/kafka.md @@ -134,7 +134,7 @@ Exemple: SELECT level, sum(total) FROM daily GROUP BY level; ``` -Pour améliorer les performances, les messages reçus sont regroupées en blocs de la taille de [max\_insert\_block\_size](../../../operations/server-configuration-parameters/settings.md#settings-max_insert_block_size). Si le bloc n'a pas été formé à l'intérieur [stream\_flush\_interval\_ms](../../../operations/server-configuration-parameters/settings.md) millisecondes, les données seront vidées dans le tableau, indépendamment de l'intégralité du bloc. +Pour améliorer les performances, les messages reçus sont regroupées en blocs de la taille de [max_insert_block_size](../../../operations/server-configuration-parameters/settings.md#settings-max_insert_block_size). Si le bloc n'a pas été formé à l'intérieur [stream_flush_interval_ms](../../../operations/server-configuration-parameters/settings.md) millisecondes, les données seront vidées dans le tableau, indépendamment de l'intégralité du bloc. Pour arrêter de recevoir des données de rubrique ou pour modifier la logique de conversion, détachez la vue matérialisée: diff --git a/docs/fr/engines/table-engines/mergetree-family/custom-partitioning-key.md b/docs/fr/engines/table-engines/mergetree-family/custom-partitioning-key.md index 3617655adb1..2fa80236adc 100644 --- a/docs/fr/engines/table-engines/mergetree-family/custom-partitioning-key.md +++ b/docs/fr/engines/table-engines/mergetree-family/custom-partitioning-key.md @@ -116,7 +116,7 @@ drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 5 12:09 201902_4_6_1 drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 1 16:48 detached ``` -Dossier ‘201901\_1\_1\_0’, ‘201901\_1\_7\_1’ et ainsi de suite sont les répertoires des parties. Chaque partie se rapporte à une partition correspondante et contient des données juste pour un certain mois (la table dans cet exemple a partitionnement par mois). +Dossier ‘201901_1_1_0’, ‘201901_1_7_1’ et ainsi de suite sont les répertoires des parties. Chaque partie se rapporte à une partition correspondante et contient des données juste pour un certain mois (la table dans cet exemple a partitionnement par mois). Le `detached` le répertoire contient des parties qui ont été détachées de la table à l'aide [DETACH](../../../sql-reference/statements/alter.md#alter_detach-partition) requête. Les parties corrompues sont également déplacées dans ce répertoire, au lieu d'être supprimées. Le serveur n'utilise pas les pièces de la `detached` directory. You can add, delete, or modify the data in this directory at any time – the server will not know about this until you run the [ATTACH](../../../sql-reference/statements/alter.md#alter_attach-partition) requête. diff --git a/docs/fr/engines/table-engines/mergetree-family/graphitemergetree.md b/docs/fr/engines/table-engines/mergetree-family/graphitemergetree.md index d1dc5e64a4f..03fae8ea261 100644 --- a/docs/fr/engines/table-engines/mergetree-family/graphitemergetree.md +++ b/docs/fr/engines/table-engines/mergetree-family/graphitemergetree.md @@ -81,7 +81,7 @@ Tous les paramètres excepté `config_section` ont la même signification que da ## Configuration De Cumul {#rollup-configuration} -Les paramètres de cumul sont définis par [graphite\_rollup](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) paramètre dans la configuration du serveur. Le nom du paramètre pourrait être tout. Vous pouvez créer plusieurs configurations et les utiliser pour différentes tables. +Les paramètres de cumul sont définis par [graphite_rollup](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) paramètre dans la configuration du serveur. Le nom du paramètre pourrait être tout. Vous pouvez créer plusieurs configurations et les utiliser pour différentes tables. Structure de configuration de cumul: diff --git a/docs/fr/engines/table-engines/mergetree-family/mergetree.md b/docs/fr/engines/table-engines/mergetree-family/mergetree.md index 3906eabfc47..475682bfc93 100644 --- a/docs/fr/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/fr/engines/table-engines/mergetree-family/mergetree.md @@ -239,7 +239,7 @@ Dans l'exemple ci-dessous, l'index ne peut pas être utilisé. SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' ``` -Pour vérifier si ClickHouse pouvez utiliser l'index lors de l'exécution d'une requête, utilisez les paramètres [force\_index\_by\_date](../../../operations/settings/settings.md#settings-force_index_by_date) et [force\_primary\_key](../../../operations/settings/settings.md). +Pour vérifier si ClickHouse pouvez utiliser l'index lors de l'exécution d'une requête, utilisez les paramètres [force_index_by_date](../../../operations/settings/settings.md#settings-force_index_by_date) et [force_primary_key](../../../operations/settings/settings.md). La clé de partitionnement par mois permet de lire uniquement les blocs de données qui contiennent des dates de la plage appropriée. Dans ce cas, le bloc de données peut contenir des données pour plusieurs dates (jusqu'à un mois entier). Dans un bloc, les données sont triées par clé primaire, qui peut ne pas contenir la date comme première colonne. Pour cette raison, l'utilisation d'une requête avec seulement une condition de date qui ne spécifie pas le préfixe de clé primaire entraînera la lecture de plus de données que pour une seule date. @@ -330,7 +330,7 @@ Les Conditions dans le `WHERE` la clause contient des appels des fonctions qui f Le `set` l'indice peut être utilisé avec toutes les fonctions. Les sous-ensembles de fonctions pour les autres index sont présentés dans le tableau ci-dessous. -| Fonction (opérateur) / Indice de | clé primaire | minmax | ngrambf\_v1 | tokenbf\_v1 | bloom\_filter | +| Fonction (opérateur) / Indice de | clé primaire | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter | |--------------------------------------------------------------------------------------------------------------|--------------|--------|-------------|-------------|---------------| | [égal (=, ==)](../../../sql-reference/functions/comparison-functions.md#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | | [notEquals (!=, \<\>)](../../../sql-reference/functions/comparison-functions.md#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | @@ -495,7 +495,7 @@ La partie de données est l'unité mobile minimum pour `MergeTree`-tables de mot - Volume — Ordered set of equal disks (similar to [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures)). - Storage policy — Set of volumes and the rules for moving data between them. -Les noms donnés aux entités décrites peuvent être trouvés dans les tables système, [système.storage\_policies](../../../operations/system-tables.md#system_tables-storage_policies) et [système.disque](../../../operations/system-tables.md#system_tables-disks). Pour appliquer l'une des stratégies de stockage configurées pour une table, utilisez `storage_policy` réglage de `MergeTree`-moteur de table de famille. +Les noms donnés aux entités décrites peuvent être trouvés dans les tables système, [système.storage_policies](../../../operations/system-tables.md#system_tables-storage_policies) et [système.disque](../../../operations/system-tables.md#system_tables-disks). Pour appliquer l'une des stratégies de stockage configurées pour une table, utilisez `storage_policy` réglage de `MergeTree`-moteur de table de famille. ### Configuration {#table_engine-mergetree-multiple-volumes_configure} @@ -642,7 +642,7 @@ Dans tous ces cas, à l'exception des mutations et du gel de partition, une piè Sous le capot, les mutations et la congélation des cloisons utilisent [des liens en dur](https://en.wikipedia.org/wiki/Hard_link). Les liens durs entre différents disques ne sont pas pris en charge, donc dans de tels cas, les pièces résultantes sont stockées sur les mêmes disques que les disques initiaux. En arrière - plan, les pièces sont déplacées entre les volumes en fonction de la quantité d'espace libre (`move_factor` paramètre) selon l'ordre les volumes sont déclarées dans le fichier de configuration. -Les données ne sont jamais transférées du dernier et dans le premier. On peut utiliser des tables système [système.part\_log](../../../operations/system-tables.md#system_tables-part-log) (champ `type = MOVE_PART`) et [système.partie](../../../operations/system-tables.md#system_tables-parts) (Fields `path` et `disk`) pour surveiller l'arrière-plan se déplace. Aussi, les informations détaillées peuvent être trouvées dans les journaux du serveur. +Les données ne sont jamais transférées du dernier et dans le premier. On peut utiliser des tables système [système.part_log](../../../operations/system-tables.md#system_tables-part-log) (champ `type = MOVE_PART`) et [système.partie](../../../operations/system-tables.md#system_tables-parts) (Fields `path` et `disk`) pour surveiller l'arrière-plan se déplace. Aussi, les informations détaillées peuvent être trouvées dans les journaux du serveur. L'utilisateur peut forcer le déplacement d'une partie ou d'une partition d'un volume à l'autre à l'aide de la requête [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](../../../sql-reference/statements/alter.md#alter_move-partition), toutes les restrictions pour les opérations de fond sont prises en compte. La requête initie un mouvement seul et n'attend pas que les opérations d'arrière-plan soient terminées. L'utilisateur recevra un message d'erreur si pas assez d'espace libre est disponible ou si l'une des conditions requises ne sont pas remplies. diff --git a/docs/fr/engines/table-engines/mergetree-family/replication.md b/docs/fr/engines/table-engines/mergetree-family/replication.md index 49690204869..1e74475ee79 100644 --- a/docs/fr/engines/table-engines/mergetree-family/replication.md +++ b/docs/fr/engines/table-engines/mergetree-family/replication.md @@ -59,7 +59,7 @@ Vous pouvez spécifier N'importe quel cluster Zookeeper existant et le système Si ZooKeeper n'est pas défini dans le fichier de configuration, vous ne pouvez pas créer de tables répliquées et toutes les tables répliquées existantes seront en lecture seule. -La gardienne n'est pas utilisé dans `SELECT` requêtes car la réplication n'affecte pas les performances de `SELECT` et les requêtes s'exécutent aussi vite que pour les tables non répliquées. Lors de l'interrogation de tables répliquées distribuées, le comportement de ClickHouse est contrôlé par les paramètres [max\_replica\_delay\_for\_distributed\_queries](../../../operations/settings/settings.md#settings-max_replica_delay_for_distributed_queries) et [fallback\_to\_stale\_replicas\_for\_distributed\_queries](../../../operations/settings/settings.md#settings-fallback_to_stale_replicas_for_distributed_queries). +La gardienne n'est pas utilisé dans `SELECT` requêtes car la réplication n'affecte pas les performances de `SELECT` et les requêtes s'exécutent aussi vite que pour les tables non répliquées. Lors de l'interrogation de tables répliquées distribuées, le comportement de ClickHouse est contrôlé par les paramètres [max_replica_delay_for_distributed_queries](../../../operations/settings/settings.md#settings-max_replica_delay_for_distributed_queries) et [fallback_to_stale_replicas_for_distributed_queries](../../../operations/settings/settings.md#settings-fallback_to_stale_replicas_for_distributed_queries). Pour chaque `INSERT` requête, environ dix entrées sont ajoutées à ZooKeeper par le biais de plusieurs transactions. (Pour être plus précis, c'est pour chaque bloc de données inséré; une requête D'insertion contient un bloc ou un bloc par `max_insert_block_size = 1048576` rangée.) Cela conduit à des latences légèrement plus longues pour `INSERT` par rapport aux tables non répliquées. Mais si vous suivez les recommandations pour insérer des données dans des lots de pas plus d'un `INSERT` par seconde, cela ne crée aucun problème. L'ensemble du cluster clickhouse utilisé pour coordonner un cluster ZooKeeper a un total de plusieurs centaines `INSERTs` par seconde. Le débit sur les insertions de données (le nombre de lignes par seconde) est aussi élevé que pour les non-données répliquées. @@ -71,7 +71,7 @@ Par défaut, une requête INSERT attend la confirmation de l'écriture des donn Chaque bloc de données est écrit de manière atomique. La requête D'insertion est divisée en blocs jusqu'à `max_insert_block_size = 1048576` rangée. En d'autres termes, si l' `INSERT` la requête a moins de 1048576 lignes, elle est faite de manière atomique. -Les blocs de données sont dédupliquées. Pour plusieurs écritures du même bloc de données (blocs de données de même taille contenant les mêmes lignes dans le même ordre), le bloc n'est écrit qu'une seule fois. La raison en est en cas de défaillance du réseau lorsque l'application cliente ne sait pas si les données ont été écrites dans la base de données, de sorte que le `INSERT` requête peut simplement être répété. Peu importe à quelles insertions de réplica ont été envoyées avec des données identiques. `INSERTs` sont idempotents. Les paramètres de déduplication sont contrôlés par [merge\_tree](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-merge_tree) les paramètres du serveur. +Les blocs de données sont dédupliquées. Pour plusieurs écritures du même bloc de données (blocs de données de même taille contenant les mêmes lignes dans le même ordre), le bloc n'est écrit qu'une seule fois. La raison en est en cas de défaillance du réseau lorsque l'application cliente ne sait pas si les données ont été écrites dans la base de données, de sorte que le `INSERT` requête peut simplement être répété. Peu importe à quelles insertions de réplica ont été envoyées avec des données identiques. `INSERTs` sont idempotents. Les paramètres de déduplication sont contrôlés par [merge_tree](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-merge_tree) les paramètres du serveur. Pendant la réplication, seules les données source à insérer sont transférées sur le réseau. D'autres transformations de données (fusion) sont coordonnées et effectuées sur toutes les répliques de la même manière. Cela minimise l'utilisation du réseau, ce qui signifie que la réplication fonctionne bien lorsque les répliques résident dans différents centres de données. (Notez que la duplication de données dans différents centres de données est l'objectif principal de la réplication.) diff --git a/docs/fr/engines/table-engines/special/buffer.md b/docs/fr/engines/table-engines/special/buffer.md index af26b32177b..22fde2d614e 100644 --- a/docs/fr/engines/table-engines/special/buffer.md +++ b/docs/fr/engines/table-engines/special/buffer.md @@ -36,7 +36,7 @@ Exemple: CREATE TABLE merge.hits_buffer AS merge.hits ENGINE = Buffer(merge, hits, 16, 10, 100, 10000, 1000000, 10000000, 100000000) ``` -La création d'un ‘merge.hits\_buffer’ table avec la même structure que ‘merge.hits’ et en utilisant le moteur tampon. Lors de l'écriture dans cette table, les données sont mises en mémoire tampon dans la RAM ‘merge.hits’ table. 16 tampons sont créés. Les données dans chacun d'entre eux est rincé si 100 secondes sont écoulées, ou un million de lignes ont été écrites, ou 100 MO de données ont été écrits; ou si, simultanément, 10 secondes et 10 000 lignes et 10 MO de données ont été écrites. Par exemple, si une ligne a été écrite, après 100 secondes, il sera vidé, n'importe quoi. Mais si plusieurs lignes ont été écrites, les données seront vidées plus tôt. +La création d'un ‘merge.hits_buffer’ table avec la même structure que ‘merge.hits’ et en utilisant le moteur tampon. Lors de l'écriture dans cette table, les données sont mises en mémoire tampon dans la RAM ‘merge.hits’ table. 16 tampons sont créés. Les données dans chacun d'entre eux est rincé si 100 secondes sont écoulées, ou un million de lignes ont été écrites, ou 100 MO de données ont été écrits; ou si, simultanément, 10 secondes et 10 000 lignes et 10 MO de données ont été écrites. Par exemple, si une ligne a été écrite, après 100 secondes, il sera vidé, n'importe quoi. Mais si plusieurs lignes ont été écrites, les données seront vidées plus tôt. Lorsque le serveur est arrêté, avec DROP TABLE ou DETACH TABLE, les données du tampon sont également vidées vers la table de destination. @@ -58,7 +58,7 @@ FINAL et SAMPLE ne fonctionnent pas correctement pour les tables tampon. Ces con Lors de l'ajout de données à un Tampon, un des tampons est verrouillé. Cela entraîne des retards si une opération de lecture est effectuée simultanément à partir de la table. -Les données insérées dans une table tampon peuvent se retrouver dans la table subordonnée dans un ordre différent et dans des blocs différents. Pour cette raison, une table tampon est difficile à utiliser pour écrire correctement dans un CollapsingMergeTree. Pour éviter les problèmes, vous pouvez définir ‘num\_layers’ 1. +Les données insérées dans une table tampon peuvent se retrouver dans la table subordonnée dans un ordre différent et dans des blocs différents. Pour cette raison, une table tampon est difficile à utiliser pour écrire correctement dans un CollapsingMergeTree. Pour éviter les problèmes, vous pouvez définir ‘num_layers’ 1. Si la table de destination est répliquée, certaines caractéristiques attendues des tables répliquées sont perdues lors de l'écriture dans une table tampon. Les modifications aléatoires apportées à l'ordre des lignes et des tailles des parties de données provoquent l'arrêt de la déduplication des données, ce qui signifie qu'il n'est pas possible d'avoir un ‘exactly once’ Ecrire dans des tables répliquées. diff --git a/docs/fr/engines/table-engines/special/distributed.md b/docs/fr/engines/table-engines/special/distributed.md index 566172bdb66..b40253b1b48 100644 --- a/docs/fr/engines/table-engines/special/distributed.md +++ b/docs/fr/engines/table-engines/special/distributed.md @@ -85,7 +85,7 @@ Les noms de Cluster ne doivent pas contenir de points. Paramètre `host`, `port` et , éventuellement, `user`, `password`, `secure`, `compression` sont spécifiés pour chaque serveur: - `host` – The address of the remote server. You can use either the domain or the IPv4 or IPv6 address. If you specify the domain, the server makes a DNS request when it starts, and the result is stored as long as the server is running. If the DNS request fails, the server doesn't start. If you change the DNS record, restart the server. -- `port` – The TCP port for messenger activity (‘tcp\_port’ dans la configuration, généralement définie sur 9000). Ne le confondez pas avec http\_port. +- `port` – The TCP port for messenger activity (‘tcp_port’ dans la configuration, généralement définie sur 9000). Ne le confondez pas avec http_port. - `user` – Name of the user for connecting to a remote server. Default value: default. This user must have access to connect to the specified server. Access is configured in the users.xml file. For more information, see the section [Les droits d'accès](../../../operations/access-rights.md). - `password` – The password for connecting to a remote server (not masked). Default value: empty string. - `secure` - Utilisez ssl pour la connexion, généralement vous devez également définir `port` = 9440. Le serveur doit écouter `9440` et avoir des certificats corrects. @@ -113,13 +113,13 @@ Deuxièmement, vous pouvez effectuer INSERT dans une table distribuée. Dans ce Chaque fragment peut avoir un poids défini dans le fichier de configuration. Par défaut, le poids est égal à un. Les données sont réparties entre les fragments dans la quantité proportionnelle au poids des fragments. Par exemple, si il y a deux tessons et le premier a un poids de 9 tandis que la seconde a un poids de 10, le premier sera envoyé 9 / 19 parties de lignes, et le second sera envoyé 10 / 19. -Chaque fragment peut avoir le ‘internal\_replication’ paramètre défini dans le fichier de configuration. +Chaque fragment peut avoir le ‘internal_replication’ paramètre défini dans le fichier de configuration. Si ce paramètre est défini à ‘true’, l'opération d'écriture sélectionne le premier saine réplique et écrit les données. Utilisez cette option si le tableau Distribué “looks at” tables répliquées. En d'autres termes, si la table où les données seront écrites va répliquer elle-même. Si elle est définie sur ‘false’ (par défaut), les données sont écrites dans toutes les répliques. En substance, cela signifie que la table distribuée réplique les données elle-même. C'est pire que d'utiliser des tables répliquées, car la cohérence des répliques n'est pas vérifiée et, au fil du temps, elles contiendront des données légèrement différentes. -Pour sélectionner le fragment auquel une ligne de données est envoyée, l'expression de sharding est analysée et son reste est extrait de la diviser par le poids total des fragments. La ligne est envoyée au fragment qui correspond au demi-intervalle des restes de ‘prev\_weight’ de ‘prev\_weights + weight’, où ‘prev\_weights’ c'est le poids total des tessons avec le plus petit nombre, et ‘weight’ est le poids de cet éclat. Par exemple, s'il y a deux fragments, et que le premier a un poids de 9 tandis que le second a un poids de 10, la ligne sera envoyée au premier fragment pour les restes de la plage \[0, 9), et au second pour les restes de la plage \[9, 19). +Pour sélectionner le fragment auquel une ligne de données est envoyée, l'expression de sharding est analysée et son reste est extrait de la diviser par le poids total des fragments. La ligne est envoyée au fragment qui correspond au demi-intervalle des restes de ‘prev_weight’ de ‘prev_weights + weight’, où ‘prev_weights’ c'est le poids total des tessons avec le plus petit nombre, et ‘weight’ est le poids de cet éclat. Par exemple, s'il y a deux fragments, et que le premier a un poids de 9 tandis que le second a un poids de 10, la ligne sera envoyée au premier fragment pour les restes de la plage \[0, 9), et au second pour les restes de la plage \[9, 19). L'expression de sharding peut être n'importe quelle expression de constantes et de colonnes de table qui renvoie un entier. Par exemple, vous pouvez utiliser l'expression ‘rand()’ pour la distribution aléatoire des données, ou ‘UserID’ pour la distribution par le reste de la division de L'ID de l'utilisateur (alors les données d'un seul utilisateur résideront sur un seul fragment, ce qui simplifie l'exécution et la jointure par les utilisateurs). Si l'une des colonnes n'est pas assez répartie uniformément, vous pouvez l'envelopper dans une fonction de hachage: intHash64 (UserID). @@ -132,11 +132,11 @@ Vous devriez être préoccupé par le système de sharding dans les cas suivants - Les requêtes sont utilisées qui nécessitent des données de jointure (IN ou JOIN) par une clé spécifique. Si les données sont partagées par cette clé, vous pouvez utiliser local in ou JOIN au lieu de GLOBAL IN ou global JOIN, ce qui est beaucoup plus efficace. - Un grand nombre de serveurs est utilisé (des centaines ou plus) avec un grand nombre de petites requêtes (requêtes de clients individuels - sites Web, annonceurs ou partenaires). Pour que les petites requêtes n'affectent pas l'ensemble du cluster, il est logique de localiser les données d'un seul client sur un seul fragment. Alternativement, comme nous l'avons fait dans Yandex.Metrica, vous pouvez configurer le sharding à deux niveaux: divisez le cluster entier en “layers”, où une couche peut être constituée de plusieurs éclats. Les données d'un seul client sont situées sur une seule couche, mais des fragments peuvent être ajoutés à une couche si nécessaire, et les données sont distribuées aléatoirement à l'intérieur de celles-ci. Des tables distribuées sont créées pour chaque couche et une seule table distribuée partagée est créée pour les requêtes globales. -Les données sont écrites de manière asynchrone. Lorsqu'il est inséré dans la table, le bloc de données est simplement écrit dans le système de fichiers local. Les données sont envoyées aux serveurs distants en arrière-plan dès que possible. La période d'envoi des données est gérée par [distributed\_directory\_monitor\_sleep\_time\_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms) et [distributed\_directory\_monitor\_max\_sleep\_time\_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms) paramètre. Le `Distributed` moteur envoie chaque fichier de données insérées séparément, mais vous pouvez activer le lot envoi de fichiers avec l' [distributed\_directory\_monitor\_batch\_inserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts) paramètre. Ce paramètre améliore les performances du cluster en utilisant mieux les ressources réseau et serveur local. Vous devriez vérifier si les données sont envoyées avec succès en vérifiant la liste des fichiers (données en attente d'envoi) dans le répertoire de la table: `/var/lib/clickhouse/data/database/table/`. +Les données sont écrites de manière asynchrone. Lorsqu'il est inséré dans la table, le bloc de données est simplement écrit dans le système de fichiers local. Les données sont envoyées aux serveurs distants en arrière-plan dès que possible. La période d'envoi des données est gérée par [distributed_directory_monitor_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms) et [distributed_directory_monitor_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms) paramètre. Le `Distributed` moteur envoie chaque fichier de données insérées séparément, mais vous pouvez activer le lot envoi de fichiers avec l' [distributed_directory_monitor_batch_inserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts) paramètre. Ce paramètre améliore les performances du cluster en utilisant mieux les ressources réseau et serveur local. Vous devriez vérifier si les données sont envoyées avec succès en vérifiant la liste des fichiers (données en attente d'envoi) dans le répertoire de la table: `/var/lib/clickhouse/data/database/table/`. Si le serveur a cessé d'exister ou a subi un redémarrage Brutal (par exemple, après une panne de périphérique) après une insertion dans une table distribuée, les données insérées peuvent être perdues. Si une partie de données endommagée est détectée dans le répertoire de la table, elle est transférée ‘broken’ sous-répertoire et n'est plus utilisé. -Lorsque l'option max\_parallel\_replicas est activée, le traitement des requêtes est parallélisé entre toutes les répliques d'un seul fragment. Pour plus d'informations, consultez la section [max\_parallel\_replicas](../../../operations/settings/settings.md#settings-max_parallel_replicas). +Lorsque l'option max_parallel_replicas est activée, le traitement des requêtes est parallélisé entre toutes les répliques d'un seul fragment. Pour plus d'informations, consultez la section [max_parallel_replicas](../../../operations/settings/settings.md#settings-max_parallel_replicas). ## Les Colonnes Virtuelles {#virtual-columns} diff --git a/docs/fr/engines/table-engines/special/external-data.md b/docs/fr/engines/table-engines/special/external-data.md index acf63600f61..df867a23215 100644 --- a/docs/fr/engines/table-engines/special/external-data.md +++ b/docs/fr/engines/table-engines/special/external-data.md @@ -27,10 +27,10 @@ Vous pouvez avoir plusieurs sections comme ça, pour le nombre de tables étant **–file** – Path to the file with the table dump, or -, which refers to stdin. Une seule table peut être récupérée à partir de stdin. -Les paramètres suivants sont facultatifs: **–name**– Name of the table. If omitted, \_data is used. +Les paramètres suivants sont facultatifs: **–name**– Name of the table. If omitted, _data is used. **–format** – Data format in the file. If omitted, TabSeparated is used. -L'un des paramètres suivants est requis:**–types** – A list of comma-separated column types. For example: `UInt64,String`. The columns will be named \_1, \_2, … +L'un des paramètres suivants est requis:**–types** – A list of comma-separated column types. For example: `UInt64,String`. The columns will be named _1, _2, … **–structure**– The table structure in the format`UserID UInt64`, `URL String`. Définit les noms et les types de colonnes. Les fichiers spécifiés dans ‘file’ sera analysé par le format spécifié dans ‘format’, en utilisant les types de données spécifié dans ‘types’ ou ‘structure’. La table sera téléchargée sur le serveur et accessible en tant que table temporaire avec le nom dans ‘name’. @@ -48,7 +48,7 @@ $ cat /etc/passwd | sed 's/:/\t/g' | clickhouse-client --query="SELECT shell, co /bin/sync 1 ``` -Lors de l'utilisation de L'interface HTTP, les données externes sont transmises au format multipart/form-data. Chaque tableau est transmis en tant que fichier séparé. Le nom de la table est tiré du nom du fichier. Le ‘query\_string’ est passé les paramètres ‘name\_format’, ‘name\_types’, et ‘name\_structure’, où ‘name’ est le nom de la table que ces paramètres correspondent. La signification des paramètres est la même que lors de l'utilisation du client de ligne de commande. +Lors de l'utilisation de L'interface HTTP, les données externes sont transmises au format multipart/form-data. Chaque tableau est transmis en tant que fichier séparé. Le nom de la table est tiré du nom du fichier. Le ‘query_string’ est passé les paramètres ‘name_format’, ‘name_types’, et ‘name_structure’, où ‘name’ est le nom de la table que ces paramètres correspondent. La signification des paramètres est la même que lors de l'utilisation du client de ligne de commande. Exemple: diff --git a/docs/fr/engines/table-engines/special/join.md b/docs/fr/engines/table-engines/special/join.md index 01b14911e26..bc10be5b9bb 100644 --- a/docs/fr/engines/table-engines/special/join.md +++ b/docs/fr/engines/table-engines/special/join.md @@ -92,15 +92,15 @@ Vous ne pouvez pas effectuer un `SELECT` requête directement à partir de la ta Lors de la création d'un tableau, les paramètres suivants sont appliqués: -- [join\_use\_nulls](../../../operations/settings/settings.md#join_use_nulls) -- [max\_rows\_in\_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) -- [max\_bytes\_in\_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) -- [join\_overflow\_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) -- [join\_any\_take\_last\_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) +- [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) +- [max_rows_in_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) +- [max_bytes_in_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) +- [join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) +- [join_any_take_last_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) Le `Join`- les tables de moteur ne peuvent pas être utilisées dans `GLOBAL JOIN` opérations. -Le `Join`-moteur permet d'utiliser [join\_use\_nulls](../../../operations/settings/settings.md#join_use_nulls) réglage de la `CREATE TABLE` déclaration. Et [SELECT](../../../sql-reference/statements/select/index.md) requête permet d'utiliser `join_use_nulls` trop. Si vous avez différents `join_use_nulls` paramètres, vous pouvez obtenir une table de jointure d'erreur. Il dépend de type de JOINTURE. Lorsque vous utilisez [joinGet](../../../sql-reference/functions/other-functions.md#joinget) fonction, vous devez utiliser le même `join_use_nulls` réglage en `CRATE TABLE` et `SELECT` déclaration. +Le `Join`-moteur permet d'utiliser [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) réglage de la `CREATE TABLE` déclaration. Et [SELECT](../../../sql-reference/statements/select/index.md) requête permet d'utiliser `join_use_nulls` trop. Si vous avez différents `join_use_nulls` paramètres, vous pouvez obtenir une table de jointure d'erreur. Il dépend de type de JOINTURE. Lorsque vous utilisez [joinGet](../../../sql-reference/functions/other-functions.md#joinget) fonction, vous devez utiliser le même `join_use_nulls` réglage en `CRATE TABLE` et `SELECT` déclaration. ## Le Stockage De Données {#data-storage} diff --git a/docs/fr/engines/table-engines/special/merge.md b/docs/fr/engines/table-engines/special/merge.md index 991204c7dd6..6b41e490113 100644 --- a/docs/fr/engines/table-engines/special/merge.md +++ b/docs/fr/engines/table-engines/special/merge.md @@ -31,7 +31,7 @@ L'utilisation traditionnelle de la `Merge` moteur pour travailler avec un grand Exemple 2: -Disons que vous avez une ancienne table (WatchLog\_old) et que vous avez décidé de changer de partitionnement sans déplacer les données vers une nouvelle table (WatchLog\_new) et que vous devez voir les données des deux tables. +Disons que vous avez une ancienne table (WatchLog_old) et que vous avez décidé de changer de partitionnement sans déplacer les données vers une nouvelle table (WatchLog_new) et que vous devez voir les données des deux tables. ``` sql CREATE TABLE WatchLog_old(date Date, UserId Int64, EventType String, Cnt UInt64) diff --git a/docs/fr/engines/table-engines/special/url.md b/docs/fr/engines/table-engines/special/url.md index 36068845546..7b8e53b1d73 100644 --- a/docs/fr/engines/table-engines/special/url.md +++ b/docs/fr/engines/table-engines/special/url.md @@ -24,7 +24,7 @@ en-têtes supplémentaires pour obtenir une réponse du serveur. respectivement. Pour le traitement `POST` demandes, le serveur distant doit prendre en charge [Encodage de transfert en morceaux](https://en.wikipedia.org/wiki/Chunked_transfer_encoding). -Vous pouvez limiter le nombre maximal de sauts de redirection HTTP GET en utilisant [max\_http\_get\_redirects](../../../operations/settings/settings.md#setting-max_http_get_redirects) paramètre. +Vous pouvez limiter le nombre maximal de sauts de redirection HTTP GET en utilisant [max_http_get_redirects](../../../operations/settings/settings.md#setting-max_http_get_redirects) paramètre. **Exemple:** diff --git a/docs/fr/faq/general.md b/docs/fr/faq/general.md index bd2765be2d2..797709fdb20 100644 --- a/docs/fr/faq/general.md +++ b/docs/fr/faq/general.md @@ -17,7 +17,7 @@ La plupart des implémentations MapReduce vous permettent d'exécuter du code ar ## Que Faire si j'ai un problème avec les encodages lors de l'utilisation D'Oracle via ODBC? {#oracle-odbc-encodings} -Si vous utilisez Oracle via le pilote ODBC comme source de dictionnaires externes, vous devez définir la valeur correcte pour `NLS_LANG` variable d'environnement dans `/etc/default/clickhouse`. Pour plus d'informations, voir le [FAQ Oracle NLS\_LANG](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html). +Si vous utilisez Oracle via le pilote ODBC comme source de dictionnaires externes, vous devez définir la valeur correcte pour `NLS_LANG` variable d'environnement dans `/etc/default/clickhouse`. Pour plus d'informations, voir le [FAQ Oracle NLS_LANG](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html). **Exemple** diff --git a/docs/fr/getting-started/example-datasets/amplab-benchmark.md b/docs/fr/getting-started/example-datasets/amplab-benchmark.md index 066bd128ff6..4a51defe7fe 100644 --- a/docs/fr/getting-started/example-datasets/amplab-benchmark.md +++ b/docs/fr/getting-started/example-datasets/amplab-benchmark.md @@ -9,7 +9,7 @@ toc_title: AMPLab Big Data Benchmark Tu vois https://amplab.cs.berkeley.edu/benchmark/ -Inscrivez-vous pour un compte GRATUIT à https://aws.amazon.com. il nécessite une carte de crédit, e-mail et numéro de téléphone. Obtenir une nouvelle clé d'accès à https://console.aws.amazon.com/iam/home?nc2=h\_m\_sc\#security\_credential +Inscrivez-vous pour un compte GRATUIT à https://aws.amazon.com. il nécessite une carte de crédit, e-mail et numéro de téléphone. Obtenir une nouvelle clé d'accès à https://console.aws.amazon.com/iam/home?nc2=h_m_sc#security_credential Exécutez ce qui suit dans la console: diff --git a/docs/fr/getting-started/example-datasets/metrica.md b/docs/fr/getting-started/example-datasets/metrica.md index 6cc8f93b1c2..f9d6c7b437b 100644 --- a/docs/fr/getting-started/example-datasets/metrica.md +++ b/docs/fr/getting-started/example-datasets/metrica.md @@ -9,7 +9,7 @@ toc_title: "Yandex.Metrica De Donn\xE9es" Dataset se compose de deux tables contenant des données anonymisées sur les hits (`hits_v1`) et les visites (`visits_v1`) de Yandex.Metrica. Vous pouvez en savoir plus sur Yandex.Metrica dans [Histoire de ClickHouse](../../introduction/history.md) section. -L'ensemble de données se compose de deux tables, l'une d'elles peut être téléchargée sous forme compressée `tsv.xz` fichier ou comme partitions préparées. En outre, une version étendue de l' `hits` table contenant 100 millions de lignes est disponible comme TSV à https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits\_100m\_obfuscated\_v1.tsv.xz et comme partitions préparées à https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits\_100m\_obfuscated\_v1.tar.xz. +L'ensemble de données se compose de deux tables, l'une d'elles peut être téléchargée sous forme compressée `tsv.xz` fichier ou comme partitions préparées. En outre, une version étendue de l' `hits` table contenant 100 millions de lignes est disponible comme TSV à https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits_100m_obfuscated_v1.tsv.xz et comme partitions préparées à https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits_100m_obfuscated_v1.tar.xz. ## Obtention de Tables à partir de Partitions préparées {#obtaining-tables-from-prepared-partitions} diff --git a/docs/fr/getting-started/example-datasets/nyc-taxi.md b/docs/fr/getting-started/example-datasets/nyc-taxi.md index 92818ac9d3f..e351b9ec543 100644 --- a/docs/fr/getting-started/example-datasets/nyc-taxi.md +++ b/docs/fr/getting-started/example-datasets/nyc-taxi.md @@ -197,7 +197,7 @@ Les données de ce tableau utilisent 142 GO. (L'importation de données directement depuis Postgres est également possible en utilisant `COPY ... TO PROGRAM`.) -Unfortunately, all the fields associated with the weather (precipitation…average\_wind\_speed) were filled with NULL. Because of this, we will remove them from the final data set. +Unfortunately, all the fields associated with the weather (precipitation…average_wind_speed) were filled with NULL. Because of this, we will remove them from the final data set. Pour commencer, nous allons créer une table sur un serveur unique. Plus tard, nous ferons le tableau distribué. diff --git a/docs/fr/getting-started/install.md b/docs/fr/getting-started/install.md index 3659369fb4a..db014bec974 100644 --- a/docs/fr/getting-started/install.md +++ b/docs/fr/getting-started/install.md @@ -9,9 +9,9 @@ toc_title: Installation ## Configuration Système Requise {#system-requirements} -ClickHouse peut fonctionner sur N'importe quel Linux, FreeBSD ou Mac OS X avec une architecture CPU x86\_64, AArch64 ou PowerPC64LE. +ClickHouse peut fonctionner sur N'importe quel Linux, FreeBSD ou Mac OS X avec une architecture CPU x86_64, AArch64 ou PowerPC64LE. -Les binaires pré-construits officiels sont généralement compilés pour le jeu d'instructions x86\_64 et leverage SSE 4.2, donc sauf indication contraire, l'utilisation du processeur qui le prend en charge devient une exigence système supplémentaire. Voici la commande pour vérifier si le processeur actuel prend en charge SSE 4.2: +Les binaires pré-construits officiels sont généralement compilés pour le jeu d'instructions x86_64 et leverage SSE 4.2, donc sauf indication contraire, l'utilisation du processeur qui le prend en charge devient une exigence système supplémentaire. Voici la commande pour vérifier si le processeur actuel prend en charge SSE 4.2: ``` bash $ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not supported" diff --git a/docs/fr/index.md b/docs/fr/index.md index fe714af8d49..d11744f1ab0 100644 --- a/docs/fr/index.md +++ b/docs/fr/index.md @@ -13,10 +13,10 @@ Dans un “normal” SGBD orienté ligne, les données sont stockées dans cet o | Rangée | WatchID | JavaEnable | Intitulé | GoodEvent | EventTime | |--------|-------------|------------|----------------------------------|-----------|---------------------| -| \#0 | 89354350662 | 1 | Relations Avec Les Investisseurs | 1 | 2016-05-18 05:19:20 | -| \#1 | 90329509958 | 0 | Contacter | 1 | 2016-05-18 08:10:20 | -| \#2 | 89953706054 | 1 | Mission | 1 | 2016-05-18 07:38:00 | -| \#N | … | … | … | … | … | +| #0 | 89354350662 | 1 | Relations Avec Les Investisseurs | 1 | 2016-05-18 05:19:20 | +| #1 | 90329509958 | 0 | Contacter | 1 | 2016-05-18 08:10:20 | +| #2 | 89953706054 | 1 | Mission | 1 | 2016-05-18 07:38:00 | +| #N | … | … | … | … | … | En d'autres termes, toutes les valeurs liées à une ligne sont physiquement stockées l'une à côté de l'autre. @@ -24,7 +24,7 @@ Des exemples d'un SGBD orienté ligne sont MySQL, Postgres et MS SQL Server. Dans un SGBD orienté colonne, les données sont stockées comme ceci: -| Rangée: | \#0 | \#1 | \#2 | \#N | +| Rangée: | #0 | #1 | #2 | #N | |-------------|----------------------------------|---------------------|---------------------|-----| | WatchID: | 89354350662 | 90329509958 | 89953706054 | … | | JavaEnable: | 1 | 0 | 1 | … | diff --git a/docs/fr/interfaces/formats.md b/docs/fr/interfaces/formats.md index 437da7f0c4f..2022683e940 100644 --- a/docs/fr/interfaces/formats.md +++ b/docs/fr/interfaces/formats.md @@ -201,7 +201,7 @@ Paramètre `format_template_resultset` spécifie le chemin d'accès au fichier, - `min` est la ligne avec des valeurs minimales dans `format_template_row` format (lorsque les extrêmes sont définis sur 1) - `max` est la ligne avec des valeurs maximales en `format_template_row` format (lorsque les extrêmes sont définis sur 1) - `rows` le nombre total de lignes de sortie -- `rows_before_limit` est le nombre minimal de lignes qu'il y aurait eu sans limite. Sortie uniquement si la requête contient LIMIT. Si la requête contient GROUP BY, rows\_before\_limit\_at\_least est le nombre exact de lignes qu'il y aurait eu sans limite. +- `rows_before_limit` est le nombre minimal de lignes qu'il y aurait eu sans limite. Sortie uniquement si la requête contient LIMIT. Si la requête contient GROUP BY, rows_before_limit_at_least est le nombre exact de lignes qu'il y aurait eu sans limite. - `time` est le temps d'exécution de la requête en secondes - `rows_read` est le nombre de lignes a été lu - `bytes_read` est le nombre d'octets (non compressé) a été lu @@ -352,21 +352,21 @@ L'analyse permet la présence du champ supplémentaire `tskv` sans le signe éga Format des valeurs séparées par des virgules ([RFC](https://tools.ietf.org/html/rfc4180)). -Lors du formatage, les lignes sont entourées de guillemets doubles. Un guillemet double à l'intérieur d'une chaîne est affiché sous la forme de deux guillemets doubles dans une rangée. Il n'y a pas d'autres règles pour échapper les caractères. Date et date-heure sont entre guillemets. Les nombres sont produits sans guillemets. Les valeurs sont séparées par un caractère délimiteur, qui est `,` par défaut. Le caractère délimiteur est défini dans le paramètre [format\_csv\_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter). Les lignes sont séparées à L'aide du saut de ligne Unix (LF). Les tableaux sont sérialisés au format CSV comme suit: tout d'abord, le tableau est sérialisé en une chaîne comme au format TabSeparated, puis la chaîne résultante est sortie au format CSV entre guillemets doubles. Les Tuples au format CSV sont sérialisés en tant que colonnes séparées (c'est-à-dire que leur imbrication dans le tuple est perdue). +Lors du formatage, les lignes sont entourées de guillemets doubles. Un guillemet double à l'intérieur d'une chaîne est affiché sous la forme de deux guillemets doubles dans une rangée. Il n'y a pas d'autres règles pour échapper les caractères. Date et date-heure sont entre guillemets. Les nombres sont produits sans guillemets. Les valeurs sont séparées par un caractère délimiteur, qui est `,` par défaut. Le caractère délimiteur est défini dans le paramètre [format_csv_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter). Les lignes sont séparées à L'aide du saut de ligne Unix (LF). Les tableaux sont sérialisés au format CSV comme suit: tout d'abord, le tableau est sérialisé en une chaîne comme au format TabSeparated, puis la chaîne résultante est sortie au format CSV entre guillemets doubles. Les Tuples au format CSV sont sérialisés en tant que colonnes séparées (c'est-à-dire que leur imbrication dans le tuple est perdue). ``` bash $ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV" < data.csv ``` -\*Par défaut, le délimiteur est `,`. Voir la [format\_csv\_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter) réglage pour plus d'informations. +\*Par défaut, le délimiteur est `,`. Voir la [format_csv_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter) réglage pour plus d'informations. Lors de l'analyse, toutes les valeurs peuvent être analysés avec ou sans guillemets. Les guillemets doubles et simples sont pris en charge. Les lignes peuvent également être organisées sans guillemets. Dans ce cas, ils sont analysés jusqu'au caractère délimiteur ou au saut de ligne (CR ou LF). En violation de la RFC, lors de l'analyse des lignes sans guillemets, les espaces et les onglets de début et de fin sont ignorés. Pour le saut de ligne, les types Unix (LF), Windows (CR LF) et Mac OS Classic (CR LF) sont tous pris en charge. Les valeurs d'entrée non cotées vides sont remplacées par des valeurs par défaut pour les colonnes respectives, si -[input\_format\_defaults\_for\_omitted\_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) +[input_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) est activé. -`NULL` est formaté en tant qu' `\N` ou `NULL` ou une chaîne vide non cotée (voir paramètres [input\_format\_csv\_unquoted\_null\_literal\_as\_null](../operations/settings/settings.md#settings-input_format_csv_unquoted_null_literal_as_null) et [input\_format\_defaults\_for\_omitted\_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields)). +`NULL` est formaté en tant qu' `\N` ou `NULL` ou une chaîne vide non cotée (voir paramètres [input_format_csv_unquoted_null_literal_as_null](../operations/settings/settings.md#settings-input_format_csv_unquoted_null_literal_as_null) et [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields)). Le format CSV prend en charge la sortie des totaux et des extrêmes de la même manière que `TabSeparated`. @@ -451,12 +451,12 @@ SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase WITH TOTA } ``` -Le JSON est compatible avec JavaScript. Pour ce faire, certains caractères sont en outre échappés: la barre oblique `/` s'est échappée comme l' `\/`; sauts de ligne alternatifs `U+2028` et `U+2029`, qui cassent certains navigateurs, sont échappés comme `\uXXXX`. Les caractères de contrôle ASCII sont échappés: retour arrière, flux de formulaire, saut de ligne, retour chariot et tabulation horizontale sont remplacés par `\b`, `\f`, `\n`, `\r`, `\t` , ainsi que les octets restants dans la plage 00-1F en utilisant `\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) à 0. +Le JSON est compatible avec JavaScript. Pour ce faire, certains caractères sont en outre échappés: la barre oblique `/` s'est échappée comme l' `\/`; sauts de ligne alternatifs `U+2028` et `U+2029`, qui cassent certains navigateurs, sont échappés comme `\uXXXX`. Les caractères de contrôle ASCII sont échappés: retour arrière, flux de formulaire, saut de ligne, retour chariot et tabulation horizontale sont remplacés par `\b`, `\f`, `\n`, `\r`, `\t` , ainsi que les octets restants dans la plage 00-1F en utilisant `\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) à 0. `rows` – The total number of output rows. `rows_before_limit_at_least` Le nombre minimal de lignes aurait été sans limite. Sortie uniquement si la requête contient LIMIT. -Si la requête contient GROUP BY, rows\_before\_limit\_at\_least est le nombre exact de lignes qu'il y aurait eu sans limite. +Si la requête contient GROUP BY, rows_before_limit_at_least est le nombre exact de lignes qu'il y aurait eu sans limite. `totals` – Total values (when using WITH TOTALS). @@ -543,7 +543,7 @@ ClickHouse ignore les espaces entre les éléments et les virgules après les ob Clickhouse remplace les valeurs omises par les valeurs par défaut pour le [types de données](../sql-reference/data-types/index.md). -Si `DEFAULT expr` clickhouse utilise différentes règles de substitution en fonction de [input\_format\_defaults\_for\_omitted\_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) paramètre. +Si `DEFAULT expr` clickhouse utilise différentes règles de substitution en fonction de [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) paramètre. Considérons le tableau suivant: @@ -586,7 +586,7 @@ Contrairement à l' [JSON](#json) format, il n'y a pas de substitution de séque ### Utilisation de Structures imbriquées {#jsoneachrow-nested} -Si vous avez une table avec [Imbriqué](../sql-reference/data-types/nested-data-structures/nested.md) colonnes de type de données, vous pouvez insérer des données JSON avec la même structure. Activer cette fonctionnalité avec le [input\_format\_import\_nested\_json](../operations/settings/settings.md#settings-input_format_import_nested_json) paramètre. +Si vous avez une table avec [Imbriqué](../sql-reference/data-types/nested-data-structures/nested.md) colonnes de type de données, vous pouvez insérer des données JSON avec la même structure. Activer cette fonctionnalité avec le [input_format_import_nested_json](../operations/settings/settings.md#settings-input_format_import_nested_json) paramètre. Par exemple, considérez le tableau suivant: @@ -600,7 +600,7 @@ Comme vous pouvez le voir dans la `Nested` description du type de données, Clic INSERT INTO json_each_row_nested FORMAT JSONEachRow {"n.s": ["abc", "def"], "n.i": [1, 23]} ``` -Pour insérer des données en tant qu'objet JSON hiérarchique, définissez [input\_format\_import\_nested\_json=1](../operations/settings/settings.md#settings-input_format_import_nested_json). +Pour insérer des données en tant qu'objet JSON hiérarchique, définissez [input_format_import_nested_json=1](../operations/settings/settings.md#settings-input_format_import_nested_json). ``` json { @@ -782,7 +782,7 @@ The minimum set of characters that you need to escape when passing data in Value C'est le format qui est utilisé dans `INSERT INTO t VALUES ...`, mais vous pouvez également l'utiliser pour le formatage des résultats de requête. -Voir aussi: [input\_format\_values\_interpret\_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions) et [input\_format\_values\_deduce\_templates\_of\_expressions](../operations/settings/settings.md#settings-input_format_values_deduce_templates_of_expressions) paramètre. +Voir aussi: [input_format_values_interpret_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions) et [input_format_values_deduce_templates_of_expressions](../operations/settings/settings.md#settings-input_format_values_deduce_templates_of_expressions) paramètre. ## Vertical {#vertical} @@ -1045,7 +1045,7 @@ Les noms de colonnes doivent: - commencer avec `[A-Za-z_]` - par la suite contenir uniquement `[A-Za-z0-9_]` -Sortie Avro fichier de compression et sync intervalle peut être configuré avec [output\_format\_avro\_codec](../operations/settings/settings.md#settings-output_format_avro_codec) et [output\_format\_avro\_sync\_interval](../operations/settings/settings.md#settings-output_format_avro_sync_interval) respectivement. +Sortie Avro fichier de compression et sync intervalle peut être configuré avec [output_format_avro_codec](../operations/settings/settings.md#settings-output_format_avro_codec) et [output_format_avro_sync_interval](../operations/settings/settings.md#settings-output_format_avro_sync_interval) respectivement. ## AvroConfluent {#data-format-avro-confluent} @@ -1055,7 +1055,7 @@ Chaque message Avro intègre un id de schéma qui peut être résolu dans le sch Les schémas sont mis en cache une fois résolus. -L'URL du registre de schéma est configurée avec [format\_avro\_schema\_registry\_url](../operations/settings/settings.md#settings-format_avro_schema_registry_url) +L'URL du registre de schéma est configurée avec [format_avro_schema_registry_url](../operations/settings/settings.md#settings-format_avro_schema_registry_url) ### Types De Données Correspondant {#data_types-matching-1} @@ -1198,13 +1198,13 @@ peut contenir un chemin absolu, soit un chemin relatif au répertoire courant su Si vous utilisez le client dans le [mode batch](../interfaces/cli.md#cli_usage), le chemin d'accès au schéma doit être relatif pour des raisons de sécurité. Si vous entrez ou sortez des données via le [Interface HTTP](../interfaces/http.md) le nom de fichier spécifié dans le format de schéma -doit être situé dans le répertoire spécifié dans [format\_schema\_path](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-format_schema_path) +doit être situé dans le répertoire spécifié dans [format_schema_path](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-format_schema_path) dans la configuration du serveur. ## Sauter Les Erreurs {#skippingerrors} -Certains formats tels que `CSV`, `TabSeparated`, `TSKV`, `JSONEachRow`, `Template`, `CustomSeparated` et `Protobuf` pouvez ignorer brisé ligne si erreur d'analyse s'est produite et poursuivre l'analyse à partir du début de la ligne suivante. Voir [input\_format\_allow\_errors\_num](../operations/settings/settings.md#settings-input_format_allow_errors_num) et -[input\_format\_allow\_errors\_ratio](../operations/settings/settings.md#settings-input_format_allow_errors_ratio) paramètre. +Certains formats tels que `CSV`, `TabSeparated`, `TSKV`, `JSONEachRow`, `Template`, `CustomSeparated` et `Protobuf` pouvez ignorer brisé ligne si erreur d'analyse s'est produite et poursuivre l'analyse à partir du début de la ligne suivante. Voir [input_format_allow_errors_num](../operations/settings/settings.md#settings-input_format_allow_errors_num) et +[input_format_allow_errors_ratio](../operations/settings/settings.md#settings-input_format_allow_errors_ratio) paramètre. Limitation: - En cas d'erreur d'analyse `JSONEachRow` ignore toutes les données jusqu'à la nouvelle ligne (ou EOF), donc les lignes doivent être délimitées par `\n` pour compter les erreurs correctement. - `Template` et `CustomSeparated` utilisez delimiter après la dernière colonne et delimiter entre les lignes pour trouver le début de la ligne suivante, donc sauter les erreurs ne fonctionne que si au moins l'une d'entre elles n'est pas vide. diff --git a/docs/fr/interfaces/http.md b/docs/fr/interfaces/http.md index a414bba2c2f..ea590a2b870 100644 --- a/docs/fr/interfaces/http.md +++ b/docs/fr/interfaces/http.md @@ -11,7 +11,7 @@ L'interface HTTP vous permet D'utiliser ClickHouse sur n'importe quelle plate-fo Par défaut, clickhouse-server écoute HTTP sur le port 8123 (cela peut être modifié dans la configuration). -Si vous faites une requête GET / sans Paramètres, elle renvoie le code de réponse 200 et la chaîne définie dans [http\_server\_default\_response](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-http_server_default_response) valeur par défaut “Ok.” (avec un saut de ligne à la fin) +Si vous faites une requête GET / sans Paramètres, elle renvoie le code de réponse 200 et la chaîne définie dans [http_server_default_response](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-http_server_default_response) valeur par défaut “Ok.” (avec un saut de ligne à la fin) ``` bash $ curl 'http://localhost:8123/' @@ -147,12 +147,12 @@ $ echo 'DROP TABLE t' | curl 'http://localhost:8123/' --data-binary @- Pour les requêtes réussies qui ne renvoient pas de table de données, un corps de réponse vide est renvoyé. -Vous pouvez utiliser le format de compression ClickHouse interne lors de la transmission de données. Les données compressées ont un format non standard, et vous devrez utiliser le spécial `clickhouse-compressor` programme de travail (il s'est installé avec le `clickhouse-client` paquet). Pour augmenter l'efficacité de l'insertion de données, vous pouvez désactiver la vérification de la somme de contrôle côté serveur en utilisant [http\_native\_compression\_disable\_checksumming\_on\_decompress](../operations/settings/settings.md#settings-http_native_compression_disable_checksumming_on_decompress) paramètre. +Vous pouvez utiliser le format de compression ClickHouse interne lors de la transmission de données. Les données compressées ont un format non standard, et vous devrez utiliser le spécial `clickhouse-compressor` programme de travail (il s'est installé avec le `clickhouse-client` paquet). Pour augmenter l'efficacité de l'insertion de données, vous pouvez désactiver la vérification de la somme de contrôle côté serveur en utilisant [http_native_compression_disable_checksumming_on_decompress](../operations/settings/settings.md#settings-http_native_compression_disable_checksumming_on_decompress) paramètre. Si vous avez spécifié `compress=1` dans l'URL, le serveur compresse les données qu'il vous envoie. Si vous avez spécifié `decompress=1` dans L'URL, le serveur décompresse les mêmes données que vous transmettez `POST` méthode. -Vous pouvez également choisir d'utiliser [La compression HTTP](https://en.wikipedia.org/wiki/HTTP_compression). Pour envoyer un compressé `POST` demande, ajouter l'en-tête de requête `Content-Encoding: compression_method`. Pour que ClickHouse compresse la réponse, vous devez ajouter `Accept-Encoding: compression_method`. Supports ClickHouse `gzip`, `br`, et `deflate` [méthodes de compression](https://en.wikipedia.org/wiki/HTTP_compression#Content-Encoding_tokens). Pour activer la compression HTTP, vous devez utiliser le ClickHouse [enable\_http\_compression](../operations/settings/settings.md#settings-enable_http_compression) paramètre. Vous pouvez configurer le niveau de compression des données dans le [http\_zlib\_compression\_level](#settings-http_zlib_compression_level) pour toutes les méthodes de compression. +Vous pouvez également choisir d'utiliser [La compression HTTP](https://en.wikipedia.org/wiki/HTTP_compression). Pour envoyer un compressé `POST` demande, ajouter l'en-tête de requête `Content-Encoding: compression_method`. Pour que ClickHouse compresse la réponse, vous devez ajouter `Accept-Encoding: compression_method`. Supports ClickHouse `gzip`, `br`, et `deflate` [méthodes de compression](https://en.wikipedia.org/wiki/HTTP_compression#Content-Encoding_tokens). Pour activer la compression HTTP, vous devez utiliser le ClickHouse [enable_http_compression](../operations/settings/settings.md#settings-enable_http_compression) paramètre. Vous pouvez configurer le niveau de compression des données dans le [http_zlib_compression_level](#settings-http_zlib_compression_level) pour toutes les méthodes de compression. Vous pouvez l'utiliser pour réduire le trafic réseau lors de la transmission d'une grande quantité de données, ou pour créer des vidages qui sont immédiatement compressés. @@ -214,7 +214,7 @@ $ echo 'SELECT 1' | curl -H 'X-ClickHouse-User: user' -H 'X-ClickHouse-Key: pass ``` Si le nom d'utilisateur n'est spécifié, le `default` le nom est utilisé. Si le mot de passe n'est spécifié, le mot de passe vide est utilisé. -Vous pouvez également utiliser les paramètres D'URL pour spécifier des paramètres pour le traitement d'une seule requête ou de profils entiers de paramètres. Exemple: http: / / localhost: 8123/?profil = web & max\_rows\_to\_read=1000000000 & query=sélectionner + 1 +Vous pouvez également utiliser les paramètres D'URL pour spécifier des paramètres pour le traitement d'une seule requête ou de profils entiers de paramètres. Exemple: http: / / localhost: 8123/?profil = web & max_rows_to_read=1000000000 & query=sélectionner + 1 Pour plus d'informations, voir le [Paramètre](../operations/settings/index.md) section. @@ -236,7 +236,7 @@ Pour plus d'informations sur les autres paramètres, consultez la section “SET De même, vous pouvez utiliser des sessions ClickHouse dans le protocole HTTP. Pour ce faire, vous devez ajouter l' `session_id` GET paramètre à la demande. Vous pouvez utiliser n'importe quelle chaîne comme ID de session. Par défaut, la session est terminée après 60 secondes d'inactivité. Pour modifier ce délai d'attente, de modifier la `default_session_timeout` dans la configuration du serveur, ou ajoutez le `session_timeout` GET paramètre à la demande. Pour vérifier l'état de la session, utilisez `session_check=1` paramètre. Une seule requête à la fois peut être exécutée dans une seule session. -Vous pouvez recevoir des informations sur le déroulement d'une requête en `X-ClickHouse-Progress` en-têtes de réponse. Pour ce faire, activez [send\_progress\_in\_http\_headers](../operations/settings/settings.md#settings-send_progress_in_http_headers). Exemple de l'en-tête de séquence: +Vous pouvez recevoir des informations sur le déroulement d'une requête en `X-ClickHouse-Progress` en-têtes de réponse. Pour ce faire, activez [send_progress_in_http_headers](../operations/settings/settings.md#settings-send_progress_in_http_headers). Exemple de l'en-tête de séquence: ``` text X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128"} @@ -253,9 +253,9 @@ Possibles champs d'en-tête: - `written_bytes` — Volume of data written in bytes. Les requêtes en cours d'exécution ne s'arrêtent pas automatiquement si la connexion HTTP est perdue. L'analyse et le formatage des données sont effectués côté serveur et l'utilisation du réseau peut s'avérer inefficace. -Facultatif ‘query\_id’ le paramètre peut être passé comme ID de requête (n'importe quelle chaîne). Pour plus d'informations, consultez la section “Settings, replace\_running\_query”. +Facultatif ‘query_id’ le paramètre peut être passé comme ID de requête (n'importe quelle chaîne). Pour plus d'informations, consultez la section “Settings, replace_running_query”. -Facultatif ‘quota\_key’ le paramètre peut être passé comme clé de quota (n'importe quelle chaîne). Pour plus d'informations, consultez la section “Quotas”. +Facultatif ‘quota_key’ le paramètre peut être passé comme clé de quota (n'importe quelle chaîne). Pour plus d'informations, consultez la section “Quotas”. L'interface HTTP permet de transmettre des données externes (tables temporaires externes) pour l'interrogation. Pour plus d'informations, consultez la section “External data for query processing”. @@ -377,11 +377,11 @@ Comme vous pouvez le voir dans l'exemple, si `` est configuré da > `` est responsable de la correspondance de la partie d'en-tête de la requête HTTP. Il est compatible avec les expressions régulières de RE2. C'est une option de configuration. S'il n'est pas défini dans le fichier de configuration, il ne correspond pas à la partie d'en-tête de la requête HTTP. > > `` contient la partie de traitement principale. Maintenant `` pouvez configurer ``, ``, ``, ``, ``, ``. -> \> `` prend actuellement en charge trois types: **predefined\_query\_handler**, **dynamic\_query\_handler**, **statique**. +> \> `` prend actuellement en charge trois types: **predefined_query_handler**, **dynamic_query_handler**, **statique**. > \> -> \> `` - utiliser avec le type predefined\_query\_handler, exécute la requête lorsque le gestionnaire est appelé. +> \> `` - utiliser avec le type predefined_query_handler, exécute la requête lorsque le gestionnaire est appelé. > \> -> \> `` - utiliser avec le type dynamic\_query\_handler, extrait et exécute la valeur correspondant au `` valeur dans les paramètres de requête HTTP. +> \> `` - utiliser avec le type dynamic_query_handler, extrait et exécute la valeur correspondant au `` valeur dans les paramètres de requête HTTP. > \> > \> `` - utiliser avec le type statique, code d'état de réponse. > \> @@ -391,9 +391,9 @@ Comme vous pouvez le voir dans l'exemple, si `` est configuré da Viennent ensuite les méthodes de configuration pour les différents ``. -## predefined\_query\_handler {#predefined_query_handler} +## predefined_query_handler {#predefined_query_handler} -`` prend en charge les paramètres de réglage et les valeurs query\_params. Vous pouvez configurer `` dans le type de ``. +`` prend en charge les paramètres de réglage et les valeurs query_params. Vous pouvez configurer `` dans le type de ``. `` la valeur est une requête prédéfinie de ``, qui est exécuté par ClickHouse lorsqu'une requête HTTP est mise en correspondance et que le résultat de la requête est renvoyé. C'est une configuration incontournable. @@ -428,13 +428,13 @@ max_alter_threads 2 !!! note "précaution" Dans un `` prend en charge un seul `` d'un type d'insertion. -## dynamic\_query\_handler {#dynamic_query_handler} +## dynamic_query_handler {#dynamic_query_handler} Dans ``, la requête est écrite sous la forme de param de la requête HTTP. La différence est que dans ``, la requête est écrite dans le fichier de configuration. Vous pouvez configurer `` dans ``. Clickhouse extrait et exécute la valeur correspondant au `` valeur dans l'url de la requête HTTP. La valeur par défaut de `` être `/query` . C'est une option de configuration. Si il n'y a pas de définition dans le fichier de configuration, le paramètre n'est pas passé. -Pour expérimenter cette fonctionnalité, l'exemple définit les valeurs de max\_threads et max\_alter\_threads et demande si les paramètres ont été définis avec succès. +Pour expérimenter cette fonctionnalité, l'exemple définit les valeurs de max_threads et max_alter_threads et demande si les paramètres ont été définis avec succès. Exemple: @@ -459,7 +459,7 @@ max_alter_threads 2 ## statique {#static} -`` peut-retour [content\_type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type), [statut](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status) et response\_content. response\_content peut renvoyer le contenu spécifié +`` peut-retour [content_type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type), [statut](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status) et response_content. response_content peut renvoyer le contenu spécifié Exemple: diff --git a/docs/fr/interfaces/mysql.md b/docs/fr/interfaces/mysql.md index b3ec2e30e26..051538d45b2 100644 --- a/docs/fr/interfaces/mysql.md +++ b/docs/fr/interfaces/mysql.md @@ -7,7 +7,7 @@ toc_title: Interface MySQL # Interface MySQL {#mysql-interface} -ClickHouse prend en charge le protocole de fil MySQL. Il peut être activé par [mysql\_port](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-mysql_port) paramètre dans le fichier de configuration: +ClickHouse prend en charge le protocole de fil MySQL. Il peut être activé par [mysql_port](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-mysql_port) paramètre dans le fichier de configuration: ``` xml 9004 diff --git a/docs/fr/interfaces/third-party/client-libraries.md b/docs/fr/interfaces/third-party/client-libraries.md index c3385d207a3..5a86d12a09c 100644 --- a/docs/fr/interfaces/third-party/client-libraries.md +++ b/docs/fr/interfaces/third-party/client-libraries.md @@ -11,7 +11,7 @@ toc_title: "Biblioth\xE8ques Clientes" Yandex ne **pas** maintenir les bibliothèques énumérées ci-dessous et n'ont pas fait de tests approfondis pour assurer leur qualité. - Python - - [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm) + - [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm) - [clickhouse-chauffeur](https://github.com/mymarilyn/clickhouse-driver) - [clickhouse-client](https://github.com/yurial/clickhouse-client) - [aiochclient](https://github.com/maximdanilchenko/aiochclient) @@ -47,7 +47,7 @@ toc_title: "Biblioth\xE8ques Clientes" - [clickhouse-Scala-client](https://github.com/crobox/clickhouse-scala-client) - Kotlin - [AORM](https://github.com/TanVD/AORM) -- C\# +- C# - [Octonica.ClickHouseClient](https://github.com/Octonica/ClickHouseClient) - [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net) - [ClickHouse.Client](https://github.com/DarkWanderer/ClickHouse.Client) diff --git a/docs/fr/interfaces/third-party/integrations.md b/docs/fr/interfaces/third-party/integrations.md index 889408909b5..1b74a49dc89 100644 --- a/docs/fr/interfaces/third-party/integrations.md +++ b/docs/fr/interfaces/third-party/integrations.md @@ -19,15 +19,15 @@ toc_title: "Int\xE9gration" - [clickhouse-mysql-lecteur de données](https://github.com/Altinity/clickhouse-mysql-data-reader) - [horgh-réplicateur](https://github.com/larsnovikov/horgh-replicator) - [PostgreSQL](https://www.postgresql.org) - - [clickhousedb\_fdw](https://github.com/Percona-Lab/clickhousedb_fdw) - - [infi.clickhouse\_fdw](https://github.com/Infinidat/infi.clickhouse_fdw) (utiliser [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) + - [clickhousedb_fdw](https://github.com/Percona-Lab/clickhousedb_fdw) + - [infi.clickhouse_fdw](https://github.com/Infinidat/infi.clickhouse_fdw) (utiliser [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - [pg2ch](https://github.com/mkabilov/pg2ch) - - [clickhouse\_fdw](https://github.com/adjust/clickhouse_fdw) + - [clickhouse_fdw](https://github.com/adjust/clickhouse_fdw) - [MSSQL](https://en.wikipedia.org/wiki/Microsoft_SQL_Server) - [ClickHouseMigrator](https://github.com/zlzforever/ClickHouseMigrator) - Files d'attente de messages - [Kafka](https://kafka.apache.org) - - [clickhouse\_sinker](https://github.com/housepower/clickhouse_sinker) (utiliser [Allez client](https://github.com/ClickHouse/clickhouse-go/)) + - [clickhouse_sinker](https://github.com/housepower/clickhouse_sinker) (utiliser [Allez client](https://github.com/ClickHouse/clickhouse-go/)) - [stream-loader-clickhouse](https://github.com/adform/stream-loader) - Traitement de flux - [Flink](https://flink.apache.org) @@ -51,12 +51,12 @@ toc_title: "Int\xE9gration" - [Grafana](https://grafana.com/) - [clickhouse-grafana](https://github.com/Vertamedia/clickhouse-grafana) - [Prometheus](https://prometheus.io/) - - [clickhouse\_exporter](https://github.com/f1yegor/clickhouse_exporter) + - [clickhouse_exporter](https://github.com/f1yegor/clickhouse_exporter) - [PromHouse](https://github.com/Percona-Lab/PromHouse) - - [clickhouse\_exporter](https://github.com/hot-wifi/clickhouse_exporter) (utiliser [Allez client](https://github.com/kshvakov/clickhouse/)) + - [clickhouse_exporter](https://github.com/hot-wifi/clickhouse_exporter) (utiliser [Allez client](https://github.com/kshvakov/clickhouse/)) - [Nagios](https://www.nagios.org/) - - [check\_clickhouse](https://github.com/exogroup/check_clickhouse/) - - [check\_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py) + - [check_clickhouse](https://github.com/exogroup/check_clickhouse/) + - [check_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py) - [Zabbix](https://www.zabbix.com) - [clickhouse-Zabbix-modèle](https://github.com/Altinity/clickhouse-zabbix-template) - [Sematext](https://sematext.com/) @@ -76,7 +76,7 @@ toc_title: "Int\xE9gration" - Python - [SQLAlchemy](https://www.sqlalchemy.org) - - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (utiliser [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) + - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (utiliser [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - [Panda](https://pandas.pydata.org) - [pandahouse](https://github.com/kszucs/pandahouse) - PHP @@ -91,7 +91,7 @@ toc_title: "Int\xE9gration" - Scala - [Akka](https://akka.io) - [clickhouse-Scala-client](https://github.com/crobox/clickhouse-scala-client) -- C\# +- C# - [ADO.NET](https://docs.microsoft.com/en-us/dotnet/framework/data/adonet/ado-net-overview) - [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net) - [ClickHouse.Client](https://github.com/DarkWanderer/ClickHouse.Client) @@ -99,7 +99,7 @@ toc_title: "Int\xE9gration" - [ClickHouse.Net.Les Migrations](https://github.com/ilyabreev/ClickHouse.Net.Migrations) - Elixir - [Ecto](https://github.com/elixir-ecto/ecto) - - [clickhouse\_ecto](https://github.com/appodeal/clickhouse_ecto) + - [clickhouse_ecto](https://github.com/appodeal/clickhouse_ecto) - Ruby - [Ruby on Rails](https://rubyonrails.org/) - [activecube](https://github.com/bitquery/activecube) diff --git a/docs/fr/operations/access-rights.md b/docs/fr/operations/access-rights.md index 350a19bb7de..c5bf9f2b695 100644 --- a/docs/fr/operations/access-rights.md +++ b/docs/fr/operations/access-rights.md @@ -134,10 +134,10 @@ Gestion des requêtes: - Configurez un répertoire pour le stockage des configurations. - Clickhouse stocke les configurations d'entité d'accès dans le dossier défini dans [access\_control\_path](server-configuration-parameters/settings.md#access_control_path) paramètre de configuration du serveur. + Clickhouse stocke les configurations d'entité d'accès dans le dossier défini dans [access_control_path](server-configuration-parameters/settings.md#access_control_path) paramètre de configuration du serveur. - Activez le contrôle D'accès piloté par SQL et la gestion de compte pour au moins un compte d'utilisateur. - Par défaut, le contrôle D'accès piloté par SQL et la gestion des comptes sont activés pour tous les utilisateurs. Vous devez configurer au moins un utilisateur dans le `users.xml` fichier de configuration et affecter 1 au [access\_management](settings/settings-users.md#access_management-user-setting) paramètre. + Par défaut, le contrôle D'accès piloté par SQL et la gestion des comptes sont activés pour tous les utilisateurs. Vous devez configurer au moins un utilisateur dans le `users.xml` fichier de configuration et affecter 1 au [access_management](settings/settings-users.md#access_management-user-setting) paramètre. [Article Original](https://clickhouse.tech/docs/en/operations/access_rights/) diff --git a/docs/fr/operations/configuration-files.md b/docs/fr/operations/configuration-files.md index 71569cc9385..8ecb1ed0bb0 100644 --- a/docs/fr/operations/configuration-files.md +++ b/docs/fr/operations/configuration-files.md @@ -20,7 +20,7 @@ Si `replace` est spécifié, il remplace l'élément entier par celui spécifié Si `remove` est spécifié, il supprime l'élément. -La configuration peut également définir “substitutions”. Si un élément a le `incl` attribut, la substitution correspondante du fichier sera utilisée comme valeur. Par défaut, le chemin d'accès au fichier avec des substitutions est `/etc/metrika.xml`. Ceci peut être changé dans le [include\_from](server-configuration-parameters/settings.md#server_configuration_parameters-include_from) élément dans la configuration du serveur. Les valeurs de substitution sont spécifiées dans `/yandex/substitution_name` les éléments de ce fichier. Si une substitution spécifiée dans `incl` n'existe pas, il est enregistré dans le journal. Pour empêcher ClickHouse de consigner les substitutions manquantes, spécifiez `optional="true"` attribut (par exemple, les paramètres de [macro](server-configuration-parameters/settings.md)). +La configuration peut également définir “substitutions”. Si un élément a le `incl` attribut, la substitution correspondante du fichier sera utilisée comme valeur. Par défaut, le chemin d'accès au fichier avec des substitutions est `/etc/metrika.xml`. Ceci peut être changé dans le [include_from](server-configuration-parameters/settings.md#server_configuration_parameters-include_from) élément dans la configuration du serveur. Les valeurs de substitution sont spécifiées dans `/yandex/substitution_name` les éléments de ce fichier. Si une substitution spécifiée dans `incl` n'existe pas, il est enregistré dans le journal. Pour empêcher ClickHouse de consigner les substitutions manquantes, spécifiez `optional="true"` attribut (par exemple, les paramètres de [macro](server-configuration-parameters/settings.md)). Les Substitutions peuvent également être effectuées à partir de ZooKeeper. Pour ce faire, spécifiez l'attribut `from_zk = "/path/to/node"`. La valeur de l'élément est remplacé par le contenu du noeud au `/path/to/node` dans ZooKeeper. Vous pouvez également placer un sous-arbre XML entier sur le nœud ZooKeeper et il sera entièrement inséré dans l'élément source. diff --git a/docs/fr/operations/monitoring.md b/docs/fr/operations/monitoring.md index 275e8a5d8ac..91aee4bc1f8 100644 --- a/docs/fr/operations/monitoring.md +++ b/docs/fr/operations/monitoring.md @@ -35,7 +35,7 @@ Clickhouse recueille: - Différentes mesures de la façon dont le serveur utilise les ressources de calcul. - Statistiques communes sur le traitement des requêtes. -Vous pouvez trouver des mesures dans le [système.métrique](../operations/system-tables.md#system_tables-metrics), [système.événement](../operations/system-tables.md#system_tables-events), et [système.asynchronous\_metrics](../operations/system-tables.md#system_tables-asynchronous_metrics) table. +Vous pouvez trouver des mesures dans le [système.métrique](../operations/system-tables.md#system_tables-metrics), [système.événement](../operations/system-tables.md#system_tables-events), et [système.asynchronous_metrics](../operations/system-tables.md#system_tables-asynchronous_metrics) table. Vous pouvez configurer ClickHouse pour exporter des métriques vers [Graphite](https://github.com/graphite-project). Voir la [Graphite section](server-configuration-parameters/settings.md#server_configuration_parameters-graphite) dans le fichier de configuration du serveur ClickHouse. Avant de configurer l'exportation des métriques, vous devez configurer Graphite en suivant leur [guide](https://graphite.readthedocs.io/en/latest/install.html). @@ -43,4 +43,4 @@ Vous pouvez configurer ClickHouse pour exporter des métriques vers [Prometheus] De plus, vous pouvez surveiller la disponibilité du serveur via L'API HTTP. Envoyer la `HTTP GET` demande à `/ping`. Si le serveur est disponible, il répond avec `200 OK`. -Pour surveiller les serveurs dans une configuration de cluster, vous devez [max\_replica\_delay\_for\_distributed\_queries](settings/settings.md#settings-max_replica_delay_for_distributed_queries) paramètre et utiliser la ressource HTTP `/replicas_status`. Une demande de `/replicas_status` retourner `200 OK` si la réplique est disponible et n'est pas retardé derrière les autres réplicas. Si une réplique est retardée, elle revient `503 HTTP_SERVICE_UNAVAILABLE` avec des informations sur l'écart. +Pour surveiller les serveurs dans une configuration de cluster, vous devez [max_replica_delay_for_distributed_queries](settings/settings.md#settings-max_replica_delay_for_distributed_queries) paramètre et utiliser la ressource HTTP `/replicas_status`. Une demande de `/replicas_status` retourner `200 OK` si la réplique est disponible et n'est pas retardé derrière les autres réplicas. Si une réplique est retardée, elle revient `503 HTTP_SERVICE_UNAVAILABLE` avec des informations sur l'écart. diff --git a/docs/fr/operations/optimizing-performance/sampling-query-profiler.md b/docs/fr/operations/optimizing-performance/sampling-query-profiler.md index ba8f3ecd110..eb2cdf7f12c 100644 --- a/docs/fr/operations/optimizing-performance/sampling-query-profiler.md +++ b/docs/fr/operations/optimizing-performance/sampling-query-profiler.md @@ -11,11 +11,11 @@ ClickHouse exécute un profileur d'échantillonnage qui permet d'analyser l'exé Utilisation du générateur de profils: -- Installation de la [trace\_log](../server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) la section de la configuration du serveur. +- Installation de la [trace_log](../server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) la section de la configuration du serveur. - Cette section configure le [trace\_log](../../operations/system-tables.md#system_tables-trace_log) tableau système contenant les résultats du fonctionnement du profileur. Il est configuré par défaut. Rappelez-vous que les données de ce tableau est valable que pour un serveur en cours d'exécution. Après le redémarrage du serveur, ClickHouse ne nettoie pas la table et toute l'adresse de mémoire virtuelle stockée peut devenir invalide. + Cette section configure le [trace_log](../../operations/system-tables.md#system_tables-trace_log) tableau système contenant les résultats du fonctionnement du profileur. Il est configuré par défaut. Rappelez-vous que les données de ce tableau est valable que pour un serveur en cours d'exécution. Après le redémarrage du serveur, ClickHouse ne nettoie pas la table et toute l'adresse de mémoire virtuelle stockée peut devenir invalide. -- Installation de la [query\_profiler\_cpu\_time\_period\_ns](../settings/settings.md#query_profiler_cpu_time_period_ns) ou [query\_profiler\_real\_time\_period\_ns](../settings/settings.md#query_profiler_real_time_period_ns) paramètre. Les deux paramètres peuvent être utilisés simultanément. +- Installation de la [query_profiler_cpu_time_period_ns](../settings/settings.md#query_profiler_cpu_time_period_ns) ou [query_profiler_real_time_period_ns](../settings/settings.md#query_profiler_real_time_period_ns) paramètre. Les deux paramètres peuvent être utilisés simultanément. Ces paramètres vous permettent de configurer les minuteries du profileur. Comme il s'agit des paramètres de session, vous pouvez obtenir une fréquence d'échantillonnage différente pour l'ensemble du serveur, les utilisateurs individuels ou les profils d'utilisateurs, pour votre session interactive et pour chaque requête individuelle. @@ -25,7 +25,7 @@ Pour analyser les `trace_log` système de table: - Installer le `clickhouse-common-static-dbg` paquet. Voir [Installer à partir de paquets DEB](../../getting-started/install.md#install-from-deb-packages). -- Autoriser les fonctions d'introspection par [allow\_introspection\_functions](../settings/settings.md#settings-allow_introspection_functions) paramètre. +- Autoriser les fonctions d'introspection par [allow_introspection_functions](../settings/settings.md#settings-allow_introspection_functions) paramètre. Pour des raisons de sécurité, les fonctions d'introspection sont désactivées par défaut. diff --git a/docs/fr/operations/requirements.md b/docs/fr/operations/requirements.md index 936d6f57f5b..1d79f3fcf82 100644 --- a/docs/fr/operations/requirements.md +++ b/docs/fr/operations/requirements.md @@ -9,7 +9,7 @@ toc_title: Exigence ## CPU {#cpu} -Pour l'installation à partir de paquets deb prédéfinis, utilisez un processeur avec l'architecture x86\_64 et la prise en charge des instructions SSE 4.2. Pour exécuter ClickHouse avec des processeurs qui ne prennent pas en charge SSE 4.2 ou qui ont une architecture AArch64 ou PowerPC64LE, vous devez créer ClickHouse à partir de sources. +Pour l'installation à partir de paquets deb prédéfinis, utilisez un processeur avec l'architecture x86_64 et la prise en charge des instructions SSE 4.2. Pour exécuter ClickHouse avec des processeurs qui ne prennent pas en charge SSE 4.2 ou qui ont une architecture AArch64 ou PowerPC64LE, vous devez créer ClickHouse à partir de sources. ClickHouse implémente le traitement parallèle des données et utilise toutes les ressources matérielles disponibles. Lors du choix d'un processeur, tenez compte du fait que ClickHouse fonctionne plus efficacement dans les configurations avec un grand nombre de cœurs mais une fréquence d'horloge plus faible que dans les configurations avec moins de cœurs et une fréquence d'horloge plus élevée. Par exemple, 16 cœurs avec 2600 MHz est préférable à 8 cœurs avec 3600 MHz. diff --git a/docs/fr/operations/server-configuration-parameters/settings.md b/docs/fr/operations/server-configuration-parameters/settings.md index 45be3c5c009..af83f3f5ed7 100644 --- a/docs/fr/operations/server-configuration-parameters/settings.md +++ b/docs/fr/operations/server-configuration-parameters/settings.md @@ -7,7 +7,7 @@ toc_title: "Les Param\xE8tres Du Serveur" # Les Paramètres Du Serveur {#server-settings} -## builtin\_dictionaries\_reload\_interval {#builtin-dictionaries-reload-interval} +## builtin_dictionaries_reload_interval {#builtin-dictionaries-reload-interval} L'intervalle en secondes avant de recharger les dictionnaires intégrés. @@ -68,7 +68,7 @@ Si aucune condition n'est remplie pour une partie de données, ClickHouse utilis ``` -## default\_database {#default-database} +## default_database {#default-database} La base de données par défaut. @@ -80,7 +80,7 @@ Pour obtenir une liste de bases de données, utilisez la [SHOW DATABASES](../../ default ``` -## default\_profile {#default-profile} +## default_profile {#default-profile} Profil des paramètres par défaut. @@ -92,7 +92,7 @@ Les paramètres des profils sont situés dans le fichier spécifié dans le para default ``` -## dictionaries\_config {#server_configuration_parameters-dictionaries_config} +## dictionaries_config {#server_configuration_parameters-dictionaries_config} Chemin d'accès au fichier de configuration des dictionnaires externes. @@ -109,7 +109,7 @@ Voir aussi “[Dictionnaires externes](../../sql-reference/dictionaries/external *_dictionary.xml ``` -## dictionaries\_lazy\_load {#server_configuration_parameters-dictionaries_lazy_load} +## dictionaries_lazy_load {#server_configuration_parameters-dictionaries_lazy_load} Chargement paresseux des dictionnaires. @@ -125,7 +125,7 @@ La valeur par défaut est `true`. true ``` -## format\_schema\_path {#server_configuration_parameters-format_schema_path} +## format_schema_path {#server_configuration_parameters-format_schema_path} Le chemin d'accès au répertoire avec des régimes pour l'entrée de données, tels que les schémas pour l' [CapnProto](../../interfaces/formats.md#capnproto) format. @@ -146,11 +146,11 @@ Paramètre: - port – The port on the Graphite server. - interval – The interval for sending, in seconds. - timeout – The timeout for sending data, in seconds. -- root\_path – Prefix for keys. +- root_path – Prefix for keys. - metrics – Sending data from the [système.métrique](../../operations/system-tables.md#system_tables-metrics) table. - events – Sending deltas data accumulated for the time period from the [système.événement](../../operations/system-tables.md#system_tables-events) table. -- events\_cumulative – Sending cumulative data from the [système.événement](../../operations/system-tables.md#system_tables-events) table. -- asynchronous\_metrics – Sending data from the [système.asynchronous\_metrics](../../operations/system-tables.md#system_tables-asynchronous_metrics) table. +- events_cumulative – Sending cumulative data from the [système.événement](../../operations/system-tables.md#system_tables-events) table. +- asynchronous_metrics – Sending data from the [système.asynchronous_metrics](../../operations/system-tables.md#system_tables-asynchronous_metrics) table. Vous pouvez configurer plusieurs `` clause. Par exemple, vous pouvez l'utiliser pour envoyer des données différentes à différents intervalles. @@ -170,7 +170,7 @@ Vous pouvez configurer plusieurs `` clause. Par exemple, vous pouvez l ``` -## graphite\_rollup {#server_configuration_parameters-graphite-rollup} +## graphite_rollup {#server_configuration_parameters-graphite-rollup} Paramètres pour l'amincissement des données pour le Graphite. @@ -198,7 +198,7 @@ Pour plus de détails, voir [GraphiteMergeTree](../../engines/table-engines/merg ``` -## http\_port/https\_port {#http-porthttps-port} +## http_port/https_port {#http-porthttps-port} Port de connexion au serveur via HTTP(S). @@ -212,7 +212,7 @@ Si `http_port` est spécifié, la configuration OpenSSL est ignorée même si el 9999 ``` -## http\_server\_default\_response {#server_configuration_parameters-http_server_default_response} +## http_server_default_response {#server_configuration_parameters-http_server_default_response} Page affichée par défaut lorsque vous accédez au serveur HTTP(S) ClickHouse. La valeur par défaut est “Ok.” (avec un saut de ligne à la fin) @@ -227,7 +227,7 @@ Ouvrir `https://tabix.io/` lors de l'accès à `http://localhost: http_port`. ``` -## include\_from {#server_configuration_parameters-include_from} +## include_from {#server_configuration_parameters-include_from} Le chemin d'accès au fichier avec des substitutions. @@ -239,7 +239,7 @@ Pour plus d'informations, consultez la section “[Fichiers de Configuration](.. /etc/metrica.xml ``` -## interserver\_http\_port {#interserver-http-port} +## interserver_http_port {#interserver-http-port} Port pour l'échange de données entre les serveurs ClickHouse. @@ -249,7 +249,7 @@ Port pour l'échange de données entre les serveurs ClickHouse. 9009 ``` -## interserver\_http\_host {#interserver-http-host} +## interserver_http_host {#interserver-http-host} Le nom d'hôte qui peut être utilisé par d'autres serveurs pour accéder à ce serveur. @@ -263,7 +263,7 @@ Utile pour rompre avec une interface réseau spécifique. example.yandex.ru ``` -## interserver\_http\_credentials {#server-settings-interserver-http-credentials} +## interserver_http_credentials {#server-settings-interserver-http-credentials} Le nom d'utilisateur et le mot de passe utilisés pour [réplication](../../engines/table-engines/mergetree-family/replication.md) avec les moteurs \* répliqués. Ces informations d'identification sont utilisées uniquement pour la communication entre les répliques et ne sont pas liées aux informations d'identification des clients ClickHouse. Le serveur vérifie ces informations d'identification pour la connexion de répliques et utilise les mêmes informations d'identification lors de la connexion à d'autres répliques. Donc, ces informations d'identification doivent être identiques pour tous les réplicas dans un cluster. Par défaut, l'authentification n'est pas utilisé. @@ -282,7 +282,7 @@ Cette section contient les paramètres suivants: ``` -## keep\_alive\_timeout {#keep-alive-timeout} +## keep_alive_timeout {#keep-alive-timeout} Le nombre de secondes que ClickHouse attend pour les demandes entrantes avant de fermer la connexion. Par défaut est de 3 secondes. @@ -292,7 +292,7 @@ Le nombre de secondes que ClickHouse attend pour les demandes entrantes avant de 3 ``` -## listen\_host {#server_configuration_parameters-listen_host} +## listen_host {#server_configuration_parameters-listen_host} Restriction sur les hôtes dont les demandes peuvent provenir. Si vous voulez que le serveur réponde à tous, spécifiez `::`. @@ -343,10 +343,10 @@ L'écriture dans le syslog est également prise en charge. Exemple de Config: Touches: -- use\_syslog — Required setting if you want to write to the syslog. +- use_syslog — Required setting if you want to write to the syslog. - address — The host\[:port\] of syslogd. If omitted, the local daemon is used. - hostname — Optional. The name of the host that logs are sent from. -- facility — [Le mot clé syslog facility](https://en.wikipedia.org/wiki/Syslog#Facility) en majuscules avec la “LOG\_” préfixe: (`LOG_USER`, `LOG_DAEMON`, `LOG_LOCAL3` et ainsi de suite). +- facility — [Le mot clé syslog facility](https://en.wikipedia.org/wiki/Syslog#Facility) en majuscules avec la “LOG_” préfixe: (`LOG_USER`, `LOG_DAEMON`, `LOG_LOCAL3` et ainsi de suite). Valeur par défaut: `LOG_USER` si `address` est spécifié, `LOG_DAEMON otherwise.` - format – Message format. Possible values: `bsd` et `syslog.` @@ -364,7 +364,7 @@ Pour plus d'informations, consultez la section “[Création de tables répliqu ``` -## mark\_cache\_size {#server-mark-cache-size} +## mark_cache_size {#server-mark-cache-size} Taille approximative (en octets) du cache des marques utilisées par les [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) famille. @@ -376,7 +376,7 @@ Le cache est partagé pour le serveur et la mémoire est allouée au besoin. La 5368709120 ``` -## max\_concurrent\_queries {#max-concurrent-queries} +## max_concurrent_queries {#max-concurrent-queries} Nombre maximal de demandes traitées simultanément. @@ -386,7 +386,7 @@ Nombre maximal de demandes traitées simultanément. 100 ``` -## max\_connections {#max-connections} +## max_connections {#max-connections} Le nombre maximal de connexions entrantes. @@ -396,7 +396,7 @@ Le nombre maximal de connexions entrantes. 4096 ``` -## max\_open\_files {#max-open-files} +## max_open_files {#max-open-files} Le nombre maximal de fichiers ouverts. @@ -410,7 +410,7 @@ Nous vous recommandons d'utiliser cette option sous Mac OS X depuis le `getrlimi 262144 ``` -## max\_table\_size\_to\_drop {#max-table-size-to-drop} +## max_table_size_to_drop {#max-table-size-to-drop} Restriction sur la suppression de tables. @@ -428,7 +428,7 @@ La valeur 0 signifie que vous pouvez supprimer toutes les tables sans aucune res 0 ``` -## merge\_tree {#server_configuration_parameters-merge_tree} +## merge_tree {#server_configuration_parameters-merge_tree} Réglage fin des tables dans le [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). @@ -463,7 +463,7 @@ Clés pour les paramètres Serveur/client: - sessionTimeout – Time for caching the session on the server. - extendedVerification – Automatically extended verification of certificates after the session ends. Acceptable values: `true`, `false`. - requireTLSv1 – Require a TLSv1 connection. Acceptable values: `true`, `false`. -- requireTLSv1\_1 – Require a TLSv1.1 connection. Acceptable values: `true`, `false`. +- requireTLSv1_1 – Require a TLSv1.1 connection. Acceptable values: `true`, `false`. - requireTLSv1 – Require a TLSv1.2 connection. Acceptable values: `true`, `false`. - fips – Activates OpenSSL FIPS mode. Supported if the library's OpenSSL version supports FIPS. - privateKeyPassphraseHandler – Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: ``, `KeyFileHandler`, `test`, ``. @@ -501,11 +501,11 @@ Clés pour les paramètres Serveur/client: ``` -## part\_log {#server_configuration_parameters-part-log} +## part_log {#server_configuration_parameters-part-log} Journalisation des événements associés à [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). Par exemple, ajouter ou fusionner des données. Vous pouvez utiliser le journal pour simuler des algorithmes de fusion et comparer leurs caractéristiques. Vous pouvez visualiser le processus de fusion. -Les requêtes sont enregistrées dans le [système.part\_log](../../operations/system-tables.md#system_tables-part-log) table, pas dans un fichier séparé. Vous pouvez configurer le nom de cette table dans le `table` paramètre (voir ci-dessous). +Les requêtes sont enregistrées dans le [système.part_log](../../operations/system-tables.md#system_tables-part-log) table, pas dans un fichier séparé. Vous pouvez configurer le nom de cette table dans le `table` paramètre (voir ci-dessous). Utilisez les paramètres suivants pour configurer la journalisation: @@ -548,7 +548,7 @@ Paramètre: - `port` – Port for `endpoint`. - `metrics` – Flag that sets to expose metrics from the [système.métrique](../system-tables.md#system_tables-metrics) table. - `events` – Flag that sets to expose metrics from the [système.événement](../system-tables.md#system_tables-events) table. -- `asynchronous_metrics` – Flag that sets to expose current metrics values from the [système.asynchronous\_metrics](../system-tables.md#system_tables-asynchronous_metrics) table. +- `asynchronous_metrics` – Flag that sets to expose current metrics values from the [système.asynchronous_metrics](../system-tables.md#system_tables-asynchronous_metrics) table. **Exemple** @@ -562,11 +562,11 @@ Paramètre: ``` -## query\_log {#server_configuration_parameters-query-log} +## query_log {#server_configuration_parameters-query-log} -Réglage de la journalisation des requêtes reçues avec [log\_queries=1](../settings/settings.md) paramètre. +Réglage de la journalisation des requêtes reçues avec [log_queries=1](../settings/settings.md) paramètre. -Les requêtes sont enregistrées dans le [système.query\_log](../../operations/system-tables.md#system_tables-query_log) table, pas dans un fichier séparé. Vous pouvez modifier le nom de la table dans le `table` paramètre (voir ci-dessous). +Les requêtes sont enregistrées dans le [système.query_log](../../operations/system-tables.md#system_tables-query_log) table, pas dans un fichier séparé. Vous pouvez modifier le nom de la table dans le `table` paramètre (voir ci-dessous). Utilisez les paramètres suivants pour configurer la journalisation: @@ -588,11 +588,11 @@ Si la table n'existe pas, ClickHouse la créera. Si la structure du journal des ``` -## query\_thread\_log {#server_configuration_parameters-query-thread-log} +## query_thread_log {#server_configuration_parameters-query-thread-log} -Réglage de la journalisation des threads de requêtes reçues avec [log\_query\_threads=1](../settings/settings.md#settings-log-query-threads) paramètre. +Réglage de la journalisation des threads de requêtes reçues avec [log_query_threads=1](../settings/settings.md#settings-log-query-threads) paramètre. -Les requêtes sont enregistrées dans le [système.query\_thread\_log](../../operations/system-tables.md#system_tables-query-thread-log) table, pas dans un fichier séparé. Vous pouvez modifier le nom de la table dans le `table` paramètre (voir ci-dessous). +Les requêtes sont enregistrées dans le [système.query_thread_log](../../operations/system-tables.md#system_tables-query-thread-log) table, pas dans un fichier séparé. Vous pouvez modifier le nom de la table dans le `table` paramètre (voir ci-dessous). Utilisez les paramètres suivants pour configurer la journalisation: @@ -614,9 +614,9 @@ Si la table n'existe pas, ClickHouse la créera. Si la structure du journal des ``` -## trace\_log {#server_configuration_parameters-trace_log} +## trace_log {#server_configuration_parameters-trace_log} -Paramètres pour le [trace\_log](../../operations/system-tables.md#system_tables-trace_log) opération de table de système. +Paramètres pour le [trace_log](../../operations/system-tables.md#system_tables-trace_log) opération de table de système. Paramètre: @@ -636,7 +636,7 @@ Le fichier de configuration du serveur par défaut `config.xml` contient la sect ``` -## query\_masking\_rules {#query-masking-rules} +## query_masking_rules {#query-masking-rules} Règles basées sur Regexp, qui seront appliquées aux requêtes ainsi qu'à tous les messages de journal avant de les stocker dans les journaux du serveur, `system.query_log`, `system.text_log`, `system.processes` table, et dans les journaux envoyés au client. Qui permet à la prévention de @@ -667,7 +667,7 @@ Les règles de masquage sont appliquées à l'ensemble de la requête (pour évi Pour les requêtes distribuées chaque serveur doivent être configurés séparément, sinon, les sous-requêtes transmises à d'autres les nœuds seront stockés sans masquage. -## remote\_servers {#server-settings-remote-servers} +## remote_servers {#server-settings-remote-servers} Configuration des clusters utilisés par le [Distribué](../../engines/table-engines/special/distributed.md) moteur de table et par le `cluster` table de fonction. @@ -681,7 +681,7 @@ Pour la valeur de l' `incl` attribut, voir la section “[Fichiers de Configurat **Voir Aussi** -- [skip\_unavailable\_shards](../settings/settings.md#settings-skip_unavailable_shards) +- [skip_unavailable_shards](../settings/settings.md#settings-skip_unavailable_shards) ## fuseau {#server_configuration_parameters-timezone} @@ -697,7 +697,7 @@ Le fuseau horaire est nécessaire pour les conversions entre les formats String Europe/Moscow ``` -## tcp\_port {#server_configuration_parameters-tcp_port} +## tcp_port {#server_configuration_parameters-tcp_port} Port pour communiquer avec les clients via le protocole TCP. @@ -707,7 +707,7 @@ Port pour communiquer avec les clients via le protocole TCP. 9000 ``` -## tcp\_port\_secure {#server_configuration_parameters-tcp_port_secure} +## tcp_port_secure {#server_configuration_parameters-tcp_port_secure} Port TCP pour une communication sécurisée avec les clients. Utilisez le avec [OpenSSL](#server_configuration_parameters-openssl) paramètre. @@ -721,7 +721,7 @@ Entier positif. 9440 ``` -## mysql\_port {#server_configuration_parameters-mysql_port} +## mysql_port {#server_configuration_parameters-mysql_port} Port pour communiquer avec les clients via le protocole MySQL. @@ -735,7 +735,7 @@ Exemple 9004 ``` -## tmp\_path {#server-settings-tmp_path} +## tmp_path {#server-settings-tmp_path} Chemin d'accès aux données temporaires pour le traitement des requêtes volumineuses. @@ -748,7 +748,7 @@ Chemin d'accès aux données temporaires pour le traitement des requêtes volumi /var/lib/clickhouse/tmp/ ``` -## tmp\_policy {#server-settings-tmp-policy} +## tmp_policy {#server-settings-tmp-policy} La politique de [`storage_configuration`](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) pour stocker des fichiers temporaires. Si cela n'est pas [`tmp_path`](#server-settings-tmp_path) est utilisé, sinon elle est ignorée. @@ -759,11 +759,11 @@ Si cela n'est pas [`tmp_path`](#server-settings-tmp_path) est utilisé, sinon el - `max_data_part_size_bytes` est ignoré - vous devez avoir exactement un volume dans cette politique -## uncompressed\_cache\_size {#server-settings-uncompressed_cache_size} +## uncompressed_cache_size {#server-settings-uncompressed_cache_size} Taille du Cache (en octets) pour les données non compressées utilisées par les [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). -Il y a un cache partagé pour le serveur. La mémoire est allouée à la demande. Le cache est utilisé si l'option [use\_uncompressed\_cache](../settings/settings.md#setting-use_uncompressed_cache) est activé. +Il y a un cache partagé pour le serveur. La mémoire est allouée à la demande. Le cache est utilisé si l'option [use_uncompressed_cache](../settings/settings.md#setting-use_uncompressed_cache) est activé. Le cache non compressé est avantageux pour les requêtes très courtes dans des cas individuels. @@ -773,7 +773,7 @@ Le cache non compressé est avantageux pour les requêtes très courtes dans des 8589934592 ``` -## user\_files\_path {#server_configuration_parameters-user_files_path} +## user_files_path {#server_configuration_parameters-user_files_path} Le répertoire avec les fichiers utilisateur. Utilisé dans la fonction de table [fichier()](../../sql-reference/table-functions/file.md). @@ -783,7 +783,7 @@ Le répertoire avec les fichiers utilisateur. Utilisé dans la fonction de table /var/lib/clickhouse/user_files/ ``` -## users\_config {#users-config} +## users_config {#users-config} Chemin d'accès au fichier qui contient: @@ -851,13 +851,13 @@ Cette section contient les paramètres suivants: - [Réplication](../../engines/table-engines/mergetree-family/replication.md) - [Guide du programmeur ZooKeeper](http://zookeeper.apache.org/doc/current/zookeeperProgrammers.html) -## use\_minimalistic\_part\_header\_in\_zookeeper {#server-settings-use_minimalistic_part_header_in_zookeeper} +## use_minimalistic_part_header_in_zookeeper {#server-settings-use_minimalistic_part_header_in_zookeeper} Méthode de stockage pour les en-têtes de partie de données dans ZooKeeper. Ce paramètre s'applique uniquement à l' `MergeTree` famille. Il peut être spécifié: -- À l'échelle mondiale dans le [merge\_tree](#server_configuration_parameters-merge_tree) la section de la `config.xml` fichier. +- À l'échelle mondiale dans le [merge_tree](#server_configuration_parameters-merge_tree) la section de la `config.xml` fichier. ClickHouse utilise le paramètre pour toutes les tables du serveur. Vous pouvez modifier le réglage à tout moment. Les tables existantes changent de comportement lorsque le paramètre change. @@ -879,21 +879,21 @@ Si `use_minimalistic_part_header_in_zookeeper = 1`, puis [répliqué](../../engi **Valeur par défaut:** 0. -## disable\_internal\_dns\_cache {#server-settings-disable-internal-dns-cache} +## disable_internal_dns_cache {#server-settings-disable-internal-dns-cache} Désactive le cache DNS interne. Recommandé pour l'utilisation de ClickHouse dans les systèmes avec des infrastructures en constante évolution telles que Kubernetes. **Valeur par défaut:** 0. -## dns\_cache\_update\_period {#server-settings-dns-cache-update-period} +## dns_cache_update_period {#server-settings-dns-cache-update-period} La période de mise à jour des adresses IP stockées dans le cache DNS interne de ClickHouse (en secondes). La mise à jour est effectuée de manière asynchrone, dans un thread système séparé. **Valeur par défaut**: 15. -## access\_control\_path {#access_control_path} +## access_control_path {#access_control_path} Chemin d'accès à un dossier dans lequel un serveur clickhouse stocke les configurations utilisateur et rôle créées par les commandes SQL. diff --git a/docs/fr/operations/settings/permissions-for-queries.md b/docs/fr/operations/settings/permissions-for-queries.md index 9107438e4b5..afab946adb3 100644 --- a/docs/fr/operations/settings/permissions-for-queries.md +++ b/docs/fr/operations/settings/permissions-for-queries.md @@ -18,7 +18,7 @@ Les requêtes dans ClickHouse peuvent être divisées en plusieurs types: Les paramètres suivants règlent les autorisations utilisateur selon le type de requête: - [ReadOnly](#settings_readonly) — Restricts permissions for all types of queries except DDL queries. -- [allow\_ddl](#settings_allow_ddl) — Restricts permissions for DDL queries. +- [allow_ddl](#settings_allow_ddl) — Restricts permissions for DDL queries. `KILL QUERY` peut être réalisée avec tous les paramètres. @@ -43,7 +43,7 @@ de modifier uniquement des paramètres spécifiques, pour plus de détails, voir Valeur par défaut: 0 -## allow\_ddl {#settings_allow_ddl} +## allow_ddl {#settings_allow_ddl} Permet ou interdit [DDL](https://en.wikipedia.org/wiki/Data_definition_language) requête. diff --git a/docs/fr/operations/settings/query-complexity.md b/docs/fr/operations/settings/query-complexity.md index 7242c2e0222..bdeb3e35242 100644 --- a/docs/fr/operations/settings/query-complexity.md +++ b/docs/fr/operations/settings/query-complexity.md @@ -14,8 +14,8 @@ Presque toutes les restrictions ne s'appliquent qu'à `SELECT`. Pour le traiteme ClickHouse vérifie les restrictions pour les parties de données, pas pour chaque ligne. Cela signifie que vous pouvez dépasser la valeur de restriction de la taille de la partie données. Restrictions sur l' “maximum amount of something” peut prendre la valeur 0, ce qui signifie “unrestricted”. -La plupart des restrictions ont également un ‘overflow\_mode’ paramètre signification que faire lorsque la limite est dépassée. -Il peut prendre deux valeurs: `throw` ou `break`. Les Restrictions sur l'agrégation (group\_by\_overflow\_mode) ont également la valeur `any`. +La plupart des restrictions ont également un ‘overflow_mode’ paramètre signification que faire lorsque la limite est dépassée. +Il peut prendre deux valeurs: `throw` ou `break`. Les Restrictions sur l'agrégation (group_by_overflow_mode) ont également la valeur `any`. `throw` – Throw an exception (default). @@ -23,7 +23,7 @@ Il peut prendre deux valeurs: `throw` ou `break`. Les Restrictions sur l'agréga `any (only for group_by_overflow_mode)` – Continuing aggregation for the keys that got into the set, but don't add new keys to the set. -## max\_memory\_usage {#settings_max_memory_usage} +## max_memory_usage {#settings_max_memory_usage} La quantité maximale de RAM à utiliser pour exécuter une requête sur un seul serveur. @@ -40,47 +40,47 @@ L'utilisation de la mémoire n'est pas totalement suivies pour les états des fo La consommation de mémoire est également limitée par les paramètres `max_memory_usage_for_user` et `max_memory_usage_for_all_queries`. -## max\_memory\_usage\_for\_user {#max-memory-usage-for-user} +## max_memory_usage_for_user {#max-memory-usage-for-user} Quantité maximale de RAM à utiliser pour exécuter les requêtes d'un utilisateur sur un seul serveur. Les valeurs par défaut sont définies dans [Paramètre.h](https://github.com/ClickHouse/ClickHouse/blob/master/src/Core/Settings.h#L288). Par défaut, le montant n'est pas limité (`max_memory_usage_for_user = 0`). -Voir aussi la description de [max\_memory\_usage](#settings_max_memory_usage). +Voir aussi la description de [max_memory_usage](#settings_max_memory_usage). -## max\_memory\_usage\_for\_all\_queries {#max-memory-usage-for-all-queries} +## max_memory_usage_for_all_queries {#max-memory-usage-for-all-queries} La quantité maximale de RAM à utiliser pour exécuter toutes les requêtes sur un seul serveur. Les valeurs par défaut sont définies dans [Paramètre.h](https://github.com/ClickHouse/ClickHouse/blob/master/src/Core/Settings.h#L289). Par défaut, le montant n'est pas limité (`max_memory_usage_for_all_queries = 0`). -Voir aussi la description de [max\_memory\_usage](#settings_max_memory_usage). +Voir aussi la description de [max_memory_usage](#settings_max_memory_usage). -## max\_rows\_to\_read {#max-rows-to-read} +## max_rows_to_read {#max-rows-to-read} Les restrictions suivantes peut être vérifiée sur chaque bloc (au lieu de sur chaque ligne). Autrement dit, les restrictions peuvent être brisées un peu. Lors de l'exécution d'une requête dans plusieurs threads, les restrictions suivantes s'appliquent à chaque thread séparément. Un nombre maximum de lignes pouvant être lues à partir d'un tableau lors de l'exécution d'une requête. -## max\_bytes\_to\_read {#max-bytes-to-read} +## max_bytes_to_read {#max-bytes-to-read} Nombre maximal d'octets (données non compressées) pouvant être lus à partir d'une table lors de l'exécution d'une requête. -## read\_overflow\_mode {#read-overflow-mode} +## read_overflow_mode {#read-overflow-mode} Que faire lorsque le volume de lecture de données dépasse l'une des limites: ‘throw’ ou ‘break’. Par défaut, les jeter. -## max\_rows\_to\_group\_by {#settings-max-rows-to-group-by} +## max_rows_to_group_by {#settings-max-rows-to-group-by} Un nombre maximum de clés uniques reçues de l'agrégation. Ce paramètre permet de limiter la consommation de mémoire lors de l'agrégation. -## group\_by\_overflow\_mode {#group-by-overflow-mode} +## group_by_overflow_mode {#group-by-overflow-mode} Que faire lorsque le nombre de clés uniques pour l'agrégation dépasse la limite: ‘throw’, ‘break’, ou ‘any’. Par défaut, les jeter. À l'aide de la ‘any’ valeur vous permet d'exécuter une approximation de GROUP BY. La qualité de cette approximation dépend de la nature statistique des données. -## max\_bytes\_before\_external\_group\_by {#settings-max_bytes_before_external_group_by} +## max_bytes_before_external_group_by {#settings-max_bytes_before_external_group_by} Active ou désactive l'exécution de `GROUP BY` clauses dans la mémoire externe. Voir [Groupe par dans la mémoire externe](../../sql-reference/statements/select/group-by.md#select-group-by-in-external-memory). @@ -91,31 +91,31 @@ Valeurs possibles: Valeur par défaut: 0. -## max\_rows\_to\_sort {#max-rows-to-sort} +## max_rows_to_sort {#max-rows-to-sort} Un nombre maximum de lignes avant le tri. Cela vous permet de limiter la consommation de mémoire lors du tri. -## max\_bytes\_to\_sort {#max-bytes-to-sort} +## max_bytes_to_sort {#max-bytes-to-sort} Un nombre maximal d'octets avant le tri. -## sort\_overflow\_mode {#sort-overflow-mode} +## sort_overflow_mode {#sort-overflow-mode} Que faire si le nombre de lignes reçues avant le tri dépasse l'une des limites: ‘throw’ ou ‘break’. Par défaut, les jeter. -## max\_result\_rows {#setting-max_result_rows} +## max_result_rows {#setting-max_result_rows} Limite sur le nombre de lignes dans le résultat. Également vérifié pour les sous-requêtes, et sur des serveurs distants lors de l'exécution de parties d'une requête distribuée. -## max\_result\_bytes {#max-result-bytes} +## max_result_bytes {#max-result-bytes} Limite sur le nombre d'octets dans le résultat. Le même que le réglage précédent. -## result\_overflow\_mode {#result-overflow-mode} +## result_overflow_mode {#result-overflow-mode} Que faire si le volume du résultat dépasse l'une des limites: ‘throw’ ou ‘break’. Par défaut, les jeter. -Utiliser ‘break’ est similaire à L'utilisation de LIMIT. `Break` interrompt l'exécution seulement au niveau du bloc. Cela signifie que la quantité de lignes renvoyées est supérieure à [max\_result\_rows](#setting-max_result_rows) multiples de [max\_block\_size](settings.md#setting-max_block_size) et dépend de l' [max\_threads](settings.md#settings-max_threads). +Utiliser ‘break’ est similaire à L'utilisation de LIMIT. `Break` interrompt l'exécution seulement au niveau du bloc. Cela signifie que la quantité de lignes renvoyées est supérieure à [max_result_rows](#setting-max_result_rows) multiples de [max_block_size](settings.md#setting-max_block_size) et dépend de l' [max_threads](settings.md#settings-max_threads). Exemple: @@ -134,103 +134,103 @@ Résultat: 6666 rows in set. ... ``` -## max\_execution\_time {#max-execution-time} +## max_execution_time {#max-execution-time} Durée maximale d'exécution de la requête en secondes. Pour le moment, il n'est pas vérifié pour l'une des étapes de tri, ni lors de la fusion et de la finalisation des fonctions d'agrégat. -## timeout\_overflow\_mode {#timeout-overflow-mode} +## timeout_overflow_mode {#timeout-overflow-mode} -Que faire si la requête est exécutée plus de ‘max\_execution\_time’: ‘throw’ ou ‘break’. Par défaut, les jeter. +Que faire si la requête est exécutée plus de ‘max_execution_time’: ‘throw’ ou ‘break’. Par défaut, les jeter. -## min\_execution\_speed {#min-execution-speed} +## min_execution_speed {#min-execution-speed} -Vitesse d'exécution minimale en lignes par seconde. Vérifié sur chaque bloc de données quand ‘timeout\_before\_checking\_execution\_speed’ expirer. Si la vitesse d'exécution est inférieure, une exception est levée. +Vitesse d'exécution minimale en lignes par seconde. Vérifié sur chaque bloc de données quand ‘timeout_before_checking_execution_speed’ expirer. Si la vitesse d'exécution est inférieure, une exception est levée. -## min\_execution\_speed\_bytes {#min-execution-speed-bytes} +## min_execution_speed_bytes {#min-execution-speed-bytes} -Un nombre minimum d'exécution d'octets par seconde. Vérifié sur chaque bloc de données quand ‘timeout\_before\_checking\_execution\_speed’ expirer. Si la vitesse d'exécution est inférieure, une exception est levée. +Un nombre minimum d'exécution d'octets par seconde. Vérifié sur chaque bloc de données quand ‘timeout_before_checking_execution_speed’ expirer. Si la vitesse d'exécution est inférieure, une exception est levée. -## max\_execution\_speed {#max-execution-speed} +## max_execution_speed {#max-execution-speed} -Un nombre maximal d'exécution de lignes par seconde. Vérifié sur chaque bloc de données quand ‘timeout\_before\_checking\_execution\_speed’ expirer. Si la vitesse d'exécution est élevée, la vitesse d'exécution sera réduit. +Un nombre maximal d'exécution de lignes par seconde. Vérifié sur chaque bloc de données quand ‘timeout_before_checking_execution_speed’ expirer. Si la vitesse d'exécution est élevée, la vitesse d'exécution sera réduit. -## max\_execution\_speed\_bytes {#max-execution-speed-bytes} +## max_execution_speed_bytes {#max-execution-speed-bytes} -Un nombre maximal d'exécution d'octets par seconde. Vérifié sur chaque bloc de données quand ‘timeout\_before\_checking\_execution\_speed’ expirer. Si la vitesse d'exécution est élevée, la vitesse d'exécution sera réduit. +Un nombre maximal d'exécution d'octets par seconde. Vérifié sur chaque bloc de données quand ‘timeout_before_checking_execution_speed’ expirer. Si la vitesse d'exécution est élevée, la vitesse d'exécution sera réduit. -## timeout\_before\_checking\_execution\_speed {#timeout-before-checking-execution-speed} +## timeout_before_checking_execution_speed {#timeout-before-checking-execution-speed} -Vérifie que la vitesse d'exécution n'est pas trop lent (pas moins de ‘min\_execution\_speed’), après l'expiration du temps spécifié en secondes. +Vérifie que la vitesse d'exécution n'est pas trop lent (pas moins de ‘min_execution_speed’), après l'expiration du temps spécifié en secondes. -## max\_columns\_to\_read {#max-columns-to-read} +## max_columns_to_read {#max-columns-to-read} Nombre maximal de colonnes pouvant être lues à partir d'une table dans une seule requête. Si une requête nécessite la lecture d'un plus grand nombre de colonnes, il lève une exception. -## max\_temporary\_columns {#max-temporary-columns} +## max_temporary_columns {#max-temporary-columns} Nombre maximal de colonnes temporaires qui doivent être conservées en RAM en même temps lors de l'exécution d'une requête, y compris les colonnes constantes. S'il y a plus de colonnes temporaires que cela, il lève une exception. -## max\_temporary\_non\_const\_columns {#max-temporary-non-const-columns} +## max_temporary_non_const_columns {#max-temporary-non-const-columns} -La même chose que ‘max\_temporary\_columns’ mais sans compter constante colonnes. +La même chose que ‘max_temporary_columns’ mais sans compter constante colonnes. Notez que les colonnes constantes sont formées assez souvent lors de l'exécution d'une requête, mais elles nécessitent environ zéro ressource informatique. -## max\_subquery\_depth {#max-subquery-depth} +## max_subquery_depth {#max-subquery-depth} Profondeur maximale de sous-requêtes. Si les sous-requêtes sont plus profondes, une exception est levée. Par défaut, 100. -## max\_pipeline\_depth {#max-pipeline-depth} +## max_pipeline_depth {#max-pipeline-depth} Profondeur maximale du pipeline. Correspond au nombre de transformations que chaque bloc de données lors du traitement des requêtes. Compté dans les limites d'un seul serveur. Si la profondeur du pipeline est supérieure, une exception est levée. Par défaut, 1000. -## max\_ast\_depth {#max-ast-depth} +## max_ast_depth {#max-ast-depth} Profondeur maximale d'une requête arbre syntaxique. En cas de dépassement, une exception est levée. À ce moment, il n'est pas vérifié pendant l'analyse, mais seulement après l'analyse de la requête. Autrement dit, un arbre syntaxique trop profond peut être créé pendant l'analyse, mais la requête échouera. Par défaut, 1000. -## max\_ast\_elements {#max-ast-elements} +## max_ast_elements {#max-ast-elements} Un nombre maximal d'éléments dans une requête arbre syntaxique. En cas de dépassement, une exception est levée. De la même manière que le paramètre précédent, il est vérifié qu'après l'analyse de la requête. Par défaut, 50 000. -## max\_rows\_in\_set {#max-rows-in-set} +## max_rows_in_set {#max-rows-in-set} Nombre maximal de lignes pour un ensemble de données dans la clause in créée à partir d'une sous-requête. -## max\_bytes\_in\_set {#max-bytes-in-set} +## max_bytes_in_set {#max-bytes-in-set} Nombre maximal d'octets (données non compressées) utilisés par un ensemble de la clause in créé à partir d'une sous-requête. -## set\_overflow\_mode {#set-overflow-mode} +## set_overflow_mode {#set-overflow-mode} Que faire lorsque la quantité de données dépasse l'une des limites: ‘throw’ ou ‘break’. Par défaut, les jeter. -## max\_rows\_in\_distinct {#max-rows-in-distinct} +## max_rows_in_distinct {#max-rows-in-distinct} Un nombre maximum de lignes différentes lors de L'utilisation de DISTINCT. -## max\_bytes\_in\_distinct {#max-bytes-in-distinct} +## max_bytes_in_distinct {#max-bytes-in-distinct} Nombre maximal d'octets utilisés par une table de hachage lors de L'utilisation de DISTINCT. -## distinct\_overflow\_mode {#distinct-overflow-mode} +## distinct_overflow_mode {#distinct-overflow-mode} Que faire lorsque la quantité de données dépasse l'une des limites: ‘throw’ ou ‘break’. Par défaut, les jeter. -## max\_rows\_to\_transfer {#max-rows-to-transfer} +## max_rows_to_transfer {#max-rows-to-transfer} Nombre maximal de lignes pouvant être transmises à un serveur distant ou enregistrées dans une table temporaire lors de L'utilisation de GLOBAL IN. -## max\_bytes\_to\_transfer {#max-bytes-to-transfer} +## max_bytes_to_transfer {#max-bytes-to-transfer} Nombre maximal d'octets (données non compressées) pouvant être transmis à un serveur distant ou enregistrés dans une table temporaire lors de L'utilisation de GLOBAL IN. -## transfer\_overflow\_mode {#transfer-overflow-mode} +## transfer_overflow_mode {#transfer-overflow-mode} Que faire lorsque la quantité de données dépasse l'une des limites: ‘throw’ ou ‘break’. Par défaut, les jeter. -## max\_rows\_in\_join {#settings-max_rows_in_join} +## max_rows_in_join {#settings-max_rows_in_join} Limite le nombre de lignes dans la table de hachage utilisée lors de la jonction de tables. @@ -238,7 +238,7 @@ Ce réglage s'applique à [SELECT … JOIN](../../sql-reference/statements/selec Si une requête contient plusieurs jointures, ClickHouse vérifie ce paramètre pour chaque résultat intermédiaire. -ClickHouse peut procéder à différentes actions lorsque la limite est atteinte. L'utilisation de la [join\_overflow\_mode](#settings-join_overflow_mode) réglage pour choisir l'action. +ClickHouse peut procéder à différentes actions lorsque la limite est atteinte. L'utilisation de la [join_overflow_mode](#settings-join_overflow_mode) réglage pour choisir l'action. Valeurs possibles: @@ -247,7 +247,7 @@ Valeurs possibles: Valeur par défaut: 0. -## max\_bytes\_in\_join {#settings-max_bytes_in_join} +## max_bytes_in_join {#settings-max_bytes_in_join} Limite la taille en octets de la table de hachage utilisée lors de l'assemblage de tables. @@ -255,7 +255,7 @@ Ce réglage s'applique à [SELECT … JOIN](../../sql-reference/statements/selec Si la requête contient des jointures, ClickHouse vérifie ce paramètre pour chaque résultat intermédiaire. -ClickHouse peut procéder à différentes actions lorsque la limite est atteinte. Utiliser [join\_overflow\_mode](#settings-join_overflow_mode) paramètres pour choisir l'action. +ClickHouse peut procéder à différentes actions lorsque la limite est atteinte. Utiliser [join_overflow_mode](#settings-join_overflow_mode) paramètres pour choisir l'action. Valeurs possibles: @@ -264,12 +264,12 @@ Valeurs possibles: Valeur par défaut: 0. -## join\_overflow\_mode {#settings-join_overflow_mode} +## join_overflow_mode {#settings-join_overflow_mode} Définit l'action que ClickHouse effectue lorsque l'une des limites de jointure suivantes est atteinte: -- [max\_bytes\_in\_join](#settings-max_bytes_in_join) -- [max\_rows\_in\_join](#settings-max_rows_in_join) +- [max_bytes_in_join](#settings-max_bytes_in_join) +- [max_rows_in_join](#settings-max_rows_in_join) Valeurs possibles: @@ -283,7 +283,7 @@ Valeur par défaut: `THROW`. - [Clause de JOINTURE](../../sql-reference/statements/select/join.md#select-join) - [Rejoindre le moteur de table](../../engines/table-engines/special/join.md) -## max\_partitions\_per\_insert\_block {#max-partitions-per-insert-block} +## max_partitions_per_insert_block {#max-partitions-per-insert-block} Limite le nombre maximal de partitions dans un seul bloc inséré. @@ -296,6 +296,6 @@ Valeur par défaut: 100. Lors de l'insertion de données, ClickHouse calcule le nombre de partitions dans le bloc inséré. Si le nombre de partitions est plus que `max_partitions_per_insert_block`, ClickHouse lève une exception avec le texte suivant: -> “Too many partitions for single INSERT block (more than” + toString (max\_parts) + “). The limit is controlled by ‘max\_partitions\_per\_insert\_block’ setting. A large number of partitions is a common misconception. It will lead to severe negative performance impact, including slow server startup, slow INSERT queries and slow SELECT queries. Recommended total number of partitions for a table is under 1000..10000. Please note, that partitioning is not intended to speed up SELECT queries (ORDER BY key is sufficient to make range queries fast). Partitions are intended for data manipulation (DROP PARTITION, etc).” +> “Too many partitions for single INSERT block (more than” + toString (max_parts) + “). The limit is controlled by ‘max_partitions_per_insert_block’ setting. A large number of partitions is a common misconception. It will lead to severe negative performance impact, including slow server startup, slow INSERT queries and slow SELECT queries. Recommended total number of partitions for a table is under 1000..10000. Please note, that partitioning is not intended to speed up SELECT queries (ORDER BY key is sufficient to make range queries fast). Partitions are intended for data manipulation (DROP PARTITION, etc).” [Article Original](https://clickhouse.tech/docs/en/operations/settings/query_complexity/) diff --git a/docs/fr/operations/settings/settings-users.md b/docs/fr/operations/settings/settings-users.md index 2be41d90533..d33b8f1335b 100644 --- a/docs/fr/operations/settings/settings-users.md +++ b/docs/fr/operations/settings/settings-users.md @@ -43,7 +43,7 @@ La Structure de la `users` section: ``` -### nom\_utilisateur/mot de passe {#user-namepassword} +### nom_utilisateur/mot de passe {#user-namepassword} Le mot de passe peut être spécifié en texte clair ou en SHA256 (format hexadécimal). @@ -75,7 +75,7 @@ Le mot de passe peut être spécifié en texte clair ou en SHA256 (format hexad La première ligne du résultat est le mot de passe. La deuxième ligne est le double hachage SHA1 correspondant. -### access\_management {#access_management-user-setting} +### access_management {#access_management-user-setting} Ce paramètre active de désactive l'utilisation de SQL-driven [le contrôle d'accès et de gestion de compte](../access-rights.md#access-control) pour l'utilisateur. @@ -86,7 +86,7 @@ Valeurs possibles: Valeur par défaut: 0. -### nom\_utilisateur / réseaux {#user-namenetworks} +### nom_utilisateur / réseaux {#user-namenetworks} Liste des réseaux à partir desquels L'utilisateur peut se connecter au serveur ClickHouse. @@ -128,18 +128,18 @@ Pour ouvrir l'accès uniquement à partir de localhost, spécifier: 127.0.0.1 ``` -### nom\_utilisateur / profil {#user-nameprofile} +### nom_utilisateur / profil {#user-nameprofile} Vous pouvez attribuer un profil des paramètres pour l'utilisateur. Les profils de paramètres sont configurés dans une section distincte du `users.xml` fichier. Pour plus d'informations, voir [Profils des paramètres](settings-profiles.md). -### nom\_utilisateur / quota {#user-namequota} +### nom_utilisateur / quota {#user-namequota} Les Quotas vous permettent de suivre ou de limiter l'utilisation des ressources sur une période donnée. Les Quotas sont configurés dans le `quotas` la section de la `users.xml` fichier de configuration. Vous pouvez attribuer un jeu de quotas à l'utilisateur. Pour une description détaillée de la configuration des quotas, voir [Quota](../quotas.md#quotas). -### nom\_utilisateur/bases de données {#user-namedatabases} +### nom_utilisateur/bases de données {#user-namedatabases} Dans cette section, vous pouvez limiter les lignes renvoyées par ClickHouse pour `SELECT` requêtes faites par l'utilisateur actuel, implémentant ainsi la sécurité de base au niveau de la ligne. diff --git a/docs/fr/operations/settings/settings.md b/docs/fr/operations/settings/settings.md index 06748ad8c70..208e4ab1eae 100644 --- a/docs/fr/operations/settings/settings.md +++ b/docs/fr/operations/settings/settings.md @@ -5,7 +5,7 @@ machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd # Paramètre {#settings} -## distributed\_product\_mode {#distributed-product-mode} +## distributed_product_mode {#distributed-product-mode} Modifie le comportement de [distribués sous-requêtes](../../sql-reference/operators/in.md). @@ -25,7 +25,7 @@ Valeurs possibles: - `global` — Replaces the `IN`/`JOIN` requête avec `GLOBAL IN`/`GLOBAL JOIN.` - `allow` — Allows the use of these types of subqueries. -## enable\_optimize\_predicate\_expression {#enable-optimize-predicate-expression} +## enable_optimize_predicate_expression {#enable-optimize-predicate-expression} Active la poussée du prédicat `SELECT` requête. @@ -49,7 +49,7 @@ Si `enable_optimize_predicate_expression = 1`, alors le temps d'exécution de ce Si `enable_optimize_predicate_expression = 0` puis le temps d'exécution de la deuxième requête est beaucoup plus long, parce que le `WHERE` la clause s'applique à toutes les données après la sous-requête des finitions. -## fallback\_to\_stale\_replicas\_for\_distributed\_queries {#settings-fallback_to_stale_replicas_for_distributed_queries} +## fallback_to_stale_replicas_for_distributed_queries {#settings-fallback_to_stale_replicas_for_distributed_queries} Force une requête à un réplica obsolète si les données mises à jour ne sont pas disponibles. Voir [Réplication](../../engines/table-engines/mergetree-family/replication.md). @@ -59,7 +59,7 @@ Utilisé lors de l'exécution `SELECT` à partir d'une table distribuée qui poi Par défaut, 1 (activé). -## force\_index\_by\_date {#settings-force_index_by_date} +## force_index_by_date {#settings-force_index_by_date} Désactive l'exécution de la requête si l'index ne peut pas être utilisé par jour. @@ -67,7 +67,7 @@ Fonctionne avec les tables de la famille MergeTree. Si `force_index_by_date=1`, Clickhouse vérifie si la requête a une condition de clé de date qui peut être utilisée pour restreindre les plages de données. S'il n'y a pas de condition appropriée, il lève une exception. Cependant, il ne vérifie pas si la condition réduit la quantité de données à lire. Par exemple, la condition `Date != ' 2000-01-01 '` est acceptable même lorsqu'il correspond à toutes les données de la table (c'est-à-dire que l'exécution de la requête nécessite une analyse complète). Pour plus d'informations sur les plages de données dans les tables MergeTree, voir [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). -## force\_primary\_key {#force-primary-key} +## force_primary_key {#force-primary-key} Désactive l'exécution de la requête si l'indexation par la clé primaire n'est pas possible. @@ -75,17 +75,17 @@ Fonctionne avec les tables de la famille MergeTree. Si `force_primary_key=1`, Clickhouse vérifie si la requête a une condition de clé primaire qui peut être utilisée pour restreindre les plages de données. S'il n'y a pas de condition appropriée, il lève une exception. Cependant, il ne vérifie pas si la condition réduit la quantité de données à lire. Pour plus d'informations sur les plages de données dans les tables MergeTree, voir [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). -## format\_schema {#format-schema} +## format_schema {#format-schema} Ce paramètre est utile lorsque vous utilisez des formats nécessitant une définition de schéma, tels que [Cap'n Proto](https://capnproto.org/) ou [Protobuf](https://developers.google.com/protocol-buffers/). La valeur dépend du format. -## fsync\_metadata {#fsync-metadata} +## fsync_metadata {#fsync-metadata} Active ou désactive [fsync](http://pubs.opengroup.org/onlinepubs/9699919799/functions/fsync.html) lors de l'écriture `.sql` fichier. Activé par défaut. Il est logique de le désactiver si le serveur a des millions de tables minuscules qui sont constamment créées et détruites. -## enable\_http\_compression {#settings-enable_http_compression} +## enable_http_compression {#settings-enable_http_compression} Active ou désactive la compression de données dans la réponse à une requête HTTP. @@ -98,15 +98,15 @@ Valeurs possibles: Valeur par défaut: 0. -## http\_zlib\_compression\_level {#settings-http_zlib_compression_level} +## http_zlib_compression_level {#settings-http_zlib_compression_level} -Définit le niveau de compression des données dans la réponse à une requête HTTP si [enable\_http\_compression = 1](#settings-enable_http_compression). +Définit le niveau de compression des données dans la réponse à une requête HTTP si [enable_http_compression = 1](#settings-enable_http_compression). Valeurs possibles: nombres de 1 à 9. Valeur par défaut: 3. -## http\_native\_compression\_disable\_checksumming\_on\_decompress {#settings-http_native_compression_disable_checksumming_on_decompress} +## http_native_compression_disable_checksumming_on_decompress {#settings-http_native_compression_disable_checksumming_on_decompress} Active ou désactive la vérification de la somme de contrôle lors de la décompression des données HTTP POST du client. Utilisé uniquement pour le format de compression natif ClickHouse (non utilisé avec `gzip` ou `deflate`). @@ -119,7 +119,7 @@ Valeurs possibles: Valeur par défaut: 0. -## send\_progress\_in\_http\_headers {#settings-send_progress_in_http_headers} +## send_progress_in_http_headers {#settings-send_progress_in_http_headers} Active ou désactive `X-ClickHouse-Progress` - Têtes de réponse HTTP dans `clickhouse-server` réponse. @@ -132,7 +132,7 @@ Valeurs possibles: Valeur par défaut: 0. -## max\_http\_get\_redirects {#setting-max_http_get_redirects} +## max_http_get_redirects {#setting-max_http_get_redirects} Limite le nombre maximal de sauts de redirection HTTP GET pour [URL](../../engines/table-engines/special/url.md)-tables de moteur. Le paramètre s'applique aux deux types de tables: celles créées par [CREATE TABLE](../../sql-reference/statements/create.md#create-table-query) requête et par la [URL](../../sql-reference/table-functions/url.md) table de fonction. @@ -143,7 +143,7 @@ Valeurs possibles: Valeur par défaut: 0. -## input\_format\_allow\_errors\_num {#settings-input_format_allow_errors_num} +## input_format_allow_errors_num {#settings-input_format_allow_errors_num} Définit le nombre maximal d'erreurs acceptables lors de la lecture à partir de formats de texte (CSV, TSV, etc.). @@ -155,7 +155,7 @@ Si une erreur s'est produite lors de la lecture de lignes mais que le compteur d Si les deux `input_format_allow_errors_num` et `input_format_allow_errors_ratio` sont dépassés, ClickHouse lève une exception. -## input\_format\_allow\_errors\_ratio {#settings-input_format_allow_errors_ratio} +## input_format_allow_errors_ratio {#settings-input_format_allow_errors_ratio} Définit le pourcentage maximal d'erreurs autorisées lors de la lecture à partir de formats de texte (CSV, TSV, etc.). Le pourcentage d'erreurs est défini comme un nombre à virgule flottante compris entre 0 et 1. @@ -168,7 +168,7 @@ Si une erreur s'est produite lors de la lecture de lignes mais que le compteur d Si les deux `input_format_allow_errors_num` et `input_format_allow_errors_ratio` sont dépassés, ClickHouse lève une exception. -## input\_format\_values\_interpret\_expressions {#settings-input_format_values_interpret_expressions} +## input_format_values_interpret_expressions {#settings-input_format_values_interpret_expressions} Active ou désactive L'analyseur SQL complet si l'analyseur de flux rapide ne peut pas analyser les données. Ce paramètre est utilisé uniquement pour la [Valeur](../../interfaces/formats.md#data-format-values) format lors de l'insertion des données. Pour plus d'informations sur l'analyse syntaxique, consultez [Syntaxe](../../sql-reference/syntax.md) section. @@ -218,7 +218,7 @@ INSERT INTO datetime_t SELECT now() Ok. ``` -## input\_format\_values\_deduce\_templates\_of\_expressions {#settings-input_format_values_deduce_templates_of_expressions} +## input_format_values_deduce_templates_of_expressions {#settings-input_format_values_deduce_templates_of_expressions} Active ou désactive la déduction de modèle pour les expressions SQL dans [Valeur](../../interfaces/formats.md#data-format-values) format. Il permet d'analyser et d'interpréter des expressions dans `Values` beaucoup plus rapide si les expressions dans des lignes consécutives ont la même structure. ClickHouse tente de déduire le modèle d'une expression, d'analyser les lignes suivantes à l'aide de ce modèle et d'évaluer l'expression sur un lot de lignes analysées avec succès. @@ -239,7 +239,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( - Si `input_format_values_interpret_expressions=0` et `format_values_deduce_templates_of_expressions=1`, les expressions des première, deuxième et troisième lignes sont analysées à l'aide de template `lower(String)` et interprété ensemble, l'expression dans la quatrième ligne est analysée avec un autre modèle (`upper(String)`). - Si `input_format_values_interpret_expressions=1` et `format_values_deduce_templates_of_expressions=1`, le même que dans le cas précédent, mais permet également d'interpréter les expressions séparément s'il n'est pas possible de déduire le modèle. -## input\_format\_values\_accurate\_types\_of\_literals {#settings-input-format-values-accurate-types-of-literals} +## input_format_values_accurate_types_of_literals {#settings-input-format-values-accurate-types-of-literals} Ce paramètre est utilisé uniquement lorsque `input_format_values_deduce_templates_of_expressions = 1`. Il peut arriver que les expressions pour une colonne aient la même structure, mais contiennent des littéraux numériques de types différents, par exemple @@ -261,7 +261,7 @@ Valeurs possibles: Valeur par défaut: 1. -## input\_format\_defaults\_for\_omitted\_fields {#session_settings-input_format_defaults_for_omitted_fields} +## input_format_defaults_for_omitted_fields {#session_settings-input_format_defaults_for_omitted_fields} Lors de l'exécution de `INSERT` requêtes, remplacez les valeurs de colonne d'entrée omises par les valeurs par défaut des colonnes respectives. Cette option s'applique uniquement aux [JSONEachRow](../../interfaces/formats.md#jsoneachrow), [CSV](../../interfaces/formats.md#csv) et [TabSeparated](../../interfaces/formats.md#tabseparated) format. @@ -275,17 +275,17 @@ Valeurs possibles: Valeur par défaut: 1. -## input\_format\_tsv\_empty\_as\_default {#settings-input-format-tsv-empty-as-default} +## input_format_tsv_empty_as_default {#settings-input-format-tsv-empty-as-default} Lorsque cette option est activée, remplacez les champs de saisie vides dans TSV par des valeurs par défaut. Pour les expressions par défaut complexes `input_format_defaults_for_omitted_fields` doit être activé en trop. Désactivé par défaut. -## input\_format\_null\_as\_default {#settings-input-format-null-as-default} +## input_format_null_as_default {#settings-input-format-null-as-default} Active ou désactive l'utilisation des valeurs par défaut si les données `NULL` mais le type de données de la colonne correspondante dans pas `Nullable(T)` (pour les formats de saisie de texte). -## input\_format\_skip\_unknown\_fields {#settings-input-format-skip-unknown-fields} +## input_format_skip_unknown_fields {#settings-input-format-skip-unknown-fields} Active ou désactive le saut d'insertion de données supplémentaires. @@ -305,7 +305,7 @@ Valeurs possibles: Valeur par défaut: 0. -## input\_format\_import\_nested\_json {#settings-input_format_import_nested_json} +## input_format_import_nested_json {#settings-input_format_import_nested_json} Active ou désactive l'insertion de données JSON avec des objets imbriqués. @@ -324,7 +324,7 @@ Voir aussi: - [Utilisation de Structures imbriquées](../../interfaces/formats.md#jsoneachrow-nested) avec l' `JSONEachRow` format. -## input\_format\_with\_names\_use\_header {#settings-input-format-with-names-use-header} +## input_format_with_names_use_header {#settings-input-format-with-names-use-header} Active ou désactive la vérification de l'ordre des colonnes lors de l'insertion de données. @@ -342,7 +342,7 @@ Valeurs possibles: Valeur par défaut: 1. -## date\_time\_input\_format {#settings-date_time_input_format} +## date_time_input_format {#settings-date_time_input_format} Permet de choisir un analyseur de la représentation textuelle de la date et de l'heure. @@ -365,7 +365,7 @@ Voir aussi: - [Type de données DateTime.](../../sql-reference/data-types/datetime.md) - [Fonctions pour travailler avec des dates et des heures.](../../sql-reference/functions/date-time-functions.md) -## join\_default\_strictness {#settings-join_default_strictness} +## join_default_strictness {#settings-join_default_strictness} Définit la rigueur par défaut pour [JOIN clauses](../../sql-reference/statements/select/join.md#select-join). @@ -378,7 +378,7 @@ Valeurs possibles: Valeur par défaut: `ALL`. -## join\_any\_take\_last\_row {#settings-join_any_take_last_row} +## join_any_take_last_row {#settings-join_any_take_last_row} Modifie le comportement des opérations de jointure avec `ANY` rigueur. @@ -396,9 +396,9 @@ Voir aussi: - [Clause de JOINTURE](../../sql-reference/statements/select/join.md#select-join) - [Rejoindre le moteur de table](../../engines/table-engines/special/join.md) -- [join\_default\_strictness](#settings-join_default_strictness) +- [join_default_strictness](#settings-join_default_strictness) -## join\_use\_nulls {#join_use_nulls} +## join_use_nulls {#join_use_nulls} Définit le type de [JOIN](../../sql-reference/statements/select/join.md) comportement. Lors de la fusion de tables, des cellules vides peuvent apparaître. ClickHouse les remplit différemment en fonction de ce paramètre. @@ -409,7 +409,7 @@ Valeurs possibles: Valeur par défaut: 0. -## max\_block\_size {#setting-max_block_size} +## max_block_size {#setting-max_block_size} Dans ClickHouse, les données sont traitées par Blocs (Ensembles de parties de colonne). Les cycles de traitement internes pour un seul bloc sont assez efficaces, mais il y a des dépenses notables sur chaque bloc. Le `max_block_size` le paramètre est une recommandation pour la taille du bloc (dans un nombre de lignes) à charger à partir des tables. La taille du bloc ne doit pas être trop petite, de sorte que les dépenses sur chaque bloc sont toujours perceptibles, mais pas trop grande pour que la requête avec limite qui est terminée après le premier bloc soit traitée rapidement. L'objectif est d'éviter de consommer trop de mémoire lors de l'extraction d'un grand nombre de colonnes dans plusieurs threads et de préserver au moins certains localité de cache. @@ -417,13 +417,13 @@ Valeur par défaut: de 65 536. Les blocs de la taille de `max_block_size` ne sont pas toujours chargées de la table. Si il est évident que moins de données doivent être récupérées, un bloc plus petit est traitée. -## preferred\_block\_size\_bytes {#preferred-block-size-bytes} +## preferred_block_size_bytes {#preferred-block-size-bytes} Utilisé dans le même but que `max_block_size`, mais il définit la taille de bloc recommandée en octets en l'adaptant au nombre de lignes dans le bloc. Cependant, la taille du bloc ne peut pas être supérieure à `max_block_size` rangée. Par défaut: 1 000 000. Cela ne fonctionne que lors de la lecture des moteurs MergeTree. -## merge\_tree\_min\_rows\_for\_concurrent\_read {#setting-merge-tree-min-rows-for-concurrent-read} +## merge_tree_min_rows_for_concurrent_read {#setting-merge-tree-min-rows-for-concurrent-read} Si le nombre de lignes à lire à partir d'un fichier d'un [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table dépasse `merge_tree_min_rows_for_concurrent_read` ensuite, ClickHouse essaie d'effectuer une lecture simultanée de ce fichier sur plusieurs threads. @@ -433,7 +433,7 @@ Valeurs possibles: Valeur par défaut: 163840. -## merge\_tree\_min\_bytes\_for\_concurrent\_read {#setting-merge-tree-min-bytes-for-concurrent-read} +## merge_tree_min_bytes_for_concurrent_read {#setting-merge-tree-min-bytes-for-concurrent-read} Si le nombre d'octets à lire à partir d'un fichier d'un [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md)- table de moteur dépasse `merge_tree_min_bytes_for_concurrent_read` puis ClickHouse essaie de lire simultanément à partir de ce fichier dans plusieurs threads. @@ -443,7 +443,7 @@ Valeur Possible: Valeur par défaut: 251658240. -## merge\_tree\_min\_rows\_for\_seek {#setting-merge-tree-min-rows-for-seek} +## merge_tree_min_rows_for_seek {#setting-merge-tree-min-rows-for-seek} Si la distance entre deux blocs de données à lire dans un fichier est inférieure à `merge_tree_min_rows_for_seek` lignes, puis ClickHouse ne cherche pas à travers le fichier mais lit les données séquentiellement. @@ -453,7 +453,7 @@ Valeurs possibles: Valeur par défaut: 0. -## merge\_tree\_min\_bytes\_for\_seek {#setting-merge-tree-min-bytes-for-seek} +## merge_tree_min_bytes_for_seek {#setting-merge-tree-min-bytes-for-seek} Si la distance entre deux blocs de données à lire dans un fichier est inférieure à `merge_tree_min_bytes_for_seek` octets, puis ClickHouse lit séquentiellement une plage de fichier qui contient les deux blocs, évitant ainsi la recherche supplémentaire. @@ -463,7 +463,7 @@ Valeurs possibles: Valeur par défaut: 0. -## merge\_tree\_coarse\_index\_granularité {#setting-merge-tree-coarse-index-granularity} +## merge_tree_coarse_index_granularité {#setting-merge-tree-coarse-index-granularity} Lors de la recherche de données, ClickHouse vérifie les marques de données dans le fichier d'index. Si ClickHouse trouve que les clés requises sont dans une certaine plage, il divise cette plage en `merge_tree_coarse_index_granularity` subranges et recherche les clés requises récursivement. @@ -473,11 +473,11 @@ Valeurs possibles: Valeur par défaut: 8. -## merge\_tree\_max\_rows\_to\_use\_cache {#setting-merge-tree-max-rows-to-use-cache} +## merge_tree_max_rows_to_use_cache {#setting-merge-tree-max-rows-to-use-cache} Si ClickHouse devrait lire plus de `merge_tree_max_rows_to_use_cache` lignes dans une requête, il n'utilise pas le cache des blocs non compressés. -Le cache des blocs non compressés stocke les données extraites pour les requêtes. ClickHouse utilise ce cache pour accélérer les réponses aux petites requêtes répétées. Ce paramètre protège le cache contre le saccage par les requêtes qui lisent une grande quantité de données. Le [uncompressed\_cache\_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) le paramètre serveur définit la taille du cache des blocs non compressés. +Le cache des blocs non compressés stocke les données extraites pour les requêtes. ClickHouse utilise ce cache pour accélérer les réponses aux petites requêtes répétées. Ce paramètre protège le cache contre le saccage par les requêtes qui lisent une grande quantité de données. Le [uncompressed_cache_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) le paramètre serveur définit la taille du cache des blocs non compressés. Valeurs possibles: @@ -485,11 +485,11 @@ Valeurs possibles: Default value: 128 ✕ 8192. -## merge\_tree\_max\_bytes\_to\_use\_cache {#setting-merge-tree-max-bytes-to-use-cache} +## merge_tree_max_bytes_to_use_cache {#setting-merge-tree-max-bytes-to-use-cache} Si ClickHouse devrait lire plus de `merge_tree_max_bytes_to_use_cache` octets dans une requête, il n'utilise pas le cache de non compressé blocs. -Le cache des blocs non compressés stocke les données extraites pour les requêtes. ClickHouse utilise ce cache pour accélérer les réponses aux petites requêtes répétées. Ce paramètre protège le cache contre le saccage par les requêtes qui lisent une grande quantité de données. Le [uncompressed\_cache\_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) le paramètre serveur définit la taille du cache des blocs non compressés. +Le cache des blocs non compressés stocke les données extraites pour les requêtes. ClickHouse utilise ce cache pour accélérer les réponses aux petites requêtes répétées. Ce paramètre protège le cache contre le saccage par les requêtes qui lisent une grande quantité de données. Le [uncompressed_cache_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) le paramètre serveur définit la taille du cache des blocs non compressés. Valeur Possible: @@ -497,7 +497,7 @@ Valeur Possible: Valeur par défaut: 2013265920. -## min\_bytes\_to\_use\_direct\_io {#settings-min-bytes-to-use-direct-io} +## min_bytes_to_use_direct_io {#settings-min-bytes-to-use-direct-io} Volume de données minimum requis pour utiliser l'accès direct aux E/S sur le disque de stockage. @@ -510,11 +510,11 @@ Valeurs possibles: Valeur par défaut: 0. -## log\_queries {#settings-log-queries} +## log_queries {#settings-log-queries} Configuration de la journalisation des requêtes. -Les requêtes envoyées à ClickHouse avec cette configuration sont enregistrées selon les règles du [query\_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query-log) paramètre de configuration du serveur. +Les requêtes envoyées à ClickHouse avec cette configuration sont enregistrées selon les règles du [query_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query-log) paramètre de configuration du serveur. Exemple: @@ -522,7 +522,7 @@ Exemple: log_queries=1 ``` -## log\_queries\_min\_type {#settings-log-queries-min-type} +## log_queries_min_type {#settings-log-queries-min-type} `query_log` type minimal à enregistrer. @@ -540,11 +540,11 @@ Peut être utilisé pour limiter le nombre de entiries va va `query_log`, dites log_queries_min_type='EXCEPTION_WHILE_PROCESSING' ``` -## log\_query\_threads {#settings-log-query-threads} +## log_query_threads {#settings-log-query-threads} Configuration de la journalisation des threads de requête. -Les threads de requêtes exécutés par ClickHouse avec cette configuration sont journalisés selon les règles du [query\_thread\_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) paramètre de configuration du serveur. +Les threads de requêtes exécutés par ClickHouse avec cette configuration sont journalisés selon les règles du [query_thread_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) paramètre de configuration du serveur. Exemple: @@ -552,19 +552,19 @@ Exemple: log_query_threads=1 ``` -## max\_insert\_block\_size {#settings-max_insert_block_size} +## max_insert_block_size {#settings-max_insert_block_size} La taille des blocs à former pour l'insertion dans une table. Ce paramètre s'applique uniquement dans les cas où le serveur formes les blocs. Par exemple, pour une insertion via L'interface HTTP, le serveur analyse le format de données et forme des blocs de la taille spécifiée. -Mais lors de l'utilisation de clickhouse-client, le client analyse les données ‘max\_insert\_block\_size’ le réglage sur le serveur n'affecte pas la taille des blocs insérés. +Mais lors de l'utilisation de clickhouse-client, le client analyse les données ‘max_insert_block_size’ le réglage sur le serveur n'affecte pas la taille des blocs insérés. Le paramètre n'a pas non plus de but lors de L'utilisation D'INSERT SELECT, car les données sont insérées à l'aide des mêmes blocs qui sont formés après SELECT. Valeur par défaut: 1 048 576 octets. La valeur par défaut est légèrement supérieure à `max_block_size`. La raison en est que certains moteurs de table (`*MergeTree`) former une partie de données sur le disque pour chaque bloc inséré, qui est une entité assez grande. Pareillement, `*MergeTree` les tables trient les données lors de l'insertion et une taille de bloc suffisamment grande permet de trier plus de données dans la RAM. -## min\_insert\_block\_size\_rows {#min-insert-block-size-rows} +## min_insert_block_size_rows {#min-insert-block-size-rows} Définit le nombre minimum de lignes dans le bloc qui peut être inséré dans un tableau par un `INSERT` requête. Les blocs de plus petite taille sont écrasés en plus gros. @@ -575,7 +575,7 @@ Valeurs possibles: Valeur par défaut: 1048576. -## min\_insert\_block\_size\_bytes {#min-insert-block-size-bytes} +## min_insert_block_size_bytes {#min-insert-block-size-bytes} Définit le nombre minimal d'octets dans le bloc qui peut être inséré dans un tableau par un `INSERT` requête. Les blocs de plus petite taille sont écrasés en plus gros. @@ -586,7 +586,7 @@ Valeurs possibles: Valeur par défaut: 268435456. -## max\_replica\_delay\_for\_distributed\_queries {#settings-max_replica_delay_for_distributed_queries} +## max_replica_delay_for_distributed_queries {#settings-max_replica_delay_for_distributed_queries} Désactive les répliques en retard pour les requêtes distribuées. Voir [Réplication](../../engines/table-engines/mergetree-family/replication.md). @@ -596,22 +596,22 @@ Valeur par défaut: 300. Utilisé lors de l'exécution `SELECT` à partir d'une table distribuée qui pointe vers des tables répliquées. -## max\_threads {#settings-max_threads} +## max_threads {#settings-max_threads} -Nombre maximal de threads de traitement des requêtes, à l'exclusion des threads de récupération de données à partir de serveurs distants (voir ‘max\_distributed\_connections’ paramètre). +Nombre maximal de threads de traitement des requêtes, à l'exclusion des threads de récupération de données à partir de serveurs distants (voir ‘max_distributed_connections’ paramètre). Ce paramètre s'applique aux threads qui effectuent les mêmes étapes du pipeline de traitement des requêtes en parallèle. -Par exemple, lors de la lecture d'une table, s'il est possible d'évaluer des expressions avec des fonctions, filtrer avec WHERE et pré-agréger pour GROUP BY en parallèle en utilisant au moins ‘max\_threads’ nombre de threads, puis ‘max\_threads’ sont utilisés. +Par exemple, lors de la lecture d'une table, s'il est possible d'évaluer des expressions avec des fonctions, filtrer avec WHERE et pré-agréger pour GROUP BY en parallèle en utilisant au moins ‘max_threads’ nombre de threads, puis ‘max_threads’ sont utilisés. Valeur par défaut: nombre de cœurs de processeur physiques. Si moins d'une requête SELECT est normalement exécutée sur un serveur à la fois, définissez ce paramètre sur une valeur légèrement inférieure au nombre réel de cœurs de processeur. -Pour les requêtes qui sont terminées rapidement en raison d'une limite, vous pouvez définir une valeur inférieure ‘max\_threads’. Par exemple, si le nombre d'entrées se trouvent dans chaque bloc et max\_threads = 8, 8 blocs sont récupérées, même s'il aurait été suffisante pour lire un seul. +Pour les requêtes qui sont terminées rapidement en raison d'une limite, vous pouvez définir une valeur inférieure ‘max_threads’. Par exemple, si le nombre d'entrées se trouvent dans chaque bloc et max_threads = 8, 8 blocs sont récupérées, même s'il aurait été suffisante pour lire un seul. Le plus petit de la `max_threads` valeur, moins la mémoire est consommée. -## max\_insert\_threads {#settings-max-insert-threads} +## max_insert_threads {#settings-max-insert-threads} Nombre maximal de threads à exécuter `INSERT SELECT` requête. @@ -622,61 +622,61 @@ Valeurs possibles: Valeur par défaut: 0. -Parallèle `INSERT SELECT` n'a d'effet que si l' `SELECT` une partie est exécutée en parallèle, voir [max\_threads](#settings-max_threads) paramètre. +Parallèle `INSERT SELECT` n'a d'effet que si l' `SELECT` une partie est exécutée en parallèle, voir [max_threads](#settings-max_threads) paramètre. Des valeurs plus élevées conduiront à une utilisation de la mémoire plus élevée. -## max\_compress\_block\_size {#max-compress-block-size} +## max_compress_block_size {#max-compress-block-size} La taille maximale des blocs de données non compressées avant la compression pour l'écriture dans une table. Par défaut, 1 048 576 (1 MiB). Si la taille est réduite, le taux de compression est considérablement réduit, la vitesse de compression et de décompression augmente légèrement en raison de la localisation du cache, et la consommation de mémoire est réduite. Il n'y aucune raison de modifier ce paramètre. Ne confondez pas les blocs pour la compression (un morceau de mémoire constitué d'octets) avec des blocs pour le traitement des requêtes (Un ensemble de lignes d'une table). -## min\_compress\_block\_size {#min-compress-block-size} +## min_compress_block_size {#min-compress-block-size} -Pour [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md)" table. Afin de réduire la latence lors du traitement des requêtes, un bloc est compressé lors de l'écriture de la marque suivante si sa taille est au moins ‘min\_compress\_block\_size’. Par défaut, 65 536. +Pour [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md)" table. Afin de réduire la latence lors du traitement des requêtes, un bloc est compressé lors de l'écriture de la marque suivante si sa taille est au moins ‘min_compress_block_size’. Par défaut, 65 536. -La taille réelle du bloc, si les données non compressées sont inférieures à ‘max\_compress\_block\_size’ pas moins de cette valeur et pas moins que le volume de données pour une marque. +La taille réelle du bloc, si les données non compressées sont inférieures à ‘max_compress_block_size’ pas moins de cette valeur et pas moins que le volume de données pour une marque. -Regardons un exemple. Supposons que ‘index\_granularity’ a 8192 lors de la création de la table. +Regardons un exemple. Supposons que ‘index_granularity’ a 8192 lors de la création de la table. -Nous écrivons une colonne de type UInt32 (4 octets par valeur). Lors de l'écriture de 8192 lignes, le total sera de 32 KO de données. Puisque min\_compress\_block\_size = 65 536, un bloc compressé sera formé pour toutes les deux marques. +Nous écrivons une colonne de type UInt32 (4 octets par valeur). Lors de l'écriture de 8192 lignes, le total sera de 32 KO de données. Puisque min_compress_block_size = 65 536, un bloc compressé sera formé pour toutes les deux marques. Nous écrivons une colonne URL avec le type de chaîne (taille moyenne de 60 octets par valeur). Lors de l'écriture de 8192 lignes, la moyenne sera légèrement inférieure à 500 Ko de données. Comme il s'agit de plus de 65 536, un bloc compressé sera formé pour chaque marque. Dans ce cas, lors de la lecture de données du disque dans la plage d'une seule marque, les données supplémentaires ne seront pas décompressées. Il n'y aucune raison de modifier ce paramètre. -## max\_query\_size {#settings-max_query_size} +## max_query_size {#settings-max_query_size} La partie maximale d'une requête qui peut être prise en RAM pour l'analyse avec L'analyseur SQL. La requête INSERT contient également des données pour INSERT qui sont traitées par un analyseur de flux séparé (qui consomme O (1) RAM), qui n'est pas inclus dans cette restriction. Valeur par défaut: 256 Ko. -## interactive\_delay {#interactive-delay} +## interactive_delay {#interactive-delay} Intervalle en microsecondes pour vérifier si l'exécution de la requête a été annulée et envoyer la progression. Valeur par défaut: 100 000 (vérifie l'Annulation et envoie la progression dix fois par seconde). -## connect\_timeout, receive\_timeout, send\_timeout {#connect-timeout-receive-timeout-send-timeout} +## connect_timeout, receive_timeout, send_timeout {#connect-timeout-receive-timeout-send-timeout} Délais d'attente en secondes sur le socket utilisé pour communiquer avec le client. Valeur par défaut: 10, 300, 300. -## cancel\_http\_readonly\_queries\_on\_client\_close {#cancel-http-readonly-queries-on-client-close} +## cancel_http_readonly_queries_on_client_close {#cancel-http-readonly-queries-on-client-close} Cancels HTTP read-only queries (e.g. SELECT) when a client closes the connection without waiting for the response. Valeur par défaut: 0 -## poll\_interval {#poll-interval} +## poll_interval {#poll-interval} Verrouillez une boucle d'attente pendant le nombre de secondes spécifié. Valeur par défaut: 10. -## max\_distributed\_connections {#max-distributed-connections} +## max_distributed_connections {#max-distributed-connections} Nombre maximal de connexions simultanées avec des serveurs distants pour le traitement distribué d'une seule requête vers une seule table distribuée. Nous vous recommandons de définir une valeur au moins égale au nombre de serveurs dans le cluster. @@ -684,20 +684,20 @@ Valeur par défaut: 1024. Les paramètres suivants ne sont utilisés que lors de la création de tables distribuées (et lors du lancement d'un serveur), il n'y a donc aucune raison de les modifier lors de l'exécution. -## distributed\_connections\_pool\_size {#distributed-connections-pool-size} +## distributed_connections_pool_size {#distributed-connections-pool-size} Nombre maximal de connexions simultanées avec des serveurs distants pour le traitement distribué de toutes les requêtes vers une seule table distribuée. Nous vous recommandons de définir une valeur au moins égale au nombre de serveurs dans le cluster. Valeur par défaut: 1024. -## connect\_timeout\_with\_failover\_ms {#connect-timeout-with-failover-ms} +## connect_timeout_with_failover_ms {#connect-timeout-with-failover-ms} Délai d'attente en millisecondes pour la connexion à un serveur distant pour un moteur de table distribué, si ‘shard’ et ‘replica’ les sections sont utilisées dans la définition du cluster. En cas d'échec, plusieurs tentatives sont faites pour se connecter à diverses répliques. Valeur par défaut: 50. -## connections\_with\_failover\_max\_tries {#connections-with-failover-max-tries} +## connections_with_failover_max_tries {#connections-with-failover-max-tries} Nombre maximal de tentatives de connexion avec chaque réplique pour le moteur de table distribué. @@ -708,27 +708,27 @@ Valeur par défaut: 3. Indique s'il faut compter les valeurs extrêmes (les minimums et les maximums dans les colonnes d'un résultat de requête). Accepte 0 ou 1. Par défaut, 0 (désactivé). Pour plus d'informations, consultez la section “Extreme values”. -## use\_uncompressed\_cache {#setting-use_uncompressed_cache} +## use_uncompressed_cache {#setting-use_uncompressed_cache} Indique s'il faut utiliser un cache de blocs non compressés. Accepte 0 ou 1. Par défaut, 0 (désactivé). -L'utilisation du cache non compressé (uniquement pour les tables de la famille MergeTree) peut réduire considérablement la latence et augmenter le débit lorsque vous travaillez avec un grand nombre de requêtes courtes. Activez ce paramètre pour les utilisateurs qui envoient des requêtes courtes fréquentes. Faites également attention à la [uncompressed\_cache\_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuration parameter (only set in the config file) – the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed and the least-used data is automatically deleted. +L'utilisation du cache non compressé (uniquement pour les tables de la famille MergeTree) peut réduire considérablement la latence et augmenter le débit lorsque vous travaillez avec un grand nombre de requêtes courtes. Activez ce paramètre pour les utilisateurs qui envoient des requêtes courtes fréquentes. Faites également attention à la [uncompressed_cache_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuration parameter (only set in the config file) – the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed and the least-used data is automatically deleted. -Pour les requêtes qui lisent au moins un volume de données assez important (un million de lignes ou plus), le cache non compressé est désactivé automatiquement pour économiser de l'espace pour les requêtes vraiment petites. Cela signifie que vous pouvez garder la ‘use\_uncompressed\_cache’ toujours la valeur 1. +Pour les requêtes qui lisent au moins un volume de données assez important (un million de lignes ou plus), le cache non compressé est désactivé automatiquement pour économiser de l'espace pour les requêtes vraiment petites. Cela signifie que vous pouvez garder la ‘use_uncompressed_cache’ toujours la valeur 1. -## replace\_running\_query {#replace-running-query} +## replace_running_query {#replace-running-query} -Lors de l'utilisation de L'interface HTTP, le ‘query\_id’ le paramètre peut être passé. C'est n'importe quelle chaîne qui sert d'Identificateur de requête. -Si une requête d'un utilisateur avec le même ‘query\_id’ il existe déjà à ce moment, le comportement dépend de la ‘replace\_running\_query’ paramètre. +Lors de l'utilisation de L'interface HTTP, le ‘query_id’ le paramètre peut être passé. C'est n'importe quelle chaîne qui sert d'Identificateur de requête. +Si une requête d'un utilisateur avec le même ‘query_id’ il existe déjà à ce moment, le comportement dépend de la ‘replace_running_query’ paramètre. -`0` (default) – Throw an exception (don't allow the query to run if a query with the same ‘query\_id’ est déjà en cours d'exécution). +`0` (default) – Throw an exception (don't allow the query to run if a query with the same ‘query_id’ est déjà en cours d'exécution). `1` – Cancel the old query and start running the new one. Yandex.Metrica utilise ce paramètre défini sur 1 pour implémenter des suggestions de conditions de segmentation. Après avoir entré le caractère suivant, si l'ancienne requête n'est pas encore terminée, elle doit être annulée. -## stream\_flush\_interval\_ms {#stream-flush-interval-ms} +## stream_flush_interval_ms {#stream-flush-interval-ms} -Fonctionne pour les tables avec des flux dans le cas d'une expiration, ou lorsqu'un thread génère [max\_insert\_block\_size](#settings-max_insert_block_size) rangée. +Fonctionne pour les tables avec des flux dans le cas d'une expiration, ou lorsqu'un thread génère [max_insert_block_size](#settings-max_insert_block_size) rangée. La valeur par défaut est 7500. @@ -787,7 +787,7 @@ Cet algorithme choisit la première réplique de l'ensemble ou une réplique al Le `first_or_random` algorithme résout le problème de la `in_order` algorithme. Avec `in_order`, si une réplique tombe en panne, la suivante obtient une double charge tandis que les répliques restantes gèrent la quantité habituelle de trafic. Lors de l'utilisation de la `first_or_random` algorithme, la charge est répartie uniformément entre les répliques qui sont encore disponibles. -## prefer\_localhost\_replica {#settings-prefer-localhost-replica} +## prefer_localhost_replica {#settings-prefer-localhost-replica} Active / désactive préférable d'utiliser le réplica localhost lors du traitement des requêtes distribuées. @@ -799,19 +799,19 @@ Valeurs possibles: Valeur par défaut: 1. !!! warning "Avertissement" - Désactivez ce paramètre si vous utilisez [max\_parallel\_replicas](#settings-max_parallel_replicas). + Désactivez ce paramètre si vous utilisez [max_parallel_replicas](#settings-max_parallel_replicas). -## totals\_mode {#totals-mode} +## totals_mode {#totals-mode} -Comment calculer les totaux lorsque HAVING est présent, ainsi que lorsque max\_rows\_to\_group\_by et group\_by\_overflow\_mode = ‘any’ sont présents. +Comment calculer les totaux lorsque HAVING est présent, ainsi que lorsque max_rows_to_group_by et group_by_overflow_mode = ‘any’ sont présents. Voir la section “WITH TOTALS modifier”. -## totals\_auto\_threshold {#totals-auto-threshold} +## totals_auto_threshold {#totals-auto-threshold} Le seuil de `totals_mode = 'auto'`. Voir la section “WITH TOTALS modifier”. -## max\_parallel\_replicas {#settings-max_parallel_replicas} +## max_parallel_replicas {#settings-max_parallel_replicas} Nombre maximal de répliques pour chaque fragment lors de l'exécution d'une requête. Par souci de cohérence (pour obtenir différentes parties du même partage de données), Cette option ne fonctionne que lorsque la clé d'échantillonnage est définie. @@ -824,7 +824,7 @@ Activer la compilation des requêtes. Par défaut, 0 (désactivé). La compilation n'est utilisée que pour une partie du pipeline de traitement des requêtes: pour la première étape de l'agrégation (GROUP BY). Si cette partie du pipeline a été compilée, la requête peut s'exécuter plus rapidement en raison du déploiement de cycles courts et des appels de fonction d'agrégation intégrés. L'amélioration maximale des performances (jusqu'à quatre fois plus rapide dans de rares cas) est observée pour les requêtes avec plusieurs fonctions d'agrégat simples. Typiquement, le gain de performance est insignifiant. Dans de très rares cas, il peut ralentir l'exécution de la requête. -## min\_count\_to\_compile {#min-count-to-compile} +## min_count_to_compile {#min-count-to-compile} Combien de fois utiliser potentiellement un morceau de code compilé avant d'exécuter la compilation. Par défaut, 3. For testing, the value can be set to 0: compilation runs synchronously and the query waits for the end of the compilation process before continuing execution. For all other cases, use values ​​starting with 1. Compilation normally takes about 5-10 seconds. @@ -833,27 +833,27 @@ Si la valeur est 1 ou plus, la compilation se produit de manière asynchrone dan Le code compilé est requis pour chaque combinaison différente de fonctions d'agrégat utilisées dans la requête et le type de clés dans la clause GROUP BY. The results of the compilation are saved in the build directory in the form of .so files. There is no restriction on the number of compilation results since they don't use very much space. Old results will be used after server restarts, except in the case of a server upgrade – in this case, the old results are deleted. -## output\_format\_json\_quote\_64bit\_integers {#session_settings-output_format_json_quote_64bit_integers} +## output_format_json_quote_64bit_integers {#session_settings-output_format_json_quote_64bit_integers} Si la valeur est true, les entiers apparaissent entre guillemets lors de l'utilisation des formats JSON\* Int64 et UInt64 (pour la compatibilité avec la plupart des implémentations JavaScript); sinon, les entiers sont sortis sans les guillemets. -## format\_csv\_delimiter {#settings-format_csv_delimiter} +## format_csv_delimiter {#settings-format_csv_delimiter} Caractère interprété comme un délimiteur dans les données CSV. Par défaut, le délimiteur est `,`. -## input\_format\_csv\_unquoted\_null\_literal\_as\_null {#settings-input_format_csv_unquoted_null_literal_as_null} +## input_format_csv_unquoted_null_literal_as_null {#settings-input_format_csv_unquoted_null_literal_as_null} Pour le format D'entrée CSV active ou désactive l'analyse des `NULL` comme littéral (synonyme de `\N`). -## output\_format\_csv\_crlf\_end\_of\_line {#settings-output-format-csv-crlf-end-of-line} +## output_format_csv_crlf_end_of_line {#settings-output-format-csv-crlf-end-of-line} Utilisez le séparateur de ligne de style DOS/Windows (CRLF) en CSV au lieu du style Unix (LF). -## output\_format\_tsv\_crlf\_end\_of\_line {#settings-output-format-tsv-crlf-end-of-line} +## output_format_tsv_crlf_end_of_line {#settings-output-format-tsv-crlf-end-of-line} Utilisez le séparateur de ligne de style DOC/Windows (CRLF) dans TSV au lieu du style Unix (LF). -## insert\_quorum {#settings-insert_quorum} +## insert_quorum {#settings-insert_quorum} Active les Écritures de quorum. @@ -868,7 +868,7 @@ Quorum écrit Toutes les répliques du quorum sont cohérentes, c'est-à-dire qu'elles contiennent des données de toutes les `INSERT` requête. Le `INSERT` la séquence est linéarisé. -Lors de la lecture des données écrites à partir du `insert_quorum`, vous pouvez utiliser le [select\_sequential\_consistency](#settings-select_sequential_consistency) option. +Lors de la lecture des données écrites à partir du `insert_quorum`, vous pouvez utiliser le [select_sequential_consistency](#settings-select_sequential_consistency) option. Clickhouse génère une exception @@ -877,10 +877,10 @@ Clickhouse génère une exception Voir aussi: -- [insert\_quorum\_timeout](#settings-insert_quorum_timeout) -- [select\_sequential\_consistency](#settings-select_sequential_consistency) +- [insert_quorum_timeout](#settings-insert_quorum_timeout) +- [select_sequential_consistency](#settings-select_sequential_consistency) -## insert\_quorum\_timeout {#settings-insert_quorum_timeout} +## insert_quorum_timeout {#settings-insert_quorum_timeout} Ecrire dans quorum timeout en secondes. Si le délai d'attente est passé et qu'aucune écriture n'a encore eu lieu, ClickHouse génère une exception et le client doit répéter la requête pour écrire le même bloc dans le même réplica ou tout autre réplica. @@ -888,10 +888,10 @@ Valeur par défaut: 60 secondes. Voir aussi: -- [insert\_quorum](#settings-insert_quorum) -- [select\_sequential\_consistency](#settings-select_sequential_consistency) +- [insert_quorum](#settings-insert_quorum) +- [select_sequential_consistency](#settings-select_sequential_consistency) -## select\_sequential\_consistency {#settings-select_sequential_consistency} +## select_sequential_consistency {#settings-select_sequential_consistency} Active ou désactive la cohérence séquentielle pour `SELECT` requête: @@ -908,10 +908,10 @@ Lorsque la cohérence séquentielle est activée, ClickHouse permet au client d' Voir aussi: -- [insert\_quorum](#settings-insert_quorum) -- [insert\_quorum\_timeout](#settings-insert_quorum_timeout) +- [insert_quorum](#settings-insert_quorum) +- [insert_quorum_timeout](#settings-insert_quorum_timeout) -## insert\_deduplicate {#settings-insert-deduplicate} +## insert_deduplicate {#settings-insert-deduplicate} Active ou désactive la déduplication des blocs `INSERT` (Répliqués\* les tableaux). @@ -924,7 +924,7 @@ Valeur par défaut: 1. Par défaut, les blocs insérés dans les tables répliquées `INSERT` déclaration sont dédupliquées (voir [Réplication Des Données](../../engines/table-engines/mergetree-family/replication.md)). -## déduplicate\_blocks\_in\_dependent\_materialized\_views {#settings-deduplicate-blocks-in-dependent-materialized-views} +## déduplicate_blocks_in_dependent_materialized_views {#settings-deduplicate-blocks-in-dependent-materialized-views} Active ou désactive la vérification de déduplication des vues matérialisées qui reçoivent des données à partir de tables\* répliquées. @@ -942,7 +942,7 @@ Si un bloc inséré est ignoré en raison de la déduplication dans la table sou Dans le même temps, ce comportement “breaks” `INSERT` idempotence. Si un `INSERT` dans la table principale a été un succès et `INSERT` into a materialized view failed (e.g. because of communication failure with Zookeeper) a client will get an error and can retry the operation. However, the materialized view won't receive the second insert because it will be discarded by deduplication in the main (source) table. The setting `deduplicate_blocks_in_dependent_materialized_views` permet de changer ce comportement. Lors d'une nouvelle tentative, une vue matérialisée recevra l'insertion répétée et effectuera une vérification de déduplication par elle-même, ignorant le résultat de la vérification pour la table source, et insérera des lignes perdues en raison de la première défaillance. -## max\_network\_bytes {#settings-max-network-bytes} +## max_network_bytes {#settings-max-network-bytes} Limite le volume de données (en octets) qui est reçu ou transmis sur le réseau lors de l'exécution d'une requête. Ce paramètre s'applique à chaque individu requête. @@ -953,7 +953,7 @@ Valeurs possibles: Valeur par défaut: 0. -## max\_network\_bandwidth {#settings-max-network-bandwidth} +## max_network_bandwidth {#settings-max-network-bandwidth} Limite la vitesse de l'échange de données sur le réseau en octets par seconde. Ce paramètre s'applique à toutes les requêtes. @@ -964,7 +964,7 @@ Valeurs possibles: Valeur par défaut: 0. -## max\_network\_bandwidth\_for\_user {#settings-max-network-bandwidth-for-user} +## max_network_bandwidth_for_user {#settings-max-network-bandwidth-for-user} Limite la vitesse de l'échange de données sur le réseau en octets par seconde. Ce paramètre s'applique à toutes les requêtes exécutées simultanément par un seul utilisateur. @@ -975,7 +975,7 @@ Valeurs possibles: Valeur par défaut: 0. -## max\_network\_bandwidth\_for\_all\_users {#settings-max-network-bandwidth-for-all-users} +## max_network_bandwidth_for_all_users {#settings-max-network-bandwidth-for-all-users} Limite la vitesse à laquelle les données sont échangées sur le réseau en octets par seconde. Ce paramètre s'applique à toutes les requêtes exécutées simultanément sur le serveur. @@ -986,7 +986,7 @@ Valeurs possibles: Valeur par défaut: 0. -## count\_distinct\_implementation {#settings-count_distinct_implementation} +## count_distinct_implementation {#settings-count_distinct_implementation} Spécifie de l' `uniq*` les fonctions doivent être utilisées pour [COUNT(DISTINCT …)](../../sql-reference/aggregate-functions/reference.md#agg_function-count) construction. @@ -1000,7 +1000,7 @@ Valeurs possibles: Valeur par défaut: `uniqExact`. -## skip\_unavailable\_shards {#settings-skip_unavailable_shards} +## skip_unavailable_shards {#settings-skip_unavailable_shards} Active ou désactive le saut silencieux des fragments indisponibles. @@ -1030,13 +1030,13 @@ Valeurs possibles: Valeur par défaut: 0. -## optimize\_skip\_unused\_shards {#settings-optimize_skip_unused_shards} +## optimize_skip_unused_shards {#settings-optimize_skip_unused_shards} Active ou désactive le saut des fragments inutilisés pour les requêtes SELECT qui ont la condition de clé de sharding dans PREWHERE/WHERE (suppose que les données sont distribuées par la clé de sharding, sinon ne rien faire). Valeur par défaut: 0 -## force\_optimize\_skip\_unused\_shards {#settings-force_optimize_skip_unused_shards} +## force_optimize_skip_unused_shards {#settings-force_optimize_skip_unused_shards} Active ou désactive l'exécution de la requête si [`optimize_skip_unused_shards`](#settings-optimize_skip_unused_shards) activé et sauter des fragments inutilisés n'est pas possible. Si le saut n'est pas possible et le paramètre est activé, une exception sera levée. @@ -1048,7 +1048,7 @@ Valeurs possibles: Valeur par défaut: 0 -## optimize\_throw\_if\_noop {#setting-optimize_throw_if_noop} +## optimize_throw_if_noop {#setting-optimize_throw_if_noop} Active ou désactive le lancement d'une exception si [OPTIMIZE](../../sql-reference/statements/misc.md#misc_operations-optimize) la requête n'a pas effectué de fusion. @@ -1061,19 +1061,19 @@ Valeurs possibles: Valeur par défaut: 0. -## distributed\_replica\_error\_half\_life {#settings-distributed_replica_error_half_life} +## distributed_replica_error_half_life {#settings-distributed_replica_error_half_life} - Type: secondes - Valeur par défaut: 60 secondes -Contrôle la vitesse à laquelle les erreurs dans les tables distribuées sont mises à zéro. Si une réplique est indisponible pendant un certain temps, accumule 5 erreurs et distributed\_replica\_error\_half\_life est défini sur 1 seconde, la réplique est considérée comme normale 3 secondes après la dernière erreur. +Contrôle la vitesse à laquelle les erreurs dans les tables distribuées sont mises à zéro. Si une réplique est indisponible pendant un certain temps, accumule 5 erreurs et distributed_replica_error_half_life est défini sur 1 seconde, la réplique est considérée comme normale 3 secondes après la dernière erreur. Voir aussi: - [Tableau moteur Distribués](../../engines/table-engines/special/distributed.md) -- [distributed\_replica\_error\_cap](#settings-distributed_replica_error_cap) +- [distributed_replica_error_cap](#settings-distributed_replica_error_cap) -## distributed\_replica\_error\_cap {#settings-distributed_replica_error_cap} +## distributed_replica_error_cap {#settings-distributed_replica_error_cap} - Type: unsigned int - Valeur par défaut: 1000 @@ -1083,9 +1083,9 @@ Le nombre d'erreurs de chaque réplique est plafonné à cette valeur, empêchan Voir aussi: - [Tableau moteur Distribués](../../engines/table-engines/special/distributed.md) -- [distributed\_replica\_error\_half\_life](#settings-distributed_replica_error_half_life) +- [distributed_replica_error_half_life](#settings-distributed_replica_error_half_life) -## distributed\_directory\_monitor\_sleep\_time\_ms {#distributed_directory_monitor_sleep_time_ms} +## distributed_directory_monitor_sleep_time_ms {#distributed_directory_monitor_sleep_time_ms} Intervalle de Base pour le [Distribué](../../engines/table-engines/special/distributed.md) tableau moteur à envoyer des données. L'intervalle réel augmente de façon exponentielle en cas d'erreurs. @@ -1095,9 +1095,9 @@ Valeurs possibles: Valeur par défaut: 100 millisecondes. -## distributed\_directory\_monitor\_max\_sleep\_time\_ms {#distributed_directory_monitor_max_sleep_time_ms} +## distributed_directory_monitor_max_sleep_time_ms {#distributed_directory_monitor_max_sleep_time_ms} -Intervalle maximal pour le [Distribué](../../engines/table-engines/special/distributed.md) tableau moteur à envoyer des données. Limite la croissance exponentielle de l'intervalle défini dans [distributed\_directory\_monitor\_sleep\_time\_ms](#distributed_directory_monitor_sleep_time_ms) paramètre. +Intervalle maximal pour le [Distribué](../../engines/table-engines/special/distributed.md) tableau moteur à envoyer des données. Limite la croissance exponentielle de l'intervalle défini dans [distributed_directory_monitor_sleep_time_ms](#distributed_directory_monitor_sleep_time_ms) paramètre. Valeurs possibles: @@ -1105,7 +1105,7 @@ Valeurs possibles: Valeur par défaut: 30000 millisecondes (30 secondes). -## distributed\_directory\_monitor\_batch\_inserts {#distributed_directory_monitor_batch_inserts} +## distributed_directory_monitor_batch_inserts {#distributed_directory_monitor_batch_inserts} Active / désactive l'envoi des données insérées par lots. @@ -1118,7 +1118,7 @@ Valeurs possibles: Valeur par défaut: 0. -## os\_thread\_priority {#setting-os-thread-priority} +## os_thread_priority {#setting-os-thread-priority} Définit la priorité ([beau](https://en.wikipedia.org/wiki/Nice_(Unix))) pour les threads qui exécutent des requêtes. Le planificateur du système d'exploitation considère cette priorité lors du choix du prochain thread à exécuter sur chaque noyau CPU disponible. @@ -1133,7 +1133,7 @@ Des valeurs plus faibles signifient une priorité plus élevée. Les discussions Valeur par défaut: 0. -## query\_profiler\_real\_time\_period\_ns {#query_profiler_real_time_period_ns} +## query_profiler_real_time_period_ns {#query_profiler_real_time_period_ns} Définit la période pour une horloge réelle de la [requête profiler](../../operations/optimizing-performance/sampling-query-profiler.md). La vraie minuterie d'horloge compte le temps d'horloge murale. @@ -1154,9 +1154,9 @@ Valeur par défaut: 1000000000 nanosecondes (une fois par seconde). Voir aussi: -- Système de table [trace\_log](../../operations/system-tables.md#system_tables-trace_log) +- Système de table [trace_log](../../operations/system-tables.md#system_tables-trace_log) -## query\_profiler\_cpu\_time\_period\_ns {#query_profiler_cpu_time_period_ns} +## query_profiler_cpu_time_period_ns {#query_profiler_cpu_time_period_ns} Définit la période pour une minuterie D'horloge CPU du [requête profiler](../../operations/optimizing-performance/sampling-query-profiler.md). Cette minuterie ne compte que le temps CPU. @@ -1177,9 +1177,9 @@ Valeur par défaut: 1000000000 nanosecondes. Voir aussi: -- Système de table [trace\_log](../../operations/system-tables.md#system_tables-trace_log) +- Système de table [trace_log](../../operations/system-tables.md#system_tables-trace_log) -## allow\_introspection\_functions {#settings-allow_introspection_functions} +## allow_introspection_functions {#settings-allow_introspection_functions} Active des désactive [obscures fonctions](../../sql-reference/functions/introspection.md) pour le profilage de requête. @@ -1193,23 +1193,23 @@ Valeur par défaut: 0. **Voir Aussi** - [Échantillonnage Du Profileur De Requête](../optimizing-performance/sampling-query-profiler.md) -- Système de table [trace\_log](../../operations/system-tables.md#system_tables-trace_log) +- Système de table [trace_log](../../operations/system-tables.md#system_tables-trace_log) -## input\_format\_parallel\_parsing {#input-format-parallel-parsing} +## input_format_parallel_parsing {#input-format-parallel-parsing} - Type: bool - Valeur par défaut: True Activer l'analyse parallèle des formats de données en préservant l'ordre. Pris en charge uniquement pour les formats TSV, TKSV, CSV et jsoneachrow. -## min\_chunk\_bytes\_for\_parallel\_parsing {#min-chunk-bytes-for-parallel-parsing} +## min_chunk_bytes_for_parallel_parsing {#min-chunk-bytes-for-parallel-parsing} - Type: unsigned int - Valeur par défaut: 1 MiB La taille minimale du bloc en octets, que chaque thread analysera en parallèle. -## output\_format\_avro\_codec {#settings-output_format_avro_codec} +## output_format_avro_codec {#settings-output_format_avro_codec} Définit le codec de compression utilisé pour le fichier Avro de sortie. @@ -1223,7 +1223,7 @@ Valeurs possibles: Valeur par défaut: `snappy` (si disponible) ou `deflate`. -## output\_format\_avro\_sync\_interval {#settings-output_format_avro_sync_interval} +## output_format_avro_sync_interval {#settings-output_format_avro_sync_interval} Définit la taille minimale des données (en octets) entre les marqueurs de synchronisation pour le fichier Avro de sortie. @@ -1233,7 +1233,7 @@ Valeurs possibles: 32 (32 octets) - 1073741824 (1 GiB) Valeur par défaut: 32768 (32 Ko) -## format\_avro\_schema\_registry\_url {#settings-format_avro_schema_registry_url} +## format_avro_schema_registry_url {#settings-format_avro_schema_registry_url} Définit L'URL de Registre de schéma Confluent à utiliser avec [AvroConfluent](../../interfaces/formats.md#data-format-avro-confluent) format @@ -1241,7 +1241,7 @@ Type: URL Valeur par défaut: vide -## background\_pool\_size {#background_pool_size} +## background_pool_size {#background_pool_size} Définit le nombre de threads effectuant des opérations d'arrière-plan dans les moteurs de table (par exemple, fusionne dans [Moteur MergeTree](../../engines/table-engines/mergetree-family/index.md) table). Ce paramètre est appliqué au démarrage du serveur ClickHouse et ne peut pas être modifié dans une session utilisateur. En ajustant ce paramètre, vous gérez la charge du processeur et du disque. Une taille de pool plus petite utilise moins de ressources CPU et disque, mais les processus d'arrière-plan avancent plus lentement, ce qui pourrait éventuellement avoir un impact sur les performances des requêtes. diff --git a/docs/fr/operations/system-tables.md b/docs/fr/operations/system-tables.md index d779f20a96a..bf875892478 100644 --- a/docs/fr/operations/system-tables.md +++ b/docs/fr/operations/system-tables.md @@ -13,7 +13,7 @@ Les tables système n'ont pas de fichiers avec des données sur le disque ou de Les tables système sont en lecture seule. Ils sont situés dans la ‘system’ la base de données. -## système.asynchronous\_metrics {#system_tables-asynchronous_metrics} +## système.asynchronous_metrics {#system_tables-asynchronous_metrics} Contient des mesures qui sont calculées périodiquement en arrière-plan. Par exemple, la quantité de RAM utilisée. @@ -48,7 +48,7 @@ SELECT * FROM system.asynchronous_metrics LIMIT 10 - [Surveiller](monitoring.md) — Base concepts of ClickHouse monitoring. - [système.métrique](#system_tables-metrics) — Contains instantly calculated metrics. - [système.événement](#system_tables-events) — Contains a number of events that have occurred. -- [système.metric\_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [système.metric_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. ## système.cluster {#system-clusters} @@ -72,8 +72,8 @@ Veuillez noter que `errors_count` est mise à jour une fois par requête à la g **Voir aussi** - [Tableau moteur Distribués](../engines/table-engines/special/distributed.md) -- [paramètre distributed\_replica\_error\_cap](settings/settings.md#settings-distributed_replica_error_cap) -- [paramètre distributed\_replica\_error\_half\_life](settings/settings.md#settings-distributed_replica_error_half_life) +- [paramètre distributed_replica_error_cap](settings/settings.md#settings-distributed_replica_error_cap) +- [paramètre distributed_replica_error_half_life](settings/settings.md#settings-distributed_replica_error_half_life) ## système.colonne {#system-columns} @@ -145,7 +145,7 @@ Cette table contient une seule colonne de chaîne appelée ‘name’ – the na Chaque base de données que le serveur connaît a une entrée correspondante dans la table. Cette table système est utilisée pour implémenter `SHOW DATABASES` requête. -## système.detached\_parts {#system_tables-detached_parts} +## système.detached_parts {#system_tables-detached_parts} Contient des informations sur les pièces détachées de [MergeTree](../engines/table-engines/mergetree-family/mergetree.md) table. Le `reason` colonne spécifie pourquoi la pièce a été détachée. Pour les pièces détachées par l'utilisateur, la raison est vide. De telles pièces peuvent être attachées avec [ALTER TABLE ATTACH PARTITION\|PART](../sql-reference/statements/alter.md#alter_attach-partition) commande. Pour la description des autres colonnes, voir [système.partie](#system_tables-parts). Si le nom de pièce n'est pas valide, les valeurs de certaines colonnes peuvent être `NULL`. Ces pièces peuvent être supprimés avec [ALTER TABLE DROP DETACHED PART](../sql-reference/statements/alter.md#alter_drop-detached). @@ -239,9 +239,9 @@ SELECT * FROM system.events LIMIT 5 **Voir Aussi** -- [système.asynchronous\_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [système.asynchronous_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. - [système.métrique](#system_tables-metrics) — Contains instantly calculated metrics. -- [système.metric\_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [système.metric_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. - [Surveiller](monitoring.md) — Base concepts of ClickHouse monitoring. ## système.fonction {#system-functions} @@ -253,9 +253,9 @@ Colonne: - `name`(`String`) – The name of the function. - `is_aggregate`(`UInt8`) — Whether the function is aggregate. -## système.graphite\_retentions {#system-graphite-retentions} +## système.graphite_retentions {#system-graphite-retentions} -Contient des informations sur les paramètres [graphite\_rollup](server-configuration-parameters/settings.md#server_configuration_parameters-graphite) qui sont utilisés dans les tableaux avec [\* GraphiteMergeTree](../engines/table-engines/mergetree-family/graphitemergetree.md) moteur. +Contient des informations sur les paramètres [graphite_rollup](server-configuration-parameters/settings.md#server_configuration_parameters-graphite) qui sont utilisés dans les tableaux avec [\* GraphiteMergeTree](../engines/table-engines/mergetree-family/graphitemergetree.md) moteur. Colonne: @@ -324,12 +324,12 @@ SELECT * FROM system.metrics LIMIT 10 **Voir Aussi** -- [système.asynchronous\_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [système.asynchronous_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. - [système.événement](#system_tables-events) — Contains a number of events that occurred. -- [système.metric\_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [système.metric_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. - [Surveiller](monitoring.md) — Base concepts of ClickHouse monitoring. -## système.metric\_log {#system_tables-metric_log} +## système.metric_log {#system_tables-metric_log} Contient l'historique des valeurs de métriques des tables `system.metrics` et `system.events` périodiquement vidé sur le disque. Pour activer la collection d'historique des métriques `system.metric_log`, créer `/etc/clickhouse-server/config.d/metric_log.xml` avec le contenu suivant: @@ -380,7 +380,7 @@ CurrentMetric_ReplicatedChecks: 0 **Voir aussi** -- [système.asynchronous\_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [système.asynchronous_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. - [système.événement](#system_tables-events) — Contains a number of events that occurred. - [système.métrique](#system_tables-metrics) — Contains instantly calculated metrics. - [Surveiller](monitoring.md) — Base concepts of ClickHouse monitoring. @@ -391,7 +391,7 @@ Cette table contient une seule colonne UInt64 nommée ‘number’ qui contient Vous pouvez utiliser cette table pour les tests, ou si vous avez besoin de faire une recherche de force brute. Les lectures de cette table ne sont pas parallélisées. -## système.numbers\_mt {#system-numbers-mt} +## système.numbers_mt {#system-numbers-mt} Le même que ‘system.numbers’ mais les lectures sont parallélisées. Les nombres peuvent être retournés dans n'importe quel ordre. Utilisé pour les tests. @@ -483,9 +483,9 @@ Colonne: - `marks_size` (`UInt64`) – Alias for `marks_bytes`. -## système.part\_log {#system_tables-part-log} +## système.part_log {#system_tables-part-log} -Le `system.part_log` la table est créée uniquement si [part\_log](server-configuration-parameters/settings.md#server_configuration_parameters-part-log) serveur paramètre est spécifié. +Le `system.part_log` la table est créée uniquement si [part_log](server-configuration-parameters/settings.md#server_configuration_parameters-part-log) serveur paramètre est spécifié. Ce tableau contient des informations sur les événements survenus avec [les parties de données](../engines/table-engines/mergetree-family/custom-partitioning-key.md) dans le [MergeTree](../engines/table-engines/mergetree-family/mergetree.md) table de famille, telles que l'ajout ou la fusion de données. @@ -528,11 +528,11 @@ Colonne: - `rows_read` (UInt64) – The number of rows read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. - `bytes_read` (UInt64) – The number of uncompressed bytes read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. - `total_rows_approx` (UInt64) – The approximation of the total number of rows that should be read. For distributed processing, on the requestor server, this is the total for all remote servers. It can be updated during request processing, when new sources to process become known. -- `memory_usage` (UInt64) – Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max\_memory\_usage](../operations/settings/query-complexity.md#settings_max_memory_usage) paramètre. +- `memory_usage` (UInt64) – Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max_memory_usage](../operations/settings/query-complexity.md#settings_max_memory_usage) paramètre. - `query` (String) – The query text. For `INSERT` il n'inclut pas les données à insérer. - `query_id` (String) – Query ID, if defined. -## système.text\_log {#system_tables-text_log} +## système.text_log {#system_tables-text_log} Contient des entrées de journalisation. Niveau de journalisation qui va à cette table peut être limité `text_log.level` paramètre de serveur. @@ -559,16 +559,16 @@ Colonne: - `source_file` (`LowCardinality(String)`)- Fichier Source à partir duquel la journalisation a été effectuée. - `source_line` (`UInt64`)- Ligne Source à partir de laquelle la journalisation a été effectuée. -## système.query\_log {#system_tables-query_log} +## système.query_log {#system_tables-query_log} Contient des informations sur l'exécution de requêtes. Pour chaque requête, vous pouvez voir l'Heure de début du traitement, la durée du traitement, les messages d'erreur et d'autres informations. !!! note "Note" Le tableau ne contient pas les données d'entrée pour `INSERT` requête. -Clickhouse crée cette table uniquement si [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) serveur paramètre est spécifié. Ce paramètre définit les règles de journalisation, tels que l'intervalle d'enregistrement ou le nom de la table, la requête sera connecté. +Clickhouse crée cette table uniquement si [query_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) serveur paramètre est spécifié. Ce paramètre définit les règles de journalisation, tels que l'intervalle d'enregistrement ou le nom de la table, la requête sera connecté. -Pour activer la journalisation des requêtes, définissez [log\_queries](settings/settings.md#settings-log-queries) paramètre 1. Pour plus de détails, voir le [Paramètre](settings/settings.md) section. +Pour activer la journalisation des requêtes, définissez [log_queries](settings/settings.md#settings-log-queries) paramètre 1. Pour plus de détails, voir le [Paramètre](settings/settings.md) section. Le `system.query_log` table enregistre deux types de requêtes: @@ -636,22 +636,22 @@ Chaque requête crée une ou deux lignes dans le `query_log` le tableau, en fonc 2. Si une erreur s'est produite pendant le traitement de la requête, deux événements avec les types 1 et 4 sont créés. 3. Si une erreur s'est produite avant le lancement de la requête, un seul événement de type 3 est créé. -Par défaut, les journaux sont ajoutés à la table à des intervalles de 7,5 secondes. Vous pouvez définir cet intervalle dans la [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) configuration du serveur (voir `flush_interval_milliseconds` paramètre). Pour vider les journaux de force du tampon mémoire dans la table, utilisez le `SYSTEM FLUSH LOGS` requête. +Par défaut, les journaux sont ajoutés à la table à des intervalles de 7,5 secondes. Vous pouvez définir cet intervalle dans la [query_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) configuration du serveur (voir `flush_interval_milliseconds` paramètre). Pour vider les journaux de force du tampon mémoire dans la table, utilisez le `SYSTEM FLUSH LOGS` requête. Lorsque la table est supprimée manuellement, il sera automatiquement créé à la volée. Notez que tous les précédents journaux seront supprimés. !!! note "Note" La période de stockage des journaux est illimitée. Les journaux ne sont pas automatiquement supprimés de la table. Vous devez organiser vous-même la suppression des journaux obsolètes. -Vous pouvez spécifier une clé de partitionnement arbitraire pour `system.query_log` la table dans le [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) configuration du serveur (voir `partition_by` paramètre). +Vous pouvez spécifier une clé de partitionnement arbitraire pour `system.query_log` la table dans le [query_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) configuration du serveur (voir `partition_by` paramètre). -## système.query\_thread\_log {#system_tables-query-thread-log} +## système.query_thread_log {#system_tables-query-thread-log} La table contient des informations sur chaque thread d'exécution de requête. -Clickhouse crée cette table uniquement si [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) serveur paramètre est spécifié. Ce paramètre définit les règles de journalisation, tels que l'intervalle d'enregistrement ou le nom de la table, la requête sera connecté. +Clickhouse crée cette table uniquement si [query_thread_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) serveur paramètre est spécifié. Ce paramètre définit les règles de journalisation, tels que l'intervalle d'enregistrement ou le nom de la table, la requête sera connecté. -Pour activer la journalisation des requêtes, définissez [log\_query\_threads](settings/settings.md#settings-log-query-threads) paramètre 1. Pour plus de détails, voir le [Paramètre](settings/settings.md) section. +Pour activer la journalisation des requêtes, définissez [log_query_threads](settings/settings.md#settings-log-query-threads) paramètre 1. Pour plus de détails, voir le [Paramètre](settings/settings.md) section. Colonne: @@ -701,20 +701,20 @@ Colonne: - `ProfileEvents.Names` (Array(String)) — Counters that measure different metrics for this thread. The description of them could be found in the table [système.événement](#system_tables-events) - `ProfileEvents.Values` (Array(UInt64)) — Values of metrics for this thread that are listed in the `ProfileEvents.Names` colonne. -Par défaut, les journaux sont ajoutés à la table à des intervalles de 7,5 secondes. Vous pouvez définir cet intervalle dans la [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) configuration du serveur (voir `flush_interval_milliseconds` paramètre). Pour vider les journaux de force du tampon mémoire dans la table, utilisez le `SYSTEM FLUSH LOGS` requête. +Par défaut, les journaux sont ajoutés à la table à des intervalles de 7,5 secondes. Vous pouvez définir cet intervalle dans la [query_thread_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) configuration du serveur (voir `flush_interval_milliseconds` paramètre). Pour vider les journaux de force du tampon mémoire dans la table, utilisez le `SYSTEM FLUSH LOGS` requête. Lorsque la table est supprimée manuellement, il sera automatiquement créé à la volée. Notez que tous les précédents journaux seront supprimés. !!! note "Note" La période de stockage des journaux est illimitée. Les journaux ne sont pas automatiquement supprimés de la table. Vous devez organiser vous-même la suppression des journaux obsolètes. -Vous pouvez spécifier une clé de partitionnement arbitraire pour `system.query_thread_log` la table dans le [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) configuration du serveur (voir `partition_by` paramètre). +Vous pouvez spécifier une clé de partitionnement arbitraire pour `system.query_thread_log` la table dans le [query_thread_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) configuration du serveur (voir `partition_by` paramètre). -## système.trace\_log {#system_tables-trace_log} +## système.trace_log {#system_tables-trace_log} Contient des traces de pile collectées par le profileur de requête d'échantillonnage. -Clickhouse crée cette table lorsque le [trace\_log](server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) la section de configuration du serveur est définie. Aussi l' [query\_profiler\_real\_time\_period\_ns](settings/settings.md#query_profiler_real_time_period_ns) et [query\_profiler\_cpu\_time\_period\_ns](settings/settings.md#query_profiler_cpu_time_period_ns) paramètres doivent être définis. +Clickhouse crée cette table lorsque le [trace_log](server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) la section de configuration du serveur est définie. Aussi l' [query_profiler_real_time_period_ns](settings/settings.md#query_profiler_real_time_period_ns) et [query_profiler_cpu_time_period_ns](settings/settings.md#query_profiler_cpu_time_period_ns) paramètres doivent être définis. Pour analyser les journaux, utilisez `addressToLine`, `addressToSymbol` et `demangle` fonctions d'introspection. @@ -737,7 +737,7 @@ Colonne: - `thread_number` ([UInt32](../sql-reference/data-types/int-uint.md)) — Thread identifier. -- `query_id` ([Chaîne](../sql-reference/data-types/string.md)) — Query identifier that can be used to get details about a query that was running from the [query\_log](#system_tables-query_log) système de table. +- `query_id` ([Chaîne](../sql-reference/data-types/string.md)) — Query identifier that can be used to get details about a query that was running from the [query_log](#system_tables-query_log) système de table. - `trace` ([Tableau (UInt64)](../sql-reference/data-types/array.md)) — Stack trace at the moment of sampling. Each element is a virtual memory address inside ClickHouse server process. @@ -824,7 +824,7 @@ Colonne: - `parts_to_check` (`UInt32`) - Le nombre de parties des données dans la file d'attente pour la vérification. Une pièce est placée dans la file d'attente de vérification s'il y a un soupçon qu'elle pourrait être endommagée. - `zookeeper_path` (`String`)- Chemin d'accès aux données de la table dans ZooKeeper. - `replica_name` (`String`) - Réplique nom de la Gardienne. Différentes répliques d'une même table ont des noms différents. -- `replica_path` (`String`)- Chemin vers les données de réplique dans ZooKeeper. La même chose que la concaténation ‘zookeeper\_path/replicas/replica\_path’. +- `replica_path` (`String`)- Chemin vers les données de réplique dans ZooKeeper. La même chose que la concaténation ‘zookeeper_path/replicas/replica_path’. - `columns_version` (`Int32`)- Numéro de Version de la structure de la table. Indique combien de fois ALTER a été effectué. Si les répliques ont des versions différentes, cela signifie que certaines répliques n'ont pas encore Toutes les modifications. - `queue_size` (`UInt32`),- La taille de la file d'attente pour les opérations en attente d'être exécuté. Les opérations comprennent l'insertion de blocs de données, les fusions et certaines autres actions. Il coïncide généralement avec `future_parts`. - `inserts_in_queue` (`UInt32`) - Nombre d'insertions de blocs de données qui doivent être faits. Les Insertions sont généralement répliquées assez rapidement. Si ce nombre est grand, cela signifie que quelque chose est faux. @@ -845,7 +845,7 @@ Les 4 colonnes suivantes ont une valeur non nulle uniquement lorsqu'il y a une s - `active_replicas` (`UInt8`) - Le nombre de répliques de cette table qui ont une session dans ZooKeeper (c'est-à-dire le nombre de répliques fonctionnelles). Si vous demandez toutes les colonnes, la table peut fonctionner un peu lentement, car plusieurs lectures de ZooKeeper sont faites pour chaque ligne. -Si vous ne demandez pas les 4 dernières colonnes (log\_max\_index, log\_pointer, total\_replicas, active\_replicas), la table fonctionne rapidement. +Si vous ne demandez pas les 4 dernières colonnes (log_max_index, log_pointer, total_replicas, active_replicas), la table fonctionne rapidement. Par exemple, vous pouvez vérifier que tout fonctionne correctement comme ceci: @@ -932,7 +932,7 @@ SELECT * FROM system.settings WHERE changed AND name='load_balancing' - [Autorisations pour les requêtes](settings/permissions-for-queries.md#settings_readonly) - [Contraintes sur les paramètres](settings/constraints-on-settings.md) -## système.tableau\_moteurs {#system.table_engines} +## système.tableau_moteurs {#system.table_engines} ``` text ┌─name───────────────────┬─value───────┐ @@ -943,7 +943,7 @@ SELECT * FROM system.settings WHERE changed AND name='load_balancing' └────────────────────────┴─────────────┘ ``` -## système.merge\_tree\_settings {#system-merge_tree_settings} +## système.merge_tree_settings {#system-merge_tree_settings} Contient des informations sur les paramètres pour `MergeTree` table. @@ -955,7 +955,7 @@ Colonne: - `type` (String) — Setting type (implementation specific string value). - `changed` (UInt8) — Whether the setting was explicitly defined in the config or explicitly changed. -## système.tableau\_moteurs {#system-table-engines} +## système.tableau_moteurs {#system-table-engines} Contient une description des moteurs de table pris en charge par le serveur et leurs informations de support de fonctionnalité. @@ -1118,25 +1118,25 @@ Le tableau contient des informations sur [mutation](../sql-reference/statements/ **base de données**, **table** - Le nom de la base de données et de la table à laquelle la mutation a été appliquée. -**mutation\_id** - Le numéro d'identification de la mutation. Pour les tables répliquées ces ID correspondent aux noms znode dans le `/mutations/` répertoire de la Gardienne. Pour les tables non compliquées, Les Id correspondent aux noms de fichiers dans le répertoire de données de la table. +**mutation_id** - Le numéro d'identification de la mutation. Pour les tables répliquées ces ID correspondent aux noms znode dans le `/mutations/` répertoire de la Gardienne. Pour les tables non compliquées, Les Id correspondent aux noms de fichiers dans le répertoire de données de la table. **commande** - La chaîne de commande mutation (la partie de la requête après `ALTER TABLE [db.]table`). -**create\_time** - Quand cette commande de mutation a été soumise pour exécution. +**create_time** - Quand cette commande de mutation a été soumise pour exécution. -**block\_numbers.partition\_id**, **block\_numbers.nombre** - Une colonne imbriquée. Pour les mutations de tables répliquées, il contient un enregistrement pour chaque partition: l'ID de partition et le numéro de bloc acquis par la mutation (dans chaque partition, seules les parties contenant des blocs avec des nombres inférieurs au numéro de bloc acquis par la mutation dans cette partition seront mutées). Dans les tables non répliquées, les numéros de bloc de toutes les partitions forment une seule séquence. Cela signifie que pour les mutations de tables non répliquées, la colonne contiendra un enregistrement avec un seul numéro de bloc acquis par la mutation. +**block_numbers.partition_id**, **block_numbers.nombre** - Une colonne imbriquée. Pour les mutations de tables répliquées, il contient un enregistrement pour chaque partition: l'ID de partition et le numéro de bloc acquis par la mutation (dans chaque partition, seules les parties contenant des blocs avec des nombres inférieurs au numéro de bloc acquis par la mutation dans cette partition seront mutées). Dans les tables non répliquées, les numéros de bloc de toutes les partitions forment une seule séquence. Cela signifie que pour les mutations de tables non répliquées, la colonne contiendra un enregistrement avec un seul numéro de bloc acquis par la mutation. -**parts\_to\_do** - Le nombre de parties de données qui doivent être mutées pour que la mutation se termine. +**parts_to_do** - Le nombre de parties de données qui doivent être mutées pour que la mutation se termine. -**\_done** - La mutation est faite? Notez que même si `parts_to_do = 0` il est possible qu'une mutation d'une table répliquée ne soit pas encore effectuée en raison d'un INSERT de longue durée qui créera une nouvelle partie de données qui devra être mutée. +**_done** - La mutation est faite? Notez que même si `parts_to_do = 0` il est possible qu'une mutation d'une table répliquée ne soit pas encore effectuée en raison d'un INSERT de longue durée qui créera une nouvelle partie de données qui devra être mutée. S'il y avait des problèmes avec la mutation de certaines parties, les colonnes suivantes contiennent des informations supplémentaires: -**latest\_failed\_part** - Le nom de la partie la plus récente qui n'a pas pu être mutée. +**latest_failed_part** - Le nom de la partie la plus récente qui n'a pas pu être mutée. -**latest\_fail\_time** - Le temps de la partie la plus récente mutation de l'échec. +**latest_fail_time** - Le temps de la partie la plus récente mutation de l'échec. -**latest\_fail\_reason** - Le message d'exception qui a provoqué l'échec de la mutation de pièce la plus récente. +**latest_fail_reason** - Le message d'exception qui a provoqué l'échec de la mutation de pièce la plus récente. ## système.disque {#system_tables-disks} @@ -1150,7 +1150,7 @@ Colonne: - `total_space` ([UInt64](../sql-reference/data-types/int-uint.md)) — Disk volume in bytes. - `keep_free_space` ([UInt64](../sql-reference/data-types/int-uint.md)) — Amount of disk space that should stay free on disk in bytes. Defined in the `keep_free_space_bytes` paramètre de configuration du disque. -## système.storage\_policies {#system_tables-storage_policies} +## système.storage_policies {#system_tables-storage_policies} Contient des informations sur les stratégies de stockage et les volumes définis [configuration du serveur](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes_configure). diff --git a/docs/fr/operations/tips.md b/docs/fr/operations/tips.md index a4fe5f844de..dc963e5e27a 100644 --- a/docs/fr/operations/tips.md +++ b/docs/fr/operations/tips.md @@ -59,7 +59,7 @@ Lors de la création de RAID-10, sélectionnez `far` disposition. Si votre budget le permet, choisissez RAID-10. Si vous avez plus de 4 disques, Utilisez RAID-6 (préféré) ou RAID-50, au lieu de RAID-5. -Lorsque vous utilisez RAID-5, RAID-6 ou RAID-50, augmentez toujours stripe\_cache\_size, car la valeur par défaut n'est généralement pas le meilleur choix. +Lorsque vous utilisez RAID-5, RAID-6 ou RAID-50, augmentez toujours stripe_cache_size, car la valeur par défaut n'est généralement pas le meilleur choix. ``` bash $ echo 4096 | sudo tee /sys/block/md2/md/stripe_cache_size diff --git a/docs/fr/operations/troubleshooting.md b/docs/fr/operations/troubleshooting.md index 0608569c61c..f7d3153a4d8 100644 --- a/docs/fr/operations/troubleshooting.md +++ b/docs/fr/operations/troubleshooting.md @@ -105,7 +105,7 @@ Vérifier: - Paramètres du point de terminaison. - Vérifier [listen\_host](server-configuration-parameters/settings.md#server_configuration_parameters-listen_host) et [tcp\_port](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) paramètre. + Vérifier [listen_host](server-configuration-parameters/settings.md#server_configuration_parameters-listen_host) et [tcp_port](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) paramètre. Clickhouse server accepte les connexions localhost uniquement par défaut. @@ -117,7 +117,7 @@ Vérifier: Vérifier: - - Le [tcp\_port\_secure](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) paramètre. + - Le [tcp_port_secure](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) paramètre. - Paramètres pour [SSL sertificates](server-configuration-parameters/settings.md#server_configuration_parameters-openssl). Utilisez les paramètres appropriés lors de la connexion. Par exemple, l'utilisation de la `port_secure` paramètre avec `clickhouse_client`. diff --git a/docs/fr/sql-reference/aggregate-functions/parametric-functions.md b/docs/fr/sql-reference/aggregate-functions/parametric-functions.md index a8d6d533cb8..ed00ad7a0a3 100644 --- a/docs/fr/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/fr/sql-reference/aggregate-functions/parametric-functions.md @@ -494,6 +494,6 @@ Solution: Write in the GROUP BY query SearchPhrase HAVING uniqUpTo(4)(UserID) >= [Article Original](https://clickhouse.tech/docs/en/query_language/agg_functions/parametric_functions/) -## sumMapFiltered(keys\_to\_keep) (clés, valeurs) {#summapfilteredkeys-to-keepkeys-values} +## sumMapFiltered(keys_to_keep) (clés, valeurs) {#summapfilteredkeys-to-keepkeys-values} Même comportement que [sumMap](reference.md#agg_functions-summap) sauf qu'un tableau de clés est passé en paramètre. Cela peut être particulièrement utile lorsque vous travaillez avec une forte cardinalité de touches. diff --git a/docs/fr/sql-reference/aggregate-functions/reference.md b/docs/fr/sql-reference/aggregate-functions/reference.md index 456a9e9c4b3..dcbf3bdd95a 100644 --- a/docs/fr/sql-reference/aggregate-functions/reference.md +++ b/docs/fr/sql-reference/aggregate-functions/reference.md @@ -31,7 +31,7 @@ Dans les deux cas le type de la valeur renvoyée est [UInt64](../../sql-referenc **Détail** -Clickhouse soutient le `COUNT(DISTINCT ...)` syntaxe. Le comportement de cette construction dépend de la [count\_distinct\_implementation](../../operations/settings/settings.md#settings-count_distinct_implementation) paramètre. Il définit lequel des [uniq\*](#agg_function-uniq) fonctions est utilisée pour effectuer l'opération. La valeur par défaut est la [uniqExact](#agg_function-uniqexact) fonction. +Clickhouse soutient le `COUNT(DISTINCT ...)` syntaxe. Le comportement de cette construction dépend de la [count_distinct_implementation](../../operations/settings/settings.md#settings-count_distinct_implementation) paramètre. Il définit lequel des [uniq\*](#agg_function-uniq) fonctions est utilisée pour effectuer l'opération. La valeur par défaut est la [uniqExact](#agg_function-uniqexact) fonction. Le `SELECT count() FROM table` la requête n'est pas optimisé, car le nombre d'entrées dans la table n'est pas stockée séparément. Il choisit une petite colonne de la table et compte le nombre de valeurs qu'il contient. @@ -721,7 +721,7 @@ La fonction prend un nombre variable de paramètres. Les paramètres peuvent êt - [uniqcombiné](#agg_function-uniqcombined) - [uniqHLL12](#agg_function-uniqhll12) -## groupArray(x), groupArray (max\_size) (x) {#agg_function-grouparray} +## groupArray(x), groupArray (max_size) (x) {#agg_function-grouparray} Crée un tableau de valeurs de l'argument. Les valeurs peuvent être ajoutées au tableau dans une (indéterminée) de commande. @@ -967,7 +967,7 @@ FROM t └───────────┴──────────────────────────────────┴───────────────────────┘ ``` -## groupUniqArray(x), groupUniqArray (max\_size) (x) {#groupuniqarrayx-groupuniqarraymax-sizex} +## groupUniqArray(x), groupUniqArray (max_size) (x) {#groupuniqarrayx-groupuniqarraymax-sizex} Crée un tableau à partir de différentes valeurs d'argument. La consommation de mémoire est la même que pour la `uniqExact` fonction. diff --git a/docs/fr/sql-reference/ansi.md b/docs/fr/sql-reference/ansi.md index 8fa5e272085..033d4c2c927 100644 --- a/docs/fr/sql-reference/ansi.md +++ b/docs/fr/sql-reference/ansi.md @@ -36,8 +36,8 @@ Le tableau suivant répertorie les cas où la fonctionnalité de requête foncti | E021-01 | Type de données CARACTÈRE | Aucun{.text-danger} | | | E021-02 | TYPE DE DONNÉES variable de caractère | Aucun{.text-danger} | `String` se comporte de la même manière, mais sans limite de longueur entre parenthèses | | E021-03 | Littéraux de caractères | Partiel{.text-warning} | Aucune concaténation automatique de littéraux consécutifs et prise en charge du jeu de caractères | -| E021-04 | Fonction CHARACTER\_LENGTH | Partiel{.text-warning} | Aucun `USING` clause | -| E021-05 | Fonction OCTET\_LENGTH | Aucun{.text-danger} | `LENGTH` se comporte de la même façon | +| E021-04 | Fonction CHARACTER_LENGTH | Partiel{.text-warning} | Aucun `USING` clause | +| E021-05 | Fonction OCTET_LENGTH | Aucun{.text-danger} | `LENGTH` se comporte de la même façon | | E021-06 | SUBSTRING | Partiel{.text-warning} | Pas de support pour `SIMILAR` et `ESCAPE` clauses, pas de `SUBSTRING_REGEX` variante | | E021-07 | Concaténation de caractères | Partiel{.text-warning} | Aucun `COLLATE` clause | | E021-08 | Fonctions supérieures et inférieures | Oui{.text-success} | | @@ -144,7 +144,7 @@ Le tableau suivant répertorie les cas où la fonctionnalité de requête foncti | F051-03 | Type de données D'horodatage (y compris la prise en charge du littéral D'horodatage) avec une précision de secondes fractionnaires d'au moins 0 et 6 | Aucun{.text-danger} | `DateTime64` temps fournit des fonctionnalités similaires | | F051-04 | Prédicat de comparaison sur les types de données DATE, heure et horodatage | Partiel{.text-warning} | Un seul type de données disponible | | F051-05 | Distribution explicite entre les types datetime et les types de chaînes de caractères | Oui{.text-success} | | -| F051-06 | CURRENT\_DATE | Aucun{.text-danger} | `today()` est similaire | +| F051-06 | CURRENT_DATE | Aucun{.text-danger} | `today()` est similaire | | F051-07 | LOCALTIME | Aucun{.text-danger} | `now()` est similaire | | F051-08 | LOCALTIMESTAMP | Aucun{.text-danger} | | | **F081** | **UNION et sauf dans les vues** | **Partiel**{.text-warning} | | diff --git a/docs/fr/sql-reference/data-types/aggregatefunction.md b/docs/fr/sql-reference/data-types/aggregatefunction.md index fce2d89580a..18874cd3cb7 100644 --- a/docs/fr/sql-reference/data-types/aggregatefunction.md +++ b/docs/fr/sql-reference/data-types/aggregatefunction.md @@ -5,7 +5,7 @@ toc_priority: 52 toc_title: AggregateFunction (nom, types_of_arguments...) --- -# AggregateFunction(name, types\_of\_arguments…) {#data-type-aggregatefunction} +# AggregateFunction(name, types_of_arguments…) {#data-type-aggregatefunction} Aggregate functions can have an implementation-defined intermediate state that can be serialized to an AggregateFunction(…) data type and stored in a table, usually, by means of [une vue matérialisée](../../sql-reference/statements/create.md#create-view). La manière courante de produire un État de fonction d'agrégat est d'appeler la fonction d'agrégat avec le `-State` suffixe. Pour obtenir le résultat final de l'agrégation dans l'avenir, vous devez utiliser la même fonction d'agrégation avec la `-Merge`suffixe. diff --git a/docs/fr/sql-reference/data-types/datetime.md b/docs/fr/sql-reference/data-types/datetime.md index cd6a3052c01..915270e4d2b 100644 --- a/docs/fr/sql-reference/data-types/datetime.md +++ b/docs/fr/sql-reference/data-types/datetime.md @@ -31,7 +31,7 @@ Le [clickhouse-client](../../interfaces/cli.md) applique le fuseau horaire du se Clickhouse affiche les valeurs dans `YYYY-MM-DD hh:mm:ss` format de texte par défaut. Vous pouvez modifier la sortie avec le [formatDateTime](../../sql-reference/functions/date-time-functions.md#formatdatetime) fonction. -Lorsque vous insérez des données dans ClickHouse, vous pouvez utiliser différents formats de chaînes de date et d'heure, en fonction de la valeur du [date\_time\_input\_format](../../operations/settings/settings.md#settings-date_time_input_format) paramètre. +Lorsque vous insérez des données dans ClickHouse, vous pouvez utiliser différents formats de chaînes de date et d'heure, en fonction de la valeur du [date_time_input_format](../../operations/settings/settings.md#settings-date_time_input_format) paramètre. ## Exemple {#examples} diff --git a/docs/fr/sql-reference/data-types/fixedstring.md b/docs/fr/sql-reference/data-types/fixedstring.md index c754176b519..5ba09187581 100644 --- a/docs/fr/sql-reference/data-types/fixedstring.md +++ b/docs/fr/sql-reference/data-types/fixedstring.md @@ -22,7 +22,7 @@ Le `FixedString` type est efficace lorsque les données ont la longueur de préc Exemples de valeurs qui peuvent être stockées efficacement dans `FixedString`-tapé colonnes: - La représentation binaire des adresses IP (`FixedString(16)` pour IPv6). -- Language codes (ru\_RU, en\_US … ). +- Language codes (ru_RU, en_US … ). - Currency codes (USD, RUB … ). - Représentation binaire des hachages (`FixedString(16)` pour MD5, `FixedString(32)` pour SHA256). diff --git a/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md b/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md index 26bf656b2c2..cc238f02f3a 100644 --- a/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md +++ b/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md @@ -27,7 +27,7 @@ Voici une structure hiérarchique: Cette hiérarchie peut être exprimée comme la table de dictionnaire suivante. -| id\_région | région\_parent | nom\_région | +| id_région | région_parent | nom_région | |------------|----------------|--------------------| | 1 | 0 | Russie | | 2 | 1 | Moscou | diff --git a/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index f1e65631a56..3bdc8d37d60 100644 --- a/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -9,7 +9,7 @@ toc_title: "Stockage des dictionnaires en m\xE9moire" Il existe une variété de façons de stocker les dictionnaires en mémoire. -Nous vous recommandons [plat](#flat), [haché](#dicts-external_dicts_dict_layout-hashed) et [complex\_key\_hashed](#complex-key-hashed). qui fournissent la vitesse de traitement optimale. +Nous vous recommandons [plat](#flat), [haché](#dicts-external_dicts_dict_layout-hashed) et [complex_key_hashed](#complex-key-hashed). qui fournissent la vitesse de traitement optimale. La mise en cache n'est pas recommandée en raison de performances potentiellement médiocres et de difficultés à sélectionner les paramètres optimaux. En savoir plus dans la section “[cache](#cache)”. @@ -54,14 +54,14 @@ LAYOUT(LAYOUT_TYPE(param value)) -- layout settings - [plat](#flat) - [haché](#dicts-external_dicts_dict_layout-hashed) -- [sparse\_hashed](#dicts-external_dicts_dict_layout-sparse_hashed) +- [sparse_hashed](#dicts-external_dicts_dict_layout-sparse_hashed) - [cache](#cache) - [direct](#direct) -- [range\_hashed](#range-hashed) -- [complex\_key\_hashed](#complex-key-hashed) -- [complex\_key\_cache](#complex-key-cache) -- [complex\_key\_direct](#complex-key-direct) -- [ip\_trie](#ip-trie) +- [range_hashed](#range-hashed) +- [complex_key_hashed](#complex-key-hashed) +- [complex_key_cache](#complex-key-cache) +- [complex_key_direct](#complex-key-direct) +- [ip_trie](#ip-trie) ### plat {#flat} @@ -107,7 +107,7 @@ ou LAYOUT(HASHED()) ``` -### sparse\_hashed {#dicts-external_dicts_dict_layout-sparse_hashed} +### sparse_hashed {#dicts-external_dicts_dict_layout-sparse_hashed} Semblable à `hashed`, mais utilise moins de mémoire en faveur de plus D'utilisation du processeur. @@ -123,7 +123,7 @@ Exemple de Configuration: LAYOUT(SPARSE_HASHED()) ``` -### complex\_key\_hashed {#complex-key-hashed} +### complex_key_hashed {#complex-key-hashed} Ce type de stockage est pour une utilisation avec composite [touches](external-dicts-dict-structure.md). Semblable à `hashed`. @@ -139,7 +139,7 @@ Exemple de Configuration: LAYOUT(COMPLEX_KEY_HASHED()) ``` -### range\_hashed {#range-hashed} +### range_hashed {#range-hashed} Le dictionnaire est stocké en mémoire sous la forme d'une table de hachage avec un tableau ordonné de gammes et leurs valeurs correspondantes. @@ -294,7 +294,7 @@ Définissez une taille de cache suffisamment grande. Vous devez expérimenter po !!! warning "Avertissement" N'utilisez pas ClickHouse comme source, car le traitement des requêtes avec des lectures aléatoires est lent. -### complex\_key\_cache {#complex-key-cache} +### complex_key_cache {#complex-key-cache} Ce type de stockage est pour une utilisation avec composite [touches](external-dicts-dict-structure.md). Semblable à `cache`. @@ -320,11 +320,11 @@ ou LAYOUT(DIRECT()) ``` -### complex\_key\_direct {#complex-key-direct} +### complex_key_direct {#complex-key-direct} Ce type de stockage est destiné à être utilisé avec des [clés](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md) composites. Similaire à `direct` -### ip\_trie {#ip-trie} +### ip_trie {#ip-trie} Ce type de stockage permet de mapper des préfixes de réseau (adresses IP) à des métadonnées telles que ASN. diff --git a/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts.md index 468734dab06..d68b7a7f112 100644 --- a/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ b/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts.md @@ -15,9 +15,9 @@ ClickHouse: - Met à jour périodiquement les dictionnaires et charge dynamiquement les valeurs manquantes. En d'autres mots, les dictionnaires peuvent être chargés dynamiquement. - Permet de créer des dictionnaires externes avec des fichiers xml ou [Les requêtes DDL](../../statements/create.md#create-dictionary-query). -La configuration des dictionnaires externes peut être située dans un ou plusieurs fichiers xml. Le chemin d'accès à la configuration spécifiée dans le [dictionaries\_config](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config) paramètre. +La configuration des dictionnaires externes peut être située dans un ou plusieurs fichiers xml. Le chemin d'accès à la configuration spécifiée dans le [dictionaries_config](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config) paramètre. -Les dictionnaires peuvent être chargés au démarrage du serveur ou à la première utilisation, en fonction [dictionaries\_lazy\_load](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load) paramètre. +Les dictionnaires peuvent être chargés au démarrage du serveur ou à la première utilisation, en fonction [dictionaries_lazy_load](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load) paramètre. Le [dictionnaire](../../../operations/system-tables.md#system_tables-dictionaries) la table système contient des informations sur les dictionnaires configurés sur le serveur. Pour chaque dictionnaire, vous pouvez y trouver: diff --git a/docs/fr/sql-reference/functions/array-functions.md b/docs/fr/sql-reference/functions/array-functions.md index ef09800614f..40568841372 100644 --- a/docs/fr/sql-reference/functions/array-functions.md +++ b/docs/fr/sql-reference/functions/array-functions.md @@ -1028,7 +1028,7 @@ Résultat: ## arrayAUC {#arrayauc} -Calculer AUC (zone sous la courbe, qui est un concept dans l'apprentissage automatique, voir plus de détails: https://en.wikipedia.org/wiki/Receiver\_operating\_characteristic\#Area\_under\_the\_curve). +Calculer AUC (zone sous la courbe, qui est un concept dans l'apprentissage automatique, voir plus de détails: https://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve). **Syntaxe** diff --git a/docs/fr/sql-reference/functions/bitmap-functions.md b/docs/fr/sql-reference/functions/bitmap-functions.md index 1ab6bc9bb4f..15cb68ffc52 100644 --- a/docs/fr/sql-reference/functions/bitmap-functions.md +++ b/docs/fr/sql-reference/functions/bitmap-functions.md @@ -65,7 +65,7 @@ SELECT bitmapToArray(bitmapBuild([1, 2, 3, 4, 5])) AS res ## bitmapSubsetInRange {#bitmap-functions-bitmapsubsetinrange} -Retourne le sous-ensemble dans la plage spécifiée (n'inclut pas le range\_end). +Retourne le sous-ensemble dans la plage spécifiée (n'inclut pas le range_end). ``` sql bitmapSubsetInRange(bitmap, range_start, range_end) @@ -241,7 +241,7 @@ SELECT bitmapCardinality(bitmapBuild([1, 2, 3, 4, 5])) AS res ## bitmapMin {#bitmapmin} -Retrun la plus petite valeur de type UInt64 dans l'ensemble, UINT32\_MAX si l'ensemble est vide. +Retrun la plus petite valeur de type UInt64 dans l'ensemble, UINT32_MAX si l'ensemble est vide. bitmapMin(bitmap) @@ -288,8 +288,8 @@ Transformer un tableau de valeurs d'une image à l'autre tableau de valeurs, le **Paramètre** - `bitmap` – bitmap object. -- `from_array` – UInt32 array. For idx in range \[0, from\_array.size()), if bitmap contains from\_array\[idx\], then replace it with to\_array\[idx\]. Note that the result depends on array ordering if there are common elements between from\_array and to\_array. -- `to_array` – UInt32 array, its size shall be the same to from\_array. +- `from_array` – UInt32 array. For idx in range \[0, from_array.size()), if bitmap contains from_array\[idx\], then replace it with to_array\[idx\]. Note that the result depends on array ordering if there are common elements between from_array and to_array. +- `to_array` – UInt32 array, its size shall be the same to from_array. **Exemple** diff --git a/docs/fr/sql-reference/functions/date-time-functions.md b/docs/fr/sql-reference/functions/date-time-functions.md index 4def4bbe6bf..d1c16b42d07 100644 --- a/docs/fr/sql-reference/functions/date-time-functions.md +++ b/docs/fr/sql-reference/functions/date-time-functions.md @@ -163,7 +163,7 @@ Arrondit une date avec le temps au début de l " intervalle de dix minutes. Arrondit la date avec le temps jusqu'au début de l'intervalle de quinze minutes. -## toStartOfInterval(time\_or\_data, intervalle x Unité \[, time\_zone\]) {#tostartofintervaltime-or-data-interval-x-unit-time-zone} +## toStartOfInterval(time_or_data, intervalle x Unité \[, time_zone\]) {#tostartofintervaltime-or-data-interval-x-unit-time-zone} Ceci est une généralisation d'autres fonctions nommées `toStartOf*`. Exemple, `toStartOfInterval(t, INTERVAL 1 year)` renvoie la même chose que `toStartOfYear(t)`, diff --git a/docs/fr/sql-reference/functions/geo.md b/docs/fr/sql-reference/functions/geo.md index dd6bd9164ba..a89f03c7216 100644 --- a/docs/fr/sql-reference/functions/geo.md +++ b/docs/fr/sql-reference/functions/geo.md @@ -209,10 +209,10 @@ Renvoie un tableau de chaînes codées geohash de précision donnée qui tombent **Les valeurs d'entrée** -- longitude\_min-longitude min, valeur flottante dans la plage `[-180°, 180°]` -- latitude\_min-latitude min, valeur flottante dans la plage `[-90°, 90°]` -- longitude\_max-longitude maximale, valeur flottante dans la plage `[-180°, 180°]` -- latitude\_max-latitude maximale, valeur flottante dans la plage `[-90°, 90°]` +- longitude_min-longitude min, valeur flottante dans la plage `[-180°, 180°]` +- latitude_min-latitude min, valeur flottante dans la plage `[-90°, 90°]` +- longitude_max-longitude maximale, valeur flottante dans la plage `[-180°, 180°]` +- latitude_max-latitude maximale, valeur flottante dans la plage `[-90°, 90°]` - précision - geohash précision, `UInt8` dans la gamme `[1, 12]` Veuillez noter que tous les paramètres de coordonnées doit être du même type: soit `Float32` ou `Float64`. diff --git a/docs/fr/sql-reference/functions/hash-functions.md b/docs/fr/sql-reference/functions/hash-functions.md index 2545b3cf336..3b0f92dd4f8 100644 --- a/docs/fr/sql-reference/functions/hash-functions.md +++ b/docs/fr/sql-reference/functions/hash-functions.md @@ -317,7 +317,7 @@ Calcule JumpConsistentHash forme un UInt64. Accepte deux arguments: une clé de type UInt64 et le nombre de compartiments. Renvoie Int32. Pour plus d'informations, voir le lien: [JumpConsistentHash](https://arxiv.org/pdf/1406.2294.pdf) -## murmurHash2\_32, murmurHash2\_64 {#murmurhash2-32-murmurhash2-64} +## murmurHash2_32, murmurHash2_64 {#murmurhash2-32-murmurhash2-64} Produit un [MurmurHash2](https://github.com/aappleby/smhasher) la valeur de hachage. @@ -385,7 +385,7 @@ Résultat: └──────────────────────┴─────────────────────┘ ``` -## murmurHash3\_32, murmurHash3\_64 {#murmurhash3-32-murmurhash3-64} +## murmurHash3_32, murmurHash3_64 {#murmurhash3-32-murmurhash3-64} Produit un [MurmurHash3](https://github.com/aappleby/smhasher) la valeur de hachage. @@ -415,7 +415,7 @@ SELECT murmurHash3_32(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23: └─────────────┴────────┘ ``` -## murmurHash3\_128 {#murmurhash3-128} +## murmurHash3_128 {#murmurhash3-128} Produit de 128 bits [MurmurHash3](https://github.com/aappleby/smhasher) la valeur de hachage. diff --git a/docs/fr/sql-reference/functions/introspection.md b/docs/fr/sql-reference/functions/introspection.md index a5b9190cd2b..91299217dc7 100644 --- a/docs/fr/sql-reference/functions/introspection.md +++ b/docs/fr/sql-reference/functions/introspection.md @@ -16,11 +16,11 @@ Pour le bon fonctionnement des fonctions d'introspection: - Installer le `clickhouse-common-static-dbg` paquet. -- Définir le [allow\_introspection\_functions](../../operations/settings/settings.md#settings-allow_introspection_functions) réglage sur 1. +- Définir le [allow_introspection_functions](../../operations/settings/settings.md#settings-allow_introspection_functions) réglage sur 1. For security reasons introspection functions are disabled by default. -Clickhouse enregistre les rapports du profileur [trace\_log](../../operations/system-tables.md#system_tables-trace_log) système de table. Assurez-vous que la table et le profileur sont correctement configurés. +Clickhouse enregistre les rapports du profileur [trace_log](../../operations/system-tables.md#system_tables-trace_log) système de table. Assurez-vous que la table et le profileur sont correctement configurés. ## addressToLine {#addresstoline} diff --git a/docs/fr/sql-reference/functions/json-functions.md b/docs/fr/sql-reference/functions/json-functions.md index 37940d53f0d..5f92c99d0f5 100644 --- a/docs/fr/sql-reference/functions/json-functions.md +++ b/docs/fr/sql-reference/functions/json-functions.md @@ -75,7 +75,7 @@ SELECT isValidJSON('{"a": "hello", "b": [-100, 200.0, 300]}') = 1 SELECT isValidJSON('not a json') = 0 ``` -## JSONHas(json\[, indices\_or\_keys\]…) {#jsonhasjson-indices-or-keys} +## JSONHas(json\[, indices_or_keys\]…) {#jsonhasjson-indices-or-keys} Si la valeur existe dans le document JSON, `1` sera retourné. @@ -108,7 +108,7 @@ SELECT JSONExtractKey('{"a": "hello", "b": [-100, 200.0, 300]}', -2) = 'a' SELECT JSONExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 1) = 'hello' ``` -## JSONLength(json\[, indices\_or\_keys\]…) {#jsonlengthjson-indices-or-keys} +## JSONLength(json\[, indices_or_keys\]…) {#jsonlengthjson-indices-or-keys} Renvoie la longueur D'un tableau JSON ou d'un objet JSON. @@ -121,7 +121,7 @@ SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 3 SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}') = 2 ``` -## JSONType(json\[, indices\_or\_keys\]…) {#jsontypejson-indices-or-keys} +## JSONType(json\[, indices_or_keys\]…) {#jsontypejson-indices-or-keys} De retour le type d'une valeur JSON. @@ -135,13 +135,13 @@ SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'a') = 'String' SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 'Array' ``` -## JSONExtractUInt(json\[, indices\_or\_keys\]…) {#jsonextractuintjson-indices-or-keys} +## JSONExtractUInt(json\[, indices_or_keys\]…) {#jsonextractuintjson-indices-or-keys} -## JSONExtractInt(json\[, indices\_or\_keys\]…) {#jsonextractintjson-indices-or-keys} +## JSONExtractInt(json\[, indices_or_keys\]…) {#jsonextractintjson-indices-or-keys} -## JSONExtractFloat(json\[, indices\_or\_keys\]…) {#jsonextractfloatjson-indices-or-keys} +## JSONExtractFloat(json\[, indices_or_keys\]…) {#jsonextractfloatjson-indices-or-keys} -## JSONExtractBool(json\[, indices\_or\_keys\]…) {#jsonextractbooljson-indices-or-keys} +## JSONExtractBool(json\[, indices_or_keys\]…) {#jsonextractbooljson-indices-or-keys} Analyse un JSON et extrait une valeur. Ces fonctions sont similaires à `visitParam` fonction. @@ -155,7 +155,7 @@ SELECT JSONExtractFloat('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 2) = 200 SELECT JSONExtractUInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', -1) = 300 ``` -## JSONExtractString(json\[, indices\_or\_keys\]…) {#jsonextractstringjson-indices-or-keys} +## JSONExtractString(json\[, indices_or_keys\]…) {#jsonextractstringjson-indices-or-keys} Analyse un JSON et extrait une chaîne. Cette fonction est similaire à `visitParamExtractString` fonction. @@ -173,7 +173,7 @@ SELECT JSONExtractString('{"abc":"\\u263"}', 'abc') = '' SELECT JSONExtractString('{"abc":"hello}', 'abc') = '' ``` -## JSONExtract(json\[, indices\_or\_keys…\], Return\_type) {#jsonextractjson-indices-or-keys-return-type} +## JSONExtract(json\[, indices_or_keys…\], Return_type) {#jsonextractjson-indices-or-keys-return-type} Analyse un JSON et extrait une valeur du type de données clickhouse donné. @@ -194,7 +194,7 @@ SELECT JSONExtract('{"day": "Thursday"}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday SELECT JSONExtract('{"day": 5}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \'Tuesday\' = 2, \'Wednesday\' = 3, \'Thursday\' = 4, \'Friday\' = 5, \'Saturday\' = 6)') = 'Friday' ``` -## JSONExtractKeysAndValues(json\[, indices\_or\_keys…\], Value\_type) {#jsonextractkeysandvaluesjson-indices-or-keys-value-type} +## JSONExtractKeysAndValues(json\[, indices_or_keys…\], Value_type) {#jsonextractkeysandvaluesjson-indices-or-keys-value-type} Analyse les paires clé-valeur à partir D'un JSON où les valeurs sont du type de données clickhouse donné. @@ -204,7 +204,7 @@ Exemple: SELECT JSONExtractKeysAndValues('{"x": {"a": 5, "b": 7, "c": 11}}', 'x', 'Int8') = [('a',5),('b',7),('c',11)] ``` -## JSONExtractRaw(json\[, indices\_or\_keys\]…) {#jsonextractrawjson-indices-or-keys} +## JSONExtractRaw(json\[, indices_or_keys\]…) {#jsonextractrawjson-indices-or-keys} Renvoie une partie de JSON en tant que chaîne non analysée. @@ -216,7 +216,7 @@ Exemple: SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = '[-100, 200.0, 300]' ``` -## JSONExtractArrayRaw(json\[, indices\_or\_keys…\]) {#jsonextractarrayrawjson-indices-or-keys} +## JSONExtractArrayRaw(json\[, indices_or_keys…\]) {#jsonextractarrayrawjson-indices-or-keys} Retourne un tableau avec des éléments de tableau JSON, chacun représenté comme une chaîne non analysée. diff --git a/docs/fr/sql-reference/functions/other-functions.md b/docs/fr/sql-reference/functions/other-functions.md index 410b0ba38ce..e5c6abedd75 100644 --- a/docs/fr/sql-reference/functions/other-functions.md +++ b/docs/fr/sql-reference/functions/other-functions.md @@ -419,7 +419,7 @@ ORDER BY h ASC Transforme une valeur en fonction explicitement définis cartographie de certains éléments à l'autre. Il existe deux variantes de cette fonction: -### de transformation(x, array\_from, array\_to, par défaut) {#transformx-array-from-array-to-default} +### de transformation(x, array_from, array_to, par défaut) {#transformx-array-from-array-to-default} `x` – What to transform. @@ -439,7 +439,7 @@ Type: Lorsque la même lettre est indiquée (T ou U), pour les types numériques, il se peut qu'il ne s'agisse pas de types correspondants, mais de types ayant un type commun. Par exemple, le premier argument peut avoir le type Int64, tandis que le second a le type Array(UInt16). -Si l' ‘x’ la valeur est égale à l'un des éléments dans la ‘array\_from’ tableau, elle renvoie l'élément existant (qui est numéroté de même) de la ‘array\_to’ tableau. Sinon, elle renvoie ‘default’. S'il y a plusieurs éléments correspondants dans ‘array\_from’ il renvoie l'un des matches. +Si l' ‘x’ la valeur est égale à l'un des éléments dans la ‘array_from’ tableau, elle renvoie l'élément existant (qui est numéroté de même) de la ‘array_to’ tableau. Sinon, elle renvoie ‘default’. S'il y a plusieurs éléments correspondants dans ‘array_from’ il renvoie l'un des matches. Exemple: @@ -461,10 +461,10 @@ ORDER BY c DESC └───────────┴────────┘ ``` -### de transformation(x, array\_from, array\_to) {#transformx-array-from-array-to} +### de transformation(x, array_from, array_to) {#transformx-array-from-array-to} Diffère de la première variation en ce que le ‘default’ l'argument est omis. -Si l' ‘x’ la valeur est égale à l'un des éléments dans la ‘array\_from’ tableau, elle renvoie l'élément correspondant (qui est numéroté de même) de la ‘array\_to’ tableau. Sinon, elle renvoie ‘x’. +Si l' ‘x’ la valeur est égale à l'un des éléments dans la ‘array_from’ tableau, elle renvoie l'élément correspondant (qui est numéroté de même) de la ‘array_to’ tableau. Sinon, elle renvoie ‘x’. Type: @@ -1082,7 +1082,7 @@ joinGet(join_storage_table_name, `value_column`, join_keys) Retourne la liste des valeurs correspond à la liste des clés. -Si certain n'existe pas dans la table source alors `0` ou `null` seront renvoyés basé sur [join\_use\_nulls](../../operations/settings/settings.md#join_use_nulls) paramètre. +Si certain n'existe pas dans la table source alors `0` ou `null` seront renvoyés basé sur [join_use_nulls](../../operations/settings/settings.md#join_use_nulls) paramètre. Plus d'infos sur `join_use_nulls` dans [Opération de jointure](../../engines/table-engines/special/join.md). @@ -1121,15 +1121,15 @@ Résultat: └──────────────────────────────────────────────────┘ ``` -## modelEvaluate(model\_name, …) {#function-modelevaluate} +## modelEvaluate(model_name, …) {#function-modelevaluate} Évaluer le modèle externe. Accepte un nom de modèle et le modèle de l'argumentation. Renvoie Float64. -## throwIf (x \[, custom\_message\]) {#throwifx-custom-message} +## throwIf (x \[, custom_message\]) {#throwifx-custom-message} Lever une exception si l'argument est non nul. -custom\_message - est un paramètre optionnel: une chaîne constante, fournit un message d'erreur +custom_message - est un paramètre optionnel: une chaîne constante, fournit un message d'erreur ``` sql SELECT throwIf(number = 3, 'Too many') FROM numbers(10); diff --git a/docs/fr/sql-reference/functions/string-functions.md b/docs/fr/sql-reference/functions/string-functions.md index bc7e10a8165..1482952426c 100644 --- a/docs/fr/sql-reference/functions/string-functions.md +++ b/docs/fr/sql-reference/functions/string-functions.md @@ -31,12 +31,12 @@ La fonction fonctionne également pour les tableaux. Renvoie la longueur d'une chaîne en points de code Unicode (pas en caractères), en supposant que la chaîne contient un ensemble d'octets qui composent le texte codé en UTF-8. Si cette hypothèse n'est pas remplie, elle renvoie un résultat (elle ne lance pas d'exception). Le type de résultat est UInt64. -## char\_length, CHAR\_LENGTH {#char-length} +## char_length, CHAR_LENGTH {#char-length} Renvoie la longueur d'une chaîne en points de code Unicode (pas en caractères), en supposant que la chaîne contient un ensemble d'octets qui composent le texte codé en UTF-8. Si cette hypothèse n'est pas remplie, elle renvoie un résultat (elle ne lance pas d'exception). Le type de résultat est UInt64. -## character\_length, CHARACTER\_LENGTH {#character-length} +## character_length, CHARACTER_LENGTH {#character-length} Renvoie la longueur d'une chaîne en points de code Unicode (pas en caractères), en supposant que la chaîne contient un ensemble d'octets qui composent le texte codé en UTF-8. Si cette hypothèse n'est pas remplie, elle renvoie un résultat (elle ne lance pas d'exception). Le type de résultat est UInt64. @@ -77,7 +77,7 @@ toValidUTF8( input_string ) Paramètre: -- input\_string — Any set of bytes represented as the [Chaîne](../../sql-reference/data-types/string.md) type de données objet. +- input_string — Any set of bytes represented as the [Chaîne](../../sql-reference/data-types/string.md) type de données objet. Valeur renvoyée: chaîne UTF-8 valide. diff --git a/docs/fr/sql-reference/functions/type-conversion-functions.md b/docs/fr/sql-reference/functions/type-conversion-functions.md index 44673df0fa8..c17b24c69dc 100644 --- a/docs/fr/sql-reference/functions/type-conversion-functions.md +++ b/docs/fr/sql-reference/functions/type-conversion-functions.md @@ -246,7 +246,7 @@ YYYY-MM-DD YYYY-MM-DD hh:mm:ss ``` -À titre d'exception, si vous convertissez des types numériques UInt32, Int32, UInt64 ou Int64 à Date, et si le nombre est supérieur ou égal à 65536, le nombre est interprété comme un horodatage Unix (et non comme le nombre de jours) et est arrondi à la date. Cela permet de prendre en charge l'occurrence commune de l'écriture ‘toDate(unix\_timestamp)’, qui autrement serait une erreur et nécessiterait d'écrire le plus lourd ‘toDate(toDateTime(unix\_timestamp))’. +À titre d'exception, si vous convertissez des types numériques UInt32, Int32, UInt64 ou Int64 à Date, et si le nombre est supérieur ou égal à 65536, le nombre est interprété comme un horodatage Unix (et non comme le nombre de jours) et est arrondi à la date. Cela permet de prendre en charge l'occurrence commune de l'écriture ‘toDate(unix_timestamp)’, qui autrement serait une erreur et nécessiterait d'écrire le plus lourd ‘toDate(toDateTime(unix_timestamp))’. La Conversion entre une date et une date avec l'heure est effectuée de manière naturelle: en ajoutant une heure nulle ou en supprimant l'heure. diff --git a/docs/fr/sql-reference/functions/url-functions.md b/docs/fr/sql-reference/functions/url-functions.md index dce3aaaf3bb..2bb2203a10b 100644 --- a/docs/fr/sql-reference/functions/url-functions.md +++ b/docs/fr/sql-reference/functions/url-functions.md @@ -125,11 +125,11 @@ Retourne le chemin d'accès. Exemple: `/top/news.html` Le chemin n'inclut pas la ### pathFull {#pathfull} -La même chose que ci-dessus, mais y compris la chaîne de requête et le fragment. Exemple: / top / nouvelles.le html?page = 2 \# commentaires +La même chose que ci-dessus, mais y compris la chaîne de requête et le fragment. Exemple: / top / nouvelles.le html?page = 2 # commentaires ### queryString {#querystring} -Retourne la chaîne de requête. Exemple: page = 1 & lr=213. query-string n'inclut pas le point d'interrogation initial, ainsi que \# et tout ce qui suit \#. +Retourne la chaîne de requête. Exemple: page = 1 & lr=213. query-string n'inclut pas le point d'interrogation initial, ainsi que # et tout ce qui suit #. ### fragment {#fragment} @@ -137,7 +137,7 @@ Renvoie l'identificateur de fragment. fragment n'inclut pas le symbole de hachag ### queryStringAndFragment {#querystringandfragment} -Renvoie la chaîne de requête et l'Identificateur de fragment. Exemple: page = 1 \# 29390. +Renvoie la chaîne de requête et l'Identificateur de fragment. Exemple: page = 1 # 29390. ### extractURLParameter (URL, nom) {#extracturlparameterurl-name} diff --git a/docs/fr/sql-reference/functions/ym-dict-functions.md b/docs/fr/sql-reference/functions/ym-dict-functions.md index 73e5a9129ae..f1e4461e24a 100644 --- a/docs/fr/sql-reference/functions/ym-dict-functions.md +++ b/docs/fr/sql-reference/functions/ym-dict-functions.md @@ -17,12 +17,12 @@ ClickHouse soutient le travail avec plusieurs géobases alternatives (hiérarchi Le ‘clickhouse-server’ config spécifie le fichier avec l'échelon régional::`/opt/geo/regions_hierarchy.txt` -Outre ce fichier, il recherche également les fichiers à proximité qui ont le symbole \_ et tout suffixe ajouté au nom (avant l'extension de fichier). +Outre ce fichier, il recherche également les fichiers à proximité qui ont le symbole _ et tout suffixe ajouté au nom (avant l'extension de fichier). Par exemple, il trouvera également le fichier `/opt/geo/regions_hierarchy_ua.txt` si présente. `ua` est appelée la clé du dictionnaire. Pour un dictionnaire sans suffixe, la clé est une chaîne vide. -Tous les dictionnaires sont rechargés dans l'exécution (une fois toutes les secondes, comme défini dans le paramètre de configuration builtin\_dictionaries\_reload\_interval, ou une fois par heure par défaut). Cependant, la liste des dictionnaires disponibles est définie une fois, lorsque le serveur démarre. +Tous les dictionnaires sont rechargés dans l'exécution (une fois toutes les secondes, comme défini dans le paramètre de configuration builtin_dictionaries_reload_interval, ou une fois par heure par défaut). Cependant, la liste des dictionnaires disponibles est définie une fois, lorsque le serveur démarre. All functions for working with regions have an optional argument at the end – the dictionary key. It is referred to as the geobase. Exemple: @@ -107,7 +107,7 @@ Exemple: `regionToCountry(toUInt32(213)) = 225` convertit Moscou (213) en Russie Convertit une région en continent. Dans tous les autres cas, cette fonction est la même que ‘regionToCity’. Exemple: `regionToContinent(toUInt32(213)) = 10001` convertit Moscou (213) en Eurasie (10001). -### regionToTopContinent (\#regiontotopcontinent) {#regiontotopcontinent-regiontotopcontinent} +### regionToTopContinent (#regiontotopcontinent) {#regiontotopcontinent-regiontotopcontinent} Trouve le continent le plus élevé dans la hiérarchie de la région. diff --git a/docs/fr/sql-reference/operators/in.md b/docs/fr/sql-reference/operators/in.md index be4f7ea717f..d87fe41a04f 100644 --- a/docs/fr/sql-reference/operators/in.md +++ b/docs/fr/sql-reference/operators/in.md @@ -117,9 +117,9 @@ Pour une requête non distribuée, utilisez `IN` / `JOIN`. Soyez prudent lorsque vous utilisez des sous-requêtes dans le `IN` / `JOIN` clauses pour le traitement des requêtes distribuées. -Regardons quelques exemples. Supposons que chaque serveur du cluster a un **local\_table**. Chaque serveur dispose également d'une **table distributed\_table** table avec le **Distribué** type, qui regarde tous les serveurs du cluster. +Regardons quelques exemples. Supposons que chaque serveur du cluster a un **local_table**. Chaque serveur dispose également d'une **table distributed_table** table avec le **Distribué** type, qui regarde tous les serveurs du cluster. -Pour une requête à l' **table distributed\_table**, la requête sera envoyée à tous les serveurs distants et exécutée sur eux en utilisant le **local\_table**. +Pour une requête à l' **table distributed_table**, la requête sera envoyée à tous les serveurs distants et exécutée sur eux en utilisant le **local_table**. Par exemple, la requête @@ -153,7 +153,7 @@ En d'autres termes, l'ensemble de données de la clause IN sera collecté sur ch Cela fonctionnera correctement et de manière optimale si vous êtes prêt pour ce cas et que vous avez réparti les données entre les serveurs de cluster de telle sorte que les données d'un seul ID utilisateur résident entièrement sur un seul serveur. Dans ce cas, toutes les données nécessaires seront disponibles localement sur chaque serveur. Sinon, le résultat sera erroné. Nous nous référons à cette variation de la requête que “local IN”. -Pour corriger le fonctionnement de la requête lorsque les données sont réparties aléatoirement sur les serveurs de cluster, vous pouvez spécifier **table distributed\_table** à l'intérieur d'une sous-requête. La requête ressemblerait à ceci: +Pour corriger le fonctionnement de la requête lorsque les données sont réparties aléatoirement sur les serveurs de cluster, vous pouvez spécifier **table distributed_table** à l'intérieur d'une sous-requête. La requête ressemblerait à ceci: ``` sql SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM distributed_table WHERE CounterID = 34) diff --git a/docs/fr/sql-reference/statements/alter.md b/docs/fr/sql-reference/statements/alter.md index 298df06304a..64fe21046a3 100644 --- a/docs/fr/sql-reference/statements/alter.md +++ b/docs/fr/sql-reference/statements/alter.md @@ -208,7 +208,7 @@ Les opérations suivantes avec [partition](../../engines/table-engines/mergetree - [ATTACH PART\|PARTITION](#alter_attach-partition) – Adds a part or partition from the `detached` répertoire à la table. - [ATTACH PARTITION FROM](#alter_attach-partition-from) – Copies the data partition from one table to another and adds. - [REPLACE PARTITION](#alter_replace-partition) - Copie la partition de données d'une table à l'autre et la remplace. -- [MOVE PARTITION TO TABLE](#alter_move_to_table-partition)(\#alter\_move\_to\_table-partition) - déplace la partition de données d'une table à l'autre. +- [MOVE PARTITION TO TABLE](#alter_move_to_table-partition)(#alter_move_to_table-partition) - déplace la partition de données d'une table à l'autre. - [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) - Rétablit la valeur d'une colonne spécifiée dans une partition. - [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) - Réinitialise l'index secondaire spécifié dans une partition. - [FREEZE PARTITION](#alter_freeze-partition) – Creates a backup of a partition. @@ -436,7 +436,7 @@ Vous pouvez spécifier l'expression de partition dans `ALTER ... PARTITION` requ - Comme une valeur de l' `partition` la colonne de la `system.parts` table. Exemple, `ALTER TABLE visits DETACH PARTITION 201901`. - Comme expression de la colonne de la table. Les constantes et les expressions constantes sont prises en charge. Exemple, `ALTER TABLE visits DETACH PARTITION toYYYYMM(toDate('2019-01-25'))`. - À l'aide de l'ID de partition. Partition ID est un identifiant de chaîne de la partition (lisible par l'homme, si possible) qui est utilisé comme noms de partitions dans le système de fichiers et dans ZooKeeper. L'ID de partition doit être spécifié dans `PARTITION ID` clause, entre guillemets simples. Exemple, `ALTER TABLE visits DETACH PARTITION ID '201901'`. -- Dans le [ALTER ATTACH PART](#alter_attach-partition) et [DROP DETACHED PART](#alter_drop-detached) requête, pour spécifier le nom d'une partie, utilisez le littéral de chaîne avec une valeur de `name` la colonne de la [système.detached\_parts](../../operations/system-tables.md#system_tables-detached_parts) table. Exemple, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. +- Dans le [ALTER ATTACH PART](#alter_attach-partition) et [DROP DETACHED PART](#alter_drop-detached) requête, pour spécifier le nom d'une partie, utilisez le littéral de chaîne avec une valeur de `name` la colonne de la [système.detached_parts](../../operations/system-tables.md#system_tables-detached_parts) table. Exemple, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. L'utilisation de guillemets lors de la spécification de la partition dépend du type d'expression de partition. Par exemple, pour la `String` type, vous devez spécifier son nom entre guillemets (`'`). Pour l' `Date` et `Int*` types aucune citation n'est nécessaire. diff --git a/docs/fr/sql-reference/statements/insert-into.md b/docs/fr/sql-reference/statements/insert-into.md index 95026d142e3..987594bae65 100644 --- a/docs/fr/sql-reference/statements/insert-into.md +++ b/docs/fr/sql-reference/statements/insert-into.md @@ -20,7 +20,7 @@ La requête peut spécifier une liste de colonnes à insérer `[(c1, c2, c3)]`. - Les valeurs calculées à partir `DEFAULT` expressions spécifiées dans la définition de la table. - Zéros et chaînes vides, si `DEFAULT` les expressions ne sont pas définies. -Si [strict\_insert\_defaults=1](../../operations/settings/settings.md), les colonnes qui n'ont pas `DEFAULT` défini doit être répertorié dans la requête. +Si [strict_insert_defaults=1](../../operations/settings/settings.md), les colonnes qui n'ont pas `DEFAULT` défini doit être répertorié dans la requête. Les données peuvent être transmises à L'INSERT dans n'importe quel [format](../../interfaces/formats.md#formats) soutenu par ClickHouse. Le format doit être spécifié explicitement dans la requête: diff --git a/docs/fr/sql-reference/statements/misc.md b/docs/fr/sql-reference/statements/misc.md index bd8375c3ec0..4631f856266 100644 --- a/docs/fr/sql-reference/statements/misc.md +++ b/docs/fr/sql-reference/statements/misc.md @@ -57,7 +57,7 @@ Pour `MergeTree` moteurs de la famille, le `CHECK TABLE` query affiche un État Si la table est corrompue, vous pouvez copier les données non corrompues dans une autre table. Pour ce faire: 1. Créez une nouvelle table avec la même structure que la table endommagée. Pour ce faire exécutez la requête `CREATE TABLE AS `. -2. Définir le [max\_threads](../../operations/settings/settings.md#settings-max_threads) la valeur 1 pour traiter la requête suivante dans un seul thread. Pour ce faire, exécutez la requête `SET max_threads = 1`. +2. Définir le [max_threads](../../operations/settings/settings.md#settings-max_threads) la valeur 1 pour traiter la requête suivante dans un seul thread. Pour ce faire, exécutez la requête `SET max_threads = 1`. 3. Exécuter la requête `INSERT INTO SELECT * FROM `. Cette demande copie les données non corrompues de la table endommagée vers une autre table. Seules les données avant la partie corrompue seront copiées. 4. Redémarrez l' `clickhouse-client` pour réinitialiser l' `max_threads` valeur. @@ -253,7 +253,7 @@ Le `OPTMIZE` la requête est également prise en charge pour [MaterializedView]( Lorsque `OPTIMIZE` est utilisé avec le [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/replication.md) famille de moteurs de table, ClickHouse crée une tâche pour la fusion et attend l'exécution sur tous les nœuds (si le `replication_alter_partitions_sync` paramètre est activé). -- Si `OPTIMIZE` n'effectue pas de fusion pour une raison quelconque, il ne notifie pas le client. Pour activer les notifications, utilisez [optimize\_throw\_if\_noop](../../operations/settings/settings.md#setting-optimize_throw_if_noop) paramètre. +- Si `OPTIMIZE` n'effectue pas de fusion pour une raison quelconque, il ne notifie pas le client. Pour activer les notifications, utilisez [optimize_throw_if_noop](../../operations/settings/settings.md#setting-optimize_throw_if_noop) paramètre. - Si vous spécifiez un `PARTITION`, seule la partition spécifiée est optimisé. [Comment définir l'expression de la partition](alter.md#alter-how-to-specify-part-expr). - Si vous spécifiez `FINAL`, l'optimisation est effectuée, même lorsque toutes les données sont déjà dans une partie. - Si vous spécifiez `DEDUPLICATE`, alors des lignes complètement identiques seront dédupliquées (toutes les colonnes sont comparées), cela n'a de sens que pour le moteur MergeTree. diff --git a/docs/fr/sql-reference/statements/select/group-by.md b/docs/fr/sql-reference/statements/select/group-by.md index 059e313fed7..9d1b5c276d5 100644 --- a/docs/fr/sql-reference/statements/select/group-by.md +++ b/docs/fr/sql-reference/statements/select/group-by.md @@ -57,7 +57,7 @@ Cette ligne supplémentaire est uniquement produite en `JSON*`, `TabSeparated*`, - Dans `Pretty*` formats, la ligne est sortie comme une table séparée après le résultat principal. - Dans les autres formats, il n'est pas disponible. -`WITH TOTALS` peut être exécuté de différentes manières lorsqu'il est présent. Le comportement dépend de l' ‘totals\_mode’ paramètre. +`WITH TOTALS` peut être exécuté de différentes manières lorsqu'il est présent. Le comportement dépend de l' ‘totals_mode’ paramètre. ### Configuration Du Traitement Des Totaux {#configuring-totals-processing} @@ -67,9 +67,9 @@ Les autres alternatives incluent uniquement les lignes qui passent à travers av `after_having_exclusive` – Don't include rows that didn't pass through `max_rows_to_group_by`. En d'autres termes, ‘totals’ aura moins ou le même nombre de lignes que si `max_rows_to_group_by` ont été omis. -`after_having_inclusive` – Include all the rows that didn't pass through ‘max\_rows\_to\_group\_by’ dans ‘totals’. En d'autres termes, ‘totals’ aura plus ou le même nombre de lignes que si `max_rows_to_group_by` ont été omis. +`after_having_inclusive` – Include all the rows that didn't pass through ‘max_rows_to_group_by’ dans ‘totals’. En d'autres termes, ‘totals’ aura plus ou le même nombre de lignes que si `max_rows_to_group_by` ont été omis. -`after_having_auto` – Count the number of rows that passed through HAVING. If it is more than a certain amount (by default, 50%), include all the rows that didn't pass through ‘max\_rows\_to\_group\_by’ dans ‘totals’. Sinon, ne pas les inclure. +`after_having_auto` – Count the number of rows that passed through HAVING. If it is more than a certain amount (by default, 50%), include all the rows that didn't pass through ‘max_rows_to_group_by’ dans ‘totals’. Sinon, ne pas les inclure. `totals_auto_threshold` – By default, 0.5. The coefficient for `after_having_auto`. @@ -117,11 +117,11 @@ L'agrégation est l'une des caractéristiques les plus importantes d'un SGBD ori ### Groupe par dans la mémoire externe {#select-group-by-in-external-memory} Vous pouvez activer le dumping des données temporaires sur le disque pour limiter l'utilisation de la mémoire pendant `GROUP BY`. -Le [max\_bytes\_before\_external\_group\_by](../../../operations/settings/settings.md#settings-max_bytes_before_external_group_by) réglage détermine le seuil de consommation de RAM pour le dumping `GROUP BY` données temporaires dans le système de fichiers. Si elle est définie sur 0 (valeur par défaut), elle est désactivée. +Le [max_bytes_before_external_group_by](../../../operations/settings/settings.md#settings-max_bytes_before_external_group_by) réglage détermine le seuil de consommation de RAM pour le dumping `GROUP BY` données temporaires dans le système de fichiers. Si elle est définie sur 0 (valeur par défaut), elle est désactivée. Lors de l'utilisation de `max_bytes_before_external_group_by`, nous vous recommandons de définir `max_memory_usage` environ deux fois plus élevé. Ceci est nécessaire car il y a deux étapes à l'agrégation: la lecture des données et la formation des données intermédiaires (1) et la fusion des données intermédiaires (2). Le Dumping des données dans le système de fichiers ne peut se produire qu'au cours de l'étape 1. Si les données temporaires n'ont pas été vidées, l'étape 2 peut nécessiter jusqu'à la même quantité de mémoire qu'à l'étape 1. -Par exemple, si [max\_memory\_usage](../../../operations/settings/settings.md#settings_max_memory_usage) a été défini sur 10000000000 et que vous souhaitez utiliser l'agrégation externe, il est logique de définir `max_bytes_before_external_group_by` à 10000000000, et `max_memory_usage` à 20000000000. Lorsque l'agrégation externe est déclenchée (s'il y a eu au moins un vidage de données temporaires), la consommation maximale de RAM n'est que légèrement supérieure à `max_bytes_before_external_group_by`. +Par exemple, si [max_memory_usage](../../../operations/settings/settings.md#settings_max_memory_usage) a été défini sur 10000000000 et que vous souhaitez utiliser l'agrégation externe, il est logique de définir `max_bytes_before_external_group_by` à 10000000000, et `max_memory_usage` à 20000000000. Lorsque l'agrégation externe est déclenchée (s'il y a eu au moins un vidage de données temporaires), la consommation maximale de RAM n'est que légèrement supérieure à `max_bytes_before_external_group_by`. Avec le traitement des requêtes distribuées, l'agrégation externe est effectuée sur des serveurs distants. Pour que le serveur demandeur n'utilise qu'une petite quantité de RAM, définissez `distributed_aggregation_memory_efficient` 1. diff --git a/docs/fr/sql-reference/statements/select/join.md b/docs/fr/sql-reference/statements/select/join.md index 335086349d3..4233a120674 100644 --- a/docs/fr/sql-reference/statements/select/join.md +++ b/docs/fr/sql-reference/statements/select/join.md @@ -40,7 +40,7 @@ Autres types de jointure disponibles dans ClickHouse: ## Setting {#join-settings} !!! note "Note" - La valeur de rigueur par défaut peut être remplacée à l'aide [join\_default\_strictness](../../../operations/settings/settings.md#settings-join_default_strictness) paramètre. + La valeur de rigueur par défaut peut être remplacée à l'aide [join_default_strictness](../../../operations/settings/settings.md#settings-join_default_strictness) paramètre. ### ASOF joindre L'utilisation {#asof-join-usage} @@ -101,7 +101,7 @@ Soyez prudent lorsque vous utilisez `GLOBAL`. Pour plus d'informations, voir le ### Traitement des cellules vides ou nulles {#processing-of-empty-or-null-cells} -Lors de la jonction de tables, les cellules vides peuvent apparaître. Paramètre [join\_use\_nulls](../../../operations/settings/settings.md#join_use_nulls) définir comment clickhouse remplit ces cellules. +Lors de la jonction de tables, les cellules vides peuvent apparaître. Paramètre [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) définir comment clickhouse remplit ces cellules. Si l' `JOIN` les touches sont [Nullable](../../data-types/nullable.md) champs, les lignes où au moins une des clés a la valeur [NULL](../../../sql-reference/syntax.md#null-literal) ne sont pas jointes. @@ -138,10 +138,10 @@ Par défaut, ClickHouse utilise [jointure de hachage](https://en.wikipedia.org/w Si vous devez restreindre la consommation de mémoire de l'opération join utilisez les paramètres suivants: -- [max\_rows\_in\_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) — Limits number of rows in the hash table. -- [max\_bytes\_in\_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) — Limits size of the hash table. +- [max_rows_in_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) — Limits number of rows in the hash table. +- [max_bytes_in_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) — Limits size of the hash table. -Lorsque l'une de ces limites est atteinte, ClickHouse agit comme [join\_overflow\_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) réglage des instructions. +Lorsque l'une de ces limites est atteinte, ClickHouse agit comme [join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) réglage des instructions. ## Exemple {#examples} diff --git a/docs/fr/sql-reference/statements/system.md b/docs/fr/sql-reference/statements/system.md index 6b957eb9695..e8c9ed85cbc 100644 --- a/docs/fr/sql-reference/statements/system.md +++ b/docs/fr/sql-reference/statements/system.md @@ -24,12 +24,12 @@ toc_title: SYSTEM ## RELOAD DICTIONARIES {#query_language-system-reload-dictionaries} Recharge tous les dictionnaires qui ont déjà été chargés avec succès. -Par défaut, les dictionnaires sont chargés paresseusement (voir [dictionaries\_lazy\_load](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load)), donc au lieu d'être chargés automatiquement au démarrage, ils sont initialisés lors du premier accès via la fonction dictGet ou sélectionnez dans les tables avec ENGINE = Dictionary . Le `SYSTEM RELOAD DICTIONARIES` query recharge ces dictionnaires (chargés). +Par défaut, les dictionnaires sont chargés paresseusement (voir [dictionaries_lazy_load](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load)), donc au lieu d'être chargés automatiquement au démarrage, ils sont initialisés lors du premier accès via la fonction dictGet ou sélectionnez dans les tables avec ENGINE = Dictionary . Le `SYSTEM RELOAD DICTIONARIES` query recharge ces dictionnaires (chargés). Retourne toujours `Ok.` quel que soit le résultat de la mise à jour du dictionnaire. -## Recharger le dictionnaire Dictionary\_name {#query_language-system-reload-dictionary} +## Recharger le dictionnaire Dictionary_name {#query_language-system-reload-dictionary} -Recharge complètement un dictionnaire `dictionary_name`, quel que soit l'état du dictionnaire (LOADED / NOT\_LOADED / FAILED). +Recharge complètement un dictionnaire `dictionary_name`, quel que soit l'état du dictionnaire (LOADED / NOT_LOADED / FAILED). Retourne toujours `Ok.` quel que soit le résultat de la mise à jour du dictionnaire. L'état du dictionnaire peut être vérifié en interrogeant le `system.dictionaries` table. @@ -41,7 +41,7 @@ SELECT name, status FROM system.dictionaries; Réinitialise le cache DNS interne de ClickHouse. Parfois (pour les anciennes versions de ClickHouse), il est nécessaire d'utiliser cette commande lors de la modification de l'infrastructure (modification de l'adresse IP d'un autre serveur ClickHouse ou du serveur utilisé par les dictionnaires). -Pour une gestion du cache plus pratique (automatique), voir paramètres disable\_internal\_dns\_cache, dns\_cache\_update\_period. +Pour une gestion du cache plus pratique (automatique), voir paramètres disable_internal_dns_cache, dns_cache_update_period. ## DROP MARK CACHE {#query_language-system-drop-mark-cache} @@ -49,7 +49,7 @@ Réinitialise le cache de marque. Utilisé dans le développement de ClickHouse ## FLUSH LOGS {#query_language-system-flush_logs} -Flushes buffers of log messages to system tables (e.g. system.query\_log). Allows you to not wait 7.5 seconds when debugging. +Flushes buffers of log messages to system tables (e.g. system.query_log). Allows you to not wait 7.5 seconds when debugging. ## RELOAD CONFIG {#query_language-system-reload-config} diff --git a/docs/fr/sql-reference/syntax.md b/docs/fr/sql-reference/syntax.md index 27a2e231956..b8b24c9bbb5 100644 --- a/docs/fr/sql-reference/syntax.md +++ b/docs/fr/sql-reference/syntax.md @@ -15,9 +15,9 @@ Le `INSERT` requête utilise les deux analyseurs: INSERT INTO t VALUES (1, 'Hello, world'), (2, 'abc'), (3, 'def') ``` -Le `INSERT INTO t VALUES` fragment est analysé par l'analyseur complet, et les données `(1, 'Hello, world'), (2, 'abc'), (3, 'def')` est analysé par l'analyseur de flux rapide. Vous pouvez également activer l'analyseur complet pour les données à l'aide de la [input\_format\_values\_interpret\_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions) paramètre. Lorsque `input_format_values_interpret_expressions = 1`, ClickHouse essaie d'abord d'analyser les valeurs avec l'analyseur de flux rapide. S'il échoue, ClickHouse essaie d'utiliser l'analyseur complet pour les données, en le traitant comme un SQL [expression](#syntax-expressions). +Le `INSERT INTO t VALUES` fragment est analysé par l'analyseur complet, et les données `(1, 'Hello, world'), (2, 'abc'), (3, 'def')` est analysé par l'analyseur de flux rapide. Vous pouvez également activer l'analyseur complet pour les données à l'aide de la [input_format_values_interpret_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions) paramètre. Lorsque `input_format_values_interpret_expressions = 1`, ClickHouse essaie d'abord d'analyser les valeurs avec l'analyseur de flux rapide. S'il échoue, ClickHouse essaie d'utiliser l'analyseur complet pour les données, en le traitant comme un SQL [expression](#syntax-expressions). -Les données peuvent avoir n'importe quel format. Lorsqu'une requête est reçue, le serveur calcule pas plus que [max\_query\_size](../operations/settings/settings.md#settings-max_query_size) octets de la requête en RAM (par défaut, 1 Mo), et le reste est analysé en flux. +Les données peuvent avoir n'importe quel format. Lorsqu'une requête est reçue, le serveur calcule pas plus que [max_query_size](../operations/settings/settings.md#settings-max_query_size) octets de la requête en RAM (par défaut, 1 Mo), et le reste est analysé en flux. Il permet d'éviter les problèmes avec de grandes `INSERT` requête. Lors de l'utilisation de la `Values` format dans un `INSERT` de la requête, il peut sembler que les données sont analysées de même que les expressions dans un `SELECT` requête, mais ce n'est pas vrai. Le `Values` le format est beaucoup plus limitée. diff --git a/docs/fr/sql-reference/table-functions/file.md b/docs/fr/sql-reference/table-functions/file.md index 620ac6b2786..a58821d021d 100644 --- a/docs/fr/sql-reference/table-functions/file.md +++ b/docs/fr/sql-reference/table-functions/file.md @@ -15,7 +15,7 @@ file(path, format, structure) **Les paramètres d'entrée** -- `path` — The relative path to the file from [user\_files\_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Chemin d'accès à la prise en charge des fichiers suivant les globs en mode Lecture seule: `*`, `?`, `{abc,def}` et `{N..M}` où `N`, `M` — numbers, \``'abc', 'def'` — strings. +- `path` — The relative path to the file from [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Chemin d'accès à la prise en charge des fichiers suivant les globs en mode Lecture seule: `*`, `?`, `{abc,def}` et `{N..M}` où `N`, `M` — numbers, \``'abc', 'def'` — strings. - `format` — The [format](../../interfaces/formats.md#formats) de le fichier. - `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. @@ -72,12 +72,12 @@ Les Constructions avec `{}` sont similaires à l' [fonction de table à distance 1. Supposons que nous ayons plusieurs fichiers avec les chemins relatifs suivants: -- ‘some\_dir/some\_file\_1’ -- ‘some\_dir/some\_file\_2’ -- ‘some\_dir/some\_file\_3’ -- ‘another\_dir/some\_file\_1’ -- ‘another\_dir/some\_file\_2’ -- ‘another\_dir/some\_file\_3’ +- ‘some_dir/some_file_1’ +- ‘some_dir/some_file_2’ +- ‘some_dir/some_file_3’ +- ‘another_dir/some_file_1’ +- ‘another_dir/some_file_2’ +- ‘another_dir/some_file_3’ 1. Interroger la quantité de lignes dans ces fichiers: diff --git a/docs/fr/sql-reference/table-functions/hdfs.md b/docs/fr/sql-reference/table-functions/hdfs.md index 67c892e0de9..51b742d8018 100644 --- a/docs/fr/sql-reference/table-functions/hdfs.md +++ b/docs/fr/sql-reference/table-functions/hdfs.md @@ -55,12 +55,12 @@ Les Constructions avec `{}` sont similaires à l' [fonction de table à distance 1. Supposons que nous ayons plusieurs fichiers avec les URI suivants sur HDFS: -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_3’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_3’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_3’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_3’ 1. Interroger la quantité de lignes dans ces fichiers: diff --git a/docs/fr/sql-reference/table-functions/index.md b/docs/fr/sql-reference/table-functions/index.md index ed25e5d6d5d..89a8200e385 100644 --- a/docs/fr/sql-reference/table-functions/index.md +++ b/docs/fr/sql-reference/table-functions/index.md @@ -16,12 +16,12 @@ Vous pouvez utiliser les fonctions de table dans: The method for creating a temporary table that is available only in the current query. The table is deleted when the query finishes. -- [Créer une TABLE en tant que \< table\_function ()\>](../statements/create.md#create-table-query) requête. +- [Créer une TABLE en tant que \< table_function ()\>](../statements/create.md#create-table-query) requête. It's one of the methods of creating a table. !!! warning "Avertissement" - Vous ne pouvez pas utiliser les fonctions de table si [allow\_ddl](../../operations/settings/permissions-for-queries.md#settings_allow_ddl) paramètre est désactivé. + Vous ne pouvez pas utiliser les fonctions de table si [allow_ddl](../../operations/settings/permissions-for-queries.md#settings_allow_ddl) paramètre est désactivé. | Fonction | Description | |-----------------------|-------------------------------------------------------------------------------------------------------------------------------------| diff --git a/docs/fr/sql-reference/table-functions/remote.md b/docs/fr/sql-reference/table-functions/remote.md index 3e911b61d75..380a9986116 100644 --- a/docs/fr/sql-reference/table-functions/remote.md +++ b/docs/fr/sql-reference/table-functions/remote.md @@ -80,6 +80,6 @@ Le `remote` table de fonction peut être utile dans les cas suivants: Si l'utilisateur n'est pas spécifié, `default` est utilisée. Si le mot de passe n'est spécifié, un mot de passe vide est utilisé. -`remoteSecure` - la même chose que `remote` but with secured connection. Default port — [tcp\_port\_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) de config ou 9440. +`remoteSecure` - la même chose que `remote` but with secured connection. Default port — [tcp_port_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) de config ou 9440. [Article Original](https://clickhouse.tech/docs/en/query_language/table_functions/remote/) diff --git a/docs/fr/whats-new/security-changelog.md b/docs/fr/whats-new/security-changelog.md index bef992b7a77..6046ef96bb2 100644 --- a/docs/fr/whats-new/security-changelog.md +++ b/docs/fr/whats-new/security-changelog.md @@ -53,7 +53,7 @@ Crédits: Andrey Krasichkov et Evgeny Sidorov de Yandex Information Security Tea ### CVE-2018-14668 {#cve-2018-14668} -“remote” la fonction de table a permis des symboles arbitraires dans “user”, “password” et “default\_database” champs qui ont conduit à des attaques de falsification de requêtes inter-protocoles. +“remote” la fonction de table a permis des symboles arbitraires dans “user”, “password” et “default_database” champs qui ont conduit à des attaques de falsification de requêtes inter-protocoles. Crédits: Andrey Krasichkov de L'équipe de sécurité de L'Information Yandex diff --git a/docs/ja/development/architecture.md b/docs/ja/development/architecture.md index 3b1da1b8552..8dc1d471007 100644 --- a/docs/ja/development/architecture.md +++ b/docs/ja/development/architecture.md @@ -47,7 +47,7 @@ A `Block` メモリ内のテーブルのサブセット(チャンク)を表 ブロック内の列に対して関数を計算するとき、その結果を含む別の列をブロックに追加します。 後で、不要な列はブロックから削除できますが、変更はできません。 共通の部分式を排除するのに便利です。 -ブロックの作成のための各処理チャンクのデータです。 同じタイプの計算では、列名と型は異なるブロックで同じままであり、列データのみが変更されることに注意してください。 ブロックサイズが小さいと、shared\_ptrsと列名をコピーするための一時的な文字列のオーバーヘッドが高くなるため、ブロックヘッダーからブロックデータを分割 +ブロックの作成のための各処理チャンクのデータです。 同じタイプの計算では、列名と型は異なるブロックで同じままであり、列データのみが変更されることに注意してください。 ブロックサイズが小さいと、shared_ptrsと列名をコピーするための一時的な文字列のオーバーヘッドが高くなるため、ブロックヘッダーからブロックデータを分割 ## ブロックの流れ {#block-streams} diff --git a/docs/ja/development/build.md b/docs/ja/development/build.md index 035d5e7efb1..33709cb65f3 100644 --- a/docs/ja/development/build.md +++ b/docs/ja/development/build.md @@ -9,7 +9,7 @@ toc_title: "Linux\u4E0A\u3067ClickHouse\u3092\u69CB\u7BC9\u3059\u308B\u65B9\u6CD 次のチュートリアルはUbuntu Linuxシステムに基づいています。 適切な変更により、他のLinuxディストリビューションでも動作するはずです。 -サポートされるプラットフォーム:x86\_64およびAArch64。 Power9のサポートは実験的です。 +サポートされるプラットフォーム:x86_64およびAArch64。 Power9のサポートは実験的です。 ## Git、CMake、Pythonと忍者をインストールします {#install-git-cmake-python-and-ninja} diff --git a/docs/ja/development/contrib.md b/docs/ja/development/contrib.md index b8917511e4c..2e16b2bc72a 100644 --- a/docs/ja/development/contrib.md +++ b/docs/ja/development/contrib.md @@ -29,7 +29,7 @@ toc_title: "\u30B5\u30FC\u30C9\u30D1\u30FC\u30C6\u30A3\u88FD\u30E9\u30A4\u30D6\u | libpcg-ランダム | [Apacheライセンス2.0](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libpcg-random/LICENSE-APACHE.txt) | | libressl | [OpenSSLライセンス](https://github.com/ClickHouse-Extras/ssl/blob/master/COPYING) | | リブドカフカ | [BSD2条項ライセンス](https://github.com/edenhill/librdkafka/blob/363dcad5a23dc29381cc626620e68ae418b3af19/LICENSE) | -| libwidechar\_width | [CC0 1.0ユニバーサル](https://github.com/ClickHouse/ClickHouse/blob/master/libs/libwidechar_width/LICENSE) | +| libwidechar_width | [CC0 1.0ユニバーサル](https://github.com/ClickHouse/ClickHouse/blob/master/libs/libwidechar_width/LICENSE) | | llvm | [BSD3条項ライセンス](https://github.com/ClickHouse-Extras/llvm/blob/163def217817c90fb982a6daf384744d8472b92b/llvm/LICENSE.TXT) | | lz4 | [BSD2条項ライセンス](https://github.com/lz4/lz4/blob/c10863b98e1503af90616ae99725ecd120265dfb/LICENSE) | | mariadb-コネクタ-c | [LGPL v2.1](https://github.com/ClickHouse-Extras/mariadb-connector-c/blob/3.1/COPYING.LIB) | diff --git a/docs/ja/development/developer-instruction.md b/docs/ja/development/developer-instruction.md index 6c7e1474dab..f3a703f9879 100644 --- a/docs/ja/development/developer-instruction.md +++ b/docs/ja/development/developer-instruction.md @@ -44,7 +44,7 @@ Ubuntuでこれを行うには、コマンドラインターミナルで実行 git clone --recursive git@github.com:your_github_username/ClickHouse.git cd ClickHouse -注:、代理して下さい *your\_github\_username* 適切なもので! +注:、代理して下さい *your_github_username* 適切なもので! このコマンドディレクトリの作成 `ClickHouse` プロジェクトの作業コピーを含む。 @@ -154,7 +154,7 @@ ClickHouseを構築する準備ができたので、別のディレクトリを mkdir build cd build -いくつかの異なるディレクトリ(build\_release、build\_debugなど)を持つことができます。)ビルドの異なるタイプのために。 +いくつかの異なるディレクトリ(build_release、build_debugなど)を持つことができます。)ビルドの異なるタイプのために。 中の間 `build` cmakeを実行してビルドを構成します。 最初の実行の前に、コンパイラ(この例ではバージョン9gccコンパイラ)を指定する環境変数を定義する必要があります。 diff --git a/docs/ja/development/style.md b/docs/ja/development/style.md index d0f6fb89900..ef962067650 100644 --- a/docs/ja/development/style.md +++ b/docs/ja/development/style.md @@ -356,7 +356,7 @@ class IBlockInputStream bool info_successfully_loaded = false; ``` -**9.** の名前 `define`sおよびグローバル定数を使用ALL\_CAPSをアンダースコア(\_). +**9.** の名前 `define`sおよびグローバル定数を使用ALL_CAPSをアンダースコア(_). ``` cpp #define MAX_SRC_TABLE_NAMES_TO_STORE 1000 @@ -396,7 +396,7 @@ FileQueueProcessor( timer (not m_timer) ``` -**14.** の定数に対して `enum`、大文字でキャメルケースを使用します。 ALL\_CAPSも許容されます。 もし `enum` は非ローカルである。 `enum class`. +**14.** の定数に対して `enum`、大文字でキャメルケースを使用します。 ALL_CAPSも許容されます。 もし `enum` は非ローカルである。 `enum class`. ``` cpp enum class CompressionMethod @@ -709,7 +709,7 @@ auto s = std::string{"Hello"}; **4.**OS:LinuxのUbuntuの、正確よりも古いではありません。 -**5.**コードはx86\_64CPUアーキテクチャ用に書かれている。 +**5.**コードはx86_64CPUアーキテクチャ用に書かれている。 CPU命令セットは、サーバー間でサポートされる最小のセットです。 現在、SSE4.2です。 diff --git a/docs/ja/development/tests.md b/docs/ja/development/tests.md index 0d9fa40cb9b..e41032a6b76 100644 --- a/docs/ja/development/tests.md +++ b/docs/ja/development/tests.md @@ -202,7 +202,7 @@ Clangにはさらに便利な警告があります。 `-Weverything` デフォ ClickHouseファジングは、両方を使用して実装されます [libFuzzer](https://llvm.org/docs/LibFuzzer.html) とランダムSQLクエリ。 すべてのファズテストは、サニタイザー(アドレスと未定義)で実行する必要があります。 -LibFuzzerは、ライブラリコードの分離ファズテストに使用されます。 ファザーはテストコードの一部として実装され “\_fuzzer” 名前の接尾辞。 +LibFuzzerは、ライブラリコードの分離ファズテストに使用されます。 ファザーはテストコードの一部として実装され “_fuzzer” 名前の接尾辞。 Fuzzerの例はで見つけることができます `src/Parsers/tests/lexer_fuzzer.cpp`. LibFuzzer固有の設定、辞書、およびコーパスは次の場所に格納されます `tests/fuzz`. ご協力をお願いいたし書きファズ試験べての機能を取り扱うユーザー入力します。 diff --git a/docs/ja/engines/table-engines/index.md b/docs/ja/engines/table-engines/index.md index d73e9ee32ef..e1a7b085a29 100644 --- a/docs/ja/engines/table-engines/index.md +++ b/docs/ja/engines/table-engines/index.md @@ -62,7 +62,7 @@ toc_title: "\u306F\u3058\u3081\u306B" - [分散](special/distributed.md#distributed) - [マテリアライズドビュー](special/materializedview.md#materializedview) - [辞書](special/dictionary.md#dictionary) -- \[Merge\](special/merge.md\#merge +- \[Merge\](special/merge.md#merge - [ファイル](special/file.md#file) - [Null](special/null.md#null) - [セット](special/set.md#set) diff --git a/docs/ja/engines/table-engines/integrations/hdfs.md b/docs/ja/engines/table-engines/integrations/hdfs.md index 8b7380d2e7e..53e5cd3df34 100644 --- a/docs/ja/engines/table-engines/integrations/hdfs.md +++ b/docs/ja/engines/table-engines/integrations/hdfs.md @@ -73,12 +73,12 @@ SELECT * FROM hdfs_engine_table LIMIT 2 1. HDFS上に次のUriを持つTSV形式のファイルがいくつかあるとします: -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_3’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_3’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_3’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_3’ 1. あはいくつかの方法が考えられているテーブルの構成は、すべてのファイル: diff --git a/docs/ja/engines/table-engines/integrations/kafka.md b/docs/ja/engines/table-engines/integrations/kafka.md index 6b5d6cc4d8b..08e3e951263 100644 --- a/docs/ja/engines/table-engines/integrations/kafka.md +++ b/docs/ja/engines/table-engines/integrations/kafka.md @@ -134,7 +134,7 @@ Kafka(kafka_broker_list, kafka_topic_list, kafka_group_name, kafka_format SELECT level, sum(total) FROM daily GROUP BY level; ``` -パフォーマンスを向上させるために、受信したメッセージは [max\_insert\_block\_size](../../../operations/server-configuration-parameters/settings.md#settings-max_insert_block_size). ブロックが内に形成されていない場合 [stream\_flush\_interval\_ms](../../../operations/server-configuration-parameters/settings.md) ミリ秒、データは関係なく、ブロックの完全性のテーブルにフラッシュされます。 +パフォーマンスを向上させるために、受信したメッセージは [max_insert_block_size](../../../operations/server-configuration-parameters/settings.md#settings-max_insert_block_size). ブロックが内に形成されていない場合 [stream_flush_interval_ms](../../../operations/server-configuration-parameters/settings.md) ミリ秒、データは関係なく、ブロックの完全性のテーブルにフラッシュされます。 リクエストを受けた話題のデータは変更に変換ロジック、切り離しを実現ビュー: diff --git a/docs/ja/engines/table-engines/mergetree-family/custom-partitioning-key.md b/docs/ja/engines/table-engines/mergetree-family/custom-partitioning-key.md index f1058c21c03..30cd3c72af5 100644 --- a/docs/ja/engines/table-engines/mergetree-family/custom-partitioning-key.md +++ b/docs/ja/engines/table-engines/mergetree-family/custom-partitioning-key.md @@ -116,7 +116,7 @@ drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 5 12:09 201902_4_6_1 drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 1 16:48 detached ``` -フォルダ ‘201901\_1\_1\_0’, ‘201901\_1\_7\_1’ そして、部品のディレクトリです。 各部に関する対応する分割データが含まれまで一定の月のテーブルこの例では、分割による。 +フォルダ ‘201901_1_1_0’, ‘201901_1_7_1’ そして、部品のディレクトリです。 各部に関する対応する分割データが含まれまで一定の月のテーブルこの例では、分割による。 その `detached` ディレクトリに含まれる部品のこともあったかを使って、テーブル [DETACH](../../../sql-reference/statements/alter.md#alter_detach-partition) クエリ。 破損した部分も、削除されるのではなく、このディレクトリに移動されます。 サーバーは、サーバーからの部品を使用しません。 `detached` directory. You can add, delete, or modify the data in this directory at any time – the server will not know about this until you run the [ATTACH](../../../sql-reference/statements/alter.md#alter_attach-partition) クエリ。 diff --git a/docs/ja/engines/table-engines/mergetree-family/graphitemergetree.md b/docs/ja/engines/table-engines/mergetree-family/graphitemergetree.md index 4e7522f5f48..18ab9390f0f 100644 --- a/docs/ja/engines/table-engines/mergetree-family/graphitemergetree.md +++ b/docs/ja/engines/table-engines/mergetree-family/graphitemergetree.md @@ -81,7 +81,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ## ロールアップ構成 {#rollup-configuration} -ロールアップの設定は、 [graphite\_rollup](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) サーバー構成のパラメータ。 パラメーターの名前は任意です。 複数の構成を作成し、異なるテーブルに使用できます。 +ロールアップの設定は、 [graphite_rollup](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) サーバー構成のパラメータ。 パラメーターの名前は任意です。 複数の構成を作成し、異なるテーブルに使用できます。 ロールアップ構成構造: diff --git a/docs/ja/engines/table-engines/mergetree-family/mergetree.md b/docs/ja/engines/table-engines/mergetree-family/mergetree.md index 55538136a57..f93d5ea4a64 100644 --- a/docs/ja/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ja/engines/table-engines/mergetree-family/mergetree.md @@ -239,7 +239,7 @@ ClickHouseの主キー指標のトリムで不正なデータを毎月パーテ SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' ``` -クエリの実行時にClickHouseがインデックスを使用できるかどうかを確認するには、設定を使用します [force\_index\_by\_date](../../../operations/settings/settings.md#settings-force_index_by_date) と [force\_primary\_key](../../../operations/settings/settings.md). +クエリの実行時にClickHouseがインデックスを使用できるかどうかを確認するには、設定を使用します [force_index_by_date](../../../operations/settings/settings.md#settings-force_index_by_date) と [force_primary_key](../../../operations/settings/settings.md). の分割による月で読み込みのみこれらのデータブロックを含むからスピーチへのマークの範囲内で適切に取扱います。 この場合、データブロックには多くの日付(月全体まで)のデータが含まれる場合があります。 ブロック内では、データは主キーによってソートされます。 このため、主キープレフィックスを指定しない日付条件のみのクエリを使用すると、単一の日付よりも多くのデータが読み取られます。 @@ -330,7 +330,7 @@ INDEX sample_index3 (lower(str), str) TYPE ngrambf_v1(3, 256, 2, 0) GRANULARITY その `set` 索引はすべての機能と使用することができる。 その他のインデックスの関数サブセットを以下の表に示します。 -| 関数(演算子)/インデックス | 主キー | minmax | ngrambf\_v1 | tokenbf\_v1 | bloom\_filter name | +| 関数(演算子)/インデックス | 主キー | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter name | |-----------------------------------------------------------------------------------------------------------|--------|--------|-------------|-------------|--------------------| | [等しい(=,==)](../../../sql-reference/functions/comparison-functions.md#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | | [notEquals(!=, \<\>)](../../../sql-reference/functions/comparison-functions.md#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | @@ -642,7 +642,7 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' フードの下で、突然変異および仕切りの凍結は利用します [ハードリンク](https://en.wikipedia.org/wiki/Hard_link). ハードリンクとディスクには対応していないため、この場合、パーツの保管と同じディスクの初期ます。 バックグラウンドでは、部品は空き領域の量に基づいてボリューム間で移動されます (`move_factor` パラメータ)ボリュームが設定ファイルで宣言されている順序に従って。 -データは、最後のデータから最初のデータに転送されることはありません。 システムテーブル [システムpart\_log](../../../operations/system-tables.md#system_tables-part-log) (フィールド `type = MOVE_PART`)と [システム部品](../../../operations/system-tables.md#system_tables-parts) (フィールド `path` と `disk`)背景の動きを監視します。 また、詳細情報はサーバーログに記載されています。 +データは、最後のデータから最初のデータに転送されることはありません。 システムテーブル [システムpart_log](../../../operations/system-tables.md#system_tables-part-log) (フィールド `type = MOVE_PART`)と [システム部品](../../../operations/system-tables.md#system_tables-parts) (フィールド `path` と `disk`)背景の動きを監視します。 また、詳細情報はサーバーログに記載されています。 ユーザーの力で移動中の一部またはパーティションから量別のクエリ [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](../../../sql-reference/statements/alter.md#alter_move-partition) バックグラウンド操作のすべての制限が考慮されます。 クエリは単独で移動を開始し、バックグラウンド操作が完了するまで待機しません。 十分な空き領域がない場合、または必要な条件のいずれかが満たされていない場合、ユーザーはエラーメッセージを表示します。 diff --git a/docs/ja/engines/table-engines/mergetree-family/replication.md b/docs/ja/engines/table-engines/mergetree-family/replication.md index bcc5de5e364..4e0d2bc0831 100644 --- a/docs/ja/engines/table-engines/mergetree-family/replication.md +++ b/docs/ja/engines/table-engines/mergetree-family/replication.md @@ -59,7 +59,7 @@ ZooKeeperクラスタのアドレスを設定する例: 場合飼育係な設定コンフィグファイルを創り上げられないんで再現しテーブル、および既存の複製のテーブル読み取り専用になります。 -飼育係はで使用されていません `SELECT` レプリケーションのパフォーマン `SELECT` また、クエリは非レプリケートテーブルの場合と同様に高速に実行されます。 時の照会に配布再現し、テーブルClickHouse行動制御の設定 [max\_replica\_delay\_for\_distributed\_queries](../../../operations/settings/settings.md#settings-max_replica_delay_for_distributed_queries) と [フォールバック\_to\_stale\_replicas\_for\_distributed\_queries](../../../operations/settings/settings.md#settings-fallback_to_stale_replicas_for_distributed_queries). +飼育係はで使用されていません `SELECT` レプリケーションのパフォーマン `SELECT` また、クエリは非レプリケートテーブルの場合と同様に高速に実行されます。 時の照会に配布再現し、テーブルClickHouse行動制御の設定 [max_replica_delay_for_distributed_queries](../../../operations/settings/settings.md#settings-max_replica_delay_for_distributed_queries) と [フォールバック_to_stale_replicas_for_distributed_queries](../../../operations/settings/settings.md#settings-fallback_to_stale_replicas_for_distributed_queries). それぞれのため `INSERT` クエリー、契約時に応募を追加飼育係を務取引等 (より正確には、これは挿入されたデータの各ブロックに対するものです。 `max_insert_block_size = 1048576` 行。)これは、 `INSERT` 非レプリケートテーブルと比較します。 しかし、推奨事項に従ってデータを複数のバッチで挿入する場合 `INSERT` 毎秒、それは問題を作成しません。 一つのZooKeeperクラスターを調整するために使用されるClickHouseクラスター全体の合計は数百です `INSERTs` 毎秒 データ挿入のスループット(秒あたりの行数)は、レプリケートされていないデータの場合と同じくらい高くなります。 @@ -71,7 +71,7 @@ ZooKeeperクラスタのアドレスを設定する例: データの各ブロックは原子的に書き込まれます。 挿入クエリは、次のブロックに分割されます `max_insert_block_size = 1048576` 行。 言い換えれば、 `INSERT` クエリには1048576行未満があり、アトミックに作成されます。 -データブロックは重複排除されます。 同じデータブロック(同じ順序で同じ行を含む同じサイズのデータブロック)の複数の書き込みの場合、ブロックは一度だけ書き込まれます。 この理由は、クライアントアプリケーションがデータがDBに書き込まれたかどうかを知らないときにネットワーク障害が発生した場合です。 `INSERT` クエリーするだけで簡単に繰り返します。 どのレプリカ挿入が同一のデータで送信されたかは関係ありません。 `INSERTs` 冪等である。 重複排除圧縮パラメータの制御 [merge\_tree](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-merge_tree) サーバー設定。 +データブロックは重複排除されます。 同じデータブロック(同じ順序で同じ行を含む同じサイズのデータブロック)の複数の書き込みの場合、ブロックは一度だけ書き込まれます。 この理由は、クライアントアプリケーションがデータがDBに書き込まれたかどうかを知らないときにネットワーク障害が発生した場合です。 `INSERT` クエリーするだけで簡単に繰り返します。 どのレプリカ挿入が同一のデータで送信されたかは関係ありません。 `INSERTs` 冪等である。 重複排除圧縮パラメータの制御 [merge_tree](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-merge_tree) サーバー設定。 複製時に、元のデータの挿入には転送されます。 さらなるデータ変換(マージ)は、すべてのレプリカで同じ方法で調整され、実行されます。 つまり、レプリカが異なるデータセンターに存在する場合、レプリケーションは適切に機能します。 レプリケーションの主な目的は、異なるデータセンターでデータを複製することです。) diff --git a/docs/ja/engines/table-engines/special/buffer.md b/docs/ja/engines/table-engines/special/buffer.md index c3ef6bd1157..deb38c04692 100644 --- a/docs/ja/engines/table-engines/special/buffer.md +++ b/docs/ja/engines/table-engines/special/buffer.md @@ -36,7 +36,7 @@ Buffer(database, table, num_layers, min_time, max_time, min_rows, max_rows, min_ CREATE TABLE merge.hits_buffer AS merge.hits ENGINE = Buffer(merge, hits, 16, 10, 100, 10000, 1000000, 10000000, 100000000) ``` -作成 ‘merge.hits\_buffer’ と同じ構造を持つテーブル ‘merge.hits’ そして、バッファエンジンを使用。 このテーブルに書き込むとき、データはRAMにバッファされ、後で ‘merge.hits’ テーブル。 16のバッファが作成されます。 それぞれのデータは、100秒が経過した場合、または百万行が書き込まれた場合、または100MBのデータが書き込まれた場合、または10秒が経過して10,000行と10 たとえば、ただ一つの行が書き込まれている場合、100秒後には何があってもフラッシュされます。 しかし、多くの行が書き込まれている場合、データは早くフラッシュされます。 +作成 ‘merge.hits_buffer’ と同じ構造を持つテーブル ‘merge.hits’ そして、バッファエンジンを使用。 このテーブルに書き込むとき、データはRAMにバッファされ、後で ‘merge.hits’ テーブル。 16のバッファが作成されます。 それぞれのデータは、100秒が経過した場合、または百万行が書き込まれた場合、または100MBのデータが書き込まれた場合、または10秒が経過して10,000行と10 たとえば、ただ一つの行が書き込まれている場合、100秒後には何があってもフラッシュされます。 しかし、多くの行が書き込まれている場合、データは早くフラッシュされます。 DROP TABLEまたはDETACH TABLEを使用してサーバーを停止すると、バッファーデータも宛先テーブルにフラッシュされます。 @@ -58,7 +58,7 @@ DROP TABLEまたはDETACH TABLEを使用してサーバーを停止すると、 を追加する場合にデータをバッファのバッファがロックされています。 これにより、テーブルから読み取り操作が同時に実行される場合に遅延が発生します。 -バッファテーブルに挿入されるデータは、下位テーブル内で異なる順序と異なるブロックになる場合があります。 このため、CollapsingMergeTreeに正しく書き込むためにバッファテーブルを使用することは困難です。 問題を回避するには、以下を設定できます ‘num\_layers’ に1. +バッファテーブルに挿入されるデータは、下位テーブル内で異なる順序と異なるブロックになる場合があります。 このため、CollapsingMergeTreeに正しく書き込むためにバッファテーブルを使用することは困難です。 問題を回避するには、以下を設定できます ‘num_layers’ に1. の場合は先テーブルがそのままに再現され、期待の特徴を再現でテーブルが失われた書き込みバッファへの表に示す。 行の順序とデータ部分のサイズがランダムに変更されると、データ重複排除が動作を終了します。 ‘exactly once’ 書く再現します。 diff --git a/docs/ja/engines/table-engines/special/distributed.md b/docs/ja/engines/table-engines/special/distributed.md index 429a34ce022..87d8c1bc6d5 100644 --- a/docs/ja/engines/table-engines/special/distributed.md +++ b/docs/ja/engines/table-engines/special/distributed.md @@ -85,13 +85,13 @@ logs – The cluster name in the server's config file. パラメータ `host`, `port`、および必要に応じて `user`, `password`, `secure`, `compression` サーバーごとに指定されます: - `host` – The address of the remote server. You can use either the domain or the IPv4 or IPv6 address. If you specify the domain, the server makes a DNS request when it starts, and the result is stored as long as the server is running. If the DNS request fails, the server doesn't start. If you change the DNS record, restart the server. -- `port` – The TCP port for messenger activity (‘tcp\_port’ 設定では、通常9000に設定されています)。 Http\_portと混同しないでください。 +- `port` – The TCP port for messenger activity (‘tcp_port’ 設定では、通常9000に設定されています)。 Http_portと混同しないでください。 - `user` – Name of the user for connecting to a remote server. Default value: default. This user must have access to connect to the specified server. Access is configured in the users.xml file. For more information, see the section [アクセス権](../../../operations/access-rights.md). - `password` – The password for connecting to a remote server (not masked). Default value: empty string. - `secure` -接続にsslを使用します。 `port` = 9440. サーバーがリッスンする `9440` 正しい証明書を持っています。 - `compression` -データ圧縮を使用します。 デフォルト値:true。 -When specifying replicas, one of the available replicas will be selected for each of the shards when reading. You can configure the algorithm for load balancing (the preference for which replica to access) – see the [load\_balancing](../../../operations/settings/settings.md#settings-load_balancing) 設定。 +When specifying replicas, one of the available replicas will be selected for each of the shards when reading. You can configure the algorithm for load balancing (the preference for which replica to access) – see the [load_balancing](../../../operations/settings/settings.md#settings-load_balancing) 設定。 サーバーとの接続が確立されていない場合は、短いタイムアウトで接続しようとします。 接続に失敗した場合は、すべてのレプリカに対して次のレプリカが選択されます。 すべてのレプリカで接続試行が失敗した場合、その試行は同じ方法で何度か繰り返されます。 リモートサーバーは接続を受け入れるかもしれませんが、動作しないか、または不十分に動作する可能性があります。 @@ -113,13 +113,13 @@ The Distributed engine requires writing clusters to the config file. Clusters fr 各シャードには、設定ファイルで定義された重みを設定できます。 デフォルトでは、重みは重みと等しくなります。 データは、シャードの重みに比例する量でシャードに分散されます。 たとえば、シャードが二つあり、最初のシャードが9の重みを持ち、次のシャードが10の重みを持つ場合、最初のシャードは行の9/19部分に送信され、次のシャー -各シャードは、 ‘internal\_replication’ 設定ファイルで定義されたパラメータ。 +各シャードは、 ‘internal_replication’ 設定ファイルで定義されたパラメータ。 このパラメータが ‘true’ 書き込み操作は、最初の正常なレプリカを選択し、それにデータを書き込みます。 分散テーブルの場合は、この代替を使用します “looks at” 複製されたテーブル。 言い換えれば、データが書き込まれるテーブルがそれ自体を複製する場合。 に設定されている場合 ‘false’ データはすべてのレプリカに書き込まれます。 本質的に、これは分散テーブルがデータ自体を複製することを意味します。 これは、レプリケートされたテーブルを使用するよりも悪いことです。 -データの行が送信されるシャードを選択するために、シャーディング式が分析され、残りの部分がシャードの総重量で除算されます。 この行は、残りの半分の間隔に対応するシャードに送信されます。 ‘prev\_weight’ に ‘prev\_weights + weight’,ここで ‘prev\_weights’ は、最小の数を持つ破片の総重量です。 ‘weight’ この破片の重量です。 たとえば、シャードが二つあり、最初のシャードの重みが9で、次のシャードの重みが10である場合、行は範囲\[0,9)の残りのシャードの最初のシャードに送信され、 +データの行が送信されるシャードを選択するために、シャーディング式が分析され、残りの部分がシャードの総重量で除算されます。 この行は、残りの半分の間隔に対応するシャードに送信されます。 ‘prev_weight’ に ‘prev_weights + weight’,ここで ‘prev_weights’ は、最小の数を持つ破片の総重量です。 ‘weight’ この破片の重量です。 たとえば、シャードが二つあり、最初のシャードの重みが9で、次のシャードの重みが10である場合、行は範囲\[0,9)の残りのシャードの最初のシャードに送信され、 シャーディング式には、整数を返す定数およびテーブル列の任意の式を使用できます。 たとえば、次の式を使用できます ‘rand()’ データのランダム分布の場合、または ‘UserID’ ユーザーのIDを分割することからの残りの部分による分配のために(単一のユーザーのデータは単一のシャード上に存在し、ユーザーによる実行と結合を簡素化する)。 いずれかの列が十分に均等に分散されていない場合は、ハッシュ関数intHash64(UserID)でラップできます。 @@ -132,11 +132,11 @@ SELECT queries are sent to all the shards and work regardless of how data is dis - 特定のキーによるデータの結合(INまたはJOIN)を必要とするクエリが使用されます。 このキーによってデータがシャードされる場合は、GLOBAL INまたはGLOBAL JOINの代わりにlocal INまたはJOINを使用できます。 - 多数のサーバーが、多数の小さなクエリ(個々のクライアント-ウェブサイト、広告主、またはパートナーのクエリ)で使用されます(数百以上)。 小さなクエリがクラスタ全体に影響を与えないようにするには、単一のシャード上の単一のクライアントのデータを検索することが理にかなってい また、我々はYandexのでやったように。Metricaでは、biレベルのシャーディングを設定できます:クラスタ全体を次のように分割します “layers” ここで、レイヤーは複数のシャードで構成されます。 単一のクライアントのデータは単一のレイヤー上にありますが、必要に応じてシャードをレイヤーに追加することができ、データはランダムに分散されます。 分散テーブルはレイヤごとに作成され、グローバルクエリ用に単一の共有分散テーブルが作成されます。 -データは非同期に書き込まれます。 テーブルに挿入すると、データブロックはローカルファイルシステムに書き込まれます。 データはできるだけ早くバックグラウンドでリモートサーバーに送信されます。 データを送信するための期間は、 [distributed\_directory\_monitor\_sleep\_time\_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms) と [distributed\_directory\_monitor\_max\_sleep\_time\_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms) 設定。 その `Distributed` エンジンは、挿入されたデータを含む各ファイルを別々に送信しますが、 [distributed\_directory\_monitor\_batch\_inserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts) 設定。 この設定の改善にクラスターの性能をより一層の活用地域のサーバやネットワーク資源です。 を確認しておきましょうか否かのデータが正常に送信されるチェックリストファイル(データまたは間に-をはさんだ)はテーブルディレクトリ: `/var/lib/clickhouse/data/database/table/`. +データは非同期に書き込まれます。 テーブルに挿入すると、データブロックはローカルファイルシステムに書き込まれます。 データはできるだけ早くバックグラウンドでリモートサーバーに送信されます。 データを送信するための期間は、 [distributed_directory_monitor_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms) と [distributed_directory_monitor_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms) 設定。 その `Distributed` エンジンは、挿入されたデータを含む各ファイルを別々に送信しますが、 [distributed_directory_monitor_batch_inserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts) 設定。 この設定の改善にクラスターの性能をより一層の活用地域のサーバやネットワーク資源です。 を確認しておきましょうか否かのデータが正常に送信されるチェックリストファイル(データまたは間に-をはさんだ)はテーブルディレクトリ: `/var/lib/clickhouse/data/database/table/`. 分散テーブルへの挿入後にサーバーが存在しなくなった場合、またはデバイスの障害後などに大まかな再起動が行われた場合は、挿入されたデータが失われ テーブルディレクトリで破損したデータ部分が検出されると、その部分は ‘broken’ 使用されなくなりました。 -Max\_parallel\_replicasオプションを有効にすると、単一のシャード内のすべてのレプリカでクエリ処理が並列化されます。 詳細については [max\_parallel\_replicas](../../../operations/settings/settings.md#settings-max_parallel_replicas). +Max_parallel_replicasオプションを有効にすると、単一のシャード内のすべてのレプリカでクエリ処理が並列化されます。 詳細については [max_parallel_replicas](../../../operations/settings/settings.md#settings-max_parallel_replicas). ## 仮想列 {#virtual-columns} diff --git a/docs/ja/engines/table-engines/special/external-data.md b/docs/ja/engines/table-engines/special/external-data.md index afa727b6ebb..ffa726be923 100644 --- a/docs/ja/engines/table-engines/special/external-data.md +++ b/docs/ja/engines/table-engines/special/external-data.md @@ -27,10 +27,10 @@ ClickHouseでは、クエリの処理に必要なデータをSELECTクエリと **–file** – Path to the file with the table dump, or -, which refers to stdin. Stdinから取得できるのは単一のテーブルのみです。 -次のパラメータは省略可能です: **–name**– Name of the table. If omitted, \_data is used. +次のパラメータは省略可能です: **–name**– Name of the table. If omitted, _data is used. **–format** – Data format in the file. If omitted, TabSeparated is used. -次のいずれかのパラメータが必要です:**–types** – A list of comma-separated column types. For example: `UInt64,String`. The columns will be named \_1, \_2, … +次のいずれかのパラメータが必要です:**–types** – A list of comma-separated column types. For example: `UInt64,String`. The columns will be named _1, _2, … **–structure**– The table structure in the format`UserID UInt64`, `URL String`. 列名と型を定義します。 で指定されたファイル ‘file’ に指定された形式で解析されます。 ‘format’ で指定されたデータ型を使用します ‘types’ または ‘structure’. のテーブルがアップロードサーバへのアクセスが一時テーブルの名前 ‘name’. @@ -48,7 +48,7 @@ $ cat /etc/passwd | sed 's/:/\t/g' | clickhouse-client --query="SELECT shell, co /bin/sync 1 ``` -HTTPインターフェイスを使用する場合、外部データはmultipart/form-data形式で渡されます。 各テーブルは別々のファイルとして送信されます。 テーブル名は、ファイル名から取得されます。 その ‘query\_string’ パラメータが渡されます ‘name\_format’, ‘name\_types’,and ‘name\_structure’,ここで ‘name’ これらのパラメーターが対応するテーブルの名前を指定します。 パラメーターの意味は、コマンドラインクライアントを使用する場合と同じです。 +HTTPインターフェイスを使用する場合、外部データはmultipart/form-data形式で渡されます。 各テーブルは別々のファイルとして送信されます。 テーブル名は、ファイル名から取得されます。 その ‘query_string’ パラメータが渡されます ‘name_format’, ‘name_types’,and ‘name_structure’,ここで ‘name’ これらのパラメーターが対応するテーブルの名前を指定します。 パラメーターの意味は、コマンドラインクライアントを使用する場合と同じです。 例: diff --git a/docs/ja/engines/table-engines/special/join.md b/docs/ja/engines/table-engines/special/join.md index e88bc12a484..bf96f64e7fc 100644 --- a/docs/ja/engines/table-engines/special/join.md +++ b/docs/ja/engines/table-engines/special/join.md @@ -92,15 +92,15 @@ SELECT joinGet('id_val_join', 'val', toUInt32(1)) テーブルを作成するときは、次の設定が適用されます: -- [join\_use\_nulls](../../../operations/settings/settings.md#join_use_nulls) -- [max\_rows\_in\_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) -- [max\_bytes\_in\_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) -- [join\_overflow\_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) -- [join\_any\_take\_last\_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) +- [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) +- [max_rows_in_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) +- [max_bytes_in_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) +- [join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) +- [join_any_take_last_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) その `Join`-エンジンテーブルは使用できません `GLOBAL JOIN` 作戦だ -その `Join`-エンジンは使用を許可する [join\_use\_nulls](../../../operations/settings/settings.md#join_use_nulls) の設定 `CREATE TABLE` 声明。 と [SELECT](../../../sql-reference/statements/select/index.md) クエリを使用できる `join_use_nulls` あまりにも。 あなたが違う場合 `join_use_nulls` 設定することができるエラー入社。 それは結合の種類に依存します。 使用するとき [joinGet](../../../sql-reference/functions/other-functions.md#joinget) 関数、あなたは同じを使用する必要があります `join_use_nulls` 設定 `CRATE TABLE` と `SELECT` 声明。 +その `Join`-エンジンは使用を許可する [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) の設定 `CREATE TABLE` 声明。 と [SELECT](../../../sql-reference/statements/select/index.md) クエリを使用できる `join_use_nulls` あまりにも。 あなたが違う場合 `join_use_nulls` 設定することができるエラー入社。 それは結合の種類に依存します。 使用するとき [joinGet](../../../sql-reference/functions/other-functions.md#joinget) 関数、あなたは同じを使用する必要があります `join_use_nulls` 設定 `CRATE TABLE` と `SELECT` 声明。 ## データ保存 {#data-storage} diff --git a/docs/ja/engines/table-engines/special/merge.md b/docs/ja/engines/table-engines/special/merge.md index 1d2ff7ce794..0a22b71964e 100644 --- a/docs/ja/engines/table-engines/special/merge.md +++ b/docs/ja/engines/table-engines/special/merge.md @@ -31,7 +31,7 @@ Regular expressions — [re2](https://github.com/google/re2) (PCREのサブセ 例2: -古いテーブル(WatchLog\_old)があり、データを新しいテーブル(WatchLog\_new)に移動せずにパーティション分割を変更することにしたとしましょう。 +古いテーブル(WatchLog_old)があり、データを新しいテーブル(WatchLog_new)に移動せずにパーティション分割を変更することにしたとしましょう。 ``` sql CREATE TABLE WatchLog_old(date Date, UserId Int64, EventType String, Cnt UInt64) diff --git a/docs/ja/engines/table-engines/special/url.md b/docs/ja/engines/table-engines/special/url.md index ded439caa4c..bf920d23be8 100644 --- a/docs/ja/engines/table-engines/special/url.md +++ b/docs/ja/engines/table-engines/special/url.md @@ -24,7 +24,7 @@ HTTPまたはHTTPSを使用します。 これは必要ありません それぞれ。 処理のため `POST` 要求は、リモートサーバーが [チャンク転送エンコード](https://en.wikipedia.org/wiki/Chunked_transfer_encoding). -HTTP GETリダイレクトホップの最大数を制限するには [max\_http\_get\_redirects](../../../operations/settings/settings.md#setting-max_http_get_redirects) 設定。 +HTTP GETリダイレクトホップの最大数を制限するには [max_http_get_redirects](../../../operations/settings/settings.md#setting-max_http_get_redirects) 設定。 **例:** diff --git a/docs/ja/faq/general.md b/docs/ja/faq/general.md index 260826160d2..e7e3f9328f4 100644 --- a/docs/ja/faq/general.md +++ b/docs/ja/faq/general.md @@ -17,7 +17,7 @@ MapReduceのようなシステムは、reduce操作が分散ソートに基づ ## になっているのでしょうか?しているのでエンコーディング利用の場合OracleをODBC? {#oracle-odbc-encodings} -外部ディクショナリのソースとしてODBCドライバを介してOracleを使用する場合は、外部ディクショナリの正しい値を設定する必要が `NLS_LANG` 環境変数in `/etc/default/clickhouse`. 詳細については、を参照してください [Oracle NLS\_LANG FAQ](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html). +外部ディクショナリのソースとしてODBCドライバを介してOracleを使用する場合は、外部ディクショナリの正しい値を設定する必要が `NLS_LANG` 環境変数in `/etc/default/clickhouse`. 詳細については、を参照してください [Oracle NLS_LANG FAQ](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html). **例** diff --git a/docs/ja/getting-started/example-datasets/amplab-benchmark.md b/docs/ja/getting-started/example-datasets/amplab-benchmark.md index 0c5d8707784..a3f66e1a0de 100644 --- a/docs/ja/getting-started/example-datasets/amplab-benchmark.md +++ b/docs/ja/getting-started/example-datasets/amplab-benchmark.md @@ -8,7 +8,7 @@ toc_title: "AMPLab Big Data ベンチマーク" https://amplab.cs.berkeley.edu/benchmark/ を参照して下さい。 https://aws.amazon.com で無料アカウントにサインアップしてください。クレジットカード、電子メール、電話番号が必要です。 -https://console.aws.amazon.com/iam/home?nc2=h\_m\_sc\#security\_credential で新しいアクセスキーを取得します。 +https://console.aws.amazon.com/iam/home?nc2=h_m_sc#security_credential で新しいアクセスキーを取得します。 コンソールで以下を実行します: diff --git a/docs/ja/getting-started/playground.md b/docs/ja/getting-started/playground.md index da210b228b6..3a2e59aa104 100644 --- a/docs/ja/getting-started/playground.md +++ b/docs/ja/getting-started/playground.md @@ -41,10 +41,10 @@ ClickHouse をサポートするソフトウェア製品の詳細情報は[こ また、以下の設定がなされています。 -- [max\_result\_bytes=10485760](../operations/settings/query_complexity/#max-result-bytes) -- [max\_result\_rows=2000](../operations/settings/query_complexity/#setting-max_result_rows) -- [result\_overflow\_mode=break](../operations/settings/query_complexity/#result-overflow-mode) -- [max\_execution\_time=60000](../operations/settings/query_complexity/#max-execution-time) +- [max_result_bytes=10485760](../operations/settings/query_complexity/#max-result-bytes) +- [max_result_rows=2000](../operations/settings/query_complexity/#setting-max_result_rows) +- [result_overflow_mode=break](../operations/settings/query_complexity/#result-overflow-mode) +- [max_execution_time=60000](../operations/settings/query_complexity/#max-execution-time) ## 例 {#examples} diff --git a/docs/ja/index.md b/docs/ja/index.md index f35e45ddc09..3c5117b3a2c 100644 --- a/docs/ja/index.md +++ b/docs/ja/index.md @@ -11,10 +11,10 @@ ClickHouseは、クエリのオンライン分析処理(OLAP)用の列指向 | Row | WatchID | JavaEnable | Title | GoodEvent | EventTime | |-----|-------------|------------|--------------------|-----------|---------------------| -| \#0 | 89354350662 | 1 | Investor Relations | 1 | 2016-05-18 05:19:20 | -| \#1 | 90329509958 | 0 | Contact us | 1 | 2016-05-18 08:10:20 | -| \#2 | 89953706054 | 1 | Mission | 1 | 2016-05-18 07:38:00 | -| \#N | … | … | … | … | … | +| #0 | 89354350662 | 1 | Investor Relations | 1 | 2016-05-18 05:19:20 | +| #1 | 90329509958 | 0 | Contact us | 1 | 2016-05-18 08:10:20 | +| #2 | 89953706054 | 1 | Mission | 1 | 2016-05-18 07:38:00 | +| #N | … | … | … | … | … | つまり、行に関連するすべての値は物理的に隣り合わせに格納されます。 @@ -22,7 +22,7 @@ ClickHouseは、クエリのオンライン分析処理(OLAP)用の列指向 列指向のDBMSでは、データは次のように保存されます: -| Row: | \#0 | \#1 | \#2 | \#N | +| Row: | #0 | #1 | #2 | #N | |-------------|---------------------|---------------------|---------------------|-----| | WatchID: | 89354350662 | 90329509958 | 89953706054 | … | | JavaEnable: | 1 | 0 | 1 | … | diff --git a/docs/ja/interfaces/formats.md b/docs/ja/interfaces/formats.md index acb5bd71262..35cfd4f67fb 100644 --- a/docs/ja/interfaces/formats.md +++ b/docs/ja/interfaces/formats.md @@ -202,7 +202,7 @@ SELECT * FROM nestedt FORMAT TSV - `min` は最小値を持つ行です。 `format_template_row` 書式(極値が1に設定されている場合) - `max` は最大値を持つ行です。 `format_template_row` 書式(極値が1に設定されている場合) - `rows` 出力行の合計数です -- `rows_before_limit` そこにあったであろう行の最小数は制限なしです。 出力の場合のみを含むクエリを制限します。 クエリにGROUP BYが含まれている場合、rows\_before\_limit\_at\_leastは制限なしで行われていた行の正確な数です。 +- `rows_before_limit` そこにあったであろう行の最小数は制限なしです。 出力の場合のみを含むクエリを制限します。 クエリにGROUP BYが含まれている場合、rows_before_limit_at_leastは制限なしで行われていた行の正確な数です。 - `time` リクエストの実行時間を秒単位で指定します - `rows_read` 読み込まれた行数です - `bytes_read` (圧縮されていない)読み込まれたバイト数です @@ -353,21 +353,21 @@ Both data output and parsing are supported in this format. For parsing, any orde カンマ区切りの値の形式 ([RFC](https://tools.ietf.org/html/rfc4180)). -書式設定の場合、行は二重引用符で囲まれます。 文字列内の二重引用符は、行の二重引用符として出力されます。 文字をエスケープするルールは他にありません。 Dateとdate-timeは二重引用符で囲みます。 数値は引用符なしで出力されます。 値は区切り文字で区切られます。 `,` デフォルトでは。 区切り文字は設定で定義されています [format\_csv\_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter). 行は、Unixラインフィード(LF)を使用して区切られます。 最初に、配列はTabSeparated形式のように文字列にシリアル化され、結果の文字列は二重引用符でCSVに出力されます。 CSV形式のタプルは、個別の列としてシリアル化されます(つまり、タプル内の入れ子は失われます)。 +書式設定の場合、行は二重引用符で囲まれます。 文字列内の二重引用符は、行の二重引用符として出力されます。 文字をエスケープするルールは他にありません。 Dateとdate-timeは二重引用符で囲みます。 数値は引用符なしで出力されます。 値は区切り文字で区切られます。 `,` デフォルトでは。 区切り文字は設定で定義されています [format_csv_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter). 行は、Unixラインフィード(LF)を使用して区切られます。 最初に、配列はTabSeparated形式のように文字列にシリアル化され、結果の文字列は二重引用符でCSVに出力されます。 CSV形式のタプルは、個別の列としてシリアル化されます(つまり、タプル内の入れ子は失われます)。 ``` bash $ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV" < data.csv ``` -\*デフォルトでは、区切り文字は `,`. を参照。 [format\_csv\_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter) より多くの情報のための設定。 +\*デフォルトでは、区切り文字は `,`. を参照。 [format_csv_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter) より多くの情報のための設定。 解析時には、すべての値を引用符の有無にかかわらず解析できます。 二重引用符と単一引quotesの両方がサポートされています。 行は引用符なしで配置することもできます。 この場合、区切り文字または改行(CRまたはLF)まで解析されます。 RFCに違反して、引用符なしで行を解析すると、先頭と末尾のスペースとタブは無視されます。 ラインフィードでは、Unix(LF)、Windows(CR LF)、およびMac OS Classic(CR LF)タイプがすべてサポートされています。 空の引用符で囲まれていない入力値は、それぞれの列のデフォルト値に置き換えられます。 -[input\_format\_defaults\_for\_omitted\_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) +[input_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) 有効です。 -`NULL` として書式設定される `\N` または `NULL` または空の引用符で囲まれていない文字列(設定を参照 [input\_format\_csv\_unquoted\_null\_literal\_as\_null](../operations/settings/settings.md#settings-input_format_csv_unquoted_null_literal_as_null) と [input\_format\_defaults\_for\_omitted\_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields)). +`NULL` として書式設定される `\N` または `NULL` または空の引用符で囲まれていない文字列(設定を参照 [input_format_csv_unquoted_null_literal_as_null](../operations/settings/settings.md#settings-input_format_csv_unquoted_null_literal_as_null) と [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields)). CSV形式では、合計と極値の出力は次のようにサポートされます `TabSeparated`. @@ -452,12 +452,12 @@ SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase WITH TOTA } ``` -JSONはJavaScriptと互換性があります。 これを確実にするために、一部の文字は追加でエスケープされます。 `/` としてエスケープ `\/`;代替の改行 `U+2028` と `U+2029` ブラウザによってはエスケープされます `\uXXXX`. ASCII制御文字はエスケープされます。 `\b`, `\f`, `\n`, `\r`, `\t` を使用して、00-1F範囲の残りのバイトと同様に `\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) 0にする。 +JSONはJavaScriptと互換性があります。 これを確実にするために、一部の文字は追加でエスケープされます。 `/` としてエスケープ `\/`;代替の改行 `U+2028` と `U+2029` ブラウザによってはエスケープされます `\uXXXX`. ASCII制御文字はエスケープされます。 `\b`, `\f`, `\n`, `\r`, `\t` を使用して、00-1F範囲の残りのバイトと同様に `\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) 0にする。 `rows` – The total number of output rows. `rows_before_limit_at_least` そこにある行の最小数は制限なしであったでしょう。 出力の場合のみを含むクエリを制限します。 -クエリにGROUP BYが含まれている場合、rows\_before\_limit\_at\_leastは制限なしで行われていた行の正確な数です。 +クエリにGROUP BYが含まれている場合、rows_before_limit_at_leastは制限なしで行われていた行の正確な数です。 `totals` – Total values (when using WITH TOTALS). @@ -544,7 +544,7 @@ ClickHouseを無視した空間要素には、カンマの後にオブジェク ClickHouseは省略された値を対応するデフォルト値に置き換えます [データ型](../sql-reference/data-types/index.md). -もし `DEFAULT expr` に応じて異なる置換規則を使用します。 [input\_format\_defaults\_for\_omitted\_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) 設定。 +もし `DEFAULT expr` に応じて異なる置換規則を使用します。 [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) 設定。 次の表を考えます: @@ -587,7 +587,7 @@ CREATE TABLE IF NOT EXISTS example_table ### 入れ子構造の使用 {#jsoneachrow-nested} -あなたがテーブルを持っている場合 [入れ子](../sql-reference/data-types/nested-data-structures/nested.md) データ型の列には、同じ構造でJSONデータを挿入することができます。 この機能を有効にするには [input\_format\_import\_nested\_json](../operations/settings/settings.md#settings-input_format_import_nested_json) 設定。 +あなたがテーブルを持っている場合 [入れ子](../sql-reference/data-types/nested-data-structures/nested.md) データ型の列には、同じ構造でJSONデータを挿入することができます。 この機能を有効にするには [input_format_import_nested_json](../operations/settings/settings.md#settings-input_format_import_nested_json) 設定。 たとえば、次の表を考えてみましょう: @@ -601,7 +601,7 @@ CREATE TABLE json_each_row_nested (n Nested (s String, i Int32) ) ENGINE = Memor INSERT INTO json_each_row_nested FORMAT JSONEachRow {"n.s": ["abc", "def"], "n.i": [1, 23]} ``` -挿入データとしての階層JSONオブジェクト [input\_format\_import\_nested\_json=1](../operations/settings/settings.md#settings-input_format_import_nested_json). +挿入データとしての階層JSONオブジェクト [input_format_import_nested_json=1](../operations/settings/settings.md#settings-input_format_import_nested_json). ``` json { @@ -783,7 +783,7 @@ The minimum set of characters that you need to escape when passing data in Value これはで使用される形式です `INSERT INTO t VALUES ...` ただし、クエリ結果の書式設定にも使用できます。 -も参照。: [input\_format\_values\_interpret\_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions) と [input\_format\_values\_deduce\_templates\_of\_expressions](../operations/settings/settings.md#settings-input_format_values_deduce_templates_of_expressions) 設定。 +も参照。: [input_format_values_interpret_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions) と [input_format_values_deduce_templates_of_expressions](../operations/settings/settings.md#settings-input_format_values_deduce_templates_of_expressions) 設定。 ## 垂直 {#vertical} @@ -1046,7 +1046,7 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Avro" > file.avro - で始まる `[A-Za-z_]` - その後のみ `[A-Za-z0-9_]` -出力Avroファイルの圧縮および同期間隔は以下で設定できます [output\_format\_avro\_codec](../operations/settings/settings.md#settings-output_format_avro_codec) と [output\_format\_avro\_sync\_interval](../operations/settings/settings.md#settings-output_format_avro_sync_interval) それぞれ。 +出力Avroファイルの圧縮および同期間隔は以下で設定できます [output_format_avro_codec](../operations/settings/settings.md#settings-output_format_avro_codec) と [output_format_avro_sync_interval](../operations/settings/settings.md#settings-output_format_avro_sync_interval) それぞれ。 ## アブロコンフルエント {#data-format-avro-confluent} @@ -1056,7 +1056,7 @@ AvroConfluent支援復号単一のオブジェクトアブロのメッセージ スキーマがキャッシュ一度に解決されます。 -スキーマのレジストリのURLは設定され [format\_avro\_schema\_registry\_url](../operations/settings/settings.md#settings-format_avro_schema_registry_url) +スキーマのレジストリのURLは設定され [format_avro_schema_registry_url](../operations/settings/settings.md#settings-format_avro_schema_registry_url) ### データ型の一致 {#data_types-matching-1} @@ -1199,13 +1199,13 @@ e.g. `schemafile.proto:MessageType`. クライアントを使用する場合 [バッチモード](../interfaces/cli.md#cli_usage) は、パスのスキーマ"相対的"に指定する必要があります。 を介してデータを入力または出力する場合 [HTTPインターフェ](../interfaces/http.md) 形式スキーマで指定されたファイル名 -指定されたディレクトリにあるはずです。 [format\_schema\_path](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-format_schema_path) +指定されたディレクトリにあるはずです。 [format_schema_path](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-format_schema_path) サーバー構成で。 ## スキップエラー {#skippingerrors} -次のような形式があります `CSV`, `TabSeparated`, `TSKV`, `JSONEachRow`, `Template`, `CustomSeparated` と `Protobuf` 解析エラーが発生した場合に壊れた行をスキップし、次の行の先頭から解析を続行できます。 見る [input\_format\_allow\_errors\_num](../operations/settings/settings.md#settings-input_format_allow_errors_num) と -[input\_format\_allow\_errors\_ratio](../operations/settings/settings.md#settings-input_format_allow_errors_ratio) 設定。 +次のような形式があります `CSV`, `TabSeparated`, `TSKV`, `JSONEachRow`, `Template`, `CustomSeparated` と `Protobuf` 解析エラーが発生した場合に壊れた行をスキップし、次の行の先頭から解析を続行できます。 見る [input_format_allow_errors_num](../operations/settings/settings.md#settings-input_format_allow_errors_num) と +[input_format_allow_errors_ratio](../operations/settings/settings.md#settings-input_format_allow_errors_ratio) 設定。 制限: -解析エラーの場合 `JSONEachRow` 新しい行(またはEOF)まですべてのデータをスキップします。 `\n` エラーを正しく数える。 - `Template` と `CustomSeparated` 最後の列の後にdelimiterを使用し、行の間にdelimiterを使用すると、次の行の先頭を見つけることができます。 diff --git a/docs/ja/interfaces/http.md b/docs/ja/interfaces/http.md index 31f2b54af6d..79c4ba372ee 100644 --- a/docs/ja/interfaces/http.md +++ b/docs/ja/interfaces/http.md @@ -11,7 +11,7 @@ HTTPイトのご利用ClickHouseにプラットフォームからゆるプログ デフォルトでは、clickhouse-serverはポート8123でHTTPをリッスンします(これは設定で変更できます)。 -パラメータなしでGET/requestを行うと、200の応答コードとで定義された文字列が返されます [http\_server\_default\_response](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-http_server_default_response) デフォルト値 “Ok.” (最後に改行があります) +パラメータなしでGET/requestを行うと、200の応答コードとで定義された文字列が返されます [http_server_default_response](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-http_server_default_response) デフォルト値 “Ok.” (最後に改行があります) ``` bash $ curl 'http://localhost:8123/' @@ -147,12 +147,12 @@ $ echo 'DROP TABLE t' | curl 'http://localhost:8123/' --data-binary @- データテーブルを返さない正常な要求の場合、空の応答本文が返されます。 -データを送信するときは、内部ClickHouse圧縮形式を使用できます。 圧縮されたデータは非標準形式であり、特別な形式を使用する必要があります `clickhouse-compressor` それで動作するようにプログラム(それは `clickhouse-client` パッケージ)。 データ挿入の効率を高めるために、以下を使用してサーバー側のチェックサム検証を無効にできます [http\_native\_compression\_disable\_checksumming\_on\_decompress](../operations/settings/settings.md#settings-http_native_compression_disable_checksumming_on_decompress) 設定。 +データを送信するときは、内部ClickHouse圧縮形式を使用できます。 圧縮されたデータは非標準形式であり、特別な形式を使用する必要があります `clickhouse-compressor` それで動作するようにプログラム(それは `clickhouse-client` パッケージ)。 データ挿入の効率を高めるために、以下を使用してサーバー側のチェックサム検証を無効にできます [http_native_compression_disable_checksumming_on_decompress](../operations/settings/settings.md#settings-http_native_compression_disable_checksumming_on_decompress) 設定。 指定した場合 `compress=1` URLでは、サーバーが送信するデータを圧縮します。 指定した場合 `decompress=1` このURLでは、サーバーは渡すデータと同じデータを解凍します。 `POST` 方法。 -また、使用することもできます [HTTP圧縮](https://en.wikipedia.org/wiki/HTTP_compression). 圧縮を送信するには `POST` リクエストヘッダーを追加します `Content-Encoding: compression_method`. ClickHouseが応答を圧縮するには、次のように追加する必要があります `Accept-Encoding: compression_method`. ClickHouseサポート `gzip`, `br`,and `deflate` [圧縮方法](https://en.wikipedia.org/wiki/HTTP_compression#Content-Encoding_tokens). HTTP圧縮を有効にするには、ClickHouseを使用する必要があります [enable\_http\_compression](../operations/settings/settings.md#settings-enable_http_compression) 設定。 データ圧縮レベルは、 [http\_zlib\_compression\_level](#settings-http_zlib_compression_level) すべての圧縮方法の設定。 +また、使用することもできます [HTTP圧縮](https://en.wikipedia.org/wiki/HTTP_compression). 圧縮を送信するには `POST` リクエストヘッダーを追加します `Content-Encoding: compression_method`. ClickHouseが応答を圧縮するには、次のように追加する必要があります `Accept-Encoding: compression_method`. ClickHouseサポート `gzip`, `br`,and `deflate` [圧縮方法](https://en.wikipedia.org/wiki/HTTP_compression#Content-Encoding_tokens). HTTP圧縮を有効にするには、ClickHouseを使用する必要があります [enable_http_compression](../operations/settings/settings.md#settings-enable_http_compression) 設定。 データ圧縮レベルは、 [http_zlib_compression_level](#settings-http_zlib_compression_level) すべての圧縮方法の設定。 利用することができ削減ネットワーク通信の送受信には大量のデータをダンプすると直ちに圧縮されます。 @@ -214,7 +214,7 @@ $ echo 'SELECT 1' | curl -H 'X-ClickHouse-User: user' -H 'X-ClickHouse-Key: pass ``` ユーザ名が指定されていない場合、 `default` 名前が使用されます。 パスワードを指定しない場合は、空のパスワードが使用されます。 -にお使いいただけますURLパラメータで指定した設定処理の単一クエリーまたは全体をプロファイルを設定します。 例:http://localhost:8123/?profile=web&max\_rows\_to\_read=1000000000&query=SELECT+1 +にお使いいただけますURLパラメータで指定した設定処理の単一クエリーまたは全体をプロファイルを設定します。 例:http://localhost:8123/?profile=web&max_rows_to_read=1000000000&query=SELECT+1 詳細については、を参照してください [設定](../operations/settings/index.md) セクション @@ -236,7 +236,7 @@ $ echo 'SELECT number FROM system.numbers LIMIT 10' | curl 'http://localhost:812 同様に、HttpプロトコルでClickHouseセッションを使用できます。 これを行うには、 `session_id` 要求のパラメータを取得します。 セッションIDとして任意の文字列を使用できます。 既定では、セッションは非アクティブの60秒後に終了します。 このタイムアウトを変更するには、 `default_session_timeout` サーバー構成で設定するか、または `session_timeout` 要求のパラメータを取得します。 セッションステータスを確認するには、 `session_check=1` パラメータ。 単一のセッション内で一度に一つのクエリだけを実行できます。 -クエリの進行状況に関する情報を受け取ることができます `X-ClickHouse-Progress` 応答ヘッダー。 これを行うには、有効にします [send\_progress\_in\_http\_headers](../operations/settings/settings.md#settings-send_progress_in_http_headers). ヘッダシーケンスの例: +クエリの進行状況に関する情報を受け取ることができます `X-ClickHouse-Progress` 応答ヘッダー。 これを行うには、有効にします [send_progress_in_http_headers](../operations/settings/settings.md#settings-send_progress_in_http_headers). ヘッダシーケンスの例: ``` text X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128"} @@ -253,9 +253,9 @@ X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_ro - `written_bytes` — Volume of data written in bytes. HTTP接続が失われても、要求の実行は自動的に停止しません。 解析とデータの書式設定はサーバー側で実行され、ネットワークを使用すると無効になる可能性があります。 -任意 ‘query\_id’ パラメータは、クエリID(任意の文字列)として渡すことができます。 詳細については “Settings, replace\_running\_query”. +任意 ‘query_id’ パラメータは、クエリID(任意の文字列)として渡すことができます。 詳細については “Settings, replace_running_query”. -任意 ‘quota\_key’ パラメータとして渡すことができ、クォーターキー(切文字列). 詳細については “Quotas”. +任意 ‘quota_key’ パラメータとして渡すことができ、クォーターキー(切文字列). 詳細については “Quotas”. HTTPイ 詳細については “External data for query processing”. @@ -377,11 +377,11 @@ $ curl -v 'http://localhost:8123/predefined_query' > `` HTTPリクエストのヘッダー部分の照合を担当します。 RE2の正規表現と互換性があります。 これはオプションの構成です。 構成ファイルで定義されていない場合、HTTP要求のヘッダー部分と一致しません。 > > `` 主要な処理の部品を含んでいます。 さて `` 構成できます ``, ``, ``, ``, ``, ``. -> \> `` 現在サポート: **predefined\_query\_handler**, **dynamic\_query\_handler**, **静的**. +> \> `` 現在サポート: **predefined_query_handler**, **dynamic_query_handler**, **静的**. > \> -> \> `` -predefined\_query\_handler型で使用し、ハンドラが呼び出されたときにクエリを実行します。 +> \> `` -predefined_query_handler型で使用し、ハンドラが呼び出されたときにクエリを実行します。 > \> -> \> `` -dynamic\_query\_handler型で使用すると、それに対応する値を抽出して実行します。 `` HTTP要求パラメータの値。 +> \> `` -dynamic_query_handler型で使用すると、それに対応する値を抽出して実行します。 `` HTTP要求パラメータの値。 > \> > \> `` -静的タイプ、応答ステータスコードで使用します。 > \> @@ -391,9 +391,9 @@ $ curl -v 'http://localhost:8123/predefined_query' 次に、異なる設定方法を示します ``. -## predefined\_query\_handler {#predefined_query_handler} +## predefined_query_handler {#predefined_query_handler} -`` 設定とquery\_params値の設定をサポートします。 設定できます `` のタイプで ``. +`` 設定とquery_params値の設定をサポートします。 設定できます `` のタイプで ``. `` 値は以下の定義済みクエリです `` これは、Http要求が一致し、クエリの結果が返されたときにClickHouseによって実行されます。 これは必須構成です。 @@ -428,13 +428,13 @@ max_alter_threads 2 !!! note "注意" 一つで `` 一つだけをサポート `` 挿入タイプ。 -## dynamic\_query\_handler {#dynamic_query_handler} +## dynamic_query_handler {#dynamic_query_handler} で ``、クエリは、HTTP要求のparamの形式で書かれています。 違いは、 ``、クエリは設定ファイルに書き込まれます。 設定できます `` で ``. クリックハウスは、 `` HTTP要求のurlの値。 のデフォルト値 `` は `/query` . これはオプションの構成です。 設定ファイルに定義がない場合、paramは渡されません。 -この機能を試すために、この例ではmax\_threadsとmax\_alter\_threadsの値を定義し、設定が正常に設定されたかどうかを照会します。 +この機能を試すために、この例ではmax_threadsとmax_alter_threadsの値を定義し、設定が正常に設定されたかどうかを照会します。 例: @@ -459,7 +459,7 @@ max_alter_threads 2 ## 静的 {#static} -`` 戻れる [content\_type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type), [状態](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status) そしてresponse\_content。 response\_contentは、指定された内容を返すことができます +`` 戻れる [content_type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type), [状態](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status) そしてresponse_content。 response_contentは、指定された内容を返すことができます 例: diff --git a/docs/ja/interfaces/mysql.md b/docs/ja/interfaces/mysql.md index 49b20339f3e..21a3f0fb98a 100644 --- a/docs/ja/interfaces/mysql.md +++ b/docs/ja/interfaces/mysql.md @@ -7,7 +7,7 @@ toc_title: "MySQL\u30A4\u30F3\u30BF" # MySQLインタ {#mysql-interface} -ClickHouseはMySQL wire protocolをサポートしています。 で有効にすることができる [mysql\_port](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-mysql_port) 設定ファイルでの設定: +ClickHouseはMySQL wire protocolをサポートしています。 で有効にすることができる [mysql_port](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-mysql_port) 設定ファイルでの設定: ``` xml 9004 diff --git a/docs/ja/interfaces/third-party/client-libraries.md b/docs/ja/interfaces/third-party/client-libraries.md index b88fe0c7389..ffe7b641c38 100644 --- a/docs/ja/interfaces/third-party/client-libraries.md +++ b/docs/ja/interfaces/third-party/client-libraries.md @@ -11,7 +11,7 @@ toc_title: "\u30AF\u30E9\u30A4\u30A2\u30F3\u30C8" Yandexのは **ない** 以下のライブラリを維持し、その品質を保証するための広範なテストを行っていません。 - Python - - [インフィclickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm) + - [インフィclickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm) - [clickhouse-ドライバ](https://github.com/mymarilyn/clickhouse-driver) - [clickhouse-クライアント](https://github.com/yurial/clickhouse-client) - [aiochclient](https://github.com/maximdanilchenko/aiochclient) @@ -47,7 +47,7 @@ toc_title: "\u30AF\u30E9\u30A4\u30A2\u30F3\u30C8" - [clickhouse-scala-クライアント](https://github.com/crobox/clickhouse-scala-client) - コトリン - [AORM](https://github.com/TanVD/AORM) -- C\# +- C# - [Octonica.ClickHouseClient](https://github.com/Octonica/ClickHouseClient) - [クリックハウスAdo](https://github.com/killwort/ClickHouse-Net) - [クリックハウスクライアン](https://github.com/DarkWanderer/ClickHouse.Client) diff --git a/docs/ja/interfaces/third-party/integrations.md b/docs/ja/interfaces/third-party/integrations.md index f88adfd4f4f..851eeec665f 100644 --- a/docs/ja/interfaces/third-party/integrations.md +++ b/docs/ja/interfaces/third-party/integrations.md @@ -19,15 +19,15 @@ toc_title: "\u7D71\u5408" - [clickhouse-mysql-データリーダー](https://github.com/Altinity/clickhouse-mysql-data-reader) - [horgh-レプリケーター](https://github.com/larsnovikov/horgh-replicator) - [PostgreSQL](https://www.postgresql.org) - - [clickhousedb\_fdw](https://github.com/Percona-Lab/clickhousedb_fdw) - - [インフィclickhouse\_fdw](https://github.com/Infinidat/infi.clickhouse_fdw) (用途 [インフィclickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) + - [clickhousedb_fdw](https://github.com/Percona-Lab/clickhousedb_fdw) + - [インフィclickhouse_fdw](https://github.com/Infinidat/infi.clickhouse_fdw) (用途 [インフィclickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - [pg2ch](https://github.com/mkabilov/pg2ch) - - [clickhouse\_fdw](https://github.com/adjust/clickhouse_fdw) + - [clickhouse_fdw](https://github.com/adjust/clickhouse_fdw) - [MSSQL](https://en.wikipedia.org/wiki/Microsoft_SQL_Server) - [ClickHouseMigrator](https://github.com/zlzforever/ClickHouseMigrator) - メッセージキュ - [カフカ](https://kafka.apache.org) - - [clickhouse\_sinker](https://github.com/housepower/clickhouse_sinker) (用途 [Goクライアント](https://github.com/ClickHouse/clickhouse-go/)) + - [clickhouse_sinker](https://github.com/housepower/clickhouse_sinker) (用途 [Goクライアント](https://github.com/ClickHouse/clickhouse-go/)) - [stream-loader-clickhouse](https://github.com/adform/stream-loader) - ストリーム処理 - [フリンク](https://flink.apache.org) @@ -51,12 +51,12 @@ toc_title: "\u7D71\u5408" - [グラファナ](https://grafana.com/) - [クリックハウス-グラファナ](https://github.com/Vertamedia/clickhouse-grafana) - [プロメテウス](https://prometheus.io/) - - [clickhouse\_exporter](https://github.com/f1yegor/clickhouse_exporter) + - [clickhouse_exporter](https://github.com/f1yegor/clickhouse_exporter) - [プロムハウス](https://github.com/Percona-Lab/PromHouse) - - [clickhouse\_exporter](https://github.com/hot-wifi/clickhouse_exporter) (用途 [Goクライアント](https://github.com/kshvakov/clickhouse/)) + - [clickhouse_exporter](https://github.com/hot-wifi/clickhouse_exporter) (用途 [Goクライアント](https://github.com/kshvakov/clickhouse/)) - [ナギオス](https://www.nagios.org/) - - [check\_clickhouse](https://github.com/exogroup/check_clickhouse/) - - [check\_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py) + - [check_clickhouse](https://github.com/exogroup/check_clickhouse/) + - [check_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py) - [Zabbix](https://www.zabbix.com) - [clickhouse-zabbix-テンプレート](https://github.com/Altinity/clickhouse-zabbix-template) - [Sematext](https://sematext.com/) @@ -76,7 +76,7 @@ toc_title: "\u7D71\u5408" - Python - [SQLAlchemy](https://www.sqlalchemy.org) - - [sqlalchemy-クリックハウス](https://github.com/cloudflare/sqlalchemy-clickhouse) (用途 [インフィclickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) + - [sqlalchemy-クリックハウス](https://github.com/cloudflare/sqlalchemy-clickhouse) (用途 [インフィclickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - [パンダ](https://pandas.pydata.org) - [パンダハウス](https://github.com/kszucs/pandahouse) - PHP @@ -91,7 +91,7 @@ toc_title: "\u7D71\u5408" - Scala - [アッカ](https://akka.io) - [clickhouse-scala-クライアント](https://github.com/crobox/clickhouse-scala-client) -- C\# +- C# - [ADO.NET](https://docs.microsoft.com/en-us/dotnet/framework/data/adonet/ado-net-overview) - [クリックハウスAdo](https://github.com/killwort/ClickHouse-Net) - [クリックハウスクライアン](https://github.com/DarkWanderer/ClickHouse.Client) @@ -99,7 +99,7 @@ toc_title: "\u7D71\u5408" - [ClickHouse.Net.Migrations](https://github.com/ilyabreev/ClickHouse.Net.Migrations) - エリクサー - [Ecto](https://github.com/elixir-ecto/ecto) - - [clickhouse\_ecto](https://github.com/appodeal/clickhouse_ecto) + - [clickhouse_ecto](https://github.com/appodeal/clickhouse_ecto) - Ruby - [Ruby on rails](https://rubyonrails.org/) - [activecube](https://github.com/bitquery/activecube) diff --git a/docs/ja/operations/access-rights.md b/docs/ja/operations/access-rights.md index e02e7529ecb..c3fea6bfe5c 100644 --- a/docs/ja/operations/access-rights.md +++ b/docs/ja/operations/access-rights.md @@ -135,10 +135,10 @@ Roleは、ユーザーアカウントに付与できるaccessエンティティ - 設定ディレクトリ構成を保管します。 - ClickHouseは、アクセスエンティティ設定を [access\_control\_path](server-configuration-parameters/settings.md#access_control_path) サーバー構成パラメータ。 + ClickHouseは、アクセスエンティティ設定を [access_control_path](server-configuration-parameters/settings.md#access_control_path) サーバー構成パラメータ。 - SQL駆動型のアクセス制御とアカウント管理を有効にします。 - デフォルトのSQL型のアクセス制御及び特別口座の口座管理オのすべてのユーザー ユーザーを設定する必要があります。 `users.xml` に1を割り当てます。 [access\_management](settings/settings-users.md#access_management-user-setting) 設定。 + デフォルトのSQL型のアクセス制御及び特別口座の口座管理オのすべてのユーザー ユーザーを設定する必要があります。 `users.xml` に1を割り当てます。 [access_management](settings/settings-users.md#access_management-user-setting) 設定。 [元の記事](https://clickhouse.tech/docs/en/operations/access_rights/) diff --git a/docs/ja/operations/configuration-files.md b/docs/ja/operations/configuration-files.md index 9457c89e3d9..b269508f408 100644 --- a/docs/ja/operations/configuration-files.md +++ b/docs/ja/operations/configuration-files.md @@ -20,7 +20,7 @@ ClickHouseは複数のファイル構成管理をサポートします。 主サ もし `remove` 指定されると、要素を削除します。 -この設定はまた、 “substitutions”. 要素が `incl` 属性は、ファイルからの対応する置換が値として使用されます。 デフォルトでは、ファイルへのパスとの置換を行う `/etc/metrika.xml`. これはで変えることができます [include\_from](server-configuration-parameters/settings.md#server_configuration_parameters-include_from) サーバー設定の要素。 置換値は、次のように指定されます `/yandex/substitution_name` このファイル内の要素。 で指定された置換の場合 `incl` 存在しない場合は、ログに記録されます。 ClickHouseが不足している置換をログに記録しないようにするには、 `optional="true"` 属性(たとえば、 [マクロ](server-configuration-parameters/settings.md)). +この設定はまた、 “substitutions”. 要素が `incl` 属性は、ファイルからの対応する置換が値として使用されます。 デフォルトでは、ファイルへのパスとの置換を行う `/etc/metrika.xml`. これはで変えることができます [include_from](server-configuration-parameters/settings.md#server_configuration_parameters-include_from) サーバー設定の要素。 置換値は、次のように指定されます `/yandex/substitution_name` このファイル内の要素。 で指定された置換の場合 `incl` 存在しない場合は、ログに記録されます。 ClickHouseが不足している置換をログに記録しないようにするには、 `optional="true"` 属性(たとえば、 [マクロ](server-configuration-parameters/settings.md)). 置換はZooKeeperからも実行できます。 これを行うには、属性を指定します `from_zk = "/path/to/node"`. 要素の値は、ノードの内容に置き換えられます。 `/path/to/node` 飼育係で。 また、ZooKeeperノードにXMLサブツリー全体を配置することもできます。 diff --git a/docs/ja/operations/monitoring.md b/docs/ja/operations/monitoring.md index 0d2ecd7f5b4..6bbe4958caf 100644 --- a/docs/ja/operations/monitoring.md +++ b/docs/ja/operations/monitoring.md @@ -35,7 +35,7 @@ ClickHouse serverには、自己状態の監視のための計測器が組み込 - 異なるメトリクスのサーバがどのように利用計算資源です。 - クエリ処理に関する一般的な統計。 -メトリックは、次のとおりです。 [システムメトリック](../operations/system-tables.md#system_tables-metrics), [システムイベント](../operations/system-tables.md#system_tables-events),and [システムasynchronous\_metrics](../operations/system-tables.md#system_tables-asynchronous_metrics) テーブル +メトリックは、次のとおりです。 [システムメトリック](../operations/system-tables.md#system_tables-metrics), [システムイベント](../operations/system-tables.md#system_tables-events),and [システムasynchronous_metrics](../operations/system-tables.md#system_tables-asynchronous_metrics) テーブル を設定することができClickHouse輸出の指標に [黒鉛](https://github.com/graphite-project). を参照。 [グラファイト部](server-configuration-parameters/settings.md#server_configuration_parameters-graphite) ClickHouseサーバー設定ファイル内。 指標のエクスポートを設定する前に、公式に従ってGraphiteを設定する必要があります [ガイド](https://graphite.readthedocs.io/en/latest/install.html). @@ -43,4 +43,4 @@ ClickHouse serverには、自己状態の監視のための計測器が組み込 さらに、HTTP APIを使用してサーバーの可用性を監視できます。 送信 `HTTP GET` リクエスト先 `/ping`. サーバーが利用可能な場合は、次のように応答します `200 OK`. -監視サーバーにクラスター構成設定してください [max\_replica\_delay\_for\_distributed\_queries](settings/settings.md#settings-max_replica_delay_for_distributed_queries) パラメータとHTTPリソースの使用 `/replicas_status`. への要求 `/replicas_status` ツづゥツ。 `200 OK` レプリカが使用可能で、他のレプリカより遅れていない場合。 レプリカが遅延すると、次のようになります `503 HTTP_SERVICE_UNAVAILABLE` ギャップについての情報と。 +監視サーバーにクラスター構成設定してください [max_replica_delay_for_distributed_queries](settings/settings.md#settings-max_replica_delay_for_distributed_queries) パラメータとHTTPリソースの使用 `/replicas_status`. への要求 `/replicas_status` ツづゥツ。 `200 OK` レプリカが使用可能で、他のレプリカより遅れていない場合。 レプリカが遅延すると、次のようになります `503 HTTP_SERVICE_UNAVAILABLE` ギャップについての情報と。 diff --git a/docs/ja/operations/optimizing-performance/sampling-query-profiler.md b/docs/ja/operations/optimizing-performance/sampling-query-profiler.md index e88b62ce22c..120830a3e16 100644 --- a/docs/ja/operations/optimizing-performance/sampling-query-profiler.md +++ b/docs/ja/operations/optimizing-performance/sampling-query-profiler.md @@ -11,11 +11,11 @@ ClickHouse運転サンプリングプロファイラでの分析クエリを実 Profilerを使用するには: -- セットアップ [trace\_log](../server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) サーバー構成のセクション。 +- セットアップ [trace_log](../server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) サーバー構成のセクション。 - このセクションでは、 [trace\_log](../../operations/system-tables.md#system_tables-trace_log) プロファイラ機能の結果を含むシステムテーブル。 これは既定で構成されています。 この表のデータは、実行中のサーバーに対してのみ有効です。 後、サーバを再起動ClickHouseないクリーンのテーブルに格納された仮想メモリアドレスが無効になります。 + このセクションでは、 [trace_log](../../operations/system-tables.md#system_tables-trace_log) プロファイラ機能の結果を含むシステムテーブル。 これは既定で構成されています。 この表のデータは、実行中のサーバーに対してのみ有効です。 後、サーバを再起動ClickHouseないクリーンのテーブルに格納された仮想メモリアドレスが無効になります。 -- セットアップ [query\_profiler\_cpu\_time\_period\_ns](../settings/settings.md#query_profiler_cpu_time_period_ns) または [query\_profiler\_real\_time\_period\_ns](../settings/settings.md#query_profiler_real_time_period_ns) 設定。 両方の設定を同時に使用できます。 +- セットアップ [query_profiler_cpu_time_period_ns](../settings/settings.md#query_profiler_cpu_time_period_ns) または [query_profiler_real_time_period_ns](../settings/settings.md#query_profiler_real_time_period_ns) 設定。 両方の設定を同時に使用できます。 これらの設定を許可する設定プロファイラータイマー. これらはセッション設定であるため、サーバー全体、個々のユーザーまたはユーザープロファイル、対話式セッション、および個々のクエリごとに異なるサンプリング周波数 @@ -25,7 +25,7 @@ Profilerを使用するには: - インストール `clickhouse-common-static-dbg` パッケージ。 見る [DEBパッケージから](../../getting-started/install.md#install-from-deb-packages). -- によるイントロスペクション関数を許可する。 [allow\_introspection\_functions](../settings/settings.md#settings-allow_introspection_functions) 設定。 +- によるイントロスペクション関数を許可する。 [allow_introspection_functions](../settings/settings.md#settings-allow_introspection_functions) 設定。 セキュリティ上の理由から、introspection関数は既定で無効になっています。 diff --git a/docs/ja/operations/requirements.md b/docs/ja/operations/requirements.md index f4998bda005..f354d265c48 100644 --- a/docs/ja/operations/requirements.md +++ b/docs/ja/operations/requirements.md @@ -9,7 +9,7 @@ toc_title: "\u8981\u4EF6" ## CPU {#cpu} -ビルド済みのdebパッケージからインストールするには、X86\_64アーキテクチャのCPUを使用し、SSE4.2命令をサポートします。 走ClickHouseプロセッサをサポートしていないSSE4.2てAArch64はPowerPC64LE建築、協力して進めることが必要でありClickHouseから。 +ビルド済みのdebパッケージからインストールするには、X86_64アーキテクチャのCPUを使用し、SSE4.2命令をサポートします。 走ClickHouseプロセッサをサポートしていないSSE4.2てAArch64はPowerPC64LE建築、協力して進めることが必要でありClickHouseから。 ClickHouseを実装した並列データの処理-利用のすべてのハードウェア資料を備えています。 プロセッサを選択するときは、ClickHouseが多数のコアを持つ構成でより効率的に動作するが、コアが少なくクロックレートが高い構成ではatよりも低いク たとえば、16 2600MHzのコアは、8 3600MHzのコアよりも好ましいです。 diff --git a/docs/ja/operations/server-configuration-parameters/settings.md b/docs/ja/operations/server-configuration-parameters/settings.md index 98a31fa5f60..0ec71b2af69 100644 --- a/docs/ja/operations/server-configuration-parameters/settings.md +++ b/docs/ja/operations/server-configuration-parameters/settings.md @@ -7,7 +7,7 @@ toc_title: "\u30B5\u30FC\u30D0\u30FC\u8A2D\u5B9A" # サーバー設定 {#server-settings} -## builtin\_dictionaries\_reload\_interval {#builtin-dictionaries-reload-interval} +## builtin_dictionaries_reload_interval {#builtin-dictionaries-reload-interval} 組み込み辞書を再ロードするまでの秒単位の間隔。 @@ -68,7 +68,7 @@ ClickHouseはx秒ごとに組み込みの辞書を再読み込みします。 ``` -## default\_database {#default-database} +## default_database {#default-database} 既定のデータベース。 @@ -80,7 +80,7 @@ ClickHouseはx秒ごとに組み込みの辞書を再読み込みします。 default ``` -## default\_profile {#default-profile} +## default_profile {#default-profile} 既定の設定プロファイル。 @@ -92,7 +92,7 @@ ClickHouseはx秒ごとに組み込みの辞書を再読み込みします。 default ``` -## dictionaries\_config {#server_configuration_parameters-dictionaries_config} +## dictionaries_config {#server_configuration_parameters-dictionaries_config} 外部辞書の設定ファイルへのパス。 @@ -109,7 +109,7 @@ ClickHouseはx秒ごとに組み込みの辞書を再読み込みします。 *_dictionary.xml ``` -## dictionaries\_lazy\_load {#server_configuration_parameters-dictionaries_lazy_load} +## dictionaries_lazy_load {#server_configuration_parameters-dictionaries_lazy_load} 辞書の遅延読み込み。 @@ -125,7 +125,7 @@ ClickHouseはx秒ごとに組み込みの辞書を再読み込みします。 true ``` -## format\_schema\_path {#server_configuration_parameters-format_schema_path} +## format_schema_path {#server_configuration_parameters-format_schema_path} 入力データのスキーマを含むディレクトリへのパス。 [CapnProto](../../interfaces/formats.md#capnproto) 形式。 @@ -146,11 +146,11 @@ ClickHouseはx秒ごとに組み込みの辞書を再読み込みします。 - port – The port on the Graphite server. - interval – The interval for sending, in seconds. - timeout – The timeout for sending data, in seconds. -- root\_path – Prefix for keys. +- root_path – Prefix for keys. - metrics – Sending data from the [システムメトリック](../../operations/system-tables.md#system_tables-metrics) テーブル。 - events – Sending deltas data accumulated for the time period from the [システムイベント](../../operations/system-tables.md#system_tables-events) テーブル。 -- events\_cumulative – Sending cumulative data from the [システムイベント](../../operations/system-tables.md#system_tables-events) テーブル。 -- asynchronous\_metrics – Sending data from the [システムasynchronous\_metrics](../../operations/system-tables.md#system_tables-asynchronous_metrics) テーブル。 +- events_cumulative – Sending cumulative data from the [システムイベント](../../operations/system-tables.md#system_tables-events) テーブル。 +- asynchronous_metrics – Sending data from the [システムasynchronous_metrics](../../operations/system-tables.md#system_tables-asynchronous_metrics) テーブル。 複数の設定が可能です `` 句。 たとえば、異なる間隔で異なるデータを送信するためにこれを使用できます。 @@ -170,7 +170,7 @@ ClickHouseはx秒ごとに組み込みの辞書を再読み込みします。 ``` -## graphite\_rollup {#server_configuration_parameters-graphite-rollup} +## graphite_rollup {#server_configuration_parameters-graphite-rollup} グラファイトのデータを薄くする設定。 @@ -198,7 +198,7 @@ ClickHouseはx秒ごとに組み込みの辞書を再読み込みします。 ``` -## http\_port/https\_port {#http-porthttps-port} +## http_port/https_port {#http-porthttps-port} HTTP経由でサーバーに接続するためのポート。 @@ -212,7 +212,7 @@ HTTP経由でサーバーに接続するためのポート。 9999 ``` -## http\_server\_default\_response {#server_configuration_parameters-http_server_default_response} +## http_server_default_response {#server_configuration_parameters-http_server_default_response} ClickHouse HTTP(s)サーバーにアクセスするときにデフォルトで表示されるページ。 既定値は次のとおりです “Ok.” (最後に改行があります) @@ -227,7 +227,7 @@ ClickHouse HTTP(s)サーバーにアクセスするときにデフォルトで ``` -## include\_from {#server_configuration_parameters-include_from} +## include_from {#server_configuration_parameters-include_from} 置換されたファイルへのパス。 @@ -239,7 +239,7 @@ ClickHouse HTTP(s)サーバーにアクセスするときにデフォルトで /etc/metrica.xml ``` -## interserver\_http\_port {#interserver-http-port} +## interserver_http_port {#interserver-http-port} ClickHouseサーバー間でデータを交換するポート。 @@ -249,7 +249,7 @@ ClickHouseサーバー間でデータを交換するポート。 9009 ``` -## interserver\_http\_host {#interserver-http-host} +## interserver_http_host {#interserver-http-host} このサーバーへのアクセスに他のサーバーが使用できるホスト名。 @@ -263,7 +263,7 @@ ClickHouseサーバー間でデータを交換するポート。 example.yandex.ru ``` -## interserver\_http\_credentials {#server-settings-interserver-http-credentials} +## interserver_http_credentials {#server-settings-interserver-http-credentials} 認証時に使用されるユーザー名とパスワード [複製](../../engines/table-engines/mergetree-family/replication.md) 複製された\*エンジンで。 これらの資格情報は、レプリカ間の通信にのみ使用され、ClickHouseクライアントの資格情報とは無関係です。 サーバーにあるチェックにこれらの資格の接続にはレプリカと同じ資格を接続する場合はその他のレプリカ. なので、これらの資格を設定する同じすべてのレプリカ、クラスター 既定では、認証は使用されません。 @@ -282,7 +282,7 @@ ClickHouseサーバー間でデータを交換するポート。 ``` -## keep\_alive\_timeout {#keep-alive-timeout} +## keep_alive_timeout {#keep-alive-timeout} ClickHouseが接続を閉じる前に受信要求を待機する秒数。 既定値は3秒です。 @@ -292,7 +292,7 @@ ClickHouseが接続を閉じる前に受信要求を待機する秒数。 既定 3 ``` -## listen\_host {#server_configuration_parameters-listen_host} +## listen_host {#server_configuration_parameters-listen_host} 要求元のホストに対する制限。 したい場合はサーバーの回答をしているが、それらを指定し `::`. @@ -343,10 +343,10 @@ Syslogへの書き込みもサポートされています。 設定の例: キー: -- use\_syslog — Required setting if you want to write to the syslog. +- use_syslog — Required setting if you want to write to the syslog. - address — The host\[:port\] of syslogd. If omitted, the local daemon is used. - hostname — Optional. The name of the host that logs are sent from. -- facility — [Syslog機能キーワード](https://en.wikipedia.org/wiki/Syslog#Facility) 大文字では “LOG\_” 接頭辞: (`LOG_USER`, `LOG_DAEMON`, `LOG_LOCAL3`、というように)。 +- facility — [Syslog機能キーワード](https://en.wikipedia.org/wiki/Syslog#Facility) 大文字では “LOG_” 接頭辞: (`LOG_USER`, `LOG_DAEMON`, `LOG_LOCAL3`、というように)。 デフォルト値: `LOG_USER` もし `address` 指定される, `LOG_DAEMON otherwise.` - format – Message format. Possible values: `bsd` と `syslog.` @@ -364,7 +364,7 @@ Syslogへの書き込みもサポートされています。 設定の例: ``` -## mark\_cache\_size {#server-mark-cache-size} +## mark_cache_size {#server-mark-cache-size} テーブルエンジンが使用するマークのキャッシュのおおよそのサイズ(バイト単位) [メルゲツリー](../../engines/table-engines/mergetree-family/mergetree.md) 家族だ @@ -376,7 +376,7 @@ Syslogへの書き込みもサポートされています。 設定の例: 5368709120 ``` -## max\_concurrent\_queries {#max-concurrent-queries} +## max_concurrent_queries {#max-concurrent-queries} 同時に処理される要求の最大数。 @@ -386,7 +386,7 @@ Syslogへの書き込みもサポートされています。 設定の例: 100 ``` -## max\_connections {#max-connections} +## max_connections {#max-connections} 受信接続の最大数。 @@ -396,7 +396,7 @@ Syslogへの書き込みもサポートされています。 設定の例: 4096 ``` -## max\_open\_files {#max-open-files} +## max_open_files {#max-open-files} 開いているファイルの最大数。 @@ -410,7 +410,7 @@ Syslogへの書き込みもサポートされています。 設定の例: 262144 ``` -## max\_table\_size\_to\_drop {#max-table-size-to-drop} +## max_table_size_to_drop {#max-table-size-to-drop} テーブルの削除に関する制限。 @@ -428,7 +428,7 @@ ClickHouseサーバーを再起動せずにテーブルを削除する必要が 0 ``` -## merge\_tree {#server_configuration_parameters-merge_tree} +## merge_tree {#server_configuration_parameters-merge_tree} テーブルの微調整 [メルゲツリー](../../engines/table-engines/mergetree-family/mergetree.md). @@ -463,7 +463,7 @@ SSLのサポートは以下によって提供されます `libpoco` 図書館 - sessionTimeout – Time for caching the session on the server. - extendedVerification – Automatically extended verification of certificates after the session ends. Acceptable values: `true`, `false`. - requireTLSv1 – Require a TLSv1 connection. Acceptable values: `true`, `false`. -- requireTLSv1\_1 – Require a TLSv1.1 connection. Acceptable values: `true`, `false`. +- requireTLSv1_1 – Require a TLSv1.1 connection. Acceptable values: `true`, `false`. - requireTLSv1 – Require a TLSv1.2 connection. Acceptable values: `true`, `false`. - fips – Activates OpenSSL FIPS mode. Supported if the library's OpenSSL version supports FIPS. - privateKeyPassphraseHandler – Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: ``, `KeyFileHandler`, `test`, ``. @@ -501,11 +501,11 @@ SSLのサポートは以下によって提供されます `libpoco` 図書館 ``` -## part\_log {#server_configuration_parameters-part-log} +## part_log {#server_configuration_parameters-part-log} 関連付けられたイベントのログ記録 [メルゲツリー](../../engines/table-engines/mergetree-family/mergetree.md). たとえば、データの追加やマージなどです。 利用できるログを統合アルゴリズムと比較しています。 マージプロセスを視覚化できます。 -クエリは [システムpart\_log](../../operations/system-tables.md#system_tables-part-log) 別のファイルではなく、テーブル。 このテーブルの名前は、 `table` パラメータ(下記参照)。 +クエリは [システムpart_log](../../operations/system-tables.md#system_tables-part-log) 別のファイルではなく、テーブル。 このテーブルの名前は、 `table` パラメータ(下記参照)。 以下のパラメータの設定ロギング: @@ -548,7 +548,7 @@ SSLのサポートは以下によって提供されます `libpoco` 図書館 - `port` – Port for `endpoint`. - `metrics` – Flag that sets to expose metrics from the [システムメトリック](../system-tables.md#system_tables-metrics) テーブル。 - `events` – Flag that sets to expose metrics from the [システムイベント](../system-tables.md#system_tables-events) テーブル。 -- `asynchronous_metrics` – Flag that sets to expose current metrics values from the [システムasynchronous\_metrics](../system-tables.md#system_tables-asynchronous_metrics) テーブル。 +- `asynchronous_metrics` – Flag that sets to expose current metrics values from the [システムasynchronous_metrics](../system-tables.md#system_tables-asynchronous_metrics) テーブル。 **例** @@ -562,11 +562,11 @@ SSLのサポートは以下によって提供されます `libpoco` 図書館 ``` -## query\_log {#server_configuration_parameters-query-log} +## query_log {#server_configuration_parameters-query-log} -クエリをログに記録するための設定 [log\_queries=1](../settings/settings.md) 設定。 +クエリをログに記録するための設定 [log_queries=1](../settings/settings.md) 設定。 -クエリは [システムquery\_log](../../operations/system-tables.md#system_tables-query_log) 別のファイルではなく、テーブル。 テーブルの名前を変更することができます。 `table` パラメータ(下記参照)。 +クエリは [システムquery_log](../../operations/system-tables.md#system_tables-query_log) 別のファイルではなく、テーブル。 テーブルの名前を変更することができます。 `table` パラメータ(下記参照)。 以下のパラメータの設定ロギング: @@ -588,11 +588,11 @@ SSLのサポートは以下によって提供されます `libpoco` 図書館 ``` -## query\_thread\_log {#server_configuration_parameters-query-thread-log} +## query_thread_log {#server_configuration_parameters-query-thread-log} -クエリのスレッドをログに記録するための設定 [log\_query\_threads=1](../settings/settings.md#settings-log-query-threads) 設定。 +クエリのスレッドをログに記録するための設定 [log_query_threads=1](../settings/settings.md#settings-log-query-threads) 設定。 -クエリは [システムquery\_thread\_log](../../operations/system-tables.md#system_tables-query-thread-log) 別のファイルではなく、テーブル。 テーブルの名前を変更することができます。 `table` パラメータ(下記参照)。 +クエリは [システムquery_thread_log](../../operations/system-tables.md#system_tables-query-thread-log) 別のファイルではなく、テーブル。 テーブルの名前を変更することができます。 `table` パラメータ(下記参照)。 以下のパラメータの設定ロギング: @@ -614,9 +614,9 @@ SSLのサポートは以下によって提供されます `libpoco` 図書館 ``` -## trace\_log {#server_configuration_parameters-trace_log} +## trace_log {#server_configuration_parameters-trace_log} -の設定 [trace\_log](../../operations/system-tables.md#system_tables-trace_log) システムテーブル操作。 +の設定 [trace_log](../../operations/system-tables.md#system_tables-trace_log) システムテーブル操作。 パラメータ: @@ -636,7 +636,7 @@ SSLのサポートは以下によって提供されます `libpoco` 図書館 ``` -## query\_masking\_rules {#query-masking-rules} +## query_masking_rules {#query-masking-rules} サーバーログに格納する前に、すべてのログメッセージだけでなく、クエリにも適用されます, `system.query_log`, `system.text_log`, `system.processes` クライアントに送信されたログ。 これにより @@ -667,7 +667,7 @@ SQLクエリからの機密データ漏洩(名前、電子メール、個人な 分散クエリの場合は、各サーバーを個別に構成する必要があります。 ノードはマスクなしで保存されます。 -## remote\_servers {#server-settings-remote-servers} +## remote_servers {#server-settings-remote-servers} によって使用されるクラスタの構成 [分散](../../engines/table-engines/special/distributed.md) テーブルエンジンと `cluster` テーブル関数。 @@ -681,7 +681,7 @@ SQLクエリからの機密データ漏洩(名前、電子メール、個人な **も参照。** -- [skip\_unavailable\_shards](../settings/settings.md#settings-skip_unavailable_shards) +- [skip_unavailable_shards](../settings/settings.md#settings-skip_unavailable_shards) ## タイムゾーン {#server_configuration_parameters-timezone} @@ -697,7 +697,7 @@ UTCタイムゾーンまたは地理的位置(たとえば、Africa/Abidjan)のI Europe/Moscow ``` -## tcp\_port {#server_configuration_parameters-tcp_port} +## tcp_port {#server_configuration_parameters-tcp_port} TCPプロトコル経由でクライアントと通信するポート。 @@ -707,7 +707,7 @@ TCPプロトコル経由でクライアントと通信するポート。 9000 ``` -## tcp\_port\_secure {#server_configuration_parameters-tcp_port_secure} +## tcp_port_secure {#server_configuration_parameters-tcp_port_secure} クライアントとの安全な通信用のTCPポート。 それを使用して [OpenSSL](#server_configuration_parameters-openssl) 設定。 @@ -721,7 +721,7 @@ TCPプロトコル経由でクライアントと通信するポート。 9440 ``` -## mysql\_port {#server_configuration_parameters-mysql_port} +## mysql_port {#server_configuration_parameters-mysql_port} MySQLプロトコ @@ -735,7 +735,7 @@ MySQLプロトコ 9004 ``` -## tmp\_path {#server-settings-tmp_path} +## tmp_path {#server-settings-tmp_path} 大規模なクエリを処理するための一時データへのパス。 @@ -748,7 +748,7 @@ MySQLプロトコ /var/lib/clickhouse/tmp/ ``` -## tmp\_policy {#server-settings-tmp-policy} +## tmp_policy {#server-settings-tmp-policy} ポリシーから [`storage_configuration`](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) 一時ファイルを格納する。 設定されていない場合 [`tmp_path`](#server-settings-tmp_path) それ以外の場合は無視されます。 @@ -759,11 +759,11 @@ MySQLプロトコ - `max_data_part_size_bytes` 無視される なければならない同一数量の政策 -## uncompressed\_cache\_size {#server-settings-uncompressed_cache_size} +## uncompressed_cache_size {#server-settings-uncompressed_cache_size} テーブルエンジンが使用する非圧縮データのキャッシュサイズ(バイト単位) [メルゲツリー](../../engines/table-engines/mergetree-family/mergetree.md). -サーバーの共有キャッシュが一つあります。 メモリが割り当てられます。 キャッシュが使用されるのは [use\_uncompressed\_cache](../settings/settings.md#setting-use_uncompressed_cache) 有効です。 +サーバーの共有キャッシュが一つあります。 メモリが割り当てられます。 キャッシュが使用されるのは [use_uncompressed_cache](../settings/settings.md#setting-use_uncompressed_cache) 有効です。 非圧縮キャッシュは、個々のケースで非常に短いクエリで有利です。 @@ -773,7 +773,7 @@ MySQLプロトコ 8589934592 ``` -## user\_files\_path {#server_configuration_parameters-user_files_path} +## user_files_path {#server_configuration_parameters-user_files_path} ユ テーブル関数で使用されます [ファイル()](../../sql-reference/table-functions/file.md). @@ -783,7 +783,7 @@ MySQLプロトコ /var/lib/clickhouse/user_files/ ``` -## users\_config {#users-config} +## users_config {#users-config} 以下のファイルへのパス: @@ -851,13 +851,13 @@ ClickHouse用飼育係の保存メタデータのレプリカの使用時に再 - [複製](../../engines/table-engines/mergetree-family/replication.md) - [ZooKeeperプログラマガイド](http://zookeeper.apache.org/doc/current/zookeeperProgrammers.html) -## use\_minimalistic\_part\_header\_in\_zookeeper {#server-settings-use_minimalistic_part_header_in_zookeeper} +## use_minimalistic_part_header_in_zookeeper {#server-settings-use_minimalistic_part_header_in_zookeeper} ZooKeeperのデータ部分ヘッダーの格納方法。 この設定は、 `MergeTree` 家族だ 指定できます: -- グローバルに [merge\_tree](#server_configuration_parameters-merge_tree) のセクション `config.xml` ファイル +- グローバルに [merge_tree](#server_configuration_parameters-merge_tree) のセクション `config.xml` ファイル ClickHouseは、サーバー上のすべてのテーブルの設定を使用します。 設定はいつでも変更できます。 既存のテーブルは、設定が変更されると動作を変更します。 @@ -879,21 +879,21 @@ ZooKeeperのデータ部分ヘッダーの格納方法。 **デフォルト値:** 0. -## disable\_internal\_dns\_cache {#server-settings-disable-internal-dns-cache} +## disable_internal_dns_cache {#server-settings-disable-internal-dns-cache} 内部DNSキャッシュを無効にします。 システムの作動のClickHouseのために推薦される 頻繁に変化するインフラなどのKubernetes. **デフォルト値:** 0. -## dns\_cache\_update\_period {#server-settings-dns-cache-update-period} +## dns_cache_update_period {#server-settings-dns-cache-update-period} ClickHouse内部DNSキャッシュに格納されているIPアドレスの更新期間(秒単位)。 更新は、別のシステムスレッドで非同期に実行されます。 **デフォルト値**: 15. -## access\_control\_path {#access_control_path} +## access_control_path {#access_control_path} パフォルダがClickHouseサーバー店舗ユーザーの役割構成で作成したSQLコマンド. diff --git a/docs/ja/operations/settings/permissions-for-queries.md b/docs/ja/operations/settings/permissions-for-queries.md index c299337d62f..d77bed96464 100644 --- a/docs/ja/operations/settings/permissions-for-queries.md +++ b/docs/ja/operations/settings/permissions-for-queries.md @@ -18,7 +18,7 @@ toc_title: "\u30AF\u30A8\u30EA\u306E\u6A29\u9650" 次の設定では、クエリの種類によってユーザー権限を調整します: - [読み取り専用](#settings_readonly) — Restricts permissions for all types of queries except DDL queries. -- [allow\_ddl](#settings_allow_ddl) — Restricts permissions for DDL queries. +- [allow_ddl](#settings_allow_ddl) — Restricts permissions for DDL queries. `KILL QUERY` 任意の設定で実行できます。 @@ -43,7 +43,7 @@ toc_title: "\u30AF\u30A8\u30EA\u306E\u6A29\u9650" デフォルト値:0 -## allow\_ddl {#settings_allow_ddl} +## allow_ddl {#settings_allow_ddl} 許可または拒否 [DDL](https://en.wikipedia.org/wiki/Data_definition_language) クエリ。 diff --git a/docs/ja/operations/settings/query-complexity.md b/docs/ja/operations/settings/query-complexity.md index 45b7a09f17f..f98e55ef572 100644 --- a/docs/ja/operations/settings/query-complexity.md +++ b/docs/ja/operations/settings/query-complexity.md @@ -14,8 +14,8 @@ toc_title: "\u30AF\u30A8\u30EA\u306E\u8907\u96D1\u3055\u306E\u5236\u9650" ClickHouseは、各行ではなく、データ部分の制限をチェックします。 これは、データ部分のサイズで制限の値を超えることができることを意味します。 の制限 “maximum amount of something” 値0を取ることができます。 “unrestricted”. -ほとんどの制限には ‘overflow\_mode’ 設定、制限を超えたときに何をすべきかを意味します。 -での値: `throw` または `break`. 集計の制限(group\_by\_overflow\_mode)にも値があります `any`. +ほとんどの制限には ‘overflow_mode’ 設定、制限を超えたときに何をすべきかを意味します。 +での値: `throw` または `break`. 集計の制限(group_by_overflow_mode)にも値があります `any`. `throw` – Throw an exception (default). @@ -23,7 +23,7 @@ ClickHouseは、各行ではなく、データ部分の制限をチェックし `any (only for group_by_overflow_mode)` – Continuing aggregation for the keys that got into the set, but don't add new keys to the set. -## max\_memory\_usage {#settings_max_memory_usage} +## max_memory_usage {#settings_max_memory_usage} 単一サーバーでクエリを実行するために使用するRAMの最大量。 @@ -40,47 +40,47 @@ ClickHouseは、各行ではなく、データ部分の制限をチェックし メモリ消費もパラメータによって制限されます `max_memory_usage_for_user` と `max_memory_usage_for_all_queries`. -## max\_memory\_usage\_for\_user {#max-memory-usage-for-user} +## max_memory_usage_for_user {#max-memory-usage-for-user} 単一サーバー上でユーザーのクエリを実行するために使用するRAMの最大量。 デフォルト値は [設定。h](https://github.com/ClickHouse/ClickHouse/blob/master/src/Core/Settings.h#L288). デフォルトでは、金額は制限されません (`max_memory_usage_for_user = 0`). -の説明も参照してください [max\_memory\_usage](#settings_max_memory_usage). +の説明も参照してください [max_memory_usage](#settings_max_memory_usage). -## max\_memory\_usage\_for\_all\_queries {#max-memory-usage-for-all-queries} +## max_memory_usage_for_all_queries {#max-memory-usage-for-all-queries} 単一サーバー上ですべてのクエリを実行するために使用するRAMの最大量。 デフォルト値は [設定。h](https://github.com/ClickHouse/ClickHouse/blob/master/src/Core/Settings.h#L289). デフォルトでは、金額は制限されません (`max_memory_usage_for_all_queries = 0`). -の説明も参照してください [max\_memory\_usage](#settings_max_memory_usage). +の説明も参照してください [max_memory_usage](#settings_max_memory_usage). -## max\_rows\_to\_read {#max-rows-to-read} +## max_rows_to_read {#max-rows-to-read} 次の制限は、各ブロック(各行ではなく)で確認できます。 つまり、制限は少し壊れる可能性があります。 複数のスレッドでクエリを実行する場合、次の制限は各スレッドに個別に適用されます。 クエリの実行時にテーブルから読み取ることができる最大行数。 -## max\_bytes\_to\_read {#max-bytes-to-read} +## max_bytes_to_read {#max-bytes-to-read} クエリの実行時にテーブルから読み取ることができる最大バイト数(非圧縮データ)。 -## read\_overflow\_mode {#read-overflow-mode} +## read_overflow_mode {#read-overflow-mode} 読み込まれるデータ量がいずれかの制限を超えた場合の対処方法: ‘throw’ または ‘break’. デフォルトでは、throw。 -## max\_rows\_to\_group\_by {#settings-max-rows-to-group-by} +## max_rows_to_group_by {#settings-max-rows-to-group-by} 集計から受け取った一意のキーの最大数。 この設定を使用すると、集計時のメモリ消費量を制限できます。 -## group\_by\_overflow\_mode {#group-by-overflow-mode} +## group_by_overflow_mode {#group-by-overflow-mode} 集計の一意キーの数が制限を超えた場合の対処方法: ‘throw’, ‘break’,または ‘any’. デフォルトでは、throw。 を使用して ‘any’ valueでは、GROUP BYの近似を実行できます。 この近似の品質は、データの統計的性質に依存します。 -## max\_bytes\_before\_external\_group\_by {#settings-max_bytes_before_external_group_by} +## max_bytes_before_external_group_by {#settings-max_bytes_before_external_group_by} の実行を有効または無効にします。 `GROUP BY` 外部メモリ内の句。 見る [外部メモリのGROUP BY](../../sql-reference/statements/select/group-by.md#select-group-by-in-external-memory). @@ -91,31 +91,31 @@ ClickHouseは、各行ではなく、データ部分の制限をチェックし デフォルト値は0です。 -## max\_rows\_to\_sort {#max-rows-to-sort} +## max_rows_to_sort {#max-rows-to-sort} 並べ替え前の最大行数。 これにより、ソート時のメモリ消費を制限できます。 -## max\_bytes\_to\_sort {#max-bytes-to-sort} +## max_bytes_to_sort {#max-bytes-to-sort} 並べ替え前の最大バイト数。 -## sort\_overflow\_mode {#sort-overflow-mode} +## sort_overflow_mode {#sort-overflow-mode} ソート前に受信した行数がいずれかの制限を超えた場合の対処方法: ‘throw’ または ‘break’. デフォルトでは、throw。 -## max\_result\_rows {#setting-max_result_rows} +## max_result_rows {#setting-max_result_rows} 結果の行数を制限します。 またチェックサブクエリは、windowsアプリケーションの実行時にパーツの分散を返します。 -## max\_result\_bytes {#max-result-bytes} +## max_result_bytes {#max-result-bytes} 結果のバイト数を制限します。 前の設定と同じです。 -## result\_overflow\_mode {#result-overflow-mode} +## result_overflow_mode {#result-overflow-mode} 結果の量が制限のいずれかを超えた場合の対処方法: ‘throw’ または ‘break’. デフォルトでは、throw。 -を使用して ‘break’ LIMITの使用に似ています。 `Break` ブロックレベルでのみ実行を中断します。 これは、返される行の量が [max\_result\_rows](#setting-max_result_rows) の倍数 [max\_block\_size](settings.md#setting-max_block_size) そして依存する [max\_threads](settings.md#settings-max_threads). +を使用して ‘break’ LIMITの使用に似ています。 `Break` ブロックレベルでのみ実行を中断します。 これは、返される行の量が [max_result_rows](#setting-max_result_rows) の倍数 [max_block_size](settings.md#setting-max_block_size) そして依存する [max_threads](settings.md#settings-max_threads). 例: @@ -134,103 +134,103 @@ FORMAT Null; 6666 rows in set. ... ``` -## max\_execution\_time {#max-execution-time} +## max_execution_time {#max-execution-time} クエリの最大実行時間を秒単位で指定します。 現時点では、ソートステージのいずれか、または集計関数のマージおよびファイナライズ時にはチェックされません。 -## timeout\_overflow\_mode {#timeout-overflow-mode} +## timeout_overflow_mode {#timeout-overflow-mode} -クエリが実行される時間よりも長い場合の対処方法 ‘max\_execution\_time’: ‘throw’ または ‘break’. デフォルトでは、throw。 +クエリが実行される時間よりも長い場合の対処方法 ‘max_execution_time’: ‘throw’ または ‘break’. デフォルトでは、throw。 -## min\_execution\_speed {#min-execution-speed} +## min_execution_speed {#min-execution-speed} -毎秒行単位の最小実行速度。 すべてのデータブロックで ‘timeout\_before\_checking\_execution\_speed’ 有効期限が切れます。 実行速度が低い場合は、例外がスローされます。 +毎秒行単位の最小実行速度。 すべてのデータブロックで ‘timeout_before_checking_execution_speed’ 有効期限が切れます。 実行速度が低い場合は、例外がスローされます。 -## min\_execution\_speed\_bytes {#min-execution-speed-bytes} +## min_execution_speed_bytes {#min-execution-speed-bytes} -秒あたりの最小実行バイト数。 すべてのデータブロックで ‘timeout\_before\_checking\_execution\_speed’ 有効期限が切れます。 実行速度が低い場合は、例外がスローされます。 +秒あたりの最小実行バイト数。 すべてのデータブロックで ‘timeout_before_checking_execution_speed’ 有効期限が切れます。 実行速度が低い場合は、例外がスローされます。 -## max\_execution\_speed {#max-execution-speed} +## max_execution_speed {#max-execution-speed} -毎秒の実行行の最大数。 すべてのデータブロックで ‘timeout\_before\_checking\_execution\_speed’ 有効期限が切れます。 実行速度が高い場合は、実行速度が低下します。 +毎秒の実行行の最大数。 すべてのデータブロックで ‘timeout_before_checking_execution_speed’ 有効期限が切れます。 実行速度が高い場合は、実行速度が低下します。 -## max\_execution\_speed\_bytes {#max-execution-speed-bytes} +## max_execution_speed_bytes {#max-execution-speed-bytes} -毎秒の実行バイト数の最大値。 すべてのデータブロックで ‘timeout\_before\_checking\_execution\_speed’ 有効期限が切れます。 実行速度が高い場合は、実行速度が低下します。 +毎秒の実行バイト数の最大値。 すべてのデータブロックで ‘timeout_before_checking_execution_speed’ 有効期限が切れます。 実行速度が高い場合は、実行速度が低下します。 -## timeout\_before\_checking\_execution\_speed {#timeout-before-checking-execution-speed} +## timeout_before_checking_execution_speed {#timeout-before-checking-execution-speed} -実行速度が遅すぎないことをチェックします ‘min\_execution\_speed’)、指定された時間が秒単位で経過した後。 +実行速度が遅すぎないことをチェックします ‘min_execution_speed’)、指定された時間が秒単位で経過した後。 -## max\_columns\_to\_read {#max-columns-to-read} +## max_columns_to_read {#max-columns-to-read} 単一のクエリ内のテーブルから読み取ることができる列の最大数。 クエリでより多くの列を読み取る必要がある場合は、例外がスローされます。 -## max\_temporary\_columns {#max-temporary-columns} +## max_temporary_columns {#max-temporary-columns} 定数列を含む、クエリを実行するときに同時にRAMに保持する必要がある一時列の最大数。 これよりも多くの一時列がある場合、例外がスローされます。 -## max\_temporary\_non\_const\_columns {#max-temporary-non-const-columns} +## max_temporary_non_const_columns {#max-temporary-non-const-columns} -同じことと ‘max\_temporary\_columns’ しかし、定数列を数えずに。 +同じことと ‘max_temporary_columns’ しかし、定数列を数えずに。 定数列は、クエリを実行するときにかなり頻繁に形成されますが、計算リソースはほぼゼロです。 -## max\_subquery\_depth {#max-subquery-depth} +## max_subquery_depth {#max-subquery-depth} サブクエリの最大ネスト深さ。 サブクエリが深い場合は、例外がスローされます。 既定では100です。 -## max\_pipeline\_depth {#max-pipeline-depth} +## max_pipeline_depth {#max-pipeline-depth} 最大パイプライン深さ。 クエリ処理中に各データブロックが処理する変換の数に対応します。 単一サーバーの範囲内でカウントされます。 パイプラインの深さが大きい場合は、例外がスローされます。 既定では、1000です。 -## max\_ast\_depth {#max-ast-depth} +## max_ast_depth {#max-ast-depth} クエリ構文ツリーの最大ネスト深さ。 超過すると、例外がスローされます。 現時点では、解析中にはチェックされず、クエリの解析後にのみチェックされます。 つまり、解析中に深すぎる構文ツリーを作成することができますが、クエリは失敗します。 既定では、1000です。 -## max\_ast\_elements {#max-ast-elements} +## max_ast_elements {#max-ast-elements} クエリ構文ツリー内の要素の最大数。 超過すると、例外がスローされます。 前の設定と同じように、クエリを解析した後にのみチェックされます。 既定では、50,000です。 -## max\_rows\_in\_set {#max-rows-in-set} +## max_rows_in_set {#max-rows-in-set} サブクエリから作成されたIN句内のデータ-セットの最大行数。 -## max\_bytes\_in\_set {#max-bytes-in-set} +## max_bytes_in_set {#max-bytes-in-set} サブクエリから作成されたIN句のセットで使用される最大バイト数(非圧縮データ)。 -## set\_overflow\_mode {#set-overflow-mode} +## set_overflow_mode {#set-overflow-mode} データ量がいずれかの制限を超えた場合の対処方法: ‘throw’ または ‘break’. デフォルトでは、throw。 -## max\_rows\_in\_distinct {#max-rows-in-distinct} +## max_rows_in_distinct {#max-rows-in-distinct} DISTINCTを使用する場合の最大行数。 -## max\_bytes\_in\_distinct {#max-bytes-in-distinct} +## max_bytes_in_distinct {#max-bytes-in-distinct} DISTINCTを使用するときにハッシュテーブルで使用される最大バイト数。 -## distinct\_overflow\_mode {#distinct-overflow-mode} +## distinct_overflow_mode {#distinct-overflow-mode} データ量がいずれかの制限を超えた場合の対処方法: ‘throw’ または ‘break’. デフォルトでは、throw。 -## max\_rows\_to\_transfer {#max-rows-to-transfer} +## max_rows_to_transfer {#max-rows-to-transfer} グローバルINを使用するときに、リモートサーバーに渡すか、一時テーブルに保存できる最大行数。 -## max\_bytes\_to\_transfer {#max-bytes-to-transfer} +## max_bytes_to_transfer {#max-bytes-to-transfer} グローバルINを使用するときに、リモートサーバーに渡すか、一時テーブルに保存できる最大バイト数(非圧縮データ)。 -## transfer\_overflow\_mode {#transfer-overflow-mode} +## transfer_overflow_mode {#transfer-overflow-mode} データ量がいずれかの制限を超えた場合の対処方法: ‘throw’ または ‘break’. デフォルトでは、throw。 -## max\_rows\_in\_join {#settings-max_rows_in_join} +## max_rows_in_join {#settings-max_rows_in_join} テーブルを結合するときに使用されるハッシュテーブル内の行数を制限します。 @@ -238,7 +238,7 @@ DISTINCTを使用するときにハッシュテーブルで使用される最大 クエリに複数の結合が含まれている場合、ClickHouseはこの設定で中間結果をすべてチェックします。 -ClickHouseは、制限に達したときにさまざまなアクションを続行できます。 使用する [join\_overflow\_mode](#settings-join_overflow_mode) アクションを選択する設定。 +ClickHouseは、制限に達したときにさまざまなアクションを続行できます。 使用する [join_overflow_mode](#settings-join_overflow_mode) アクションを選択する設定。 可能な値: @@ -247,7 +247,7 @@ ClickHouseは、制限に達したときにさまざまなアクションを続 デフォルト値は0です。 -## max\_bytes\_in\_join {#settings-max_bytes_in_join} +## max_bytes_in_join {#settings-max_bytes_in_join} 制限サイズをバイトのハッシュテーブルが参加す。 @@ -255,7 +255,7 @@ ClickHouseは、制限に達したときにさまざまなアクションを続 クエリに結合が含まれている場合、ClickHouseは中間結果ごとにこの設定をチェックします。 -ClickHouseは、制限に達したときにさまざまなアクションを続行できます。 使用 [join\_overflow\_mode](#settings-join_overflow_mode) アクションを選択する設定。 +ClickHouseは、制限に達したときにさまざまなアクションを続行できます。 使用 [join_overflow_mode](#settings-join_overflow_mode) アクションを選択する設定。 可能な値: @@ -264,12 +264,12 @@ ClickHouseは、制限に達したときにさまざまなアクションを続 デフォルト値は0です。 -## join\_overflow\_mode {#settings-join_overflow_mode} +## join_overflow_mode {#settings-join_overflow_mode} 次のいずれかの結合制限に達したときにClickHouseが実行するアクションを定義します: -- [max\_bytes\_in\_join](#settings-max_bytes_in_join) -- [max\_rows\_in\_join](#settings-max_rows_in_join) +- [max_bytes_in_join](#settings-max_bytes_in_join) +- [max_rows_in_join](#settings-max_rows_in_join) 可能な値: @@ -283,7 +283,7 @@ ClickHouseは、制限に達したときにさまざまなアクションを続 - [JOIN句](../../sql-reference/statements/select/join.md#select-join) - [結合テーブルエンジン](../../engines/table-engines/special/join.md) -## max\_partitions\_per\_insert\_block {#max-partitions-per-insert-block} +## max_partitions_per_insert_block {#max-partitions-per-insert-block} 単一挿入ブロック内のパーティションの最大数を制限します。 @@ -296,6 +296,6 @@ ClickHouseは、制限に達したときにさまざまなアクションを続 を挿入する際、データClickHouse計算パーティションの数に挿入されます。 パーティションの数が `max_partitions_per_insert_block`,ClickHouseは、次のテキストで例外をスローします: -> “Too many partitions for single INSERT block (more than” +toString(max\_parts)+ “). The limit is controlled by ‘max\_partitions\_per\_insert\_block’ setting. A large number of partitions is a common misconception. It will lead to severe negative performance impact, including slow server startup, slow INSERT queries and slow SELECT queries. Recommended total number of partitions for a table is under 1000..10000. Please note, that partitioning is not intended to speed up SELECT queries (ORDER BY key is sufficient to make range queries fast). Partitions are intended for data manipulation (DROP PARTITION, etc).” +> “Too many partitions for single INSERT block (more than” +toString(max_parts)+ “). The limit is controlled by ‘max_partitions_per_insert_block’ setting. A large number of partitions is a common misconception. It will lead to severe negative performance impact, including slow server startup, slow INSERT queries and slow SELECT queries. Recommended total number of partitions for a table is under 1000..10000. Please note, that partitioning is not intended to speed up SELECT queries (ORDER BY key is sufficient to make range queries fast). Partitions are intended for data manipulation (DROP PARTITION, etc).” [元の記事](https://clickhouse.tech/docs/en/operations/settings/query_complexity/) diff --git a/docs/ja/operations/settings/settings-users.md b/docs/ja/operations/settings/settings-users.md index 7b6fca249d7..dc72c6ceb6f 100644 --- a/docs/ja/operations/settings/settings-users.md +++ b/docs/ja/operations/settings/settings-users.md @@ -43,7 +43,7 @@ toc_title: "\u30E6\u30FC\u30B6\u30FC\u8A2D\u5B9A" ``` -### user\_name/パスワード {#user-namepassword} +### user_name/パスワード {#user-namepassword} パスワードは、平文またはSHA256(hex形式)で指定できます。 @@ -75,7 +75,7 @@ toc_title: "\u30E6\u30FC\u30B6\u30FC\u8A2D\u5B9A" 結果の最初の行はパスワードです。 第二の行は、対応する二重SHA1ハッシュです。 -### access\_management {#access_management-user-setting} +### access_management {#access_management-user-setting} この設定では、SQLドリブンの使用を無効にできます [アクセス制御とアカウント管理](../access-rights.md#access-control) ユーザーのために。 @@ -86,7 +86,7 @@ toc_title: "\u30E6\u30FC\u30B6\u30FC\u8A2D\u5B9A" デフォルト値は0です。 -### user\_name/ネットワーク {#user-namenetworks} +### user_name/ネットワーク {#user-namenetworks} ユーザーがClickHouseサーバーに接続できるネットワークのリスト。 @@ -128,18 +128,18 @@ toc_title: "\u30E6\u30FC\u30B6\u30FC\u8A2D\u5B9A" 127.0.0.1 ``` -### user\_name/プロファイル {#user-nameprofile} +### user_name/プロファイル {#user-nameprofile} を割り当てることができる設定プロファイルをユーザーです。 設定プロファイルは、 `users.xml` ファイル 詳細については、 [設定のプロファイル](settings-profiles.md). -### user\_name/クォータ {#user-namequota} +### user_name/クォータ {#user-namequota} クォータを使用すると、一定期間のリソース使用量を追跡または制限できます。 クォータは `quotas` のセクション `users.xml` 設定ファイル。 ユーザにクォータセットを割り当てることができます。 クォータ設定の詳細については、以下を参照してください [クォータ](../quotas.md#quotas). -### user\_name/データベース {#user-namedatabases} +### user_name/データベース {#user-namedatabases} このセクションでは、ClickHouseによって返される行を以下の目的で制限することができます `SELECT` クエリーによる、現在のユーザが実施基本列レベルです。 diff --git a/docs/ja/operations/settings/settings.md b/docs/ja/operations/settings/settings.md index 721f161ebda..530edf780f0 100644 --- a/docs/ja/operations/settings/settings.md +++ b/docs/ja/operations/settings/settings.md @@ -5,7 +5,7 @@ machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd # 設定 {#settings} -## distributed\_product\_mode {#distributed-product-mode} +## distributed_product_mode {#distributed-product-mode} の動作を変更します [分散サブクエリ](../../sql-reference/operators/in.md). @@ -25,7 +25,7 @@ ClickHouse applies this setting when the query contains the product of distribut - `global` — Replaces the `IN`/`JOIN` クエリ `GLOBAL IN`/`GLOBAL JOIN.` - `allow` — Allows the use of these types of subqueries. -## enable\_optimize\_predicate\_expression {#enable-optimize-predicate-expression} +## enable_optimize_predicate_expression {#enable-optimize-predicate-expression} 述語プッシュダウンをオンにする `SELECT` クエリ。 @@ -49,7 +49,7 @@ ClickHouse applies this setting when the query contains the product of distribut もし `enable_optimize_predicate_expression = 0` 次に、第二のクエリの実行時間ははるかに長くなります。 `WHERE` 句は、サブクエリの終了後にすべてのデータに適用されます。 -## フォールバック\_to\_stale\_replicas\_for\_distributed\_queries {#settings-fallback_to_stale_replicas_for_distributed_queries} +## フォールバック_to_stale_replicas_for_distributed_queries {#settings-fallback_to_stale_replicas_for_distributed_queries} 更新されたデータが使用できない場合は、クエリを古いレプリカに強制します。 見る [複製](../../engines/table-engines/mergetree-family/replication.md). @@ -59,7 +59,7 @@ ClickHouseは、テーブルの古いレプリカから最も関連性の高い デフォルトでは、1(有効)です。 -## force\_index\_by\_date {#settings-force_index_by_date} +## force_index_by_date {#settings-force_index_by_date} インデックスを日付で使用できない場合は、クエリの実行を無効にします。 @@ -67,7 +67,7 @@ MergeTreeファミリ内のテーブルで動作します。 もし `force_index_by_date=1`,ClickHouseは、クエリにデータ範囲を制限するために使用できる日付キー条件があるかどうかをチェックします。 適切な条件がない場合は、例外をスローします。 ただし、条件によって読み取るデータ量が減少するかどうかはチェックされません。 たとえば、次の条件 `Date != ' 2000-01-01 '` テーブル内のすべてのデータと一致する場合でも許容されます(つまり、クエリを実行するにはフルスキャンが必要です)。 MergeTreeテーブル内のデータ範囲の詳細については、以下を参照してください [メルゲツリー](../../engines/table-engines/mergetree-family/mergetree.md). -## force\_primary\_key {#force-primary-key} +## force_primary_key {#force-primary-key} 主キーによる索引付けが不可能な場合は、クエリの実行を無効にします。 @@ -75,17 +75,17 @@ MergeTreeファミリ内のテーブルで動作します。 もし `force_primary_key=1`,ClickHouseは、クエリにデータ範囲を制限するために使用できる主キー条件があるかどうかを確認します。 適切な条件がない場合は、例外をスローします。 ただし、条件によって読み取るデータ量が減少するかどうかはチェックされません。 MergeTreeテーブルのデータ範囲の詳細については、以下を参照してください [メルゲツリー](../../engines/table-engines/mergetree-family/mergetree.md). -## format\_schema {#format-schema} +## format_schema {#format-schema} このパラメーターは、次のようなスキーマ定義を必要とする形式を使用している場合に便利です [Cap'N Proto](https://capnproto.org/) または [プロトブフ](https://developers.google.com/protocol-buffers/). 値は形式によって異なります。 -## fsync\_metadata {#fsync-metadata} +## fsync_metadata {#fsync-metadata} 有効または無効にします [fsync](http://pubs.opengroup.org/onlinepubs/9699919799/functions/fsync.html) 書くとき `.sql` ファイル 既定で有効になっています。 ことは、あってはならないことで無効にすることもできれば、サーバは、数百万の小さなテーブルが続々と生まれてくると破壊されました。 -## enable\_http\_compression {#settings-enable_http_compression} +## enable_http_compression {#settings-enable_http_compression} HTTP要求に対する応答のデータ圧縮を有効または無効にします。 @@ -98,15 +98,15 @@ HTTP要求に対する応答のデータ圧縮を有効または無効にしま デフォルト値は0です。 -## http\_zlib\_compression\_level {#settings-http_zlib_compression_level} +## http_zlib_compression_level {#settings-http_zlib_compression_level} -HTTP要求に対する応答のデータ圧縮のレベルを次の場合に設定します [enable\_http\_compression=1](#settings-enable_http_compression). +HTTP要求に対する応答のデータ圧縮のレベルを次の場合に設定します [enable_http_compression=1](#settings-enable_http_compression). 可能な値:1から9までの数値。 デフォルト値は3です。 -## http\_native\_compression\_disable\_checksumming\_on\_decompress {#settings-http_native_compression_disable_checksumming_on_decompress} +## http_native_compression_disable_checksumming_on_decompress {#settings-http_native_compression_disable_checksumming_on_decompress} クライア ClickHouseネイティブ圧縮フォーマットにのみ使用されます( `gzip` または `deflate`). @@ -119,7 +119,7 @@ HTTP要求に対する応答のデータ圧縮のレベルを次の場合に設 デフォルト値は0です。 -## send\_progress\_in\_http\_headers {#settings-send_progress_in_http_headers} +## send_progress_in_http_headers {#settings-send_progress_in_http_headers} 有効または無効にします `X-ClickHouse-Progress` HTTP応答ヘッダー `clickhouse-server` 応答。 @@ -132,7 +132,7 @@ HTTP要求に対する応答のデータ圧縮のレベルを次の場合に設 デフォルト値は0です。 -## max\_http\_get\_redirects {#setting-max_http_get_redirects} +## max_http_get_redirects {#setting-max_http_get_redirects} HTTP GETリダイレクトホップの最大数を制限します。 [URL](../../engines/table-engines/special/url.md)-エンジンテーブル。 この設定は、両方のタイプのテーブルに適用されます。 [CREATE TABLE](../../sql-reference/statements/create.md#create-table-query) クエリとによって [url](../../sql-reference/table-functions/url.md) テーブル関数。 @@ -143,7 +143,7 @@ HTTP GETリダイレクトホップの最大数を制限します。 [URL](../.. デフォルト値は0です。 -## input\_format\_allow\_errors\_num {#settings-input_format_allow_errors_num} +## input_format_allow_errors_num {#settings-input_format_allow_errors_num} テキスト形式(CSV、TSVなど)から読み取るときに許容されるエラーの最大数を設定します。). @@ -155,7 +155,7 @@ HTTP GETリダイレクトホップの最大数を制限します。 [URL](../.. 両方の場合 `input_format_allow_errors_num` と `input_format_allow_errors_ratio` を超えた場合、ClickHouseは例外をスローします。 -## input\_format\_allow\_errors\_ratio {#settings-input_format_allow_errors_ratio} +## input_format_allow_errors_ratio {#settings-input_format_allow_errors_ratio} テキスト形式(CSV、TSVなど)から読み取るときに許容されるエラーの最大割合を設定します。). エラーの割合は、0から1の間の浮動小数点数として設定されます。 @@ -168,7 +168,7 @@ HTTP GETリダイレクトホップの最大数を制限します。 [URL](../.. 両方の場合 `input_format_allow_errors_num` と `input_format_allow_errors_ratio` を超えた場合、ClickHouseは例外をスローします。 -## input\_format\_values\_interpret\_expressions {#settings-input_format_values_interpret_expressions} +## input_format_values_interpret_expressions {#settings-input_format_values_interpret_expressions} を有効または無効にしのSQLのパーサの場合の高速ストリームのパーサで構文解析のデータです。 この設定は、 [値](../../interfaces/formats.md#data-format-values) データ挿入時の書式。 構文解析の詳細については、以下を参照してください [構文](../../sql-reference/syntax.md) セクション @@ -218,7 +218,7 @@ INSERT INTO datetime_t SELECT now() Ok. ``` -## input\_format\_values\_deduce\_templates\_of\_expressions {#settings-input_format_values_deduce_templates_of_expressions} +## input_format_values_deduce_templates_of_expressions {#settings-input_format_values_deduce_templates_of_expressions} SQL式のテンプレート控除を有効または無効にします。 [値](../../interfaces/formats.md#data-format-values) 形式。 この解析と解釈表現 `Values` 連続する行の式が同じ構造を持つ場合、はるかに高速です。 ClickHouseは式のテンプレートを推論し、このテンプレートを使用して次の行を解析し、正常に解析された行のバッチで式を評価しようとします。 @@ -239,7 +239,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( - もし `input_format_values_interpret_expressions=0` と `format_values_deduce_templates_of_expressions=1`、第一、第二および第三の行の式は、テンプレートを使用して解析されます `lower(String)` そして一緒に解釈されると、forth行の式は別のテンプレートで解析されます (`upper(String)`). - もし `input_format_values_interpret_expressions=1` と `format_values_deduce_templates_of_expressions=1`、前の場合と同じですが、テンプレートを推論できない場合は、式を別々に解釈することもできます。 -## input\_format\_values\_accurate\_types\_of\_literals {#settings-input-format-values-accurate-types-of-literals} +## input_format_values_accurate_types_of_literals {#settings-input-format-values-accurate-types-of-literals} この設定は次の場合にのみ使用されます `input_format_values_deduce_templates_of_expressions = 1`. いくつかの列の式は同じ構造を持ちますが、異なる型の数値リテラルが含まれています。 @@ -261,7 +261,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( デフォルト値:1。 -## input\_format\_defaults\_for\_omitted\_fields {#session_settings-input_format_defaults_for_omitted_fields} +## input_format_defaults_for_omitted_fields {#session_settings-input_format_defaults_for_omitted_fields} 実行するとき `INSERT` 省略された入力列の値を、それぞれの列のデフォルト値に置き換えます。 このオプションは [JSONEachRow](../../interfaces/formats.md#jsoneachrow), [CSV](../../interfaces/formats.md#csv) と [TabSeparated](../../interfaces/formats.md#tabseparated) フォーマット。 @@ -275,17 +275,17 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( デフォルト値:1。 -## input\_format\_tsv\_empty\_as\_default {#settings-input-format-tsv-empty-as-default} +## input_format_tsv_empty_as_default {#settings-input-format-tsv-empty-as-default} 有効にすると、TSVの空の入力フィールドを既定値に置き換えます。 複雑な既定の式の場合 `input_format_defaults_for_omitted_fields` 有効にする必要があります。 既定では無効です。 -## input\_format\_null\_as\_default {#settings-input-format-null-as-default} +## input_format_null_as_default {#settings-input-format-null-as-default} を有効または無効にし用のデフォルト値が入力データを含む `NULL` しかし、notの対応する列のデータ型 `Nullable(T)` (テキスト入力形式の場合)。 -## input\_format\_skip\_unknown\_fields {#settings-input-format-skip-unknown-fields} +## input_format_skip_unknown_fields {#settings-input-format-skip-unknown-fields} 追加データのスキップ挿入を有効または無効にします。 @@ -305,7 +305,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( デフォルト値は0です。 -## input\_format\_import\_nested\_json {#settings-input_format_import_nested_json} +## input_format_import_nested_json {#settings-input_format_import_nested_json} を有効または無効にし、挿入のJSONデータをネストしたオブジェクト。 @@ -324,7 +324,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( - [入れ子構造の使用](../../interfaces/formats.md#jsoneachrow-nested) と `JSONEachRow` 形式。 -## input\_format\_with\_names\_use\_header {#settings-input-format-with-names-use-header} +## input_format_with_names_use_header {#settings-input-format-with-names-use-header} データ挿入時の列順序の確認を有効または無効にします。 @@ -342,7 +342,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( デフォルト値:1。 -## date\_time\_input\_format {#settings-date_time_input_format} +## date_time_input_format {#settings-date_time_input_format} 日付と時刻のテキスト表現のパーサーを選択できます。 @@ -365,7 +365,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( - [DateTimeデータ型。](../../sql-reference/data-types/datetime.md) - [日付と時刻を操作するための関数。](../../sql-reference/functions/date-time-functions.md) -## join\_default\_strictness {#settings-join_default_strictness} +## join_default_strictness {#settings-join_default_strictness} デフォルトの厳密さを [結合句](../../sql-reference/statements/select/join.md#select-join). @@ -378,7 +378,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( デフォルト値: `ALL`. -## join\_any\_take\_last\_row {#settings-join_any_take_last_row} +## join_any_take_last_row {#settings-join_any_take_last_row} 結合操作の動作を `ANY` 厳密さ。 @@ -396,9 +396,9 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( - [JOIN句](../../sql-reference/statements/select/join.md#select-join) - [結合テーブルエンジン](../../engines/table-engines/special/join.md) -- [join\_default\_strictness](#settings-join_default_strictness) +- [join_default_strictness](#settings-join_default_strictness) -## join\_use\_nulls {#join_use_nulls} +## join_use_nulls {#join_use_nulls} のタイプを設定します。 [JOIN](../../sql-reference/statements/select/join.md) 行動。 際融合のテーブル、空細胞が表示される場合があります。 ClickHouseは、この設定に基づいて異なる塗りつぶします。 @@ -409,7 +409,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( デフォルト値は0です。 -## max\_block\_size {#setting-max_block_size} +## max_block_size {#setting-max_block_size} ClickHouseでは、データはブロック(列部分のセット)によって処理されます。 単一ブロックの内部処理サイクルは十分に効率的ですが、各ブロックには顕著な支出があります。 その `max_block_size` 設定は、テーブルからロードするブロックのサイズ(行数)の推奨事項です。 ブロックサイズは小さすぎるので、各ブロックの支出はまだ目立ちますが、最初のブロックの後に完了したLIMITのクエリがすばやく処理されるよう 目標は、複数のスレッドで多数の列を抽出するときにメモリを消費するのを避け、少なくともいくつかのキャッシュの局所性を保持することです。 @@ -417,13 +417,13 @@ ClickHouseでは、データはブロック(列部分のセット)によっ ブロックのサイズ `max_block_size` ていないから読み込まれます。 場合ことは明らかであることなくデータを取得され、さらに小型のブロックを処理します。 -## preferred\_block\_size\_bytes {#preferred-block-size-bytes} +## preferred_block_size_bytes {#preferred-block-size-bytes} と同じ目的のために使用される `max_block_size` しかし、ブロック内の行数に適応させることによって、推奨されるブロックサイズをバイト単位で設定します。 ただし、ブロックサイズは次のようになります `max_block_size` 行。 既定では、1,000,000です。 MergeTreeエンジンから読み取るときにのみ動作します。 -## merge\_tree\_min\_rows\_for\_concurrent\_read {#setting-merge-tree-min-rows-for-concurrent-read} +## merge_tree_min_rows_for_concurrent_read {#setting-merge-tree-min-rows-for-concurrent-read} のファイルから読み込まれる行数が [メルゲツリー](../../engines/table-engines/mergetree-family/mergetree.md) テーブル超過 `merge_tree_min_rows_for_concurrent_read` その後ClickHouseしようとして行な兼職の状況からの読み出しこのファイルに複数のスレッド)。 @@ -433,7 +433,7 @@ ClickHouseでは、データはブロック(列部分のセット)によっ デフォルト値は163840です。 -## merge\_tree\_min\_bytes\_for\_concurrent\_read {#setting-merge-tree-min-bytes-for-concurrent-read} +## merge_tree_min_bytes_for_concurrent_read {#setting-merge-tree-min-bytes-for-concurrent-read} のファイルから読み取るバイト数が [メルゲツリー](../../engines/table-engines/mergetree-family/mergetree.md)-エンジン表超過 `merge_tree_min_bytes_for_concurrent_read` 次に、ClickHouseはこのファイルから複数のスレッドで同時に読み取ろうとします。 @@ -443,7 +443,7 @@ ClickHouseでは、データはブロック(列部分のセット)によっ デフォルト値は251658240です。 -## merge\_tree\_min\_rows\_for\_seek {#setting-merge-tree-min-rows-for-seek} +## merge_tree_min_rows_for_seek {#setting-merge-tree-min-rows-for-seek} 一つのファイルに読み込まれる二つのデータブロック間の距離が `merge_tree_min_rows_for_seek` その後、ClickHouseはファイルをシークせず、データを順番に読み込みます。 @@ -453,7 +453,7 @@ ClickHouseでは、データはブロック(列部分のセット)によっ デフォルト値は0です。 -## merge\_tree\_min\_bytes\_for\_seek {#setting-merge-tree-min-bytes-for-seek} +## merge_tree_min_bytes_for_seek {#setting-merge-tree-min-bytes-for-seek} 一つのファイルに読み込まれる二つのデータブロック間の距離が `merge_tree_min_bytes_for_seek` その後、ClickHouseは両方のブロックを含むファイルの範囲を順番に読み込むため、余分なシークを避けます。 @@ -463,7 +463,7 @@ ClickHouseでは、データはブロック(列部分のセット)によっ デフォルト値は0です。 -## merge\_tree\_coarse\_index\_granularity {#setting-merge-tree-coarse-index-granularity} +## merge_tree_coarse_index_granularity {#setting-merge-tree-coarse-index-granularity} する場合のデータClickHouseチェックのデータにファイルです。 必要なキーがある範囲にあることをClickHouseが検出すると、この範囲を次のように分割します `merge_tree_coarse_index_granularity` 必要なキーを再帰的に検索します。 @@ -473,11 +473,11 @@ ClickHouseでは、データはブロック(列部分のセット)によっ デフォルト値:8。 -## merge\_tree\_max\_rows\_to\_use\_cache {#setting-merge-tree-max-rows-to-use-cache} +## merge_tree_max_rows_to_use_cache {#setting-merge-tree-max-rows-to-use-cache} ClickHouseがより多くを読むべきであれば `merge_tree_max_rows_to_use_cache` あるクエリでは、非圧縮ブロックのキャッシュを使用しません。 -のキャッシュされた、圧縮解除されたブロックの店舗データを抽出したためます。 ClickHouseこのキャッシュの高速化対応小の繰り返します。 この設定は、大量のデータを読み取るクエリによってキャッシュが破損するのを防ぎます。 その [uncompressed\_cache\_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) サーバー設定は、非圧縮ブロックのキャッシュのサイズを定義します。 +のキャッシュされた、圧縮解除されたブロックの店舗データを抽出したためます。 ClickHouseこのキャッシュの高速化対応小の繰り返します。 この設定は、大量のデータを読み取るクエリによってキャッシュが破損するのを防ぎます。 その [uncompressed_cache_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) サーバー設定は、非圧縮ブロックのキャッシュのサイズを定義します。 可能な値: @@ -485,11 +485,11 @@ ClickHouseがより多くを読むべきであれば `merge_tree_max_rows_to_use Default value: 128 ✕ 8192. -## merge\_tree\_max\_bytes\_to\_use\_cache {#setting-merge-tree-max-bytes-to-use-cache} +## merge_tree_max_bytes_to_use_cache {#setting-merge-tree-max-bytes-to-use-cache} ClickHouseがより多くを読むべきであれば `merge_tree_max_bytes_to_use_cache` 一つのクエリでは、非圧縮ブロックのキャッシュを使用しません。 -のキャッシュされた、圧縮解除されたブロックの店舗データを抽出したためます。 ClickHouseこのキャッシュの高速化対応小の繰り返します。 この設定は、大量のデータを読み取るクエリによってキャッシュが破損するのを防ぎます。 その [uncompressed\_cache\_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) サーバー設定は、非圧縮ブロックのキャッシュのサイズを定義します。 +のキャッシュされた、圧縮解除されたブロックの店舗データを抽出したためます。 ClickHouseこのキャッシュの高速化対応小の繰り返します。 この設定は、大量のデータを読み取るクエリによってキャッシュが破損するのを防ぎます。 その [uncompressed_cache_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) サーバー設定は、非圧縮ブロックのキャッシュのサイズを定義します。 可能な値: @@ -497,7 +497,7 @@ ClickHouseがより多くを読むべきであれば `merge_tree_max_bytes_to_us 既定値:2013265920。 -## min\_bytes\_to\_use\_direct\_io {#settings-min-bytes-to-use-direct-io} +## min_bytes_to_use_direct_io {#settings-min-bytes-to-use-direct-io} ストレージディスクへの直接I/Oアクセスを使用するために必要な最小データ量。 @@ -510,11 +510,11 @@ ClickHouseこの設定からデータを読み込むときます。 読み取る デフォルト値は0です。 -## log\_queries {#settings-log-queries} +## log_queries {#settings-log-queries} クエリログの設定。 -この設定でClickHouseに送信されたクエリは、 [query\_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query-log) サーバー構成パラメータ。 +この設定でClickHouseに送信されたクエリは、 [query_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query-log) サーバー構成パラメータ。 例: @@ -522,7 +522,7 @@ ClickHouseこの設定からデータを読み込むときます。 読み取る log_queries=1 ``` -## log\_queries\_min\_type {#settings-log-queries-min-type} +## log_queries_min_type {#settings-log-queries-min-type} `query_log` ログに記録する最小タイプ。 @@ -540,11 +540,11 @@ log_queries=1 log_queries_min_type='EXCEPTION_WHILE_PROCESSING' ``` -## log\_query\_threads {#settings-log-query-threads} +## log_query_threads {#settings-log-query-threads} クエリスレッドログの設定。 -この設定でClickHouseによって実行されたクエリのスレッドは、 [query\_thread\_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) サーバー構成パラメータ。 +この設定でClickHouseによって実行されたクエリのスレッドは、 [query_thread_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) サーバー構成パラメータ。 例: @@ -552,19 +552,19 @@ log_queries_min_type='EXCEPTION_WHILE_PROCESSING' log_query_threads=1 ``` -## max\_insert\_block\_size {#settings-max_insert_block_size} +## max_insert_block_size {#settings-max_insert_block_size} テーブルに挿入するために形成するブロックのサイズ。 この設定は、サーバーがブロックを形成する場合にのみ適用されます。 たとえば、HTTPインターフェイスを介した挿入の場合、サーバーはデータ形式を解析し、指定されたサイズのブロックを形成します。 -しかし、clickhouse-clientを使用すると、クライアントはデータ自体を解析し、 ‘max\_insert\_block\_size’ サーバーでの設定は、挿入されたブロックのサイズには影響しません。 +しかし、clickhouse-clientを使用すると、クライアントはデータ自体を解析し、 ‘max_insert_block_size’ サーバーでの設定は、挿入されたブロックのサイズには影響しません。 データはSELECT後に形成されるのと同じブロックを使用して挿入されるため、INSERT SELECTを使用するときにもこの設定には目的がありません。 既定値は1,048,576です。 デフォルトは、 `max_block_size`. この理由は、特定のテーブルエンジンが原因です (`*MergeTree`)挿入されたブロックごとにディスク上にデータ部分を形成します。 同様に, `*MergeTree` テーブルデータを並べ替え時の挿入やるのに十分な大きさのブロックサイズを選別データにアプリです。 -## min\_insert\_block\_size\_rows {#min-insert-block-size-rows} +## min_insert_block_size_rows {#min-insert-block-size-rows} テーブルに挿入できるブロック内の最小行数を設定します。 `INSERT` クエリ。 小さめサイズのブロックをつぶし入ります。 @@ -575,7 +575,7 @@ log_query_threads=1 デフォルト値は1048576です。 -## min\_insert\_block\_size\_bytes {#min-insert-block-size-bytes} +## min_insert_block_size_bytes {#min-insert-block-size-bytes} テーブルに挿入できるブロック内の最小バイト数を設定します。 `INSERT` クエリ。 小さめサイズのブロックをつぶし入ります。 @@ -586,7 +586,7 @@ log_query_threads=1 デフォルト値:268435456。 -## max\_replica\_delay\_for\_distributed\_queries {#settings-max_replica_delay_for_distributed_queries} +## max_replica_delay_for_distributed_queries {#settings-max_replica_delay_for_distributed_queries} 分散クエリの遅延レプリカを無効にします。 見る [複製](../../engines/table-engines/mergetree-family/replication.md). @@ -596,22 +596,22 @@ log_query_threads=1 実行時使用 `SELECT` 複製されたテーブルを指す分散テーブルから。 -## max\_threads {#settings-max_threads} +## max_threads {#settings-max_threads} -リモートサーバーからデータを取得するためのスレッドを除く、クエリ処理スレッドの最大数 ‘max\_distributed\_connections’ 変数)。 +リモートサーバーからデータを取得するためのスレッドを除く、クエリ処理スレッドの最大数 ‘max_distributed_connections’ 変数)。 このパラメータに適用されるスレッドは、それらのスレッドが同じ段階での問合せ処理パイプライン。 -たとえば、テーブルから読み込むときに、関数で式を評価することができる場合は、少なくともを使用してWHEREとGROUP BYの事前集計を並列に使用してフィル ‘max\_threads’ その後、スレッドの数 ‘max\_threads’ 使用されます。 +たとえば、テーブルから読み込むときに、関数で式を評価することができる場合は、少なくともを使用してWHEREとGROUP BYの事前集計を並列に使用してフィル ‘max_threads’ その後、スレッドの数 ‘max_threads’ 使用されます。 デフォルト値:物理CPUコアの数。 サーバーで同時に実行されるSELECTクエリが通常より少ない場合は、このパラメーターを実際のプロセッサコア数よりわずかに小さい値に設定します。 -制限のために迅速に完了するクエリの場合は、低い値を設定できます ‘max\_threads’. たとえば、必要な数のエントリがすべてのブロックにあり、max\_threads=8の場合、8つのブロックが取得されます。 +制限のために迅速に完了するクエリの場合は、低い値を設定できます ‘max_threads’. たとえば、必要な数のエントリがすべてのブロックにあり、max_threads=8の場合、8つのブロックが取得されます。 小さいほど `max_threads` 値は、より少ないメモリが消費されます。 -## max\_insert\_threads {#settings-max-insert-threads} +## max_insert_threads {#settings-max-insert-threads} 実行するスレッドの最大数 `INSERT SELECT` クエリ。 @@ -622,61 +622,61 @@ log_query_threads=1 デフォルト値は0です。 -平行 `INSERT SELECT` のみ効果があります。 `SELECT` パーツは並列で実行されます。 [max\_threads](#settings-max_threads) 設定。 +平行 `INSERT SELECT` のみ効果があります。 `SELECT` パーツは並列で実行されます。 [max_threads](#settings-max_threads) 設定。 値を大きくすると、メモリ使用量が増えます。 -## max\_compress\_block\_size {#max-compress-block-size} +## max_compress_block_size {#max-compress-block-size} テーブルに書き込むための圧縮前の非圧縮データのブロックの最大サイズ。 既定では、1,048,576(1MiB)です。 サイズを小さくすると、圧縮率が大幅に低下し、キャッシュの局所性のために圧縮と解凍の速度がわずかに増加し、メモリ消費が減少します。 通常、この設定を変更する理由はありません。 圧縮のためのブロック(バイトで構成されるメモリの塊)とクエリ処理のためのブロック(テーブルからの行のセット)を混同しないでください。 -## min\_compress\_block\_size {#min-compress-block-size} +## min_compress_block_size {#min-compress-block-size} -のために [メルゲツリー](../../engines/table-engines/mergetree-family/mergetree.md)"テーブル。 削減のため、遅延が処理クエリーのブロックの圧縮を書くとき、次のマークがそのサイズは少なくとも ‘min\_compress\_block\_size’. 既定では、65,536です。 +のために [メルゲツリー](../../engines/table-engines/mergetree-family/mergetree.md)"テーブル。 削減のため、遅延が処理クエリーのブロックの圧縮を書くとき、次のマークがそのサイズは少なくとも ‘min_compress_block_size’. 既定では、65,536です。 -圧縮されていないデータが以下の場合、ブロックの実際のサイズ ‘max\_compress\_block\_size’ は、この値以上であり、一つのマークのデータ量以上である。 +圧縮されていないデータが以下の場合、ブロックの実際のサイズ ‘max_compress_block_size’ は、この値以上であり、一つのマークのデータ量以上である。 -例を見てみましょう。 仮定すると ‘index\_granularity’ テーブル作成時に8192に設定されました。 +例を見てみましょう。 仮定すると ‘index_granularity’ テーブル作成時に8192に設定されました。 -UInt32型の列(値あたり4バイト)を書いています。 8192行を書き込むと、合計は32KBのデータになります。 Min\_compress\_block\_size=65,536ので、圧縮されたブロックはすべてのマークに対して形成されます。 +UInt32型の列(値あたり4バイト)を書いています。 8192行を書き込むと、合計は32KBのデータになります。 Min_compress_block_size=65,536ので、圧縮されたブロックはすべてのマークに対して形成されます。 文字列タイプ(値あたり60バイトの平均サイズ)のURL列を作成しています。 8192行を書き込むと、平均は500KBのデータよりわずかに小さくなります。 これは65,536以上であるため、各マークに圧縮されたブロックが形成されます。 この場合、単一のマークの範囲でディスクからデータを読み取るとき、余分なデータは解凍されません。 通常、この設定を変更する理由はありません。 -## max\_query\_size {#settings-max_query_size} +## max_query_size {#settings-max_query_size} SQLパーサーを使用して解析するためにRAMに取り込むことができるクエリの最大部分。 INSERTクエリには、別のストリームパーサー(O(1)RAMを消費する)によって処理されるINSERTのデータも含まれていますが、この制限には含まれていません。 デフォルト値:256KiB。 -## interactive\_delay {#interactive-delay} +## interactive_delay {#interactive-delay} 区間マイクロ秒単位で確認を行うための要求実行中止となり、送信を行います。 デフォルト値:100,000(キャンセルをチェックし、進行状況を秒単位で送信します)。 -## connect\_timeout,receive\_timeout,send\_timeout {#connect-timeout-receive-timeout-send-timeout} +## connect_timeout,receive_timeout,send_timeout {#connect-timeout-receive-timeout-send-timeout} クライアントとの通信に使用されるソケットの秒単位のタイムアウト。 デフォルト値:10,300,300。 -## cancel\_http\_readonly\_queries\_on\_client\_close {#cancel-http-readonly-queries-on-client-close} +## cancel_http_readonly_queries_on_client_close {#cancel-http-readonly-queries-on-client-close} Cancels HTTP read-only queries (e.g. SELECT) when a client closes the connection without waiting for the response. デフォルト値:0 -## poll\_interval {#poll-interval} +## poll_interval {#poll-interval} 指定された秒数の待機ループをロックします。 デフォルト値は10です。 -## max\_distributed\_connections {#max-distributed-connections} +## max_distributed_connections {#max-distributed-connections} 単一の分散テーブルへの単一のクエリの分散処理のためのリモートサーバーとの同時接続の最大数。 クラスター内のサーバー数以上の値を設定することをお勧めします。 @@ -684,20 +684,20 @@ Cancels HTTP read-only queries (e.g. SELECT) when a client closes the connectio 次のパラメーターは、分散テーブルを作成するとき(およびサーバーを起動するとき)にのみ使用されるため、実行時に変更する理由はありません。 -## distributed\_connections\_pool\_size {#distributed-connections-pool-size} +## distributed_connections_pool_size {#distributed-connections-pool-size} 単一の分散テーブルに対するすべてのクエリの分散処理のためのリモートサーバーとの同時接続の最大数。 クラスター内のサーバー数以上の値を設定することをお勧めします。 デフォルト値:1024。 -## connect\_timeout\_with\_failover\_ms {#connect-timeout-with-failover-ms} +## connect_timeout_with_failover_ms {#connect-timeout-with-failover-ms} 分散テーブルエンジンのリモートサーバーに接続するためのタイムアウト時間(ミリ秒)。 ‘shard’ と ‘replica’ セクションは、クラスタ定義で使用されます。 失敗した場合は、さまざまなレプリカへの接続が試行されます。 デフォルト値は50です。 -## connections\_with\_failover\_max\_tries {#connections-with-failover-max-tries} +## connections_with_failover_max_tries {#connections-with-failover-max-tries} 分散テーブルエンジンの各レプリカとの接続試行の最大数。 @@ -708,33 +708,33 @@ Cancels HTTP read-only queries (e.g. SELECT) when a client closes the connectio 極端な値(クエリ結果の列の最小値と最大値)を数えるかどうか。 0または1を受け入れます。 既定では、0(無効)です。 詳細については “Extreme values”. -## use\_uncompressed\_cache {#setting-use_uncompressed_cache} +## use_uncompressed_cache {#setting-use_uncompressed_cache} 非圧縮ブロックのキャッシュを使用するかどうか。 0または1を受け入れます。 既定では、0(無効)です。 -非圧縮キャッシュ(MergeTreeファミリ内のテーブルのみ)を使用すると、多数の短いクエリを処理する場合に、待ち時間を大幅に削減してスループットを向上させ この設定を有効にユーザーに送信頻繁に短います。 またに注意を払って下さい [uncompressed\_cache\_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuration parameter (only set in the config file) – the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed and the least-used data is automatically deleted. +非圧縮キャッシュ(MergeTreeファミリ内のテーブルのみ)を使用すると、多数の短いクエリを処理する場合に、待ち時間を大幅に削減してスループットを向上させ この設定を有効にユーザーに送信頻繁に短います。 またに注意を払って下さい [uncompressed_cache_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuration parameter (only set in the config file) – the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed and the least-used data is automatically deleted. -少なくとも幾分大きな量のデータ(百万行以上)を読み取るクエリの場合、圧縮されていないキャッシュは自動的に無効になり、本当に小さなクエリの これは保つことができることを意味する ‘use\_uncompressed\_cache’ 設定は常に1に設定します。 +少なくとも幾分大きな量のデータ(百万行以上)を読み取るクエリの場合、圧縮されていないキャッシュは自動的に無効になり、本当に小さなクエリの これは保つことができることを意味する ‘use_uncompressed_cache’ 設定は常に1に設定します。 -## replace\_running\_query {#replace-running-query} +## replace_running_query {#replace-running-query} -HTTPインターフェイスを使用する場合、 ‘query\_id’ 変数は渡すことができます。 これは、クエリ識別子として機能する任意の文字列です。 -同じユーザーからのクエリが同じ場合 ‘query\_id’ この時点で既に存在しているので、動作は ‘replace\_running\_query’ パラメータ。 +HTTPインターフェイスを使用する場合、 ‘query_id’ 変数は渡すことができます。 これは、クエリ識別子として機能する任意の文字列です。 +同じユーザーからのクエリが同じ場合 ‘query_id’ この時点で既に存在しているので、動作は ‘replace_running_query’ パラメータ。 -`0` (default) – Throw an exception (don't allow the query to run if a query with the same ‘query\_id’ すでに実行されています)。 +`0` (default) – Throw an exception (don't allow the query to run if a query with the same ‘query_id’ すでに実行されています)。 `1` – Cancel the old query and start running the new one. Yandex.Metricaこのパラメータセットが1の実施のための提案のための分割ます。 次の文字を入力した後、古いクエリがまだ終了していない場合は、キャンセルする必要があります。 -## stream\_flush\_interval\_ms {#stream-flush-interval-ms} +## stream_flush_interval_ms {#stream-flush-interval-ms} -作品のテーブルストリーミングの場合はタイムアウトした場合、またはスレッドを生成す [max\_insert\_block\_size](#settings-max_insert_block_size) 行。 +作品のテーブルストリーミングの場合はタイムアウトした場合、またはスレッドを生成す [max_insert_block_size](#settings-max_insert_block_size) 行。 既定値は7500です。 値が小さいほど、データがテーブルにフラッシュされる頻度が高くなります。 値を小さく設定すると、パフォーマンスが低下します。 -## load\_balancing {#settings-load_balancing} +## load_balancing {#settings-load_balancing} 分散クエリ処理に使用されるレプリカ選択のアルゴリズムを指定します。 @@ -787,31 +787,31 @@ load_balancing = first_or_random その `first_or_random` アルゴリズムはの問題を解決します `in_order` アルゴリズム と `in_order` あるレプリカがダウンした場合、残りのレプリカは通常のトラフィック量を処理しますが、次のレプリカは二重負荷を受けます。 を使用する場合 `first_or_random` アルゴリズムでは、負荷はまだ利用可能なレプリカ間で均等に分散されます。 -## prefer\_localhost\_replica {#settings-prefer-localhost-replica} +## prefer_localhost_replica {#settings-prefer-localhost-replica} を有効/無効にしが好ましいのlocalhostレプリカ処理時に分布します。 可能な値: - 1 — ClickHouse always sends a query to the localhost replica if it exists. -- 0 — ClickHouse uses the balancing strategy specified by the [load\_balancing](#settings-load_balancing) 設定。 +- 0 — ClickHouse uses the balancing strategy specified by the [load_balancing](#settings-load_balancing) 設定。 デフォルト値:1。 !!! warning "警告" - 使用する場合は、この設定を無効にします [max\_parallel\_replicas](#settings-max_parallel_replicas). + 使用する場合は、この設定を無効にします [max_parallel_replicas](#settings-max_parallel_replicas). -## totals\_mode {#totals-mode} +## totals_mode {#totals-mode} -有するときの合計の計算方法、およびmax\_rows\_to\_group\_byおよびgroup\_by\_overflow\_mode= ‘any’ 存在する。 +有するときの合計の計算方法、およびmax_rows_to_group_byおよびgroup_by_overflow_mode= ‘any’ 存在する。 セクションを参照 “WITH TOTALS modifier”. -## totals\_auto\_threshold {#totals-auto-threshold} +## totals_auto_threshold {#totals-auto-threshold} のしきい値 `totals_mode = 'auto'`. セクションを参照 “WITH TOTALS modifier”. -## max\_parallel\_replicas {#settings-max_parallel_replicas} +## max_parallel_replicas {#settings-max_parallel_replicas} クエリ実行時の各シャードのレプリカの最大数。 のための一貫性を異なる部分に同じデータを分割)、このオプションにしているときだけサンプリングキーを設定します。 @@ -824,7 +824,7 @@ load_balancing = first_or_random コンパイルは、クエリ処理パイプラインの一部にのみ使用されます。 この部分のパイプラインのためのクエリを実行するアによる展開の短サイクルinlining集計機能。 複数の単純な集計関数を使用するクエリでは、最大のパフォーマンスの向上が見られます。 通常、性能は軽微であります。 非常に珍しい例で遅くなクエリを実行します。 -## min\_count\_to\_compile {#min-count-to-compile} +## min_count_to_compile {#min-count-to-compile} り方を潜在的に利用コチャンクのコードの実行前に作成する。 デフォルトでは3. For testing, the value can be set to 0: compilation runs synchronously and the query waits for the end of the compilation process before continuing execution. For all other cases, use values ​​starting with 1. Compilation normally takes about 5-10 seconds. @@ -833,27 +833,27 @@ For testing, the value can be set to 0: compilation runs synchronously and the q コンパイルされたコードは、クエリで使用される集計関数とGROUP BY句内のキーの種類のそれぞれの異なる組み合わせに必要です。 The results of the compilation are saved in the build directory in the form of .so files. There is no restriction on the number of compilation results since they don't use very much space. Old results will be used after server restarts, except in the case of a server upgrade – in this case, the old results are deleted. -## output\_format\_json\_quote\_64bit\_integers {#session_settings-output_format_json_quote_64bit_integers} +## output_format_json_quote_64bit_integers {#session_settings-output_format_json_quote_64bit_integers} 値がtrueの場合、json\*Int64およびUInt64形式(ほとんどのJavaScript実装との互換性のため)を使用するときに整数が引用符で表示されます。 -## format\_csv\_delimiter {#settings-format_csv_delimiter} +## format_csv_delimiter {#settings-format_csv_delimiter} CSVデータの区切り文字として解釈される文字。 デフォルトでは、区切り文字は `,`. -## input\_format\_csv\_unquoted\_null\_literal\_as\_null {#settings-input_format_csv_unquoted_null_literal_as_null} +## input_format_csv_unquoted_null_literal_as_null {#settings-input_format_csv_unquoted_null_literal_as_null} CSV入力形式の場合、引用符なしの解析を有効または無効にします `NULL` リテラルとして(のシノニム `\N`). -## output\_format\_csv\_crlf\_end\_of\_line {#settings-output-format-csv-crlf-end-of-line} +## output_format_csv_crlf_end_of_line {#settings-output-format-csv-crlf-end-of-line} CSVでは、UNIXスタイル(LF)の代わりにDOS/Windowsスタイルの行区切り記号(CRLF)を使用します。 -## output\_format\_tsv\_crlf\_end\_of\_line {#settings-output-format-tsv-crlf-end-of-line} +## output_format_tsv_crlf_end_of_line {#settings-output-format-tsv-crlf-end-of-line} TSVでは、UNIXスタイル(LF)の代わりにDOC/Windowsスタイルの行区切り記号(CRLF)を使用します。 -## insert\_quorum {#settings-insert_quorum} +## insert_quorum {#settings-insert_quorum} を決議の定足数を書き込みます. @@ -868,7 +868,7 @@ TSVでは、UNIXスタイル(LF)の代わりにDOC/Windowsスタイルの行区 クォーラム内のすべてのレプリカは一貫性があります。 `INSERT` クエリ。 その `INSERT` シーケンスは線形化されます。 -から書き込まれたデータを読み取ると `insert_quorum` を使用することができます [select\_sequential\_consistency](#settings-select_sequential_consistency) オプション +から書き込まれたデータを読み取ると `insert_quorum` を使用することができます [select_sequential_consistency](#settings-select_sequential_consistency) オプション ClickHouseは例外を生成します @@ -877,10 +877,10 @@ ClickHouseは例外を生成します も参照。: -- [insert\_quorum\_timeout](#settings-insert_quorum_timeout) -- [select\_sequential\_consistency](#settings-select_sequential_consistency) +- [insert_quorum_timeout](#settings-insert_quorum_timeout) +- [select_sequential_consistency](#settings-select_sequential_consistency) -## insert\_quorum\_timeout {#settings-insert_quorum_timeout} +## insert_quorum_timeout {#settings-insert_quorum_timeout} 書き込み数が定員タイムアウトを秒で指定します。 タイムアウトが経過し、まだ書き込みが行われていない場合、ClickHouseは例外を生成し、クライアントは同じまたは他のレプリカに同じブロックを書き込む @@ -888,10 +888,10 @@ ClickHouseは例外を生成します も参照。: -- [insert\_quorum](#settings-insert_quorum) -- [select\_sequential\_consistency](#settings-select_sequential_consistency) +- [insert_quorum](#settings-insert_quorum) +- [select_sequential_consistency](#settings-select_sequential_consistency) -## select\_sequential\_consistency {#settings-select_sequential_consistency} +## select_sequential_consistency {#settings-select_sequential_consistency} の順次整合性を有効または無効にします `SELECT` クエリ: @@ -908,10 +908,10 @@ ClickHouseは例外を生成します も参照。: -- [insert\_quorum](#settings-insert_quorum) -- [insert\_quorum\_timeout](#settings-insert_quorum_timeout) +- [insert_quorum](#settings-insert_quorum) +- [insert_quorum_timeout](#settings-insert_quorum_timeout) -## insert\_deduplicate {#settings-insert-deduplicate} +## insert_deduplicate {#settings-insert-deduplicate} 重複除外のブロックを有効または無効にします。 `INSERT` (複製された\*テーブルの場合)。 @@ -924,7 +924,7 @@ ClickHouseは例外を生成します デフォルトでは、ブロックは `INSERT` ステートメントは重複排除されます [データ複製](../../engines/table-engines/mergetree-family/replication.md)). -## deduplicate\_blocks\_in\_dependent\_materialized\_views {#settings-deduplicate-blocks-in-dependent-materialized-views} +## deduplicate_blocks_in_dependent_materialized_views {#settings-deduplicate-blocks-in-dependent-materialized-views} を有効または無効にし、重複排除圧縮をチェックを実現し意見を受け取るデータから複製\*ます。 @@ -942,7 +942,7 @@ ClickHouseは例外を生成します 同時に、この動作 “breaks” `INSERT` べき等性。 もし `INSERT` メインテーブルに成功したと `INSERT` into a materialized view failed (e.g. because of communication failure with Zookeeper) a client will get an error and can retry the operation. However, the materialized view won't receive the second insert because it will be discarded by deduplication in the main (source) table. The setting `deduplicate_blocks_in_dependent_materialized_views` この動作を変更できます。 再試行すると、マテリアライズドビューは繰り返し挿入を受け取り、重複除外チェックを単独で実行します, ソーステーブルのチェック結果を無視し、最初の失敗のために失われた行を挿入します。 -## max\_network\_bytes {#settings-max-network-bytes} +## max_network_bytes {#settings-max-network-bytes} クエリの実行時にネットワーク経由で受信または送信されるデータ量(バイト単位)を制限します。 この設定は、個々のクエリごとに適用されます。 @@ -953,7 +953,7 @@ ClickHouseは例外を生成します デフォルト値は0です。 -## max\_network\_bandwidth {#settings-max-network-bandwidth} +## max_network_bandwidth {#settings-max-network-bandwidth} ネットワーク上でのデータ交換の速度を毎秒バイト単位で制限します。 この設定はすべての照会に適用されます。 @@ -964,7 +964,7 @@ ClickHouseは例外を生成します デフォルト値は0です。 -## max\_network\_bandwidth\_for\_user {#settings-max-network-bandwidth-for-user} +## max_network_bandwidth_for_user {#settings-max-network-bandwidth-for-user} ネットワーク上でのデータ交換の速度を毎秒バイト単位で制限します。 この設定は、単一ユーザーが同時に実行するすべてのクエリに適用されます。 @@ -975,7 +975,7 @@ ClickHouseは例外を生成します デフォルト値は0です。 -## max\_network\_bandwidth\_for\_all\_users {#settings-max-network-bandwidth-for-all-users} +## max_network_bandwidth_for_all_users {#settings-max-network-bandwidth-for-all-users} ネットワーク経由でデータが交換される速度を毎秒バイト単位で制限します。 この設定が適用されるのはすべての同時走行に関するお問い合わせます。 @@ -986,7 +986,7 @@ ClickHouseは例外を生成します デフォルト値は0です。 -## count\_distinct\_implementation {#settings-count_distinct_implementation} +## count_distinct_implementation {#settings-count_distinct_implementation} これは、 `uniq*` を実行するために使用する必要があります。 [COUNT(DISTINCT …)](../../sql-reference/aggregate-functions/reference.md#agg_function-count) 建設。 @@ -1000,7 +1000,7 @@ ClickHouseは例外を生成します デフォルト値: `uniqExact`. -## skip\_unavailable\_shards {#settings-skip_unavailable_shards} +## skip_unavailable_shards {#settings-skip_unavailable_shards} を有効または無効にし静キの不可欠片. @@ -1030,13 +1030,13 @@ ClickHouseは例外を生成します デフォルト値は0です。 -## optimize\_skip\_unused\_shards {#settings-optimize_skip_unused_shards} +## optimize_skip_unused_shards {#settings-optimize_skip_unused_shards} PREWHERE/WHEREにシャーディングキー条件があるSELECTクエリの未使用のシャードのスキップを有効または無効にします(データがシャーディングキーによって配布される デフォルト値:0 -## force\_optimize\_skip\_unused\_shards {#settings-force_optimize_skip_unused_shards} +## force_optimize_skip_unused_shards {#settings-force_optimize_skip_unused_shards} を有効または無効にしクエリの実行の場合 [`optimize_skip_unused_shards`](#settings-optimize_skip_unused_shards) 未使用のシャードを有効にしてスキップすることはできません。 スキップが不可能で、設定が有効になっている場合は例外がスローされます。 @@ -1048,7 +1048,7 @@ PREWHERE/WHEREにシャーディングキー条件があるSELECTクエリの未 デフォルト値:0 -## optimize\_throw\_if\_noop {#setting-optimize_throw_if_noop} +## optimize_throw_if_noop {#setting-optimize_throw_if_noop} 例外のスローを有効または無効にします。 [OPTIMIZE](../../sql-reference/statements/misc.md#misc_operations-optimize) クエリがマージを実行しませんでした。 @@ -1066,7 +1066,7 @@ PREWHERE/WHEREにシャーディングキー条件があるSELECTクエリの未 - タイプ:秒 - デフォルト値:60秒 -分散テーブルのエラーをゼロにする速度を制御します。 レプリカがしばらく使用できず、5つのエラーが蓄積され、distributed\_replica\_error\_half\_lifeが1秒に設定されている場合、レプリカは最後のエラーの3秒後に正常と見なされま +分散テーブルのエラーをゼロにする速度を制御します。 レプリカがしばらく使用できず、5つのエラーが蓄積され、distributed_replica_error_half_lifeが1秒に設定されている場合、レプリカは最後のエラーの3秒後に正常と見なされま も参照。: @@ -1085,7 +1085,7 @@ PREWHERE/WHEREにシャーディングキー条件があるSELECTクエリの未 - [分散テーブルエンジン](../../engines/table-engines/special/distributed.md) - [ディストリビューター](#settings-distributed_replica_error_half_life) -## distributed\_directory\_monitor\_sleep\_time\_ms {#distributed_directory_monitor_sleep_time_ms} +## distributed_directory_monitor_sleep_time_ms {#distributed_directory_monitor_sleep_time_ms} の基本区間 [分散](../../engines/table-engines/special/distributed.md) データを送信する表エンジン。 実際の間隔は、エラーが発生した場合に指数関数的に増加します。 @@ -1095,9 +1095,9 @@ PREWHERE/WHEREにシャーディングキー条件があるSELECTクエリの未 既定値は100ミリ秒です。 -## distributed\_directory\_monitor\_max\_sleep\_time\_ms {#distributed_directory_monitor_max_sleep_time_ms} +## distributed_directory_monitor_max_sleep_time_ms {#distributed_directory_monitor_max_sleep_time_ms} -の最大間隔 [分散](../../engines/table-engines/special/distributed.md) データを送信する表エンジン。 の区間の指数関数的成長を制限する。 [distributed\_directory\_monitor\_sleep\_time\_ms](#distributed_directory_monitor_sleep_time_ms) 設定。 +の最大間隔 [分散](../../engines/table-engines/special/distributed.md) データを送信する表エンジン。 の区間の指数関数的成長を制限する。 [distributed_directory_monitor_sleep_time_ms](#distributed_directory_monitor_sleep_time_ms) 設定。 可能な値: @@ -1105,7 +1105,7 @@ PREWHERE/WHEREにシャーディングキー条件があるSELECTクエリの未 デフォルト値:30000ミリ秒(30秒)。 -## distributed\_directory\_monitor\_batch\_inserts {#distributed_directory_monitor_batch_inserts} +## distributed_directory_monitor_batch_inserts {#distributed_directory_monitor_batch_inserts} 挿入されたデータのバッチ送信を有効または無効にします。 @@ -1118,7 +1118,7 @@ PREWHERE/WHEREにシャーディングキー条件があるSELECTクエリの未 デフォルト値は0です。 -## os\_thread\_priority {#setting-os-thread-priority} +## os_thread_priority {#setting-os-thread-priority} 優先度を設定します ([ニース](https://en.wikipedia.org/wiki/Nice_(Unix)))クエリを実行するスレッドの場合。 OSスケジューラは、使用可能な各CPUコアで実行する次のスレッドを選択する際に、この優先順位を考慮します。 @@ -1133,7 +1133,7 @@ PREWHERE/WHEREにシャーディングキー条件があるSELECTクエリの未 デフォルト値は0です。 -## query\_profiler\_real\_time\_period\_ns {#query_profiler_real_time_period_ns} +## query_profiler_real_time_period_ns {#query_profiler_real_time_period_ns} の実クロックタイマーの期間を設定します。 [クエリプロファイラ](../../operations/optimizing-performance/sampling-query-profiler.md). リアルクロックタイマーカウント壁掛時計。 @@ -1154,9 +1154,9 @@ PREWHERE/WHEREにシャーディングキー条件があるSELECTクエリの未 も参照。: -- システム表 [trace\_log](../../operations/system-tables.md#system_tables-trace_log) +- システム表 [trace_log](../../operations/system-tables.md#system_tables-trace_log) -## query\_profiler\_cpu\_time\_period\_ns {#query_profiler_cpu_time_period_ns} +## query_profiler_cpu_time_period_ns {#query_profiler_cpu_time_period_ns} のCPUクロックタイマーの期間を設定します。 [クエリプロファイラ](../../operations/optimizing-performance/sampling-query-profiler.md). このタイマーカウントのみのCPU時間。 @@ -1177,9 +1177,9 @@ PREWHERE/WHEREにシャーディングキー条件があるSELECTクエリの未 も参照。: -- システム表 [trace\_log](../../operations/system-tables.md#system_tables-trace_log) +- システム表 [trace_log](../../operations/system-tables.md#system_tables-trace_log) -## allow\_introspection\_functions {#settings-allow_introspection_functions} +## allow_introspection_functions {#settings-allow_introspection_functions} ディスエーブルの有効 [イントロスペクション関数](../../sql-reference/functions/introspection.md) クエリプロファイル用。 @@ -1193,23 +1193,23 @@ PREWHERE/WHEREにシャーディングキー条件があるSELECTクエリの未 **も参照。** - [サンプリングクロファイラ](../optimizing-performance/sampling-query-profiler.md) -- システム表 [trace\_log](../../operations/system-tables.md#system_tables-trace_log) +- システム表 [trace_log](../../operations/system-tables.md#system_tables-trace_log) -## input\_format\_parallel\_parsing {#input-format-parallel-parsing} +## input_format_parallel_parsing {#input-format-parallel-parsing} - タイプ:bool - 既定値:True データ形式の順序保持並列解析を有効にします。 TSV、TKSV、CSVおよびJSONEachRow形式でのみサポートされます。 -## min\_chunk\_bytes\_for\_parallel\_parsing {#min-chunk-bytes-for-parallel-parsing} +## min_chunk_bytes_for_parallel_parsing {#min-chunk-bytes-for-parallel-parsing} - 型:unsigned int - デフォルト値:1MiB 各スレッドが並列に解析する最小チャンクサイズをバイト単位で表します。 -## output\_format\_avro\_codec {#settings-output_format_avro_codec} +## output_format_avro_codec {#settings-output_format_avro_codec} 出力Avroファイルに使用する圧縮コーデックを設定します。 @@ -1223,7 +1223,7 @@ PREWHERE/WHEREにシャーディングキー条件があるSELECTクエリの未 デフォルト値: `snappy` (利用可能な場合)または `deflate`. -## output\_format\_avro\_sync\_interval {#settings-output_format_avro_sync_interval} +## output_format_avro_sync_interval {#settings-output_format_avro_sync_interval} 出力Avroファイルの同期マーカー間の最小データサイズ(バイト単位)を設定します。 @@ -1233,7 +1233,7 @@ PREWHERE/WHEREにシャーディングキー条件があるSELECTクエリの未 デフォルト値:32768(32KiB) -## format\_avro\_schema\_registry\_url {#settings-format_avro_schema_registry_url} +## format_avro_schema_registry_url {#settings-format_avro_schema_registry_url} 使用するConfluentスキーマレジストリURLを設定します [アブロコンフルエント](../../interfaces/formats.md#data-format-avro-confluent) 形式 @@ -1241,7 +1241,7 @@ PREWHERE/WHEREにシャーディングキー条件があるSELECTクエリの未 既定値:空 -## background\_pool\_size {#background_pool_size} +## background_pool_size {#background_pool_size} セットのスレッド数を行う背景事業のテーブルエンジン(例えば、合併に [MergeTreeエンジン](../../engines/table-engines/mergetree-family/index.md) テーブル)。 この設定はClickHouseサーバーの起動時に適用され、ユーザーセッションでは変更できません。 この設定を調整することで、CPUとディスクの負荷を管理します。 小さなプールサイズを以下のCPUやディスクの資源が背景のプロセスの事前の遅れが影響をクエリす。 diff --git a/docs/ja/operations/system-tables.md b/docs/ja/operations/system-tables.md index e7488b0fa44..095038b2b72 100644 --- a/docs/ja/operations/system-tables.md +++ b/docs/ja/operations/system-tables.md @@ -13,7 +13,7 @@ toc_title: "\u30B7\u30B9\u30C6\u30E0\u8868" システムテーブルは読み取り専用です。 彼らはに位置しています ‘system’ データベース。 -## システムasynchronous\_metrics {#system_tables-asynchronous_metrics} +## システムasynchronous_metrics {#system_tables-asynchronous_metrics} バックグラウンドで定期的に計算される指標が含まれます。 例えば、使用中のRAMの量。 @@ -48,7 +48,7 @@ SELECT * FROM system.asynchronous_metrics LIMIT 10 - [監視](monitoring.md) — Base concepts of ClickHouse monitoring. - [システムメトリック](#system_tables-metrics) — Contains instantly calculated metrics. - [システムイベント](#system_tables-events) — Contains a number of events that have occurred. -- [システムmetric\_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [システムmetric_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. ## システムクラスタ {#system-clusters} @@ -72,8 +72,8 @@ SELECT * FROM system.asynchronous_metrics LIMIT 10 **も参照。** - [分散テーブルエンジン](../engines/table-engines/special/distributed.md) -- [distributed\_replica\_error\_cap設定](settings/settings.md#settings-distributed_replica_error_cap) -- [distributed\_replica\_error\_half\_life設定](settings/settings.md#settings-distributed_replica_error_half_life) +- [distributed_replica_error_cap設定](settings/settings.md#settings-distributed_replica_error_cap) +- [distributed_replica_error_half_life設定](settings/settings.md#settings-distributed_replica_error_half_life) ## システム列 {#system-columns} @@ -145,7 +145,7 @@ SELECT * FROM system.contributors WHERE name='Olga Khvostikova' 各データベースのサーバーについて知っていて対応するエントリの表に示す。 このシステムテーブルは、 `SHOW DATABASES` クエリ。 -## システムdetached\_parts {#system_tables-detached_parts} +## システムdetached_parts {#system_tables-detached_parts} についての情報が含まれて外部 [メルゲツリー](../engines/table-engines/mergetree-family/mergetree.md) テーブル その `reason` column部品が切り離された理由を指定します。 ユーザーが取り外した部品の場合、その理由は空です。 このような部品は、 [ALTER TABLE ATTACH PARTITION\|PART](../sql-reference/statements/alter.md#alter_attach-partition) コマンド その他の列の説明については、 [システム部品](#system_tables-parts). パーツ名が無効な場合、一部のカラムの値は次のようになります `NULL`. このような部分は、以下で削除できます [ALTER TABLE DROP DETACHED PART](../sql-reference/statements/alter.md#alter_drop-detached). @@ -239,9 +239,9 @@ SELECT * FROM system.events LIMIT 5 **も参照。** -- [システムasynchronous\_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [システムasynchronous_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. - [システムメトリック](#system_tables-metrics) — Contains instantly calculated metrics. -- [システムmetric\_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [システムmetric_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. - [監視](monitoring.md) — Base concepts of ClickHouse monitoring. ## システム関数 {#system-functions} @@ -253,9 +253,9 @@ SELECT * FROM system.events LIMIT 5 - `name`(`String`) – The name of the function. - `is_aggregate`(`UInt8`) — Whether the function is aggregate. -## システムgraphite\_retentions {#system-graphite-retentions} +## システムgraphite_retentions {#system-graphite-retentions} -パラメ [graphite\_rollup](server-configuration-parameters/settings.md#server_configuration_parameters-graphite) テーブルで使用される [\*GraphiteMergeTree](../engines/table-engines/mergetree-family/graphitemergetree.md) エンジンだ +パラメ [graphite_rollup](server-configuration-parameters/settings.md#server_configuration_parameters-graphite) テーブルで使用される [\*GraphiteMergeTree](../engines/table-engines/mergetree-family/graphitemergetree.md) エンジンだ 列: @@ -324,12 +324,12 @@ SELECT * FROM system.metrics LIMIT 10 **も参照。** -- [システムasynchronous\_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [システムasynchronous_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. - [システムイベント](#system_tables-events) — Contains a number of events that occurred. -- [システムmetric\_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [システムmetric_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. - [監視](monitoring.md) — Base concepts of ClickHouse monitoring. -## システムmetric\_log {#system_tables-metric_log} +## システムmetric_log {#system_tables-metric_log} を含む履歴メトリクスの値からテーブル `system.metrics` と `system.events`、定期的にディスクにフラッシュ。 メトリック履歴の収集を有効にするには `system.metric_log`,作成 `/etc/clickhouse-server/config.d/metric_log.xml` 次の内容を使って: @@ -380,7 +380,7 @@ CurrentMetric_ReplicatedChecks: 0 **も参照。** -- [システムasynchronous\_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [システムasynchronous_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. - [システムイベント](#system_tables-events) — Contains a number of events that occurred. - [システムメトリック](#system_tables-metrics) — Contains instantly calculated metrics. - [監視](monitoring.md) — Base concepts of ClickHouse monitoring. @@ -391,7 +391,7 @@ CurrentMetric_ReplicatedChecks: 0 このテーブルは、テストのため、またはブルートフォース検索を行う必要がある場合に使用できます。 この表からの読み取りは並列化されません。 -## システムnumbers\_mt {#system-numbers-mt} +## システムnumbers_mt {#system-numbers-mt} と同じ ‘system.numbers’ しかし、読み取りは並列処理されます。 番号は任意の順序で返すことができます。 テストに使用されます。 @@ -483,9 +483,9 @@ CurrentMetric_ReplicatedChecks: 0 - `marks_size` (`UInt64`) – Alias for `marks_bytes`. -## システムpart\_log {#system_tables-part-log} +## システムpart_log {#system_tables-part-log} -その `system.part_log` テーブルが作成されるのは、 [part\_log](server-configuration-parameters/settings.md#server_configuration_parameters-part-log) サーバ設定を指定します。 +その `system.part_log` テーブルが作成されるのは、 [part_log](server-configuration-parameters/settings.md#server_configuration_parameters-part-log) サーバ設定を指定します。 このテーブルについての情報が含まれてイベントが発生した [データパーツ](../engines/table-engines/mergetree-family/custom-partitioning-key.md) で [メルゲツリー](../engines/table-engines/mergetree-family/mergetree.md) データの追加やマージなどのファミリテーブル。 @@ -528,11 +528,11 @@ CurrentMetric_ReplicatedChecks: 0 - `rows_read` (UInt64) – The number of rows read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. - `bytes_read` (UInt64) – The number of uncompressed bytes read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. - `total_rows_approx` (UInt64) – The approximation of the total number of rows that should be read. For distributed processing, on the requestor server, this is the total for all remote servers. It can be updated during request processing, when new sources to process become known. -- `memory_usage` (UInt64) – Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max\_memory\_usage](../operations/settings/query-complexity.md#settings_max_memory_usage) 設定。 +- `memory_usage` (UInt64) – Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max_memory_usage](../operations/settings/query-complexity.md#settings_max_memory_usage) 設定。 - `query` (String) – The query text. For `INSERT`,挿入するデータは含まれません。 - `query_id` (String) – Query ID, if defined. -## システムtext\_log {#system_tables-text_log} +## システムtext_log {#system_tables-text_log} を含むログイン作品の応募がありました。 ログレベルがこのテーブルで限定 `text_log.level` サーバー設定。 @@ -559,16 +559,16 @@ CurrentMetric_ReplicatedChecks: 0 - `source_file` (`LowCardinality(String)`)-ロギングが行われたソースファイル。 - `source_line` (`UInt64`)-ロギングが行われたソース行。 -## システムquery\_log {#system_tables-query_log} +## システムquery_log {#system_tables-query_log} クエリの実行に関する情報が含まれます。 クエリごとに、処理開始時間、処理時間、エラーメッセージおよびその他の情報を確認できます。 !!! note "注" テーブルには以下の入力データは含まれません `INSERT` クエリ。 -ClickHouseはこのテーブルを作成します。 [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) serverパラメータを指定します。 このパラメーターは、クエリがログインするテーブルのログ間隔や名前などのログルールを設定します。 +ClickHouseはこのテーブルを作成します。 [query_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) serverパラメータを指定します。 このパラメーターは、クエリがログインするテーブルのログ間隔や名前などのログルールを設定します。 -クエリロギングを有効にするには、 [log\_queries](settings/settings.md#settings-log-queries) パラメータは1。 詳細については、 [設定](settings/settings.md) セクション +クエリロギングを有効にするには、 [log_queries](settings/settings.md#settings-log-queries) パラメータは1。 詳細については、 [設定](settings/settings.md) セクション その `system.query_log` テーブルレジスタの種類は問合せ: @@ -636,22 +636,22 @@ ClickHouseはこのテーブルを作成します。 [query\_log](server-configu 2. クエリ処理中にエラーが発生した場合は、タイプ1と4のイベントが作成されます。 3. クエリを起動する前にエラーが発生した場合は、タイプ3の単一のイベントが作成されます。 -既定では、ログは7.5秒間隔でテーブルに追加されます。 この間隔は [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) サーバ設定(参照 `flush_interval_milliseconds` 変数)。 ログをメモリバッファからテーブルに強制的にフラッシュするには、 `SYSTEM FLUSH LOGS` クエリ。 +既定では、ログは7.5秒間隔でテーブルに追加されます。 この間隔は [query_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) サーバ設定(参照 `flush_interval_milliseconds` 変数)。 ログをメモリバッファからテーブルに強制的にフラッシュするには、 `SYSTEM FLUSH LOGS` クエリ。 テーブルを手動で削除すると、その場で自動的に作成されます。 以前のログはすべて削除されます。 !!! note "注" ログの保存期間は無制限です。 ログはテーブルから自動的には削除されません。 古いログの削除を自分で整理する必要があります。 -パーティショニングキーを指定できます。 `system.query_log` のテーブル [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) サーバ設定(参照 `partition_by` 変数)。 +パーティショニングキーを指定できます。 `system.query_log` のテーブル [query_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) サーバ設定(参照 `partition_by` 変数)。 -## システムquery\_thread\_log {#system_tables-query-thread-log} +## システムquery_thread_log {#system_tables-query-thread-log} のテーブルについての情報が含まれてそれぞれの検索キーワード実行スレッド. -ClickHouseはこのテーブルを作成します。 [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) serverパラメータを指定します。 このパラメーターは、クエリがログインするテーブルのログ間隔や名前などのログルールを設定します。 +ClickHouseはこのテーブルを作成します。 [query_thread_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) serverパラメータを指定します。 このパラメーターは、クエリがログインするテーブルのログ間隔や名前などのログルールを設定します。 -クエリロギングを有効にするには、 [log\_query\_threads](settings/settings.md#settings-log-query-threads) パラメータは1。 詳細については、 [設定](settings/settings.md) セクション +クエリロギングを有効にするには、 [log_query_threads](settings/settings.md#settings-log-query-threads) パラメータは1。 詳細については、 [設定](settings/settings.md) セクション 列: @@ -701,20 +701,20 @@ ClickHouseはこのテーブルを作成します。 [query\_thread\_log](server - `ProfileEvents.Names` (Array(String)) — Counters that measure different metrics for this thread. The description of them could be found in the table [システムイベント](#system_tables-events) - `ProfileEvents.Values` (Array(UInt64)) — Values of metrics for this thread that are listed in the `ProfileEvents.Names` 列。 -既定では、ログは7.5秒間隔でテーブルに追加されます。 この間隔は [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) サーバ設定(参照 `flush_interval_milliseconds` 変数)。 ログをメモリバッファからテーブルに強制的にフラッシュするには、 `SYSTEM FLUSH LOGS` クエリ。 +既定では、ログは7.5秒間隔でテーブルに追加されます。 この間隔は [query_thread_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) サーバ設定(参照 `flush_interval_milliseconds` 変数)。 ログをメモリバッファからテーブルに強制的にフラッシュするには、 `SYSTEM FLUSH LOGS` クエリ。 テーブルを手動で削除すると、その場で自動的に作成されます。 以前のログはすべて削除されます。 !!! note "注" ログの保存期間は無制限です。 ログはテーブルから自動的には削除されません。 古いログの削除を自分で整理する必要があります。 -パーティショニングキーを指定できます。 `system.query_thread_log` のテーブル [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) サーバ設定(参照 `partition_by` 変数)。 +パーティショニングキーを指定できます。 `system.query_thread_log` のテーブル [query_thread_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) サーバ設定(参照 `partition_by` 変数)。 -## システムtrace\_log {#system_tables-trace_log} +## システムtrace_log {#system_tables-trace_log} を含むスタックトレースの収集、サンプリングクロファイラ. -ClickHouseはこのテーブルを作成します。 [trace\_log](server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) サーバの設定が設定されます。 また、 [query\_profiler\_real\_time\_period\_ns](settings/settings.md#query_profiler_real_time_period_ns) と [query\_profiler\_cpu\_time\_period\_ns](settings/settings.md#query_profiler_cpu_time_period_ns) 設定は設定する必要があります。 +ClickHouseはこのテーブルを作成します。 [trace_log](server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) サーバの設定が設定されます。 また、 [query_profiler_real_time_period_ns](settings/settings.md#query_profiler_real_time_period_ns) と [query_profiler_cpu_time_period_ns](settings/settings.md#query_profiler_cpu_time_period_ns) 設定は設定する必要があります。 ログを分析するには、 `addressToLine`, `addressToSymbol` と `demangle` イントロスペクション関数。 @@ -737,7 +737,7 @@ ClickHouseはこのテーブルを作成します。 [trace\_log](server-configu - `thread_number` ([UInt32](../sql-reference/data-types/int-uint.md)) — Thread identifier. -- `query_id` ([文字列](../sql-reference/data-types/string.md)) — Query identifier that can be used to get details about a query that was running from the [query\_log](#system_tables-query_log) システムテーブル。 +- `query_id` ([文字列](../sql-reference/data-types/string.md)) — Query identifier that can be used to get details about a query that was running from the [query_log](#system_tables-query_log) システムテーブル。 - `trace` ([配列(UInt64)](../sql-reference/data-types/array.md)) — Stack trace at the moment of sampling. Each element is a virtual memory address inside ClickHouse server process. @@ -824,7 +824,7 @@ active_replicas: 2 - `parts_to_check` (`UInt32`)-検証のためのキュー内のデータ部分の数。 破損の疑いがある場合は、部品を検証キューに入れます。 - `zookeeper_path` (`String`)-ZooKeeperのテーブルデータへのパス。 - `replica_name` (`String`)-飼育係のレプリカ名。 同じテーブルの異なるレプリカの名前は異なります。 -- `replica_path` (`String`)-ZooKeeperのレプリカデータへのパス。 連結と同じ ‘zookeeper\_path/replicas/replica\_path’. +- `replica_path` (`String`)-ZooKeeperのレプリカデータへのパス。 連結と同じ ‘zookeeper_path/replicas/replica_path’. - `columns_version` (`Int32`)-テーブル構造のバージョン番号。 変更が実行された回数を示します。 場合にレプリカは異なるバージョンで一部のレプリカさんのすべての変更はまだない。 - `queue_size` (`UInt32`)-実行待ちの操作のキューのサイズ。 操作には、データのブロックの挿入、マージ、その他の特定の操作が含まれます。 それは通常と一致します `future_parts`. - `inserts_in_queue` (`UInt32`)-必要なデータブロックの挿入数。 挿入は通常、かなり迅速に複製されます。 この数が大きい場合は、何かが間違っていることを意味します。 @@ -845,7 +845,7 @@ active_replicas: 2 - `active_replicas` (`UInt8`)-ZooKeeperにセッションがあるこのテーブルのレプリカの数(つまり、機能しているレプリカの数)。 すべての列を要求すると、ZooKeeperからのいくつかの読み取りが行ごとに行われるため、テーブルは少し遅く動作する可能性があります。 -最後の4つの列(log\_max\_index、log\_pointer、total\_replicas、active\_replicas)を要求しないと、テーブルはすぐに機能します。 +最後の4つの列(log_max_index、log_pointer、total_replicas、active_replicas)を要求しないと、テーブルはすぐに機能します。 たとえば、次のようにすべてが正常に動作していることを確認できます: @@ -932,7 +932,7 @@ SELECT * FROM system.settings WHERE changed AND name='load_balancing' - [クエリの権限](settings/permissions-for-queries.md#settings_readonly) - [設定の制約](settings/constraints-on-settings.md) -## システムtable\_engines {#system.table_engines} +## システムtable_engines {#system.table_engines} ``` text ┌─name───────────────────┬─value───────┐ @@ -943,7 +943,7 @@ SELECT * FROM system.settings WHERE changed AND name='load_balancing' └────────────────────────┴─────────────┘ ``` -## システムmerge\_tree\_settings {#system-merge_tree_settings} +## システムmerge_tree_settings {#system-merge_tree_settings} の設定に関する情報が含まれます `MergeTree` テーブル @@ -955,7 +955,7 @@ SELECT * FROM system.settings WHERE changed AND name='load_balancing' - `type` (String) — Setting type (implementation specific string value). - `changed` (UInt8) — Whether the setting was explicitly defined in the config or explicitly changed. -## システムtable\_engines {#system-table-engines} +## システムtable_engines {#system-table-engines} を含むの記述のテーブルエンジンをサポートサーバーとその特徴を支援す。 @@ -1118,25 +1118,25 @@ path: /clickhouse/tables/01-08/visits/replicas **データ**, **テーブル** -突然変異が適用されたデータベースとテーブルの名前。 -**mutation\_id** -突然変異のID。 レプリケートされたテーブルの場合、これらのIdは `/mutations/` 飼育係のディレクトリ。 未複製テーブルの場合、Idはテーブルのデータディレクトリ内のファイル名に対応します。 +**mutation_id** -突然変異のID。 レプリケートされたテーブルの場合、これらのIdは `/mutations/` 飼育係のディレクトリ。 未複製テーブルの場合、Idはテーブルのデータディレクトリ内のファイル名に対応します。 **コマンド** -突然変異コマンド文字列(後のクエリの部分 `ALTER TABLE [db.]table`). -**create\_time** -この突然変異コマンドが実行のために提出されたとき。 +**create_time** -この突然変異コマンドが実行のために提出されたとき。 -**ブロック番号partition\_id**, **ブロック番号番号** -入れ子になった列。 パーティションIDと、その変異によって取得されたブロック番号(各パーティションでは、そのパーティション内の変異によって取得されたブロック番号 非複製のテーブル、ブロック番号の全ての仕切りがひとつのシーケンスです。 こないということを意味している変異体再現し、テーブルの列として展開しているのが記録するとともにシングルブロック番号の取得による突然変異が原因です。 +**ブロック番号partition_id**, **ブロック番号番号** -入れ子になった列。 パーティションIDと、その変異によって取得されたブロック番号(各パーティションでは、そのパーティション内の変異によって取得されたブロック番号 非複製のテーブル、ブロック番号の全ての仕切りがひとつのシーケンスです。 こないということを意味している変異体再現し、テーブルの列として展開しているのが記録するとともにシングルブロック番号の取得による突然変異が原因です。 -**parts\_to\_do** -突然変異が完了するために突然変異する必要があるデータ部分の数。 +**parts_to_do** -突然変異が完了するために突然変異する必要があるデータ部分の数。 -**is\_done** -突然変異は? たとえ `parts_to_do = 0` 変更する必要がある新しいデータパーツを作成する長時間実行されるINSERTのために、複製されたテーブルの突然変異がまだ行われていない可能性があり +**is_done** -突然変異は? たとえ `parts_to_do = 0` 変更する必要がある新しいデータパーツを作成する長時間実行されるINSERTのために、複製されたテーブルの突然変異がまだ行われていない可能性があり 一部のパーツの変更に問題がある場合は、次の列に追加情報が含まれています: -**latest\_failed\_part** -変異することができなかった最新の部分の名前。 +**latest_failed_part** -変異することができなかった最新の部分の名前。 -**latest\_fail\_time** -最も最近の部分突然変異の失敗の時間。 +**latest_fail_time** -最も最近の部分突然変異の失敗の時間。 -**latest\_fail\_reason** -最新の部品突然変異の失敗を引き起こした例外メッセージ。 +**latest_fail_reason** -最新の部品突然変異の失敗を引き起こした例外メッセージ。 ## システムディスク {#system_tables-disks} diff --git a/docs/ja/operations/tips.md b/docs/ja/operations/tips.md index 630d62730fd..5c64a216e1d 100644 --- a/docs/ja/operations/tips.md +++ b/docs/ja/operations/tips.md @@ -59,7 +59,7 @@ RAID-10を作成するときは、 `far` レイアウト。 予算が許せば、RAID-10を選択します。 4つ以上のディスクがある場合は、RAID-6(優先)またはRAID-50を使用してください。 -RAID-5、RAID-6、またはRAID-50を使用する場合は、常にstripe\_cache\_sizeを増やしてください。 +RAID-5、RAID-6、またはRAID-50を使用する場合は、常にstripe_cache_sizeを増やしてください。 ``` bash $ echo 4096 | sudo tee /sys/block/md2/md/stripe_cache_size diff --git a/docs/ja/operations/troubleshooting.md b/docs/ja/operations/troubleshooting.md index 989e30eb5f2..341613009c6 100644 --- a/docs/ja/operations/troubleshooting.md +++ b/docs/ja/operations/troubleshooting.md @@ -105,7 +105,7 @@ $ sudo -u clickhouse /usr/bin/clickhouse-server --config-file /etc/clickhouse-se - エンドポイント設定。 - チェック [listen\_host](server-configuration-parameters/settings.md#server_configuration_parameters-listen_host) と [tcp\_port](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) 設定。 + チェック [listen_host](server-configuration-parameters/settings.md#server_configuration_parameters-listen_host) と [tcp_port](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) 設定。 ClickHouse serverはデフォルトでのみlocalhost接続を受け入れます。 @@ -117,7 +117,7 @@ $ sudo -u clickhouse /usr/bin/clickhouse-server --config-file /etc/clickhouse-se チェック: - - その [tcp\_port\_secure](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) 設定。 + - その [tcp_port_secure](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) 設定。 - の設定 [SSLセルティクス](server-configuration-parameters/settings.md#server_configuration_parameters-openssl). 適切なパラメータを接続 たとえば、 `port_secure` 変数との `clickhouse_client`. diff --git a/docs/ja/sql-reference/aggregate-functions/parametric-functions.md b/docs/ja/sql-reference/aggregate-functions/parametric-functions.md index 22d9dbe95ea..1df84f2657c 100644 --- a/docs/ja/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/ja/sql-reference/aggregate-functions/parametric-functions.md @@ -494,6 +494,6 @@ Solution: Write in the GROUP BY query SearchPhrase HAVING uniqUpTo(4)(UserID) >= [元の記事](https://clickhouse.tech/docs/en/query_language/agg_functions/parametric_functions/) -## sumMapFiltered(keys\_to\_keep)(キー,値) {#summapfilteredkeys-to-keepkeys-values} +## sumMapFiltered(keys_to_keep)(キー,値) {#summapfilteredkeys-to-keepkeys-values} と同じ動作 [サマップ](reference.md#agg_functions-summap) ただし、キーの配列はパラメータとして渡されます。 これは、キーの基数が高い場合に特に便利です。 diff --git a/docs/ja/sql-reference/aggregate-functions/reference.md b/docs/ja/sql-reference/aggregate-functions/reference.md index 3a4266e7a02..298de5a75b5 100644 --- a/docs/ja/sql-reference/aggregate-functions/reference.md +++ b/docs/ja/sql-reference/aggregate-functions/reference.md @@ -31,7 +31,7 @@ ClickHouseは以下の構文をサポートしています `count`: **詳細** -クリックハウスは `COUNT(DISTINCT ...)` 構文。 この構造の動作は、 [count\_distinct\_implementation](../../operations/settings/settings.md#settings-count_distinct_implementation) 設定。 それはのどれを定義します [uniq\*](#agg_function-uniq) 関数は、操作を実行するために使用されます。 デフォルトは [uniqExact](#agg_function-uniqexact) 機能。 +クリックハウスは `COUNT(DISTINCT ...)` 構文。 この構造の動作は、 [count_distinct_implementation](../../operations/settings/settings.md#settings-count_distinct_implementation) 設定。 それはのどれを定義します [uniq\*](#agg_function-uniq) 関数は、操作を実行するために使用されます。 デフォルトは [uniqExact](#agg_function-uniqexact) 機能。 その `SELECT count() FROM table` テーブル内のエントリの数が別々に格納されないため、クエリは最適化されません。 テーブルから小さな列を選択し、その中の値の数をカウントします。 @@ -721,7 +721,7 @@ uniqExact(x[, ...]) - [uniqCombined](#agg_function-uniqcombined) - [uniqHLL12](#agg_function-uniqhll12) -## groupArray(x),groupArray(max\_size)(x) {#agg_function-grouparray} +## groupArray(x),groupArray(max_size)(x) {#agg_function-grouparray} 引数値の配列を作成します。 値は、任意の(不確定な)順序で配列に追加できます。 @@ -967,7 +967,7 @@ FROM t └───────────┴──────────────────────────────────┴───────────────────────┘ ``` -## groupUniqArray(x),groupUniqArray(max\_size)(x) {#groupuniqarrayx-groupuniqarraymax-sizex} +## groupUniqArray(x),groupUniqArray(max_size)(x) {#groupuniqarrayx-groupuniqarraymax-sizex} 異なる引数値から配列を作成します。 メモリ消費量は `uniqExact` 機能。 diff --git a/docs/ja/sql-reference/ansi.md b/docs/ja/sql-reference/ansi.md index 246036125b2..fbeb4b8b6db 100644 --- a/docs/ja/sql-reference/ansi.md +++ b/docs/ja/sql-reference/ansi.md @@ -36,8 +36,8 @@ toc_title: "ANSI\u306E\u4E92\u63DB\u6027" | E021-01 | 文字データ型 | いいえ。{.text-danger} | | | E021-02 | 文字変化型データ型 | いいえ。{.text-danger} | `String` 動作同様に、長さの制限内 | | E021-03 | 文字リテラル | 部分的{.text-warning} | 連続したリテラルと文字セットの自動連結はサポートされません | -| E021-04 | CHARACTER\_LENGTH関数 | 部分的{.text-warning} | いいえ。 `USING` 句 | -| E021-05 | OCTET\_LENGTH関数 | いいえ。{.text-danger} | `LENGTH` 同様に動作します | +| E021-04 | CHARACTER_LENGTH関数 | 部分的{.text-warning} | いいえ。 `USING` 句 | +| E021-05 | OCTET_LENGTH関数 | いいえ。{.text-danger} | `LENGTH` 同様に動作します | | E021-06 | SUBSTRING | 部分的{.text-warning} | サポートなし `SIMILAR` と `ESCAPE` 句、ない `SUBSTRING_REGEX` バリアント | | E021-07 | 文字の連結 | 部分的{.text-warning} | いいえ。 `COLLATE` 句 | | E021-08 | 上部および下の機能 | はい。{.text-success} | | @@ -144,7 +144,7 @@ toc_title: "ANSI\u306E\u4E92\u63DB\u6027" | F051-03 | タイムスタンプのデータ型を含む支援のタイムスタンプ文字と小数点以下の秒の精度で少なくとも0-6 | いいえ。{.text-danger} | `DateTime64` timeは同様の機能を提供します | | F051-04 | 日付、時刻、およびタイムスタンプのデータ型の比較述語 | 部分的{.text-warning} | 使用可能なデータ型は一つだけです | | F051-05 | Datetime型と文字列型の間の明示的なキャスト | はい。{.text-success} | | -| F051-06 | CURRENT\_DATE | いいえ。{.text-danger} | `today()` 似ています | +| F051-06 | CURRENT_DATE | いいえ。{.text-danger} | `today()` 似ています | | F051-07 | LOCALTIME | いいえ。{.text-danger} | `now()` 似ています | | F051-08 | LOCALTIMESTAMP | いいえ。{.text-danger} | | | **F081** | **ビュー内の組合および除く** | **部分的**{.text-warning} | | diff --git a/docs/ja/sql-reference/data-types/aggregatefunction.md b/docs/ja/sql-reference/data-types/aggregatefunction.md index 7051880ec7b..45aea71a2a9 100644 --- a/docs/ja/sql-reference/data-types/aggregatefunction.md +++ b/docs/ja/sql-reference/data-types/aggregatefunction.md @@ -5,7 +5,7 @@ toc_priority: 52 toc_title: "AggregateFunction(\u540D\u524D,types_of_arguments...)" --- -# AggregateFunction(name, types\_of\_arguments…) {#data-type-aggregatefunction} +# AggregateFunction(name, types_of_arguments…) {#data-type-aggregatefunction} Aggregate functions can have an implementation-defined intermediate state that can be serialized to an AggregateFunction(…) data type and stored in a table, usually, by means of [マテリアライズドビュー](../../sql-reference/statements/create.md#create-view). 集計関数の状態を生成する一般的な方法は、集計関数を呼び出すことです。 `-State` 接尾辞。 将来集計の最終結果を取得するには、同じ集計関数を使用する必要があります。 `-Merge`接尾辞。 diff --git a/docs/ja/sql-reference/data-types/datetime.md b/docs/ja/sql-reference/data-types/datetime.md index c4a74696be1..8c6164881d8 100644 --- a/docs/ja/sql-reference/data-types/datetime.md +++ b/docs/ja/sql-reference/data-types/datetime.md @@ -31,7 +31,7 @@ DateTime([timezone]) ClickHouseは、次の値を出力します `YYYY-MM-DD hh:mm:ss` デフォルトのテキスト形式。 出力を変更するには [formatDateTime](../../sql-reference/functions/date-time-functions.md#formatdatetime) 機能。 -ClickHouseにデータを挿入するときは、データの値に応じて、日付と時刻の文字列の異なる形式を使用できます。 [date\_time\_input\_format](../../operations/settings/settings.md#settings-date_time_input_format) 設定。 +ClickHouseにデータを挿入するときは、データの値に応じて、日付と時刻の文字列の異なる形式を使用できます。 [date_time_input_format](../../operations/settings/settings.md#settings-date_time_input_format) 設定。 ## 例 {#examples} diff --git a/docs/ja/sql-reference/data-types/fixedstring.md b/docs/ja/sql-reference/data-types/fixedstring.md index 4e2e5aae4e9..ef94410e285 100644 --- a/docs/ja/sql-reference/data-types/fixedstring.md +++ b/docs/ja/sql-reference/data-types/fixedstring.md @@ -22,7 +22,7 @@ toc_title: "\u56FA\u5B9A\u6587\u5B57\u5217(N)" 効率的に格納できる値の例 `FixedString`-型指定された列: - IPアドレスのバイナリ表現 (`FixedString(16)` IPv6の場合)。 -- Language codes (ru\_RU, en\_US … ). +- Language codes (ru_RU, en_US … ). - Currency codes (USD, RUB … ). - ハッシュのバイナリ表現 (`FixedString(16)` MD5の場合, `FixedString(32)` SHA256)のため。 diff --git a/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md b/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md index b46e09b0284..1e99b782043 100644 --- a/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md +++ b/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md @@ -27,7 +27,7 @@ ClickHouseは、階層辞書をサポートしています [数値キー](extern この階層は、次の辞書テーブルとして表すことができます。 -| region\_id | parent\_region | region\_name | +| region_id | parent_region | region_name | |------------|----------------|--------------| | 1 | 0 | ロシア | | 2 | 1 | モスクワ | diff --git a/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 4a7d5fb1c72..167e0f1f5d1 100644 --- a/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -9,7 +9,7 @@ toc_title: "\u30E1\u30E2\u30EA\u3078\u306E\u8F9E\u66F8\u306E\u683C\u7D0D" 辞書をメモリに保存するには、さまざまな方法があります。 -私達は推薦します [平ら](#flat), [ハッシュ](#dicts-external_dicts_dict_layout-hashed) と [complex\_key\_hashed](#complex-key-hashed). 最適の処理速度を提供するかどれが。 +私達は推薦します [平ら](#flat), [ハッシュ](#dicts-external_dicts_dict_layout-hashed) と [complex_key_hashed](#complex-key-hashed). 最適の処理速度を提供するかどれが。 キャッシュ推奨されていないものになる可能性のある性能や困難の選定に最適なパラメータ。 セクションの続きを読む “[キャッシュ](#cache)”. @@ -54,13 +54,13 @@ LAYOUT(LAYOUT_TYPE(param value)) -- layout settings - [平ら](#flat) - [ハッシュ](#dicts-external_dicts_dict_layout-hashed) -- [sparse\_hashed](#dicts-external_dicts_dict_layout-sparse_hashed) +- [sparse_hashed](#dicts-external_dicts_dict_layout-sparse_hashed) - [キャッシュ](#cache) - [直接](#direct) -- [range\_hashed](#range-hashed) -- [complex\_key\_hashed](#complex-key-hashed) -- [complex\_key\_cache](#complex-key-cache) -- [ip\_trie](#ip-trie) +- [range_hashed](#range-hashed) +- [complex_key_hashed](#complex-key-hashed) +- [complex_key_cache](#complex-key-cache) +- [ip_trie](#ip-trie) ### 平ら {#flat} @@ -106,7 +106,7 @@ LAYOUT(FLAT()) LAYOUT(HASHED()) ``` -### sparse\_hashed {#dicts-external_dicts_dict_layout-sparse_hashed} +### sparse_hashed {#dicts-external_dicts_dict_layout-sparse_hashed} に類似した `hashed` が、使用メモリ賛以上のCPUます。 @@ -122,7 +122,7 @@ LAYOUT(HASHED()) LAYOUT(SPARSE_HASHED()) ``` -### complex\_key\_hashed {#complex-key-hashed} +### complex_key_hashed {#complex-key-hashed} このタイプの貯蔵は合成物との使用のためです [キー](external-dicts-dict-structure.md). に類似した `hashed`. @@ -138,7 +138,7 @@ LAYOUT(SPARSE_HASHED()) LAYOUT(COMPLEX_KEY_HASHED()) ``` -### range\_hashed {#range-hashed} +### range_hashed {#range-hashed} 辞書は、範囲とそれに対応する値の順序付き配列を持つハッシュテーブルの形式でメモリに格納されます。 @@ -293,7 +293,7 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000)) !!! warning "警告" ClickHouseをソースとして使用しないでください。 -### complex\_key\_cache {#complex-key-cache} +### complex_key_cache {#complex-key-cache} このタイプの貯蔵は合成物との使用のためです [キー](external-dicts-dict-structure.md). に類似した `cache`. @@ -319,7 +319,7 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000)) LAYOUT(DIRECT()) ``` -### ip\_trie {#ip-trie} +### ip_trie {#ip-trie} このタイプの貯蔵するマッピングするネットワーク接頭辞(IPアドレスへのメタデータなどのASN. diff --git a/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts.md index 51b3ecf1900..01e6e57d2ca 100644 --- a/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ b/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts.md @@ -15,9 +15,9 @@ toc_title: "\u4E00\u822C\u7684\u306A\u8AAC\u660E" - 辞書を定期的に更新し、欠損値を動的に読み込みます。 つまり、辞書は動的に読み込むことができます。 - Xmlファイルで外部辞書を作成することができます。 [DDLクエリ](../../statements/create.md#create-dictionary-query). -外部辞書の構成は、一つ以上のxmlファイルに配置できます。 設定へのパスは [dictionaries\_config](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config) パラメータ。 +外部辞書の構成は、一つ以上のxmlファイルに配置できます。 設定へのパスは [dictionaries_config](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config) パラメータ。 -辞書は、サーバーの起動時または最初の使用時に読み込むことができます。 [dictionaries\_lazy\_load](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load) 設定。 +辞書は、サーバーの起動時または最初の使用時に読み込むことができます。 [dictionaries_lazy_load](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load) 設定。 その [辞書](../../../operations/system-tables.md#system_tables-dictionaries) システムテーブルについての情報が含まれて辞書に設定されます。 各辞書については、そこにあります: diff --git a/docs/ja/sql-reference/functions/array-functions.md b/docs/ja/sql-reference/functions/array-functions.md index bd30262cc1e..6eab7656c25 100644 --- a/docs/ja/sql-reference/functions/array-functions.md +++ b/docs/ja/sql-reference/functions/array-functions.md @@ -1028,7 +1028,7 @@ SELECT arrayZip(['a', 'b', 'c'], [5, 2, 1]) ## アレイオーク {#arrayauc} -計算AUC(機械学習の概念である曲線の下の面積、詳細を参照してください:https://en.wikipedia.org/wiki/Receiver\_operating\_characteristic\#Area\_under\_the\_curve)。 +計算AUC(機械学習の概念である曲線の下の面積、詳細を参照してください:https://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve)。 **構文** diff --git a/docs/ja/sql-reference/functions/bitmap-functions.md b/docs/ja/sql-reference/functions/bitmap-functions.md index 70fee866961..cc57e762610 100644 --- a/docs/ja/sql-reference/functions/bitmap-functions.md +++ b/docs/ja/sql-reference/functions/bitmap-functions.md @@ -65,7 +65,7 @@ SELECT bitmapToArray(bitmapBuild([1, 2, 3, 4, 5])) AS res ## bitmapSubsetInRange {#bitmap-functions-bitmapsubsetinrange} -指定された範囲のサブセットを返します(range\_endは含まれません)。 +指定された範囲のサブセットを返します(range_endは含まれません)。 ``` sql bitmapSubsetInRange(bitmap, range_start, range_end) @@ -241,7 +241,7 @@ SELECT bitmapCardinality(bitmapBuild([1, 2, 3, 4, 5])) AS res ## bitmapMin {#bitmapmin} -セット内のUInt64型の最小値を再実行し、セットが空の場合はUINT32\_MAX。 +セット内のUInt64型の最小値を再実行し、セットが空の場合はUINT32_MAX。 bitmapMin(bitmap) @@ -288,8 +288,8 @@ SELECT bitmapMax(bitmapBuild([1, 2, 3, 4, 5])) AS res **パラメータ** - `bitmap` – bitmap object. -- `from_array` – UInt32 array. For idx in range \[0, from\_array.size()), if bitmap contains from\_array\[idx\], then replace it with to\_array\[idx\]. Note that the result depends on array ordering if there are common elements between from\_array and to\_array. -- `to_array` – UInt32 array, its size shall be the same to from\_array. +- `from_array` – UInt32 array. For idx in range \[0, from_array.size()), if bitmap contains from_array\[idx\], then replace it with to_array\[idx\]. Note that the result depends on array ordering if there are common elements between from_array and to_array. +- `to_array` – UInt32 array, its size shall be the same to from_array. **例** diff --git a/docs/ja/sql-reference/functions/date-time-functions.md b/docs/ja/sql-reference/functions/date-time-functions.md index 88dd72dc7fc..a0d51439339 100644 --- a/docs/ja/sql-reference/functions/date-time-functions.md +++ b/docs/ja/sql-reference/functions/date-time-functions.md @@ -163,7 +163,7 @@ Mode引数は、toWeek()のmode引数とまったく同じように動作しま 日付を時刻とともに切り捨てて、十分間隔の開始まで切り捨てます。 -## トスタートオフインターバル(time\_or\_data,区間x単位\[,time\_zone\]) {#tostartofintervaltime-or-data-interval-x-unit-time-zone} +## トスタートオフインターバル(time_or_data,区間x単位\[,time_zone\]) {#tostartofintervaltime-or-data-interval-x-unit-time-zone} これは、名前付きの他の関数の一般化です `toStartOf*`. 例えば, `toStartOfInterval(t, INTERVAL 1 year)` と同じを返します `toStartOfYear(t)`, diff --git a/docs/ja/sql-reference/functions/geo.md b/docs/ja/sql-reference/functions/geo.md index f6b2ebd3b62..4237e43b8dc 100644 --- a/docs/ja/sql-reference/functions/geo.md +++ b/docs/ja/sql-reference/functions/geo.md @@ -209,10 +209,10 @@ SELECT geoToH3(37.79506683, 55.71290588, 15) as h3Index **入力値** -- longitudie\_min-最小の経度、範囲内の浮動小数点値 `[-180°, 180°]` -- latitude\_min-最小緯度、範囲内の浮動小数点値 `[-90°, 90°]` +- longitudie_min-最小の経度、範囲内の浮動小数点値 `[-180°, 180°]` +- latitude_min-最小緯度、範囲内の浮動小数点値 `[-90°, 90°]` - 縦方向max-最大経度、範囲内の浮動小数点値 `[-180°, 180°]` -- latitude\_max-最大緯度、範囲内の浮動小数点値 `[-90°, 90°]` +- latitude_max-最大緯度、範囲内の浮動小数点値 `[-90°, 90°]` - 精度-ジオハッシュ精度, `UInt8` 範囲内 `[1, 12]` すべての座標パラメータは同じタイプでなければなりません。 `Float32` または `Float64`. diff --git a/docs/ja/sql-reference/functions/hash-functions.md b/docs/ja/sql-reference/functions/hash-functions.md index 6710f30e196..d48e6846bb4 100644 --- a/docs/ja/sql-reference/functions/hash-functions.md +++ b/docs/ja/sql-reference/functions/hash-functions.md @@ -317,7 +317,7 @@ JumpConsistentHashを計算すると、UInt64を形成します。 UInt64型のキーとバケットの数です。 Int32を返します。 詳細は、リンクを参照してください: [JumpConsistentHash](https://arxiv.org/pdf/1406.2294.pdf) -## murmurHash2\_32,murmurHash2\_64 {#murmurhash2-32-murmurhash2-64} +## murmurHash2_32,murmurHash2_64 {#murmurhash2-32-murmurhash2-64} を生成する。 [つぶやき2](https://github.com/aappleby/smhasher) ハッシュ値。 @@ -385,7 +385,7 @@ SELECT └──────────────────────┴─────────────────────┘ ``` -## murmurHash3\_32,murmurHash3\_64 {#murmurhash3-32-murmurhash3-64} +## murmurHash3_32,murmurHash3_64 {#murmurhash3-32-murmurhash3-64} を生成する。 [マムルハシュ3世](https://github.com/aappleby/smhasher) ハッシュ値。 @@ -415,7 +415,7 @@ SELECT murmurHash3_32(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23: └─────────────┴────────┘ ``` -## つぶやき3\_128 {#murmurhash3-128} +## つぶやき3_128 {#murmurhash3-128} 128ビットを生成する [マムルハシュ3世](https://github.com/aappleby/smhasher) ハッシュ値。 diff --git a/docs/ja/sql-reference/functions/introspection.md b/docs/ja/sql-reference/functions/introspection.md index 4aef649b3cd..8b17ac5ad7c 100644 --- a/docs/ja/sql-reference/functions/introspection.md +++ b/docs/ja/sql-reference/functions/introspection.md @@ -16,11 +16,11 @@ toc_title: "\u5185\u7701" - インストール `clickhouse-common-static-dbg` パッケージ。 -- セット [allow\_introspection\_functions](../../operations/settings/settings.md#settings-allow_introspection_functions) 1に設定します。 +- セット [allow_introspection_functions](../../operations/settings/settings.md#settings-allow_introspection_functions) 1に設定します。 For security reasons introspection functions are disabled by default. -ClickHouseはプロファイラレポートを [trace\_log](../../operations/system-tables.md#system_tables-trace_log) システムテーブル。 のテーブルプロファイラで設定されます。 +ClickHouseはプロファイラレポートを [trace_log](../../operations/system-tables.md#system_tables-trace_log) システムテーブル。 のテーブルプロファイラで設定されます。 ## アドレストリン {#addresstoline} diff --git a/docs/ja/sql-reference/functions/json-functions.md b/docs/ja/sql-reference/functions/json-functions.md index 921efc675eb..7ceb26c9c9d 100644 --- a/docs/ja/sql-reference/functions/json-functions.md +++ b/docs/ja/sql-reference/functions/json-functions.md @@ -75,7 +75,7 @@ SELECT isValidJSON('{"a": "hello", "b": [-100, 200.0, 300]}') = 1 SELECT isValidJSON('not a json') = 0 ``` -## JSONHas(json\[, indices\_or\_keys\]…) {#jsonhasjson-indices-or-keys} +## JSONHas(json\[, indices_or_keys\]…) {#jsonhasjson-indices-or-keys} 値がJSONドキュメントに存在する場合, `1` 返されます。 @@ -108,7 +108,7 @@ SELECT JSONExtractKey('{"a": "hello", "b": [-100, 200.0, 300]}', -2) = 'a' SELECT JSONExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 1) = 'hello' ``` -## JSONLength(json\[, indices\_or\_keys\]…) {#jsonlengthjson-indices-or-keys} +## JSONLength(json\[, indices_or_keys\]…) {#jsonlengthjson-indices-or-keys} JSON配列またはJSONオブジェクトの長さを返します。 @@ -121,7 +121,7 @@ SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 3 SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}') = 2 ``` -## JSONType(json\[, indices\_or\_keys\]…) {#jsontypejson-indices-or-keys} +## JSONType(json\[, indices_or_keys\]…) {#jsontypejson-indices-or-keys} JSON値の型を返します。 @@ -135,13 +135,13 @@ SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'a') = 'String' SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 'Array' ``` -## JSONExtractUInt(json\[, indices\_or\_keys\]…) {#jsonextractuintjson-indices-or-keys} +## JSONExtractUInt(json\[, indices_or_keys\]…) {#jsonextractuintjson-indices-or-keys} -## JSONExtractInt(json\[, indices\_or\_keys\]…) {#jsonextractintjson-indices-or-keys} +## JSONExtractInt(json\[, indices_or_keys\]…) {#jsonextractintjson-indices-or-keys} -## JSONExtractFloat(json\[, indices\_or\_keys\]…) {#jsonextractfloatjson-indices-or-keys} +## JSONExtractFloat(json\[, indices_or_keys\]…) {#jsonextractfloatjson-indices-or-keys} -## JSONExtractBool(json\[, indices\_or\_keys\]…) {#jsonextractbooljson-indices-or-keys} +## JSONExtractBool(json\[, indices_or_keys\]…) {#jsonextractbooljson-indices-or-keys} JSONを解析して値を抽出します。 これらの機能と類似 `visitParam` 機能。 @@ -155,7 +155,7 @@ SELECT JSONExtractFloat('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 2) = 200 SELECT JSONExtractUInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', -1) = 300 ``` -## JSONExtractString(json\[, indices\_or\_keys\]…) {#jsonextractstringjson-indices-or-keys} +## JSONExtractString(json\[, indices_or_keys\]…) {#jsonextractstringjson-indices-or-keys} JSONを解析して文字列を抽出します。 この関数は次のようになります `visitParamExtractString` 機能。 @@ -173,7 +173,7 @@ SELECT JSONExtractString('{"abc":"\\u263"}', 'abc') = '' SELECT JSONExtractString('{"abc":"hello}', 'abc') = '' ``` -## JSONExtract(json\[, indices\_or\_keys…\], Return\_type) {#jsonextractjson-indices-or-keys-return-type} +## JSONExtract(json\[, indices_or_keys…\], Return_type) {#jsonextractjson-indices-or-keys-return-type} JSONを解析し、指定されたClickHouseデータ型の値を抽出します。 @@ -194,7 +194,7 @@ SELECT JSONExtract('{"day": "Thursday"}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday SELECT JSONExtract('{"day": 5}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \'Tuesday\' = 2, \'Wednesday\' = 3, \'Thursday\' = 4, \'Friday\' = 5, \'Saturday\' = 6)') = 'Friday' ``` -## JSONExtractKeysAndValues(json\[, indices\_or\_keys…\], Value\_type) {#jsonextractkeysandvaluesjson-indices-or-keys-value-type} +## JSONExtractKeysAndValues(json\[, indices_or_keys…\], Value_type) {#jsonextractkeysandvaluesjson-indices-or-keys-value-type} 値が指定されたClickHouseデータ型であるJSONからキーと値のペアを解析します。 @@ -204,7 +204,7 @@ SELECT JSONExtract('{"day": 5}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \ SELECT JSONExtractKeysAndValues('{"x": {"a": 5, "b": 7, "c": 11}}', 'x', 'Int8') = [('a',5),('b',7),('c',11)] ``` -## JSONExtractRaw(json\[, indices\_or\_keys\]…) {#jsonextractrawjson-indices-or-keys} +## JSONExtractRaw(json\[, indices_or_keys\]…) {#jsonextractrawjson-indices-or-keys} JSONの一部を解析されていない文字列として返します。 @@ -216,7 +216,7 @@ JSONの一部を解析されていない文字列として返します。 SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = '[-100, 200.0, 300]' ``` -## JSONExtractArrayRaw(json\[, indices\_or\_keys…\]) {#jsonextractarrayrawjson-indices-or-keys} +## JSONExtractArrayRaw(json\[, indices_or_keys…\]) {#jsonextractarrayrawjson-indices-or-keys} JSON配列の要素を持つ配列を返します。 diff --git a/docs/ja/sql-reference/functions/other-functions.md b/docs/ja/sql-reference/functions/other-functions.md index 31d799f35a4..c590612aeeb 100644 --- a/docs/ja/sql-reference/functions/other-functions.md +++ b/docs/ja/sql-reference/functions/other-functions.md @@ -419,7 +419,7 @@ ORDER BY h ASC いくつかの要素と他の要素との明示的に定義されたマッピングに従って値を変換します。 あーならではのバリエーション機能: -### transform(x,array\_from,array\_to,デフォルト) {#transformx-array-from-array-to-default} +### transform(x,array_from,array_to,デフォルト) {#transformx-array-from-array-to-default} `x` – What to transform. @@ -439,7 +439,7 @@ ORDER BY h ASC 同じ文字が示されている場合(TまたはU)、数値型の場合、これらは一致する型ではなく、共通の型を持つ型である可能性があります。 たとえば、最初の引数はInt64型で、二番目の引数は配列(UInt16)型です。 -もし ‘x’ 値は、要素のいずれかに等しいです。 ‘array\_from’ これは、配列から既存の要素(つまり、同じ番号が付けられている)を返します。 ‘array\_to’ 配列 それ以外の場合は、 ‘default’. 複数の一致する要素がある場合 ‘array\_from’,一致するもののいずれかを返します。 +もし ‘x’ 値は、要素のいずれかに等しいです。 ‘array_from’ これは、配列から既存の要素(つまり、同じ番号が付けられている)を返します。 ‘array_to’ 配列 それ以外の場合は、 ‘default’. 複数の一致する要素がある場合 ‘array_from’,一致するもののいずれかを返します。 例: @@ -461,10 +461,10 @@ ORDER BY c DESC └───────────┴────────┘ ``` -### 変換(x,array\_from,array\_to) {#transformx-array-from-array-to} +### 変換(x,array_from,array_to) {#transformx-array-from-array-to} 最初のバリエーションとは異なり、 ‘default’ 引数は省略される。 -もし ‘x’ 値は、要素のいずれかに等しいです。 ‘array\_from’ これは、配列から一致する要素(つまり、同じ番号が付けられている)を返します。 ‘array\_to’ 配列 それ以外の場合は、 ‘x’. +もし ‘x’ 値は、要素のいずれかに等しいです。 ‘array_from’ これは、配列から一致する要素(つまり、同じ番号が付けられている)を返します。 ‘array_to’ 配列 それ以外の場合は、 ‘x’. タイプ: @@ -1082,7 +1082,7 @@ joinGet(join_storage_table_name, `value_column`, join_keys) キーのリストに対応する値のリストを返します。 -ソーステーブルに特定のものが存在しない場合、 `0` または `null` に基づいて返されます [join\_use\_nulls](../../operations/settings/settings.md#join_use_nulls) 設定。 +ソーステーブルに特定のものが存在しない場合、 `0` または `null` に基づいて返されます [join_use_nulls](../../operations/settings/settings.md#join_use_nulls) 設定。 詳細について `join_use_nulls` で [結合操作](../../engines/table-engines/special/join.md). @@ -1121,15 +1121,15 @@ SELECT joinGet(db_test.id_val,'val',toUInt32(number)) from numbers(4) SETTINGS j └──────────────────────────────────────────────────┘ ``` -## modelEvaluate(model\_name, …) {#function-modelevaluate} +## modelEvaluate(model_name, …) {#function-modelevaluate} 外部モデルの評価 モデル名とモデル引数を受け取ります。 Float64を返します。 -## throwIf(x\[,custom\_message\]) {#throwifx-custom-message} +## throwIf(x\[,custom_message\]) {#throwifx-custom-message} 引数がゼロ以外の場合は例外をスローします。 -custom\_message-オプションのパラメータです。 +custom_message-オプションのパラメータです。 ``` sql SELECT throwIf(number = 3, 'Too many') FROM numbers(10); diff --git a/docs/ja/sql-reference/functions/string-functions.md b/docs/ja/sql-reference/functions/string-functions.md index 87cdb186e52..2c436cd2419 100644 --- a/docs/ja/sql-reference/functions/string-functions.md +++ b/docs/ja/sql-reference/functions/string-functions.md @@ -31,12 +31,12 @@ toc_title: "\u6587\u5B57\u5217\u306E\u64CD\u4F5C" 文字列にUTF-8でエンコードされたテキストを構成するバイトのセットが含まれていると仮定して、Unicodeコードポイント(文字ではない)で文字列の長さを返 この仮定が満たされない場合、結果が返されます(例外はスローされません)。 結果の型はUInt64です。 -## char\_length,CHAR\_LENGTH {#char-length} +## char_length,CHAR_LENGTH {#char-length} 文字列にUTF-8でエンコードされたテキストを構成するバイトのセットが含まれていると仮定して、Unicodeコードポイント(文字ではない)で文字列の長さを返 この仮定が満たされない場合、結果が返されます(例外はスローされません)。 結果の型はUInt64です。 -## character\_length,CHARACTER\_LENGTH {#character-length} +## character_length,CHARACTER_LENGTH {#character-length} 文字列にUTF-8でエンコードされたテキストを構成するバイトのセットが含まれていると仮定して、Unicodeコードポイント(文字ではない)で文字列の長さを返 この仮定が満たされない場合、結果が返されます(例外はスローされません)。 結果の型はUInt64です。 @@ -77,7 +77,7 @@ toValidUTF8( input_string ) パラメータ: -- input\_string — Any set of bytes represented as the [文字列](../../sql-reference/data-types/string.md) データ型オブジェクト。 +- input_string — Any set of bytes represented as the [文字列](../../sql-reference/data-types/string.md) データ型オブジェクト。 戻り値:有効なUTF-8文字列。 diff --git a/docs/ja/sql-reference/functions/type-conversion-functions.md b/docs/ja/sql-reference/functions/type-conversion-functions.md index 447e36a58b5..7058407d5b5 100644 --- a/docs/ja/sql-reference/functions/type-conversion-functions.md +++ b/docs/ja/sql-reference/functions/type-conversion-functions.md @@ -246,7 +246,7 @@ YYYY-MM-DD YYYY-MM-DD hh:mm:ss ``` -例外として、uint32、Int32、UInt64、またはInt64の数値型からDateに変換する場合、数値が65536以上の場合、数値はUnixタイムスタンプとして解釈され(日数ではなく)、日付 この支援のための共通の発生を書く ‘toDate(unix\_timestamp)’ それ以外の場合はエラーになり、より面倒な書き込みが必要になります ‘toDate(toDateTime(unix\_timestamp))’. +例外として、uint32、Int32、UInt64、またはInt64の数値型からDateに変換する場合、数値が65536以上の場合、数値はUnixタイムスタンプとして解釈され(日数ではなく)、日付 この支援のための共通の発生を書く ‘toDate(unix_timestamp)’ それ以外の場合はエラーになり、より面倒な書き込みが必要になります ‘toDate(toDateTime(unix_timestamp))’. 日付と時刻の間の変換は、null時間を追加するか、時間を削除することによって、自然な方法で実行されます。 diff --git a/docs/ja/sql-reference/functions/url-functions.md b/docs/ja/sql-reference/functions/url-functions.md index e263a95d56f..22e8dbd028d 100644 --- a/docs/ja/sql-reference/functions/url-functions.md +++ b/docs/ja/sql-reference/functions/url-functions.md @@ -125,11 +125,11 @@ SELECT topLevelDomain('svn+ssh://www.some.svn-hosting.com:80/repo/trunk') ### パスフル {#pathfull} -上記と同じですが、クエリ文字列とフラグメントを含みます。 例:/top/news.html?ページ=2\#コメント +上記と同じですが、クエリ文字列とフラグメントを含みます。 例:/top/news.html?ページ=2#コメント ### クエリ文字列 {#querystring} -クエリ文字列を返します。 例:ページ=1&lr=213. query-stringには、最初の疑問符だけでなく、\#と\#の後のすべても含まれていません。 +クエリ文字列を返します。 例:ページ=1&lr=213. query-stringには、最初の疑問符だけでなく、#と#の後のすべても含まれていません。 ### 断片 {#fragment} @@ -137,7 +137,7 @@ SELECT topLevelDomain('svn+ssh://www.some.svn-hosting.com:80/repo/trunk') ### queryStringAndFragment {#querystringandfragment} -クエリ文字列とフラグメント識別子を返します。 例:ページ=1\#29390. +クエリ文字列とフラグメント識別子を返します。 例:ページ=1#29390. ### extractURLParameter(URL,名前) {#extracturlparameterurl-name} diff --git a/docs/ja/sql-reference/functions/ym-dict-functions.md b/docs/ja/sql-reference/functions/ym-dict-functions.md index ce9ec860203..53721e07103 100644 --- a/docs/ja/sql-reference/functions/ym-dict-functions.md +++ b/docs/ja/sql-reference/functions/ym-dict-functions.md @@ -17,12 +17,12 @@ ClickHouseは、特定の地域が属する国のさまざまな視点をサポ その ‘clickhouse-server’ configは、地域階層を持つファイルを指定します::`/opt/geo/regions_hierarchy.txt` -このファイル以外にも、名前に\_記号と接尾辞が付加された近くのファイルも検索します(ファイル拡張子の前に)。 +このファイル以外にも、名前に_記号と接尾辞が付加された近くのファイルも検索します(ファイル拡張子の前に)。 たとえば、次のファイルも検索します `/opt/geo/regions_hierarchy_ua.txt`、存在する場合。 `ua` 辞書キーと呼ばれます。 接尾辞のない辞書の場合、キーは空の文字列です。 -すべての辞書は実行時に再ロードされます(builtin\_dictionaries\_reload\_interval設定パラメータで定義されているように、一定の秒数ごとに、またはデフォルトでは時間に一度)。 ただし、使用可能な辞書のリストは、サーバーの起動時に一度に定義されます。 +すべての辞書は実行時に再ロードされます(builtin_dictionaries_reload_interval設定パラメータで定義されているように、一定の秒数ごとに、またはデフォルトでは時間に一度)。 ただし、使用可能な辞書のリストは、サーバーの起動時に一度に定義されます。 All functions for working with regions have an optional argument at the end – the dictionary key. It is referred to as the geobase. 例: @@ -107,7 +107,7 @@ LIMIT 15 地域を大陸に変換します。 他のすべての方法では、この関数は次のように同じです ‘regionToCity’. 例: `regionToContinent(toUInt32(213)) = 10001` モスクワ(213)をユーラシア(10001)に変換します。 -### regionToTopContinent(\#regiontotopcontinent) {#regiontotopcontinent-regiontotopcontinent} +### regionToTopContinent(#regiontotopcontinent) {#regiontotopcontinent-regiontotopcontinent} リージョンの階層内で最も高い大陸を検索します。 diff --git a/docs/ja/sql-reference/statements/alter.md b/docs/ja/sql-reference/statements/alter.md index 057cbebae7f..226565dd226 100644 --- a/docs/ja/sql-reference/statements/alter.md +++ b/docs/ja/sql-reference/statements/alter.md @@ -208,7 +208,7 @@ ALTER TABLE [db].name DROP CONSTRAINT constraint_name; - [ATTACH PART\|PARTITION](#alter_attach-partition) – Adds a part or partition from the `detached` テーブルへのディレクトリ。 - [ATTACH PARTITION FROM](#alter_attach-partition-from) – Copies the data partition from one table to another and adds. - [REPLACE PARTITION](#alter_replace-partition) -コピーするデータを仕切りからテーブルにも置き換え. -- [MOVE PARTITION TO TABLE](#alter_move_to_table-partition)(\#alter\_move\_to\_table-partition)-データ-パーティションをあるテーブルから別のテーブルに移動します。 +- [MOVE PARTITION TO TABLE](#alter_move_to_table-partition)(#alter_move_to_table-partition)-データ-パーティションをあるテーブルから別のテーブルに移動します。 - [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) -パーティション内の指定された列の値をリセットします。 - [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) -リセットの指定された二次インデックス、パーティション - [FREEZE PARTITION](#alter_freeze-partition) – Creates a backup of a partition. @@ -436,7 +436,7 @@ ALTER TABLE hits MOVE PARTITION '2019-09-01' TO DISK 'fast_ssd' - からの値として `partition` の列 `system.parts` テーブル。 例えば, `ALTER TABLE visits DETACH PARTITION 201901`. - テーブル列からの式として。 定数と定数式がサポートされています。 例えば, `ALTER TABLE visits DETACH PARTITION toYYYYMM(toDate('2019-01-25'))`. - パーティションIDの使用。 Partition IDは、ファイルシステムおよびZooKeeperのパーティションの名前として使用されるパーティションの文字列識別子(可能であれば人間が読める)です。 パーティションIDは、 `PARTITION ID` 一重引quotesで囲まれた句。 例えば, `ALTER TABLE visits DETACH PARTITION ID '201901'`. -- で [ALTER ATTACH PART](#alter_attach-partition) と [DROP DETACHED PART](#alter_drop-detached) パーツの名前を指定するには、文字列リテラルを使用します。 `name` の列 [システムdetached\_parts](../../operations/system-tables.md#system_tables-detached_parts) テーブル。 例えば, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. +- で [ALTER ATTACH PART](#alter_attach-partition) と [DROP DETACHED PART](#alter_drop-detached) パーツの名前を指定するには、文字列リテラルを使用します。 `name` の列 [システムdetached_parts](../../operations/system-tables.md#system_tables-detached_parts) テーブル。 例えば, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. ご利用の引用符を指定する場合、パーティションのエントランスは目を引く壁面緑化を表現。 例えば、 `String` その名前を引用符で指定する必要があります (`'`). のために `Date` と `Int*` 型引用符は必要ありません。 diff --git a/docs/ja/sql-reference/statements/insert-into.md b/docs/ja/sql-reference/statements/insert-into.md index 3a646cc214f..2709137a04e 100644 --- a/docs/ja/sql-reference/statements/insert-into.md +++ b/docs/ja/sql-reference/statements/insert-into.md @@ -20,7 +20,7 @@ INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), . - から計算される値 `DEFAULT` テーブル定義で指定された式。 - ゼロと空の文字列 `DEFAULT` 式は定義されていません。 -もし [strict\_insert\_defaults=1](../../operations/settings/settings.md),を持たない列 `DEFAULT` 定義され記載されていることを返します。 +もし [strict_insert_defaults=1](../../operations/settings/settings.md),を持たない列 `DEFAULT` 定義され記載されていることを返します。 データは、任意の場所でINSERTに渡すことができます [形式](../../interfaces/formats.md#formats) ClickHouseがサポートしています。 この形式は、クエリで明示的に指定する必要があります: diff --git a/docs/ja/sql-reference/statements/misc.md b/docs/ja/sql-reference/statements/misc.md index 8925ed645b3..d4f4b245c9a 100644 --- a/docs/ja/sql-reference/statements/misc.md +++ b/docs/ja/sql-reference/statements/misc.md @@ -57,7 +57,7 @@ CHECK TABLE [db.]name テーブルが破損している場合は、破損していないデータを別のテーブルにコピーできます。 これを行うには: 1. 破損したテーブルと同じ構造を持つ新しいテーブルを作成します。 これを行うにはクエリを実行します `CREATE TABLE AS `. -2. セット [max\_threads](../../operations/settings/settings.md#settings-max_threads) 単一のスレッドで次のクエリを処理するには、値を1に設定します。 このクエリ `SET max_threads = 1`. +2. セット [max_threads](../../operations/settings/settings.md#settings-max_threads) 単一のスレッドで次のクエリを処理するには、値を1に設定します。 このクエリ `SET max_threads = 1`. 3. クエリの実行 `INSERT INTO SELECT * FROM `. この要求により、破損していないデータが破損した表から別の表にコピーされます。 破損した部分の前のデータのみがコピーされます。 4. 再起動 `clickhouse-client` リセットするには `max_threads` 値。 @@ -253,7 +253,7 @@ OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition | PARTITION I とき `OPTIMIZE` と共に使用されます [複製マージツリー](../../engines/table-engines/mergetree-family/replication.md) テーブルエンジンのファミリでは、ClickHouseはマージ用のタスクを作成し、すべてのノードで実行を待機します。 `replication_alter_partitions_sync` 設定が有効になっています)。 -- もし `OPTIMIZE` 何らかの理由でマージを実行せず、クライアントに通知しません。 通知を有効にするには、 [optimize\_throw\_if\_noop](../../operations/settings/settings.md#setting-optimize_throw_if_noop) 設定。 +- もし `OPTIMIZE` 何らかの理由でマージを実行せず、クライアントに通知しません。 通知を有効にするには、 [optimize_throw_if_noop](../../operations/settings/settings.md#setting-optimize_throw_if_noop) 設定。 - を指定した場合 `PARTITION` 指定したパーティションのみが最適化されます。 [パーティション式の設定方法](alter.md#alter-how-to-specify-part-expr). - 指定した場合 `FINAL`、最適化は、すべてのデータが一つの部分に既にある場合でも実行されます。 - 指定した場合 `DEDUPLICATE` その後、完全に同一の行が重複除外されます(すべての列が比較されます)。 diff --git a/docs/ja/sql-reference/statements/optimize.md b/docs/ja/sql-reference/statements/optimize.md index 9781f51049e..563839668f9 100644 --- a/docs/ja/sql-reference/statements/optimize.md +++ b/docs/ja/sql-reference/statements/optimize.md @@ -15,7 +15,7 @@ The `OPTMIZE` query is also supported for the [MaterializedView](../../engines/t When `OPTIMIZE` is used with the [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/replication.md) family of table engines, ClickHouse creates a task for merging and waits for execution on all nodes (if the `replication_alter_partitions_sync` setting is enabled). -- If `OPTIMIZE` doesn’t perform a merge for any reason, it doesn’t notify the client. To enable notifications, use the [optimize\_throw\_if\_noop](../../operations/settings/settings.md#setting-optimize_throw_if_noop) setting. +- If `OPTIMIZE` doesn’t perform a merge for any reason, it doesn’t notify the client. To enable notifications, use the [optimize_throw_if_noop](../../operations/settings/settings.md#setting-optimize_throw_if_noop) setting. - If you specify a `PARTITION`, only the specified partition is optimized. [How to set partition expression](../../sql-reference/statements/alter.md#alter-how-to-specify-part-expr). - If you specify `FINAL`, optimization is performed even when all the data is already in one part. - If you specify `DEDUPLICATE`, then completely identical rows will be deduplicated (all columns are compared), it makes sense only for the MergeTree engine. diff --git a/docs/ja/sql-reference/statements/system.md b/docs/ja/sql-reference/statements/system.md index 4f39fd2ac6e..c15c3a79cd5 100644 --- a/docs/ja/sql-reference/statements/system.md +++ b/docs/ja/sql-reference/statements/system.md @@ -24,12 +24,12 @@ toc_title: SYSTEM ## RELOAD DICTIONARIES {#query_language-system-reload-dictionaries} 以前に正常に読み込まれたすべての辞書を再読み込みします。 -デフォルトでは、辞書は遅延して読み込まれます [dictionaries\_lazy\_load](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load))したがって、起動時に自動的にロードされるのではなく、dictGet関数による最初のアクセス時に初期化されるか、ENGINE=Dictionaryを使用してテーブルから選択されます。 その `SYSTEM RELOAD DICTIONARIES` クエバなどの辞書(ロード). +デフォルトでは、辞書は遅延して読み込まれます [dictionaries_lazy_load](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load))したがって、起動時に自動的にロードされるのではなく、dictGet関数による最初のアクセス時に初期化されるか、ENGINE=Dictionaryを使用してテーブルから選択されます。 その `SYSTEM RELOAD DICTIONARIES` クエバなどの辞書(ロード). 常に戻ります `Ok.` 辞書の更新の結果に関係なく。 -## 辞書Dictionary\_nameを再読み込み {#query_language-system-reload-dictionary} +## 辞書Dictionary_nameを再読み込み {#query_language-system-reload-dictionary} -辞書を完全に再読み込みします `dictionary_name` ディクショナリの状態に関係なく(LOADED/NOT\_LOADED/FAILED)。 +辞書を完全に再読み込みします `dictionary_name` ディクショナリの状態に関係なく(LOADED/NOT_LOADED/FAILED)。 常に戻ります `Ok.` 辞書の更新の結果に関係なく。 ディクショナリのステータスは、 `system.dictionaries` テーブル。 @@ -41,7 +41,7 @@ SELECT name, status FROM system.dictionaries; ClickHouseの内部DNSキャッシュをリセットします。 場合によっては(古いClickHouseバージョンの場合)、インフラストラクチャを変更するとき(別のClickHouseサーバーまたは辞書で使用されているサーバーのIPアドレスを変更す -より便利な(自動)キャッシュ管理については、"disable\_internal\_dns\_cache,dns\_cache\_update\_periodパラメーター"を参照してください。 +より便利な(自動)キャッシュ管理については、"disable_internal_dns_cache,dns_cache_update_periodパラメーター"を参照してください。 ## DROP MARK CACHE {#query_language-system-drop-mark-cache} @@ -49,7 +49,7 @@ ClickHouseの内部DNSキャッシュをリセットします。 場合によっ ## FLUSH LOGS {#query_language-system-flush_logs} -Flushes buffers of log messages to system tables (e.g. system.query\_log). Allows you to not wait 7.5 seconds when debugging. +Flushes buffers of log messages to system tables (e.g. system.query_log). Allows you to not wait 7.5 seconds when debugging. ## RELOAD CONFIG {#query_language-system-reload-config} diff --git a/docs/ja/sql-reference/syntax.md b/docs/ja/sql-reference/syntax.md index 9cd128b001c..5d15f408b5d 100644 --- a/docs/ja/sql-reference/syntax.md +++ b/docs/ja/sql-reference/syntax.md @@ -15,9 +15,9 @@ toc_title: "\u69CB\u6587" INSERT INTO t VALUES (1, 'Hello, world'), (2, 'abc'), (3, 'def') ``` -その `INSERT INTO t VALUES` フラグメントは完全なパーサーによって解析され、データは `(1, 'Hello, world'), (2, 'abc'), (3, 'def')` 高速ストリームパーサーによって解析されます。 また、データの完全なパーサーをオンにするには [input\_format\_values\_interpret\_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions) 設定。 とき `input_format_values_interpret_expressions = 1`,ClickHouseはまず、高速ストリームパーサーで値を解析しようとします。 失敗した場合、ClickHouseはデータに対して完全なパーサーを使用し、SQLのように扱います [式](#syntax-expressions). +その `INSERT INTO t VALUES` フラグメントは完全なパーサーによって解析され、データは `(1, 'Hello, world'), (2, 'abc'), (3, 'def')` 高速ストリームパーサーによって解析されます。 また、データの完全なパーサーをオンにするには [input_format_values_interpret_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions) 設定。 とき `input_format_values_interpret_expressions = 1`,ClickHouseはまず、高速ストリームパーサーで値を解析しようとします。 失敗した場合、ClickHouseはデータに対して完全なパーサーを使用し、SQLのように扱います [式](#syntax-expressions). -データの形式は任意です。 クエリが受信されると、サーバーは以下の値を計算しません [max\_query\_size](../operations/settings/settings.md#settings-max_query_size) 要求のバイトはRAM(デフォルトでは1MB)で、残りはストリーム解析されます。 +データの形式は任意です。 クエリが受信されると、サーバーは以下の値を計算しません [max_query_size](../operations/settings/settings.md#settings-max_query_size) 要求のバイトはRAM(デフォルトでは1MB)で、残りはストリーム解析されます。 これを回避する問題の大き `INSERT` クエリ。 を使用する場合 `Values` フォーマット `INSERT` これは、データがaの式と同じように解析されるように見えるかもしれません `SELECT` クエリが、これは真実ではありません。 その `Values` 形式ははるかに限られています。 diff --git a/docs/ja/sql-reference/table-functions/file.md b/docs/ja/sql-reference/table-functions/file.md index 125ff8bf238..7f93df69495 100644 --- a/docs/ja/sql-reference/table-functions/file.md +++ b/docs/ja/sql-reference/table-functions/file.md @@ -15,7 +15,7 @@ file(path, format, structure) **入力パラメータ** -- `path` — The relative path to the file from [user\_files\_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). パスファイルをサポートglobに読み取り専用モード: `*`, `?`, `{abc,def}` と `{N..M}` どこに `N`, `M` — numbers, \``'abc', 'def'` — strings. +- `path` — The relative path to the file from [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). パスファイルをサポートglobに読み取り専用モード: `*`, `?`, `{abc,def}` と `{N..M}` どこに `N`, `M` — numbers, \``'abc', 'def'` — strings. - `format` — The [形式](../../interfaces/formats.md#formats) ファイルの。 - `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. @@ -72,12 +72,12 @@ SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 U 1. 次の相対パスを持つ複数のファイルがあるとします: -- ‘some\_dir/some\_file\_1’ -- ‘some\_dir/some\_file\_2’ -- ‘some\_dir/some\_file\_3’ -- ‘another\_dir/some\_file\_1’ -- ‘another\_dir/some\_file\_2’ -- ‘another\_dir/some\_file\_3’ +- ‘some_dir/some_file_1’ +- ‘some_dir/some_file_2’ +- ‘some_dir/some_file_3’ +- ‘another_dir/some_file_1’ +- ‘another_dir/some_file_2’ +- ‘another_dir/some_file_3’ 1. これらのファイル内の行の量を照会します: diff --git a/docs/ja/sql-reference/table-functions/hdfs.md b/docs/ja/sql-reference/table-functions/hdfs.md index ccf89095ce9..23772bc2400 100644 --- a/docs/ja/sql-reference/table-functions/hdfs.md +++ b/docs/ja/sql-reference/table-functions/hdfs.md @@ -55,12 +55,12 @@ LIMIT 2 1. HDFSに次のUriを持つ複数のファイルがあるとします: -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_3’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_3’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_3’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_3’ 1. これらのファイル内の行の量を照会します: diff --git a/docs/ja/sql-reference/table-functions/index.md b/docs/ja/sql-reference/table-functions/index.md index 5d8e2592039..10ef4b991af 100644 --- a/docs/ja/sql-reference/table-functions/index.md +++ b/docs/ja/sql-reference/table-functions/index.md @@ -16,12 +16,12 @@ toc_title: "\u306F\u3058\u3081\u306B" The method for creating a temporary table that is available only in the current query. The table is deleted when the query finishes. -- [テーブルを\として作成](../statements/create.md#create-table-query) クエリ。 +- [テーブルを\として作成](../statements/create.md#create-table-query) クエリ。 It's one of the methods of creating a table. !!! warning "警告" - テーブル関数を使用することはできません。 [allow\_ddl](../../operations/settings/permissions-for-queries.md#settings_allow_ddl) 設定は無効です。 + テーブル関数を使用することはできません。 [allow_ddl](../../operations/settings/permissions-for-queries.md#settings_allow_ddl) 設定は無効です。 | 関数 | 説明 | |-----------------------|----------------------------------------------------------------------------------------------------------------------------------------| diff --git a/docs/ja/sql-reference/table-functions/remote.md b/docs/ja/sql-reference/table-functions/remote.md index 368584551fc..b2ec55ec182 100644 --- a/docs/ja/sql-reference/table-functions/remote.md +++ b/docs/ja/sql-reference/table-functions/remote.md @@ -56,7 +56,7 @@ example01-{01..02}-1 中括弧のペアが複数ある場合、対応する集合の直接積を生成します。 -中括弧の中の住所と住所の一部は、パイプ記号(\|)で区切ることができます。 この場合、対応するアドレスのセットはレプリカとして解釈され、クエリは最初の正常なレプリカに送信されます。 ただし、レプリカは、現在設定されている順序で反復処理されます。 [load\_balancing](../../operations/settings/settings.md) 設定。 +中括弧の中の住所と住所の一部は、パイプ記号(\|)で区切ることができます。 この場合、対応するアドレスのセットはレプリカとして解釈され、クエリは最初の正常なレプリカに送信されます。 ただし、レプリカは、現在設定されている順序で反復処理されます。 [load_balancing](../../operations/settings/settings.md) 設定。 例: @@ -80,6 +80,6 @@ example01-{01..02}-{1|2} ユーザーが指定されていない場合, `default` が使用される。 パスワードを指定しない場合は、空のパスワードが使用されます。 -`remoteSecure` -同じように `remote` but with secured connection. Default port — [tcp\_port\_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) 設定または9440から。 +`remoteSecure` -同じように `remote` but with secured connection. Default port — [tcp_port_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) 設定または9440から。 [元の記事](https://clickhouse.tech/docs/en/query_language/table_functions/remote/) diff --git a/docs/ja/whats-new/security-changelog.md b/docs/ja/whats-new/security-changelog.md index d56b465acfa..aa93f9c42ae 100644 --- a/docs/ja/whats-new/security-changelog.md +++ b/docs/ja/whats-new/security-changelog.md @@ -53,7 +53,7 @@ unixODBC許容荷重任意の共有オブジェクトからのファイルシス ### CVE-2018-14668 {#cve-2018-14668} -“remote” テーブル関数で任意のシンボルを許可 “user”, “password” と “default\_database” 分野を横断プロトコルの要求偽造攻撃であった。 +“remote” テーブル関数で任意のシンボルを許可 “user”, “password” と “default_database” 分野を横断プロトコルの要求偽造攻撃であった。 クレジット:Yandexの情報セキュリティチームのアンドレイKrasichkov diff --git a/docs/ru/development/contrib.md b/docs/ru/development/contrib.md index 7a9c7c7c14d..c640ecee79d 100644 --- a/docs/ru/development/contrib.md +++ b/docs/ru/development/contrib.md @@ -21,7 +21,7 @@ | libpcg-random | [Apache License 2.0](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libpcg-random/LICENSE-APACHE.txt) | | libressl | [OpenSSL License](https://github.com/ClickHouse-Extras/ssl/blob/master/COPYING) | | librdkafka | [BSD 2-Clause License](https://github.com/edenhill/librdkafka/blob/363dcad5a23dc29381cc626620e68ae418b3af19/LICENSE) | -| libwidechar\_width | [CC0 1.0 Universal](https://github.com/ClickHouse/ClickHouse/blob/master/libs/libwidechar_width/LICENSE) | +| libwidechar_width | [CC0 1.0 Universal](https://github.com/ClickHouse/ClickHouse/blob/master/libs/libwidechar_width/LICENSE) | | llvm | [BSD 3-Clause License](https://github.com/ClickHouse-Extras/llvm/blob/163def217817c90fb982a6daf384744d8472b92b/llvm/LICENSE.TXT) | | lz4 | [BSD 2-Clause License](https://github.com/lz4/lz4/blob/c10863b98e1503af90616ae99725ecd120265dfb/LICENSE) | | mariadb-connector-c | [LGPL v2.1](https://github.com/ClickHouse-Extras/mariadb-connector-c/blob/3.1/COPYING.LIB) | diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index 6775b5378cb..9edcd4cf9f9 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -150,7 +150,7 @@ ClickHouse использует для сборки некоторое коли mkdir build cd build -Вы можете иметь несколько разных директорий (build\_release, build\_debug) для разных вариантов сборки. +Вы можете иметь несколько разных директорий (build_release, build_debug) для разных вариантов сборки. Находясь в директории build, выполните конфигурацию сборки с помощью CMake. Перед первым запуском необходимо выставить переменные окружения, отвечающие за выбор компилятора (в данном примере это - gcc версии 9). diff --git a/docs/ru/development/style.md b/docs/ru/development/style.md index d97b3469467..951537e8113 100644 --- a/docs/ru/development/style.md +++ b/docs/ru/development/style.md @@ -349,7 +349,7 @@ class IBlockInputStream bool info_successfully_loaded = false; ``` -**9.** В именах `define` и глобальных констант используется ALL\_CAPS с подчёркиванием. +**9.** В именах `define` и глобальных констант используется ALL_CAPS с подчёркиванием. ``` cpp #define MAX_SRC_TABLE_NAMES_TO_STORE 1000 @@ -389,7 +389,7 @@ FileQueueProcessor( timer (not m_timer) ``` -**14.** Константы в `enum` — CamelCase с большой буквы. Также допустим ALL\_CAPS. Если `enum` не локален, то используйте `enum class`. +**14.** Константы в `enum` — CamelCase с большой буквы. Также допустим ALL_CAPS. Если `enum` не локален, то используйте `enum class`. ``` cpp enum class CompressionMethod @@ -779,7 +779,7 @@ The dictionary is configured incorrectly. **4.** ОС - Linux, Mac OS X или FreeBSD. -**5.** Код пишется под процессоры с архитектурой x86\_64, AArch64 и ppc64le. +**5.** Код пишется под процессоры с архитектурой x86_64, AArch64 и ppc64le. **6.** Используются флаги компиляции `-Wall -Wextra -Werror` и `-Weverything` с некоторыми исключениями. diff --git a/docs/ru/engines/table-engines/integrations/hdfs.md b/docs/ru/engines/table-engines/integrations/hdfs.md index 698139fa8a3..b5234b38d7d 100644 --- a/docs/ru/engines/table-engines/integrations/hdfs.md +++ b/docs/ru/engines/table-engines/integrations/hdfs.md @@ -62,12 +62,12 @@ SELECT * FROM hdfs_engine_table LIMIT 2 1. Предположим, у нас есть несколько файлов со следующими URI в HDFS: -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_3’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_3’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_3’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_3’ 1. Есть несколько возможностей создать таблицу, состояющую из этих шести файлов: diff --git a/docs/ru/engines/table-engines/integrations/kafka.md b/docs/ru/engines/table-engines/integrations/kafka.md index 8a79c8556b8..8c47e97f497 100644 --- a/docs/ru/engines/table-engines/integrations/kafka.md +++ b/docs/ru/engines/table-engines/integrations/kafka.md @@ -123,7 +123,7 @@ Kafka(kafka_broker_list, kafka_topic_list, kafka_group_name, kafka_format SELECT level, sum(total) FROM daily GROUP BY level; ``` -Для улучшения производительности полученные сообщения группируются в блоки размера [max\_insert\_block\_size](../../../operations/settings/settings.md#settings-max_insert_block_size). Если блок не удалось сформировать за [stream\_flush\_interval\_ms](../../../operations/settings/settings.md#stream-flush-interval-ms) миллисекунд, то данные будут сброшены в таблицу независимо от полноты блока. +Для улучшения производительности полученные сообщения группируются в блоки размера [max_insert_block_size](../../../operations/settings/settings.md#settings-max_insert_block_size). Если блок не удалось сформировать за [stream_flush_interval_ms](../../../operations/settings/settings.md#stream-flush-interval-ms) миллисекунд, то данные будут сброшены в таблицу независимо от полноты блока. Чтобы остановить получение данных топика или изменить логику преобразования, отсоедините материализованное представление: diff --git a/docs/ru/engines/table-engines/integrations/rabbitmq.md b/docs/ru/engines/table-engines/integrations/rabbitmq.md index f0aeee34d70..ef7b811e295 100644 --- a/docs/ru/engines/table-engines/integrations/rabbitmq.md +++ b/docs/ru/engines/table-engines/integrations/rabbitmq.md @@ -102,7 +102,7 @@ Example: 2. чтобы объявить одни и те же очереди для разных таблиц, что позволяет создавать несколько параллельных подписчиков на каждую из очередей. То есть обеспечивается лучшая производительность. В данном случае, для таких таблиц также необходимо совпадение настроек: `rabbitmq_num_consumers`, `rabbitmq_num_queues`. 3. чтобы повторно использовать созданные c `durable` настройкой очереди, так как они не удаляются автоматически (но могут быть удалены с помощью любого RabbitMQ CLI). -Для улучшения производительности полученные сообщения группируются в блоки размера [max\_insert\_block\_size](../../../operations/settings/settings.md#settings-max_insert_block_size). Если блок не удалось сформировать за [stream\_flush\_interval\_ms](../../../operations/settings/settings.md#stream-flush-interval-ms) миллисекунд, то данные будут сброшены в таблицу независимо от полноты блока. +Для улучшения производительности полученные сообщения группируются в блоки размера [max_insert_block_size](../../../operations/settings/settings.md#settings-max_insert_block_size). Если блок не удалось сформировать за [stream_flush_interval_ms](../../../operations/settings/settings.md#stream-flush-interval-ms) миллисекунд, то данные будут сброшены в таблицу независимо от полноты блока. Если параметры`rabbitmq_num_consumers` и/или `rabbitmq_num_queues` заданы вместе с параметром `rabbitmq_exchange_type`: diff --git a/docs/ru/engines/table-engines/mergetree-family/custom-partitioning-key.md b/docs/ru/engines/table-engines/mergetree-family/custom-partitioning-key.md index 89980a3bd73..32b5eee7fab 100644 --- a/docs/ru/engines/table-engines/mergetree-family/custom-partitioning-key.md +++ b/docs/ru/engines/table-engines/mergetree-family/custom-partitioning-key.md @@ -70,7 +70,7 @@ WHERE table = 'visits' - `1` – уровень куска (глубина дерева слияний, которыми этот кусок образован). !!! info "Info" - Названия кусков для таблиц старого типа образуются следующим образом: `20190117_20190123_2_2_0` (минимальная дата \_ максимальная дата \_ номер минимального блока \_ номер максимального блока \_ уровень). + Названия кусков для таблиц старого типа образуются следующим образом: `20190117_20190123_2_2_0` (минимальная дата _ максимальная дата _ номер минимального блока _ номер максимального блока _ уровень). Как видно из примера выше, таблица содержит несколько отдельных кусков для одной и той же партиции (например, куски `201901_1_3_1` и `201901_1_9_2` принадлежат партиции `201901`). Это означает, что эти куски еще не были объединены – в файловой системе они хранятся отдельно. После того как будет выполнено автоматическое слияние данных (выполняется примерно спустя 10 минут после вставки данных), исходные куски будут объединены в один более крупный кусок и помечены как неактивные. @@ -111,7 +111,7 @@ drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 5 12:09 201902_4_6_1 drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 1 16:48 detached ``` -‘201901\_1\_1\_0’, ‘201901\_1\_7\_1’ и т. д. – это директории кусков партиции. Каждый кусок содержит данные только для соответствующего месяца (таблица в данном примере содержит партиционирование по месяцам). +‘201901_1_1_0’, ‘201901_1_7_1’ и т. д. – это директории кусков партиции. Каждый кусок содержит данные только для соответствующего месяца (таблица в данном примере содержит партиционирование по месяцам). Директория `detached` содержит куски, отсоединенные от таблицы с помощью запроса [DETACH](../../../sql-reference/statements/alter/partition.md#alter_detach-partition). Поврежденные куски также попадают в эту директорию – они не удаляются с сервера. diff --git a/docs/ru/engines/table-engines/mergetree-family/graphitemergetree.md b/docs/ru/engines/table-engines/mergetree-family/graphitemergetree.md index c38065a6e5d..861a8b6e633 100644 --- a/docs/ru/engines/table-engines/mergetree-family/graphitemergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/graphitemergetree.md @@ -74,7 +74,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ## Конфигурация Rollup {#rollup-configuration} -Настройки прореживания данных задаются параметром [graphite\_rollup](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) в конфигурации сервера . Имя параметра может быть любым. Можно создать несколько конфигураций и использовать их для разных таблиц. +Настройки прореживания данных задаются параметром [graphite_rollup](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) в конфигурации сервера . Имя параметра может быть любым. Можно создать несколько конфигураций и использовать их для разных таблиц. Структура конфигурации rollup: diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 2f89317a0eb..ef1e236a95c 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -197,7 +197,7 @@ ClickHouse не требует уникального первичного кл Длинный первичный ключ будет негативно влиять на производительность вставки и потребление памяти, однако на производительность ClickHouse при запросах `SELECT` лишние столбцы в первичном ключе не влияют. -Вы можете создать таблицу без первичного ключа, используя синтаксис `ORDER BY tuple()`. В этом случае ClickHouse хранит данные в порядке вставки. Если вы хотите сохранить порядок данных при вставке данных с помощью запросов `INSERT ... SELECT`, установите [max\_insert\_threads = 1](../../../operations/settings/settings.md#settings-max-insert-threads). +Вы можете создать таблицу без первичного ключа, используя синтаксис `ORDER BY tuple()`. В этом случае ClickHouse хранит данные в порядке вставки. Если вы хотите сохранить порядок данных при вставке данных с помощью запросов `INSERT ... SELECT`, установите [max_insert_threads = 1](../../../operations/settings/settings.md#settings-max-insert-threads). Чтобы выбрать данные в первоначальном порядке, используйте [однопоточные](../../../operations/settings/settings.md#settings-max_threads) запросы `SELECT. @@ -254,7 +254,7 @@ ClickHouse будет использовать индекс по первичн SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' ``` -Чтобы проверить, сможет ли ClickHouse использовать индекс при выполнении запроса, используйте настройки [force\_index\_by\_date](../../../operations/settings/settings.md#settings-force_index_by_date) и [force\_primary\_key](../../../operations/settings/settings.md#settings-force_primary_key). +Чтобы проверить, сможет ли ClickHouse использовать индекс при выполнении запроса, используйте настройки [force_index_by_date](../../../operations/settings/settings.md#settings-force_index_by_date) и [force_primary_key](../../../operations/settings/settings.md#settings-force_primary_key). Ключ партиционирования по месяцам обеспечивает чтение только тех блоков данных, которые содержат даты из нужного диапазона. При этом блок данных может содержать данные за многие даты (до целого месяца). В пределах одного блока данные упорядочены по первичному ключу, который может не содержать дату в качестве первого столбца. В связи с этим, при использовании запроса с указанием условия только на дату, но не на префикс первичного ключа, будет читаться данных больше, чем за одну дату. @@ -327,7 +327,7 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT Индекс `set` используется со всеми функциями. Наборы функций для остальных индексов представлены в таблице ниже. -| Функция (оператор) / Индекс | primary key | minmax | ngrambf\_v1 | tokenbf\_v1 | bloom\_filter | +| Функция (оператор) / Индекс | primary key | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter | |------------------------------------------------------------------------------------------------------------|-------------|--------|-------------|-------------|---------------| | [equals (=, ==)](../../../sql-reference/functions/comparison-functions.md#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | | [notEquals(!=, \<\>)](../../../sql-reference/functions/comparison-functions.md#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | @@ -490,7 +490,7 @@ ALTER TABLE example_table - Том (Volume) — упорядоченный набор равноценных дисков (схоже с [JBOD](https://ru.wikipedia.org/wiki/JBOD)) - Политика хранения (StoragePolicy) — множество томов с правилами перемещения данных между ними. -У всех описанных сущностей при создании указываются имена, можно найти в системных таблицах [system.storage\_policies](../../../engines/table-engines/mergetree-family/mergetree.md#system_tables-storage_policies) и [system.disks](../../../engines/table-engines/mergetree-family/mergetree.md#system_tables-disks). Имя политики хранения можно указать в настройке `storage_policy` движков таблиц семейства `MergeTree`. +У всех описанных сущностей при создании указываются имена, можно найти в системных таблицах [system.storage_policies](../../../engines/table-engines/mergetree-family/mergetree.md#system_tables-storage_policies) и [system.disks](../../../engines/table-engines/mergetree-family/mergetree.md#system_tables-disks). Имя политики хранения можно указать в настройке `storage_policy` движков таблиц семейства `MergeTree`. ### Конфигурация {#table_engine-mergetree-multiple-volumes_configure} @@ -637,7 +637,7 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' Мутации и запросы заморозки партиций в реализации используют [жесткие ссылки](https://ru.wikipedia.org/wiki/%D0%96%D1%91%D1%81%D1%82%D0%BA%D0%B0%D1%8F_%D1%81%D1%81%D1%8B%D0%BB%D0%BA%D0%B0). Жесткие ссылки между различными дисками не поддерживаются, поэтому в случае таких операций куски размещаются на тех же дисках, что и исходные. -В фоне куски перемещаются между томами на основе информации о занятом месте (настройка `move_factor`) по порядку, в котором указаны тома в конфигурации. Данные никогда не перемещаются с последнего тома и на первый том. Следить за фоновыми перемещениями можно с помощью системных таблиц [system.part\_log](../../../engines/table-engines/mergetree-family/mergetree.md#system_tables-part-log) (поле `type = MOVE_PART`) и [system.parts](../../../engines/table-engines/mergetree-family/mergetree.md#system_tables-parts) (поля `path` и `disk`). Также подробная информация о перемещениях доступна в логах сервера. +В фоне куски перемещаются между томами на основе информации о занятом месте (настройка `move_factor`) по порядку, в котором указаны тома в конфигурации. Данные никогда не перемещаются с последнего тома и на первый том. Следить за фоновыми перемещениями можно с помощью системных таблиц [system.part_log](../../../engines/table-engines/mergetree-family/mergetree.md#system_tables-part-log) (поле `type = MOVE_PART`) и [system.parts](../../../engines/table-engines/mergetree-family/mergetree.md#system_tables-parts) (поля `path` и `disk`). Также подробная информация о перемещениях доступна в логах сервера. С помощью запроса [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](../../../engines/table-engines/mergetree-family/mergetree.md#alter_move-partition) пользователь может принудительно перенести кусок или партицию с одного раздела на другой. При этом учитываются все ограничения, указанные для фоновых операций. Запрос самостоятельно инициирует процесс перемещения не дожидаясь фоновых операций. В случае недостатка места или неудовлетворения ограничениям пользователь получит сообщение об ошибке. Перемещения данных не взаимодействуют с репликацией данных, поэтому на разных репликах одной и той же таблицы могут быть указаны разные политики хранения. diff --git a/docs/ru/engines/table-engines/mergetree-family/replication.md b/docs/ru/engines/table-engines/mergetree-family/replication.md index 54f390c293e..73291020aa3 100644 --- a/docs/ru/engines/table-engines/mergetree-family/replication.md +++ b/docs/ru/engines/table-engines/mergetree-family/replication.md @@ -52,7 +52,7 @@ ClickHouse хранит метаинформацию о репликах в [Apa Если в конфигурационном файле не настроен ZooKeeper, то вы не сможете создать реплицируемые таблицы, а уже имеющиеся реплицируемые таблицы будут доступны в режиме только на чтение. -При запросах `SELECT`, ZooKeeper не используется, т.е. репликация не влияет на производительность `SELECT` и запросы работают так же быстро, как и для нереплицируемых таблиц. При запросах к распределенным реплицированным таблицам поведение ClickHouse регулируется настройками [max\_replica\_delay\_for\_distributed\_queries](../../../operations/settings/settings.md#settings-max_replica_delay_for_distributed_queries) and [fallback\_to\_stale\_replicas\_for\_distributed\_queries](../../../operations/settings/settings.md). +При запросах `SELECT`, ZooKeeper не используется, т.е. репликация не влияет на производительность `SELECT` и запросы работают так же быстро, как и для нереплицируемых таблиц. При запросах к распределенным реплицированным таблицам поведение ClickHouse регулируется настройками [max_replica_delay_for_distributed_queries](../../../operations/settings/settings.md#settings-max_replica_delay_for_distributed_queries) and [fallback_to_stale_replicas_for_distributed_queries](../../../operations/settings/settings.md). При каждом запросе `INSERT`, делается около десятка записей в ZooKeeper в рамках нескольких транзакций. (Чтобы быть более точным, это для каждого вставленного блока данных; запрос INSERT содержит один блок или один блок на `max_insert_block_size = 1048576` строк.) Это приводит к некоторому увеличению задержек при `INSERT`, по сравнению с нереплицируемыми таблицами. Но если придерживаться обычных рекомендаций - вставлять данные пачками не более одного `INSERT` в секунду, то это не составляет проблем. На всём кластере ClickHouse, использующим для координации один кластер ZooKeeper, может быть в совокупности несколько сотен `INSERT` в секунду. Пропускная способность при вставке данных (количество строчек в секунду) такая же высокая, как для нереплицируемых таблиц. @@ -64,7 +64,7 @@ ClickHouse хранит метаинформацию о репликах в [Apa Каждый блок данных записывается атомарно. Запрос INSERT разбивается на блоки данных размером до `max_insert_block_size = 1048576` строк. То есть, если в запросе `INSERT` менее 1048576 строк, то он делается атомарно. -Блоки данных дедуплицируются. При многократной записи одного и того же блока данных (блоков данных одинакового размера, содержащих одни и те же строчки в одном и том же порядке), блок будет записан только один раз. Это сделано для того, чтобы в случае сбоя в сети, когда клиентское приложение не может понять, были ли данные записаны в БД, можно было просто повторить запрос `INSERT`. При этом не имеет значения, на какую реплику будут отправлены INSERT-ы с одинаковыми данными. Запрос `INSERT` идемпотентный. Параметры дедуплицирования регулируются настройками сервера [merge\_tree](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-merge_tree) +Блоки данных дедуплицируются. При многократной записи одного и того же блока данных (блоков данных одинакового размера, содержащих одни и те же строчки в одном и том же порядке), блок будет записан только один раз. Это сделано для того, чтобы в случае сбоя в сети, когда клиентское приложение не может понять, были ли данные записаны в БД, можно было просто повторить запрос `INSERT`. При этом не имеет значения, на какую реплику будут отправлены INSERT-ы с одинаковыми данными. Запрос `INSERT` идемпотентный. Параметры дедуплицирования регулируются настройками сервера [merge_tree](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-merge_tree) При репликации, по сети передаются только исходные вставляемые данные. Дальнейшие преобразования данных (слияния) координируются и делаются на всех репликах одинаковым образом. За счёт этого минимизируется использование сети, и благодаря этому, репликация хорошо работает при расположении реплик в разных дата-центрах. (Стоит заметить, что дублирование данных в разных дата-центрах, по сути, является основной задачей репликации). diff --git a/docs/ru/engines/table-engines/special/buffer.md b/docs/ru/engines/table-engines/special/buffer.md index 5866b36164f..72d2822af98 100644 --- a/docs/ru/engines/table-engines/special/buffer.md +++ b/docs/ru/engines/table-engines/special/buffer.md @@ -29,7 +29,7 @@ Buffer(database, table, num_layers, min_time, max_time, min_rows, max_rows, min_ CREATE TABLE merge.hits_buffer AS merge.hits ENGINE = Buffer(merge, hits, 16, 10, 100, 10000, 1000000, 10000000, 100000000) ``` -Создаём таблицу merge.hits\_buffer такой же структуры как merge.hits и движком Buffer. При записи в эту таблицу, данные буферизуются в оперативке и, в дальнейшем, записываются в таблицу merge.hits. Создаётся 16 буферов. Данные, имеющиеся в каждом из них будут сбрасываться, если прошло сто секунд, или записан миллион строк, или записано сто мегабайт данных; или если одновременно прошло десять секунд и записано десять тысяч строк и записано десять мегабайт данных. Для примера, если записана всего лишь одна строка, то через сто секунд она будет сброшена в любом случае. А если записано много строк, то они будут сброшены раньше. +Создаём таблицу merge.hits_buffer такой же структуры как merge.hits и движком Buffer. При записи в эту таблицу, данные буферизуются в оперативке и, в дальнейшем, записываются в таблицу merge.hits. Создаётся 16 буферов. Данные, имеющиеся в каждом из них будут сбрасываться, если прошло сто секунд, или записан миллион строк, или записано сто мегабайт данных; или если одновременно прошло десять секунд и записано десять тысяч строк и записано десять мегабайт данных. Для примера, если записана всего лишь одна строка, то через сто секунд она будет сброшена в любом случае. А если записано много строк, то они будут сброшены раньше. При остановке сервера, при DROP TABLE или DETACH TABLE, данные из буфера тоже сбрасываются в таблицу назначения. @@ -51,7 +51,7 @@ CREATE TABLE merge.hits_buffer AS merge.hits ENGINE = Buffer(merge, hits, 16, 10 При добавлении данных в Buffer, один из буферов блокируется. Это приводит к задержкам, если одновременно делается чтение из таблицы. -Данные, вставляемые в таблицу Buffer, попадают в подчинённую таблицу в порядке, возможно отличающимся от порядка вставки, и блоками, возможно отличающимися от вставленных блоков. В связи с этим, трудно корректно использовать таблицу типа Buffer для записи в CollapsingMergeTree. Чтобы избежать проблемы, можно выставить num\_layers в 1. +Данные, вставляемые в таблицу Buffer, попадают в подчинённую таблицу в порядке, возможно отличающимся от порядка вставки, и блоками, возможно отличающимися от вставленных блоков. В связи с этим, трудно корректно использовать таблицу типа Buffer для записи в CollapsingMergeTree. Чтобы избежать проблемы, можно выставить num_layers в 1. Если таблица назначения является реплицируемой, то при записи в таблицу Buffer будут потеряны некоторые ожидаемые свойства реплицируемых таблиц. Из-за произвольного изменения порядка строк и размеров блоков данных, перестаёт работать дедупликация данных, в результате чего исчезает возможность надёжной exactly once записи в реплицируемые таблицы. diff --git a/docs/ru/engines/table-engines/special/distributed.md b/docs/ru/engines/table-engines/special/distributed.md index 53c5a02e752..c8f7fe6eba7 100644 --- a/docs/ru/engines/table-engines/special/distributed.md +++ b/docs/ru/engines/table-engines/special/distributed.md @@ -78,13 +78,13 @@ logs - имя кластера в конфигурационном файле с В качестве параметров для каждого сервера указываются `host`, `port` и, не обязательно, `user`, `password`, `secure`, `compression`: - `host` - адрес удалённого сервера. Может быть указан домен, или IPv4 или IPv6 адрес. В случае указания домена, при старте сервера делается DNS запрос, и результат запоминается на всё время работы сервера. Если DNS запрос неуспешен, то сервер не запускается. Если вы изменяете DNS-запись, перезапустите сервер. -- `port` - TCP-порт для межсерверного взаимодействия (в конфиге - tcp\_port, обычно 9000). Не перепутайте с http\_port. +- `port` - TCP-порт для межсерверного взаимодействия (в конфиге - tcp_port, обычно 9000). Не перепутайте с http_port. - `user` - имя пользователя для соединения с удалённым сервером. по умолчанию - default. Этот пользователь должен иметь доступ для соединения с указанным сервером. Доступы настраиваются в файле users.xml, подробнее смотрите в разделе [Права доступа](../../../operations/access-rights.md). - `password` - пароль для соединения с удалённым сервером, в открытом виде. по умолчанию - пустая строка. - `secure` - Использовать шифрованное соединение ssl, Обычно используется с портом `port` = 9440. Сервер должен слушать порт `9440` с корректными настройками сертификатов. - `compression` - Использовать сжатие данных. По умолчанию: true. -При указании реплик, для каждого из шардов, при чтении, будет выбрана одна из доступных реплик. Можно настроить алгоритм балансировки нагрузки (то есть, предпочтения, на какую из реплик идти) - см. настройку [load\_balancing](../../../operations/settings/settings.md#settings-load_balancing). +При указании реплик, для каждого из шардов, при чтении, будет выбрана одна из доступных реплик. Можно настроить алгоритм балансировки нагрузки (то есть, предпочтения, на какую из реплик идти) - см. настройку [load_balancing](../../../operations/settings/settings.md#settings-load_balancing). Если соединение с сервером не установлено, то будет произведена попытка соединения с небольшим таймаутом. Если соединиться не удалось, то будет выбрана следующая реплика, и так для всех реплик. Если попытка соединения для всех реплик не удалась, то будут снова произведены попытки соединения по кругу, и так несколько раз. Это работает в пользу отказоустойчивости, хотя и не обеспечивает полную отказоустойчивость: удалённый сервер может принять соединение, но не работать, или плохо работать. @@ -107,13 +107,13 @@ logs - имя кластера в конфигурационном файле с У каждого шарда в конфигурационном файле может быть задан «вес» (weight). По умолчанию, вес равен единице. Данные будут распределяться по шардам в количестве, пропорциональном весу шарда. Например, если есть два шарда, и у первого выставлен вес 9, а у второго 10, то на первый будет отправляться 9 / 19 доля строк, а на второй - 10 / 19. -У каждого шарда в конфигурационном файле может быть указан параметр internal\_replication. +У каждого шарда в конфигурационном файле может быть указан параметр internal_replication. Если он выставлен в true, то для записи будет выбираться первая живая реплика и данные будут писаться на неё. Этот вариант следует использовать, если Distributed таблица «смотрит» на реплицируемые таблицы. То есть, если таблица, в которую будут записаны данные, будет сама заниматься их репликацией. Если он выставлен в false (по умолчанию), то данные будут записываться на все реплики. По сути, это означает, что Distributed таблица занимается репликацией данных самостоятельно. Это хуже, чем использование реплицируемых таблиц, так как не контролируется консистентность реплик, и они со временем будут содержать немного разные данные. -Для выбора шарда, на который отправляется строка данных, вычисляется выражение шардирования, и берётся его остаток от деления на суммарный вес шардов. Строка отправляется на шард, соответствующий полуинтервалу остатков от prev\_weights до prev\_weights + weight, где prev\_weights - сумма весов шардов с меньшим номером, а weight - вес этого шарда. Например, если есть два шарда, и у первого выставлен вес 9, а у второго 10, то строка будет отправляться на первый шард для остатков из диапазона \[0, 9), а на второй - для остатков из диапазона \[9, 19). +Для выбора шарда, на который отправляется строка данных, вычисляется выражение шардирования, и берётся его остаток от деления на суммарный вес шардов. Строка отправляется на шард, соответствующий полуинтервалу остатков от prev_weights до prev_weights + weight, где prev_weights - сумма весов шардов с меньшим номером, а weight - вес этого шарда. Например, если есть два шарда, и у первого выставлен вес 9, а у второго 10, то строка будет отправляться на первый шард для остатков из диапазона \[0, 9), а на второй - для остатков из диапазона \[9, 19). Выражением шардирование может быть произвольное выражение от констант и столбцов таблицы, возвращающее целое число. Например, вы можете использовать выражение rand() для случайного распределения данных, или UserID - для распределения по остатку от деления идентификатора посетителя (тогда данные одного посетителя будут расположены на одном шарде, что упростит выполнение IN и JOIN по посетителям). Если распределение какого-либо столбца недостаточно равномерное, вы можете обернуть его в хэш функцию: intHash64(UserID). @@ -125,10 +125,10 @@ logs - имя кластера в конфигурационном файле с - используются запросы, требующие соединение данных (IN, JOIN) по определённому ключу - тогда если данные шардированы по этому ключу, то можно использовать локальные IN, JOIN вместо GLOBAL IN, GLOBAL JOIN, что кардинально более эффективно. - используется большое количество серверов (сотни и больше) и большое количество маленьких запросов (запросы отдельных клиентов - сайтов, рекламодателей, партнёров) - тогда, для того, чтобы маленькие запросы не затрагивали весь кластер, имеет смысл располагать данные одного клиента на одном шарде, или (вариант, который используется в Яндекс.Метрике) сделать двухуровневое шардирование: разбить весь кластер на «слои», где слой может состоять из нескольких шардов; данные для одного клиента располагаются на одном слое, но в один слой можно по мере необходимости добавлять шарды, в рамках которых данные распределены произвольным образом; создаются распределённые таблицы на каждый слой и одна общая распределённая таблица для глобальных запросов. -Запись данных осуществляется полностью асинхронно. При вставке в таблицу, блок данных сначала записывается в файловую систему. Затем, в фоновом режиме отправляются на удалённые серверы при первой возможности. Период отправки регулируется настройками [distributed\_directory\_monitor\_sleep\_time\_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms) и [distributed\_directory\_monitor\_max\_sleep\_time\_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms). Движок таблиц `Distributed` отправляет каждый файл со вставленными данными отдельно, но можно включить пакетную отправку данных настройкой [distributed\_directory\_monitor\_batch\_inserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts). Эта настройка улучшает производительность кластера за счет более оптимального использования ресурсов сервера-отправителя и сети. Необходимо проверять, что данные отправлены успешно, для этого проверьте список файлов (данных, ожидающих отправки) в каталоге таблицы `/var/lib/clickhouse/data/database/table/`. Количество потоков для выполнения фоновых задач можно задать с помощью настройки [background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size). +Запись данных осуществляется полностью асинхронно. При вставке в таблицу, блок данных сначала записывается в файловую систему. Затем, в фоновом режиме отправляются на удалённые серверы при первой возможности. Период отправки регулируется настройками [distributed_directory_monitor_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms) и [distributed_directory_monitor_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms). Движок таблиц `Distributed` отправляет каждый файл со вставленными данными отдельно, но можно включить пакетную отправку данных настройкой [distributed_directory_monitor_batch_inserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts). Эта настройка улучшает производительность кластера за счет более оптимального использования ресурсов сервера-отправителя и сети. Необходимо проверять, что данные отправлены успешно, для этого проверьте список файлов (данных, ожидающих отправки) в каталоге таблицы `/var/lib/clickhouse/data/database/table/`. Количество потоков для выполнения фоновых задач можно задать с помощью настройки [background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size). Если после INSERT-а в Distributed таблицу, сервер перестал существовать или был грубо перезапущен (например, в следствие аппаратного сбоя), то записанные данные могут быть потеряны. Если в директории таблицы обнаружен повреждённый кусок данных, то он переносится в поддиректорию broken и больше не используется. -При выставлении опции max\_parallel\_replicas выполнение запроса распараллеливается по всем репликам внутри одного шарда. Подробнее смотрите раздел [max\_parallel\_replicas](../../../operations/settings/settings.md#settings-max_parallel_replicas). +При выставлении опции max_parallel_replicas выполнение запроса распараллеливается по всем репликам внутри одного шарда. Подробнее смотрите раздел [max_parallel_replicas](../../../operations/settings/settings.md#settings-max_parallel_replicas). [Оригинальная статья](https://clickhouse.tech/docs/ru/operations/table_engines/distributed/) diff --git a/docs/ru/engines/table-engines/special/external-data.md b/docs/ru/engines/table-engines/special/external-data.md index 700a718ed0c..41708978367 100644 --- a/docs/ru/engines/table-engines/special/external-data.md +++ b/docs/ru/engines/table-engines/special/external-data.md @@ -21,11 +21,11 @@ ClickHouse позволяет отправить на сервер данные, Из stdin может быть считана только одна таблица. Следующие параметры не обязательные: -**–name** - имя таблицы. Если не указано - используется \_data. +**–name** - имя таблицы. Если не указано - используется _data. **–format** - формат данных в файле. Если не указано - используется TabSeparated. Должен быть указан один из следующих параметров: -**–types** - список типов столбцов через запятую. Например, `UInt64,String`. Столбцы будут названы \_1, \_2, … +**–types** - список типов столбцов через запятую. Например, `UInt64,String`. Столбцы будут названы _1, _2, … **–structure** - структура таблицы, в форме `UserID UInt64`, `URL String`. Определяет имена и типы столбцов. Файлы, указанные в file, будут разобраны форматом, указанным в format, с использованием типов данных, указанных в types или structure. Таблица будет загружена на сервер, и доступна там в качестве временной таблицы с именем name. @@ -43,7 +43,7 @@ $ cat /etc/passwd | sed 's/:/\t/g' | clickhouse-client --query="SELECT shell, co /bin/sync 1 ``` -При использовании HTTP интерфейса, внешние данные передаются в формате multipart/form-data. Каждая таблица передаётся отдельным файлом. Имя таблицы берётся из имени файла. В query\_string передаются параметры name\_format, name\_types, name\_structure, где name - имя таблицы, которой соответствуют эти параметры. Смысл параметров такой же, как при использовании клиента командной строки. +При использовании HTTP интерфейса, внешние данные передаются в формате multipart/form-data. Каждая таблица передаётся отдельным файлом. Имя таблицы берётся из имени файла. В query_string передаются параметры name_format, name_types, name_structure, где name - имя таблицы, которой соответствуют эти параметры. Смысл параметров такой же, как при использовании клиента командной строки. Пример: diff --git a/docs/ru/engines/table-engines/special/join.md b/docs/ru/engines/table-engines/special/join.md index 2caf1187d50..aa734f5ca55 100644 --- a/docs/ru/engines/table-engines/special/join.md +++ b/docs/ru/engines/table-engines/special/join.md @@ -85,15 +85,15 @@ SELECT joinGet('id_val_join', 'val', toUInt32(1)) При создании таблицы, применяются следующие параметры : -- [join\_use\_nulls](../../../operations/settings/settings.md#join_use_nulls) -- [max\_rows\_in\_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) -- [max\_bytes\_in\_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) -- [join\_overflow\_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) -- [join\_any\_take\_last\_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) +- [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) +- [max_rows_in_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) +- [max_bytes_in_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) +- [join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) +- [join_any_take_last_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) Таблицы с движком `Join` нельзя использовать в операциях `GLOBAL JOIN`. -Движок `Join` позволяет использовать параметр [join\_use\_nulls](../../../operations/settings/settings.md#join_use_nulls) в запросе `CREATE TABLE`, который также можно использовать в запросе [SELECT](../../../engines/table-engines/special/join.md). Если у вас разные настройки `join_use_nulls`, вы можете получить сообщение об ошибке при объединении таблиц. Это зависит от типа соединения. Когда вы используете функцию [joinGet](../../../engines/table-engines/special/join.md#joinget), вам необходимо использовать один и тот же параметр `join_use_nulls` в запросах `CRATE TABLE` и `SELECT`. +Движок `Join` позволяет использовать параметр [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) в запросе `CREATE TABLE`, который также можно использовать в запросе [SELECT](../../../engines/table-engines/special/join.md). Если у вас разные настройки `join_use_nulls`, вы можете получить сообщение об ошибке при объединении таблиц. Это зависит от типа соединения. Когда вы используете функцию [joinGet](../../../engines/table-engines/special/join.md#joinget), вам необходимо использовать один и тот же параметр `join_use_nulls` в запросах `CRATE TABLE` и `SELECT`. ## Хранение данных {#khranenie-dannykh} diff --git a/docs/ru/engines/table-engines/special/url.md b/docs/ru/engines/table-engines/special/url.md index 5f4696286d7..69c1567a1c2 100644 --- a/docs/ru/engines/table-engines/special/url.md +++ b/docs/ru/engines/table-engines/special/url.md @@ -17,7 +17,7 @@ соответственно. Для обработки `POST`-запросов удаленный сервер должен поддерживать [Chunked transfer encoding](https://ru.wikipedia.org/wiki/Chunked_transfer_encoding). -Максимальное количество переходов по редиректам при выполнении HTTP-запроса методом GET можно ограничить с помощью настройки [max\_http\_get\_redirects](../../../operations/settings/settings.md#setting-max_http_get_redirects). +Максимальное количество переходов по редиректам при выполнении HTTP-запроса методом GET можно ограничить с помощью настройки [max_http_get_redirects](../../../operations/settings/settings.md#setting-max_http_get_redirects). **Пример:** diff --git a/docs/ru/faq/general.md b/docs/ru/faq/general.md index 4862a069b95..470b8dabbf1 100644 --- a/docs/ru/faq/general.md +++ b/docs/ru/faq/general.md @@ -13,7 +13,7 @@ ## Что делать, если у меня проблема с кодировками при использовании Oracle через ODBC? {#oracle-odbc-encodings} -Если вы используете Oracle через драйвер ODBC в качестве источника внешних словарей, необходимо задать правильное значение для переменной окружения `NLS_LANG` в `/etc/default/clickhouse`. Подробнее читайте в [Oracle NLS\_LANG FAQ](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html). +Если вы используете Oracle через драйвер ODBC в качестве источника внешних словарей, необходимо задать правильное значение для переменной окружения `NLS_LANG` в `/etc/default/clickhouse`. Подробнее читайте в [Oracle NLS_LANG FAQ](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html). **Пример** diff --git a/docs/ru/getting-started/example-datasets/amplab-benchmark.md b/docs/ru/getting-started/example-datasets/amplab-benchmark.md index 2516dfefba7..b503fc909f3 100644 --- a/docs/ru/getting-started/example-datasets/amplab-benchmark.md +++ b/docs/ru/getting-started/example-datasets/amplab-benchmark.md @@ -3,7 +3,7 @@ См. https://amplab.cs.berkeley.edu/benchmark/ Зарегистрируйте бесплатную учетную запись на https://aws.amazon.com - понадобится кредитная карта, email и номер телефона -Получите новый ключ доступа на https://console.aws.amazon.com/iam/home?nc2=h\_m\_sc\#security\_credential +Получите новый ключ доступа на https://console.aws.amazon.com/iam/home?nc2=h_m_sc#security_credential Выполните следующее в консоли: diff --git a/docs/ru/getting-started/example-datasets/nyc-taxi.md b/docs/ru/getting-started/example-datasets/nyc-taxi.md index 1873695f679..64202b2e765 100644 --- a/docs/ru/getting-started/example-datasets/nyc-taxi.md +++ b/docs/ru/getting-started/example-datasets/nyc-taxi.md @@ -190,7 +190,7 @@ real 75m56.214s (Импорт данных напрямую из Postgres также возможен с использованием `COPY ... TO PROGRAM`.) -К сожалению, все поля, связанные с погодой (precipitation…average\_wind\_speed) заполнены NULL. Из-за этого мы исключим их из финального набора данных. +К сожалению, все поля, связанные с погодой (precipitation…average_wind_speed) заполнены NULL. Из-за этого мы исключим их из финального набора данных. Для начала мы создадим таблицу на одном сервере. Позже мы сделаем таблицу распределенной. diff --git a/docs/ru/getting-started/playground.md b/docs/ru/getting-started/playground.md index 5cb0612dfc7..e3bae5c9756 100644 --- a/docs/ru/getting-started/playground.md +++ b/docs/ru/getting-started/playground.md @@ -31,10 +31,10 @@ ClickHouse Playground дает возможность поработать с [ - запрещены INSERT запросы Также установлены следующие опции: -- [max\_result\_bytes=10485760](../operations/settings/query_complexity/#max-result-bytes) -- [max\_result\_rows=2000](../operations/settings/query_complexity/#setting-max_result_rows) -- [result\_overflow\_mode=break](../operations/settings/query_complexity/#result-overflow-mode) -- [max\_execution\_time=60000](../operations/settings/query_complexity/#max-execution-time) +- [max_result_bytes=10485760](../operations/settings/query_complexity/#max-result-bytes) +- [max_result_rows=2000](../operations/settings/query_complexity/#setting-max_result_rows) +- [result_overflow_mode=break](../operations/settings/query_complexity/#result-overflow-mode) +- [max_execution_time=60000](../operations/settings/query_complexity/#max-execution-time) ## Примеры {#examples} diff --git a/docs/ru/index.md b/docs/ru/index.md index a7aa80b91d2..7c129f9d113 100644 --- a/docs/ru/index.md +++ b/docs/ru/index.md @@ -11,10 +11,10 @@ ClickHouse - столбцовая система управления базам | Строка | WatchID | JavaEnable | Title | GoodEvent | EventTime | |--------|-------------|------------|--------------------|-----------|---------------------| -| \#0 | 89354350662 | 1 | Investor Relations | 1 | 2016-05-18 05:19:20 | -| \#1 | 90329509958 | 0 | Contact us | 1 | 2016-05-18 08:10:20 | -| \#2 | 89953706054 | 1 | Mission | 1 | 2016-05-18 07:38:00 | -| \#N | … | … | … | … | … | +| #0 | 89354350662 | 1 | Investor Relations | 1 | 2016-05-18 05:19:20 | +| #1 | 90329509958 | 0 | Contact us | 1 | 2016-05-18 08:10:20 | +| #2 | 89953706054 | 1 | Mission | 1 | 2016-05-18 07:38:00 | +| #N | … | … | … | … | … | То есть, значения, относящиеся к одной строке, физически хранятся рядом. @@ -23,7 +23,7 @@ ClickHouse - столбцовая система управления базам В столбцовых СУБД, данные хранятся в таком порядке: -| Строка: | \#0 | \#1 | \#2 | \#N | +| Строка: | #0 | #1 | #2 | #N | |-------------|---------------------|---------------------|---------------------|-----| | WatchID: | 89354350662 | 90329509958 | 89953706054 | … | | JavaEnable: | 1 | 0 | 1 | … | diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index dd68f7eb646..441cb8c5c91 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -324,19 +324,19 @@ x=1 y=\N Формат Comma Separated Values ([RFC](https://tools.ietf.org/html/rfc4180)). -При форматировании, строки выводятся в двойных кавычках. Двойная кавычка внутри строки выводится как две двойные кавычки подряд. Других правил экранирования нет. Даты и даты-с-временем выводятся в двойных кавычках. Числа выводятся без кавычек. Значения разделяются символом-разделителем, по умолчанию — `,`. Символ-разделитель определяется настройкой [format\_csv\_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter). Строки разделяются unix переводом строки (LF). Массивы сериализуются в CSV следующим образом: сначала массив сериализуется в строку, как в формате TabSeparated, а затем полученная строка выводится в CSV в двойных кавычках. Кортежи в формате CSV сериализуются, как отдельные столбцы (то есть, теряется их вложенность в кортеж). +При форматировании, строки выводятся в двойных кавычках. Двойная кавычка внутри строки выводится как две двойные кавычки подряд. Других правил экранирования нет. Даты и даты-с-временем выводятся в двойных кавычках. Числа выводятся без кавычек. Значения разделяются символом-разделителем, по умолчанию — `,`. Символ-разделитель определяется настройкой [format_csv_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter). Строки разделяются unix переводом строки (LF). Массивы сериализуются в CSV следующим образом: сначала массив сериализуется в строку, как в формате TabSeparated, а затем полученная строка выводится в CSV в двойных кавычках. Кортежи в формате CSV сериализуются, как отдельные столбцы (то есть, теряется их вложенность в кортеж). ``` bash $ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV" < data.csv ``` -\*По умолчанию — `,`. См. настройку [format\_csv\_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter) для дополнительной информации. +\*По умолчанию — `,`. См. настройку [format_csv_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter) для дополнительной информации. При парсинге, все значения могут парситься как в кавычках, так и без кавычек. Поддерживаются как двойные, так и одинарные кавычки. Строки также могут быть без кавычек. В этом случае они парсятся до символа-разделителя или перевода строки (CR или LF). В нарушение RFC, в случае парсинга строк не в кавычках, начальные и конечные пробелы и табы игнорируются. В качестве перевода строки, поддерживаются как Unix (LF), так и Windows (CR LF) и Mac OS Classic (LF CR) варианты. -`NULL` форматируется в виде `\N` или `NULL` или пустой неэкранированной строки (см. настройки [input\_format\_csv\_unquoted\_null\_literal\_as\_null](../operations/settings/settings.md#settings-input_format_csv_unquoted_null_literal_as_null) и [input\_format\_defaults\_for\_omitted\_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields)). +`NULL` форматируется в виде `\N` или `NULL` или пустой неэкранированной строки (см. настройки [input_format_csv_unquoted_null_literal_as_null](../operations/settings/settings.md#settings-input_format_csv_unquoted_null_literal_as_null) и [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields)). -Если установлена настройка [input\_format\_defaults\_for\_omitted\_fields = 1](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) и тип столбца не `Nullable(T)`, то пустые значения без кавычек заменяются значениями по умолчанию для типа данных столбца. +Если установлена настройка [input_format_defaults_for_omitted_fields = 1](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) и тип столбца не `Nullable(T)`, то пустые значения без кавычек заменяются значениями по умолчанию для типа данных столбца. Формат CSV поддерживает вывод totals и extremes аналогично `TabSeparated`. @@ -421,12 +421,12 @@ SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase WITH TOTA } ``` -JSON совместим с JavaScript. Для этого, дополнительно экранируются некоторые символы: символ прямого слеша `/` экранируется в виде `\/`; альтернативные переводы строк `U+2028`, `U+2029`, на которых ломаются некоторые браузеры, экранируются в виде `\uXXXX`-последовательностей. Экранируются ASCII control characters: backspace, form feed, line feed, carriage return, horizontal tab в виде `\b`, `\f`, `\n`, `\r`, `\t` соответственно, а также остальные байты из диапазона 00-1F с помощью `\uXXXX`-последовательностей. Невалидные UTF-8 последовательности заменяются на replacement character � и, таким образом, выводимый текст будет состоять из валидных UTF-8 последовательностей. Числа типа UInt64 и Int64, для совместимости с JavaScript, по умолчанию выводятся в двойных кавычках. Чтобы они выводились без кавычек, можно установить конфигурационный параметр [output\_format\_json\_quote\_64bit\_integers](../operations/settings/settings.md#session_settings-output_format_json_quote_64bit_integers) равным 0. +JSON совместим с JavaScript. Для этого, дополнительно экранируются некоторые символы: символ прямого слеша `/` экранируется в виде `\/`; альтернативные переводы строк `U+2028`, `U+2029`, на которых ломаются некоторые браузеры, экранируются в виде `\uXXXX`-последовательностей. Экранируются ASCII control characters: backspace, form feed, line feed, carriage return, horizontal tab в виде `\b`, `\f`, `\n`, `\r`, `\t` соответственно, а также остальные байты из диапазона 00-1F с помощью `\uXXXX`-последовательностей. Невалидные UTF-8 последовательности заменяются на replacement character � и, таким образом, выводимый текст будет состоять из валидных UTF-8 последовательностей. Числа типа UInt64 и Int64, для совместимости с JavaScript, по умолчанию выводятся в двойных кавычках. Чтобы они выводились без кавычек, можно установить конфигурационный параметр [output_format_json_quote_64bit_integers](../operations/settings/settings.md#session_settings-output_format_json_quote_64bit_integers) равным 0. `rows` - общее количество выведенных строчек. `rows_before_limit_at_least` - не менее скольких строчек получилось бы, если бы не было LIMIT-а. Выводится только если запрос содержит LIMIT. -В случае, если запрос содержит GROUP BY, rows\_before\_limit\_at\_least - точное число строк, которое получилось бы, если бы не было LIMIT-а. +В случае, если запрос содержит GROUP BY, rows_before_limit_at_least - точное число строк, которое получилось бы, если бы не было LIMIT-а. `totals` - тотальные значения (при использовании WITH TOTALS). @@ -434,7 +434,7 @@ JSON совместим с JavaScript. Для этого, дополнитель Этот формат подходит только для вывода результата выполнения запроса, но не для парсинга (приёма данных для вставки в таблицу). -ClickHouse поддерживает [NULL](../sql-reference/syntax.md), который при выводе JSON будет отображен как `null`. Чтобы включить отображение в результате значений `+nan`, `-nan`, `+inf`, `-inf`, установите параметр [output\_format\_json\_quote\_denormals](../operations/settings/settings.md#settings-output_format_json_quote_denormals) равным 1. +ClickHouse поддерживает [NULL](../sql-reference/syntax.md), который при выводе JSON будет отображен как `null`. Чтобы включить отображение в результате значений `+nan`, `-nan`, `+inf`, `-inf`, установите параметр [output_format_json_quote_denormals](../operations/settings/settings.md#settings-output_format_json_quote_denormals) равным 1. Смотрите также формат [JSONEachRow](#jsoneachrow) . @@ -511,7 +511,7 @@ ClickHouse игнорирует пробелы между элементами ClickHouse заменяет опущенные значения значениями по умолчанию для соответствующих [data types](../sql-reference/data-types/index.md). -Если указано `DEFAULT expr`, то ClickHouse использует различные правила подстановки в зависимости от настройки [input\_format\_defaults\_for\_omitted\_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields). +Если указано `DEFAULT expr`, то ClickHouse использует различные правила подстановки в зависимости от настройки [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields). Рассмотрим следующую таблицу: @@ -554,7 +554,7 @@ CREATE TABLE IF NOT EXISTS example_table ### Использование вложенных структур {#jsoneachrow-nested} -Если у вас есть таблица со столбцами типа [Nested](../sql-reference/data-types/nested-data-structures/nested.md), то в неё можно вставить данные из JSON-документа с такой же структурой. Функциональность включается настройкой [input\_format\_import\_nested\_json](../operations/settings/settings.md#settings-input_format_import_nested_json). +Если у вас есть таблица со столбцами типа [Nested](../sql-reference/data-types/nested-data-structures/nested.md), то в неё можно вставить данные из JSON-документа с такой же структурой. Функциональность включается настройкой [input_format_import_nested_json](../operations/settings/settings.md#settings-input_format_import_nested_json). Например, рассмотрим следующую таблицу: @@ -568,7 +568,7 @@ CREATE TABLE json_each_row_nested (n Nested (s String, i Int32) ) ENGINE = Memor INSERT INTO json_each_row_nested FORMAT JSONEachRow {"n.s": ["abc", "def"], "n.i": [1, 23]} ``` -Чтобы вставить данные как иерархический объект JSON, установите [input\_format\_import\_nested\_json=1](../operations/settings/settings.md#settings-input_format_import_nested_json). +Чтобы вставить данные как иерархический объект JSON, установите [input_format_import_nested_json=1](../operations/settings/settings.md#settings-input_format_import_nested_json). ``` json { @@ -963,7 +963,7 @@ ClickHouse пишет и читает сообщения `Protocol Buffers` в Схемы кэшируются после разрешения. -URL-адрес реестра схем настраивается с помощью [format\_avro\_schema\_registry\_url](../operations/settings/settings.md#format_avro_schema_registry_url). +URL-адрес реестра схем настраивается с помощью [format_avro_schema_registry_url](../operations/settings/settings.md#format_avro_schema_registry_url). ### Соответствие типов данных {#sootvetstvie-tipov-dannykh-0} @@ -1112,6 +1112,6 @@ $ cat filename.orc | clickhouse-client --query="INSERT INTO some_table FORMAT OR относительно текущей директории на клиенте. Если клиент используется в [batch режиме](../interfaces/cli.md#cli_usage), то в записи схемы допускается только относительный путь, из соображений безопасности. Если для ввода/вывода данных используется [HTTP-интерфейс](../interfaces/http.md), то файл со схемой должен располагаться на сервере в каталоге, -указанном в параметре [format\_schema\_path](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-format_schema_path) конфигурации сервера. +указанном в параметре [format_schema_path](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-format_schema_path) конфигурации сервера. [Оригинальная статья](https://clickhouse.tech/docs/ru/interfaces/formats/) diff --git a/docs/ru/interfaces/http.md b/docs/ru/interfaces/http.md index b1cc4c79b25..ffe4b2e5276 100644 --- a/docs/ru/interfaces/http.md +++ b/docs/ru/interfaces/http.md @@ -3,7 +3,7 @@ HTTP интерфейс позволяет использовать ClickHouse на любой платформе, из любого языка программирования. У нас он используется для работы из Java и Perl, а также из shell-скриптов. В других отделах, HTTP интерфейс используется из Perl, Python и Go. HTTP интерфейс более ограничен по сравнению с родным интерфейсом, но является более совместимым. По умолчанию, clickhouse-server слушает HTTP на порту 8123 (это можно изменить в конфиге). -Если запросить GET / без параметров, то вернётся строка заданная с помощью настройки [http\_server\_default\_response](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-http_server_default_response). Значение по умолчанию «Ok.» (с переводом строки на конце). +Если запросить GET / без параметров, то вернётся строка заданная с помощью настройки [http_server_default_response](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-http_server_default_response). Значение по умолчанию «Ok.» (с переводом строки на конце). ``` bash $ curl 'http://localhost:8123/' @@ -141,12 +141,12 @@ $ echo 'DROP TABLE t' | curl 'http://localhost:8123/' --data-binary @- Для запросов, которые не возвращают таблицу с данными, в случае успеха, выдаётся пустое тело ответа. -Вы можете использовать внутренний формат сжатия Clickhouse при передаче данных. Формат сжатых данных нестандартный, и вам придётся использовать для работы с ним специальную программу `clickhouse-compressor` (устанавливается вместе с пакетом `clickhouse-client`). Для повышения эффективности вставки данных можно отключить проверку контрольной суммы на стороне сервера с помощью настройки[http\_native\_compression\_disable\_checksumming\_on\_decompress](../operations/settings/settings.md#settings-http_native_compression_disable_checksumming_on_decompress). +Вы можете использовать внутренний формат сжатия Clickhouse при передаче данных. Формат сжатых данных нестандартный, и вам придётся использовать для работы с ним специальную программу `clickhouse-compressor` (устанавливается вместе с пакетом `clickhouse-client`). Для повышения эффективности вставки данных можно отключить проверку контрольной суммы на стороне сервера с помощью настройки[http_native_compression_disable_checksumming_on_decompress](../operations/settings/settings.md#settings-http_native_compression_disable_checksumming_on_decompress). Если вы указали `compress = 1` в URL, то сервер сжимает данные, которые он отправляет. Если вы указали `decompress = 1` в URL, сервер распаковывает те данные, которые вы передаёте методом `POST`. -Также, можно использовать [HTTP compression](https://en.wikipedia.org/wiki/HTTP_compression). Для отправки сжатого запроса `POST`, добавьте заголовок `Content-Encoding: compression_method`. Чтобы ClickHouse сжимал ответ, добавьте заголовок `Accept-Encoding: compression_method`. ClickHouse поддерживает следующие [методы сжатия](https://en.wikipedia.org/wiki/HTTP_compression#Content-Encoding_tokens): `gzip`, `br`, and `deflate`. Чтобы включить HTTP compression, используйте настройку ClickHouse [enable\_http\_compression](../operations/settings/settings.md#settings-enable_http_compression). Уровень сжатия данных для всех методов сжатия можно настроить с помощью настройки [http\_zlib\_compression\_level](#settings-http_zlib_compression_level). +Также, можно использовать [HTTP compression](https://en.wikipedia.org/wiki/HTTP_compression). Для отправки сжатого запроса `POST`, добавьте заголовок `Content-Encoding: compression_method`. Чтобы ClickHouse сжимал ответ, добавьте заголовок `Accept-Encoding: compression_method`. ClickHouse поддерживает следующие [методы сжатия](https://en.wikipedia.org/wiki/HTTP_compression#Content-Encoding_tokens): `gzip`, `br`, and `deflate`. Чтобы включить HTTP compression, используйте настройку ClickHouse [enable_http_compression](../operations/settings/settings.md#settings-enable_http_compression). Уровень сжатия данных для всех методов сжатия можно настроить с помощью настройки [http_zlib_compression_level](#settings-http_zlib_compression_level). Это может быть использовано для уменьшения трафика по сети при передаче большого количества данных, а также для создания сразу сжатых дампов. @@ -208,7 +208,7 @@ $ echo 'SELECT 1' | curl -H 'X-ClickHouse-User: user' -H 'X-ClickHouse-Key: pass ``` Если пользователь не задан,то используется `default`. Если пароль не задан, то используется пустой пароль. -Также в параметрах URL вы можете указать любые настройки, которые будут использованы для обработки одного запроса, или целые профили настроек. Пример:http://localhost:8123/?profile=web&max\_rows\_to\_read=1000000000&query=SELECT+1 +Также в параметрах URL вы можете указать любые настройки, которые будут использованы для обработки одного запроса, или целые профили настроек. Пример:http://localhost:8123/?profile=web&max_rows_to_read=1000000000&query=SELECT+1 Подробнее смотрите в разделе [Настройки](../operations/settings/index.md). @@ -230,7 +230,7 @@ $ echo 'SELECT number FROM system.numbers LIMIT 10' | curl 'http://localhost:812 Аналогично можно использовать ClickHouse-сессии в HTTP-протоколе. Для этого необходимо добавить к запросу GET параметр `session_id`. В качестве идентификатора сессии можно использовать произвольную строку. По умолчанию через 60 секунд бездействия сессия будет прервана. Можно изменить этот таймаут, изменяя настройку `default_session_timeout` в конфигурации сервера, или добавив к запросу GET параметр `session_timeout`. Статус сессии можно проверить с помощью параметра `session_check=1`. В рамках одной сессии одновременно может исполняться только один запрос. -Прогресс выполнения запроса можно отслеживать с помощью заголовков ответа `X-ClickHouse-Progress`. Для этого включите [send\_progress\_in\_http\_headers](../operations/settings/settings.md#settings-send_progress_in_http_headers). Пример последовательности заголовков: +Прогресс выполнения запроса можно отслеживать с помощью заголовков ответа `X-ClickHouse-Progress`. Для этого включите [send_progress_in_http_headers](../operations/settings/settings.md#settings-send_progress_in_http_headers). Пример последовательности заголовков: ``` text X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128"} @@ -247,9 +247,9 @@ X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_ro - `written_bytes` — объём записанных данных в байтах. Запущенные запросы не останавливаются автоматически при разрыве HTTP соединения. Парсинг и форматирование данных производится на стороне сервера и использование сети может быть неэффективным. -Может быть передан необязательный параметр query\_id - идентификатор запроса, произвольная строка. Подробнее смотрите раздел «Настройки, replace\_running\_query». +Может быть передан необязательный параметр query_id - идентификатор запроса, произвольная строка. Подробнее смотрите раздел «Настройки, replace_running_query». -Может быть передан необязательный параметр quota\_key - ключ квоты, произвольная строка. Подробнее смотрите раздел «Квоты». +Может быть передан необязательный параметр quota_key - ключ квоты, произвольная строка. Подробнее смотрите раздел «Квоты». HTTP интерфейс позволяет передать внешние данные (внешние временные таблицы) для использования запроса. Подробнее смотрите раздел «Внешние данные для обработки запроса» diff --git a/docs/ru/interfaces/mysql.md b/docs/ru/interfaces/mysql.md index 493df82be53..79368aa1ba2 100644 --- a/docs/ru/interfaces/mysql.md +++ b/docs/ru/interfaces/mysql.md @@ -1,6 +1,6 @@ # MySQL-интерфейс {#mysql-interface} -ClickHouse поддерживает взаимодействие по протоколу MySQL. Данная функция включается настройкой [mysql\_port](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-mysql_port) в конфигурационном файле: +ClickHouse поддерживает взаимодействие по протоколу MySQL. Данная функция включается настройкой [mysql_port](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-mysql_port) в конфигурационном файле: ``` xml 9004 diff --git a/docs/ru/interfaces/third-party/client-libraries.md b/docs/ru/interfaces/third-party/client-libraries.md index 52ff782b790..66a2252b23a 100644 --- a/docs/ru/interfaces/third-party/client-libraries.md +++ b/docs/ru/interfaces/third-party/client-libraries.md @@ -4,7 +4,7 @@ Яндекс не поддерживает перечисленные ниже библиотеки и не проводит тщательного тестирования для проверки их качества. - Python: - - [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm) + - [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm) - [clickhouse-driver](https://github.com/mymarilyn/clickhouse-driver) - [clickhouse-client](https://github.com/yurial/clickhouse-client) - [aiochclient](https://github.com/maximdanilchenko/aiochclient) @@ -39,7 +39,7 @@ - [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client) - Kotlin - [AORM](https://github.com/TanVD/AORM) -- C\# +- C# - [Octonica.ClickHouseClient](https://github.com/Octonica/ClickHouseClient) - [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net) - [ClickHouse.Client](https://github.com/DarkWanderer/ClickHouse.Client) diff --git a/docs/ru/interfaces/third-party/integrations.md b/docs/ru/interfaces/third-party/integrations.md index 106cd6c859b..527f65343db 100644 --- a/docs/ru/interfaces/third-party/integrations.md +++ b/docs/ru/interfaces/third-party/integrations.md @@ -12,15 +12,15 @@ - [clickhouse-mysql-data-reader](https://github.com/Altinity/clickhouse-mysql-data-reader) - [horgh-replicator](https://github.com/larsnovikov/horgh-replicator) - [PostgreSQL](https://www.postgresql.org) - - [clickhousedb\_fdw](https://github.com/Percona-Lab/clickhousedb_fdw) - - [infi.clickhouse\_fdw](https://github.com/Infinidat/infi.clickhouse_fdw) (использует [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) + - [clickhousedb_fdw](https://github.com/Percona-Lab/clickhousedb_fdw) + - [infi.clickhouse_fdw](https://github.com/Infinidat/infi.clickhouse_fdw) (использует [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - [pg2ch](https://github.com/mkabilov/pg2ch) - - [clickhouse\_fdw](https://github.com/adjust/clickhouse_fdw) + - [clickhouse_fdw](https://github.com/adjust/clickhouse_fdw) - [MSSQL](https://en.wikipedia.org/wiki/Microsoft_SQL_Server) - [ClickHouseMightrator](https://github.com/zlzforever/ClickHouseMigrator) - Очереди сообщений - [Kafka](https://kafka.apache.org) - - [clickhouse\_sinker](https://github.com/housepower/clickhouse_sinker) (использует [Go client](https://github.com/ClickHouse/clickhouse-go/)) + - [clickhouse_sinker](https://github.com/housepower/clickhouse_sinker) (использует [Go client](https://github.com/ClickHouse/clickhouse-go/)) - [stream-loader-clickhouse](https://github.com/adform/stream-loader) - Потоковая обработка - [Flink](https://flink.apache.org) @@ -44,12 +44,12 @@ - [Grafana](https://grafana.com/) - [clickhouse-grafana](https://github.com/Vertamedia/clickhouse-grafana) - [Prometheus](https://prometheus.io/) - - [clickhouse\_exporter](https://github.com/f1yegor/clickhouse_exporter) + - [clickhouse_exporter](https://github.com/f1yegor/clickhouse_exporter) - [PromHouse](https://github.com/Percona-Lab/PromHouse) - - [clickhouse\_exporter](https://github.com/hot-wifi/clickhouse_exporter) (использует [Go client](https://github.com/kshvakov/clickhouse/)) + - [clickhouse_exporter](https://github.com/hot-wifi/clickhouse_exporter) (использует [Go client](https://github.com/kshvakov/clickhouse/)) - [Nagios](https://www.nagios.org/) - - [check\_clickhouse](https://github.com/exogroup/check_clickhouse/) - - [check\_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py) + - [check_clickhouse](https://github.com/exogroup/check_clickhouse/) + - [check_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py) - [Zabbix](https://www.zabbix.com) - [clickhouse-zabbix-template](https://github.com/Altinity/clickhouse-zabbix-template) - [Sematext](https://sematext.com/) @@ -69,7 +69,7 @@ - Python - [SQLAlchemy](https://www.sqlalchemy.org) - - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (использует [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) + - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (использует [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - [pandas](https://pandas.pydata.org) - [pandahouse](https://github.com/kszucs/pandahouse) - PHP @@ -84,7 +84,7 @@ - Scala - [Akka](https://akka.io) - [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client) -- C\# +- C# - [ADO.NET](https://docs.microsoft.com/en-us/dotnet/framework/data/adonet/ado-net-overview) - [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net) - [ClickHouse.Client](https://github.com/DarkWanderer/ClickHouse.Client) @@ -92,7 +92,7 @@ - [ClickHouse.Net.Migrations](https://github.com/ilyabreev/ClickHouse.Net.Migrations) - Elixir - [Ecto](https://github.com/elixir-ecto/ecto) - - [clickhouse\_ecto](https://github.com/appodeal/clickhouse_ecto) + - [clickhouse_ecto](https://github.com/appodeal/clickhouse_ecto) - Ruby - [Ruby on Rails](https://rubyonrails.org/) - [activecube](https://github.com/bitquery/activecube) diff --git a/docs/ru/operations/configuration-files.md b/docs/ru/operations/configuration-files.md index b8ab21c5f85..df56ab9ac7d 100644 --- a/docs/ru/operations/configuration-files.md +++ b/docs/ru/operations/configuration-files.md @@ -12,7 +12,7 @@ Если указано `remove` - удалить элемент. -Также в конфиге могут быть указаны «подстановки». Если у элемента присутствует атрибут `incl`, то в качестве значения будет использована соответствующая подстановка из файла. По умолчанию, путь к файлу с подстановками - `/etc/metrika.xml`. Он может быть изменён в конфигурации сервера в элементе [include\_from](server-configuration-parameters/settings.md#server_configuration_parameters-include_from). Значения подстановок указываются в элементах `/yandex/имя_подстановки` этого файла. Если подстановка, заданная в `incl` отсутствует, то в лог попадает соответствующая запись. Чтобы ClickHouse не писал в лог об отсутствии подстановки, необходимо указать атрибут `optional="true"` (например, настройка [macros](server-configuration-parameters/settings.md)). +Также в конфиге могут быть указаны «подстановки». Если у элемента присутствует атрибут `incl`, то в качестве значения будет использована соответствующая подстановка из файла. По умолчанию, путь к файлу с подстановками - `/etc/metrika.xml`. Он может быть изменён в конфигурации сервера в элементе [include_from](server-configuration-parameters/settings.md#server_configuration_parameters-include_from). Значения подстановок указываются в элементах `/yandex/имя_подстановки` этого файла. Если подстановка, заданная в `incl` отсутствует, то в лог попадает соответствующая запись. Чтобы ClickHouse не писал в лог об отсутствии подстановки, необходимо указать атрибут `optional="true"` (например, настройка [macros](server-configuration-parameters/settings.md)). Подстановки могут также выполняться из ZooKeeper. Для этого укажите у элемента атрибут `from_zk = "/path/to/node"`. Значение элемента заменится на содержимое узла `/path/to/node` в ZooKeeper. В ZooKeeper-узел также можно положить целое XML-поддерево, оно будет целиком вставлено в исходный элемент. diff --git a/docs/ru/operations/monitoring.md b/docs/ru/operations/monitoring.md index 2ca14c2af88..a1013f5eae5 100644 --- a/docs/ru/operations/monitoring.md +++ b/docs/ru/operations/monitoring.md @@ -36,6 +36,6 @@ ClickHouse собирает: Также, можно отслеживать доступность сервера через HTTP API. Отправьте `HTTP GET` к ресурсу `/ping`. Если сервер доступен, он отвечает `200 OK`. -Для мониторинга серверов в кластерной конфигурации необходимо установить параметр [max\_replica\_delay\_for\_distributed\_queries](settings/settings.md#settings-max_replica_delay_for_distributed_queries) и использовать HTTP ресурс `/replicas_status`. Если реплика доступна и не отстаёт от других реплик, то запрос к `/replicas_status` возвращает `200 OK`. Если реплика отстаёт, то запрос возвращает `503 HTTP_SERVICE_UNAVAILABLE`, включая информацию о размере отставания. +Для мониторинга серверов в кластерной конфигурации необходимо установить параметр [max_replica_delay_for_distributed_queries](settings/settings.md#settings-max_replica_delay_for_distributed_queries) и использовать HTTP ресурс `/replicas_status`. Если реплика доступна и не отстаёт от других реплик, то запрос к `/replicas_status` возвращает `200 OK`. Если реплика отстаёт, то запрос возвращает `503 HTTP_SERVICE_UNAVAILABLE`, включая информацию о размере отставания. [Оригинальная статья](https://clickhouse.tech/docs/ru/operations/system_tables/monitoring) diff --git a/docs/ru/operations/optimizing-performance/sampling-query-profiler.md b/docs/ru/operations/optimizing-performance/sampling-query-profiler.md index 3ca1cfe6acb..0c075180530 100644 --- a/docs/ru/operations/optimizing-performance/sampling-query-profiler.md +++ b/docs/ru/operations/optimizing-performance/sampling-query-profiler.md @@ -9,11 +9,11 @@ ClickHouse runs sampling profiler that allows analyzing query execution. Using p To use profiler: -- Setup the [trace\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) section of the server configuration. +- Setup the [trace_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) section of the server configuration. This section configures the [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) system table containing the results of the profiler functioning. It is configured by default. Remember that data in this table is valid only for a running server. After the server restart, ClickHouse doesn’t clean up the table and all the stored virtual memory address may become invalid. -- Setup the [query\_profiler\_cpu\_time\_period\_ns](../../operations/settings/settings.md#query_profiler_cpu_time_period_ns) or [query\_profiler\_real\_time\_period\_ns](../../operations/settings/settings.md#query_profiler_real_time_period_ns) settings. Both settings can be used simultaneously. +- Setup the [query_profiler_cpu_time_period_ns](../../operations/settings/settings.md#query_profiler_cpu_time_period_ns) or [query_profiler_real_time_period_ns](../../operations/settings/settings.md#query_profiler_real_time_period_ns) settings. Both settings can be used simultaneously. These settings allow you to configure profiler timers. As these are the session settings, you can get different sampling frequency for the whole server, individual users or user profiles, for your interactive session, and for each individual query. @@ -23,7 +23,7 @@ To analyze the `trace_log` system table: - Install the `clickhouse-common-static-dbg` package. See [Install from DEB Packages](../../getting-started/install.md#install-from-deb-packages). -- Allow introspection functions by the [allow\_introspection\_functions](../../operations/settings/settings.md#settings-allow_introspection_functions) setting. +- Allow introspection functions by the [allow_introspection_functions](../../operations/settings/settings.md#settings-allow_introspection_functions) setting. For security reasons, introspection functions are disabled by default. diff --git a/docs/ru/operations/requirements.md b/docs/ru/operations/requirements.md index 459ccf4de25..e1ecafd9f3b 100644 --- a/docs/ru/operations/requirements.md +++ b/docs/ru/operations/requirements.md @@ -2,7 +2,7 @@ ## Процессор {#protsessor} -В случае установки из готовых deb-пакетов используйте процессоры с архитектурой x86\_64 и поддержкой инструкций SSE 4.2. Для запуска ClickHouse на процессорах без поддержки SSE 4.2 или на процессорах с архитектурой AArch64 и PowerPC64LE необходимо собирать ClickHouse из исходников. +В случае установки из готовых deb-пакетов используйте процессоры с архитектурой x86_64 и поддержкой инструкций SSE 4.2. Для запуска ClickHouse на процессорах без поддержки SSE 4.2 или на процессорах с архитектурой AArch64 и PowerPC64LE необходимо собирать ClickHouse из исходников. ClickHouse реализует параллельную обработку данных и использует все доступные аппаратные ресурсы. При выборе процессора учитывайте, что ClickHouse работает более эффективно в конфигурациях с большим количеством ядер, но с более низкой тактовой частотой, чем в конфигурациях с меньшим количеством ядер и более высокой тактовой частотой. Например, 16 ядер с 2600 MHz предпочтительнее, чем 8 ядер с 3600 MHz. diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 0abb568ffc7..4443993d7e8 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -1,6 +1,6 @@ # Конфигурационные параметры сервера {#server-configuration-parameters-reference} -## builtin\_dictionaries\_reload\_interval {#builtin-dictionaries-reload-interval} +## builtin_dictionaries_reload_interval {#builtin-dictionaries-reload-interval} Интервал (в секундах) перезагрузки встроенных словарей. @@ -61,7 +61,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part ``` -## default\_database {#default-database} +## default_database {#default-database} База данных по умолчанию. @@ -73,7 +73,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part default ``` -## default\_profile {#default-profile} +## default_profile {#default-profile} Профиль настроек по умолчанию. @@ -85,7 +85,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part default ``` -## dictionaries\_config {#server_configuration_parameters-dictionaries_config} +## dictionaries_config {#server_configuration_parameters-dictionaries_config} Путь к конфигурации внешних словарей. @@ -102,7 +102,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part *_dictionary.xml ``` -## dictionaries\_lazy\_load {#server_configuration_parameters-dictionaries_lazy_load} +## dictionaries_lazy_load {#server_configuration_parameters-dictionaries_lazy_load} Отложенная загрузка словарей. @@ -118,7 +118,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part true ``` -## format\_schema\_path {#server_configuration_parameters-format_schema_path} +## format_schema_path {#server_configuration_parameters-format_schema_path} Путь к каталогу со схемами для входных данных. Например со схемами для формата [CapnProto](../../interfaces/formats.md#capnproto). @@ -139,11 +139,11 @@ ClickHouse проверяет условия для `min_part_size` и `min_part - port – Порт сервера Graphite. - interval – Период отправки в секундах. - timeout – Таймаут отправки данных в секундах. -- root\_path – Префикс для ключей. +- root_path – Префикс для ключей. - metrics – Отправка данных из таблицы [system.metrics](../../operations/server-configuration-parameters/settings.md#system_tables-metrics). - events – Отправка дельты данных, накопленной за промежуток времени из таблицы [system.events](../../operations/server-configuration-parameters/settings.md#system_tables-events). -- events\_cumulative – Отправка суммарных данных из таблицы [system.events](../../operations/server-configuration-parameters/settings.md#system_tables-events). -- asynchronous\_metrics – Отправка данных из таблицы [system.asynchronous\_metrics](../../operations/server-configuration-parameters/settings.md#system_tables-asynchronous_metrics). +- events_cumulative – Отправка суммарных данных из таблицы [system.events](../../operations/server-configuration-parameters/settings.md#system_tables-events). +- asynchronous_metrics – Отправка данных из таблицы [system.asynchronous_metrics](../../operations/server-configuration-parameters/settings.md#system_tables-asynchronous_metrics). Можно определить несколько секций ``, например, для передачи различных данных с различной частотой. @@ -163,7 +163,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part ``` -## graphite\_rollup {#server_configuration_parameters-graphite-rollup} +## graphite_rollup {#server_configuration_parameters-graphite-rollup} Настройка прореживания данных для Graphite. @@ -191,7 +191,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part ``` -## http\_port/https\_port {#http-porthttps-port} +## http_port/https_port {#http-porthttps-port} Порт для обращений к серверу по протоколу HTTP(s). @@ -205,7 +205,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part 9999 ``` -## http\_server\_default\_response {#server_configuration_parameters-http_server_default_response} +## http_server_default_response {#server_configuration_parameters-http_server_default_response} Страница, показываемая по умолчанию, при обращении к HTTP(s) серверу ClickHouse. Значение по умолчанию «Ok.» (с переводом строки на конце). @@ -220,7 +220,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part ``` -## include\_from {#server_configuration_parameters-include_from} +## include_from {#server_configuration_parameters-include_from} Путь к файлу с подстановками. @@ -232,7 +232,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part /etc/metrica.xml ``` -## interserver\_http\_port {#interserver-http-port} +## interserver_http_port {#interserver-http-port} Порт для обмена между серверами ClickHouse. @@ -242,7 +242,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part 9009 ``` -## interserver\_http\_host {#interserver-http-host} +## interserver_http_host {#interserver-http-host} Имя хоста, которое могут использовать другие серверы для обращения к этому. @@ -256,7 +256,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part example.yandex.ru ``` -## interserver\_http\_credentials {#server-settings-interserver-http-credentials} +## interserver_http_credentials {#server-settings-interserver-http-credentials} Имя пользователя и пароль, использующиеся для аутентификации при [репликации](../../operations/server-configuration-parameters/settings.md) движками Replicated\*. Это имя пользователя и пароль используются только для взаимодействия между репликами кластера и никак не связаны с аутентификацией клиентов ClickHouse. Сервер проверяет совпадение имени и пароля для соединяющихся с ним реплик, а также использует это же имя и пароль для соединения с другими репликами. Соответственно, эти имя и пароль должны быть прописаны одинаковыми для всех реплик кластера. По умолчанию аутентификация не используется. @@ -275,7 +275,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part ``` -## keep\_alive\_timeout {#keep-alive-timeout} +## keep_alive_timeout {#keep-alive-timeout} Время в секундах, в течение которого ClickHouse ожидает входящих запросов прежде, чем закрыть соединение. @@ -285,7 +285,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part 3 ``` -## listen\_host {#server_configuration_parameters-listen_host} +## listen_host {#server_configuration_parameters-listen_host} Ограничение по хостам, с которых может прийти запрос. Если необходимо, чтобы сервер отвечал всем, то надо указать `::`. @@ -336,14 +336,14 @@ ClickHouse проверяет условия для `min_part_size` и `min_part Ключи для syslog: -- use\_syslog - обязательная настройка, если требуется запись в syslog +- use_syslog - обязательная настройка, если требуется запись в syslog - address - хост\[:порт\] демона syslogd. Если не указан, используется локальный - hostname - опционально, имя хоста, с которого отсылаются логи -- facility - [категория syslog](https://en.wikipedia.org/wiki/Syslog#Facility), записанная в верхнем регистре, с префиксом «LOG\_»: (`LOG_USER`, `LOG_DAEMON`, `LOG_LOCAL3` и прочие). +- facility - [категория syslog](https://en.wikipedia.org/wiki/Syslog#Facility), записанная в верхнем регистре, с префиксом «LOG_»: (`LOG_USER`, `LOG_DAEMON`, `LOG_LOCAL3` и прочие). Значения по умолчанию: при указанном `address` - `LOG_USER`, иначе - `LOG_DAEMON` - format - формат сообщений. Возможные значения - `bsd` и `syslog` -## send\_crash\_reports {#server_configuration_parameters-logger} +## send_crash_reports {#server_configuration_parameters-logger} Настройки для отправки сообщений о сбоях в команду разработчиков ядра ClickHouse через [Sentry](https://sentry.io). Включение этих настроек, особенно в pre-production среде, может дать очень ценную информацию и поможет развитию ClickHouse. @@ -381,7 +381,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part ``` -## mark\_cache\_size {#server-mark-cache-size} +## mark_cache_size {#server-mark-cache-size} Приблизительный размер (в байтах) кэша засечек, используемых движками таблиц семейства [MergeTree](../../operations/server-configuration-parameters/settings.md). @@ -393,7 +393,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part 5368709120 ``` -## max\_server\_memory\_usage {#max_server_memory_usage} +## max_server_memory_usage {#max_server_memory_usage} Ограничивает объём оперативной памяти, используемой сервером ClickHouse. Настройка может быть задана только для профиля `default`. @@ -410,7 +410,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part **См. также** -- [max\_memory\_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) +- [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) - [max_server_memory_usage_to_ram_ratio](#max_server_memory_usage_to_ram_ratio) ## max_server_memory_usage_to_ram_ratio {#max_server_memory_usage_to_ram_ratio} @@ -438,7 +438,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part - [max_server_memory_usage](#max_server_memory_usage) -## max\_concurrent\_queries {#max-concurrent-queries} +## max_concurrent_queries {#max-concurrent-queries} Максимальное количество одновременно обрабатываемых запросов. @@ -448,7 +448,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part 100 ``` -## max\_connections {#max-connections} +## max_connections {#max-connections} Максимальное количество входящих соединений. @@ -458,7 +458,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part 4096 ``` -## max\_open\_files {#max-open-files} +## max_open_files {#max-open-files} Максимальное количество открытых файлов. @@ -472,7 +472,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part 262144 ``` -## max\_table\_size\_to\_drop {#max-table-size-to-drop} +## max_table_size_to_drop {#max-table-size-to-drop} Ограничение на удаление таблиц. @@ -490,7 +490,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part 0 ``` -## max\_thread\_pool\_size {#max-thread-pool-size} +## max_thread_pool_size {#max-thread-pool-size} Максимальное количество потоков в глобальном пуле потоков. @@ -502,7 +502,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part 12000 ``` -## merge\_tree {#server_configuration_parameters-merge_tree} +## merge_tree {#server_configuration_parameters-merge_tree} Тонкая настройка таблиц семейства [MergeTree](../../operations/server-configuration-parameters/settings.md). @@ -537,8 +537,8 @@ ClickHouse проверяет условия для `min_part_size` и `min_part - sessionTimeout - Время кеширования сессии на сервере. - extendedVerification - Автоматическая расширенная проверка сертификатов после завершении сессии. Допустимые значения: `true`, `false`. - requireTLSv1 - Требование соединения TLSv1. Допустимые значения: `true`, `false`. -- requireTLSv1\_1 - Требование соединения TLSv1.1. Допустимые значения: `true`, `false`. -- requireTLSv1\_2 - Требование соединения TLSv1.2. Допустимые значения: `true`, `false`. +- requireTLSv1_1 - Требование соединения TLSv1.1. Допустимые значения: `true`, `false`. +- requireTLSv1_2 - Требование соединения TLSv1.2. Допустимые значения: `true`, `false`. - fips - Активация режима OpenSSL FIPS. Поддерживается, если версия OpenSSL, с которой собрана библиотека поддерживает fips. - privateKeyPassphraseHandler - Класс (подкласс PrivateKeyPassphraseHandler)запрашивающий кодовую фразу доступа к секретному ключу. Например, ``, `KeyFileHandler`, `test`, ``. - invalidCertificateHandler - Класс (подкласс CertificateHandler) для подтверждения не валидных сертификатов. Например, ` ConsoleCertificateHandler `. @@ -575,11 +575,11 @@ ClickHouse проверяет условия для `min_part_size` и `min_part ``` -## part\_log {#server_configuration_parameters-part-log} +## part_log {#server_configuration_parameters-part-log} Логирование событий, связанных с данными типа [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). Например, события добавления или мержа данных. Лог можно использовать для симуляции алгоритмов слияния, чтобы сравнивать их характеристики. Также, можно визуализировать процесс слияния. -Запросы логируются не в отдельный файл, а в таблицу [system.part\_log](../../operations/system-tables/part_log.md#system_tables-part-log). Вы можете изменить название этой таблицы в параметре `table` (см. ниже). +Запросы логируются не в отдельный файл, а в таблицу [system.part_log](../../operations/system-tables/part_log.md#system_tables-part-log). Вы можете изменить название этой таблицы в параметре `table` (см. ниже). При настройке логирования используются следующие параметры: @@ -623,7 +623,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part - `port` – порт по которому будет доступен endpoint для сбора метрик. - `metrics` – флаг для экспорта текущих значений метрик из таблицы [system.metrics](../system-tables/metrics.md#system_tables-metrics). - `events` – флаг для экспорта текущих значений метрик из таблицы [system.events](../system-tables/events.md#system_tables-events). -- `asynchronous_metrics` – флаг для экспорта текущих значений значения метрик из таблицы [system.asynchronous\_metrics](../system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics). +- `asynchronous_metrics` – флаг для экспорта текущих значений значения метрик из таблицы [system.asynchronous_metrics](../system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics). **Пример** @@ -637,11 +637,11 @@ ClickHouse проверяет условия для `min_part_size` и `min_part ``` -## query\_log {#server_configuration_parameters-query-log} +## query_log {#server_configuration_parameters-query-log} -Настройка логирования запросов, принятых с настройкой [log\_queries=1](../../operations/settings/settings.md). +Настройка логирования запросов, принятых с настройкой [log_queries=1](../../operations/settings/settings.md). -Запросы логируются не в отдельный файл, а в системную таблицу [system.query\_log](../../operations/system-tables/query_log.md#system_tables-query_log). Вы можете изменить название этой таблицы в параметре `table` (см. ниже). +Запросы логируются не в отдельный файл, а в системную таблицу [system.query_log](../../operations/system-tables/query_log.md#system_tables-query_log). Вы можете изменить название этой таблицы в параметре `table` (см. ниже). При настройке логирования используются следующие параметры: @@ -664,11 +664,11 @@ ClickHouse проверяет условия для `min_part_size` и `min_part ``` -## query\_thread\_log {#server_configuration_parameters-query_thread_log} +## query_thread_log {#server_configuration_parameters-query_thread_log} -Настройка логирования потоков выполнения запросов, принятых с настройкой [log\_query\_threads=1](../settings/settings.md#settings-log-query-threads). +Настройка логирования потоков выполнения запросов, принятых с настройкой [log_query_threads=1](../settings/settings.md#settings-log-query-threads). -Запросы логируются не в отдельный файл, а в системную таблицу [system.query\_thread\_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log). Вы можете изменить название этой таблицы в параметре `table` (см. ниже). +Запросы логируются не в отдельный файл, а в системную таблицу [system.query_thread_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log). Вы можете изменить название этой таблицы в параметре `table` (см. ниже). При настройке логирования используются следующие параметры: @@ -691,9 +691,9 @@ ClickHouse проверяет условия для `min_part_size` и `min_part ``` -## text\_log {#server_configuration_parameters-text_log} +## text_log {#server_configuration_parameters-text_log} -Настройка логирования текстовых сообщений в системную таблицу [text\_log](../../operations/system-tables/text_log.md#system_tables-text_log). +Настройка логирования текстовых сообщений в системную таблицу [text_log](../../operations/system-tables/text_log.md#system_tables-text_log). Параметры: @@ -719,9 +719,9 @@ ClickHouse проверяет условия для `min_part_size` и `min_part ``` -## trace\_log {#server_configuration_parameters-trace_log} +## trace_log {#server_configuration_parameters-trace_log} -Настройки для [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) system table operation. +Настройки для [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) system table operation. Parameters: @@ -742,7 +742,7 @@ The default server configuration file `config.xml` contains the following settin ``` -## query\_masking\_rules {#query-masking-rules} +## query_masking_rules {#query-masking-rules} Правила основанные на регурялных выражениях, которые будут применены для всех запросов а также для всех сообщений перед сохранением их в лог на сервере, `system.query_log`, `system.text_log`, `system.processes` таблицы, а также в логах отсылаемых клиенту. Это позволяет предотвратить @@ -772,7 +772,7 @@ The default server configuration file `config.xml` contains the following settin Для распределенных запросов каждый сервер должен быть сконфигурирован отдельно, иначе, подзапросы, переданные на другие узлы, будут сохраняться без маскировки. -## remote\_servers {#server-settings-remote-servers} +## remote_servers {#server-settings-remote-servers} Конфигурация кластеров, которые использует движок таблиц [Distributed](../../operations/server-configuration-parameters/settings.md) и табличная функция `cluster`. @@ -786,7 +786,7 @@ The default server configuration file `config.xml` contains the following settin **Смотрите также** -- [skip\_unavailable\_shards](../settings/settings.md#settings-skip_unavailable_shards) +- [skip_unavailable_shards](../settings/settings.md#settings-skip_unavailable_shards) ## timezone {#server_configuration_parameters-timezone} @@ -802,7 +802,7 @@ The default server configuration file `config.xml` contains the following settin Europe/Moscow ``` -## tcp\_port {#server_configuration_parameters-tcp_port} +## tcp_port {#server_configuration_parameters-tcp_port} Порт для взаимодействия с клиентами по протоколу TCP. @@ -812,7 +812,7 @@ The default server configuration file `config.xml` contains the following settin 9000 ``` -## tcp\_port\_secure {#server_configuration_parameters-tcp_port_secure} +## tcp_port_secure {#server_configuration_parameters-tcp_port_secure} TCP порт для защищённого обмена данными с клиентами. Используйте с настройкой [OpenSSL](#server_configuration_parameters-openssl). @@ -826,7 +826,7 @@ TCP порт для защищённого обмена данными с кли 9440 ``` -## mysql\_port {#server_configuration_parameters-mysql_port} +## mysql_port {#server_configuration_parameters-mysql_port} Порт для взаимодействия с клиентами по протоколу MySQL. @@ -853,7 +853,7 @@ TCP порт для защищённого обмена данными с кли /var/lib/clickhouse/tmp/ ``` -## tmp\_policy {#tmp-policy} +## tmp_policy {#tmp-policy} Политика из [storage_configuration](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) для хранения временных файлов. @@ -865,11 +865,11 @@ TCP порт для защищённого обмена данными с кли - `max_data_part_size_bytes` игнорируется. - В данной политике у вас должен быть ровно один том. -## uncompressed\_cache\_size {#server-settings-uncompressed_cache_size} +## uncompressed_cache_size {#server-settings-uncompressed_cache_size} Размер кеша (в байтах) для несжатых данных, используемых движками таблиц семейства [MergeTree](../../operations/server-configuration-parameters/settings.md). -Кеш единый для сервера. Память выделяется по требованию. Кеш используется в том случае, если включена опция [use\_uncompressed\_cache](../settings/settings.md). +Кеш единый для сервера. Память выделяется по требованию. Кеш используется в том случае, если включена опция [use_uncompressed_cache](../settings/settings.md). Несжатый кеш выгодно использовать для очень коротких запросов в отдельных случаях. @@ -879,7 +879,7 @@ TCP порт для защищённого обмена данными с кли 8589934592 ``` -## user\_files\_path {#server_configuration_parameters-user_files_path} +## user_files_path {#server_configuration_parameters-user_files_path} Каталог с пользовательскими файлами. Используется в табличной функции [file()](../../operations/server-configuration-parameters/settings.md). @@ -889,7 +889,7 @@ TCP порт для защищённого обмена данными с кли /var/lib/clickhouse/user_files/ ``` -## users\_config {#users-config} +## users_config {#users-config} Путь к файлу, который содержит: @@ -957,13 +957,13 @@ ClickHouse использует ZooKeeper для хранения метадан - [Репликация](../../engines/table-engines/mergetree-family/replication.md) - [ZooKeeper Programmer’s Guide](http://zookeeper.apache.org/doc/current/zookeeperProgrammers.html) -## use\_minimalistic\_part\_header\_in\_zookeeper {#server-settings-use_minimalistic_part_header_in_zookeeper} +## use_minimalistic_part_header_in_zookeeper {#server-settings-use_minimalistic_part_header_in_zookeeper} Способ хранения заголовков кусков данных в ZooKeeper. Параметр применяется только к семейству таблиц `MergeTree`. Его можно установить: -- Глобально в разделе [merge\_tree](#server_configuration_parameters-merge_tree) файла `config.xml`. +- Глобально в разделе [merge_tree](#server_configuration_parameters-merge_tree) файла `config.xml`. ClickHouse использует этот параметр для всех таблиц на сервере. Вы можете изменить настройку в любое время. Существующие таблицы изменяют свое поведение при изменении параметра. @@ -985,14 +985,14 @@ ClickHouse использует ZooKeeper для хранения метадан **Значение по умолчанию**: 0. -## disable\_internal\_dns\_cache {#server-settings-disable-internal-dns-cache} +## disable_internal_dns_cache {#server-settings-disable-internal-dns-cache} Отключает внутренний кеш DNS записей. Используется при эксплуатации ClickHouse в системах с часто меняющейся инфраструктурой, таких как Kubernetes. **Значение по умолчанию**: 0. -## dns\_cache\_update\_period {#server-settings-dns-cache-update-period} +## dns_cache_update_period {#server-settings-dns-cache-update-period} Период обновления IP адресов у записей во внутреннем DNS кеше ClickHouse (в секундах). Обновление выполняется асинхронно, отдельным системным потоком. diff --git a/docs/ru/operations/settings/merge-tree-settings.md b/docs/ru/operations/settings/merge-tree-settings.md index c0123cfb5f8..e78d4c98683 100644 --- a/docs/ru/operations/settings/merge-tree-settings.md +++ b/docs/ru/operations/settings/merge-tree-settings.md @@ -29,7 +29,7 @@ ALTER TABLE foo MODIFY SETTING max_suspicious_broken_parts = 100; ``` -## parts\_to\_throw\_insert {#parts-to-throw-insert} +## parts_to_throw_insert {#parts-to-throw-insert} Eсли число кусков в партиции превышает значение `parts_to_throw_insert`, INSERT прерывается с исключением `Too many parts (N). Merges are processing significantly slower than inserts`. @@ -43,7 +43,7 @@ Eсли число кусков в партиции превышает знач Можно установить большее значение 600 (1200), это уменьшит вероятность возникновения ошибки `Too many parts`, но в тоже время вы позже обнаружите возможную проблему со слияниями (например, из-за недостатка места на диске) и деградацию производительности `SELECT`. -## parts\_to\_delay\_insert {#parts-to-delay-insert} +## parts_to_delay_insert {#parts-to-delay-insert} Eсли число кусков в партиции превышает значение `parts_to_delay_insert`, `INSERT` искусственно замедляется. @@ -55,9 +55,9 @@ Eсли число кусков в партиции превышает знач ClickHouse искусственно выполняет `INSERT` дольше (добавляет ‘sleep’), чтобы фоновый механизм слияния успевал слиять куски быстрее, чем они добавляются. -## max\_delay\_to\_insert {#max-delay-to-insert} +## max_delay_to_insert {#max-delay-to-insert} -Величина в секундах, которая используется для расчета задержки `INSERT`, если число кусков в партиции превышает значение [parts\_to\_delay\_insert](#parts-to-delay-insert). +Величина в секундах, которая используется для расчета задержки `INSERT`, если число кусков в партиции превышает значение [parts_to_delay_insert](#parts-to-delay-insert). Возможные значения: @@ -73,9 +73,9 @@ k = 1 + parts_count_in_partition - parts_to_delay_insert delay_milliseconds = pow(max_delay_to_insert * 1000, k / max_k) ``` -Т.е. если в партиции уже 299 кусков и parts\_to\_throw\_insert = 300, parts\_to\_delay\_insert = 150, max\_delay\_to\_insert = 1, `INSERT` замедлится на `pow( 1 * 1000, (1 + 299 - 150) / (300 - 150) ) = 1000` миллисекунд. +Т.е. если в партиции уже 299 кусков и parts_to_throw_insert = 300, parts_to_delay_insert = 150, max_delay_to_insert = 1, `INSERT` замедлится на `pow( 1 * 1000, (1 + 299 - 150) / (300 - 150) ) = 1000` миллисекунд. -## max\_parts\_in\_total {#max-parts-in-total} +## max_parts_in_total {#max-parts-in-total} Eсли суммарное число активных кусков во всех партициях таблицы превышает значение `max_parts_in_total`, INSERT прерывается с исключением `Too many parts (N)`. @@ -87,7 +87,7 @@ Eсли суммарное число активных кусков во все Большое число кусков в таблице снижает производительность запросов ClickHouse и увеличивает время старта ClickHouse. Чаще всего это следствие неправильного дизайна (ошибки при выборе стратегии партиционирования -- слишком мелкие партиции). -## replicated\_deduplication\_window {#replicated-deduplication-window} +## replicated_deduplication_window {#replicated-deduplication-window} Количество хеш-сумм последних вставленных блоков, хранящихся в Zookeeper. @@ -100,7 +100,7 @@ Eсли суммарное число активных кусков во все Команда `Insert` создает один или несколько блоков (кусков). При вставке в Replicated таблицы ClickHouse для [дедупликации вставок](../../engines/table-engines/mergetree-family/replication.md) записывает в Zookeeper хеш-суммы созданных кусков. Но хранятся хеш-суммы не всех кусков, а только последние `replicated_deduplication_window`. Наиболее старые хеш-суммы удаляются из Zookeeper. Большое число `replicated_deduplication_window` замедляет `Insert`-ы. Хеш-сумма рассчитывается от композиции имен и типов полей, а также данных вставленного куска (потока байт). -## replicated\_deduplication\_window\_seconds {#replicated-deduplication-window-seconds} +## replicated_deduplication_window_seconds {#replicated-deduplication-window-seconds} Число секунд, после которых хеш-суммы вставленных блоков удаляются из Zookeeper. @@ -112,7 +112,7 @@ Eсли суммарное число активных кусков во все Аналогично [replicated_deduplication_window](#replicated-deduplication-window), задает, сколько времени хранить хеш-суммы блоков для дедупликции `Insert`-в. Хеш-суммы старше `replicated_deduplication_window_seconds` удаляются из Zookeeper, даже если их меньше чем `replicated_deduplication_window`. -## old\_parts\_lifetime {#old-parts-lifetime} +## old_parts_lifetime {#old-parts-lifetime} Время (в секундах) хранения неактивных кусков, для защиты от потери данных при спонтанной перезагрузке сервера или О.С. @@ -125,11 +125,11 @@ Eсли суммарное число активных кусков во все После слияния нескольких кусков в новый кусок, ClickHouse помечает исходные куски как неактивные и удаляет их после `old_parts_lifetime` секунд. Неактивные куски удаляются, если они не используются в текущих запросах, т.е. если счетчик ссылок куска – `refcount` равен нулю. -Неактивные куски удаляются не сразу, потому что при записи нового куска не вызывается `fsync`, т.е. некоторое время новый кусок находится только в оперативной памяти сервера (кеше О.С.). Т.о. при спонтанной перезагрузке сервера новый (смерженный) кусок может быть потерян или испорчен. В этом случае ClickHouse в процессе старта при проверке целостности кусков обнаружит проблему, вернет неактивные куски в список активных и позже заново их смержит. Сломанный кусок в этом случае переименовывается (добавляется префикс broken\_) и перемещается в папку detached. Если проверка целостности не обнаруживает проблем в смерженном куске, то исходные неактивные куски переименовываются (добавляется префикс ignored\_) и перемещаются в папку detached. +Неактивные куски удаляются не сразу, потому что при записи нового куска не вызывается `fsync`, т.е. некоторое время новый кусок находится только в оперативной памяти сервера (кеше О.С.). Т.о. при спонтанной перезагрузке сервера новый (смерженный) кусок может быть потерян или испорчен. В этом случае ClickHouse в процессе старта при проверке целостности кусков обнаружит проблему, вернет неактивные куски в список активных и позже заново их смержит. Сломанный кусок в этом случае переименовывается (добавляется префикс broken_) и перемещается в папку detached. Если проверка целостности не обнаруживает проблем в смерженном куске, то исходные неактивные куски переименовываются (добавляется префикс ignored_) и перемещаются в папку detached. -Стандартное значение Linux dirty\_expire\_centisecs - 30 секунд (максимальное время, которое записанные данные хранятся только в оперативной памяти), но при больших нагрузках на дисковую систему, данные могут быть записаны намного позже. Экспериментально было найдено время - 480 секунд, за которое гарантированно новый кусок будет записан на диск. +Стандартное значение Linux dirty_expire_centisecs - 30 секунд (максимальное время, которое записанные данные хранятся только в оперативной памяти), но при больших нагрузках на дисковую систему, данные могут быть записаны намного позже. Экспериментально было найдено время - 480 секунд, за которое гарантированно новый кусок будет записан на диск. -## max\_bytes\_to\_merge\_at\_max\_space\_in\_pool {#max-bytes-to-merge-at-max-space-in-pool} +## max_bytes_to_merge_at_max_space_in_pool {#max-bytes-to-merge-at-max-space-in-pool} Максимальный суммарный размер кусков (в байтах) в одном слиянии, при наличии свободных ресурсов в фоновом пуле. `max_bytes_to_merge_at_max_space_in_pool` -- примерно соответствует максимально возможному размеру куска, созданного автоматическим фоновым слиянием. @@ -144,7 +144,7 @@ Eсли суммарное число активных кусков во все Слияния, инициированные `optimize final`, не учитывают `max_bytes_to_merge_at_max_space_in_pool` и размеры кусков и слияют куски только с учетом наличия ресурсов в фоновом пуле, пока не останется один кусок в партиции. -## max\_bytes\_to\_merge\_at\_min\_space\_in\_pool {#max-bytes-to-merge-at-min-space-in-pool} +## max_bytes_to_merge_at_min_space_in_pool {#max-bytes-to-merge-at-min-space-in-pool} Максимальный суммарный размер кусков (в байтах) в одном слиянии, при минимальных свободных ресурсах в фоновом пуле. @@ -157,7 +157,7 @@ Eсли суммарное число активных кусков во все `max_bytes_to_merge_at_min_space_in_pool` задает максимальный суммарный размер кусков, для которых можно начать слияние, несмотря на недостаток свободных ресурсов в фоновом пуле (дискового пространства). Это необходимо, чтобы уменьшить количество маленьких кусков и вероятность ошибки `Too many parts`. Слияния резервируют дисковое пространство, удваивая суммарный размер кусков в слиянии. Таким образом, при малом количестве свободного места на диске может сложится ситуация, что свободное место есть, но оно уже зарезервировано идущими слиянияними, поэтому другие слияния не могут начаться, и количество маленьких кусков в партиции растет с каждым инсертом. -## merge\_max\_block\_size {#merge-max-block-size} +## merge_max_block_size {#merge-max-block-size} Количество строк в блоках, которые читаются из слияемых кусков. @@ -169,7 +169,7 @@ Eсли суммарное число активных кусков во все Слияние читает строки из кусков блоками по `merge_max_block_size` строк, производит слияние и пишет результат в новый кусок. Читаемый блок помещается в оперативную память, т.е. `merge_max_block_size` влияет на размер оперативной памяти, необходимой для слияния. Таким образом, слияния могут потреблять большое количество оперативной памяти для таблиц, хранящих очень большие строки (если средний размер строки 100кб, то при слиянии 10 кусков будет использовано (100кб * 10 * 8192) =~ 8ГБ ОЗУ). Уменьшив `merge_max_block_size`, можно сократить размер оперативной памяти, необходимой для слияния. -## max\_part\_loading\_threads {#max-part-loading-threads} +## max_part_loading_threads {#max-part-loading-threads} Максимальное количество потоков, которые читают куски при старте ClickHouse. diff --git a/docs/ru/operations/settings/permissions-for-queries.md b/docs/ru/operations/settings/permissions-for-queries.md index bbc122041de..cb65e652331 100644 --- a/docs/ru/operations/settings/permissions-for-queries.md +++ b/docs/ru/operations/settings/permissions-for-queries.md @@ -11,7 +11,7 @@ Разрешения пользователя по типу запроса регулируются параметрами: - [readonly](#settings_readonly) — ограничивает разрешения для всех типов запросов, кроме DDL. -- [allow\_ddl](#settings_allow_ddl) — ограничивает разрешения для DDL запросов. +- [allow_ddl](#settings_allow_ddl) — ограничивает разрешения для DDL запросов. `KILL QUERY` выполняется с любыми настройками. @@ -37,7 +37,7 @@ 0 -## allow\_ddl {#settings_allow_ddl} +## allow_ddl {#settings_allow_ddl} Разрешает/запрещает [DDL](https://ru.wikipedia.org/wiki/Data_Definition_Language) запросы. diff --git a/docs/ru/operations/settings/query-complexity.md b/docs/ru/operations/settings/query-complexity.md index d228732acdf..aeb5445a054 100644 --- a/docs/ru/operations/settings/query-complexity.md +++ b/docs/ru/operations/settings/query-complexity.md @@ -8,8 +8,8 @@ Ограничения проверяются на каждый блок обработанных данных, а не на каждую строку. В связи с этим, ограничения могут быть превышены на размер блока. Ограничения вида «максимальное количество чего-нибудь» могут принимать значение 0, которое обозначает «не ограничено». -Для большинства ограничений также присутствует настройка вида overflow\_mode - что делать, когда ограничение превышено. -Оно может принимать одно из двух значений: `throw` или `break`; а для ограничения на агрегацию (group\_by\_overflow\_mode) есть ещё значение `any`. +Для большинства ограничений также присутствует настройка вида overflow_mode - что делать, когда ограничение превышено. +Оно может принимать одно из двух значений: `throw` или `break`; а для ограничения на агрегацию (group_by_overflow_mode) есть ещё значение `any`. `throw` - кинуть исключение (по умолчанию). @@ -17,7 +17,7 @@ `any (только для group_by_overflow_mode)` - продолжить агрегацию по ключам, которые успели войти в набор, но не добавлять новые ключи в набор. -## max\_memory\_usage {#settings_max_memory_usage} +## max_memory_usage {#settings_max_memory_usage} Максимальный возможный объём оперативной памяти для выполнения запроса на одном сервере. @@ -34,29 +34,29 @@ Потребление памяти ограничивается также параметрами `max_memory_usage_for_user` и [max_server_memory_usage](../server-configuration-parameters/settings.md#max_server_memory_usage). -## max\_memory\_usage\_for\_user {#max-memory-usage-for-user} +## max_memory_usage_for_user {#max-memory-usage-for-user} Максимальный возможный объём оперативной памяти для запросов пользователя на одном сервере. Значения по умолчанию определены в файле [Settings.h](https://github.com/ClickHouse/ClickHouse/blob/master/src/Core/Settings.h#L288). По умолчанию размер не ограничен (`max_memory_usage_for_user = 0`). -Смотрите также описание настройки [max\_memory\_usage](#settings_max_memory_usage). +Смотрите также описание настройки [max_memory_usage](#settings_max_memory_usage). -## max\_rows\_to\_read {#max-rows-to-read} +## max_rows_to_read {#max-rows-to-read} Следующие ограничения могут проверяться на каждый блок (а не на каждую строку). То есть, ограничения могут быть немного нарушены. Максимальное количество строчек, которое можно прочитать из таблицы при выполнении запроса. -## max\_bytes\_to\_read {#max-bytes-to-read} +## max_bytes_to_read {#max-bytes-to-read} Максимальное количество байт (несжатых данных), которое можно прочитать из таблицы при выполнении запроса. -## read\_overflow\_mode {#read-overflow-mode} +## read_overflow_mode {#read-overflow-mode} Что делать, когда количество прочитанных данных превысило одно из ограничений: throw или break. По умолчанию: throw. -## max\_rows\_to\_read_leaf {#max-rows-to-read-leaf} +## max_rows_to_read_leaf {#max-rows-to-read-leaf} Следующие ограничения могут проверяться на каждый блок (а не на каждую строку). То есть, ограничения могут быть немного нарушены. @@ -68,7 +68,7 @@ настройкой `max_rows_to_read=150` выбросит исключение, т.к. в общем он прочитает 200 строк. Но запрос с настройкой `max_rows_to_read_leaf=150` завершится успешно, потому что каждый из шардов прочитает максимум 100 строк. -## max\_bytes\_to\_read_leaf {#max-bytes-to-read-leaf} +## max_bytes_to_read_leaf {#max-bytes-to-read-leaf} Максимальное количество байт (несжатых данных), которое можно прочитать из таблицы на удалённом сервере при выполнении распределенного запроса. Распределенные запросы могут создавать несколько подзапросов к каждому из шардов в @@ -78,20 +78,20 @@ и установленной настройкой `max_bytes_to_read=150` выбросит исключение, т.к. в общем он прочитает 200 байт. Но запрос с настройкой `max_bytes_to_read_leaf=150` завершится успешно, потому что каждый из шардов прочитает максимум 100 байт. -## read\_overflow\_mode_leaf {#read-overflow-mode-leaf} +## read_overflow_mode_leaf {#read-overflow-mode-leaf} Что делать, когда количество прочитанных данных на удаленном сервере превысило одно из ограничений: throw или break. По умолчанию: throw. -## max\_rows\_to\_group\_by {#settings-max-rows-to-group-by} +## max_rows_to_group_by {#settings-max-rows-to-group-by} Максимальное количество уникальных ключей, получаемых в процессе агрегации. Позволяет ограничить потребление оперативки при агрегации. -## group\_by\_overflow\_mode {#group-by-overflow-mode} +## group_by_overflow_mode {#group-by-overflow-mode} Что делать, когда количество уникальных ключей при агрегации превысило ограничение: throw, break или any. По умолчанию: throw. Использование значения any позволяет выполнить GROUP BY приближённо. Качество такого приближённого вычисления сильно зависит от статистических свойств данных. -## max\_bytes\_before\_external\_group\_by {#settings-max_bytes_before_external_group_by} +## max_bytes_before_external_group_by {#settings-max_bytes_before_external_group_by} Включает или отключает выполнение секций `GROUP BY` во внешней памяти. Смотрите [GROUP BY во внешней памяти](../../sql-reference/statements/select/group-by.md#select-group-by-in-external-memory). @@ -102,31 +102,31 @@ Значение по умолчанию — 0. -## max\_rows\_to\_sort {#max-rows-to-sort} +## max_rows_to_sort {#max-rows-to-sort} Максимальное количество строк до сортировки. Позволяет ограничить потребление оперативки при сортировке. -## max\_bytes\_to\_sort {#max-bytes-to-sort} +## max_bytes_to_sort {#max-bytes-to-sort} Максимальное количество байт до сортировки. -## sort\_overflow\_mode {#sort-overflow-mode} +## sort_overflow_mode {#sort-overflow-mode} Что делать, если количество строк, полученное перед сортировкой, превысило одно из ограничений: throw или break. По умолчанию: throw. -## max\_result\_rows {#setting-max_result_rows} +## max_result_rows {#setting-max_result_rows} Ограничение на количество строк результата. Проверяются также для подзапросов и на удалённых серверах при выполнении части распределённого запроса. -## max\_result\_bytes {#max-result-bytes} +## max_result_bytes {#max-result-bytes} Ограничение на количество байт результата. Аналогично. -## result\_overflow\_mode {#result-overflow-mode} +## result_overflow_mode {#result-overflow-mode} Что делать, если объём результата превысил одно из ограничений: throw или break. По умолчанию: throw. -Использование break по смыслу похоже на LIMIT. Break прерывает выполнение только на уровне блока. Т.е. число строк которые вернет запрос будет больше чем ограничение [max\_result\_rows](#setting-max_result_rows), кратно [max\_block\_size](../settings/settings.md#setting-max_block_size) и зависит от [max\_threads](../settings/settings.md#settings-max_threads). +Использование break по смыслу похоже на LIMIT. Break прерывает выполнение только на уровне блока. Т.е. число строк которые вернет запрос будет больше чем ограничение [max_result_rows](#setting-max_result_rows), кратно [max_block_size](../settings/settings.md#setting-max_block_size) и зависит от [max_threads](../settings/settings.md#settings-max_threads). Пример: @@ -145,103 +145,103 @@ FORMAT Null; 6666 rows in set. ... ``` -## max\_execution\_time {#max-execution-time} +## max_execution_time {#max-execution-time} Максимальное время выполнения запроса в секундах. На данный момент не проверяется при одной из стадий сортировки а также при слиянии и финализации агрегатных функций. -## timeout\_overflow\_mode {#timeout-overflow-mode} +## timeout_overflow_mode {#timeout-overflow-mode} -Что делать, если запрос выполняется дольше max\_execution\_time: throw или break. По умолчанию: throw. +Что делать, если запрос выполняется дольше max_execution_time: throw или break. По умолчанию: throw. -## min\_execution\_speed {#min-execution-speed} +## min_execution_speed {#min-execution-speed} -Минимальная скорость выполнения запроса в строчках в секунду. Проверяется на каждый блок данных по истечении timeout\_before\_checking\_execution\_speed. Если скорость выполнения запроса оказывается меньше, то кидается исключение. +Минимальная скорость выполнения запроса в строчках в секунду. Проверяется на каждый блок данных по истечении timeout_before_checking_execution_speed. Если скорость выполнения запроса оказывается меньше, то кидается исключение. -## min\_execution\_speed\_bytes {#min-execution-speed-bytes} +## min_execution_speed_bytes {#min-execution-speed-bytes} -Минимальная скорость выполнения запроса в строках на байт. Он проверяется для каждого блока данных после timeout\_before\_checking\_execution\_speed. Если скорость выполнения запроса меньше, исключение. +Минимальная скорость выполнения запроса в строках на байт. Он проверяется для каждого блока данных после timeout_before_checking_execution_speed. Если скорость выполнения запроса меньше, исключение. -## max\_execution\_speed {#max-execution-speed} +## max_execution_speed {#max-execution-speed} -Максимальная скорость выполнения запроса в строках в секунду. Он проверяется для каждого блока данных после timeout\_before\_checking\_execution\_speed. Если скорость выполнения запроса выше, скорость будет снижена. +Максимальная скорость выполнения запроса в строках в секунду. Он проверяется для каждого блока данных после timeout_before_checking_execution_speed. Если скорость выполнения запроса выше, скорость будет снижена. -## max\_execution\_speed\_bytes {#max-execution-speed-bytes} +## max_execution_speed_bytes {#max-execution-speed-bytes} -Максимальная скорость выполнения запроса в байтах в секунду. Он проверяется для каждого блока данных после timeout\_before\_checking\_execution\_speed. Если скорость выполнения запроса выше, скорость будет снижена. +Максимальная скорость выполнения запроса в байтах в секунду. Он проверяется для каждого блока данных после timeout_before_checking_execution_speed. Если скорость выполнения запроса выше, скорость будет снижена. -## timeout\_before\_checking\_execution\_speed {#timeout-before-checking-execution-speed} +## timeout_before_checking_execution_speed {#timeout-before-checking-execution-speed} -Проверять, что скорость выполнения запроса не слишком низкая (не меньше min\_execution\_speed), после прошествия указанного времени в секундах. +Проверять, что скорость выполнения запроса не слишком низкая (не меньше min_execution_speed), после прошествия указанного времени в секундах. -## max\_columns\_to\_read {#max-columns-to-read} +## max_columns_to_read {#max-columns-to-read} Максимальное количество столбцов, которых можно читать из таблицы в одном запросе. Если запрос требует чтения большего количества столбцов - кинуть исключение. -## max\_temporary\_columns {#max-temporary-columns} +## max_temporary_columns {#max-temporary-columns} Максимальное количество временных столбцов, которых необходимо одновременно держать в оперативке, в процессе выполнения запроса, включая константные столбцы. Если временных столбцов оказалось больше - кидается исключение. -## max\_temporary\_non\_const\_columns {#max-temporary-non-const-columns} +## max_temporary_non_const_columns {#max-temporary-non-const-columns} -То же самое, что и max\_temporary\_columns, но без учёта столбцов-констант. +То же самое, что и max_temporary_columns, но без учёта столбцов-констант. Стоит заметить, что столбцы-константы довольно часто образуются в процессе выполнения запроса, но расходуют примерно нулевое количество вычислительных ресурсов. -## max\_subquery\_depth {#max-subquery-depth} +## max_subquery_depth {#max-subquery-depth} Максимальная вложенность подзапросов. Если подзапросы более глубокие - кидается исключение. По умолчанию: 100. -## max\_pipeline\_depth {#max-pipeline-depth} +## max_pipeline_depth {#max-pipeline-depth} Максимальная глубина конвейера выполнения запроса. Соответствует количеству преобразований, которое проходит каждый блок данных в процессе выполнения запроса. Считается в пределах одного сервера. Если глубина конвейера больше - кидается исключение. По умолчанию: 1000. -## max\_ast\_depth {#max-ast-depth} +## max_ast_depth {#max-ast-depth} Максимальная вложенность синтаксического дерева запроса. Если превышена - кидается исключение. На данный момент, проверяются не во время парсинга а уже после парсинга запроса. То есть, во время парсинга может быть создано слишком глубокое синтаксическое дерево, но запрос не будет выполнен. По умолчанию: 1000. -## max\_ast\_elements {#max-ast-elements} +## max_ast_elements {#max-ast-elements} Максимальное количество элементов синтаксического дерева запроса. Если превышено - кидается исключение. Аналогично, проверяется уже после парсинга запроса. По умолчанию: 50 000. -## max\_rows\_in\_set {#max-rows-in-set} +## max_rows_in_set {#max-rows-in-set} Максимальное количество строчек для множества в секции IN, создаваемого из подзапроса. -## max\_bytes\_in\_set {#max-bytes-in-set} +## max_bytes_in_set {#max-bytes-in-set} Максимальное количество байт (несжатых данных), занимаемое множеством в секции IN, создаваемым из подзапроса. -## set\_overflow\_mode {#set-overflow-mode} +## set_overflow_mode {#set-overflow-mode} Что делать, когда количество данных превысило одно из ограничений: throw или break. По умолчанию: throw. -## max\_rows\_in\_distinct {#max-rows-in-distinct} +## max_rows_in_distinct {#max-rows-in-distinct} Максимальное количество различных строчек при использовании DISTINCT. -## max\_bytes\_in\_distinct {#max-bytes-in-distinct} +## max_bytes_in_distinct {#max-bytes-in-distinct} Максимальное количество байт, занимаемых хэш-таблицей, при использовании DISTINCT. -## distinct\_overflow\_mode {#distinct-overflow-mode} +## distinct_overflow_mode {#distinct-overflow-mode} Что делать, когда количество данных превысило одно из ограничений: throw или break. По умолчанию: throw. -## max\_rows\_to\_transfer {#max-rows-to-transfer} +## max_rows_to_transfer {#max-rows-to-transfer} Максимальное количество строчек, которых можно передать на удалённый сервер или сохранить во временную таблицу, при использовании GLOBAL IN. -## max\_bytes\_to\_transfer {#max-bytes-to-transfer} +## max_bytes_to_transfer {#max-bytes-to-transfer} Максимальное количество байт (несжатых данных), которых можно передать на удалённый сервер или сохранить во временную таблицу, при использовании GLOBAL IN. -## transfer\_overflow\_mode {#transfer-overflow-mode} +## transfer_overflow_mode {#transfer-overflow-mode} Что делать, когда количество данных превысило одно из ограничений: throw или break. По умолчанию: throw. -## max\_rows\_in\_join {#settings-max_rows_in_join} +## max_rows_in_join {#settings-max_rows_in_join} Ограничивает количество строк в хэш-таблице, используемой при соединении таблиц. @@ -249,7 +249,7 @@ FORMAT Null; Если запрос содержит несколько `JOIN`, то ClickHouse проверяет значение настройки для каждого промежуточного результата. -При достижении предела ClickHouse может выполнять различные действия. Используйте настройку [join\_overflow\_mode](#settings-join_overflow_mode) для выбора действия. +При достижении предела ClickHouse может выполнять различные действия. Используйте настройку [join_overflow_mode](#settings-join_overflow_mode) для выбора действия. Возможные значения: @@ -258,7 +258,7 @@ FORMAT Null; Значение по умолчанию — 0. -## max\_bytes\_in\_join {#settings-max_bytes_in_join} +## max_bytes_in_join {#settings-max_bytes_in_join} Ограничивает размер (в байтах) хэш-таблицы, используемой при объединении таблиц. @@ -266,7 +266,7 @@ FORMAT Null; Если запрос содержит несколько `JOIN`, то ClickHouse проверяет значение настройки для каждого промежуточного результата. -При достижении предела ClickHouse может выполнять различные действия. Используйте настройку [join\_overflow\_mode](#settings-join_overflow_mode) для выбора действия. +При достижении предела ClickHouse может выполнять различные действия. Используйте настройку [join_overflow_mode](#settings-join_overflow_mode) для выбора действия. Возможные значения: @@ -275,12 +275,12 @@ FORMAT Null; Значение по умолчанию — 0. -## join\_overflow\_mode {#settings-join_overflow_mode} +## join_overflow_mode {#settings-join_overflow_mode} Определяет, какое действие ClickHouse выполняет при достижении любого из следующих ограничений для `JOIN`: -- [max\_bytes\_in\_join](#settings-max_bytes_in_join) -- [max\_rows\_in\_join](#settings-max_rows_in_join) +- [max_bytes_in_join](#settings-max_bytes_in_join) +- [max_rows_in_join](#settings-max_rows_in_join) Возможные значения: @@ -294,7 +294,7 @@ FORMAT Null; - [Секция JOIN](../../sql-reference/statements/select/join.md#select-join) - [Движоy таблиц Join](../../engines/table-engines/special/join.md) -## max\_partitions\_per\_insert\_block {#max-partitions-per-insert-block} +## max_partitions_per_insert_block {#max-partitions-per-insert-block} Ограничивает максимальное количество партиций в одном вставленном блоке. @@ -307,6 +307,6 @@ FORMAT Null; При вставке данных, ClickHouse вычисляет количество партиций во вставленном блоке. Если число партиций больше, чем `max_partitions_per_insert_block`, ClickHouse генерирует исключение со следующим текстом: -> «Too many partitions for single INSERT block (more than» + toString(max\_parts) + «). The limit is controlled by ‘max\_partitions\_per\_insert\_block’ setting. Large number of partitions is a common misconception. It will lead to severe negative performance impact, including slow server startup, slow INSERT queries and slow SELECT queries. Recommended total number of partitions for a table is under 1000..10000. Please note, that partitioning is not intended to speed up SELECT queries (ORDER BY key is sufficient to make range queries fast). Partitions are intended for data manipulation (DROP PARTITION, etc).» +> «Too many partitions for single INSERT block (more than» + toString(max_parts) + «). The limit is controlled by ‘max_partitions_per_insert_block’ setting. Large number of partitions is a common misconception. It will lead to severe negative performance impact, including slow server startup, slow INSERT queries and slow SELECT queries. Recommended total number of partitions for a table is under 1000..10000. Please note, that partitioning is not intended to speed up SELECT queries (ORDER BY key is sufficient to make range queries fast). Partitions are intended for data manipulation (DROP PARTITION, etc).» [Оригинальная статья](https://clickhouse.tech/docs/ru/operations/settings/query_complexity/) diff --git a/docs/ru/operations/settings/settings-users.md b/docs/ru/operations/settings/settings-users.md index 95701f0f639..7c12780823a 100644 --- a/docs/ru/operations/settings/settings-users.md +++ b/docs/ru/operations/settings/settings-users.md @@ -36,7 +36,7 @@ ``` -### user\_name/password {#user-namepassword} +### user_name/password {#user-namepassword} Пароль можно указать в текстовом виде или в виде SHA256 (шестнадцатеричный формат). @@ -83,7 +83,7 @@ Значение по умолчанию: 0. -### user\_name/networks {#user-namenetworks} +### user_name/networks {#user-namenetworks} Список сетей, из которых пользователь может подключиться к серверу ClickHouse. @@ -125,17 +125,17 @@ 127.0.0.1 ``` -### user\_name/profile {#user-nameprofile} +### user_name/profile {#user-nameprofile} Пользователю можно назначить профиль настроек. Профили настроек конфигурируются в отдельной секции файла `users.xml`. Подробнее читайте в разделе [Профили настроек](settings-profiles.md). -### user\_name/quota {#user-namequota} +### user_name/quota {#user-namequota} Квотирование позволяет отслеживать или ограничивать использование ресурсов в течение определённого периода времени. Квоты настраиваются в разделе `quotas` конфигурационного файла `users.xml`. Пользователю можно назначить квоты. Подробное описание настройки квот смотрите в разделе [Квоты](../quotas.md#quotas). -### user\_name/databases {#user-namedatabases} +### user_name/databases {#user-namedatabases} В этом разделе вы можете ограничить выдачу ClickHouse запросами `SELECT` для конкретного пользователя, таким образом реализуя базовую защиту на уровне строк. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 2969dcc8430..f4ec5edeb8f 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -5,7 +5,7 @@ toc_title: Settings # Настройки {#settings} -## distributed\_product\_mode {#distributed-product-mode} +## distributed_product_mode {#distributed-product-mode} Изменяет поведение [распределенных подзапросов](../../sql-reference/operators/index.md). @@ -25,7 +25,7 @@ ClickHouse применяет настройку в тех случаях, ко - `global` — заменяет запрос `IN`/`JOIN` на `GLOBAL IN`/`GLOBAL JOIN.` - `allow` — разрешает использование таких подзапросов. -## enable\_optimize\_predicate\_expression {#enable-optimize-predicate-expression} +## enable_optimize_predicate_expression {#enable-optimize-predicate-expression} Включает пробрасывание предикатов в подзапросы для запросов `SELECT`. @@ -49,7 +49,7 @@ ClickHouse применяет настройку в тех случаях, ко Если `enable_optimize_predicate_expression = 0`, то время выполнения второго запроса намного больше, потому что секция `WHERE` применяется к данным уже после завершения подзапроса. -## fallback\_to\_stale\_replicas\_for\_distributed\_queries {#settings-fallback_to_stale_replicas_for_distributed_queries} +## fallback_to_stale_replicas_for_distributed_queries {#settings-fallback_to_stale_replicas_for_distributed_queries} Форсирует запрос в устаревшую реплику в случае, если актуальные данные недоступны. См. [Репликация](../../engines/table-engines/mergetree-family/replication.md). @@ -59,7 +59,7 @@ ClickHouse применяет настройку в тех случаях, ко По умолчанию - 1 (включена). -## force\_index\_by\_date {#settings-force_index_by_date} +## force_index_by_date {#settings-force_index_by_date} Запрещает выполнение запросов, если использовать индекс по дате невозможно. @@ -67,7 +67,7 @@ ClickHouse применяет настройку в тех случаях, ко При `force_index_by_date=1` ClickHouse проверяет, есть ли в запросе условие на ключ даты, которое может использоваться для отсечения диапазонов данных. Если подходящего условия нет - кидается исключение. При этом не проверяется, действительно ли условие уменьшает объём данных для чтения. Например, условие `Date != '2000-01-01'` подходит даже в том случае, когда соответствует всем данным в таблице (т.е. для выполнения запроса требуется full scan). Подробнее про диапазоны данных в таблицах MergeTree читайте в разделе [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). -## force\_primary\_key {#settings-force_primary_key} +## force_primary_key {#settings-force_primary_key} Запрещает выполнение запросов, если использовать индекс по первичному ключу невозможно. @@ -75,17 +75,17 @@ ClickHouse применяет настройку в тех случаях, ко При `force_primary_key=1` ClickHouse проверяет, есть ли в запросе условие на первичный ключ, которое может использоваться для отсечения диапазонов данных. Если подходящего условия нет - кидается исключение. При этом не проверяется, действительно ли условие уменьшает объём данных для чтения. Подробнее про диапазоны данных в таблицах MergeTree читайте в разделе [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). -## format\_schema {#format-schema} +## format_schema {#format-schema} Параметр применяется в том случае, когда используются форматы, требующие определения схемы, например [Cap’n Proto](https://capnproto.org/) или [Protobuf](https://developers.google.com/protocol-buffers/). Значение параметра зависит от формата. -## fsync\_metadata {#fsync-metadata} +## fsync_metadata {#fsync-metadata} Включает или отключает [fsync](http://pubs.opengroup.org/onlinepubs/9699919799/functions/fsync.html) при записи `.sql` файлов. По умолчанию включено. Имеет смысл выключать, если на сервере миллионы мелких таблиц-чанков, которые постоянно создаются и уничтожаются. -## enable\_http\_compression {#settings-enable_http_compression} +## enable_http_compression {#settings-enable_http_compression} Включает или отключает сжатие данных в ответе на HTTP-запрос. @@ -98,15 +98,15 @@ ClickHouse применяет настройку в тех случаях, ко Значение по умолчанию: 0. -## http\_zlib\_compression\_level {#settings-http_zlib_compression_level} +## http_zlib_compression_level {#settings-http_zlib_compression_level} -Задаёт уровень сжатия данных в ответе на HTTP-запрос, если [enable\_http\_compression = 1](#settings-enable_http_compression). +Задаёт уровень сжатия данных в ответе на HTTP-запрос, если [enable_http_compression = 1](#settings-enable_http_compression). Возможные значения: числа от 1 до 9. Значение по умолчанию: 3. -## http\_native\_compression\_disable\_checksumming\_on\_decompress {#settings-http_native_compression_disable_checksumming_on_decompress} +## http_native_compression_disable_checksumming_on_decompress {#settings-http_native_compression_disable_checksumming_on_decompress} Включает или отключает проверку контрольной суммы при распаковке данных HTTP POST от клиента. Используется только для собственного (`Navite`) формата сжатия ClickHouse (ни `gzip`, ни `deflate`). @@ -119,7 +119,7 @@ ClickHouse применяет настройку в тех случаях, ко Значение по умолчанию: 0. -## send\_progress\_in\_http\_headers {#settings-send_progress_in_http_headers} +## send_progress_in_http_headers {#settings-send_progress_in_http_headers} Включает или отключает HTTP-заголовки `X-ClickHouse-Progress` в ответах `clickhouse-server`. @@ -132,7 +132,7 @@ ClickHouse применяет настройку в тех случаях, ко Значение по умолчанию: 0. -## max\_http\_get\_redirects {#setting-max_http_get_redirects} +## max_http_get_redirects {#setting-max_http_get_redirects} Ограничивает максимальное количество переходов по редиректам в таблицах с движком [URL](../../engines/table-engines/special/url.md) при выполнении HTTP запросов методом GET. Настройка применяется для обоих типов таблиц: созданных запросом [CREATE TABLE](../../sql_reference/create/#create-table-query) и с помощью табличной функции [url](../../sql-reference/table-functions/url.md). @@ -143,7 +143,7 @@ ClickHouse применяет настройку в тех случаях, ко Значение по умолчанию: 0. -## input\_format\_allow\_errors\_num {#input-format-allow-errors-num} +## input_format_allow_errors_num {#input-format-allow-errors-num} Устанавливает максимальное количество допустимых ошибок при чтении из текстовых форматов (CSV, TSV и т.п.). @@ -155,7 +155,7 @@ ClickHouse применяет настройку в тех случаях, ко В случае превышения `input_format_allow_errors_num` ClickHouse генерирует исключение. -## input\_format\_allow\_errors\_ratio {#input-format-allow-errors-ratio} +## input_format_allow_errors_ratio {#input-format-allow-errors-ratio} Устанавливает максимальную долю допустимых ошибок при чтении из текстовых форматов (CSV, TSV и т.п.). Доля ошибок задаётся в виде числа с плавающей запятой от 0 до 1. @@ -168,7 +168,7 @@ ClickHouse применяет настройку в тех случаях, ко В случае превышения `input_format_allow_errors_ratio` ClickHouse генерирует исключение. -## input\_format\_values\_interpret\_expressions {#settings-input_format_values_interpret_expressions} +## input_format_values_interpret_expressions {#settings-input_format_values_interpret_expressions} Включает или отключает парсер SQL, если потоковый парсер не может проанализировать данные. Этот параметр используется только для формата [Values](../../interfaces/formats.md#data-format-values) при вставке данных. Дополнительные сведения о парсерах читайте в разделе [Синтаксис](../../sql-reference/syntax.md). @@ -218,7 +218,7 @@ INSERT INTO datetime_t SELECT now() Ok. ``` -## input\_format\_values\_deduce\_templates\_of\_expressions {#settings-input_format_values_deduce_templates_of_expressions} +## input_format_values_deduce_templates_of_expressions {#settings-input_format_values_deduce_templates_of_expressions} Включает или отключает попытку вычисления шаблона для выражений SQL в формате [Values](../../interfaces/formats.md#data-format-values). Это позволяет гораздо быстрее парсить и интерпретировать выражения в `Values`, если выражения в последовательных строках имеют одинаковую структуру. ClickHouse пытается вычислить шаблон выражения, распарсить следующие строки с помощью этого шаблона и вычислить выражение в пачке успешно проанализированных строк. @@ -239,7 +239,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( - Если `input_format_values_interpret_expressions=0` и `format_values_deduce_templates_of_expressions=1`, выражения в первой, второй и третьей строках парсятся с помощью шаблона `lower(String)` и интерпретируется вместе, выражение в четвертой строке парсится с другим шаблоном (`upper(String)`). - Если `input_format_values_interpret_expressions=1` и `format_values_deduce_templates_of_expressions=1`, то же самое, что и в предыдущем случае, но также позволяет выполнять резервную интерпретацию выражений отдельно, если невозможно вычислить шаблон. -## input\_format\_values\_accurate\_types\_of\_literals {#settings-input-format-values-accurate-types-of-literals} +## input_format_values_accurate_types_of_literals {#settings-input-format-values-accurate-types-of-literals} Эта настройка используется, только когда `input_format_values_deduce_templates_of_expressions = 1`. Выражения для некоторых столбцов могут иметь одинаковую структуру, но содержат числовые литералы разных типов, например: @@ -261,7 +261,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( Значение по умолчанию: 1. -## input\_format\_defaults\_for\_omitted\_fields {#session_settings-input_format_defaults_for_omitted_fields} +## input_format_defaults_for_omitted_fields {#session_settings-input_format_defaults_for_omitted_fields} При вставке данных запросом `INSERT`, заменяет пропущенные поля значениям по умолчанию для типа данных столбца. @@ -281,7 +281,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( Значение по умолчанию: 1. -## input\_format\_tsv\_empty\_as\_default {#settings-input-format-tsv-empty-as-default} +## input_format_tsv_empty_as_default {#settings-input-format-tsv-empty-as-default} Если эта настройка включена, замените пустые поля ввода в TSV значениями по умолчанию. Для сложных выражений по умолчанию также должна быть включена настройка `input_format_defaults_for_omitted_fields`. @@ -289,11 +289,11 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( Disabled by default. -## input\_format\_null\_as\_default {#settings-input-format-null-as-default} +## input_format_null_as_default {#settings-input-format-null-as-default} Включает или отключает использование значений по умолчанию в случаях, когда во входных данных содержится `NULL`, но тип соответствующего столбца не `Nullable(T)` (для текстовых форматов). -## input\_format\_skip\_unknown\_fields {#settings-input-format-skip-unknown-fields} +## input_format_skip_unknown_fields {#settings-input-format-skip-unknown-fields} Включает или отключает пропускание вставки неизвестных данных. @@ -313,7 +313,7 @@ Disabled by default. Значение по умолчанию: 0. -## input\_format\_import\_nested\_json {#settings-input_format_import_nested_json} +## input_format_import_nested_json {#settings-input_format_import_nested_json} Включает или отключает вставку данных JSON с вложенными объектами. @@ -332,7 +332,7 @@ Disabled by default. - [Использование вложенных структур](../../interfaces/formats.md#jsoneachrow-nested) with the `JSONEachRow` format. -## input\_format\_with\_names\_use\_header {#settings-input-format-with-names-use-header} +## input_format_with_names_use_header {#settings-input-format-with-names-use-header} Включает или отключает проверку порядка столбцов при вставке данных. @@ -350,7 +350,7 @@ Disabled by default. Значение по умолчанию: 1. -## date\_time\_input\_format {#settings-date_time_input_format} +## date_time_input_format {#settings-date_time_input_format} Выбор парсера для текстового представления дат и времени при обработке входного формата. @@ -373,7 +373,7 @@ Disabled by default. - [Тип данных DateTime.](../../sql-reference/data-types/datetime.md) - [Функции для работы с датой и временем.](../../sql-reference/functions/date-time-functions.md) -## join\_default\_strictness {#settings-join_default_strictness} +## join_default_strictness {#settings-join_default_strictness} Устанавливает строгость по умолчанию для [JOIN](../../sql-reference/statements/select/join.md#select-join). @@ -385,7 +385,7 @@ Disabled by default. Значение по умолчанию: `ALL`. -## join\_any\_take\_last\_row {#settings-join_any_take_last_row} +## join_any_take_last_row {#settings-join_any_take_last_row} Изменяет поведение операций, выполняемых со строгостью `ANY`. @@ -403,9 +403,9 @@ Disabled by default. - [Секция JOIN](../../sql-reference/statements/select/join.md#select-join) - [Движок таблиц Join](../../engines/table-engines/special/join.md) -- [join\_default\_strictness](#settings-join_default_strictness) +- [join_default_strictness](#settings-join_default_strictness) -## join\_use\_nulls {#join_use_nulls} +## join_use_nulls {#join_use_nulls} Устанавливает тип поведения [JOIN](../../sql-reference/statements/select/join.md). При объединении таблиц могут появиться пустые ячейки. ClickHouse заполняет их по-разному в зависимости от настроек. @@ -494,7 +494,7 @@ Disabled by default. - [JOIN strictness](../../sql-reference/statements/select/join.md#join-settings) -## max\_block\_size {#setting-max_block_size} +## max_block_size {#setting-max_block_size} Данные в ClickHouse обрабатываются по блокам (наборам кусочков столбцов). Внутренние циклы обработки для одного блока достаточно эффективны, но есть заметные издержки на каждый блок. Настройка `max_block_size` — это рекомендация, какой размер блока (в количестве строк) загружать из таблиц. Размер блока не должен быть слишком маленьким, чтобы затраты на каждый блок были заметны, но не слишком велики, чтобы запрос с LIMIT, который завершается после первого блока, обрабатывался быстро. Цель состоит в том, чтобы не использовалось слишком много оперативки при вынимании большого количества столбцов в несколько потоков; чтобы оставалась хоть какая-нибудь кэш-локальность. @@ -502,13 +502,13 @@ Disabled by default. Из таблицы не всегда загружаются блоки размера `max_block_size`. Если ясно, что нужно прочитать меньше данных, то будет считан блок меньшего размера. -## preferred\_block\_size\_bytes {#preferred-block-size-bytes} +## preferred_block_size_bytes {#preferred-block-size-bytes} Служит для тех же целей что и `max_block_size`, но задает рекомендуемый размер блоков в байтах, выбирая адаптивное количество строк в блоке. При этом размер блока не может быть более `max_block_size` строк. По умолчанию: 1,000,000. Работает только при чтении из MergeTree-движков. -## merge\_tree\_uniform\_read\_distribution {#setting-merge-tree-uniform-read-distribution} +## merge_tree_uniform_read_distribution {#setting-merge-tree-uniform-read-distribution} При чтении из таблиц [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) ClickHouse использует несколько потоков. Этот параметр включает/выключает равномерное распределение заданий по рабочим потокам. Алгоритм равномерного распределения стремится сделать время выполнения всех потоков примерно равным для одного запроса `SELECT`. @@ -519,7 +519,7 @@ Disabled by default. Значение по умолчанию: 1. -## merge\_tree\_min\_rows\_for\_concurrent\_read {#setting-merge-tree-min-rows-for-concurrent-read} +## merge_tree_min_rows_for_concurrent_read {#setting-merge-tree-min-rows-for-concurrent-read} Если количество строк, считываемых из файла таблицы [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) превышает `merge_tree_min_rows_for_concurrent_read`, то ClickHouse пытается выполнить одновременное чтение из этого файла в несколько потоков. @@ -529,7 +529,7 @@ Disabled by default. Значение по умолчанию: 163840. -## merge\_tree\_min\_bytes\_for\_concurrent\_read {#setting-merge-tree-min-bytes-for-concurrent-read} +## merge_tree_min_bytes_for_concurrent_read {#setting-merge-tree-min-bytes-for-concurrent-read} Если число байтов, которое должно быть прочитано из одного файла таблицы с движком [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md), превышает значение `merge_tree_min_bytes_for_concurrent_read`, то ClickHouse выполняет одновременное чтение в несколько потоков из этого файла. @@ -539,7 +539,7 @@ Disabled by default. Значение по умолчанию: 251658240. -## merge\_tree\_min\_rows\_for\_seek {#setting-merge-tree-min-rows-for-seek} +## merge_tree_min_rows_for_seek {#setting-merge-tree-min-rows-for-seek} Если расстояние между двумя блоками данных для чтения в одном файле меньше, чем `merge_tree_min_rows_for_seek` строк, то ClickHouse не перескакивает (seek) через блоки, а считывает данные последовательно. @@ -549,7 +549,7 @@ Disabled by default. Значение по умолчанию: 0. -## merge\_tree\_min\_bytes\_for\_seek {#setting-merge-tree-min-bytes-for-seek} +## merge_tree_min_bytes_for_seek {#setting-merge-tree-min-bytes-for-seek} Если расстояние между двумя блоками данных для чтения в одном файле меньше, чем `merge_tree_min_bytes_for_seek` байтов, то ClickHouse не перескакивает (seek) через блоки, а считывает данные последовательно. @@ -559,7 +559,7 @@ Disabled by default. Значение по умолчанию: 0. -## merge\_tree\_coarse\_index\_granularity {#setting-merge-tree-coarse-index-granularity} +## merge_tree_coarse_index_granularity {#setting-merge-tree-coarse-index-granularity} При поиске данных ClickHouse проверяет засечки данных в файле индекса. Если ClickHouse обнаруживает, что требуемые ключи находятся в некотором диапазоне, он делит этот диапазон на `merge_tree_coarse_index_granularity` поддиапазонов и выполняет в них рекурсивный поиск нужных ключей. @@ -569,11 +569,11 @@ Disabled by default. Значение по умолчанию: 8. -## merge\_tree\_max\_rows\_to\_use\_cache {#setting-merge-tree-max-rows-to-use-cache} +## merge_tree_max_rows_to_use_cache {#setting-merge-tree-max-rows-to-use-cache} Если требуется прочитать более, чем `merge_tree_max_rows_to_use_cache` строк в одном запросе, ClickHouse не используют кэш несжатых блоков. -Кэш несжатых блоков хранит данные, извлечённые при выполнении запросов. ClickHouse использует этот кэш для ускорения ответов на повторяющиеся небольшие запросы. Настройка защищает кэш от замусоривания запросами, для выполнения которых необходимо извлечь большое количество данных. Настройка сервера [uncompressed\_cache\_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) определяет размер кэша несжатых блоков. +Кэш несжатых блоков хранит данные, извлечённые при выполнении запросов. ClickHouse использует этот кэш для ускорения ответов на повторяющиеся небольшие запросы. Настройка защищает кэш от замусоривания запросами, для выполнения которых необходимо извлечь большое количество данных. Настройка сервера [uncompressed_cache_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) определяет размер кэша несжатых блоков. Возможные значения: @@ -581,11 +581,11 @@ Disabled by default. Значение по умолчанию: 128 ✕ 8192. -## merge\_tree\_max\_bytes\_to\_use\_cache {#setting-merge-tree-max-bytes-to-use-cache} +## merge_tree_max_bytes_to_use_cache {#setting-merge-tree-max-bytes-to-use-cache} Если требуется прочитать более, чем `merge_tree_max_bytes_to_use_cache` байтов в одном запросе, ClickHouse не используют кэш несжатых блоков. -Кэш несжатых блоков хранит данные, извлечённые при выполнении запросов. ClickHouse использует кэш для ускорения ответов на повторяющиеся небольшие запросы. Настройка защищает кэш от переполнения. Настройка сервера [uncompressed\_cache\_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) определяет размер кэша несжатых блоков. +Кэш несжатых блоков хранит данные, извлечённые при выполнении запросов. ClickHouse использует кэш для ускорения ответов на повторяющиеся небольшие запросы. Настройка защищает кэш от переполнения. Настройка сервера [uncompressed_cache_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) определяет размер кэша несжатых блоков. Возможное значение: @@ -593,7 +593,7 @@ Disabled by default. Значение по умолчанию: 2013265920. -## min\_bytes\_to\_use\_direct\_io {#settings-min-bytes-to-use-direct-io} +## min_bytes_to_use_direct_io {#settings-min-bytes-to-use-direct-io} Минимальный объём данных, необходимый для прямого (небуферизованного) чтения/записи (direct I/O) на диск. @@ -631,11 +631,11 @@ ClickHouse использует этот параметр при чтении д Значение по умолчанию: `1`. -## log\_queries {#settings-log-queries} +## log_queries {#settings-log-queries} Установка логирования запроса. -Запросы, переданные в ClickHouse с этой установкой, логируются согласно правилам конфигурационного параметра сервера [query\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query-log). +Запросы, переданные в ClickHouse с этой установкой, логируются согласно правилам конфигурационного параметра сервера [query_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query-log). Пример: @@ -643,7 +643,7 @@ ClickHouse использует этот параметр при чтении д log_queries=1 ``` -## log\_queries\_min\_type {#settings-log-queries-min-type} +## log_queries_min_type {#settings-log-queries-min-type} Задаёт минимальный уровень логирования в `query_log`. @@ -661,11 +661,11 @@ log_queries=1 log_queries_min_type='EXCEPTION_WHILE_PROCESSING' ``` -## log\_query\_threads {#settings-log-query-threads} +## log_query_threads {#settings-log-query-threads} Установка логирования информации о потоках выполнения запроса. -Лог информации о потоках выполнения запросов, переданных в ClickHouse с этой установкой, записывается согласно правилам конфигурационного параметра сервера [query\_thread\_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log). +Лог информации о потоках выполнения запросов, переданных в ClickHouse с этой установкой, записывается согласно правилам конфигурационного параметра сервера [query_thread_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log). Пример: @@ -673,19 +673,19 @@ log_queries_min_type='EXCEPTION_WHILE_PROCESSING' log_query_threads=1 ``` -## max\_insert\_block\_size {#settings-max_insert_block_size} +## max_insert_block_size {#settings-max_insert_block_size} Формировать блоки указанного размера, при вставке в таблицу. Эта настройка действует только в тех случаях, когда сервер сам формирует такие блоки. Например, при INSERT-е через HTTP интерфейс, сервер парсит формат данных, и формирует блоки указанного размера. -А при использовании clickhouse-client, клиент сам парсит данные, и настройка max\_insert\_block\_size на сервере не влияет на размер вставляемых блоков. +А при использовании clickhouse-client, клиент сам парсит данные, и настройка max_insert_block_size на сервере не влияет на размер вставляемых блоков. При использовании INSERT SELECT, настройка так же не имеет смысла, так как данные будут вставляться теми блоками, которые вышли после SELECT-а. Значение по умолчанию: 1,048,576. Это значение намного больше, чем `max_block_size`. Это сделано, потому что некоторые движки таблиц (`*MergeTree`) будут на каждый вставляемый блок формировать кусок данных на диске, что является довольно большой сущностью. Также, в таблицах типа `*MergeTree`, данные сортируются при вставке, и достаточно большой размер блока позволяет отсортировать больше данных в оперативке. -## min\_insert\_block\_size\_rows {#min-insert-block-size-rows} +## min_insert_block_size_rows {#min-insert-block-size-rows} Устанавливает минимальное количество строк в блоке, который может быть вставлен в таблицу запросом `INSERT`. Блоки меньшего размера склеиваются в блоки большего размера. @@ -696,7 +696,7 @@ log_query_threads=1 Значение по умолчанию: 1048576. -## min\_insert\_block\_size\_bytes {#min-insert-block-size-bytes} +## min_insert_block_size_bytes {#min-insert-block-size-bytes} Устанавливает минимальное количество байтов в блоке, который может быть вставлен в таблицу запросом `INSERT`. Блоки меньшего размера склеиваются в блоки большего размера. @@ -707,7 +707,7 @@ log_query_threads=1 Значение по умолчанию: 268435456. -## max\_replica\_delay\_for\_distributed\_queries {#settings-max_replica_delay_for_distributed_queries} +## max_replica_delay_for_distributed_queries {#settings-max_replica_delay_for_distributed_queries} Отключает отстающие реплики при распределенных запросах. См. [Репликация](../../engines/table-engines/mergetree-family/replication.md). @@ -719,20 +719,20 @@ log_query_threads=1 ## max_threads {#settings-max_threads} -Максимальное количество потоков обработки запроса без учёта потоков для чтения данных с удалённых серверов (смотрите параметр max\_distributed\_connections). +Максимальное количество потоков обработки запроса без учёта потоков для чтения данных с удалённых серверов (смотрите параметр max_distributed_connections). Этот параметр относится к потокам, которые выполняют параллельно одни стадии конвейера выполнения запроса. -Например, при чтении из таблицы, если есть возможность вычислять выражения с функциями, фильтровать с помощью WHERE и предварительно агрегировать для GROUP BY параллельно, используя хотя бы количество потоков max\_threads, то используются max\_threads. +Например, при чтении из таблицы, если есть возможность вычислять выражения с функциями, фильтровать с помощью WHERE и предварительно агрегировать для GROUP BY параллельно, используя хотя бы количество потоков max_threads, то используются max_threads. Значение по умолчанию: количество процессорных ядер без учёта Hyper-Threading. Если на сервере обычно исполняется менее одного запроса SELECT одновременно, то выставите этот параметр в значение чуть меньше количества реальных процессорных ядер. -Для запросов, которые быстро завершаются из-за LIMIT-а, имеет смысл выставить max\_threads поменьше. Например, если нужное количество записей находится в каждом блоке, то при max\_threads = 8 будет считано 8 блоков, хотя достаточно было прочитать один. +Для запросов, которые быстро завершаются из-за LIMIT-а, имеет смысл выставить max_threads поменьше. Например, если нужное количество записей находится в каждом блоке, то при max_threads = 8 будет считано 8 блоков, хотя достаточно было прочитать один. Чем меньше `max_threads`, тем меньше будет использоваться оперативки. -## max\_insert\_threads {#settings-max-insert-threads} +## max_insert_threads {#settings-max-insert-threads} Максимальное количество потоков для выполнения запроса `INSERT SELECT`. @@ -746,34 +746,34 @@ log_query_threads=1 Параллельный `INSERT SELECT` действует только в том случае, если часть SELECT выполняется параллельно, см. настройку [max_threads](#settings-max_threads). Чем больше значение `max_insert_threads`, тем больше потребление оперативной памяти. -## max\_compress\_block\_size {#max-compress-block-size} +## max_compress_block_size {#max-compress-block-size} Максимальный размер блоков не сжатых данных перед сжатием при записи в таблицу. По умолчанию - 1 048 576 (1 MiB). При уменьшении размера, незначительно уменьшается коэффициент сжатия, незначительно возрастает скорость сжатия и разжатия за счёт кэш-локальности, и уменьшается потребление оперативки. Как правило, не имеет смысла менять эту настройку. Не путайте блоки для сжатия (кусок памяти, состоящий из байт) и блоки для обработки запроса (пачка строк из таблицы). -## min\_compress\_block\_size {#min-compress-block-size} +## min_compress_block_size {#min-compress-block-size} -Для таблиц типа [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). В целях уменьшения задержек при обработке запросов, блок сжимается при записи следующей засечки, если его размер не меньше min\_compress\_block\_size. По умолчанию - 65 536. +Для таблиц типа [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). В целях уменьшения задержек при обработке запросов, блок сжимается при записи следующей засечки, если его размер не меньше min_compress_block_size. По умолчанию - 65 536. -Реальный размер блока, если несжатых данных меньше max\_compress\_block\_size, будет не меньше этого значения и не меньше объёма данных на одну засечку. +Реальный размер блока, если несжатых данных меньше max_compress_block_size, будет не меньше этого значения и не меньше объёма данных на одну засечку. -Рассмотрим пример. Пусть index\_granularity, указанная при создании таблицы - 8192. +Рассмотрим пример. Пусть index_granularity, указанная при создании таблицы - 8192. -Пусть мы записываем столбец типа UInt32 (4 байта на значение). При записи 8192 строк, будет всего 32 КБ данных. Так как min\_compress\_block\_size = 65 536, сжатый блок будет сформирован на каждые две засечки. +Пусть мы записываем столбец типа UInt32 (4 байта на значение). При записи 8192 строк, будет всего 32 КБ данных. Так как min_compress_block_size = 65 536, сжатый блок будет сформирован на каждые две засечки. Пусть мы записываем столбец URL типа String (средний размер - 60 байт на значение). При записи 8192 строк, будет, в среднем, чуть меньше 500 КБ данных. Так как это больше 65 536 строк, то сжатый блок будет сформирован на каждую засечку. В этом случае, при чтении с диска данных из диапазона в одну засечку, не будет разжато лишних данных. Как правило, не имеет смысла менять эту настройку. -## max\_query\_size {#settings-max_query_size} +## max_query_size {#settings-max_query_size} Максимальный кусок запроса, который будет считан в оперативку для разбора парсером языка SQL. Запрос INSERT также содержит данные для INSERT-а, которые обрабатываются отдельным, потоковым парсером (расходующим O(1) оперативки), и не учитываются в этом ограничении. Значение по умолчанию: 256 Кб. -## max\_parser\_depth {#max_parser_depth} +## max_parser_depth {#max_parser_depth} Ограничивает максимальную глубину рекурсии в парсере рекурсивного спуска. Позволяет контролировать размер стека. @@ -784,31 +784,31 @@ log_query_threads=1 Значение по умолчанию: 1000. -## interactive\_delay {#interactive-delay} +## interactive_delay {#interactive-delay} Интервал в микросекундах для проверки, не запрошена ли остановка выполнения запроса, и отправки прогресса. Значение по умолчанию: 100,000 (проверять остановку запроса и отправлять прогресс десять раз в секунду). -## connect\_timeout, receive\_timeout, send\_timeout {#connect-timeout-receive-timeout-send-timeout} +## connect_timeout, receive_timeout, send_timeout {#connect-timeout-receive-timeout-send-timeout} Таймауты в секундах на сокет, по которому идёт общение с клиентом. Значение по умолчанию: 10, 300, 300. -## cancel\_http\_readonly\_queries\_on\_client\_close {#cancel-http-readonly-queries-on-client-close} +## cancel_http_readonly_queries_on_client_close {#cancel-http-readonly-queries-on-client-close} Отменяет HTTP readonly запросы (например, SELECT), когда клиент обрывает соединение до завершения получения данных. Значение по умолчанию: 0 -## poll\_interval {#poll-interval} +## poll_interval {#poll-interval} Блокироваться в цикле ожидания запроса в сервере на указанное количество секунд. Значение по умолчанию: 10. -## max\_distributed\_connections {#max-distributed-connections} +## max_distributed_connections {#max-distributed-connections} Максимальное количество одновременных соединений с удалёнными серверами при распределённой обработке одного запроса к одной таблице типа Distributed. Рекомендуется выставлять не меньше, чем количество серверов в кластере. @@ -816,20 +816,20 @@ log_query_threads=1 Следующие параметры имеют значение только на момент создания таблицы типа Distributed (и при запуске сервера), поэтому их не имеет смысла менять в рантайме. -## distributed\_connections\_pool\_size {#distributed-connections-pool-size} +## distributed_connections_pool_size {#distributed-connections-pool-size} Максимальное количество одновременных соединений с удалёнными серверами при распределённой обработке всех запросов к одной таблице типа Distributed. Рекомендуется выставлять не меньше, чем количество серверов в кластере. Значение по умолчанию: 1024. -## connect\_timeout\_with\_failover\_ms {#connect-timeout-with-failover-ms} +## connect_timeout_with_failover_ms {#connect-timeout-with-failover-ms} Таймаут в миллисекундах на соединение с удалённым сервером, для движка таблиц Distributed, если используются секции shard и replica в описании кластера. В случае неуспеха, делается несколько попыток соединений с разными репликами. Значение по умолчанию: 50. -## connection\_pool\_max\_wait\_ms {#connection-pool-max-wait-ms} +## connection_pool_max_wait_ms {#connection-pool-max-wait-ms} Время ожидания соединения в миллисекундах, когда пул соединений заполнен. @@ -840,7 +840,7 @@ log_query_threads=1 Значение по умолчанию: 0. -## connections\_with\_failover\_max\_tries {#connections-with-failover-max-tries} +## connections_with_failover_max_tries {#connections-with-failover-max-tries} Максимальное количество попыток соединения с каждой репликой, для движка таблиц Distributed. @@ -851,7 +851,7 @@ log_query_threads=1 Считать ли экстремальные значения (минимумы и максимумы по столбцам результата запроса). Принимает 0 или 1. По умолчанию - 0 (выключено). Подробнее смотрите раздел «Экстремальные значения». -## kafka\_max\_wait\_ms {#kafka-max-wait-ms} +## kafka_max_wait_ms {#kafka-max-wait-ms} Время ожидания в миллисекундах для чтения сообщений из [Kafka](../../engines/table-engines/integrations/kafka.md#kafka) перед повторной попыткой. @@ -866,26 +866,26 @@ log_query_threads=1 - [Apache Kafka](https://kafka.apache.org/) -## use\_uncompressed\_cache {#setting-use_uncompressed_cache} +## use_uncompressed_cache {#setting-use_uncompressed_cache} Использовать ли кэш разжатых блоков. Принимает 0 или 1. По умолчанию - 0 (выключено). -Использование кэша несжатых блоков (только для таблиц семейства MergeTree) может существенно сократить задержку и увеличить пропускную способность при работе с большим количеством коротких запросов. Включите эту настройку для пользователей, от которых идут частые короткие запросы. Также обратите внимание на конфигурационный параметр [uncompressed\_cache\_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) (настраивается только в конфигурационном файле) – размер кэша разжатых блоков. По умолчанию - 8 GiB. Кэш разжатых блоков заполняется по мере надобности, а наиболее невостребованные данные автоматически удаляются. +Использование кэша несжатых блоков (только для таблиц семейства MergeTree) может существенно сократить задержку и увеличить пропускную способность при работе с большим количеством коротких запросов. Включите эту настройку для пользователей, от которых идут частые короткие запросы. Также обратите внимание на конфигурационный параметр [uncompressed_cache_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) (настраивается только в конфигурационном файле) – размер кэша разжатых блоков. По умолчанию - 8 GiB. Кэш разжатых блоков заполняется по мере надобности, а наиболее невостребованные данные автоматически удаляются. Для запросов, читающих хоть немного приличный объём данных (миллион строк и больше), кэш разжатых блоков автоматически выключается, чтобы оставить место для действительно мелких запросов. Поэтому, можно держать настройку `use_uncompressed_cache` всегда выставленной в 1. -## replace\_running\_query {#replace-running-query} +## replace_running_query {#replace-running-query} -При использовании интерфейса HTTP может быть передан параметр query\_id. Это любая строка, которая служит идентификатором запроса. -Если в этот момент, уже существует запрос от того же пользователя с тем же query\_id, то поведение определяется параметром replace\_running\_query. +При использовании интерфейса HTTP может быть передан параметр query_id. Это любая строка, которая служит идентификатором запроса. +Если в этот момент, уже существует запрос от того же пользователя с тем же query_id, то поведение определяется параметром replace_running_query. -`0` - (по умолчанию) кинуть исключение (не давать выполнить запрос, если запрос с таким же query\_id уже выполняется); +`0` - (по умолчанию) кинуть исключение (не давать выполнить запрос, если запрос с таким же query_id уже выполняется); `1` - отменить старый запрос и начать выполнять новый. Эта настройка, выставленная в 1, используется в Яндекс.Метрике для реализации suggest-а значений для условий сегментации. После ввода очередного символа, если старый запрос ещё не выполнился, его следует отменить. -## replace\_running\_query\_max\_wait\_ms {#replace-running-query-max-wait-ms} +## replace_running_query_max_wait_ms {#replace-running-query-max-wait-ms} Время ожидания завершения выполнения запроса с тем же `query_id`, когда активирована настройка [replace_running_query](#replace-running-query). @@ -896,15 +896,15 @@ log_query_threads=1 Значение по умолчанию: 5000. -## stream\_flush\_interval\_ms {#stream-flush-interval-ms} +## stream_flush_interval_ms {#stream-flush-interval-ms} -Работает для таблиц со стриммингом в случае тайм-аута, или когда поток генерирует [max\_insert\_block\_size](#settings-max_insert_block_size) строк. +Работает для таблиц со стриммингом в случае тайм-аута, или когда поток генерирует [max_insert_block_size](#settings-max_insert_block_size) строк. Значение по умолчанию: 7500. Чем меньше значение, тем чаще данные сбрасываются в таблицу. Установка слишком низкого значения приводит к снижению производительности. -## load\_balancing {#settings-load_balancing} +## load_balancing {#settings-load_balancing} Задает алгоритм выбора реплик, используемый при обработке распределенных запросов. @@ -918,7 +918,7 @@ ClickHouse поддерживает следующие алгоритмы выб См. также: -- [distributed\_replica\_max\_ignored\_errors](#settings-distributed_replica_max_ignored_errors) +- [distributed_replica_max_ignored_errors](#settings-distributed_replica_max_ignored_errors) ### Random (by Default) {#load_balancing-random} @@ -970,31 +970,31 @@ load_balancing = round_robin Этот алгоритм использует циклический перебор реплик с одинаковым количеством ошибок (учитываются только запросы с алгоритмом `round_robin`). -## prefer\_localhost\_replica {#settings-prefer-localhost-replica} +## prefer_localhost_replica {#settings-prefer-localhost-replica} Включает или выключает предпочтительное использование localhost реплики при обработке распределенных запросов. Возможные значения: - 1 — ClickHouse всегда отправляет запрос на localhost реплику, если она существует. -- 0 — ClickHouse использует балансировку, заданную настройкой [load\_balancing](#settings-load_balancing). +- 0 — ClickHouse использует балансировку, заданную настройкой [load_balancing](#settings-load_balancing). Значение по умолчанию: 1. !!! warning "Warning" - Отключайте эту настройку при использовании [max\_parallel\_replicas](#settings-max_parallel_replicas). + Отключайте эту настройку при использовании [max_parallel_replicas](#settings-max_parallel_replicas). -## totals\_mode {#totals-mode} +## totals_mode {#totals-mode} -Каким образом вычислять TOTALS при наличии HAVING, а также при наличии max\_rows\_to\_group\_by и group\_by\_overflow\_mode = ‘any’. +Каким образом вычислять TOTALS при наличии HAVING, а также при наличии max_rows_to_group_by и group_by_overflow_mode = ‘any’. Смотрите раздел «Модификатор WITH TOTALS». -## totals\_auto\_threshold {#totals-auto-threshold} +## totals_auto_threshold {#totals-auto-threshold} Порог для `totals_mode = 'auto'`. Смотрите раздел «Модификатор WITH TOTALS». -## max\_parallel\_replicas {#settings-max_parallel_replicas} +## max_parallel_replicas {#settings-max_parallel_replicas} Максимальное количество используемых реплик каждого шарда при выполнении запроса. Для консистентности (чтобы получить разные части одного и того же разбиения), эта опция работает только при заданном ключе сэмплирования. @@ -1007,7 +1007,7 @@ load_balancing = round_robin Компиляция предусмотрена только для части конвейера обработки запроса - для первой стадии агрегации (GROUP BY). В случае, если эта часть конвейера была скомпилирована, запрос может работать быстрее, за счёт разворачивания коротких циклов и инлайнинга вызовов агрегатных функций. Максимальный прирост производительности (до четырёх раз в редких случаях) достигается на запросах с несколькими простыми агрегатными функциями. Как правило, прирост производительности незначителен. В очень редких случаях возможно замедление выполнения запроса. -## min\_count\_to\_compile {#min-count-to-compile} +## min_count_to_compile {#min-count-to-compile} После скольких раз, когда скомпилированный кусок кода мог пригодиться, выполнить его компиляцию. По умолчанию - 3. Для тестирования можно установить значение 0: компиляция выполняется синхронно, и запрос ожидает окончания процесса компиляции перед продолжением выполнения. Во всех остальных случаях используйте значения, начинающиеся с 1. Как правило, компиляция занимает по времени около 5-10 секунд. @@ -1016,16 +1016,16 @@ load_balancing = round_robin Скомпилированный код требуется для каждого разного сочетания используемых в запросе агрегатных функций и вида ключей в GROUP BY. Результаты компиляции сохраняются в директории build в виде .so файлов. Количество результатов компиляции не ограничено, так как они не занимают много места. При перезапуске сервера, старые результаты будут использованы, за исключением случая обновления сервера - тогда старые результаты удаляются. -## input\_format\_skip\_unknown\_fields {#input-format-skip-unknown-fields} +## input_format_skip_unknown_fields {#input-format-skip-unknown-fields} Если значение равно true, то при выполнении INSERT входные данные из столбцов с неизвестными именами будут пропущены. В противном случае эта ситуация создаст исключение. Работает для форматов JSONEachRow и TSKV. -## output\_format\_json\_quote\_64bit\_integers {#session_settings-output_format_json_quote_64bit_integers} +## output_format_json_quote_64bit_integers {#session_settings-output_format_json_quote_64bit_integers} Если значение истинно, то при использовании JSON\* форматов UInt64 и Int64 числа выводятся в кавычках (из соображений совместимости с большинством реализаций JavaScript), иначе - без кавычек. -## output\_format\_json\_quote\_denormals {#settings-output_format_json_quote_denormals} +## output_format_json_quote_denormals {#settings-output_format_json_quote_denormals} При выводе данных в формате [JSON](../../interfaces/formats.md#json) включает отображение значений `+nan`, `-nan`, `+inf`, `-inf`. @@ -1119,23 +1119,23 @@ SELECT area/period FROM account_orders FORMAT JSON; ``` -## format\_csv\_delimiter {#settings-format_csv_delimiter} +## format_csv_delimiter {#settings-format_csv_delimiter} Символ, интерпретируемый как разделитель в данных формата CSV. По умолчанию — `,`. -## input\_format\_csv\_unquoted\_null\_literal\_as\_null {#settings-input_format_csv_unquoted_null_literal_as_null} +## input_format_csv_unquoted_null_literal_as_null {#settings-input_format_csv_unquoted_null_literal_as_null} Для формата CSV включает или выключает парсинг неэкранированной строки `NULL` как литерала (синоним для `\N`) -## output\_format\_csv\_crlf\_end\_of\_line {#settings-output-format-csv-crlf-end-of-line} +## output_format_csv_crlf_end_of_line {#settings-output-format-csv-crlf-end-of-line} Использовать в качестве разделителя строк для CSV формата CRLF (DOS/Windows стиль) вместо LF (Unix стиль). -## output\_format\_tsv\_crlf\_end\_of\_line {#settings-output-format-tsv-crlf-end-of-line} +## output_format_tsv_crlf_end_of_line {#settings-output-format-tsv-crlf-end-of-line} Использовать в качестве разделителя строк для TSV формата CRLF (DOC/Windows стиль) вместо LF (Unix стиль). -## insert\_quorum {#settings-insert_quorum} +## insert_quorum {#settings-insert_quorum} Включает кворумную запись. @@ -1150,7 +1150,7 @@ SELECT area/period FROM account_orders FORMAT JSON; Все реплики в кворуме консистентны, т.е. содержат данные всех более ранних запросов `INSERT`. Последовательность `INSERT` линеаризуется. -При чтении данных, записанных с `insert_quorum` можно использовать настройку [select\_sequential\_consistency](#settings-select_sequential_consistency). +При чтении данных, записанных с `insert_quorum` можно использовать настройку [select_sequential_consistency](#settings-select_sequential_consistency). ClickHouse генерирует исключение @@ -1159,10 +1159,10 @@ ClickHouse генерирует исключение См. также: -- [insert\_quorum\_timeout](#settings-insert_quorum_timeout) -- [select\_sequential\_consistency](#settings-select_sequential_consistency) +- [insert_quorum_timeout](#settings-insert_quorum_timeout) +- [select_sequential_consistency](#settings-select_sequential_consistency) -## insert\_quorum\_timeout {#settings-insert_quorum_timeout} +## insert_quorum_timeout {#settings-insert_quorum_timeout} Время ожидания кворумной записи в секундах. Если время прошло, а запись так не состоялась, то ClickHouse сгенерирует исключение и клиент должен повторить запрос на запись того же блока на эту же или любую другую реплику. @@ -1170,10 +1170,10 @@ ClickHouse генерирует исключение См. также: -- [insert\_quorum](#settings-insert_quorum) -- [select\_sequential\_consistency](#settings-select_sequential_consistency) +- [insert_quorum](#settings-insert_quorum) +- [select_sequential_consistency](#settings-select_sequential_consistency) -## select\_sequential\_consistency {#settings-select_sequential_consistency} +## select_sequential_consistency {#settings-select_sequential_consistency} Включает или выключает последовательную консистентность для запросов `SELECT`. @@ -1190,10 +1190,10 @@ ClickHouse генерирует исключение См. также: -- [insert\_quorum](#settings-insert_quorum) -- [insert\_quorum\_timeout](#settings-insert_quorum_timeout) +- [insert_quorum](#settings-insert_quorum) +- [insert_quorum_timeout](#settings-insert_quorum_timeout) -## insert\_deduplicate {#settings-insert-deduplicate} +## insert_deduplicate {#settings-insert-deduplicate} Включает и выключает дедупликацию для запросов `INSERT` (для Replicated\* таблиц). @@ -1206,7 +1206,7 @@ ClickHouse генерирует исключение По умолчанию блоки, вставляемые в реплицируемые таблицы оператором `INSERT`, дедуплицируются (см. [Репликация данных](../../engines/table-engines/mergetree-family/replication.md)). -## deduplicate\_blocks\_in\_dependent\_materialized\_views {#settings-deduplicate-blocks-in-dependent-materialized-views} +## deduplicate_blocks_in_dependent_materialized_views {#settings-deduplicate-blocks-in-dependent-materialized-views} Включает и выключает проверку дедупликации для материализованных представлений, которые получают данные из Replicated\* таблиц. @@ -1221,7 +1221,7 @@ ClickHouse генерирует исключение Т.е. если `INSERT` в основную таблицу д.б. пропущен (сдедуплицирован), то автоматически не будет вставки и в материализованные представления. Это имплементировано для того, чтобы работали материализованные представления, которые сильно группируют данные основных `INSERT`, до такой степени что блоки вставляемые в материализованные представления получаются одинаковыми для разных `INSERT` в основную таблицу. Одновременно это «ломает» идемпотентность вставки в материализованные представления. Т.е. если `INSERT` был успешен в основную таблицу и неуспешен в таблицу материализованного представления (напр. из-за сетевого сбоя при коммуникации с Zookeeper), клиент получит ошибку и попытается повторить `INSERT`. Но вставки в материализованные представления произведено не будет, потому что дедупликация сработает на основной таблице. Настройка `deduplicate_blocks_in_dependent_materialized_views` позволяет это изменить. Т.е. при повторном `INSERT` будет произведена дедупликация на таблице материализованного представления, и повторный инсерт вставит данные в таблицу материализованного представления, которые не удалось вставить из-за сбоя первого `INSERT`. -## count\_distinct\_implementation {#settings-count_distinct_implementation} +## count_distinct_implementation {#settings-count_distinct_implementation} Задаёт, какая из функций `uniq*` используется при выполнении конструкции [COUNT(DISTINCT …)](../../sql-reference/aggregate-functions/reference/count.md#agg_function-count). @@ -1235,7 +1235,7 @@ ClickHouse генерирует исключение Значение по умолчанию: `uniqExact`. -## max\_network\_bytes {#settings-max-network-bytes} +## max_network_bytes {#settings-max-network-bytes} Ограничивает объём данных (в байтах), который принимается или передается по сети при выполнении запроса. Параметр применяется к каждому отдельному запросу. @@ -1246,7 +1246,7 @@ ClickHouse генерирует исключение Значение по умолчанию: 0. -## max\_network\_bandwidth {#settings-max-network-bandwidth} +## max_network_bandwidth {#settings-max-network-bandwidth} Ограничивает скорость обмена данными по сети в байтах в секунду. Параметр применяется к каждому отдельному запросу. @@ -1257,7 +1257,7 @@ ClickHouse генерирует исключение Значение по умолчанию: 0. -## max\_network\_bandwidth\_for\_user {#settings-max-network-bandwidth-for-user} +## max_network_bandwidth_for_user {#settings-max-network-bandwidth-for-user} Ограничивает скорость обмена данными по сети в байтах в секунду. Этот параметр применяется ко всем одновременно выполняемым запросам, запущенным одним пользователем. @@ -1268,7 +1268,7 @@ ClickHouse генерирует исключение Значение по умолчанию: 0. -## max\_network\_bandwidth\_for\_all\_users {#settings-max-network-bandwidth-for-all-users} +## max_network_bandwidth_for_all_users {#settings-max-network-bandwidth-for-all-users} Ограничивает скорость обмена данными по сети в байтах в секунду. Этот параметр применяется ко всем одновременно выполняемым запросам на сервере. @@ -1279,7 +1279,7 @@ ClickHouse генерирует исключение Значение по умолчанию: 0. -## skip\_unavailable\_shards {#settings-skip_unavailable_shards} +## skip_unavailable_shards {#settings-skip_unavailable_shards} Включает или отключает тихий пропуск недоступных шардов. @@ -1309,7 +1309,7 @@ ClickHouse генерирует исключение Значение по умолчанию: 0. -## optimize\_skip\_unused\_shards {#optimize-skip-unused-shards} +## optimize_skip_unused_shards {#optimize-skip-unused-shards} Включает или отключает пропуск неиспользуемых шардов для запросов [SELECT](../../sql-reference/statements/select/index.md) , в которых условие ключа шардирования задано в секции `WHERE/PREWHERE`. Предполагается, что данные распределены с помощью ключа шардирования, в противном случае настройка ничего не делает. @@ -1320,7 +1320,7 @@ ClickHouse генерирует исключение Значение по умолчанию: 0 -## optimize\_skip\_unused\_shards\_nesting {#optimize-skip-unused-shards-nesting} +## optimize_skip_unused_shards_nesting {#optimize-skip-unused-shards-nesting} Контролирует настройку [`optimize_skip_unused_shards`](#optimize-skip-unused-shards) (поэтому все еще требует `optimize_skip_unused_shards`) в зависимости от вложенности распределенного запроса (когда у вас есть `Distributed` таблица которая смотрит на другую `Distributed` таблицу). @@ -1332,7 +1332,7 @@ ClickHouse генерирует исключение Значение по умолчанию: 0 -## force\_optimize\_skip\_unused\_shards {#settings-force_optimize_skip_unused_shards} +## force_optimize_skip_unused_shards {#settings-force_optimize_skip_unused_shards} Разрешает или запрещает выполнение запроса, если настройка [optimize_skip_unused_shards](#optimize-skip-unused-shards) включена, а пропуск неиспользуемых шардов невозможен. Если данная настройка включена и пропуск невозможен, ClickHouse генерирует исключение. @@ -1344,7 +1344,7 @@ ClickHouse генерирует исключение Значение по умолчанию: 0 -## force\_optimize\_skip\_unused\_shards\_nesting {#settings-force_optimize_skip_unused_shards_nesting} +## force_optimize_skip_unused_shards_nesting {#settings-force_optimize_skip_unused_shards_nesting} Контролирует настройку [`force_optimize_skip_unused_shards`](#settings-force_optimize_skip_unused_shards) (поэтому все еще требует `optimize_skip_unused_shards`) в зависимости от вложенности распределенного запроса (когда у вас есть `Distributed` таблица которая смотрит на другую `Distributed` таблицу). @@ -1356,7 +1356,7 @@ ClickHouse генерирует исключение Значение по умолчанию: 0 -## force\_optimize\_skip\_unused\_shards\_no\_nested {#settings-force_optimize_skip_unused_shards_no_nested} +## force_optimize_skip_unused_shards_no_nested {#settings-force_optimize_skip_unused_shards_no_nested} Сбрасывает [`optimize_skip_unused_shards`](#settings-force_optimize_skip_unused_shards) для вложенных `Distributed` таблиц. @@ -1367,7 +1367,7 @@ ClickHouse генерирует исключение Значение по умолчанию: 0 -## optimize\_throw\_if\_noop {#setting-optimize_throw_if_noop} +## optimize_throw_if_noop {#setting-optimize_throw_if_noop} Включает или отключает генерирование исключения в случаях, когда запрос [OPTIMIZE](../../sql-reference/statements/misc.md#misc_operations-optimize) не выполняет мёрж. @@ -1380,21 +1380,21 @@ ClickHouse генерирует исключение Значение по умолчанию: 0. -## distributed\_replica\_error\_half\_life {#settings-distributed_replica_error_half_life} +## distributed_replica_error_half_life {#settings-distributed_replica_error_half_life} - Тип: секунды - Значение по умолчанию: 60 секунд -Управляет скоростью обнуления ошибок в распределенных таблицах. Если реплика недоступна в течение некоторого времени, накапливает 5 ошибок, а distributed\_replica\_error\_half\_life установлена на 1 секунду, то реплика считается нормальной через 3 секунды после последней ошибки. +Управляет скоростью обнуления ошибок в распределенных таблицах. Если реплика недоступна в течение некоторого времени, накапливает 5 ошибок, а distributed_replica_error_half_life установлена на 1 секунду, то реплика считается нормальной через 3 секунды после последней ошибки. См. также: -- [load\_balancing](#load_balancing-round_robin) +- [load_balancing](#load_balancing-round_robin) - [Table engine Distributed](../../engines/table-engines/special/distributed.md) -- [distributed\_replica\_error\_cap](#settings-distributed_replica_error_cap) -- [distributed\_replica\_max\_ignored\_errors](#settings-distributed_replica_max_ignored_errors) +- [distributed_replica_error_cap](#settings-distributed_replica_error_cap) +- [distributed_replica_max_ignored_errors](#settings-distributed_replica_max_ignored_errors) -## distributed\_replica\_error\_cap {#settings-distributed_replica_error_cap} +## distributed_replica_error_cap {#settings-distributed_replica_error_cap} - Тип: unsigned int - Значение по умолчанию: 1000 @@ -1403,12 +1403,12 @@ ClickHouse генерирует исключение См. также: -- [load\_balancing](#load_balancing-round_robin) +- [load_balancing](#load_balancing-round_robin) - [Table engine Distributed](../../engines/table-engines/special/distributed.md) -- [distributed\_replica\_error\_half\_life](#settings-distributed_replica_error_half_life) -- [distributed\_replica\_max\_ignored\_errors](#settings-distributed_replica_max_ignored_errors) +- [distributed_replica_error_half_life](#settings-distributed_replica_error_half_life) +- [distributed_replica_max_ignored_errors](#settings-distributed_replica_max_ignored_errors) -## distributed\_replica\_max\_ignored\_errors {#settings-distributed_replica_max_ignored_errors} +## distributed_replica_max_ignored_errors {#settings-distributed_replica_max_ignored_errors} - Тип: unsigned int - Значение по умолчанию: 0 @@ -1417,12 +1417,12 @@ ClickHouse генерирует исключение См. также: -- [load\_balancing](#load_balancing-round_robin) +- [load_balancing](#load_balancing-round_robin) - [Table engine Distributed](../../engines/table-engines/special/distributed.md) -- [distributed\_replica\_error\_cap](#settings-distributed_replica_error_cap) -- [distributed\_replica\_error\_half\_life](#settings-distributed_replica_error_half_life) +- [distributed_replica_error_cap](#settings-distributed_replica_error_cap) +- [distributed_replica_error_half_life](#settings-distributed_replica_error_half_life) -## distributed\_directory\_monitor\_sleep\_time\_ms {#distributed_directory_monitor_sleep_time_ms} +## distributed_directory_monitor_sleep_time_ms {#distributed_directory_monitor_sleep_time_ms} Основной интервал отправки данных движком таблиц [Distributed](../../engines/table-engines/special/distributed.md). Фактический интервал растёт экспоненциально при возникновении ошибок. @@ -1432,9 +1432,9 @@ ClickHouse генерирует исключение Значение по умолчанию: 100 миллисекунд. -## distributed\_directory\_monitor\_max\_sleep\_time\_ms {#distributed_directory_monitor_max_sleep_time_ms} +## distributed_directory_monitor_max_sleep_time_ms {#distributed_directory_monitor_max_sleep_time_ms} -Максимальный интервал отправки данных движком таблиц [Distributed](../../engines/table-engines/special/distributed.md). Ограничивает экпоненциальный рост интервала, установленого настройкой [distributed\_directory\_monitor\_sleep\_time\_ms](#distributed_directory_monitor_sleep_time_ms). +Максимальный интервал отправки данных движком таблиц [Distributed](../../engines/table-engines/special/distributed.md). Ограничивает экпоненциальный рост интервала, установленого настройкой [distributed_directory_monitor_sleep_time_ms](#distributed_directory_monitor_sleep_time_ms). Возможные значения: @@ -1442,7 +1442,7 @@ ClickHouse генерирует исключение Значение по умолчанию: 30000 миллисекунд (30 секунд). -## distributed\_directory\_monitor\_batch\_inserts {#distributed_directory_monitor_batch_inserts} +## distributed_directory_monitor_batch_inserts {#distributed_directory_monitor_batch_inserts} Включает/выключает пакетную отправку вставленных данных. @@ -1455,7 +1455,7 @@ ClickHouse генерирует исключение Значение по умолчанию: 0. -## os\_thread\_priority {#setting-os-thread-priority} +## os_thread_priority {#setting-os-thread-priority} Устанавливает приоритет ([nice](https://en.wikipedia.org/wiki/Nice_(Unix))) для потоков, исполняющих запросы. Планировщик ОС учитывает эти приоритеты при выборе следующего потока для исполнения на доступном ядре CPU. @@ -1470,7 +1470,7 @@ ClickHouse генерирует исключение Значение по умолчанию: 0. -## query\_profiler\_real\_time\_period\_ns {#query_profiler_real_time_period_ns} +## query_profiler_real_time_period_ns {#query_profiler_real_time_period_ns} Устанавливает период для таймера реального времени [профилировщика запросов](../../operations/optimizing-performance/sampling-query-profiler.md). Таймер реального времени считает wall-clock time. @@ -1491,9 +1491,9 @@ ClickHouse генерирует исключение См. также: -- Системная таблица [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) +- Системная таблица [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) -## query\_profiler\_cpu\_time\_period\_ns {#query_profiler_cpu_time_period_ns} +## query_profiler_cpu_time_period_ns {#query_profiler_cpu_time_period_ns} Устанавливает период для таймера CPU [query profiler](../../operations/optimizing-performance/sampling-query-profiler.md). Этот таймер считает только время CPU. @@ -1514,7 +1514,7 @@ ClickHouse генерирует исключение См. также: -- Системная таблица [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) +- Системная таблица [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) ## allow_introspection_functions {#settings-allow_introspection_functions} @@ -1530,23 +1530,23 @@ ClickHouse генерирует исключение **См. также** - [Sampling Query Profiler](../optimizing-performance/sampling-query-profiler.md) -- Системная таблица [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) +- Системная таблица [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) -## input\_format\_parallel\_parsing {#input-format-parallel-parsing} +## input_format_parallel_parsing {#input-format-parallel-parsing} - Тип: bool - Значение по умолчанию: True Обеспечивает параллельный анализ форматов данных с сохранением порядка. Поддерживается только для форматов TSV, TKSV, CSV и JSONEachRow. -## min\_chunk\_bytes\_for\_parallel\_parsing {#min-chunk-bytes-for-parallel-parsing} +## min_chunk_bytes_for_parallel_parsing {#min-chunk-bytes-for-parallel-parsing} - Тип: unsigned int - Значение по умолчанию: 1 MiB Минимальный размер блока в байтах, который каждый поток будет анализировать параллельно. -## output\_format\_avro\_codec {#settings-output_format_avro_codec} +## output_format_avro_codec {#settings-output_format_avro_codec} Устанавливает кодек сжатия, используемый для вывода файла Avro. @@ -1560,7 +1560,7 @@ ClickHouse генерирует исключение Значение по умолчанию: `snappy` (если доступно) или `deflate`. -## output\_format\_avro\_sync\_interval {#settings-output_format_avro_sync_interval} +## output_format_avro_sync_interval {#settings-output_format_avro_sync_interval} Устанавливает минимальный размер данных (в байтах) между маркерами синхронизации для выходного файла Avro. @@ -1570,7 +1570,7 @@ ClickHouse генерирует исключение Значение по умолчанию: 32768 (32 KiB) -## background\_pool\_size {#background_pool_size} +## background_pool_size {#background_pool_size} Задает количество потоков для выполнения фоновых операций в движках таблиц (например, слияния в таблицах c движком [MergeTree](../../engines/table-engines/mergetree-family/index.md)). Настройка применяется при запуске сервера ClickHouse и не может быть изменена во пользовательском сеансе. Настройка позволяет управлять загрузкой процессора и диска. Чем меньше пулл, тем ниже нагрузка на CPU и диск, при этом фоновые процессы замедляются, что может повлиять на скорость выполнения запроса. @@ -1610,7 +1610,7 @@ ClickHouse генерирует исключение - [Движок Distributed](../../engines/table-engines/special/distributed.md#distributed) - [Управление распределёнными таблицами](../../sql-reference/statements/system.md#query-language-system-distributed) -## validate\_polygons {#validate_polygons} +## validate_polygons {#validate_polygons} Включает или отключает генерирование исключения в функции [pointInPolygon](../../sql-reference/functions/geo/index.md#pointinpolygon), если многоугольник самопересекающийся или самокасающийся. @@ -1706,7 +1706,7 @@ SELECT idx, i FROM null_in WHERE i IN (1, NULL) SETTINGS transform_null_in = 1; - [Обработка значения NULL в операторе IN](../../sql-reference/operators/in.md#in-null-processing) -## low\_cardinality\_max\_dictionary\_size {#low_cardinality_max_dictionary_size} +## low_cardinality_max_dictionary_size {#low_cardinality_max_dictionary_size} Задает максимальный размер общего глобального словаря (в строках) для типа данных `LowCardinality`, который может быть записан в файловую систему хранилища. Настройка предотвращает проблемы с оперативной памятью в случае неограниченного увеличения словаря. Все данные, которые не могут быть закодированы из-за ограничения максимального размера словаря, ClickHouse записывает обычным способом. @@ -1716,7 +1716,7 @@ SELECT idx, i FROM null_in WHERE i IN (1, NULL) SETTINGS transform_null_in = 1; Значение по умолчанию: 8192. -## low\_cardinality\_use\_single\_dictionary\_for\_part {#low_cardinality_use_single_dictionary_for_part} +## low_cardinality_use_single_dictionary_for_part {#low_cardinality_use_single_dictionary_for_part} Включает или выключает использование единого словаря для куска (парта). @@ -1729,7 +1729,7 @@ SELECT idx, i FROM null_in WHERE i IN (1, NULL) SETTINGS transform_null_in = 1; Значение по умолчанию: 0. -## low\_cardinality\_allow\_in\_native\_format {#low_cardinality_allow_in_native_format} +## low_cardinality_allow_in_native_format {#low_cardinality_allow_in_native_format} Разрешает или запрещает использование типа данных `LowCardinality` с форматом данных [Native](../../interfaces/formats.md#native). @@ -1744,7 +1744,7 @@ SELECT idx, i FROM null_in WHERE i IN (1, NULL) SETTINGS transform_null_in = 1; Значение по умолчанию: 1. -## allow\_suspicious\_low\_cardinality\_types {#allow_suspicious_low_cardinality_types} +## allow_suspicious_low_cardinality_types {#allow_suspicious_low_cardinality_types} Разрешает или запрещает использование типа данных `LowCardinality` с типами данных с фиксированным размером 8 байт или меньше: числовые типы данных и `FixedString (8_bytes_or_less)`. @@ -1803,7 +1803,7 @@ SELECT idx, i FROM null_in WHERE i IN (1, NULL) SETTINGS transform_null_in = 1; Значение по умолчанию: 16. -## format\_avro\_schema\_registry\_url {#format_avro_schema_registry_url} +## format_avro_schema_registry_url {#format_avro_schema_registry_url} Задает URL реестра схем [Confluent](https://docs.confluent.io/current/schema-registry/index.html) для использования с форматом [AvroConfluent](../../interfaces/formats.md#data-format-avro-confluent). @@ -1849,7 +1849,7 @@ SELECT idx, i FROM null_in WHERE i IN (1, NULL) SETTINGS transform_null_in = 1; - [min_insert_block_size_bytes](#min-insert-block-size-bytes) -## output\_format\_pretty\_grid\_charset {#output-format-pretty-grid-charset} +## output_format_pretty_grid_charset {#output-format-pretty-grid-charset} Позволяет изменить кодировку, которая используется для печати грид-границ. Доступны следующие кодировки: UTF-8, ASCII. diff --git a/docs/ru/operations/system-tables/asynchronous_metrics.md b/docs/ru/operations/system-tables/asynchronous_metrics.md index a0ab5a3b5d8..5ff010bc79f 100644 --- a/docs/ru/operations/system-tables/asynchronous_metrics.md +++ b/docs/ru/operations/system-tables/asynchronous_metrics.md @@ -33,7 +33,7 @@ SELECT * FROM system.asynchronous_metrics LIMIT 10 - [Мониторинг](../../operations/monitoring.md) — основы мониторинга в ClickHouse. - [system.metrics](#system_tables-metrics) — таблица с мгновенно вычисляемыми метриками. - [system.events](#system_tables-events) — таблица с количеством произошедших событий. -- [system.metric\_log](#system_tables-metric_log) — таблица фиксирующая историю значений метрик из `system.metrics` и `system.events`. +- [system.metric_log](#system_tables-metric_log) — таблица фиксирующая историю значений метрик из `system.metrics` и `system.events`. [Оригинальная статья](https://clickhouse.tech/docs/ru/operations/system_tables/asynchronous_metrics) \ No newline at end of file diff --git a/docs/ru/operations/system-tables/events.md b/docs/ru/operations/system-tables/events.md index 9843e170f3d..0a48617bb5c 100644 --- a/docs/ru/operations/system-tables/events.md +++ b/docs/ru/operations/system-tables/events.md @@ -26,9 +26,9 @@ SELECT * FROM system.events LIMIT 5 **Смотрите также** -- [system.asynchronous\_metrics](#system_tables-asynchronous_metrics) — таблица с периодически вычисляемыми метриками. +- [system.asynchronous_metrics](#system_tables-asynchronous_metrics) — таблица с периодически вычисляемыми метриками. - [system.metrics](#system_tables-metrics) — таблица с мгновенно вычисляемыми метриками. -- [system.metric\_log](#system_tables-metric_log) — таблица фиксирующая историю значений метрик из `system.metrics` и `system.events`. +- [system.metric_log](#system_tables-metric_log) — таблица фиксирующая историю значений метрик из `system.metrics` и `system.events`. - [Мониторинг](../../operations/monitoring.md) — основы мониторинга в ClickHouse. [Оригинальная статья](https://clickhouse.tech/docs/ru/operations/system_tables/events) diff --git a/docs/ru/operations/system-tables/graphite_retentions.md b/docs/ru/operations/system-tables/graphite_retentions.md index 846f3cbb473..66fca7ba299 100644 --- a/docs/ru/operations/system-tables/graphite_retentions.md +++ b/docs/ru/operations/system-tables/graphite_retentions.md @@ -1,6 +1,6 @@ # system.graphite_retentions {#system-graphite-retentions} -Содержит информацию о том, какие параметры [graphite\_rollup](../server-configuration-parameters/settings.md#server_configuration_parameters-graphite) используются в таблицах с движками [\*GraphiteMergeTree](../../engines/table-engines/mergetree-family/graphitemergetree.md). +Содержит информацию о том, какие параметры [graphite_rollup](../server-configuration-parameters/settings.md#server_configuration_parameters-graphite) используются в таблицах с движками [\*GraphiteMergeTree](../../engines/table-engines/mergetree-family/graphitemergetree.md). Столбцы: diff --git a/docs/ru/operations/system-tables/metric_log.md b/docs/ru/operations/system-tables/metric_log.md index 9bdee44b4f3..32ebe91dd91 100644 --- a/docs/ru/operations/system-tables/metric_log.md +++ b/docs/ru/operations/system-tables/metric_log.md @@ -49,7 +49,7 @@ CurrentMetric_ReplicatedChecks: 0 **Смотрите также** -- [system.asynchronous\_metrics](#system_tables-asynchronous_metrics) — таблица с периодически вычисляемыми метриками. +- [system.asynchronous_metrics](#system_tables-asynchronous_metrics) — таблица с периодически вычисляемыми метриками. - [system.events](#system_tables-events) — таблица с количеством произошедших событий. - [system.metrics](#system_tables-metrics) — таблица с мгновенно вычисляемыми метриками. - [Мониторинг](../../operations/monitoring.md) — основы мониторинга в ClickHouse. diff --git a/docs/ru/operations/system-tables/metrics.md b/docs/ru/operations/system-tables/metrics.md index 2b6e776615f..39ebab58624 100644 --- a/docs/ru/operations/system-tables/metrics.md +++ b/docs/ru/operations/system-tables/metrics.md @@ -33,9 +33,9 @@ SELECT * FROM system.metrics LIMIT 10 **Смотрите также** -- [system.asynchronous\_metrics](#system_tables-asynchronous_metrics) — таблица с периодически вычисляемыми метриками. +- [system.asynchronous_metrics](#system_tables-asynchronous_metrics) — таблица с периодически вычисляемыми метриками. - [system.events](#system_tables-events) — таблица с количеством произошедших событий. -- [system.metric\_log](#system_tables-metric_log) — таблица фиксирующая историю значений метрик из `system.metrics` и `system.events`. +- [system.metric_log](#system_tables-metric_log) — таблица фиксирующая историю значений метрик из `system.metrics` и `system.events`. - [Мониторинг](../../operations/monitoring.md) — основы мониторинга в ClickHouse. [Оригинальная статья](https://clickhouse.tech/docs/ru/operations/system_tables/metrics) diff --git a/docs/ru/operations/system-tables/processes.md b/docs/ru/operations/system-tables/processes.md index 83247cffd92..c9216e162b3 100644 --- a/docs/ru/operations/system-tables/processes.md +++ b/docs/ru/operations/system-tables/processes.md @@ -10,7 +10,7 @@ - `rows_read` (UInt64) – количество прочитанных строк. При распределённой обработке запроса на сервере-инициаторе запроса представляет собой сумму по всем удалённым серверам. - `bytes_read` (UInt64) – количество прочитанных из таблиц байт, в несжатом виде. При распределённой обработке запроса на сервере-инициаторе запроса представляет собой сумму по всем удалённым серверам. - `total_rows_approx` (UInt64) – приблизительная оценка общего количества строк, которые должны быть прочитаны. При распределённой обработке запроса, на сервере-инициаторе запроса, представляет собой сумму по всем удалённым серверам. Может обновляться в процессе выполнения запроса, когда становятся известны новые источники для обработки. -- `memory_usage` (UInt64) – потребление памяти запросом. Может не учитывать некоторые виды выделенной памяти. Смотрите описание настройки [max\_memory\_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage). +- `memory_usage` (UInt64) – потребление памяти запросом. Может не учитывать некоторые виды выделенной памяти. Смотрите описание настройки [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage). - `query` (String) – текст запроса. Для запросов `INSERT` не содержит встаявляемые данные. - `query_id` (String) – идентификатор запроса, если был задан. diff --git a/docs/ru/operations/system-tables/query_log.md b/docs/ru/operations/system-tables/query_log.md index 5aa3b2d6ae5..340e77d85da 100644 --- a/docs/ru/operations/system-tables/query_log.md +++ b/docs/ru/operations/system-tables/query_log.md @@ -75,7 +75,7 @@ ClickHouse не удаляет данные из таблица автомати - `quota_key` ([String](../../sql-reference/data-types/string.md)) — «ключ квоты» из настроек [квот](quotas.md) (см. `keyed`). - `revision` ([UInt32](../../sql-reference/data-types/int-uint.md)) — ревизия ClickHouse. - `thread_numbers` ([Array(UInt32)](../../sql-reference/data-types/array.md)) — количество потоков, участвующих в обработке запросов. -- `ProfileEvents.Names` ([Array(String)](../../sql-reference/data-types/array.md)) — Счетчики для изменения различных метрик. Описание метрик можно получить из таблицы [system.events](#system_tables-events)(\#system\_tables-events +- `ProfileEvents.Names` ([Array(String)](../../sql-reference/data-types/array.md)) — Счетчики для изменения различных метрик. Описание метрик можно получить из таблицы [system.events](#system_tables-events)(#system_tables-events - `ProfileEvents.Values` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — метрики, перечисленные в столбце `ProfileEvents.Names`. - `Settings.Names` ([Array(String)](../../sql-reference/data-types/array.md)) — имена настроек, которые меняются, когда клиент выполняет запрос. Чтобы разрешить логирование изменений настроек, установите параметр `log_query_settings` равным 1. - `Settings.Values` ([Array(String)](../../sql-reference/data-types/array.md)) — Значения настроек, которые перечислены в столбце `Settings.Names`. diff --git a/docs/ru/operations/system-tables/replicas.md b/docs/ru/operations/system-tables/replicas.md index 6a097e75818..bb2e6a550a0 100644 --- a/docs/ru/operations/system-tables/replicas.md +++ b/docs/ru/operations/system-tables/replicas.md @@ -63,9 +63,9 @@ active_replicas: 2 - `parts_to_check` (`UInt32`) - количество кусков с данными в очереди на проверку. Кусок помещается в очередь на проверку, если есть подозрение, что он может быть битым. - `zookeeper_path` (`String`) - путь к данным таблицы в ZK. - `replica_name` (`String`) - имя реплики в ZK; разные реплики одной таблицы имеют разное имя. -- `replica_path` (`String`) - путь к данным реплики в ZK. То же самое, что конкатенация zookeeper\_path/replicas/replica\_path. +- `replica_path` (`String`) - путь к данным реплики в ZK. То же самое, что конкатенация zookeeper_path/replicas/replica_path. - `columns_version` (`Int32`) - номер версии структуры таблицы. Обозначает, сколько раз был сделан ALTER. Если на репликах разные версии, значит некоторые реплики сделали ещё не все ALTER-ы. -- `queue_size` (`UInt32`) - размер очереди действий, которые предстоит сделать. К действиям относятся вставки блоков данных, слияния, и некоторые другие действия. Как правило, совпадает с future\_parts. +- `queue_size` (`UInt32`) - размер очереди действий, которые предстоит сделать. К действиям относятся вставки блоков данных, слияния, и некоторые другие действия. Как правило, совпадает с future_parts. - `inserts_in_queue` (`UInt32`) - количество вставок блоков данных, которые предстоит сделать. Обычно вставки должны быстро реплицироваться. Если величина большая - значит что-то не так. - `merges_in_queue` (`UInt32`) - количество слияний, которые предстоит сделать. Бывают длинные слияния - то есть, это значение может быть больше нуля продолжительное время. - `part_mutations_in_queue` (`UInt32`) - количество мутаций, которые предстоит сделать. @@ -77,14 +77,14 @@ active_replicas: 2 Следующие 4 столбца имеют ненулевое значение только если активна сессия с ZK. - `log_max_index` (`UInt64`) - максимальный номер записи в общем логе действий. -- `log_pointer` (`UInt64`) - максимальный номер записи из общего лога действий, которую реплика скопировала в свою очередь для выполнения, плюс единица. Если log\_pointer сильно меньше log\_max\_index, значит что-то не так. +- `log_pointer` (`UInt64`) - максимальный номер записи из общего лога действий, которую реплика скопировала в свою очередь для выполнения, плюс единица. Если log_pointer сильно меньше log_max_index, значит что-то не так. - `last_queue_update` (`DateTime`) - When the queue was updated last time. - `absolute_delay` (`UInt64`) - How big lag in seconds the current replica has. - `total_replicas` (`UInt8`) - общее число известных реплик этой таблицы. - `active_replicas` (`UInt8`) - число реплик этой таблицы, имеющих сессию в ZK; то есть, число работающих реплик. Если запрашивать все столбцы, то таблица может работать слегка медленно, так как на каждую строчку делается несколько чтений из ZK. -Если не запрашивать последние 4 столбца (log\_max\_index, log\_pointer, total\_replicas, active\_replicas), то таблица работает быстро. +Если не запрашивать последние 4 столбца (log_max_index, log_pointer, total_replicas, active_replicas), то таблица работает быстро. Например, так можно проверить, что всё хорошо: diff --git a/docs/ru/operations/system-tables/trace_log.md b/docs/ru/operations/system-tables/trace_log.md index dec74b48487..8bab8ff646c 100644 --- a/docs/ru/operations/system-tables/trace_log.md +++ b/docs/ru/operations/system-tables/trace_log.md @@ -2,7 +2,7 @@ Содержит экземпляры трассировки стека адресов вызова, собранные с помощью семплирующего профайлера запросов. -ClickHouse создает эту таблицу когда утсановлена настройка [trace\_log](../server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) в конфигурационном файле сервереа. А также настройки [query_profiler_real_time_period_ns](../settings/settings.md#query_profiler_real_time_period_ns) и [query_profiler_cpu_time_period_ns](../settings/settings.md#query_profiler_cpu_time_period_ns). +ClickHouse создает эту таблицу когда утсановлена настройка [trace_log](../server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) в конфигурационном файле сервереа. А также настройки [query_profiler_real_time_period_ns](../settings/settings.md#query_profiler_real_time_period_ns) и [query_profiler_cpu_time_period_ns](../settings/settings.md#query_profiler_cpu_time_period_ns). Для анализа stack traces, используйте функции интроспекции `addressToLine`, `addressToSymbol` и `demangle`. diff --git a/docs/ru/operations/tips.md b/docs/ru/operations/tips.md index e537f6ef5c1..f076fab1311 100644 --- a/docs/ru/operations/tips.md +++ b/docs/ru/operations/tips.md @@ -52,7 +52,7 @@ $ echo 'madvise' | sudo tee /sys/kernel/mm/transparent_hugepage/enabled Если бюджет позволяет, лучше выбрать RAID-10. На более чем 4 дисках вместо RAID-5 нужно использовать RAID-6 (предпочтительнее) или RAID-50. -При использовании RAID-5, RAID-6 или RAID-50, нужно всегда увеличивать stripe\_cache\_size, так как значение по умолчанию выбрано не самым удачным образом. +При использовании RAID-5, RAID-6 или RAID-50, нужно всегда увеличивать stripe_cache_size, так как значение по умолчанию выбрано не самым удачным образом. ``` bash $ echo 4096 | sudo tee /sys/block/md2/md/stripe_cache_size diff --git a/docs/ru/sql-reference/aggregate-functions/reference/count.md b/docs/ru/sql-reference/aggregate-functions/reference/count.md index 68f7d3dcc6b..d99c3b2aeb2 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/count.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/count.md @@ -27,7 +27,7 @@ ClickHouse поддерживает следующие виды синтакси **Подробности** -ClickHouse поддерживает синтаксис `COUNT(DISTINCT ...)`. Поведение этой конструкции зависит от настройки [count\_distinct\_implementation](../../../operations/settings/settings.md#settings-count_distinct_implementation). Она определяет, какая из функций [uniq\*](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) используется для выполнения операции. По умолчанию — функция [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact). +ClickHouse поддерживает синтаксис `COUNT(DISTINCT ...)`. Поведение этой конструкции зависит от настройки [count_distinct_implementation](../../../operations/settings/settings.md#settings-count_distinct_implementation). Она определяет, какая из функций [uniq\*](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) используется для выполнения операции. По умолчанию — функция [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact). Запрос `SELECT count() FROM table` не оптимизирован, поскольку количество записей в таблице не хранится отдельно. Он выбирает небольшой столбец из таблицы и подсчитывает количество значений в нём. diff --git a/docs/ru/sql-reference/data-types/aggregatefunction.md b/docs/ru/sql-reference/data-types/aggregatefunction.md index 22825deb3eb..018d38d825e 100644 --- a/docs/ru/sql-reference/data-types/aggregatefunction.md +++ b/docs/ru/sql-reference/data-types/aggregatefunction.md @@ -7,7 +7,7 @@ toc_title: AggregateFunction Агрегатные функции могут обладать определяемым реализацией промежуточным состоянием, которое может быть сериализовано в тип данных, соответствующий AggregateFunction(…), и быть записано в таблицу обычно посредством [материализованного представления] (../../sql-reference/statements/create.md#create-view). Чтобы получить промежуточное состояние, обычно используются агрегатные функции с суффиксом `-State`. Чтобы в дальнейшем получить агрегированные данные необходимо использовать те же агрегатные функции с суффиксом `-Merge`. -`AggregateFunction(name, types\_of\_arguments…)` — параметрический тип данных. +`AggregateFunction(name, types_of_arguments…)` — параметрический тип данных. **Параметры** diff --git a/docs/ru/sql-reference/data-types/datetime.md b/docs/ru/sql-reference/data-types/datetime.md index e7c3d8c24f1..d60e81c74d5 100644 --- a/docs/ru/sql-reference/data-types/datetime.md +++ b/docs/ru/sql-reference/data-types/datetime.md @@ -24,7 +24,7 @@ DateTime([timezone]) ClickHouse отображает значения типа `DateTime` в формате `YYYY-MM-DD hh:mm:ss`. Отображение можно поменять с помощью функции [formatDateTime](../../sql-reference/data-types/datetime.md#formatdatetime). -При вставке данных в ClickHouse, можно использовать различные форматы даты и времени в зависимости от значения настройки [date\_time\_input\_format](../../operations/settings/settings.md#settings-date_time_input_format). +При вставке данных в ClickHouse, можно использовать различные форматы даты и времени в зависимости от значения настройки [date_time_input_format](../../operations/settings/settings.md#settings-date_time_input_format). ## Примеры {#primery} diff --git a/docs/ru/sql-reference/data-types/decimal.md b/docs/ru/sql-reference/data-types/decimal.md index aa2886d7ec0..29192cb8819 100644 --- a/docs/ru/sql-reference/data-types/decimal.md +++ b/docs/ru/sql-reference/data-types/decimal.md @@ -79,7 +79,7 @@ SELECT toDecimal32(4.2, 8) AS x, 6 * x DB::Exception: Decimal math overflow. ``` -Проверка переполнения приводит к замедлению операций. При уверенности, что типа результата хватит для его записи проверку переполнения можно отключить настройкой decimal\_check\_overflow. В этом случае при переполнении вернется неверное значение: +Проверка переполнения приводит к замедлению операций. При уверенности, что типа результата хватит для его записи проверку переполнения можно отключить настройкой decimal_check_overflow. В этом случае при переполнении вернется неверное значение: ``` sql SET decimal_check_overflow = 0; diff --git a/docs/ru/sql-reference/data-types/fixedstring.md b/docs/ru/sql-reference/data-types/fixedstring.md index 1d801bb6e92..4a26b2f76af 100644 --- a/docs/ru/sql-reference/data-types/fixedstring.md +++ b/docs/ru/sql-reference/data-types/fixedstring.md @@ -15,7 +15,7 @@ Примеры значений, которые можно эффективно хранить в столбцах типа `FixedString`: - Двоичное представление IP-адреса (`FixedString(16)` для IPv6). -- Коды языков (ru\_RU, en\_US … ). +- Коды языков (ru_RU, en_US … ). - Коды валют (USD, RUB … ). - Двоичное представление хэшей (`FixedString(16)` для MD5, `FixedString(32)` для SHA256). diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md index 1f09eb28d2e..0869b409b0b 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md @@ -20,7 +20,7 @@ ClickHouse поддерживает иерархические словари с Эту иерархию можно выразить в виде следующей таблицы-словаря. -| region\_id | parent\_region | region\_name | +| region_id | parent_region | region_name | |------------|----------------|---------------| | 1 | 0 | Russia | | 2 | 1 | Moscow | diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index d21b4b18151..61275cc6652 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -2,7 +2,7 @@ Словари можно размещать в памяти множеством способов. -Рекомендуем [flat](#flat), [hashed](#dicts-external_dicts_dict_layout-hashed) и [complex\_key\_hashed](#complex-key-hashed). Скорость обработки словарей при этом максимальна. +Рекомендуем [flat](#flat), [hashed](#dicts-external_dicts_dict_layout-hashed) и [complex_key_hashed](#complex-key-hashed). Скорость обработки словарей при этом максимальна. Размещение с кэшированием не рекомендуется использовать из-за потенциально низкой производительности и сложностей в подборе оптимальных параметров. Читайте об этом подробнее в разделе «[cache](#cache)». @@ -47,14 +47,14 @@ LAYOUT(LAYOUT_TYPE(param value)) -- layout settings - [flat](#flat) - [hashed](#dicts-external_dicts_dict_layout-hashed) -- [sparse\_hashed](#dicts-external_dicts_dict_layout-sparse_hashed) +- [sparse_hashed](#dicts-external_dicts_dict_layout-sparse_hashed) - [cache](#cache) - [direct](#direct) -- [range\_hashed](#range-hashed) -- [complex\_key\_hashed](#complex-key-hashed) -- [complex\_key\_cache](#complex-key-cache) -- [complex\_key\_direct](#complex-key-direct) -- [ip\_trie](#ip-trie) +- [range_hashed](#range-hashed) +- [complex_key_hashed](#complex-key-hashed) +- [complex_key_cache](#complex-key-cache) +- [complex_key_direct](#complex-key-direct) +- [ip_trie](#ip-trie) ### flat {#flat} @@ -100,7 +100,7 @@ LAYOUT(FLAT()) LAYOUT(HASHED()) ``` -### sparse\_hashed {#dicts-external_dicts_dict_layout-sparse_hashed} +### sparse_hashed {#dicts-external_dicts_dict_layout-sparse_hashed} Аналогичен `hashed`, но при этом занимает меньше места в памяти и генерирует более высокую загрузку CPU. @@ -118,7 +118,7 @@ LAYOUT(HASHED()) LAYOUT(SPARSE_HASHED()) ``` -### complex\_key\_hashed {#complex-key-hashed} +### complex_key_hashed {#complex-key-hashed} Тип размещения предназначен для использования с составными [ключами](external-dicts-dict-structure.md). Аналогичен `hashed`. @@ -136,7 +136,7 @@ LAYOUT(SPARSE_HASHED()) LAYOUT(COMPLEX_KEY_HASHED()) ``` -### range\_hashed {#range-hashed} +### range_hashed {#range-hashed} Словарь хранится в оперативной памяти в виде хэш-таблицы с упорядоченным массивом диапазонов и соответствующих им значений. @@ -290,7 +290,7 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000)) !!! warning "Warning" Не используйте в качестве источника ClickHouse, поскольку он медленно обрабатывает запросы со случайным чтением. -### complex\_key\_cache {#complex-key-cache} +### complex_key_cache {#complex-key-cache} Тип размещения предназначен для использования с составными [ключами](external-dicts-dict-structure.md). Аналогичен `cache`. @@ -316,11 +316,11 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000)) LAYOUT(DIRECT()) ``` -### complex\_key\_direct {#complex-key-direct} +### complex_key_direct {#complex-key-direct} Тип размещения предназначен для использования с составными [ключами](external-dicts-dict-structure.md). Аналогичен `direct`. -### ip\_trie {#ip-trie} +### ip_trie {#ip-trie} Тип размещения предназначен для сопоставления префиксов сети (IP адресов) с метаданными, такими как ASN. diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts.md index 5878b9206eb..734de8cffdf 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts.md @@ -7,9 +7,9 @@ ClickHouse: - Периодически обновляет их и динамически подгружает отсутствующие значения. - Позволяет создавать внешние словари с помощью xml-файлов или [DDL-запросов](../../statements/create/dictionary.md#create-dictionary-query). -Конфигурация внешних словарей может находится в одном или нескольких xml-файлах. Путь к конфигурации указывается в параметре [dictionaries\_config](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config). +Конфигурация внешних словарей может находится в одном или нескольких xml-файлах. Путь к конфигурации указывается в параметре [dictionaries_config](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config). -Словари могут загружаться при старте сервера или при первом использовании, в зависимости от настройки [dictionaries\_lazy\_load](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load). +Словари могут загружаться при старте сервера или при первом использовании, в зависимости от настройки [dictionaries_lazy_load](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load). Системная таблица [system.dictionaries](../../../operations/system-tables/dictionaries.md#system_tables-dictionaries) содержит информацию о словарях, сконфигурированных на сервере. Для каждого словаря там можно найти: diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index c4bc42c6ece..cdc869190e4 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -189,7 +189,7 @@ WITH toDateTime64('2020-01-01 10:20:30.999', 3) AS dt64 SELECT toStartOfSecond(d Округляет дату-с-временем вниз до начала пятнадцатиминутного интервала. -## toStartOfInterval(time\_or\_data, INTERVAL x unit \[, time\_zone\]) {#tostartofintervaltime-or-data-interval-x-unit-time-zone} +## toStartOfInterval(time_or_data, INTERVAL x unit \[, time_zone\]) {#tostartofintervaltime-or-data-interval-x-unit-time-zone} Обобщение остальных функций `toStartOf*`. Например, `toStartOfInterval(t, INTERVAL 1 year)` возвращает то же самое, что и `toStartOfYear(t)`, diff --git a/docs/ru/sql-reference/functions/hash-functions.md b/docs/ru/sql-reference/functions/hash-functions.md index 682f1410cc6..201fc934cea 100644 --- a/docs/ru/sql-reference/functions/hash-functions.md +++ b/docs/ru/sql-reference/functions/hash-functions.md @@ -312,7 +312,7 @@ SELECT metroHash64(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00: Имеет два параметра: ключ типа UInt64 и количество бакетов. Возвращает значение типа Int32. Дополнительные сведения смотрите по ссылке: [JumpConsistentHash](https://arxiv.org/pdf/1406.2294.pdf) -## murmurHash2\_32, murmurHash2\_64 {#murmurhash2-32-murmurhash2-64} +## murmurHash2_32, murmurHash2_64 {#murmurhash2-32-murmurhash2-64} Генерирует значение [MurmurHash2](https://github.com/aappleby/smhasher). @@ -380,7 +380,7 @@ SELECT └──────────────────────┴─────────────────────┘ ``` -## murmurHash3\_32, murmurHash3\_64 {#murmurhash3-32-murmurhash3-64} +## murmurHash3_32, murmurHash3_64 {#murmurhash3-32-murmurhash3-64} Генерирует значение [MurmurHash3](https://github.com/aappleby/smhasher). @@ -410,7 +410,7 @@ SELECT murmurHash3_32(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23: └─────────────┴────────┘ ``` -## murmurHash3\_128 {#murmurhash3-128} +## murmurHash3_128 {#murmurhash3-128} Генерирует значение [MurmurHash3](https://github.com/aappleby/smhasher). diff --git a/docs/ru/sql-reference/functions/introspection.md b/docs/ru/sql-reference/functions/introspection.md index 655c4be8318..c09efd35093 100644 --- a/docs/ru/sql-reference/functions/introspection.md +++ b/docs/ru/sql-reference/functions/introspection.md @@ -9,7 +9,7 @@ - Установите пакет `clickhouse-common-static-dbg`. -- Установите настройку [allow\_introspection\_functions](../../operations/settings/settings.md#settings-allow_introspection_functions) в 1. +- Установите настройку [allow_introspection_functions](../../operations/settings/settings.md#settings-allow_introspection_functions) в 1. Из соображений безопасности данные функции отключены по умолчанию. diff --git a/docs/ru/sql-reference/functions/json-functions.md b/docs/ru/sql-reference/functions/json-functions.md index 1079702e709..752b70b7c5f 100644 --- a/docs/ru/sql-reference/functions/json-functions.md +++ b/docs/ru/sql-reference/functions/json-functions.md @@ -68,7 +68,7 @@ SELECT isValidJSON('{"a": "hello", "b": [-100, 200.0, 300]}') = 1 SELECT isValidJSON('not a json') = 0 ``` -## JSONHas(json\[, indices\_or\_keys\]…) {#jsonhasjson-indices-or-keys} +## JSONHas(json\[, indices_or_keys\]…) {#jsonhasjson-indices-or-keys} Если значение существует в документе JSON, то возвращается `1`. @@ -101,7 +101,7 @@ SELECT JSONExtractKey('{"a": "hello", "b": [-100, 200.0, 300]}', -2) = 'a' SELECT JSONExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 1) = 'hello' ``` -## JSONLength(json\[, indices\_or\_keys\]…) {#jsonlengthjson-indices-or-keys} +## JSONLength(json\[, indices_or_keys\]…) {#jsonlengthjson-indices-or-keys} Возвращает длину массива JSON или объекта JSON. @@ -114,7 +114,7 @@ SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 3 SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}') = 2 ``` -## JSONType(json\[, indices\_or\_keys\]…) {#jsontypejson-indices-or-keys} +## JSONType(json\[, indices_or_keys\]…) {#jsontypejson-indices-or-keys} Возвращает тип значения JSON. @@ -128,13 +128,13 @@ SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'a') = 'String' SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 'Array' ``` -## JSONExtractUInt(json\[, indices\_or\_keys\]…) {#jsonextractuintjson-indices-or-keys} +## JSONExtractUInt(json\[, indices_or_keys\]…) {#jsonextractuintjson-indices-or-keys} -## JSONExtractInt(json\[, indices\_or\_keys\]…) {#jsonextractintjson-indices-or-keys} +## JSONExtractInt(json\[, indices_or_keys\]…) {#jsonextractintjson-indices-or-keys} -## JSONExtractFloat(json\[, indices\_or\_keys\]…) {#jsonextractfloatjson-indices-or-keys} +## JSONExtractFloat(json\[, indices_or_keys\]…) {#jsonextractfloatjson-indices-or-keys} -## JSONExtractBool(json\[, indices\_or\_keys\]…) {#jsonextractbooljson-indices-or-keys} +## JSONExtractBool(json\[, indices_or_keys\]…) {#jsonextractbooljson-indices-or-keys} Парсит JSON и извлекает значение. Эти функции аналогичны функциям `visitParam`. @@ -148,7 +148,7 @@ SELECT JSONExtractFloat('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 2) = 200 SELECT JSONExtractUInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', -1) = 300 ``` -## JSONExtractString(json\[, indices\_or\_keys\]…) {#jsonextractstringjson-indices-or-keys} +## JSONExtractString(json\[, indices_or_keys\]…) {#jsonextractstringjson-indices-or-keys} Парсит JSON и извлекает строку. Эта функция аналогична функции `visitParamExtractString`. @@ -166,7 +166,7 @@ SELECT JSONExtractString('{"abc":"\\u263"}', 'abc') = '' SELECT JSONExtractString('{"abc":"hello}', 'abc') = '' ``` -## JSONExtract(json\[, indices\_or\_keys…\], Return\_type) {#jsonextractjson-indices-or-keys-return-type} +## JSONExtract(json\[, indices_or_keys…\], Return_type) {#jsonextractjson-indices-or-keys-return-type} Парсит JSON и извлекает значение с заданным типом данных. @@ -187,7 +187,7 @@ SELECT JSONExtract('{"day": "Thursday"}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday SELECT JSONExtract('{"day": 5}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \'Tuesday\' = 2, \'Wednesday\' = 3, \'Thursday\' = 4, \'Friday\' = 5, \'Saturday\' = 6)') = 'Friday' ``` -## JSONExtractKeysAndValues(json\[, indices\_or\_keys…\], Value\_type) {#jsonextractkeysandvaluesjson-indices-or-keys-value-type} +## JSONExtractKeysAndValues(json\[, indices_or_keys…\], Value_type) {#jsonextractkeysandvaluesjson-indices-or-keys-value-type} Разбор пар ключ-значение из JSON, где значение имеет тип данных ClickHouse. @@ -197,7 +197,7 @@ SELECT JSONExtract('{"day": 5}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \ SELECT JSONExtractKeysAndValues('{"x": {"a": 5, "b": 7, "c": 11}}', 'x', 'Int8') = [('a',5),('b',7),('c',11)]; ``` -## JSONExtractRaw(json\[, indices\_or\_keys\]…) {#jsonextractrawjson-indices-or-keys} +## JSONExtractRaw(json\[, indices_or_keys\]…) {#jsonextractrawjson-indices-or-keys} Возвращает часть JSON в виде строки, содержащей неразобранную подстроку. @@ -209,7 +209,7 @@ SELECT JSONExtractKeysAndValues('{"x": {"a": 5, "b": 7, "c": 11}}', 'x', 'Int8') SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = '[-100, 200.0, 300]' ``` -## JSONExtractArrayRaw(json\[, indices\_or\_keys\]…) {#jsonextractarrayrawjson-indices-or-keys} +## JSONExtractArrayRaw(json\[, indices_or_keys\]…) {#jsonextractarrayrawjson-indices-or-keys} Возвращает массив из элементов JSON массива, каждый из которых представлен в виде строки с неразобранными подстроками из JSON. diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index f162df4a703..2179f6bd1c9 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -410,7 +410,7 @@ ORDER BY h ASC Преобразовать значение согласно явно указанному отображению одних элементов на другие. Имеется два варианта функции: -### transform(x, array\_from, array\_to, default) {#transformx-array-from-array-to-default} +### transform(x, array_from, array_to, default) {#transformx-array-from-array-to-default} `x` - что преобразовывать. @@ -430,7 +430,7 @@ ORDER BY h ASC При этом, где обозначена одна и та же буква (T или U), могут быть, в случае числовых типов, не совпадающие типы, а типы, для которых есть общий тип. Например, первый аргумент может иметь тип Int64, а второй - Array(UInt16). -Если значение x равно одному из элементов массива array\_from, то возвращает соответствующий (такой же по номеру) элемент массива array\_to; иначе возвращает default. Если имеется несколько совпадающих элементов в array\_from, то возвращает какой-нибудь из соответствующих. +Если значение x равно одному из элементов массива array_from, то возвращает соответствующий (такой же по номеру) элемент массива array_to; иначе возвращает default. Если имеется несколько совпадающих элементов в array_from, то возвращает какой-нибудь из соответствующих. Пример: @@ -452,10 +452,10 @@ ORDER BY c DESC └───────────┴────────┘ ``` -### transform(x, array\_from, array\_to) {#transformx-array-from-array-to} +### transform(x, array_from, array_to) {#transformx-array-from-array-to} Отличается от первого варианта отсутствующим аргументом default. -Если значение x равно одному из элементов массива array\_from, то возвращает соответствующий (такой же по номеру) элемент массива array\_to; иначе возвращает x. +Если значение x равно одному из элементов массива array_from, то возвращает соответствующий (такой же по номеру) элемент массива array_to; иначе возвращает x. Типы: @@ -737,7 +737,7 @@ WHERE diff != 1 ## runningDifferenceStartingWithFirstValue {#runningdifferencestartingwithfirstvalue} -То же, что и \[runningDifference\] (./other\_functions.md \# other\_functions-runningdifference), но в первой строке возвращается значение первой строки, а не ноль. +То же, что и \[runningDifference\] (./other_functions.md # other_functions-runningdifference), но в первой строке возвращается значение первой строки, а не ноль. ## MACNumToString(num) {#macnumtostringnum} @@ -1188,7 +1188,7 @@ joinGet(join_storage_table_name, `value_column`, join_keys) Возвращает значение по списку ключей. -Если значения не существует в исходной таблице, вернется `0` или `null` в соответствии с настройками [join\_use\_nulls](../../operations/settings/settings.md#join_use_nulls). +Если значения не существует в исходной таблице, вернется `0` или `null` в соответствии с настройками [join_use_nulls](../../operations/settings/settings.md#join_use_nulls). Подробнее о настройке `join_use_nulls` в [операциях Join](../../sql-reference/functions/other-functions.md). @@ -1227,16 +1227,16 @@ SELECT joinGet(db_test.id_val,'val',toUInt32(number)) from numbers(4) SETTINGS j └──────────────────────────────────────────────────┘ ``` -## modelEvaluate(model\_name, …) {#function-modelevaluate} +## modelEvaluate(model_name, …) {#function-modelevaluate} Оценивает внешнюю модель. Принимает на вход имя и аргументы модели. Возвращает Float64. -## throwIf(x\[, custom\_message\]) {#throwifx-custom-message} +## throwIf(x\[, custom_message\]) {#throwifx-custom-message} Бросает исключение, если аргумент не равен нулю. -custom\_message - необязательный параметр, константная строка, задает текст сообщения об ошибке. +custom_message - необязательный параметр, константная строка, задает текст сообщения об ошибке. ``` sql SELECT throwIf(number = 3, 'Too many') FROM numbers(10); diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 134c096de84..58ed582b399 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -24,12 +24,12 @@ Возвращает длину строки в кодовых точках Unicode (не символах), при допущении, что строка содержит набор байтов, являющийся текстом в кодировке UTF-8. Если допущение не выполнено, то возвращает какой-нибудь результат (не кидает исключение). Тип результата — UInt64. -## char\_length, CHAR\_LENGTH {#char-length} +## char_length, CHAR_LENGTH {#char-length} Возвращает длину строки в кодовых точках Unicode (не символах), при допущении, что строка содержит набор байтов, являющийся текстом в кодировке UTF-8. Если допущение не выполнено, возвращает какой-нибудь результат (не кидает исключение). Тип результата — UInt64. -## character\_length, CHARACTER\_LENGTH {#character-length} +## character_length, CHARACTER_LENGTH {#character-length} Возвращает длину строки в кодовых точках Unicode (не символах), при допущении, что строка содержит набор байтов, являющийся текстом в кодировке UTF-8. Если допущение не выполнено, возвращает какой-нибудь результат (не кидает исключение). Тип результата — UInt64. @@ -70,7 +70,7 @@ toValidUTF8( input_string ) Параметры: -- input\_string — произвольный набор байтов, представленный как объект типа [String](../../sql-reference/functions/string-functions.md). +- input_string — произвольный набор байтов, представленный как объект типа [String](../../sql-reference/functions/string-functions.md). Возвращаемое значение: Корректная строка UTF-8. diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 7571fcf6c0b..fbebc066abc 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -245,7 +245,7 @@ YYYY-MM-DD YYYY-MM-DD hh:mm:ss ``` -В качестве исключения, если делается преобразование из числа типа UInt32, Int32, UInt64, Int64 в Date, и если число больше или равно 65536, то число рассматривается как unix timestamp (а не как число дней) и округляется до даты. Это позволяет поддержать распространённый случай, когда пишут toDate(unix\_timestamp), что иначе было бы ошибкой и требовало бы написания более громоздкого toDate(toDateTime(unix\_timestamp)) +В качестве исключения, если делается преобразование из числа типа UInt32, Int32, UInt64, Int64 в Date, и если число больше или равно 65536, то число рассматривается как unix timestamp (а не как число дней) и округляется до даты. Это позволяет поддержать распространённый случай, когда пишут toDate(unix_timestamp), что иначе было бы ошибкой и требовало бы написания более громоздкого toDate(toDateTime(unix_timestamp)) Преобразование между датой и датой-с-временем производится естественным образом: добавлением нулевого времени или отбрасыванием времени. diff --git a/docs/ru/sql-reference/functions/url-functions.md b/docs/ru/sql-reference/functions/url-functions.md index 843b75567ee..54b752d7c68 100644 --- a/docs/ru/sql-reference/functions/url-functions.md +++ b/docs/ru/sql-reference/functions/url-functions.md @@ -116,11 +116,11 @@ SELECT topLevelDomain('svn+ssh://www.some.svn-hosting.com:80/repo/trunk') ### pathFull {#pathfull} -То же самое, но включая query string и fragment. Пример: /top/news.html?page=2\#comments +То же самое, но включая query string и fragment. Пример: /top/news.html?page=2#comments ### queryString {#querystring} -Возвращает query-string. Пример: page=1&lr=213. query-string не включает в себя начальный знак вопроса, а также \# и всё, что после \#. +Возвращает query-string. Пример: page=1&lr=213. query-string не включает в себя начальный знак вопроса, а также # и всё, что после #. ### fragment {#fragment} @@ -128,7 +128,7 @@ SELECT topLevelDomain('svn+ssh://www.some.svn-hosting.com:80/repo/trunk') ### queryStringAndFragment {#querystringandfragment} -Возвращает query string и fragment identifier. Пример: страница=1\#29390. +Возвращает query string и fragment identifier. Пример: страница=1#29390. ### extractURLParameter(URL, name) {#extracturlparameterurl-name} diff --git a/docs/ru/sql-reference/functions/ym-dict-functions.md b/docs/ru/sql-reference/functions/ym-dict-functions.md index 74be04e9ba3..7ac27c0d285 100644 --- a/docs/ru/sql-reference/functions/ym-dict-functions.md +++ b/docs/ru/sql-reference/functions/ym-dict-functions.md @@ -11,12 +11,12 @@ ClickHouse поддерживает работу одновременно с н В конфиге clickhouse-server указывается файл с иерархией регионов: `/opt/geo/regions_hierarchy.txt` -Кроме указанного файла, рядом ищутся файлы, к имени которых (до расширения) добавлен символ \_ и какой угодно суффикс. +Кроме указанного файла, рядом ищутся файлы, к имени которых (до расширения) добавлен символ _ и какой угодно суффикс. Например, также найдётся файл `/opt/geo/regions_hierarchy_ua.txt`, если такой есть. `ua` называется ключом словаря. Для словаря без суффикса, ключ является пустой строкой. -Все словари перезагружаются в рантайме (раз в количество секунд, заданное в конфигурационном параметре builtin\_dictionaries\_reload\_interval, по умолчанию - раз в час), но перечень доступных словарей определяется один раз, при старте сервера. +Все словари перезагружаются в рантайме (раз в количество секунд, заданное в конфигурационном параметре builtin_dictionaries_reload_interval, по умолчанию - раз в час), но перечень доступных словарей определяется один раз, при старте сервера. Во все функции по работе с регионами, в конце добавлен один необязательный аргумент - ключ словаря. Далее он обозначен как geobase. Пример: @@ -101,7 +101,7 @@ LIMIT 15 Переводит регион в континент. В остальном, аналогично функции regionToCity. Пример: `regionToContinent(toUInt32(213)) = 10001` - преобразовали Москву (213) в Евразию (10001). -### regionToTopContinent (\#regiontotopcontinent) {#regiontotopcontinent-regiontotopcontinent} +### regionToTopContinent (#regiontotopcontinent) {#regiontotopcontinent-regiontotopcontinent} Находит для региона верхний в иерархии континент. diff --git a/docs/ru/sql-reference/operators/in.md b/docs/ru/sql-reference/operators/in.md index 98006288b05..4c1290df166 100644 --- a/docs/ru/sql-reference/operators/in.md +++ b/docs/ru/sql-reference/operators/in.md @@ -112,9 +112,9 @@ FROM t_null Следует быть внимательным при использовании подзапросов в секции `IN` / `JOIN` в случае распределённой обработки запроса. -Рассмотрим это на примерах. Пусть на каждом сервере кластера есть обычная таблица **local\_table**. Пусть также есть таблица **distributed\_table** типа **Distributed**, которая смотрит на все серверы кластера. +Рассмотрим это на примерах. Пусть на каждом сервере кластера есть обычная таблица **local_table**. Пусть также есть таблица **distributed_table** типа **Distributed**, которая смотрит на все серверы кластера. -При запросе к распределённой таблице **distributed\_table**, запрос будет отправлен на все удалённые серверы, и на них будет выполнен с использованием таблицы **local\_table**. +При запросе к распределённой таблице **distributed_table**, запрос будет отправлен на все удалённые серверы, и на них будет выполнен с использованием таблицы **local_table**. Например, запрос @@ -148,7 +148,7 @@ SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID IN (SEL Это будет работать правильно и оптимально, если вы предусмотрели такой случай, и раскладываете данные по серверам кластера таким образом, чтобы данные одного UserID-а лежали только на одном сервере. В таком случае все необходимые данные будут присутствовать на каждом сервере локально. В противном случае результат будет посчитан неточно. Назовём этот вариант запроса «локальный IN». -Чтобы исправить работу запроса, когда данные размазаны по серверам кластера произвольным образом, можно было бы указать **distributed\_table** внутри подзапроса. Запрос будет выглядеть так: +Чтобы исправить работу запроса, когда данные размазаны по серверам кластера произвольным образом, можно было бы указать **distributed_table** внутри подзапроса. Запрос будет выглядеть так: ``` sql SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM distributed_table WHERE CounterID = 34) diff --git a/docs/ru/sql-reference/statements/check-table.md b/docs/ru/sql-reference/statements/check-table.md index 178b2fc3a48..fa37b24851f 100644 --- a/docs/ru/sql-reference/statements/check-table.md +++ b/docs/ru/sql-reference/statements/check-table.md @@ -36,7 +36,7 @@ CHECK TABLE [db.]name В этом случае можно скопировать оставшиеся неповрежденные данные в другую таблицу. Для этого: 1. Создайте новую таблицу с такой же структурой, как у поврежденной таблицы. Для этого выполните запрос `CREATE TABLE AS `. -2. Установите значение параметра [max\_threads](../../operations/settings/settings.md#settings-max_threads) в 1. Это нужно для того, чтобы выполнить следующий запрос в одном потоке. Установить значение параметра можно через запрос: `SET max_threads = 1`. +2. Установите значение параметра [max_threads](../../operations/settings/settings.md#settings-max_threads) в 1. Это нужно для того, чтобы выполнить следующий запрос в одном потоке. Установить значение параметра можно через запрос: `SET max_threads = 1`. 3. Выполните запрос `INSERT INTO SELECT * FROM `. В результате неповрежденные данные будут скопированы в другую таблицу. Обратите внимание, будут скопированы только те данные, которые следуют до поврежденного участка. 4. Перезапустите `clickhouse-client`, чтобы вернуть предыдущее значение параметра `max_threads`. diff --git a/docs/ru/sql-reference/statements/insert-into.md b/docs/ru/sql-reference/statements/insert-into.md index c1201687d32..281b708f3f9 100644 --- a/docs/ru/sql-reference/statements/insert-into.md +++ b/docs/ru/sql-reference/statements/insert-into.md @@ -18,7 +18,7 @@ INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), . - Значения, вычисляемые из `DEFAULT` выражений, указанных в определении таблицы. - Нули и пустые строки, если `DEFAULT` не определены. -Если [strict\_insert\_defaults=1](../../operations/settings/settings.md), то столбцы, для которых не определены `DEFAULT`, необходимо перечислить в запросе. +Если [strict_insert_defaults=1](../../operations/settings/settings.md), то столбцы, для которых не определены `DEFAULT`, необходимо перечислить в запросе. В INSERT можно передавать данные любого [формата](../../interfaces/formats.md#formats), который поддерживает ClickHouse. Для этого формат необходимо указать в запросе в явном виде: diff --git a/docs/ru/sql-reference/statements/optimize.md b/docs/ru/sql-reference/statements/optimize.md index d94ba2aa5da..dc70d86a1a0 100644 --- a/docs/ru/sql-reference/statements/optimize.md +++ b/docs/ru/sql-reference/statements/optimize.md @@ -13,7 +13,7 @@ OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition | PARTITION I Если `OPTIMIZE` применяется к таблицам семейства [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/replication.md), ClickHouse создаёт задачу на мёрж и ожидает её исполнения на всех узлах (если активирована настройка `replication_alter_partitions_sync`). -- Если `OPTIMIZE` не выполняет мёрж по любой причине, ClickHouse не оповещает об этом клиента. Чтобы включить оповещения, используйте настройку [optimize\_throw\_if\_noop](../../operations/settings/settings.md#setting-optimize_throw_if_noop). +- Если `OPTIMIZE` не выполняет мёрж по любой причине, ClickHouse не оповещает об этом клиента. Чтобы включить оповещения, используйте настройку [optimize_throw_if_noop](../../operations/settings/settings.md#setting-optimize_throw_if_noop). - Если указать `PARTITION`, то оптимизация выполняется только для указанной партиции. [Как задавать имя партиции в запросах](alter/index.md#alter-how-to-specify-part-expr). - Если указать `FINAL`, то оптимизация выполняется даже в том случае, если все данные уже лежат в одном куске. - Если указать `DEDUPLICATE`, то произойдет схлопывание полностью одинаковых строк (сравниваются значения во всех колонках), имеет смысл только для движка MergeTree. diff --git a/docs/ru/sql-reference/statements/select/group-by.md b/docs/ru/sql-reference/statements/select/group-by.md index f68a8547f13..9581f477af5 100644 --- a/docs/ru/sql-reference/statements/select/group-by.md +++ b/docs/ru/sql-reference/statements/select/group-by.md @@ -54,15 +54,15 @@ ### Настройка обработки итогов {#configuring-totals-processing} -По умолчанию `totals_mode = 'before_having'`. В этом случае totals считается по всем строчкам, включая непрошедших через HAVING и max\_rows\_to\_group\_by. +По умолчанию `totals_mode = 'before_having'`. В этом случае totals считается по всем строчкам, включая непрошедших через HAVING и max_rows_to_group_by. Остальные варианты учитывают в totals только строчки, прошедшие через HAVING, и имеют разное поведение при наличии настройки `max_rows_to_group_by` и `group_by_overflow_mode = 'any'`. `after_having_exclusive` - не учитывать строчки, не прошедшие `max_rows_to_group_by`. То есть в totals попадёт меньше или столько же строчек, чем если бы `max_rows_to_group_by` не было. -`after_having_inclusive` - учитывать в totals все строчки, не прошедшие max\_rows\_to\_group\_by. То есть в totals попадёт больше или столько же строчек, чем если бы `max_rows_to_group_by` не было. +`after_having_inclusive` - учитывать в totals все строчки, не прошедшие max_rows_to_group_by. То есть в totals попадёт больше или столько же строчек, чем если бы `max_rows_to_group_by` не было. -`after_having_auto` - считать долю строчек, прошедших через HAVING. Если она больше некоторого значения (по умолчанию - 50%), то включить все строчки, не прошедшие max\_rows\_to\_group\_by в totals, иначе - не включить. +`after_having_auto` - считать долю строчек, прошедших через HAVING. Если она больше некоторого значения (по умолчанию - 50%), то включить все строчки, не прошедшие max_rows_to_group_by в totals, иначе - не включить. `totals_auto_threshold` - по умолчанию 0.5. Коэффициент для работы `after_having_auto`. @@ -110,10 +110,10 @@ GROUP BY вычисляет для каждого встретившегося ### Группировка во внешней памяти {#select-group-by-in-external-memory} Можно включить сброс временных данных на диск, чтобы ограничить потребление оперативной памяти при выполнении `GROUP BY`. -Настройка [max\_bytes\_before\_external\_group\_by](../../../operations/settings/settings.md#settings-max_bytes_before_external_group_by) определяет пороговое значение потребления RAM, по достижении которого временные данные `GROUP BY` сбрасываются в файловую систему. Если равно 0 (по умолчанию) - значит выключено. +Настройка [max_bytes_before_external_group_by](../../../operations/settings/settings.md#settings-max_bytes_before_external_group_by) определяет пороговое значение потребления RAM, по достижении которого временные данные `GROUP BY` сбрасываются в файловую систему. Если равно 0 (по умолчанию) - значит выключено. При использовании `max_bytes_before_external_group_by`, рекомендуем выставить `max_memory_usage` приблизительно в два раза больше. Это следует сделать, потому что агрегация выполняется в две стадии: чтение и формирование промежуточных данных (1) и слияние промежуточных данных (2). Сброс данных на файловую систему может производиться только на стадии 1. Если сброса временных данных не было, то на стадии 2 может потребляться до такого же объёма памяти, как на стадии 1. -Например, если [max\_memory\_usage](../../../operations/settings/settings.md#settings_max_memory_usage) было выставлено в 10000000000, и вы хотите использовать внешнюю агрегацию, то имеет смысл выставить `max_bytes_before_external_group_by` в 10000000000, а `max_memory_usage` в 20000000000. При срабатывании внешней агрегации (если был хотя бы один сброс временных данных в файловую систему) максимальное потребление оперативки будет лишь чуть-чуть больше `max_bytes_before_external_group_by`. +Например, если [max_memory_usage](../../../operations/settings/settings.md#settings_max_memory_usage) было выставлено в 10000000000, и вы хотите использовать внешнюю агрегацию, то имеет смысл выставить `max_bytes_before_external_group_by` в 10000000000, а `max_memory_usage` в 20000000000. При срабатывании внешней агрегации (если был хотя бы один сброс временных данных в файловую систему) максимальное потребление оперативки будет лишь чуть-чуть больше `max_bytes_before_external_group_by`. При распределённой обработке запроса внешняя агрегация производится на удалённых серверах. Для того чтобы на сервере-инициаторе запроса использовалось немного оперативки, нужно выставить настройку `distributed_aggregation_memory_efficient` в 1. diff --git a/docs/ru/sql-reference/statements/select/join.md b/docs/ru/sql-reference/statements/select/join.md index 800f07a7c66..6f1b5e2cde6 100644 --- a/docs/ru/sql-reference/statements/select/join.md +++ b/docs/ru/sql-reference/statements/select/join.md @@ -35,7 +35,7 @@ FROM ## Настройки {#join-settings} !!! note "Примечание" - Значение строгости по умолчанию может быть переопределено с помощью настройки [join\_default\_strictness](../../../operations/settings/settings.md#settings-join_default_strictness). + Значение строгости по умолчанию может быть переопределено с помощью настройки [join_default_strictness](../../../operations/settings/settings.md#settings-join_default_strictness). Поведение сервера ClickHouse для операций `ANY JOIN` зависит от параметра [any_join_distinct_right_table_keys](../../../operations/settings/settings.md#any_join_distinct_right_table_keys). @@ -89,7 +89,7 @@ USING (equi_column1, ... equi_columnN, asof_column) !!! note "Примечание" `ASOF JOIN` не поддержан для движка таблиц [Join](../../../engines/table-engines/special/join.md). -Чтобы задать значение строгости по умолчанию, используйте сессионный параметр [join\_default\_strictness](../../../operations/settings/settings.md#settings-join_default_strictness). +Чтобы задать значение строгости по умолчанию, используйте сессионный параметр [join_default_strictness](../../../operations/settings/settings.md#settings-join_default_strictness). #### Распределённый join {#global-join} @@ -104,7 +104,7 @@ USING (equi_column1, ... equi_columnN, asof_column) ### Обработка пустых ячеек и NULL {#processing-of-empty-or-null-cells} -При соединении таблиц могут появляться пустые ячейки. Настройка [join\_use\_nulls](../../../operations/settings/settings.md#join_use_nulls) определяет, как ClickHouse заполняет эти ячейки. +При соединении таблиц могут появляться пустые ячейки. Настройка [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) определяет, как ClickHouse заполняет эти ячейки. Если ключами `JOIN` выступают поля типа [Nullable](../../../sql-reference/data-types/nullable.md), то строки, где хотя бы один из ключей имеет значение [NULL](../../../sql-reference/syntax.md#null-literal), не соединяются. @@ -140,10 +140,10 @@ USING (equi_column1, ... equi_columnN, asof_column) По умолчанию ClickHouse использует алгоритм [hash join](https://en.wikipedia.org/wiki/Hash_join). ClickHouse берет `` и создает для него хэш-таблицу в оперативной памяти. После некоторого порога потребления памяти ClickHouse переходит к алгоритму merge join. -- [max\_rows\_in\_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) — ограничивает количество строк в хэш-таблице. -- [max\_bytes\_in\_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) — ограничивает размер хэш-таблицы. +- [max_rows_in_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) — ограничивает количество строк в хэш-таблице. +- [max_bytes_in_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) — ограничивает размер хэш-таблицы. -По достижении любого из этих ограничений, ClickHouse действует в соответствии с настройкой [join\_overflow\_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode). +По достижении любого из этих ограничений, ClickHouse действует в соответствии с настройкой [join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode). ## Примеры {#examples} diff --git a/docs/ru/sql-reference/statements/system.md b/docs/ru/sql-reference/statements/system.md index 89536d0e2f6..1ba3a7555b9 100644 --- a/docs/ru/sql-reference/statements/system.md +++ b/docs/ru/sql-reference/statements/system.md @@ -38,12 +38,12 @@ ## RELOAD DICTIONARIES {#query_language-system-reload-dictionaries} Перегружает все словари, которые были успешно загружены до этого. -По умолчанию включена ленивая загрузка [dictionaries\_lazy\_load](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load), поэтому словари не загружаются автоматически при старте, а только при первом обращении через dictGet или SELECT к ENGINE=Dictionary. После этого такие словари (LOADED) будут перегружаться командой `system reload dictionaries`. +По умолчанию включена ленивая загрузка [dictionaries_lazy_load](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load), поэтому словари не загружаются автоматически при старте, а только при первом обращении через dictGet или SELECT к ENGINE=Dictionary. После этого такие словари (LOADED) будут перегружаться командой `system reload dictionaries`. Всегда возвращает `Ok.`, вне зависимости от результата обновления словарей. -## RELOAD DICTIONARY Dictionary\_name {#query_language-system-reload-dictionary} +## RELOAD DICTIONARY Dictionary_name {#query_language-system-reload-dictionary} -Полностью перегружает словарь `dictionary_name`, вне зависимости от состояния словаря (LOADED/NOT\_LOADED/FAILED). +Полностью перегружает словарь `dictionary_name`, вне зависимости от состояния словаря (LOADED/NOT_LOADED/FAILED). Всегда возвращает `Ok.`, вне зависимости от результата обновления словаря. Состояние словаря можно проверить запросом к `system.dictionaries`. @@ -55,7 +55,7 @@ SELECT name, status FROM system.dictionaries; Сбрасывает внутренний DNS кеш ClickHouse. Иногда (для старых версий ClickHouse) необходимо использовать эту команду при изменении инфраструктуры (смене IP адреса у другого ClickHouse сервера или сервера, используемого словарями). -Для более удобного (автоматического) управления кешем см. параметры disable\_internal\_dns\_cache, dns\_cache\_update\_period. +Для более удобного (автоматического) управления кешем см. параметры disable_internal_dns_cache, dns_cache_update_period. ## DROP MARK CACHE {#query_language-system-drop-mark-cache} @@ -73,7 +73,7 @@ SELECT name, status FROM system.dictionaries; ## FLUSH LOGS {#query_language-system-flush_logs} -Записывает буферы логов в системные таблицы (например system.query\_log). Позволяет не ждать 7.5 секунд при отладке. +Записывает буферы логов в системные таблицы (например system.query_log). Позволяет не ждать 7.5 секунд при отладке. Если буфер логов пустой, то этот запрос просто создаст системные таблицы. ## RELOAD CONFIG {#query_language-system-reload-config} diff --git a/docs/ru/sql-reference/table-functions/cluster.md b/docs/ru/sql-reference/table-functions/cluster.md index 5a98e2bac6b..b59ccf085ed 100644 --- a/docs/ru/sql-reference/table-functions/cluster.md +++ b/docs/ru/sql-reference/table-functions/cluster.md @@ -32,5 +32,5 @@ clusterAllReplicas('cluster_name', db, table) **See Also** -- [skip\_unavailable\_shards](../../operations/settings/settings.md#settings-skip_unavailable_shards) -- [load\_balancing](../../operations/settings/settings.md#settings-load_balancing) +- [skip_unavailable_shards](../../operations/settings/settings.md#settings-skip_unavailable_shards) +- [load_balancing](../../operations/settings/settings.md#settings-load_balancing) diff --git a/docs/ru/sql-reference/table-functions/file.md b/docs/ru/sql-reference/table-functions/file.md index 103ff34753c..4581fa081d7 100644 --- a/docs/ru/sql-reference/table-functions/file.md +++ b/docs/ru/sql-reference/table-functions/file.md @@ -8,7 +8,7 @@ file(path, format, structure) **Входные параметры** -- `path` — относительный путь до файла от [user\_files\_path](../../sql-reference/table-functions/file.md#server_configuration_parameters-user_files_path). Путь к файлу поддерживает следующие шаблоны в режиме доступа только для чтения `*`, `?`, `{abc,def}` и `{N..M}`, где `N`, `M` — числа, \``'abc', 'def'` — строки. +- `path` — относительный путь до файла от [user_files_path](../../sql-reference/table-functions/file.md#server_configuration_parameters-user_files_path). Путь к файлу поддерживает следующие шаблоны в режиме доступа только для чтения `*`, `?`, `{abc,def}` и `{N..M}`, где `N`, `M` — числа, \``'abc', 'def'` — строки. - `format` — [формат](../../interfaces/formats.md#formats) файла. - `structure` — структура таблицы. Формат `'colunmn1_name column1_ype, column2_name column2_type, ...'`. @@ -58,12 +58,12 @@ LIMIT 2 1. Предположим у нас есть несколько файлов со следующими относительными путями: -- ‘some\_dir/some\_file\_1’ -- ‘some\_dir/some\_file\_2’ -- ‘some\_dir/some\_file\_3’ -- ‘another\_dir/some\_file\_1’ -- ‘another\_dir/some\_file\_2’ -- ‘another\_dir/some\_file\_3’ +- ‘some_dir/some_file_1’ +- ‘some_dir/some_file_2’ +- ‘some_dir/some_file_3’ +- ‘another_dir/some_file_1’ +- ‘another_dir/some_file_2’ +- ‘another_dir/some_file_3’ 1. Запросим количество строк в этих файлах: diff --git a/docs/ru/sql-reference/table-functions/index.md b/docs/ru/sql-reference/table-functions/index.md index b8c3e51619d..83225d54e60 100644 --- a/docs/ru/sql-reference/table-functions/index.md +++ b/docs/ru/sql-reference/table-functions/index.md @@ -15,12 +15,12 @@ toc_title: "\u0412\u0432\u0435\u0434\u0435\u043D\u0438\u0435" Это способ создания временной таблицы, которая доступна только в текущем запросе. -- Запросе [CREATE TABLE AS \](../statements/create/index.md#create-table-query). +- Запросе [CREATE TABLE AS \](../statements/create/index.md#create-table-query). Это один из методов создания таблицы. !!! warning "Предупреждение" - Если настройка [allow\_ddl](../../operations/settings/permissions-for-queries.md#settings_allow_ddl) выключена, то использовать табличные функции невозможно. + Если настройка [allow_ddl](../../operations/settings/permissions-for-queries.md#settings_allow_ddl) выключена, то использовать табличные функции невозможно. | Функция | Описание | |-----------------------|---------------------------------------------------------------------------------------------------------------------------------------| diff --git a/docs/ru/sql-reference/table-functions/remote.md b/docs/ru/sql-reference/table-functions/remote.md index ab216d5de36..944500b57a0 100644 --- a/docs/ru/sql-reference/table-functions/remote.md +++ b/docs/ru/sql-reference/table-functions/remote.md @@ -49,7 +49,7 @@ example01-{01..02}-1 При наличии нескольких пар фигурных скобок, генерируется прямое произведение соответствующих множеств. -Адреса или их фрагменты в фигурных скобках можно указать через символ \|. В этом случае, соответствующие множества адресов понимаются как реплики - запрос будет отправлен на первую живую реплику. При этом, реплики перебираются в порядке, согласно текущей настройке [load\_balancing](../../operations/settings/settings.md). +Адреса или их фрагменты в фигурных скобках можно указать через символ \|. В этом случае, соответствующие множества адресов понимаются как реплики - запрос будет отправлен на первую живую реплику. При этом, реплики перебираются в порядке, согласно текущей настройке [load_balancing](../../operations/settings/settings.md). Пример: diff --git a/docs/ru/whats-new/extended-roadmap.md b/docs/ru/whats-new/extended-roadmap.md index b1d56ef005e..57a29ce90ad 100644 --- a/docs/ru/whats-new/extended-roadmap.md +++ b/docs/ru/whats-new/extended-roadmap.md @@ -42,7 +42,7 @@ Upd. Всё доделано, ожидается в релизе 20.6. ### 1.5. + ALTER RENAME COLUMN {#alter-rename-column} -[\#6861](https://github.com/ClickHouse/ClickHouse/issues/6861) +[#6861](https://github.com/ClickHouse/ClickHouse/issues/6861) Требует 1.3. Будет делать [Александр Сапин](https://github.com/alesapin). @@ -119,7 +119,7 @@ Upd. Представлен прототип неизвестной степен ClickHouse использует для хранения данных локальную файловую систему. Существует сценарий работы, в котором размещение старых (архивных) данных было бы выгодно на удалённой файловой системе. Если файловая система POSIX совместимая, то это не составляет проблем: ClickHouse успешно работает с Ceph, GlusterFS, MooseFS. Также востребованным является сценарий использования S3 (из-за доступности в облаке) или HDFS (для интеграции с Hadoop). Но эти файловые системы не являются POSIX совместимыми. Хотя для них существуют FUSE драйверы, но скорость работы сильно страдает и поддержка неполная. -ClickHouse использует небольшое подмножество функций ФС, но в то же время, и некоторые специфические части: симлинки и хардлинки, O\_DIRECT. Предлагается выделить всё взаимодействие с файловой системой в отдельный интерфейс. +ClickHouse использует небольшое подмножество функций ФС, но в то же время, и некоторые специфические части: симлинки и хардлинки, O_DIRECT. Предлагается выделить всё взаимодействие с файловой системой в отдельный интерфейс. ### 1.12. Экспериментальная реализация VFS поверх S3 и HDFS {#eksperimentalnaia-realizatsiia-vfs-poverkh-s3-i-hdfs} @@ -868,7 +868,7 @@ ClickHouse предоставляет возможность обратитьс ### 9.3. + Поддержка TLS для ZooKeeper {#podderzhka-tls-dlia-zookeeper} -[\#10174](https://github.com/ClickHouse/ClickHouse/issues/10174) +[#10174](https://github.com/ClickHouse/ClickHouse/issues/10174) Есть pull request. @@ -1359,7 +1359,7 @@ Upd. Задача в разработке. ### 20.1. Поддержка DELETE путём запоминания множества затронутых кусков и ключей {#podderzhka-delete-putiom-zapominaniia-mnozhestva-zatronutykh-kuskov-i-kliuchei} -### 20.2. Поддержка DELETE путём преобразования множества ключей в множество row\_numbers на реплике, столбца флагов и индекса по диапазонам {#podderzhka-delete-putiom-preobrazovaniia-mnozhestva-kliuchei-v-mnozhestvo-row-numbers-na-replike-stolbtsa-flagov-i-indeksa-po-diapazonam} +### 20.2. Поддержка DELETE путём преобразования множества ключей в множество row_numbers на реплике, столбца флагов и индекса по диапазонам {#podderzhka-delete-putiom-preobrazovaniia-mnozhestva-kliuchei-v-mnozhestvo-row-numbers-na-replike-stolbtsa-flagov-i-indeksa-po-diapazonam} ### 20.3. Поддержка ленивых DELETE путём запоминания выражений и преобразования к множеству ключей в фоне {#podderzhka-lenivykh-delete-putiom-zapominaniia-vyrazhenii-i-preobrazovaniia-k-mnozhestvu-kliuchei-v-fone} @@ -1720,7 +1720,7 @@ Q1. [Николай Кочетов](https://github.com/KochetovNicolai). ### 23.10. + Включение mlock бинарника {#vkliuchenie-mlock-binarnika} -Возможность mlock бинарника сделал Олег Алексеенков [\#3553](https://github.com/ClickHouse/ClickHouse/pull/3553) +Возможность mlock бинарника сделал Олег Алексеенков [#3553](https://github.com/ClickHouse/ClickHouse/pull/3553) . Поможет, когда на серверах кроме ClickHouse работает много посторонних программ (мы иногда называем их в шутку «треш-программами»). @@ -1807,11 +1807,11 @@ RAID позволяет одновременно увеличить надёжн ### 24.8. + Специализация векторизованного кода для AVX/AVX2/AVX512 и ARM NEON {#spetsializatsiia-vektorizovannogo-koda-dlia-avxavx2avx512-i-arm-neon} -[\#1017](https://github.com/ClickHouse/ClickHouse/issues/1017) +[#1017](https://github.com/ClickHouse/ClickHouse/issues/1017) Дмитрий Ковальков, ВШЭ и Яндекс. -Подавляющее большинство кода ClickHouse написана для x86\_64 с набором инструкций до SSE 4.2 включительно. Лишь отдельные редкие функции поддерживают AVX/AVX2/AVX512 с динамической диспетчеризацией. +Подавляющее большинство кода ClickHouse написана для x86_64 с набором инструкций до SSE 4.2 включительно. Лишь отдельные редкие функции поддерживают AVX/AVX2/AVX512 с динамической диспетчеризацией. В первой части задачи, следует добавить в ClickHouse реализации некоторых примитивов, оптимизированные под более новый набор инструкций. Например, AVX2 реализацию генератора случайных чисел pcg: https://github.com/lemire/simdpcg @@ -1827,7 +1827,7 @@ Upd. Есть pull request. В стадии ревью. Готово. ### 24.10. Поддержка типов half/bfloat16/unum {#podderzhka-tipov-halfbfloat16unum} -[\#7657](https://github.com/ClickHouse/ClickHouse/issues/7657) +[#7657](https://github.com/ClickHouse/ClickHouse/issues/7657) Рустам Гусейн-заде, ВШЭ. @@ -1877,7 +1877,7 @@ Upd. Прототип bitonic sort помержен, но целесообраз ### 24.14. Window функции {#window-funktsii} -[\#1469](https://github.com/ClickHouse/ClickHouse/issues/1469) +[#1469](https://github.com/ClickHouse/ClickHouse/issues/1469) Требует 2.1. @@ -1943,21 +1943,21 @@ ClickHouse также может использоваться для быстр ### 24.24. - Реализация алгоритмов differential privacy {#realizatsiia-algoritmov-differential-privacy} -[\#6874](https://github.com/ClickHouse/ClickHouse/issues/6874) +[#6874](https://github.com/ClickHouse/ClickHouse/issues/6874) Артём Вишняков, ВШЭ. Есть pull request. Отменено, так как решение имеет низкую практичность. ### 24.25. Интеграция в ClickHouse функциональности обработки HTTP User Agent {#integratsiia-v-clickhouse-funktsionalnosti-obrabotki-http-user-agent} -[\#157](https://github.com/ClickHouse/ClickHouse/issues/157) +[#157](https://github.com/ClickHouse/ClickHouse/issues/157) Есть хороший код в Яндекс.Метрике. Получено согласие от руководства. Михаил Филитов, ВШЭ. Upd. Есть pull request. Нужно ещё чистить код библиотеки. ### 24.26. Поддержка open tracing или аналогов {#podderzhka-open-tracing-ili-analogov} -[\#5182](https://github.com/ClickHouse/ClickHouse/issues/5182) +[#5182](https://github.com/ClickHouse/ClickHouse/issues/5182) Александр Кожихов, ВШЭ и Яндекс.YT. Upd. Есть pull request с прототипом. @@ -1965,7 +1965,7 @@ Upd. Александ Кузьменков взял задачу в работу ### 24.27. Реализация алгоритмов min-hash, sim-hash для нечёткого поиска полудубликатов {#realizatsiia-algoritmov-min-hash-sim-hash-dlia-nechiotkogo-poiska-poludublikatov} -[\#7649](https://github.com/ClickHouse/ClickHouse/pull/7649) +[#7649](https://github.com/ClickHouse/ClickHouse/pull/7649) ucasFL, ICT. @@ -1981,7 +1981,7 @@ Upd. Николай Кочетов взял задачу в работу. ### 24.29. Поддержка Arrow Flight {#podderzhka-arrow-flight} -[\#7554](https://github.com/ClickHouse/ClickHouse/issues/7554) +[#7554](https://github.com/ClickHouse/ClickHouse/issues/7554) Жанна Зосимова, ВШЭ. Upd. Пока поддержали Arrow как формат ввода-вывода. diff --git a/docs/ru/whats-new/security-changelog.md b/docs/ru/whats-new/security-changelog.md index 4a799a29649..ad55e1eeb51 100644 --- a/docs/ru/whats-new/security-changelog.md +++ b/docs/ru/whats-new/security-changelog.md @@ -48,7 +48,7 @@ unixODBC позволял указать путь для подключения ### CVE-2018-14668 {#cve-2018-14668} -Табличная функция «remote» допускала произвольные символы в полях «user», «password» и «default\_database», что позволяло производить атаки класса Cross Protocol Request Forgery. +Табличная функция «remote» допускала произвольные символы в полях «user», «password» и «default_database», что позволяло производить атаки класса Cross Protocol Request Forgery. Обнаружено благодаря: Андрею Красичкову из Службы Информационной Безопасности Яндекса diff --git a/docs/tools/output.md b/docs/tools/output.md index 58d6ad60955..91ec6e75999 100644 --- a/docs/tools/output.md +++ b/docs/tools/output.md @@ -7,10 +7,10 @@ In a “normal” row-oriented DBMS, data is stored in this order: Row WatchID JavaEnable Title GoodEvent EventTime ----- ------------- ------------ -------------------- ----------- --------------------- - \#0 89354350662 1 Investor Relations 1 2016-05-18 05:19:20 - \#1 90329509958 0 Contact us 1 2016-05-18 08:10:20 - \#2 89953706054 1 Mission 1 2016-05-18 07:38:00 - \#N ... ... ... ... ... + #0 89354350662 1 Investor Relations 1 2016-05-18 05:19:20 + #1 90329509958 0 Contact us 1 2016-05-18 08:10:20 + #2 89953706054 1 Mission 1 2016-05-18 07:38:00 + #N ... ... ... ... ... In other words, all the values related to a row are physically stored next to each other. @@ -20,7 +20,7 @@ Examples of a row-oriented DBMS are MySQL, Postgres, and MS SQL Server. In a column-oriented DBMS, data is stored like this: - Row: \#0 \#1 \#2 \#N + Row: #0 #1 #2 #N ------------- --------------------- --------------------- --------------------- ----- WatchID: 89354350662 90329509958 89953706054 ... JavaEnable: 1 0 1 ... diff --git a/docs/tr/development/architecture.md b/docs/tr/development/architecture.md index 37b5060e91e..5e9d177e78c 100644 --- a/docs/tr/development/architecture.md +++ b/docs/tr/development/architecture.md @@ -47,7 +47,7 @@ A `Block` bellekteki bir tablonun bir alt kümesini (yığın) temsil eden bir k Bir bloktaki sütunlar üzerinde bazı işlevleri hesapladığımızda, bloğa sonucu olan başka bir sütun ekleriz ve işlemler değişmez olduğu için işlevin argümanları için sütunlara dokunmayız. Daha sonra, gereksiz sütunlar bloktan kaldırılabilir, ancak değiştirilemez. Ortak alt ifadelerin ortadan kaldırılması için uygundur. -İşlenen her veri yığını için bloklar oluşturulur. Aynı hesaplama türü için, sütun adları ve türleri farklı bloklar için aynı kalır ve yalnızca sütun verileri değişir unutmayın. Küçük blok boyutları shared\_ptrs ve sütun adlarını kopyalamak için geçici dizeleri yüksek bir ek yükü olduğundan blok üstbilgisinden blok verileri bölmek daha iyidir. +İşlenen her veri yığını için bloklar oluşturulur. Aynı hesaplama türü için, sütun adları ve türleri farklı bloklar için aynı kalır ve yalnızca sütun verileri değişir unutmayın. Küçük blok boyutları shared_ptrs ve sütun adlarını kopyalamak için geçici dizeleri yüksek bir ek yükü olduğundan blok üstbilgisinden blok verileri bölmek daha iyidir. ## Blok Akışları {#block-streams} diff --git a/docs/tr/development/build.md b/docs/tr/development/build.md index 90cb166094d..92eaaf30ced 100644 --- a/docs/tr/development/build.md +++ b/docs/tr/development/build.md @@ -9,7 +9,7 @@ toc_title: "Linux \xFCzerinde ClickHouse nas\u0131l olu\u015Fturulur" Aşağıdaki öğretici Ubuntu Linux sistemine dayanmaktadır. Uygun değişikliklerle, başka herhangi bir Linux dağıtımı üzerinde de çalışması gerekir. -Desteklenen platformlar: x86\_64 ve AArch64. Power9 için destek deneyseldir. +Desteklenen platformlar: x86_64 ve AArch64. Power9 için destek deneyseldir. ## Git, Cmake, Python ve Ninja'yı yükleyin {#install-git-cmake-python-and-ninja} diff --git a/docs/tr/development/contrib.md b/docs/tr/development/contrib.md index de254f992dc..63cc289ec9b 100644 --- a/docs/tr/development/contrib.md +++ b/docs/tr/development/contrib.md @@ -28,7 +28,7 @@ toc_title: "Kullan\u0131lan \xDC\xE7\xFCnc\xFC Taraf K\xFCt\xFCphaneleri" | libpcg-rastgele | [Apache Lic 2.0ense 2.0](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libpcg-random/LICENSE-APACHE.txt) | | libressl | [OpenSSL Lisansı](https://github.com/ClickHouse-Extras/ssl/blob/master/COPYING) | | librdkafka | [BSD 2-Clause Lisansı](https://github.com/edenhill/librdkafka/blob/363dcad5a23dc29381cc626620e68ae418b3af19/LICENSE) | -| libwidechar\_width | [CC0 1.0 Evrensel](https://github.com/ClickHouse/ClickHouse/blob/master/libs/libwidechar_width/LICENSE) | +| libwidechar_width | [CC0 1.0 Evrensel](https://github.com/ClickHouse/ClickHouse/blob/master/libs/libwidechar_width/LICENSE) | | llvm | [BSD 3-Clause Lisansı](https://github.com/ClickHouse-Extras/llvm/blob/163def217817c90fb982a6daf384744d8472b92b/llvm/LICENSE.TXT) | | lz4 | [BSD 2-Clause Lisansı](https://github.com/lz4/lz4/blob/c10863b98e1503af90616ae99725ecd120265dfb/LICENSE) | | mariadb-bağlayıcı-c | [LGPL v2. 1](https://github.com/ClickHouse-Extras/mariadb-connector-c/blob/3.1/COPYING.LIB) | diff --git a/docs/tr/development/developer-instruction.md b/docs/tr/development/developer-instruction.md index 284989c652d..10b1ea35758 100644 --- a/docs/tr/development/developer-instruction.md +++ b/docs/tr/development/developer-instruction.md @@ -44,7 +44,7 @@ Komut satırında terminal Çalıştır: git clone --recursive git@github.com:your_github_username/ClickHouse.git cd ClickHouse -Not: lütfen, yerine *your\_github\_username* uygun olanı ile! +Not: lütfen, yerine *your_github_username* uygun olanı ile! Bu komut bir dizin oluşturacaktır `ClickHouse` projenin çalışma kopyasını içeren. @@ -154,7 +154,7 @@ Artık ClickHouse oluşturmaya hazır olduğunuza göre ayrı bir dizin oluştur mkdir build cd build -Birkaç farklı dizine (build\_release, build\_debug, vb.) sahip olabilirsiniz.) farklı yapı türleri için. +Birkaç farklı dizine (build_release, build_debug, vb.) sahip olabilirsiniz.) farklı yapı türleri için. İçinde iken `build` dizin, cmake çalıştırarak yapı yapılandırın. İlk çalıştırmadan önce, derleyici belirten ortam değişkenlerini tanımlamanız gerekir (bu örnekte sürüm 9 gcc derleyicisi). diff --git a/docs/tr/development/style.md b/docs/tr/development/style.md index 1628641df52..7c8d7f3d569 100644 --- a/docs/tr/development/style.md +++ b/docs/tr/development/style.md @@ -356,7 +356,7 @@ Diğer tüm durumlarda, anlamı açıklayan bir isim kullanın. bool info_successfully_loaded = false; ``` -**9.** İsimleri `define`s ve genel sabitler alt çizgi ile ALL\_CAPS kullanın. +**9.** İsimleri `define`s ve genel sabitler alt çizgi ile ALL_CAPS kullanın. ``` cpp #define MAX_SRC_TABLE_NAMES_TO_STORE 1000 @@ -396,7 +396,7 @@ Bağımsız değişken yapıcı gövdesinde kullanılmazsa, alt çizgi soneki at timer (not m_timer) ``` -**14.** Bir de SAB theitler için `enum`, büyük harfle CamelCase kullanın. ALL\_CAPS da kabul edilebilir. Eğer... `enum` yerel olmayan, bir `enum class`. +**14.** Bir de SAB theitler için `enum`, büyük harfle CamelCase kullanın. ALL_CAPS da kabul edilebilir. Eğer... `enum` yerel olmayan, bir `enum class`. ``` cpp enum class CompressionMethod @@ -709,7 +709,7 @@ Standart kütüphane kullanılır (`libc++`). **4.**OS: Linux UB .untu, daha eski değil. -**5.**Kod x86\_64 CPU mimarisi için yazılmıştır. +**5.**Kod x86_64 CPU mimarisi için yazılmıştır. CPU komut seti, sunucularımız arasında desteklenen minimum kümedir. Şu anda, sse 4.2. diff --git a/docs/tr/development/tests.md b/docs/tr/development/tests.md index 4406cae6429..a0766e54ae7 100644 --- a/docs/tr/development/tests.md +++ b/docs/tr/development/tests.md @@ -203,7 +203,7 @@ Hata ayıklama sürümü `jemalloc` hata ayıklama oluşturmak için kullanılı ClickHouse fuzzing hem kullanılarak uygulanmaktadır [libFuzzer](https://llvm.org/docs/LibFuzzer.html) ve rastgele SQL sorguları. Tüm fuzz testleri sanitizers (Adres ve tanımsız) ile yapılmalıdır. -LibFuzzer kütüphane kodu izole fuzz testi için kullanılır. Fuzzers test kodunun bir parçası olarak uygulanır ve “\_fuzzer” adı postfixes. +LibFuzzer kütüphane kodu izole fuzz testi için kullanılır. Fuzzers test kodunun bir parçası olarak uygulanır ve “_fuzzer” adı postfixes. Fuzzer örneği bulunabilir `src/Parsers/tests/lexer_fuzzer.cpp`. LibFuzzer özgü yapılandırmalar, sözlükler ve corpus saklanır `tests/fuzz`. Kullanıcı girişini işleyen her işlevsellik için fuzz testleri yazmanızı öneririz. diff --git a/docs/tr/engines/table-engines/index.md b/docs/tr/engines/table-engines/index.md index 01a447dbc6c..a50a0cd28e9 100644 --- a/docs/tr/engines/table-engines/index.md +++ b/docs/tr/engines/table-engines/index.md @@ -62,7 +62,7 @@ Ailede motorlar: - [Dağılı](special/distributed.md#distributed) - [MaterializedView](special/materializedview.md#materializedview) - [Sözlük](special/dictionary.md#dictionary) -- \[Mer \]ge\] (spec /ial / mer \#ge. md\#mer \#ge +- \[Mer \]ge\] (spec /ial / mer #ge. md#mer #ge - [Dosya](special/file.md#file) - [Boş](special/null.md#null) - [Koymak](special/set.md#set) diff --git a/docs/tr/engines/table-engines/integrations/hdfs.md b/docs/tr/engines/table-engines/integrations/hdfs.md index a69d717f0e4..7c9fcdd9f07 100644 --- a/docs/tr/engines/table-engines/integrations/hdfs.md +++ b/docs/tr/engines/table-engines/integrations/hdfs.md @@ -73,12 +73,12 @@ Birden çok yol bileşenleri globs olabilir. İşlenmek için dosya var olmalı 1. HDFS'DE aşağıdaki Urı'lerle TSV formatında birkaç dosyamız olduğunu varsayalım: -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_3’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_3’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_3’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_3’ 1. Altı dosyadan oluşan bir tablo oluşturmanın birkaç yolu vardır: diff --git a/docs/tr/engines/table-engines/integrations/kafka.md b/docs/tr/engines/table-engines/integrations/kafka.md index d74711b21e4..a26a4d12267 100644 --- a/docs/tr/engines/table-engines/integrations/kafka.md +++ b/docs/tr/engines/table-engines/integrations/kafka.md @@ -134,7 +134,7 @@ Bir kafka tablosu istediğiniz kadar materialized görüşe sahip olabilir, kafk SELECT level, sum(total) FROM daily GROUP BY level; ``` -Performansı artırmak için, alınan iletiler bloklar halinde gruplandırılır [max\_ınsert\_block\_size](../../../operations/server-configuration-parameters/settings.md#settings-max_insert_block_size). İçinde blok oluş ifma ifdıysa [stream\_flush\_interval\_ms](../../../operations/server-configuration-parameters/settings.md) milisaniye, veri blok bütünlüğü ne olursa olsun tabloya temizlendi. +Performansı artırmak için, alınan iletiler bloklar halinde gruplandırılır [max_ınsert_block_size](../../../operations/server-configuration-parameters/settings.md#settings-max_insert_block_size). İçinde blok oluş ifma ifdıysa [stream_flush_interval_ms](../../../operations/server-configuration-parameters/settings.md) milisaniye, veri blok bütünlüğü ne olursa olsun tabloya temizlendi. Konu verilerini almayı durdurmak veya dönüşüm mantığını değiştirmek için, hayata geçirilmiş görünümü ayırın: diff --git a/docs/tr/engines/table-engines/mergetree-family/custom-partitioning-key.md b/docs/tr/engines/table-engines/mergetree-family/custom-partitioning-key.md index 9dba2ac627e..148403a85cf 100644 --- a/docs/tr/engines/table-engines/mergetree-family/custom-partitioning-key.md +++ b/docs/tr/engines/table-engines/mergetree-family/custom-partitioning-key.md @@ -116,7 +116,7 @@ drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 5 12:09 201902_4_6_1 drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 1 16:48 detached ``` -Klasör ‘201901\_1\_1\_0’, ‘201901\_1\_7\_1’ ve böylece parçaların dizinleri vardır. Her bölüm karşılık gelen bir bölümle ilgilidir ve yalnızca belirli bir ay için veri içerir (Bu örnekteki tabloda aylara göre bölümleme vardır). +Klasör ‘201901_1_1_0’, ‘201901_1_7_1’ ve böylece parçaların dizinleri vardır. Her bölüm karşılık gelen bir bölümle ilgilidir ve yalnızca belirli bir ay için veri içerir (Bu örnekteki tabloda aylara göre bölümleme vardır). Bu `detached` dizin kullanarak tablodan ayrılmış parçaları içerir [DETACH](../../../sql-reference/statements/alter.md#alter_detach-partition) sorgu. Bozuk parçalar da silinmek yerine bu dizine taşınır. Sunucu parçaları kullanmaz `detached` directory. You can add, delete, or modify the data in this directory at any time – the server will not know about this until you run the [ATTACH](../../../sql-reference/statements/alter.md#alter_attach-partition) sorgu. diff --git a/docs/tr/engines/table-engines/mergetree-family/graphitemergetree.md b/docs/tr/engines/table-engines/mergetree-family/graphitemergetree.md index 3fa7ed529af..f34f545739b 100644 --- a/docs/tr/engines/table-engines/mergetree-family/graphitemergetree.md +++ b/docs/tr/engines/table-engines/mergetree-family/graphitemergetree.md @@ -81,7 +81,7 @@ Hariç tüm parametreler `config_section` içinde olduğu gibi aynı anlama sahi ## Toplaması Yapılandırması {#rollup-configuration} -Toplaması için ayarları tarafından tanımlanan [graphite\_rollup](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) sunucu yapılandırmasında parametre. Parametrenin adı herhangi biri olabilir. Birkaç yapılandırma oluşturabilir ve bunları farklı tablolar için kullanabilirsiniz. +Toplaması için ayarları tarafından tanımlanan [graphite_rollup](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) sunucu yapılandırmasında parametre. Parametrenin adı herhangi biri olabilir. Birkaç yapılandırma oluşturabilir ve bunları farklı tablolar için kullanabilirsiniz. Toplaması yapılandırma yapısı: diff --git a/docs/tr/engines/table-engines/mergetree-family/mergetree.md b/docs/tr/engines/table-engines/mergetree-family/mergetree.md index 4065b5fdcbd..c3ef7194c94 100644 --- a/docs/tr/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/tr/engines/table-engines/mergetree-family/mergetree.md @@ -239,7 +239,7 @@ Aşağıdaki örnekte, dizin kullanılamaz. SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' ``` -Clickhouse'un bir sorgu çalıştırırken dizini kullanıp kullanamayacağını kontrol etmek için ayarları kullanın [force\_index\_by\_date](../../../operations/settings/settings.md#settings-force_index_by_date) ve [force\_primary\_key](../../../operations/settings/settings.md). +Clickhouse'un bir sorgu çalıştırırken dizini kullanıp kullanamayacağını kontrol etmek için ayarları kullanın [force_index_by_date](../../../operations/settings/settings.md#settings-force_index_by_date) ve [force_primary_key](../../../operations/settings/settings.md). Aylara göre bölümleme anahtarı, yalnızca uygun aralıktaki tarihleri içeren veri bloklarını okumanıza izin verir. Bu durumda, veri bloğu birçok tarih için veri içerebilir (bir aya kadar). Bir blok içinde veriler, ilk sütun olarak tarihi içermeyen birincil anahtara göre sıralanır. Bu nedenle, birincil anahtar önekini belirtmeyen yalnızca bir tarih koşulu ile bir sorgu kullanarak tek bir tarih için okunacak daha fazla veri neden olur. @@ -330,7 +330,7 @@ Koşulları `WHERE` yan tümcesi, sütunlarla çalışan işlevlerin çağrılar Bu `set` dizin tüm fonksiyonları ile kullanılabilir. Diğer dizinler için işlev alt kümeleri aşağıdaki tabloda gösterilmiştir. -| Fonksiyon (operatör) / dizin | birincil anahtar | minmax | ngrambf\_v1 | tokenbf\_v1 | bloom\_filter | +| Fonksiyon (operatör) / dizin | birincil anahtar | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter | |------------------------------------------------------------------------------------------------------------|------------------|--------|-------------|-------------|---------------| | [eşitlikler (=, ==)](../../../sql-reference/functions/comparison-functions.md#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | | [notEquals(!=, \<\>)](../../../sql-reference/functions/comparison-functions.md#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | @@ -495,7 +495,7 @@ Veri kısmı için minimum hareketli birimdir `MergeTree`- motor masaları. Bir - Volume — Ordered set of equal disks (similar to [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures)). - Storage policy — Set of volumes and the rules for moving data between them. -Açıklanan varlıklara verilen isimler sistem tablolarında bulunabilir, [sistem.storage\_policies](../../../operations/system-tables.md#system_tables-storage_policies) ve [sistem.diskler](../../../operations/system-tables.md#system_tables-disks). Bir tablo için yapılandırılmış depolama ilkelerinden birini uygulamak için `storage_policy` ayarı `MergeTree`- motor aile tabloları. +Açıklanan varlıklara verilen isimler sistem tablolarında bulunabilir, [sistem.storage_policies](../../../operations/system-tables.md#system_tables-storage_policies) ve [sistem.diskler](../../../operations/system-tables.md#system_tables-disks). Bir tablo için yapılandırılmış depolama ilkelerinden birini uygulamak için `storage_policy` ayarı `MergeTree`- motor aile tabloları. ### Yapılandırma {#table_engine-mergetree-multiple-volumes_configure} @@ -642,7 +642,7 @@ Mutasyonlar ve bölüm dondurma hariç tüm bu durumlarda, bir parça verilen de Kap hoodut underun altında, [sabit linkler](https://en.wikipedia.org/wiki/Hard_link). Farklı diskler arasındaki sabit bağlantılar desteklenmez, bu nedenle bu gibi durumlarda ortaya çıkan parçalar ilk disklerle aynı disklerde saklanır. Arka planda, parçalar boş alan miktarına göre hacimler arasında taşınır (`move_factor` parametre) sırasına göre birimler yapılandırma dosyasında beyan edilir. -Veriler asla sonuncudan ve birincisine aktarılmaz. Bir sistem tabloları kullanabilirsiniz [sistem.part\_log](../../../operations/system-tables.md#system_tables-part-log) (alan `type = MOVE_PART`) ve [sistem.parçalar](../../../operations/system-tables.md#system_tables-parts) (alanlar `path` ve `disk`) arka plan hareketlerini izlemek için. Ayrıca, ayrıntılı bilgi sunucu günlüklerinde bulunabilir. +Veriler asla sonuncudan ve birincisine aktarılmaz. Bir sistem tabloları kullanabilirsiniz [sistem.part_log](../../../operations/system-tables.md#system_tables-part-log) (alan `type = MOVE_PART`) ve [sistem.parçalar](../../../operations/system-tables.md#system_tables-parts) (alanlar `path` ve `disk`) arka plan hareketlerini izlemek için. Ayrıca, ayrıntılı bilgi sunucu günlüklerinde bulunabilir. Kullanıcı, sorguyu kullanarak bir bölümü veya bölümü bir birimden diğerine taşımaya zorlayabilir [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](../../../sql-reference/statements/alter.md#alter_move-partition), arka plan işlemleri için tüm kısıtlamalar dikkate alınır. Sorgu, kendi başına bir hareket başlatır ve tamamlanması için arka plan işlemleri beklemez. Yeterli boş alan yoksa veya gerekli koşullardan herhangi biri karşılanmazsa kullanıcı bir hata mesajı alır. diff --git a/docs/tr/engines/table-engines/mergetree-family/replication.md b/docs/tr/engines/table-engines/mergetree-family/replication.md index 108ddf42d7f..133cce91bd1 100644 --- a/docs/tr/engines/table-engines/mergetree-family/replication.md +++ b/docs/tr/engines/table-engines/mergetree-family/replication.md @@ -59,7 +59,7 @@ Varolan herhangi bir ZooKeeper kümesini belirtebilirsiniz ve sistem kendi veril Zookeeper yapılandırma dosyasında ayarlanmamışsa, çoğaltılmış tablolar oluşturamazsınız ve varolan çoğaltılmış tablolar salt okunur olacaktır. -ZooKeeper kullanılmaz `SELECT` çoğaltma performansını etkilemez çünkü sorgular `SELECT` ve sorgular, çoğaltılmamış tablolar için yaptıkları kadar hızlı çalışır. Dağıtılmış çoğaltılmış tabloları sorgularken, ClickHouse davranışı ayarlar tarafından denetlenir [max\_replica\_delay\_for\_distributed\_queries](../../../operations/settings/settings.md#settings-max_replica_delay_for_distributed_queries) ve [fallback\_to\_stale\_replicas\_for\_distributed\_queries](../../../operations/settings/settings.md#settings-fallback_to_stale_replicas_for_distributed_queries). +ZooKeeper kullanılmaz `SELECT` çoğaltma performansını etkilemez çünkü sorgular `SELECT` ve sorgular, çoğaltılmamış tablolar için yaptıkları kadar hızlı çalışır. Dağıtılmış çoğaltılmış tabloları sorgularken, ClickHouse davranışı ayarlar tarafından denetlenir [max_replica_delay_for_distributed_queries](../../../operations/settings/settings.md#settings-max_replica_delay_for_distributed_queries) ve [fallback_to_stale_replicas_for_distributed_queries](../../../operations/settings/settings.md#settings-fallback_to_stale_replicas_for_distributed_queries). Her biri için `INSERT` sorgu, yaklaşık on girişleri zookeeper birkaç işlemler aracılığıyla eklenir. (Daha kesin olmak gerekirse, bu eklenen her veri bloğu içindir; bir ekleme sorgusu her bir blok veya bir blok içerir `max_insert_block_size = 1048576` satırlar.) Bu, biraz daha uzun gecikmelere yol açar `INSERT` çoğaltılmamış tablolarla karşılaştırıldığında. Ancak, birden fazla olmayan gruplar halinde veri eklemek için önerileri izlerseniz `INSERT` saniyede, herhangi bir sorun yaratmaz. Bir ZooKeeper kümesini koordine etmek için kullanılan tüm ClickHouse kümesinin toplam birkaç yüzü vardır `INSERTs` saniyede. Veri eklerindeki verim (saniyede satır sayısı), çoğaltılmamış veriler için olduğu kadar yüksektir. @@ -71,7 +71,7 @@ Varsayılan olarak, bir INSERT sorgusu yalnızca bir yinelemeden veri yazma onay Her veri bloğu atomik olarak yazılır. Ekle sorgusu kadar bloklara ayrılmıştır `max_insert_block_size = 1048576` satırlar. Diğer bir deyişle, `INSERT` sorgu 1048576 satırdan daha az, atomik olarak yapılır. -Veri blokları tekilleştirilmiştir. Aynı veri bloğunun (aynı sırayla aynı satırları içeren aynı boyuttaki veri blokları) birden fazla yazımı için, blok yalnızca bir kez yazılır. Bunun nedeni, istemci uygulaması verilerin DB'YE yazılıp yazılmadığını bilmediğinde ağ arızaları durumunda, `INSERT` sorgu sadece tekrar edilebilir. Hangi çoğaltma eklerinin aynı verilerle gönderildiği önemli değildir. `INSERTs` idempotent vardır. Tekilleştirme parametreleri tarafından kontrol edilir [merge\_tree](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-merge_tree) sunucu ayarları. +Veri blokları tekilleştirilmiştir. Aynı veri bloğunun (aynı sırayla aynı satırları içeren aynı boyuttaki veri blokları) birden fazla yazımı için, blok yalnızca bir kez yazılır. Bunun nedeni, istemci uygulaması verilerin DB'YE yazılıp yazılmadığını bilmediğinde ağ arızaları durumunda, `INSERT` sorgu sadece tekrar edilebilir. Hangi çoğaltma eklerinin aynı verilerle gönderildiği önemli değildir. `INSERTs` idempotent vardır. Tekilleştirme parametreleri tarafından kontrol edilir [merge_tree](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-merge_tree) sunucu ayarları. Çoğaltma sırasında, yalnızca eklenecek kaynak veriler ağ üzerinden aktarılır. Daha fazla veri dönüşümü (birleştirme), tüm kopyalarda aynı şekilde koordine edilir ve gerçekleştirilir. Bu, ağ kullanımını en aza indirir; bu, çoğaltmaların farklı veri merkezlerinde bulunduğu zaman çoğaltmanın iyi çalıştığı anlamına gelir. (Farklı veri merkezlerinde çoğaltmanın çoğaltmanın ana hedefi olduğunu unutmayın .) diff --git a/docs/tr/engines/table-engines/special/buffer.md b/docs/tr/engines/table-engines/special/buffer.md index 1770738db00..5de1b71267b 100644 --- a/docs/tr/engines/table-engines/special/buffer.md +++ b/docs/tr/engines/table-engines/special/buffer.md @@ -36,7 +36,7 @@ Verilerin yıkanması için koşullar, her biri için ayrı ayrı hesaplanır. ` CREATE TABLE merge.hits_buffer AS merge.hits ENGINE = Buffer(merge, hits, 16, 10, 100, 10000, 1000000, 10000000, 100000000) ``` -Oluşturma Bir ‘merge.hits\_buffer’ ile aynı yapıya sahip tablo ‘merge.hits’ ve Tampon motorunu kullanarak. Bu tabloya yazarken, veriler RAM'de arabelleğe alınır ve daha sonra ‘merge.hits’ Tablo. 16 tamponlar oluşturulur. 100 saniye geçti veya bir milyon satır yazılmış veya 100 MB veri yazılmıştır; ya da aynı anda 10 saniye geçti ve 10.000 satır ve 10 MB veri yazılmıştır, bunların her veri temizlendi. Örneğin, sadece bir satır yazılmışsa, 100 saniye sonra ne olursa olsun, yıkanacaktır. Ancak, birçok satır yazılmışsa, veriler daha erken temizlenecektir. +Oluşturma Bir ‘merge.hits_buffer’ ile aynı yapıya sahip tablo ‘merge.hits’ ve Tampon motorunu kullanarak. Bu tabloya yazarken, veriler RAM'de arabelleğe alınır ve daha sonra ‘merge.hits’ Tablo. 16 tamponlar oluşturulur. 100 saniye geçti veya bir milyon satır yazılmış veya 100 MB veri yazılmıştır; ya da aynı anda 10 saniye geçti ve 10.000 satır ve 10 MB veri yazılmıştır, bunların her veri temizlendi. Örneğin, sadece bir satır yazılmışsa, 100 saniye sonra ne olursa olsun, yıkanacaktır. Ancak, birçok satır yazılmışsa, veriler daha erken temizlenecektir. Sunucu DROP TABLE veya DETACH TABLE ile durdurulduğunda, arabellek verileri de hedef tabloya temizlendi. @@ -58,7 +58,7 @@ Son ve örnek arabellek tabloları için düzgün çalışmıyor. Bu koşullar h Bir arabelleğe veri eklerken, arabelleklerden biri kilitlenir. Bir okuma işlemi aynı anda tablodan gerçekleştiriliyor, bu gecikmelere neden olur. -Bir arabellek tablosuna eklenen veriler, alt tabloda farklı bir sırada ve farklı bloklarda sonuçlanabilir. Bu nedenle, bir arabellek tablo CollapsingMergeTree doğru yazmak için kullanmak zordur. Sorunları önlemek için şunları ayarlayabilirsiniz ‘num\_layers’ 1'e. +Bir arabellek tablosuna eklenen veriler, alt tabloda farklı bir sırada ve farklı bloklarda sonuçlanabilir. Bu nedenle, bir arabellek tablo CollapsingMergeTree doğru yazmak için kullanmak zordur. Sorunları önlemek için şunları ayarlayabilirsiniz ‘num_layers’ 1'e. Hedef tablo yinelenirse, bir arabellek tablosuna yazarken yinelenmiş tabloların bazı beklenen özellikleri kaybolur. Satır ve veri parçaları boyutlarda sipariş için rasgele değişiklikler veri çoğaltma güvenilir olması mümkün olmadığını ifade eden çalışma, kapanmasına neden ‘exactly once’ çoğaltılan tablolara yazın. diff --git a/docs/tr/engines/table-engines/special/distributed.md b/docs/tr/engines/table-engines/special/distributed.md index d4ea832194b..12157728000 100644 --- a/docs/tr/engines/table-engines/special/distributed.md +++ b/docs/tr/engines/table-engines/special/distributed.md @@ -85,7 +85,7 @@ Küme adları nokta içermemelidir. Parametre `host`, `port` ve isteğe bağlı olarak `user`, `password`, `secure`, `compression` her sunucu için belirtilir: - `host` – The address of the remote server. You can use either the domain or the IPv4 or IPv6 address. If you specify the domain, the server makes a DNS request when it starts, and the result is stored as long as the server is running. If the DNS request fails, the server doesn't start. If you change the DNS record, restart the server. -- `port` – The TCP port for messenger activity (‘tcp\_port’ yapılandırmada, genellikle 9000 olarak ayarlanır). Http\_port ile karıştırmayın. +- `port` – The TCP port for messenger activity (‘tcp_port’ yapılandırmada, genellikle 9000 olarak ayarlanır). Http_port ile karıştırmayın. - `user` – Name of the user for connecting to a remote server. Default value: default. This user must have access to connect to the specified server. Access is configured in the users.xml file. For more information, see the section [Erişim hakları](../../../operations/access-rights.md). - `password` – The password for connecting to a remote server (not masked). Default value: empty string. - `secure` - Bağlantı için ssl kullanın, genellikle de tanımlamanız gerekir `port` = 9440. Sunucu dinlem shouldeli `9440` ve doğru sertifikalara sahip. @@ -113,13 +113,13 @@ Bir kümeye veri yazmak için iki yöntem vardır: Her parça yapılandırma dosyasında tanımlanan bir ağırlığa sahip olabilir. Varsayılan olarak, ağırlık bir eşittir. Veriler, parça ağırlığı ile orantılı miktarda parçalara dağıtılır. Örneğin, iki parça varsa ve birincisi 9'luk bir ağırlığa sahipse, ikincisi 10'luk bir ağırlığa sahipse, ilk satırların 9 / 19 parçası gönderilir ve ikincisi 10 / 19 gönderilir. -Her shard olabilir ‘internal\_replication’ yapılandırma dosyasında tanımlanan parametre. +Her shard olabilir ‘internal_replication’ yapılandırma dosyasında tanımlanan parametre. Bu parametre şu şekilde ayarlanırsa ‘true’, yazma işlemi ilk sağlıklı yinelemeyi seçer ve ona veri yazar. Dağıtılmış tablo ise bu alternatifi kullanın “looks at” çoğaltılan tablolar. Başka bir deyişle, verilerin yazılacağı tablo kendilerini çoğaltacaktır. Olarak ayarlan ifmışsa ‘false’ (varsayılan), veriler tüm kopyalara yazılır. Özünde, bu, dağıtılmış tablonun verilerin kendisini çoğalttığı anlamına gelir. Bu, çoğaltılmış tabloları kullanmaktan daha kötüdür, çünkü kopyaların tutarlılığı denetlenmez ve zamanla biraz farklı veriler içerirler. -Bir veri satırının gönderildiği parçayı seçmek için, parçalama ifadesi analiz edilir ve kalan kısmı, parçaların toplam ağırlığına bölünmesinden alınır. Satır, kalanların yarı aralığına karşılık gelen parçaya gönderilir. ‘prev\_weight’ -e doğru ‘prev\_weights + weight’, nere ‘prev\_weights’ en küçük sayıya sahip parçaların toplam ağırlığı ve ‘weight’ bu parçanın ağırlığı. Örneğin, iki parça varsa ve birincisi 9'luk bir ağırlığa sahipse, ikincisi 10'luk bir ağırlığa sahipse, satır \[0, 9) aralığından kalanlar için ilk parçaya ve ikincisine \[9, 19) aralığından kalanlar için gönderilecektir. +Bir veri satırının gönderildiği parçayı seçmek için, parçalama ifadesi analiz edilir ve kalan kısmı, parçaların toplam ağırlığına bölünmesinden alınır. Satır, kalanların yarı aralığına karşılık gelen parçaya gönderilir. ‘prev_weight’ -e doğru ‘prev_weights + weight’, nere ‘prev_weights’ en küçük sayıya sahip parçaların toplam ağırlığı ve ‘weight’ bu parçanın ağırlığı. Örneğin, iki parça varsa ve birincisi 9'luk bir ağırlığa sahipse, ikincisi 10'luk bir ağırlığa sahipse, satır \[0, 9) aralığından kalanlar için ilk parçaya ve ikincisine \[9, 19) aralığından kalanlar için gönderilecektir. Sharding ifadesi, bir tamsayı döndüren sabitler ve tablo sütunlarından herhangi bir ifade olabilir. Örneğin, ifadeyi kullanabilirsiniz ‘rand()’ verilerin rastgele dağılımı için veya ‘UserID’ kullanıcının kimliğinin bölünmesinden kalanın dağıtımı için (daha sonra tek bir kullanıcının verileri, kullanıcılar tarafından çalışmayı ve katılmayı basitleştiren tek bir parçada bulunur). Sütunlardan biri yeterince eşit olarak dağıtılmazsa, onu bir karma işleve sarabilirsiniz: ınthash64(Userıd). @@ -132,11 +132,11 @@ Aşağıdaki durumlarda sharding şeması hakkında endişelenmelisiniz: - Belirli bir anahtar tarafından veri (veya birleştirme) birleştirme gerektiren sorgular kullanılır. Veriler bu anahtar tarafından parçalanırsa, GLOBAL IN veya GLOBAL JOİN yerine local IN veya JOİN kullanabilirsiniz, bu da çok daha etkilidir. - Çok sayıda küçük Sorgu ile çok sayıda sunucu (yüzlerce veya daha fazla) kullanılır (bireysel müşterilerin sorguları - web siteleri, reklamverenler veya ortaklar). Küçük sorguların tüm kümeyi etkilememesi için, tek bir istemci için tek bir parça üzerinde veri bulmak mantıklıdır. Alternatif olarak, Yandex'te yaptığımız gibi.Metrica, iki seviyeli sharding kurabilirsiniz: tüm kümeyi bölün “layers”, bir katmanın birden fazla parçadan oluşabileceği yer. Tek bir istemci için veriler tek bir katmanda bulunur, ancak kırıklar gerektiğinde bir katmana eklenebilir ve veriler rastgele dağıtılır. Her katman için dağıtılmış tablolar oluşturulur ve genel sorgular için tek bir paylaşılan dağıtılmış tablo oluşturulur. -Veriler zaman uyumsuz olarak yazılır. Tabloya eklendiğinde, veri bloğu sadece yerel dosya sistemine yazılır. Veriler en kısa sürede arka planda uzak sunuculara gönderilir. Veri gönderme süresi tarafından yönetilir [distributed\_directory\_monitor\_sleep\_time\_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms) ve [distributed\_directory\_monitor\_max\_sleep\_time\_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms) ayarlar. Bu `Distributed` motor ayrı ayrı eklenen verilerle her dosyayı gönderir, ancak toplu dosya gönderme etkinleştirebilirsiniz [distributed\_directory\_monitor\_batch\_ınserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts) ayar. Bu ayar, yerel sunucu ve ağ kaynaklarını daha iyi kullanarak küme performansını artırır. Tablo dizinindeki dosyaların listesini (gönderilmeyi bekleyen veriler) kontrol ederek verilerin başarıyla gönderilip gönderilmediğini kontrol etmelisiniz: `/var/lib/clickhouse/data/database/table/`. +Veriler zaman uyumsuz olarak yazılır. Tabloya eklendiğinde, veri bloğu sadece yerel dosya sistemine yazılır. Veriler en kısa sürede arka planda uzak sunuculara gönderilir. Veri gönderme süresi tarafından yönetilir [distributed_directory_monitor_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms) ve [distributed_directory_monitor_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms) ayarlar. Bu `Distributed` motor ayrı ayrı eklenen verilerle her dosyayı gönderir, ancak toplu dosya gönderme etkinleştirebilirsiniz [distributed_directory_monitor_batch_ınserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts) ayar. Bu ayar, yerel sunucu ve ağ kaynaklarını daha iyi kullanarak küme performansını artırır. Tablo dizinindeki dosyaların listesini (gönderilmeyi bekleyen veriler) kontrol ederek verilerin başarıyla gönderilip gönderilmediğini kontrol etmelisiniz: `/var/lib/clickhouse/data/database/table/`. Sunucu varlığını durdurdu veya (örneğin, bir aygıt arızasından sonra) dağıtılmış bir tabloya bir ekleme sonra kaba bir yeniden başlatma vardı, eklenen veriler kaybolabilir. Tablo dizininde bozuk bir veri parçası tespit edilirse, ‘broken’ alt dizin ve artık kullanılmıyor. -Max\_parallel\_replicas seçeneği etkinleştirildiğinde, sorgu işleme tek bir parça içindeki tüm yinelemeler arasında paralelleştirilir. Daha fazla bilgi için bölüme bakın [max\_parallel\_replicas](../../../operations/settings/settings.md#settings-max_parallel_replicas). +Max_parallel_replicas seçeneği etkinleştirildiğinde, sorgu işleme tek bir parça içindeki tüm yinelemeler arasında paralelleştirilir. Daha fazla bilgi için bölüme bakın [max_parallel_replicas](../../../operations/settings/settings.md#settings-max_parallel_replicas). ## Sanal Sütunlar {#virtual-columns} diff --git a/docs/tr/engines/table-engines/special/external-data.md b/docs/tr/engines/table-engines/special/external-data.md index 3dc7863c1b3..ac317098442 100644 --- a/docs/tr/engines/table-engines/special/external-data.md +++ b/docs/tr/engines/table-engines/special/external-data.md @@ -27,10 +27,10 @@ Komut satırı istemcisinde, formatta bir parametreler bölümü belirtebilirsin **–file** – Path to the file with the table dump, or -, which refers to stdin. Stdın'den yalnızca tek bir tablo alınabilir. -Aşağıdaki parametreler isteğe bağlıdır: **–name**– Name of the table. If omitted, \_data is used. +Aşağıdaki parametreler isteğe bağlıdır: **–name**– Name of the table. If omitted, _data is used. **–format** – Data format in the file. If omitted, TabSeparated is used. -Aşağıdaki parametrelerden biri gereklidir:**–types** – A list of comma-separated column types. For example: `UInt64,String`. The columns will be named \_1, \_2, … +Aşağıdaki parametrelerden biri gereklidir:**–types** – A list of comma-separated column types. For example: `UInt64,String`. The columns will be named _1, _2, … **–structure**– The table structure in the format`UserID UInt64`, `URL String`. Sütun adlarını ve türlerini tanımlar. Belirtilen dosyalar ‘file’ belirtilen biçimde ayrıştırılır ‘format’, belirtilen veri türlerini kullanarak ‘types’ veya ‘structure’. Tablo sunucuya yüklenecek ve orada adı ile geçici bir tablo olarak erişilebilir ‘name’. @@ -48,7 +48,7 @@ $ cat /etc/passwd | sed 's/:/\t/g' | clickhouse-client --query="SELECT shell, co /bin/sync 1 ``` -HTTP arabirimini kullanırken, dış veriler çok parçalı/form veri biçiminde geçirilir. Her tablo ayrı bir dosya olarak iletilir. Tablo adı dosya adından alınır. Bu ‘query\_string’ parametreleri geçirilir ‘name\_format’, ‘name\_types’, ve ‘name\_structure’, nere ‘name’ bu parametreler karşılık gelen tablonun adıdır. Parametrelerin anlamı, komut satırı istemcisini kullanırken olduğu gibi aynıdır. +HTTP arabirimini kullanırken, dış veriler çok parçalı/form veri biçiminde geçirilir. Her tablo ayrı bir dosya olarak iletilir. Tablo adı dosya adından alınır. Bu ‘query_string’ parametreleri geçirilir ‘name_format’, ‘name_types’, ve ‘name_structure’, nere ‘name’ bu parametreler karşılık gelen tablonun adıdır. Parametrelerin anlamı, komut satırı istemcisini kullanırken olduğu gibi aynıdır. Örnek: diff --git a/docs/tr/engines/table-engines/special/join.md b/docs/tr/engines/table-engines/special/join.md index f7605f1b579..28705c280bc 100644 --- a/docs/tr/engines/table-engines/special/join.md +++ b/docs/tr/engines/table-engines/special/join.md @@ -92,15 +92,15 @@ Gerçekleştir aemezsiniz `SELECT` doğrudan tablodan sorgulayın. Bunun yerine, Bir tablo oluştururken aşağıdaki ayarlar uygulanır: -- [join\_use\_nulls](../../../operations/settings/settings.md#join_use_nulls) -- [max\_rows\_in\_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) -- [max\_bytes\_in\_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) -- [join\_overflow\_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) -- [join\_any\_take\_last\_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) +- [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) +- [max_rows_in_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) +- [max_bytes_in_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) +- [join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) +- [join_any_take_last_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) Bu `Join`- motor tabloları kullanılamaz `GLOBAL JOIN` harekat. -Bu `Join`- motor kullanımına izin verir [join\_use\_nulls](../../../operations/settings/settings.md#join_use_nulls) ayarı `CREATE TABLE` deyim. Ve [SELECT](../../../sql-reference/statements/select/index.md) sorgu kullanımına izin verir `join_use_nulls` çok. Eğer farklı varsa `join_use_nulls` ayarlar, tablo birleştirme bir hata alabilirsiniz. Bu katılmak türüne bağlıdır. Kullandığınızda [joinGet](../../../sql-reference/functions/other-functions.md#joinget) fonksiyonu, aynı kullanmak zorunda `join_use_nulls` ayarı `CRATE TABLE` ve `SELECT` deyimler. +Bu `Join`- motor kullanımına izin verir [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) ayarı `CREATE TABLE` deyim. Ve [SELECT](../../../sql-reference/statements/select/index.md) sorgu kullanımına izin verir `join_use_nulls` çok. Eğer farklı varsa `join_use_nulls` ayarlar, tablo birleştirme bir hata alabilirsiniz. Bu katılmak türüne bağlıdır. Kullandığınızda [joinGet](../../../sql-reference/functions/other-functions.md#joinget) fonksiyonu, aynı kullanmak zorunda `join_use_nulls` ayarı `CRATE TABLE` ve `SELECT` deyimler. ## Veri Depolama {#data-storage} diff --git a/docs/tr/engines/table-engines/special/merge.md b/docs/tr/engines/table-engines/special/merge.md index a707954c13f..18d9573b813 100644 --- a/docs/tr/engines/table-engines/special/merge.md +++ b/docs/tr/engines/table-engines/special/merge.md @@ -31,7 +31,7 @@ Kullanmak için tipik bir yol `Merge` motor çok sayıda çalışma içindir `Ti Örnek 2: -Diyelim ki eski bir tablonuz (WatchLog\_old) var ve verileri yeni bir tabloya (WatchLog\_new) taşımadan bölümlemeyi değiştirmeye karar verdiniz ve her iki tablodaki verileri görmeniz gerekiyor. +Diyelim ki eski bir tablonuz (WatchLog_old) var ve verileri yeni bir tabloya (WatchLog_new) taşımadan bölümlemeyi değiştirmeye karar verdiniz ve her iki tablodaki verileri görmeniz gerekiyor. ``` sql CREATE TABLE WatchLog_old(date Date, UserId Int64, EventType String, Cnt UInt64) diff --git a/docs/tr/engines/table-engines/special/url.md b/docs/tr/engines/table-engines/special/url.md index 2632f1e5046..d9d56630337 100644 --- a/docs/tr/engines/table-engines/special/url.md +++ b/docs/tr/engines/table-engines/special/url.md @@ -24,7 +24,7 @@ sunucudan yanıt almak için ek başlıklar. sırasıyla. İşleme için `POST` istekleri, uzak sunucu desteklemesi gerekir [Yığınlı aktarım kodlaması](https://en.wikipedia.org/wiki/Chunked_transfer_encoding). -Kullanarak HTTP get yönlendirme şerbetçiotu sayısını sınırlayabilirsiniz [max\_http\_get\_redirects](../../../operations/settings/settings.md#setting-max_http_get_redirects) ayar. +Kullanarak HTTP get yönlendirme şerbetçiotu sayısını sınırlayabilirsiniz [max_http_get_redirects](../../../operations/settings/settings.md#setting-max_http_get_redirects) ayar. **Örnek:** diff --git a/docs/tr/faq/general.md b/docs/tr/faq/general.md index f17baee24e9..52d60896a54 100644 --- a/docs/tr/faq/general.md +++ b/docs/tr/faq/general.md @@ -17,7 +17,7 @@ Bu sistemler, yüksek gecikme süreleri nedeniyle çevrimiçi sorgular için uyg ## Oracle aracılığıyla ODBC kullanırken Kodlamalarla ilgili bir sorunum varsa ne olur? {#oracle-odbc-encodings} -Oracle ODBC sürücüsü aracılığıyla dış sözlükler kaynağı olarak kullanırsanız, doğru değeri ayarlamanız gerekir. `NLS_LANG` ortam değişkeni `/etc/default/clickhouse`. Daha fazla bilgi için, bkz: [Oracle NLS\_LANG SSS](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html). +Oracle ODBC sürücüsü aracılığıyla dış sözlükler kaynağı olarak kullanırsanız, doğru değeri ayarlamanız gerekir. `NLS_LANG` ortam değişkeni `/etc/default/clickhouse`. Daha fazla bilgi için, bkz: [Oracle NLS_LANG SSS](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html). **Örnek** diff --git a/docs/tr/getting-started/example-datasets/amplab-benchmark.md b/docs/tr/getting-started/example-datasets/amplab-benchmark.md index affeb465c84..8fe1a2301c7 100644 --- a/docs/tr/getting-started/example-datasets/amplab-benchmark.md +++ b/docs/tr/getting-started/example-datasets/amplab-benchmark.md @@ -9,7 +9,7 @@ toc_title: "AMPLab B\xFCy\xFCk Veri Benchmark" Bkz. https://amplab.cs.berkeley.edu/benchmark/ -Ücretsiz bir hesap için kaydolun https://aws.amazon.com. bir kredi kartı, e-posta ve telefon numarası gerektirir. Yeni bir erişim anahtarı alın https://console.aws.amazon.com/iam/home?nc2=h\_m\_sc\#security\_credential +Ücretsiz bir hesap için kaydolun https://aws.amazon.com. bir kredi kartı, e-posta ve telefon numarası gerektirir. Yeni bir erişim anahtarı alın https://console.aws.amazon.com/iam/home?nc2=h_m_sc#security_credential Konsolda aşağıdakileri çalıştırın: diff --git a/docs/tr/getting-started/example-datasets/metrica.md b/docs/tr/getting-started/example-datasets/metrica.md index 22f96f6761a..6a727d1ab55 100644 --- a/docs/tr/getting-started/example-datasets/metrica.md +++ b/docs/tr/getting-started/example-datasets/metrica.md @@ -9,7 +9,7 @@ toc_title: "\xDCye.Metrica Verileri" Veri kümesi, isabetlerle ilgili anonimleştirilmiş verileri içeren iki tablodan oluşur (`hits_v1`) ve ziyaret visitsler (`visits_v1`(kayıt olmak için).Metrica. Yandex hakkında daha fazla bilgi edinebilirsiniz.Metrica içinde [ClickHouse geçmişi](../../introduction/history.md) bölme. -Veri kümesi iki tablodan oluşur, bunlardan biri sıkıştırılmış olarak indirilebilir `tsv.xz` dosya veya hazırlanmış bölümler olarak. Buna ek olarak, genişletilmiş bir sürümü `hits` 100 milyon satır içeren tablo TSV olarak mevcuttur https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits\_100m\_obfuscated\_v1.tsv.xz ve hazırlanan bölümler olarak https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits\_100m\_obfuscated\_v1.tar.xz. +Veri kümesi iki tablodan oluşur, bunlardan biri sıkıştırılmış olarak indirilebilir `tsv.xz` dosya veya hazırlanmış bölümler olarak. Buna ek olarak, genişletilmiş bir sürümü `hits` 100 milyon satır içeren tablo TSV olarak mevcuttur https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits_100m_obfuscated_v1.tsv.xz ve hazırlanan bölümler olarak https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits_100m_obfuscated_v1.tar.xz. ## Hazırlanan bölümlerden tablolar elde etme {#obtaining-tables-from-prepared-partitions} diff --git a/docs/tr/getting-started/example-datasets/nyc-taxi.md b/docs/tr/getting-started/example-datasets/nyc-taxi.md index 70569d47542..7c2fa26eb05 100644 --- a/docs/tr/getting-started/example-datasets/nyc-taxi.md +++ b/docs/tr/getting-started/example-datasets/nyc-taxi.md @@ -197,7 +197,7 @@ Bu tablodaki veriler 142 GB kullanır. (Verileri doğrudan Postgres'ten içe aktarmak da mümkündür `COPY ... TO PROGRAM`.) -Unfortunately, all the fields associated with the weather (precipitation…average\_wind\_speed) were filled with NULL. Because of this, we will remove them from the final data set. +Unfortunately, all the fields associated with the weather (precipitation…average_wind_speed) were filled with NULL. Because of this, we will remove them from the final data set. Başlamak için, tek bir sunucuda bir tablo oluşturacağız. Daha sonra tabloyu dağıtacağız. diff --git a/docs/tr/getting-started/install.md b/docs/tr/getting-started/install.md index 361f59b3a10..62448efce3b 100644 --- a/docs/tr/getting-started/install.md +++ b/docs/tr/getting-started/install.md @@ -9,9 +9,9 @@ toc_title: Kurulum ## Sistem Gereksinimleri {#system-requirements} -ClickHouse, x86\_64, AArch64 veya PowerPC64LE CPU mimarisine sahip herhangi bir Linux, FreeBSD veya Mac OS X üzerinde çalışabilir. +ClickHouse, x86_64, AArch64 veya PowerPC64LE CPU mimarisine sahip herhangi bir Linux, FreeBSD veya Mac OS X üzerinde çalışabilir. -Resmi önceden oluşturulmuş ikili dosyalar genellikle x86\_64 ve kaldıraç sse 4.2 komut seti için derlenir, bu nedenle destekleyen CPU'nun aksi belirtilmedikçe ek bir sistem gereksinimi haline gelir. Geçerli CPU'nun sse 4.2 desteği olup olmadığını kontrol etmek için komut: +Resmi önceden oluşturulmuş ikili dosyalar genellikle x86_64 ve kaldıraç sse 4.2 komut seti için derlenir, bu nedenle destekleyen CPU'nun aksi belirtilmedikçe ek bir sistem gereksinimi haline gelir. Geçerli CPU'nun sse 4.2 desteği olup olmadığını kontrol etmek için komut: ``` bash $ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not supported" diff --git a/docs/tr/index.md b/docs/tr/index.md index 9ac619a1f8d..a3e84ec1c65 100644 --- a/docs/tr/index.md +++ b/docs/tr/index.md @@ -11,10 +11,10 @@ ClickHouse, sorguların çevrimiçi analitik işlenmesi (*Online Analytical Proc | Satır | WatchId | JavaEnable | Başlık | İyiOlay | OlayZamanı | |-------|-------------|------------|----------------------|---------|---------------------| -| \#0 | 89354350662 | 1 | Yatırımcı İlişkileri | 1 | 2016-05-18 05:19:20 | -| \#1 | 90329509958 | 0 | Bize ulaşın | 1 | 2016-05-18 08:10:20 | -| \#2 | 89953706054 | 1 | Görev | 1 | 2016-05-18 07:38:00 | -| \#N | … | … | … | … | … | +| #0 | 89354350662 | 1 | Yatırımcı İlişkileri | 1 | 2016-05-18 05:19:20 | +| #1 | 90329509958 | 0 | Bize ulaşın | 1 | 2016-05-18 08:10:20 | +| #2 | 89953706054 | 1 | Görev | 1 | 2016-05-18 07:38:00 | +| #N | … | … | … | … | … | Başka bir deyişle, bir satırla ilgili tüm değerler fiziksel olarak yan yana depolanır. @@ -22,7 +22,7 @@ MySQL, Postgres ve MS SQL Server gibi veritabanları satır odaklı DBMS örnekl Sütun odaklı bir DBMS’de ise veriler şu şekilde saklanır: -| Satır: | \#0 | \#1 | \#2 | \#N | +| Satır: | #0 | #1 | #2 | #N | |-------------|----------------------|---------------------|---------------------|-----| | WatchId: | 89354350662 | 90329509958 | 89953706054 | … | | JavaEnable: | 1 | 0 | 1 | … | diff --git a/docs/tr/interfaces/formats.md b/docs/tr/interfaces/formats.md index 5146edea8a2..1187b21f8c2 100644 --- a/docs/tr/interfaces/formats.md +++ b/docs/tr/interfaces/formats.md @@ -201,7 +201,7 @@ Ayar `format_template_resultset` resultset için bir biçim dizesi içeren dosya - `min` satır içinde minimum değerlere sahip mi `format_template_row` biçim (1 olarak ayarlandığında) - `max` maksimum değerleri olan satır `format_template_row` biçim (1 olarak ayarlandığında) - `rows` çıktı satırlarının toplam sayısıdır -- `rows_before_limit` minimum satır sayısı sınırı olmadan olurdu. Yalnızca sorgu sınırı içeriyorsa çıktı. Sorgu GROUP BY içeriyorsa, ROWS\_BEFORE\_LİMİT\_AT\_LEAST SINIRSIZDI olurdu satır tam sayısıdır. +- `rows_before_limit` minimum satır sayısı sınırı olmadan olurdu. Yalnızca sorgu sınırı içeriyorsa çıktı. Sorgu GROUP BY içeriyorsa, ROWS_BEFORE_LİMİT_AT_LEAST SINIRSIZDI olurdu satır tam sayısıdır. - `time` istek yürütme süresi saniyeler içinde mi - `rows_read` satır sayısı okun thedu mu - `bytes_read` bayt sayısı (sıkıştırılmamış) okundu mu @@ -352,21 +352,21 @@ Ayrıştırma, ek alanın varlığına izin verir `tskv` eşit işareti veya bir Virgülle ayrılmış değerler biçimi ([RFC](https://tools.ietf.org/html/rfc4180)). -Biçimlendirme yaparken, satırlar çift tırnak içine alınır. Bir dizenin içindeki çift alıntı, bir satırda iki çift tırnak olarak çıktılanır. Karakterlerden kaçmak için başka kural yoktur. Tarih ve Tarih-Saat çift tırnak içine alınır. Sayılar tırnak işaretleri olmadan çıktı. Değerler, bir sınırlayıcı karakterle ayrılır; `,` varsayılan olarak. Sınırlayıcı karakteri ayarında tanımlanır [format\_csv\_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter). Satırlar Unıx satır besleme (LF) kullanılarak ayrılır. Diziler CSV'DE aşağıdaki gibi serileştirilir: ilk olarak, dizi TabSeparated biçiminde olduğu gibi bir dizeye serileştirilir ve daha sonra ortaya çıkan dize çift tırnak içinde CSV'YE çıkarılır. CSV biçimindeki Tuples ayrı sütunlar olarak serileştirilir(yani, tuple'daki yuvalanmaları kaybolur). +Biçimlendirme yaparken, satırlar çift tırnak içine alınır. Bir dizenin içindeki çift alıntı, bir satırda iki çift tırnak olarak çıktılanır. Karakterlerden kaçmak için başka kural yoktur. Tarih ve Tarih-Saat çift tırnak içine alınır. Sayılar tırnak işaretleri olmadan çıktı. Değerler, bir sınırlayıcı karakterle ayrılır; `,` varsayılan olarak. Sınırlayıcı karakteri ayarında tanımlanır [format_csv_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter). Satırlar Unıx satır besleme (LF) kullanılarak ayrılır. Diziler CSV'DE aşağıdaki gibi serileştirilir: ilk olarak, dizi TabSeparated biçiminde olduğu gibi bir dizeye serileştirilir ve daha sonra ortaya çıkan dize çift tırnak içinde CSV'YE çıkarılır. CSV biçimindeki Tuples ayrı sütunlar olarak serileştirilir(yani, tuple'daki yuvalanmaları kaybolur). ``` bash $ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV" < data.csv ``` -\* Varsayılan olarak, sınırlayıcı `,`. Görmek [format\_csv\_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter) daha fazla bilgi için ayarlama. +\* Varsayılan olarak, sınırlayıcı `,`. Görmek [format_csv_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter) daha fazla bilgi için ayarlama. Ayrıştırma yaparken, tüm değerler tırnak işaretleri ile veya tırnak işaretleri olmadan ayrıştırılabilir. Hem çift hem de tek tırnak desteklenmektedir. Satırlar tırnak işaretleri olmadan da düzenlenebilir. Bu durumda, sınırlayıcı karaktere veya satır beslemesine (CR veya LF) ayrıştırılır. RFC'Yİ ihlal ederken, satırları tırnak işaretleri olmadan ayrıştırırken, önde gelen ve sondaki boşluklar ve sekmeler göz ardı edilir. Hat beslemesi için Unix (LF), Windows (CR LF) ve Mac OS Classic (CR LF) türleri desteklenir. Boş unquoted giriş değerleri, ilgili sütunlar için varsayılan değerlerle değiştirilir -[ınput\_format\_defaults\_for\_omitted\_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) +[ınput_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) etkindir. -`NULL` olarak format islanır `\N` veya `NULL` veya boş bir unquoted dize (bkz. ayarlar [ınput\_format\_csv\_unquoted\_null\_literal\_as\_null](../operations/settings/settings.md#settings-input_format_csv_unquoted_null_literal_as_null) ve [ınput\_format\_defaults\_for\_omitted\_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields)). +`NULL` olarak format islanır `\N` veya `NULL` veya boş bir unquoted dize (bkz. ayarlar [ınput_format_csv_unquoted_null_literal_as_null](../operations/settings/settings.md#settings-input_format_csv_unquoted_null_literal_as_null) ve [ınput_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields)). CSV biçimi, toplamların ve aşırılıkların çıktısını aynı şekilde destekler `TabSeparated`. @@ -451,12 +451,12 @@ SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase WITH TOTA } ``` -Json JavaScript ile uyumludur. Bunu sağlamak için, bazı karakterler ek olarak kaçar: eğik çizgi `/` olarak kaç İsar `\/`; alternatif Satır sonları `U+2028` ve `U+2029`, hangi bazı tarayıcılar kırmak, olarak kaçtı `\uXXXX`. ASCII denetim karakterleri kaçtı: backspace, form besleme, satır besleme, satır başı ve yatay sekme ile değiştirilir `\b`, `\f`, `\n`, `\r`, `\t` , 00-1f aralığında kalan baytların yanı sıra `\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) 0'a. +Json JavaScript ile uyumludur. Bunu sağlamak için, bazı karakterler ek olarak kaçar: eğik çizgi `/` olarak kaç İsar `\/`; alternatif Satır sonları `U+2028` ve `U+2029`, hangi bazı tarayıcılar kırmak, olarak kaçtı `\uXXXX`. ASCII denetim karakterleri kaçtı: backspace, form besleme, satır besleme, satır başı ve yatay sekme ile değiştirilir `\b`, `\f`, `\n`, `\r`, `\t` , 00-1f aralığında kalan baytların yanı sıra `\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) 0'a. `rows` – The total number of output rows. `rows_before_limit_at_least` Minimum satır sayısı sınırı olmadan olurdu. Yalnızca sorgu sınırı içeriyorsa çıktı. -Sorgu GROUP BY içeriyorsa, ROWS\_BEFORE\_LİMİT\_AT\_LEAST SINIRSIZDI olurdu satır tam sayısıdır. +Sorgu GROUP BY içeriyorsa, ROWS_BEFORE_LİMİT_AT_LEAST SINIRSIZDI olurdu satır tam sayısıdır. `totals` – Total values (when using WITH TOTALS). @@ -543,7 +543,7 @@ ClickHouse, nesnelerden sonra öğeler ve virgüller arasındaki boşlukları yo ClickHouse, karşılık gelen değerler için varsayılan değerlerle atlanmış değerleri değiştirir [veri türleri](../sql-reference/data-types/index.md). -Eğer `DEFAULT expr` belirtilen, ClickHouse bağlı olarak farklı ikame kuralları kullanır [ınput\_format\_defaults\_for\_omitted\_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) ayar. +Eğer `DEFAULT expr` belirtilen, ClickHouse bağlı olarak farklı ikame kuralları kullanır [ınput_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) ayar. Aşağıdaki tabloyu düşünün: @@ -586,7 +586,7 @@ Aksine [JSON](#json) biçimi, geçersiz UTF-8 dizilerinin hiçbir ikame yoktur. ### İç içe yapıların kullanımı {#jsoneachrow-nested} -İle bir tablo varsa [İçiçe](../sql-reference/data-types/nested-data-structures/nested.md) veri türü sütunları, aynı yapıya sahip json verilerini ekleyebilirsiniz. İle bu özelliği etkinleştirin [ınput\_format\_ımport\_nested\_json](../operations/settings/settings.md#settings-input_format_import_nested_json) ayar. +İle bir tablo varsa [İçiçe](../sql-reference/data-types/nested-data-structures/nested.md) veri türü sütunları, aynı yapıya sahip json verilerini ekleyebilirsiniz. İle bu özelliği etkinleştirin [ınput_format_ımport_nested_json](../operations/settings/settings.md#settings-input_format_import_nested_json) ayar. Örneğin, aşağıdaki tabloyu göz önünde bulundurun: @@ -600,7 +600,7 @@ Gibi görmek `Nested` veri türü açıklaması, ClickHouse, iç içe geçmiş y INSERT INTO json_each_row_nested FORMAT JSONEachRow {"n.s": ["abc", "def"], "n.i": [1, 23]} ``` -Hiyerarşik bir json nesnesi olarak veri eklemek için [input\_format\_import\_nested\_json = 1](../operations/settings/settings.md#settings-input_format_import_nested_json). +Hiyerarşik bir json nesnesi olarak veri eklemek için [input_format_import_nested_json = 1](../operations/settings/settings.md#settings-input_format_import_nested_json). ``` json { @@ -782,7 +782,7 @@ The minimum set of characters that you need to escape when passing data in Value Bu, kullanılan formattır `INSERT INTO t VALUES ...`, ancak sorgu sonuçlarını biçimlendirmek için de kullanabilirsiniz. -Ayrıca bakınız: [ınput\_format\_values\_interpret\_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions) ve [ınput\_format\_values\_deduce\_templates\_of\_expressions](../operations/settings/settings.md#settings-input_format_values_deduce_templates_of_expressions) ayarlar. +Ayrıca bakınız: [ınput_format_values_interpret_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions) ve [ınput_format_values_deduce_templates_of_expressions](../operations/settings/settings.md#settings-input_format_values_deduce_templates_of_expressions) ayarlar. ## Dikey {#vertical} @@ -1045,7 +1045,7 @@ Sütun adları gerekir: - ile başla `[A-Za-z_]` - daha sonra sadece içerir `[A-Za-z0-9_]` -Çıkış Avro dosya sıkıştırma ve senkronizasyon aralığı ile yapılandırılabilir [output\_format\_avro\_codec](../operations/settings/settings.md#settings-output_format_avro_codec) ve [output\_format\_avro\_sync\_interval](../operations/settings/settings.md#settings-output_format_avro_sync_interval) sırasıyla. +Çıkış Avro dosya sıkıştırma ve senkronizasyon aralığı ile yapılandırılabilir [output_format_avro_codec](../operations/settings/settings.md#settings-output_format_avro_codec) ve [output_format_avro_sync_interval](../operations/settings/settings.md#settings-output_format_avro_sync_interval) sırasıyla. ## AvroConfluent {#data-format-avro-confluent} @@ -1055,7 +1055,7 @@ Her Avro iletisi, şema Kayıt defterinin yardımıyla gerçek şemaya çözüle Şemalar çözüldükten sonra önbelleğe alınır. -Şema kayıt defteri URL'si ile yapılandırılır [format\_avro\_schema\_registry\_url](../operations/settings/settings.md#settings-format_avro_schema_registry_url) +Şema kayıt defteri URL'si ile yapılandırılır [format_avro_schema_registry_url](../operations/settings/settings.md#settings-format_avro_schema_registry_url) ### Veri Türleri Eşleştirme {#data_types-matching-1} @@ -1198,13 +1198,13 @@ mutlak bir yol veya istemci üzerinde geçerli dizine göre bir yol içerebilir. Eğer istemci kullanıyorsanız [Toplu Modu](../interfaces/cli.md#cli_usage), şemanın yolu güvenlik nedeniyle göreceli olmalıdır. Eğer giriş veya çıkış veri üzerinden [HTTP arayüzü](../interfaces/http.md) biçim şemasında belirtilen dosya adı -belirtilen dizinde bulunmalıdır [format\_schema\_path](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-format_schema_path) +belirtilen dizinde bulunmalıdır [format_schema_path](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-format_schema_path) sunucu yapılandırmasında. ## Atlama Hataları {#skippingerrors} -Gibi bazı format suchlar `CSV`, `TabSeparated`, `TSKV`, `JSONEachRow`, `Template`, `CustomSeparated` ve `Protobuf` ayrıştırma hatası oluşursa kırık satırı atlayabilir ve bir sonraki satırın başından ayrıştırmaya devam edebilir. Görmek [ınput\_format\_allow\_errors\_num](../operations/settings/settings.md#settings-input_format_allow_errors_num) ve -[ınput\_format\_allow\_errors\_ratio](../operations/settings/settings.md#settings-input_format_allow_errors_ratio) ayarlar. +Gibi bazı format suchlar `CSV`, `TabSeparated`, `TSKV`, `JSONEachRow`, `Template`, `CustomSeparated` ve `Protobuf` ayrıştırma hatası oluşursa kırık satırı atlayabilir ve bir sonraki satırın başından ayrıştırmaya devam edebilir. Görmek [ınput_format_allow_errors_num](../operations/settings/settings.md#settings-input_format_allow_errors_num) ve +[ınput_format_allow_errors_ratio](../operations/settings/settings.md#settings-input_format_allow_errors_ratio) ayarlar. Sınırlamalar: - Ayrıştırma hatası durumunda `JSONEachRow` yeni satıra (veya EOF) kadar tüm verileri atlar, bu nedenle satırlar AŞAĞIDAKİLERLE sınırlandırılmalıdır `\n` hataları doğru saymak için. - `Template` ve `CustomSeparated` bir sonraki satırın başlangıcını bulmak için son sütundan sonra sınırlayıcı ve satırlar arasındaki sınırlayıcıyı kullanın, Bu nedenle hataları atlamak yalnızca en az biri boş değilse çalışır. diff --git a/docs/tr/interfaces/http.md b/docs/tr/interfaces/http.md index 49d20ef6655..7a8d8da5e2c 100644 --- a/docs/tr/interfaces/http.md +++ b/docs/tr/interfaces/http.md @@ -11,7 +11,7 @@ HTTP arayüzü, herhangi bir programlama dilinden herhangi bir platformda Clickh Varsayılan olarak, clickhouse-server, 8123 numaralı bağlantı noktasında HTTP dinler (bu, yapılandırmada değiştirilebilir). -Parametreler olmadan bir GET / request yaparsanız, 200 yanıt kodunu ve tanımlanan dizeyi döndürür [http\_server\_default\_response](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-http_server_default_response) varsayılan değer “Ok.” (sonunda bir çizgi besleme ile) +Parametreler olmadan bir GET / request yaparsanız, 200 yanıt kodunu ve tanımlanan dizeyi döndürür [http_server_default_response](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-http_server_default_response) varsayılan değer “Ok.” (sonunda bir çizgi besleme ile) ``` bash $ curl 'http://localhost:8123/' @@ -147,12 +147,12 @@ $ echo 'DROP TABLE t' | curl 'http://localhost:8123/' --data-binary @- Veri tablosu döndürmeyen başarılı istekler için boş bir yanıt gövdesi döndürülür. -Veri iletirken dahili ClickHouse sıkıştırma formatını kullanabilirsiniz. Sıkıştırılmış veriler standart olmayan bir biçime sahiptir ve özel `clickhouse-compressor` onunla çalışmak için program (bu ile yüklü `clickhouse-client` paket). Veri ekleme verimliliğini artırmak için, sunucu tarafı sağlama toplamı doğrulamasını kullanarak devre dışı bırakabilirsiniz. [http\_native\_compression\_disable\_checksumming\_on\_decompress](../operations/settings/settings.md#settings-http_native_compression_disable_checksumming_on_decompress) ayar. +Veri iletirken dahili ClickHouse sıkıştırma formatını kullanabilirsiniz. Sıkıştırılmış veriler standart olmayan bir biçime sahiptir ve özel `clickhouse-compressor` onunla çalışmak için program (bu ile yüklü `clickhouse-client` paket). Veri ekleme verimliliğini artırmak için, sunucu tarafı sağlama toplamı doğrulamasını kullanarak devre dışı bırakabilirsiniz. [http_native_compression_disable_checksumming_on_decompress](../operations/settings/settings.md#settings-http_native_compression_disable_checksumming_on_decompress) ayar. Belirt ift ifiyseniz `compress=1` URL'de, sunucu size gönderdiği verileri sıkıştırır. Belirt ift ifiyseniz `decompress=1` URL'de, sunucu içinde geçirdiğiniz aynı verileri açar. `POST` yöntem. -Ayrıca kullanmayı seçebilirsiniz [HTTP sıkıştırma](https://en.wikipedia.org/wiki/HTTP_compression). Sıkıştırılmış bir göndermek için `POST` istek, istek başlığını Ekle `Content-Encoding: compression_method`. Clickhouse'un yanıtı sıkıştırması için şunları eklemelisiniz `Accept-Encoding: compression_method`. ClickHouse destekler `gzip`, `br`, ve `deflate` [sıkıştırma yöntemleri](https://en.wikipedia.org/wiki/HTTP_compression#Content-Encoding_tokens). HTTP sıkıştırmasını etkinleştirmek için Clickhouse'u kullanmanız gerekir [enable\_http\_compression](../operations/settings/settings.md#settings-enable_http_compression) ayar. Veri sıkıştırma düzeyini [http\_zlib\_compression\_level](#settings-http_zlib_compression_level) tüm sıkıştırma yöntemleri için ayarlama. +Ayrıca kullanmayı seçebilirsiniz [HTTP sıkıştırma](https://en.wikipedia.org/wiki/HTTP_compression). Sıkıştırılmış bir göndermek için `POST` istek, istek başlığını Ekle `Content-Encoding: compression_method`. Clickhouse'un yanıtı sıkıştırması için şunları eklemelisiniz `Accept-Encoding: compression_method`. ClickHouse destekler `gzip`, `br`, ve `deflate` [sıkıştırma yöntemleri](https://en.wikipedia.org/wiki/HTTP_compression#Content-Encoding_tokens). HTTP sıkıştırmasını etkinleştirmek için Clickhouse'u kullanmanız gerekir [enable_http_compression](../operations/settings/settings.md#settings-enable_http_compression) ayar. Veri sıkıştırma düzeyini [http_zlib_compression_level](#settings-http_zlib_compression_level) tüm sıkıştırma yöntemleri için ayarlama. Bunu, büyük miktarda veri iletirken ağ trafiğini azaltmak veya hemen sıkıştırılmış dökümler oluşturmak için kullanabilirsiniz. @@ -214,7 +214,7 @@ $ echo 'SELECT 1' | curl -H 'X-ClickHouse-User: user' -H 'X-ClickHouse-Key: pass ``` Kullanıcı adı belirtilmemişse, `default` adı kullanılır. Parola belirtilmezse, boş parola kullanılır. -Tek bir sorguyu veya ayarların tüm profillerini işlemek için herhangi bir ayar belirtmek için URL parametrelerini de kullanabilirsiniz. Örnek: http: / / localhost: 8123/?profil = web & max\_rows\_to\_read = 1000000000 & query = seç + 1 +Tek bir sorguyu veya ayarların tüm profillerini işlemek için herhangi bir ayar belirtmek için URL parametrelerini de kullanabilirsiniz. Örnek: http: / / localhost: 8123/?profil = web & max_rows_to_read = 1000000000 & query = seç + 1 Daha fazla bilgi için, bkz: [Ayarlar](../operations/settings/index.md) bölme. @@ -236,7 +236,7 @@ Diğer parametreler hakkında bilgi için bölüme bakın “SET”. Benzer şekilde, http protokolünde ClickHouse oturumlarını kullanabilirsiniz. Bunu yapmak için şunları eklemeniz gerekir: `session_id` İsteğe parametre alın. Oturum kimliği olarak herhangi bir dize kullanabilirsiniz. Varsayılan olarak, oturum 60 saniye hareketsizlik sonra sonlandırılır. Bu zaman aşımını değiştirmek için, `default_session_timeout` sunucu yapılandırmasında ayarlama veya `session_timeout` İsteğe parametre alın. Oturum durumunu kontrol etmek için `session_check=1` parametre. Bir kerede yalnızca bir sorgu, tek bir oturum içinde çalıştırılabilir. -Bir sorgunun ilerleme durumu hakkında bilgi alabilirsiniz `X-ClickHouse-Progress` yanıt başlıkları. Bunu yapmak için etkinleştir [send\_progress\_in\_http\_headers](../operations/settings/settings.md#settings-send_progress_in_http_headers). Başlık dizisi örneği: +Bir sorgunun ilerleme durumu hakkında bilgi alabilirsiniz `X-ClickHouse-Progress` yanıt başlıkları. Bunu yapmak için etkinleştir [send_progress_in_http_headers](../operations/settings/settings.md#settings-send_progress_in_http_headers). Başlık dizisi örneği: ``` text X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128"} @@ -253,9 +253,9 @@ Olası başlık alanları: - `written_bytes` — Volume of data written in bytes. Http bağlantısı kaybolursa çalışan istekler otomatik olarak durmaz. Ayrıştırma ve veri biçimlendirme sunucu tarafında gerçekleştirilir ve ağ kullanarak etkisiz olabilir. -Opsiyonel ‘query\_id’ parametre sorgu kimliği (herhangi bir dize) geçirilebilir. Daha fazla bilgi için bölüme bakın “Settings, replace\_running\_query”. +Opsiyonel ‘query_id’ parametre sorgu kimliği (herhangi bir dize) geçirilebilir. Daha fazla bilgi için bölüme bakın “Settings, replace_running_query”. -Opsiyonel ‘quota\_key’ parametre kota anahtarı (herhangi bir dize) olarak geçirilebilir. Daha fazla bilgi için bölüme bakın “Quotas”. +Opsiyonel ‘quota_key’ parametre kota anahtarı (herhangi bir dize) olarak geçirilebilir. Daha fazla bilgi için bölüme bakın “Quotas”. HTTP arabirimi, sorgulamak için dış verileri (dış geçici tablolar) geçirmenize izin verir. Daha fazla bilgi için bölüme bakın “External data for query processing”. @@ -377,11 +377,11 @@ $ curl -v 'http://localhost:8123/predefined_query' > `` HTTP isteğinin başlık kısmını eşleştirmekten sorumludur. Bu re2 düzenli ifadeler ile uyumludur. İsteğe bağlı bir yapılandırmadır. Yapılandırma dosyasında tanımlanmamışsa, HTTP isteğinin üstbilgi bölümü eşleşmiyor. > > `` ana işleme bölümünü içerir. Şimdi `` Yapılandır configureılabilir ``, ``, ``, ``, ``, ``. -> \> `` şu anda üç tip destekler: **predefined\_query\_handler**, **dynamic\_query\_handler**, **sabit**. +> \> `` şu anda üç tip destekler: **predefined_query_handler**, **dynamic_query_handler**, **sabit**. > \> -> \> `` - işleyici çağrıldığında predefined\_query\_handler türü ile kullanın, sorgu yürütür. +> \> `` - işleyici çağrıldığında predefined_query_handler türü ile kullanın, sorgu yürütür. > \> -> \> `` - dynamic\_query\_handler tipi ile kullanın, özler ve karşılık gelen değeri yürütür `` HTTP isteği params değeri. +> \> `` - dynamic_query_handler tipi ile kullanın, özler ve karşılık gelen değeri yürütür `` HTTP isteği params değeri. > \> > \> `` - statik tip, yanıt durum kodu ile kullanın. > \> @@ -391,9 +391,9 @@ $ curl -v 'http://localhost:8123/predefined_query' Sonraki farklı yapılandırma yöntemleri ``. -## predefined\_query\_handler {#predefined_query_handler} +## predefined_query_handler {#predefined_query_handler} -`` ayar ayarları ve query\_params değerlerini destekler. Yapılandırabilirsiniz `` tip ininde ``. +`` ayar ayarları ve query_params değerlerini destekler. Yapılandırabilirsiniz `` tip ininde ``. `` değer, önceden tanımlanmış bir sorgudur ``, bir HTTP isteği eşleştirildiğinde ve sorgunun sonucu döndürüldüğünde ClickHouse tarafından yürütülür. Bu bir zorunluluktur yapılandırma. @@ -428,13 +428,13 @@ max_alter_threads 2 !!! note "Dikkat" Birinde `` sadece birini destekler `` bir ekleme türü. -## dynamic\_query\_handler {#dynamic_query_handler} +## dynamic_query_handler {#dynamic_query_handler} İçinde ``, sorgu HTTP isteğinin param şeklinde yazılır. Fark şu ki ``, sorgu yapılandırma dosyasında yazılır. Yapılandırabilirsiniz `` içinde ``. ClickHouse ayıklar ve karşılık gelen değeri yürütür `` HTTP isteğinin url'sindeki değer. Varsayılan değer `` oluyor `/query` . İsteğe bağlı bir yapılandırmadır. Yapılandırma dosyasında tanım yoksa, param iletilmez. -Bu işlevselliği denemek için örnek max\_threads ve max\_alter\_threads değerlerini tanımlar ve ayarların başarıyla ayarlanıp ayarlanmadığını sorgular. +Bu işlevselliği denemek için örnek max_threads ve max_alter_threads değerlerini tanımlar ve ayarların başarıyla ayarlanıp ayarlanmadığını sorgular. Örnek: @@ -459,7 +459,7 @@ max_alter_threads 2 ## sabit {#static} -`` dön canebilir [content\_type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type), [durum](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status) ve response\_content. response\_content belirtilen içeriği döndürebilir +`` dön canebilir [content_type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type), [durum](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status) ve response_content. response_content belirtilen içeriği döndürebilir Örnek: diff --git a/docs/tr/interfaces/mysql.md b/docs/tr/interfaces/mysql.md index 72df467d0ad..1d8c10cf209 100644 --- a/docs/tr/interfaces/mysql.md +++ b/docs/tr/interfaces/mysql.md @@ -7,7 +7,7 @@ toc_title: "MySQL Aray\xFCz\xFC" # MySQL Arayüzü {#mysql-interface} -ClickHouse MySQL Tel protokolünü destekler. Tarafından etkinleştir canilebilir [mysql\_port](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-mysql_port) yapılandırma dosyasında ayarlama: +ClickHouse MySQL Tel protokolünü destekler. Tarafından etkinleştir canilebilir [mysql_port](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-mysql_port) yapılandırma dosyasında ayarlama: ``` xml 9004 diff --git a/docs/tr/interfaces/third-party/client-libraries.md b/docs/tr/interfaces/third-party/client-libraries.md index 7fba28f662a..f47de125299 100644 --- a/docs/tr/interfaces/third-party/client-libraries.md +++ b/docs/tr/interfaces/third-party/client-libraries.md @@ -11,7 +11,7 @@ toc_title: "\u0130stemci Kitapl\u0131klar\u0131" Yandex yapar **değil** Aşağıda listelenen kütüphaneleri koruyun ve kalitelerini sağlamak için kapsamlı bir test yapmadınız. - Piton - - [ınfi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm) + - [ınfi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm) - [clickhouse-sürücü](https://github.com/mymarilyn/clickhouse-driver) - [clickhouse-müşteri](https://github.com/yurial/clickhouse-client) - [aiochclient](https://github.com/maximdanilchenko/aiochclient) @@ -47,7 +47,7 @@ toc_title: "\u0130stemci Kitapl\u0131klar\u0131" - [clickhouse-Scala-istemci](https://github.com/crobox/clickhouse-scala-client) - Kotlin - [AORM](https://github.com/TanVD/AORM) -- C\# +- C# - [Octonica.ClickHouseClient](https://github.com/Octonica/ClickHouseClient) - [ClickHouse.Gürültü](https://github.com/killwort/ClickHouse-Net) - [ClickHouse.Müşteri](https://github.com/DarkWanderer/ClickHouse.Client) diff --git a/docs/tr/interfaces/third-party/integrations.md b/docs/tr/interfaces/third-party/integrations.md index d28046272b7..8e38b2c5ba3 100644 --- a/docs/tr/interfaces/third-party/integrations.md +++ b/docs/tr/interfaces/third-party/integrations.md @@ -19,15 +19,15 @@ toc_title: Entegrasyonlar - [clickhouse-mysql-data-reader](https://github.com/Altinity/clickhouse-mysql-data-reader) - [horgh-çoğaltıcı](https://github.com/larsnovikov/horgh-replicator) - [PostgreSQL](https://www.postgresql.org) - - [clickhousedb\_fdw](https://github.com/Percona-Lab/clickhousedb_fdw) - - [ınfi.clickhouse\_fdw](https://github.com/Infinidat/infi.clickhouse_fdw) (kullanma [ınfi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) + - [clickhousedb_fdw](https://github.com/Percona-Lab/clickhousedb_fdw) + - [ınfi.clickhouse_fdw](https://github.com/Infinidat/infi.clickhouse_fdw) (kullanma [ınfi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - [pg2ch](https://github.com/mkabilov/pg2ch) - - [clickhouse\_fdw](https://github.com/adjust/clickhouse_fdw) + - [clickhouse_fdw](https://github.com/adjust/clickhouse_fdw) - [MSSQL](https://en.wikipedia.org/wiki/Microsoft_SQL_Server) - [ClickHouseMigrator](https://github.com/zlzforever/ClickHouseMigrator) - Mesaj kuyrukları - [Kafka](https://kafka.apache.org) - - [clickhouse\_sinker](https://github.com/housepower/clickhouse_sinker) (kullanma [Go client](https://github.com/ClickHouse/clickhouse-go/)) + - [clickhouse_sinker](https://github.com/housepower/clickhouse_sinker) (kullanma [Go client](https://github.com/ClickHouse/clickhouse-go/)) - [stream-loader-clickhouse](https://github.com/adform/stream-loader) - Akış işleme - [Flink](https://flink.apache.org) @@ -51,12 +51,12 @@ toc_title: Entegrasyonlar - [Grafana](https://grafana.com/) - [clickhouse-grafana](https://github.com/Vertamedia/clickhouse-grafana) - [Prometheus](https://prometheus.io/) - - [clickhouse\_exporter](https://github.com/f1yegor/clickhouse_exporter) + - [clickhouse_exporter](https://github.com/f1yegor/clickhouse_exporter) - [PromHouse](https://github.com/Percona-Lab/PromHouse) - - [clickhouse\_exporter](https://github.com/hot-wifi/clickhouse_exporter) (kullanma [Go client](https://github.com/kshvakov/clickhouse/)) + - [clickhouse_exporter](https://github.com/hot-wifi/clickhouse_exporter) (kullanma [Go client](https://github.com/kshvakov/clickhouse/)) - [Nagios](https://www.nagios.org/) - - [check\_clickhouse](https://github.com/exogroup/check_clickhouse/) - - [check\_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py) + - [check_clickhouse](https://github.com/exogroup/check_clickhouse/) + - [check_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py) - [Zabbix](https://www.zabbix.com) - [clickhouse-zabbix-şablon](https://github.com/Altinity/clickhouse-zabbix-template) - [Sematext](https://sematext.com/) @@ -76,7 +76,7 @@ toc_title: Entegrasyonlar - Piton - [SQLAlchemy](https://www.sqlalchemy.org) - - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (kullanma [ınfi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) + - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (kullanma [ınfi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - [Pandalar](https://pandas.pydata.org) - [pandahouse](https://github.com/kszucs/pandahouse) - PHP @@ -91,7 +91,7 @@ toc_title: Entegrasyonlar - SC scalaala - [Akka](https://akka.io) - [clickhouse-Scala-istemci](https://github.com/crobox/clickhouse-scala-client) -- C\# +- C# - [ADO.NET](https://docs.microsoft.com/en-us/dotnet/framework/data/adonet/ado-net-overview) - [ClickHouse.Gürültü](https://github.com/killwort/ClickHouse-Net) - [ClickHouse.Müşteri](https://github.com/DarkWanderer/ClickHouse.Client) @@ -99,7 +99,7 @@ toc_title: Entegrasyonlar - [ClickHouse. Net. Migrations](https://github.com/ilyabreev/ClickHouse.Net.Migrations) - İksir - [Ecto](https://github.com/elixir-ecto/ecto) - - [clickhouse\_ecto](https://github.com/appodeal/clickhouse_ecto) + - [clickhouse_ecto](https://github.com/appodeal/clickhouse_ecto) - Ruby - [Ruby on Rails](https://rubyonrails.org/) - [activecube](https://github.com/bitquery/activecube) diff --git a/docs/tr/operations/access-rights.md b/docs/tr/operations/access-rights.md index 87c7bf5a373..f151bbfed3f 100644 --- a/docs/tr/operations/access-rights.md +++ b/docs/tr/operations/access-rights.md @@ -134,10 +134,10 @@ Yönetim sorguları: - Yapılandırmaları depolama için bir dizin Kur. - ClickHouse, erişim varlık yapılandırmalarını, [access\_control\_path](server-configuration-parameters/settings.md#access_control_path) sunucu yapılandırma parametresi. + ClickHouse, erişim varlık yapılandırmalarını, [access_control_path](server-configuration-parameters/settings.md#access_control_path) sunucu yapılandırma parametresi. - En az bir kullanıcı hesabı için SQL tabanlı erişim denetimi ve hesap yönetimini etkinleştirin. - Varsayılan olarak SQL güdümlü erişim denetimi ve hesap yönetimi, tüm kullanıcılar için açık. En az bir kullanıcı yapılandırmanız gerekir `users.xml` yapılandırma dosyası ve atama 1 [access\_management](settings/settings-users.md#access_management-user-setting) ayar. + Varsayılan olarak SQL güdümlü erişim denetimi ve hesap yönetimi, tüm kullanıcılar için açık. En az bir kullanıcı yapılandırmanız gerekir `users.xml` yapılandırma dosyası ve atama 1 [access_management](settings/settings-users.md#access_management-user-setting) ayar. [Orijinal makale](https://clickhouse.tech/docs/en/operations/access_rights/) diff --git a/docs/tr/operations/configuration-files.md b/docs/tr/operations/configuration-files.md index 59f268981a3..bc818bdcf50 100644 --- a/docs/tr/operations/configuration-files.md +++ b/docs/tr/operations/configuration-files.md @@ -20,7 +20,7 @@ Eğer `replace` belirtilen, tüm öğeyi belirtilen ile değiştirir. Eğer `remove` belirt .ilirse, öğeyi siler. -Yapılandırma ayrıca tanımlayabilir “substitutions”. Bir öğe varsa `incl` öznitelik, dosyadan karşılık gelen ikame değeri olarak kullanılacaktır. Varsayılan olarak, değiştirmeler ile dosyanın yolu `/etc/metrika.xml`. Bu değiştirilebilir [include\_from](server-configuration-parameters/settings.md#server_configuration_parameters-include_from) sunucu yapılandırmasında öğe. İkame değerleri belirtilen `/yandex/substitution_name` bu dosyadaki öğeler. Belirtilen bir ika Ame halinde `incl` yok, günlüğe kaydedilir. Clickhouse'un eksik değiştirmelerin günlüğe kaydedilmesini önlemek için `optional="true"` öznitelik (örneğin, ayarlar [makrolar](server-configuration-parameters/settings.md)). +Yapılandırma ayrıca tanımlayabilir “substitutions”. Bir öğe varsa `incl` öznitelik, dosyadan karşılık gelen ikame değeri olarak kullanılacaktır. Varsayılan olarak, değiştirmeler ile dosyanın yolu `/etc/metrika.xml`. Bu değiştirilebilir [include_from](server-configuration-parameters/settings.md#server_configuration_parameters-include_from) sunucu yapılandırmasında öğe. İkame değerleri belirtilen `/yandex/substitution_name` bu dosyadaki öğeler. Belirtilen bir ika Ame halinde `incl` yok, günlüğe kaydedilir. Clickhouse'un eksik değiştirmelerin günlüğe kaydedilmesini önlemek için `optional="true"` öznitelik (örneğin, ayarlar [makrolar](server-configuration-parameters/settings.md)). İkame da ZooKeeper yapılabilir. Bunu yapmak için özniteliği belirtin `from_zk = "/path/to/node"`. Eleman değeri, düğümün içeriği ile değiştirilir `/path/to/node` ZooKeeper. Ayrıca ZooKeeper düğümünde bir XML alt ağacının tamamını koyabilirsiniz ve kaynak öğeye tamamen eklenecektir. diff --git a/docs/tr/operations/monitoring.md b/docs/tr/operations/monitoring.md index bed1b9985ef..8488be4202d 100644 --- a/docs/tr/operations/monitoring.md +++ b/docs/tr/operations/monitoring.md @@ -35,7 +35,7 @@ ClickHouse toplar: - Sunucunun hesaplama kaynaklarını nasıl kullandığına dair farklı metrikler. - Sorgu işleme ile ilgili ortak istatistikler. -Metrikleri şu adreste bulabilirsiniz: [sistem.metrik](../operations/system-tables.md#system_tables-metrics), [sistem.etkinlik](../operations/system-tables.md#system_tables-events), ve [sistem.asynchronous\_metrics](../operations/system-tables.md#system_tables-asynchronous_metrics) Tablolar. +Metrikleri şu adreste bulabilirsiniz: [sistem.metrik](../operations/system-tables.md#system_tables-metrics), [sistem.etkinlik](../operations/system-tables.md#system_tables-events), ve [sistem.asynchronous_metrics](../operations/system-tables.md#system_tables-asynchronous_metrics) Tablolar. Clickhouse'u metrikleri dışa aktaracak şekilde yapılandırabilirsiniz [Grafit](https://github.com/graphite-project). Görmek [Graf sectionit bölümü](server-configuration-parameters/settings.md#server_configuration_parameters-graphite) ClickHouse sunucu yapılandırma dosyasında. Metriklerin dışa aktarımını yapılandırmadan önce, grafit'i resmi olarak takip ederek ayarlamanız gerekir [kılavuz](https://graphite.readthedocs.io/en/latest/install.html). @@ -43,4 +43,4 @@ Clickhouse'u metrikleri dışa aktaracak şekilde yapılandırabilirsiniz [Prome Ayrıca, http API aracılığıyla sunucu kullanılabilirliğini izleyebilirsiniz. Sen sendd the `HTTP GET` istek için `/ping`. Sunucu mevcutsa, yanıt verir `200 OK`. -Bir küme yapılandırmasındaki sunucuları izlemek için [max\_replica\_delay\_for\_distributed\_queries](settings/settings.md#settings-max_replica_delay_for_distributed_queries) parametre ve HTTP kaynağını kullanın `/replicas_status`. Bir istek için `/replicas_status` dönüşler `200 OK` çoğaltma kullanılabilir ve diğer yinelemeler gecikmiş değil. Bir çoğaltma gecikirse, döndürür `503 HTTP_SERVICE_UNAVAILABLE` boşluk hakkında bilgi ile. +Bir küme yapılandırmasındaki sunucuları izlemek için [max_replica_delay_for_distributed_queries](settings/settings.md#settings-max_replica_delay_for_distributed_queries) parametre ve HTTP kaynağını kullanın `/replicas_status`. Bir istek için `/replicas_status` dönüşler `200 OK` çoğaltma kullanılabilir ve diğer yinelemeler gecikmiş değil. Bir çoğaltma gecikirse, döndürür `503 HTTP_SERVICE_UNAVAILABLE` boşluk hakkında bilgi ile. diff --git a/docs/tr/operations/optimizing-performance/sampling-query-profiler.md b/docs/tr/operations/optimizing-performance/sampling-query-profiler.md index e73151397e9..4efa811fa8b 100644 --- a/docs/tr/operations/optimizing-performance/sampling-query-profiler.md +++ b/docs/tr/operations/optimizing-performance/sampling-query-profiler.md @@ -11,11 +11,11 @@ ClickHouse, sorgu yürütülmesini analiz etmeyi sağlayan örnekleme profiler'i Profiler kullanmak için: -- Kurulum [trace\_log](../server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) sunucu yapılandırması bölümü. +- Kurulum [trace_log](../server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) sunucu yapılandırması bölümü. - Bu bölüm yapılandırır [trace\_log](../../operations/system-tables.md#system_tables-trace_log) profiler işleyişinin sonuçlarını içeren sistem tablosu. Varsayılan olarak yapılandırılmıştır. Bu tablodaki verilerin yalnızca çalışan bir sunucu için geçerli olduğunu unutmayın. Sunucu yeniden başlatıldıktan sonra ClickHouse tabloyu temizlemez ve depolanan tüm sanal bellek adresi geçersiz hale gelebilir. + Bu bölüm yapılandırır [trace_log](../../operations/system-tables.md#system_tables-trace_log) profiler işleyişinin sonuçlarını içeren sistem tablosu. Varsayılan olarak yapılandırılmıştır. Bu tablodaki verilerin yalnızca çalışan bir sunucu için geçerli olduğunu unutmayın. Sunucu yeniden başlatıldıktan sonra ClickHouse tabloyu temizlemez ve depolanan tüm sanal bellek adresi geçersiz hale gelebilir. -- Kurulum [query\_profiler\_cpu\_time\_period\_ns](../settings/settings.md#query_profiler_cpu_time_period_ns) veya [query\_profiler\_real\_time\_period\_ns](../settings/settings.md#query_profiler_real_time_period_ns) ayarlar. Her iki ayar da aynı anda kullanılabilir. +- Kurulum [query_profiler_cpu_time_period_ns](../settings/settings.md#query_profiler_cpu_time_period_ns) veya [query_profiler_real_time_period_ns](../settings/settings.md#query_profiler_real_time_period_ns) ayarlar. Her iki ayar da aynı anda kullanılabilir. Bu ayarlar, profiler zamanlayıcılarını yapılandırmanıza izin verir. Bunlar oturum ayarları olduğundan, tüm sunucu, bireysel kullanıcılar veya kullanıcı profilleri, etkileşimli oturumunuz ve her bir sorgu için farklı örnekleme sıklığı elde edebilirsiniz. @@ -25,7 +25,7 @@ Analiz etmek `trace_log` sistem tablosu: - Yüklemek `clickhouse-common-static-dbg` paket. Görmek [DEB paketlerinden yükleyin](../../getting-started/install.md#install-from-deb-packages). -- Tarafından iç gözlem işlevlerine izin ver [allow\_introspection\_functions](../settings/settings.md#settings-allow_introspection_functions) ayar. +- Tarafından iç gözlem işlevlerine izin ver [allow_introspection_functions](../settings/settings.md#settings-allow_introspection_functions) ayar. Güvenlik nedenleriyle, iç gözlem işlevleri varsayılan olarak devre dışı bırakılır. diff --git a/docs/tr/operations/requirements.md b/docs/tr/operations/requirements.md index c45fc93e068..790d45282a8 100644 --- a/docs/tr/operations/requirements.md +++ b/docs/tr/operations/requirements.md @@ -9,7 +9,7 @@ toc_title: Gereksinimler ## CPU {#cpu} -Önceden oluşturulmuş deb paketlerinden kurulum için, x86\_64 mimarisine sahip bir CPU kullanın ve sse 4.2 yönergelerini destekleyin. Clickhouse'u SSE 4.2'yi desteklemeyen veya AArch64 veya PowerPC64LE mimarisine sahip işlemcilerle çalıştırmak için, kaynaklardan Clickhouse'u oluşturmanız gerekir. +Önceden oluşturulmuş deb paketlerinden kurulum için, x86_64 mimarisine sahip bir CPU kullanın ve sse 4.2 yönergelerini destekleyin. Clickhouse'u SSE 4.2'yi desteklemeyen veya AArch64 veya PowerPC64LE mimarisine sahip işlemcilerle çalıştırmak için, kaynaklardan Clickhouse'u oluşturmanız gerekir. ClickHouse paralel veri işleme uygular ve mevcut tüm donanım kaynaklarını kullanır. Bir işlemci seçerken, Clickhouse'un çok sayıda çekirdeğe sahip konfigürasyonlarda daha verimli çalıştığını, ancak daha az çekirdeğe ve daha yüksek bir saat hızına sahip konfigürasyonlardan daha düşük bir saat hızına sahip olduğunu göz önünde bulundurun. Örneğin, 2600 MHz'lik 16 çekirdek, 3600 MHz'lik 8 çekirdeğe tercih edilir. diff --git a/docs/tr/operations/server-configuration-parameters/settings.md b/docs/tr/operations/server-configuration-parameters/settings.md index cc5ef3e8e21..431044a0be1 100644 --- a/docs/tr/operations/server-configuration-parameters/settings.md +++ b/docs/tr/operations/server-configuration-parameters/settings.md @@ -7,7 +7,7 @@ toc_title: "Sunucu Ayarlar\u0131" # Sunucu Ayarları {#server-settings} -## buıltın\_dıctıonarıes\_reload\_ınterval {#builtin-dictionaries-reload-interval} +## buıltın_dıctıonarıes_reload_ınterval {#builtin-dictionaries-reload-interval} Dahili sözlükleri yeniden yüklemeden önce saniye cinsinden Aralık. @@ -68,7 +68,7 @@ Bir veri bölümü için herhangi bir koşul karşılanmazsa, ClickHouse `lz4` s ``` -## default\_database {#default-database} +## default_database {#default-database} Varsayılan veritabanı. @@ -80,7 +80,7 @@ Veritabanlarının bir listesini almak için [SHOW DATABASES](../../sql-referenc default ``` -## default\_profile {#default-profile} +## default_profile {#default-profile} Varsayılan ayarlar profili. @@ -92,7 +92,7 @@ Ayarlar profilleri parametrede belirtilen dosyada bulunur `user_config`. default ``` -## dictionaries\_config {#server_configuration_parameters-dictionaries_config} +## dictionaries_config {#server_configuration_parameters-dictionaries_config} Dış sözlükler için yapılandırma dosyasının yolu. @@ -109,7 +109,7 @@ Ayrıca bakınız “[Dış söz dictionarieslükler](../../sql-reference/dictio *_dictionary.xml ``` -## dictionaries\_lazy\_load {#server_configuration_parameters-dictionaries_lazy_load} +## dictionaries_lazy_load {#server_configuration_parameters-dictionaries_lazy_load} Sözlüklerin tembel yüklenmesi. @@ -125,7 +125,7 @@ Varsayılan değer `true`. true ``` -## format\_schema\_path {#server_configuration_parameters-format_schema_path} +## format_schema_path {#server_configuration_parameters-format_schema_path} Dizin için şemalar gibi giriş verileri için şemaları ile yolu [CapnProto](../../interfaces/formats.md#capnproto) biçimli. @@ -146,11 +146,11 @@ Ayarlar: - port – The port on the Graphite server. - interval – The interval for sending, in seconds. - timeout – The timeout for sending data, in seconds. -- root\_path – Prefix for keys. +- root_path – Prefix for keys. - metrics – Sending data from the [sistem.metrik](../../operations/system-tables.md#system_tables-metrics) Tablo. - events – Sending deltas data accumulated for the time period from the [sistem.etkinlik](../../operations/system-tables.md#system_tables-events) Tablo. -- events\_cumulative – Sending cumulative data from the [sistem.etkinlik](../../operations/system-tables.md#system_tables-events) Tablo. -- asynchronous\_metrics – Sending data from the [sistem.asynchronous\_metrics](../../operations/system-tables.md#system_tables-asynchronous_metrics) Tablo. +- events_cumulative – Sending cumulative data from the [sistem.etkinlik](../../operations/system-tables.md#system_tables-events) Tablo. +- asynchronous_metrics – Sending data from the [sistem.asynchronous_metrics](../../operations/system-tables.md#system_tables-asynchronous_metrics) Tablo. Birden fazla yapılandırabilirsiniz `` yanlar. Örneğin, bunu farklı aralıklarla farklı veri göndermek için kullanabilirsiniz. @@ -170,7 +170,7 @@ Birden fazla yapılandırabilirsiniz `` yanlar. Örneğin, bunu farkl ``` -## graphite\_rollup {#server_configuration_parameters-graphite-rollup} +## graphite_rollup {#server_configuration_parameters-graphite-rollup} Grafit için inceltme verileri için ayarlar. @@ -198,7 +198,7 @@ Daha fazla ayrıntı için bkz. [Graphıtemergetree](../../engines/table-engines ``` -## http\_port/https\_port {#http-porthttps-port} +## http_port/https_port {#http-porthttps-port} Http(ler) üzerinden sunucuya bağlanmak için bağlantı noktası. @@ -212,7 +212,7 @@ Eğer `http_port` belirtilmişse, OpenSSL yapılandırması ayarlanmış olsa bi 9999 ``` -## http\_server\_default\_response {#server_configuration_parameters-http_server_default_response} +## http_server_default_response {#server_configuration_parameters-http_server_default_response} ClickHouse HTTP (s) sunucusuna eriştiğinizde varsayılan olarak gösterilen sayfa. Varsayılan değer “Ok.” (sonunda bir çizgi besleme ile) @@ -227,7 +227,7 @@ Açıyor `https://tabix.io/` eriş whenirken `http://localhost: http_port`. ``` -## include\_from {#server_configuration_parameters-include_from} +## include_from {#server_configuration_parameters-include_from} Değiştirmeleri ile dosyanın yolu. @@ -239,7 +239,7 @@ Daha fazla bilgi için bölüme bakın “[Yapılandırma dosyaları](../configu /etc/metrica.xml ``` -## ınterserver\_http\_port {#interserver-http-port} +## ınterserver_http_port {#interserver-http-port} ClickHouse sunucuları arasında veri alışverişi için bağlantı noktası. @@ -249,7 +249,7 @@ ClickHouse sunucuları arasında veri alışverişi için bağlantı noktası. 9009 ``` -## ınterserver\_http\_host {#interserver-http-host} +## ınterserver_http_host {#interserver-http-host} Bu sunucuya erişmek için diğer sunucular tarafından kullanılabilecek ana bilgisayar adı. @@ -263,7 +263,7 @@ Belirli bir ağ arayüzünden kopmak için kullanışlıdır. example.yandex.ru ``` -## ınterserver\_http\_credentials {#server-settings-interserver-http-credentials} +## ınterserver_http_credentials {#server-settings-interserver-http-credentials} Sırasında kimlik doğrulaması için kullanılan kullanıcı adı ve şifre [çoğalma](../../engines/table-engines/mergetree-family/replication.md) çoğaltılan \* motorlarla. Bu kimlik bilgileri yalnızca yinelemeler arasındaki iletişim için kullanılır ve ClickHouse istemcileri için kimlik bilgileri ile ilgisizdir. Sunucu, yinelemeleri bağlamak için bu kimlik bilgilerini denetliyor ve diğer yinelemelere bağlanırken aynı kimlik bilgilerini kullanıyor. Bu nedenle, bu kimlik bilgileri kümedeki tüm yinelemeler için aynı şekilde ayarlanmalıdır. Varsayılan olarak, kimlik doğrulama kullanılmaz. @@ -282,7 +282,7 @@ Bu bölüm aşağıdaki parametreleri içerir: ``` -## keep\_alive\_timeout {#keep-alive-timeout} +## keep_alive_timeout {#keep-alive-timeout} ClickHouse bağlantıyı kapatmadan önce gelen istekleri bekler saniye sayısı. Varsayılan 3 saniye. @@ -292,7 +292,7 @@ ClickHouse bağlantıyı kapatmadan önce gelen istekleri bekler saniye sayısı 3 ``` -## listen\_host {#server_configuration_parameters-listen_host} +## listen_host {#server_configuration_parameters-listen_host} İsteklerin gelebileceği ana bilgisayarlarda kısıtlama. Sunucunun hepsini yanıtlamasını istiyorsanız, belirtin `::`. @@ -343,10 +343,10 @@ Syslog yazma da desteklenmektedir. Yapılandırma örneği: Anahtarlar: -- use\_syslog — Required setting if you want to write to the syslog. +- use_syslog — Required setting if you want to write to the syslog. - address — The host\[:port\] of syslogd. If omitted, the local daemon is used. - hostname — Optional. The name of the host that logs are sent from. -- facility — [Syslog tesisi anahtar sözcüğü](https://en.wikipedia.org/wiki/Syslog#Facility) ile büyük harf inlerle “LOG\_” önek: (`LOG_USER`, `LOG_DAEMON`, `LOG_LOCAL3` vb.). +- facility — [Syslog tesisi anahtar sözcüğü](https://en.wikipedia.org/wiki/Syslog#Facility) ile büyük harf inlerle “LOG_” önek: (`LOG_USER`, `LOG_DAEMON`, `LOG_LOCAL3` vb.). Varsayılan değer: `LOG_USER` eğer `address` belirtilen, `LOG_DAEMON otherwise.` - format – Message format. Possible values: `bsd` ve `syslog.` @@ -364,7 +364,7 @@ Daha fazla bilgi için bölüme bakın “[Çoğaltılmış tablolar oluşturma] ``` -## mark\_cache\_size {#server-mark-cache-size} +## mark_cache_size {#server-mark-cache-size} Tablo motorları tarafından kullanılan işaretlerin önbelleğinin yaklaşık boyutu (bayt cinsinden) [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) aile. @@ -376,7 +376,7 @@ Tablo motorları tarafından kullanılan işaretlerin önbelleğinin yaklaşık 5368709120 ``` -## max\_concurrent\_queries {#max-concurrent-queries} +## max_concurrent_queries {#max-concurrent-queries} Aynı anda işlenen isteklerin maksimum sayısı. @@ -386,7 +386,7 @@ Aynı anda işlenen isteklerin maksimum sayısı. 100 ``` -## max\_connections {#max-connections} +## max_connections {#max-connections} En fazla gelen bağlantı sayısı. @@ -396,7 +396,7 @@ En fazla gelen bağlantı sayısı. 4096 ``` -## max\_open\_files {#max-open-files} +## max_open_files {#max-open-files} Maksimum açık dosya sayısı. @@ -410,7 +410,7 @@ Biz beri Mac OS X bu seçeneği kullanmanızı öneririz `getrlimit()` işlev ya 262144 ``` -## max\_table\_size\_to\_drop {#max-table-size-to-drop} +## max_table_size_to_drop {#max-table-size-to-drop} Tabloları silme konusunda kısıtlama. @@ -428,7 +428,7 @@ Varsayılan değer: 50 GB. 0 ``` -## merge\_tree {#server_configuration_parameters-merge_tree} +## merge_tree {#server_configuration_parameters-merge_tree} Tablolar için ince ayar [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). @@ -463,7 +463,7 @@ Sunucu/istemci ayarları için tuşlar: - sessionTimeout – Time for caching the session on the server. - extendedVerification – Automatically extended verification of certificates after the session ends. Acceptable values: `true`, `false`. - requireTLSv1 – Require a TLSv1 connection. Acceptable values: `true`, `false`. -- requireTLSv1\_1 – Require a TLSv1.1 connection. Acceptable values: `true`, `false`. +- requireTLSv1_1 – Require a TLSv1.1 connection. Acceptable values: `true`, `false`. - requireTLSv1 – Require a TLSv1.2 connection. Acceptable values: `true`, `false`. - fips – Activates OpenSSL FIPS mode. Supported if the library's OpenSSL version supports FIPS. - privateKeyPassphraseHandler – Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: ``, `KeyFileHandler`, `test`, ``. @@ -501,11 +501,11 @@ Sunucu/istemci ayarları için tuşlar: ``` -## part\_log {#server_configuration_parameters-part-log} +## part_log {#server_configuration_parameters-part-log} İlişkili olayları günlüğe kaydetme [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). Örneğin, veri ekleme veya birleştirme. Birleştirme algoritmalarını simüle etmek ve özelliklerini karşılaştırmak için günlüğü kullanabilirsiniz. Birleştirme işlemini görselleştirebilirsiniz. -Sorgular günlüğe kaydedilir [sistem.part\_log](../../operations/system-tables.md#system_tables-part-log) tablo, ayrı bir dosyada değil. Bu tablonun adını aşağıdaki tabloda yapılandırabilirsiniz: `table` parametre (aşağıya bakınız). +Sorgular günlüğe kaydedilir [sistem.part_log](../../operations/system-tables.md#system_tables-part-log) tablo, ayrı bir dosyada değil. Bu tablonun adını aşağıdaki tabloda yapılandırabilirsiniz: `table` parametre (aşağıya bakınız). Günlüğü yapılandırmak için aşağıdaki parametreleri kullanın: @@ -548,7 +548,7 @@ Ayarlar: - `port` – Port for `endpoint`. - `metrics` – Flag that sets to expose metrics from the [sistem.metrik](../system-tables.md#system_tables-metrics) Tablo. - `events` – Flag that sets to expose metrics from the [sistem.etkinlik](../system-tables.md#system_tables-events) Tablo. -- `asynchronous_metrics` – Flag that sets to expose current metrics values from the [sistem.asynchronous\_metrics](../system-tables.md#system_tables-asynchronous_metrics) Tablo. +- `asynchronous_metrics` – Flag that sets to expose current metrics values from the [sistem.asynchronous_metrics](../system-tables.md#system_tables-asynchronous_metrics) Tablo. **Örnek** @@ -562,11 +562,11 @@ Ayarlar: ``` -## query\_log {#server_configuration_parameters-query-log} +## query_log {#server_configuration_parameters-query-log} -İle alınan günlük sorgu settinglarının ayarlanması [log\_queries = 1](../settings/settings.md) ayar. +İle alınan günlük sorgu settinglarının ayarlanması [log_queries = 1](../settings/settings.md) ayar. -Sorgular günlüğe kaydedilir [sistem.query\_log](../../operations/system-tables.md#system_tables-query_log) tablo, ayrı bir dosyada değil. Tablonun adını değiştirebilirsiniz. `table` parametre (aşağıya bakınız). +Sorgular günlüğe kaydedilir [sistem.query_log](../../operations/system-tables.md#system_tables-query_log) tablo, ayrı bir dosyada değil. Tablonun adını değiştirebilirsiniz. `table` parametre (aşağıya bakınız). Günlüğü yapılandırmak için aşağıdaki parametreleri kullanın: @@ -588,11 +588,11 @@ Tablo yoksa, ClickHouse bunu oluşturur. ClickHouse sunucusu güncelleştirildi ``` -## query\_thread\_log {#server_configuration_parameters-query-thread-log} +## query_thread_log {#server_configuration_parameters-query-thread-log} -İle alınan sorguların günlük iş parçacıklarının ayarlanması [log\_query\_threads = 1](../settings/settings.md#settings-log-query-threads) ayar. +İle alınan sorguların günlük iş parçacıklarının ayarlanması [log_query_threads = 1](../settings/settings.md#settings-log-query-threads) ayar. -Sorgular günlüğe kaydedilir [sistem.query\_thread\_log](../../operations/system-tables.md#system_tables-query-thread-log) tablo, ayrı bir dosyada değil. Tablonun adını değiştirebilirsiniz. `table` parametre (aşağıya bakınız). +Sorgular günlüğe kaydedilir [sistem.query_thread_log](../../operations/system-tables.md#system_tables-query-thread-log) tablo, ayrı bir dosyada değil. Tablonun adını değiştirebilirsiniz. `table` parametre (aşağıya bakınız). Günlüğü yapılandırmak için aşağıdaki parametreleri kullanın: @@ -614,9 +614,9 @@ Tablo yoksa, ClickHouse bunu oluşturur. Sorgu iş parçacığı günlüğü yap ``` -## trace\_log {#server_configuration_parameters-trace_log} +## trace_log {#server_configuration_parameters-trace_log} -İçin ayarlar [trace\_log](../../operations/system-tables.md#system_tables-trace_log) sistem tablosu çalışması. +İçin ayarlar [trace_log](../../operations/system-tables.md#system_tables-trace_log) sistem tablosu çalışması. Parametre: @@ -636,7 +636,7 @@ Varsayılan sunucu yapılandırma dosyası `config.xml` aşağıdaki ayarlar bö ``` -## query\_masking\_rules {#query-masking-rules} +## query_masking_rules {#query-masking-rules} Regexp tabanlı kurallar, sorgulara ve tüm günlük iletilerine sunucu günlüklerinde depolamadan önce uygulanacak, `system.query_log`, `system.text_log`, `system.processes` tablo ve istemciye gönderilen günlüklerde. Önlem allowseyi sağlayan @@ -667,7 +667,7 @@ Maskeleme kuralları tüm sorguya uygulanır (hatalı biçimlendirilmiş / ayrı Dağıtılmış sorgular için her sunucu ayrı ayrı yapılandırılmalıdır, aksi takdirde alt sorgular diğerine iletilir düğümler maskeleme olmadan saklanır. -## remote\_servers {#server-settings-remote-servers} +## remote_servers {#server-settings-remote-servers} Tarafından kullanılan küm ofelerin yapılandırması [Dağılı](../../engines/table-engines/special/distributed.md) tablo motoru ve `cluster` tablo işlevi. @@ -681,7 +681,7 @@ Değeri için `incl` öznitelik, bölümüne bakın “[Yapılandırma dosyalar **Ayrıca Bakınız** -- [skip\_unavailable\_shards](../settings/settings.md#settings-skip_unavailable_shards) +- [skip_unavailable_shards](../settings/settings.md#settings-skip_unavailable_shards) ## saat dilimi {#server_configuration_parameters-timezone} @@ -697,7 +697,7 @@ Saat dilimi, datetime alanları metin biçimine (ekranda veya dosyada yazdırıl Europe/Moscow ``` -## tcp\_port {#server_configuration_parameters-tcp_port} +## tcp_port {#server_configuration_parameters-tcp_port} TCP protokolü üzerinden istemcilerle iletişim kurmak için bağlantı noktası. @@ -707,7 +707,7 @@ TCP protokolü üzerinden istemcilerle iletişim kurmak için bağlantı noktas 9000 ``` -## tcp\_port\_secure {#server_configuration_parameters-tcp_port_secure} +## tcp_port_secure {#server_configuration_parameters-tcp_port_secure} İstemcilerle güvenli iletişim için TCP bağlantı noktası. İle kullanın [OpenSSL](#server_configuration_parameters-openssl) ayarlar. @@ -721,7 +721,7 @@ Pozitif tamsayı. 9440 ``` -## mysql\_port {#server_configuration_parameters-mysql_port} +## mysql_port {#server_configuration_parameters-mysql_port} MySQL protokolü üzerinden istemcilerle iletişim kurmak için bağlantı noktası. @@ -735,7 +735,7 @@ Pozitif tamsayı. 9004 ``` -## tmp\_path {#server-settings-tmp_path} +## tmp_path {#server-settings-tmp_path} Büyük sorguları işlemek için geçici veri yolu. @@ -748,7 +748,7 @@ Büyük sorguları işlemek için geçici veri yolu. /var/lib/clickhouse/tmp/ ``` -## tmp\_policy {#server-settings-tmp-policy} +## tmp_policy {#server-settings-tmp-policy} Politika dan [`storage_configuration`](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) geçici dosyaları saklamak için. Set değilse [`tmp_path`](#server-settings-tmp_path) kullanılır, aksi takdirde göz ardı edilir. @@ -759,11 +759,11 @@ Set değilse [`tmp_path`](#server-settings-tmp_path) kullanılır, aksi takdirde - `max_data_part_size_bytes` göz ardı edilir - bu Politikada tam olarak bir cilt olmalı -## uncompressed\_cache\_size {#server-settings-uncompressed_cache_size} +## uncompressed_cache_size {#server-settings-uncompressed_cache_size} Tablo motorları tarafından kullanılan sıkıştırılmamış veriler için önbellek boyutu (bayt cinsinden) [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). -Sunucu için bir paylaşılan önbellek var. Bellek talep üzerine tahsis edilir. Seçenek varsa önbellek kullanılır [use\_uncompressed\_cache](../settings/settings.md#setting-use_uncompressed_cache) etkindir. +Sunucu için bir paylaşılan önbellek var. Bellek talep üzerine tahsis edilir. Seçenek varsa önbellek kullanılır [use_uncompressed_cache](../settings/settings.md#setting-use_uncompressed_cache) etkindir. Sıkıştırılmamış önbellek, tek tek durumlarda çok kısa sorgular için avantajlıdır. @@ -773,7 +773,7 @@ Sıkıştırılmamış önbellek, tek tek durumlarda çok kısa sorgular için a 8589934592 ``` -## user\_files\_path {#server_configuration_parameters-user_files_path} +## user_files_path {#server_configuration_parameters-user_files_path} Kullanıcı dosyaları ile dizin. Tablo işlevinde kullanılır [Dosya()](../../sql-reference/table-functions/file.md). @@ -783,7 +783,7 @@ Kullanıcı dosyaları ile dizin. Tablo işlevinde kullanılır [Dosya()](../../ /var/lib/clickhouse/user_files/ ``` -## users\_config {#users-config} +## users_config {#users-config} İçeren dosyanın yolu: @@ -851,13 +851,13 @@ Bu bölüm aşağıdaki parametreleri içerir: - [Çoğalma](../../engines/table-engines/mergetree-family/replication.md) - [ZooKeeper programcı Kılavuzu](http://zookeeper.apache.org/doc/current/zookeeperProgrammers.html) -## use\_minimalistic\_part\_header\_in\_zookeeper {#server-settings-use_minimalistic_part_header_in_zookeeper} +## use_minimalistic_part_header_in_zookeeper {#server-settings-use_minimalistic_part_header_in_zookeeper} ZooKeeper veri parçası başlıkları için depolama yöntemi. Bu ayar yalnızca `MergeTree` aile. Belirt specifiedilebilir: -- Küresel olarak [merge\_tree](#server_configuration_parameters-merge_tree) bu bölüm `config.xml` Dosya. +- Küresel olarak [merge_tree](#server_configuration_parameters-merge_tree) bu bölüm `config.xml` Dosya. ClickHouse sunucudaki tüm tablolar için ayarı kullanır. Ayarı istediğiniz zaman değiştirebilirsiniz. Mevcut tablolar, ayar değiştiğinde davranışlarını değiştirir. @@ -879,21 +879,21 @@ Eğer `use_minimalistic_part_header_in_zookeeper = 1`, sonraları [çoğaltıyor **Varsayılan değer:** 0. -## disable\_internal\_dns\_cache {#server-settings-disable-internal-dns-cache} +## disable_internal_dns_cache {#server-settings-disable-internal-dns-cache} İç DNS önbelleğini devre dışı bırakır. Sistemlerinde ClickHouse işletim için tavsiye Kubernetes gibi sık sık değişen altyapı ile. **Varsayılan değer:** 0. -## dns\_cache\_update\_period {#server-settings-dns-cache-update-period} +## dns_cache_update_period {#server-settings-dns-cache-update-period} ClickHouse iç DNS önbelleğinde saklanan IP adreslerini güncelleme süresi (saniye cinsinden). Güncelleştirme, ayrı bir sistem iş parçacığında zaman uyumsuz olarak gerçekleştirilir. **Varsayılan değer**: 15. -## access\_control\_path {#access_control_path} +## access_control_path {#access_control_path} ClickHouse sunucusunun SQL komutları tarafından oluşturulan kullanıcı ve rol yapılandırmalarını depoladığı bir klasörün yolu. diff --git a/docs/tr/operations/settings/permissions-for-queries.md b/docs/tr/operations/settings/permissions-for-queries.md index 499d704230f..ae9f95c5172 100644 --- a/docs/tr/operations/settings/permissions-for-queries.md +++ b/docs/tr/operations/settings/permissions-for-queries.md @@ -18,7 +18,7 @@ Clickhouse'daki sorgular birkaç türe ayrılabilir: Aşağıdaki ayarlar, kullanıcı izinlerini sorgu Türüne göre düzenler: - [readonly](#settings_readonly) — Restricts permissions for all types of queries except DDL queries. -- [allow\_ddl](#settings_allow_ddl) — Restricts permissions for DDL queries. +- [allow_ddl](#settings_allow_ddl) — Restricts permissions for DDL queries. `KILL QUERY` herhangi bir ayar ile yapılabilir. @@ -43,7 +43,7 @@ sadece belirli ayarları değiştirmekten, ayrıntılar için bkz [ayarlardaki k Varsayılan değer: 0 -## allow\_ddl {#settings_allow_ddl} +## allow_ddl {#settings_allow_ddl} İzin verir veya reddeder [DDL](https://en.wikipedia.org/wiki/Data_definition_language) sorgular. diff --git a/docs/tr/operations/settings/query-complexity.md b/docs/tr/operations/settings/query-complexity.md index c1880fc2ce7..1aebc2f258a 100644 --- a/docs/tr/operations/settings/query-complexity.md +++ b/docs/tr/operations/settings/query-complexity.md @@ -15,8 +15,8 @@ Hemen hemen tüm kısıtlamalar sadece aşağıdakiler için geçerlidir `SELECT ClickHouse, her satır için değil, veri bölümleri için kısıtlamaları denetler. Bu, veri parçasının boyutu ile kısıtlama değerini aşabileceğiniz anlamına gelir. Üzerindeki kısıtlamalar “maximum amount of something” 0 değerini alabilir, yani “unrestricted”. -Çoğu kısıtlama da bir ‘overflow\_mode’ ayar, sınır aşıldığında ne yapılması gerektiği anlamına gelir. -İki değerden birini alabilir: `throw` veya `break`. Toplama (group\_by\_overflow\_mode) üzerindeki kısıtlamalar da değere sahiptir `any`. +Çoğu kısıtlama da bir ‘overflow_mode’ ayar, sınır aşıldığında ne yapılması gerektiği anlamına gelir. +İki değerden birini alabilir: `throw` veya `break`. Toplama (group_by_overflow_mode) üzerindeki kısıtlamalar da değere sahiptir `any`. `throw` – Throw an exception (default). @@ -24,7 +24,7 @@ ClickHouse, her satır için değil, veri bölümleri için kısıtlamaları den `any (only for group_by_overflow_mode)` – Continuing aggregation for the keys that got into the set, but don't add new keys to the set. -## max\_memory\_usage {#settings_max_memory_usage} +## max_memory_usage {#settings_max_memory_usage} Tek bir sunucuda bir sorgu çalıştırmak için kullanılacak en fazla RAM miktarı. @@ -41,47 +41,47 @@ Toplam işlevlerin durumları için bellek kullanımı tam olarak izlenmiyor `mi Bellek tüketimi de parametrelerle sınırlıdır `max_memory_usage_for_user` ve `max_memory_usage_for_all_queries`. -## max\_memory\_usage\_for\_user {#max-memory-usage-for-user} +## max_memory_usage_for_user {#max-memory-usage-for-user} Tek bir sunucuda bir kullanıcının sorguları çalıştırmak için kullanılacak en fazla RAM miktarı. Varsayılan değerler [Ayarlar.sa](https://github.com/ClickHouse/ClickHouse/blob/master/src/Core/Settings.h#L288). Varsayılan olarak, tutar sınırlı değildir (`max_memory_usage_for_user = 0`). -Ayrıca açıklamasına bakın [max\_memory\_usage](#settings_max_memory_usage). +Ayrıca açıklamasına bakın [max_memory_usage](#settings_max_memory_usage). -## max\_memory\_usage\_for\_all\_queries {#max-memory-usage-for-all-queries} +## max_memory_usage_for_all_queries {#max-memory-usage-for-all-queries} Tek bir sunucuda tüm sorguları çalıştırmak için kullanılacak en fazla RAM miktarı. Varsayılan değerler [Ayarlar.sa](https://github.com/ClickHouse/ClickHouse/blob/master/src/Core/Settings.h#L289). Varsayılan olarak, tutar sınırlı değildir (`max_memory_usage_for_all_queries = 0`). -Ayrıca açıklamasına bakın [max\_memory\_usage](#settings_max_memory_usage). +Ayrıca açıklamasına bakın [max_memory_usage](#settings_max_memory_usage). -## max\_rows\_to\_read {#max-rows-to-read} +## max_rows_to_read {#max-rows-to-read} Aşağıdaki kısıtlamalar her blokta kontrol edilebilir (her satır yerine). Yani, kısıtlamalar biraz kırılabilir. Birden çok iş parçacığında bir sorgu çalıştırırken, aşağıdaki kısıtlamalar her iş parçacığı için ayrı ayrı uygulanır. Bir sorgu çalıştırırken bir tablodan okunabilen satır sayısı. -## max\_bytes\_to\_read {#max-bytes-to-read} +## max_bytes_to_read {#max-bytes-to-read} Bir sorgu çalıştırırken bir tablodan okunabilen bayt sayısı (sıkıştırılmamış veri). -## read\_overflow\_mode {#read-overflow-mode} +## read_overflow_mode {#read-overflow-mode} Okunan veri hacmi sınırlardan birini aştığında ne yapmalı: ‘throw’ veya ‘break’. Varsayılan olarak, atın. -## max\_rows\_to\_group\_by {#settings-max-rows-to-group-by} +## max_rows_to_group_by {#settings-max-rows-to-group-by} Toplama alınan benzersiz anahtarların maksimum sayısı. Bu ayar, toplama sırasında bellek tüketimini sınırlamanızı sağlar. -## group\_by\_overflow\_mode {#group-by-overflow-mode} +## group_by_overflow_mode {#group-by-overflow-mode} Toplama için benzersiz anahtarların sayısı sınırı aştığında ne yapmalı: ‘throw’, ‘break’, veya ‘any’. Varsayılan olarak, atın. Kullanarak ‘any’ değer, GROUP BY'NİN bir yaklaşımını çalıştırmanızı sağlar. Bu yaklaşımın kalitesi, verilerin istatistiksel niteliğine bağlıdır. -## max\_bytes\_before\_external\_group\_by {#settings-max_bytes_before_external_group_by} +## max_bytes_before_external_group_by {#settings-max_bytes_before_external_group_by} Çalıştırmayı etkinleştirir veya devre dışı bırakır `GROUP BY` harici bellekte yan tümceleri. Görmek [Harici bellekte grupla](../../sql-reference/statements/select/group-by.md#select-group-by-in-external-memory). @@ -92,31 +92,31 @@ Olası değerler: Varsayılan değer: 0. -## max\_rows\_to\_sort {#max-rows-to-sort} +## max_rows_to_sort {#max-rows-to-sort} Sıralamadan önce en fazla satır sayısı. Bu, sıralama yaparken bellek tüketimini sınırlamanıza izin verir. -## max\_bytes\_to\_sort {#max-bytes-to-sort} +## max_bytes_to_sort {#max-bytes-to-sort} Sıralamadan önce en fazla bayt sayısı. -## sort\_overflow\_mode {#sort-overflow-mode} +## sort_overflow_mode {#sort-overflow-mode} Sıralamadan önce alınan satır sayısı sınırlardan birini aşarsa ne yapmalı: ‘throw’ veya ‘break’. Varsayılan olarak, atın. -## max\_result\_rows {#setting-max_result_rows} +## max_result_rows {#setting-max_result_rows} Sonuçtaki satır sayısını sınırlayın. Ayrıca, dağıtılmış bir sorgunun parçalarını çalıştırırken alt sorgular ve uzak sunucularda da kontrol edildi. -## max\_result\_bytes {#max-result-bytes} +## max_result_bytes {#max-result-bytes} Sonuçtaki bayt sayısını sınırlayın. Önceki ayar ile aynı. -## result\_overflow\_mode {#result-overflow-mode} +## result_overflow_mode {#result-overflow-mode} Sonucun hacmi sınırlardan birini aşarsa ne yapmalı: ‘throw’ veya ‘break’. Varsayılan olarak, atın. -Kullanım ‘break’ LİMİT kullanmaya benzer. `Break` yürütmeyi yalnızca blok düzeyinde keser. Bu, döndürülen satırların miktarının daha büyük olduğu anlamına gelir [max\_result\_rows](#setting-max_result_rows) birden çok [max\_block\_size](settings.md#setting-max_block_size) ve bağlıdır [max\_threads](settings.md#settings-max_threads). +Kullanım ‘break’ LİMİT kullanmaya benzer. `Break` yürütmeyi yalnızca blok düzeyinde keser. Bu, döndürülen satırların miktarının daha büyük olduğu anlamına gelir [max_result_rows](#setting-max_result_rows) birden çok [max_block_size](settings.md#setting-max_block_size) ve bağlıdır [max_threads](settings.md#settings-max_threads). Örnek: @@ -135,103 +135,103 @@ Sonuç: 6666 rows in set. ... ``` -## max\_execution\_time {#max-execution-time} +## max_execution_time {#max-execution-time} Saniye cinsinden maksimum sorgu yürütme süresi. Şu anda, sıralama aşamalarından biri için veya toplama işlevlerini birleştirirken ve sonlandırırken kontrol edilmez. -## timeout\_overflow\_mode {#timeout-overflow-mode} +## timeout_overflow_mode {#timeout-overflow-mode} -Sorgu daha uzun çalıştırılırsa ne yapmalı ‘max\_execution\_time’: ‘throw’ veya ‘break’. Varsayılan olarak, atın. +Sorgu daha uzun çalıştırılırsa ne yapmalı ‘max_execution_time’: ‘throw’ veya ‘break’. Varsayılan olarak, atın. -## min\_execution\_speed {#min-execution-speed} +## min_execution_speed {#min-execution-speed} -Saniyede satırlarda minimum yürütme hızı. Her veri bloğunda ne zaman kontrol edildi ‘timeout\_before\_checking\_execution\_speed’ doluyor. Yürütme hızı düşükse, bir istisna atılır. +Saniyede satırlarda minimum yürütme hızı. Her veri bloğunda ne zaman kontrol edildi ‘timeout_before_checking_execution_speed’ doluyor. Yürütme hızı düşükse, bir istisna atılır. -## min\_execution\_speed\_bytes {#min-execution-speed-bytes} +## min_execution_speed_bytes {#min-execution-speed-bytes} -Saniyede en az yürütme bayt sayısı. Her veri bloğunda ne zaman kontrol edildi ‘timeout\_before\_checking\_execution\_speed’ doluyor. Yürütme hızı düşükse, bir istisna atılır. +Saniyede en az yürütme bayt sayısı. Her veri bloğunda ne zaman kontrol edildi ‘timeout_before_checking_execution_speed’ doluyor. Yürütme hızı düşükse, bir istisna atılır. -## max\_execution\_speed {#max-execution-speed} +## max_execution_speed {#max-execution-speed} -Saniyede en fazla yürütme satırı sayısı. Her veri bloğunda ne zaman kontrol edildi ‘timeout\_before\_checking\_execution\_speed’ doluyor. Yürütme hızı yüksekse, yürütme hızı azaltılır. +Saniyede en fazla yürütme satırı sayısı. Her veri bloğunda ne zaman kontrol edildi ‘timeout_before_checking_execution_speed’ doluyor. Yürütme hızı yüksekse, yürütme hızı azaltılır. -## max\_execution\_speed\_bytes {#max-execution-speed-bytes} +## max_execution_speed_bytes {#max-execution-speed-bytes} -Saniyede en fazla yürütme bayt sayısı. Her veri bloğunda ne zaman kontrol edildi ‘timeout\_before\_checking\_execution\_speed’ doluyor. Yürütme hızı yüksekse, yürütme hızı azaltılır. +Saniyede en fazla yürütme bayt sayısı. Her veri bloğunda ne zaman kontrol edildi ‘timeout_before_checking_execution_speed’ doluyor. Yürütme hızı yüksekse, yürütme hızı azaltılır. -## timeout\_before\_checking\_execution\_speed {#timeout-before-checking-execution-speed} +## timeout_before_checking_execution_speed {#timeout-before-checking-execution-speed} -Yürütme hızının çok yavaş olmadığını kontrol eder (en az ‘min\_execution\_speed’), saniye içinde belirtilen süre dolduktan sonra. +Yürütme hızının çok yavaş olmadığını kontrol eder (en az ‘min_execution_speed’), saniye içinde belirtilen süre dolduktan sonra. -## max\_columns\_to\_read {#max-columns-to-read} +## max_columns_to_read {#max-columns-to-read} Tek bir sorguda bir tablodan okunabilen sütun sayısı. Bir sorgu daha fazla sayıda sütun okuma gerektiriyorsa, bir özel durum atar. -## max\_temporary\_columns {#max-temporary-columns} +## max_temporary_columns {#max-temporary-columns} Sabit sütunlar da dahil olmak üzere bir sorgu çalıştırırken aynı anda RAM'de tutulması gereken geçici sütun sayısı. Bundan daha fazla geçici sütun varsa, bir istisna atar. -## max\_temporary\_non\_const\_columns {#max-temporary-non-const-columns} +## max_temporary_non_const_columns {#max-temporary-non-const-columns} -Aynı şey ‘max\_temporary\_columns’, ancak sabit sütunları saymadan. +Aynı şey ‘max_temporary_columns’, ancak sabit sütunları saymadan. Bir sorgu çalıştırırken sabit sütunların oldukça sık oluşturulduğunu, ancak yaklaşık sıfır bilgi işlem kaynağı gerektirdiğini unutmayın. -## max\_subquery\_depth {#max-subquery-depth} +## max_subquery_depth {#max-subquery-depth} Alt sorguların maksimum yuvalama derinliği. Alt sorgular daha derinse, bir istisna atılır. Varsayılan olarak, 100. -## max\_pipeline\_depth {#max-pipeline-depth} +## max_pipeline_depth {#max-pipeline-depth} Maksimum boru hattı derinliği. Sorgu işleme sırasında her veri bloğunun geçtiği dönüşümlerin sayısına karşılık gelir. Tek bir sunucunun sınırları içinde sayılır. Boru hattı derinliği büyükse, bir istisna atılır. Varsayılan olarak, 1000. -## max\_ast\_depth {#max-ast-depth} +## max_ast_depth {#max-ast-depth} Sorgu sözdizimsel ağacının en fazla yuvalama derinliği. Aşılırsa, bir istisna atılır. Şu anda, ayrıştırma sırasında değil, yalnızca sorguyu ayrıştırdıktan sonra kontrol edilir. Yani, ayrıştırma sırasında çok derin bir sözdizimsel ağaç oluşturulabilir, ancak sorgu başarısız olur. Varsayılan olarak, 1000. -## max\_ast\_elements {#max-ast-elements} +## max_ast_elements {#max-ast-elements} Sorgu sözdizimsel ağacındaki en fazla öğe sayısı. Aşılırsa, bir istisna atılır. Önceki ayarla aynı şekilde, yalnızca sorguyu ayrıştırdıktan sonra kontrol edilir. Varsayılan olarak, 50.000. -## max\_rows\_in\_set {#max-rows-in-set} +## max_rows_in_set {#max-rows-in-set} Bir alt sorgudan oluşturulan In yan tümcesinde bir veri kümesi için satır sayısı. -## max\_bytes\_in\_set {#max-bytes-in-set} +## max_bytes_in_set {#max-bytes-in-set} Bir alt sorgudan oluşturulan In yan tümcesinde bir set tarafından kullanılan en fazla bayt sayısı (sıkıştırılmamış veri). -## set\_overflow\_mode {#set-overflow-mode} +## set_overflow_mode {#set-overflow-mode} Veri miktarı sınırlardan birini aştığında ne yapmalı: ‘throw’ veya ‘break’. Varsayılan olarak, atın. -## max\_rows\_ın\_distinct {#max-rows-in-distinct} +## max_rows_ın_distinct {#max-rows-in-distinct} DISTINCT kullanırken en fazla sayıda farklı satır. -## max\_bytes\_ın\_distinct {#max-bytes-in-distinct} +## max_bytes_ın_distinct {#max-bytes-in-distinct} DISTINCT kullanırken bir karma tablo tarafından kullanılan bayt sayısı. -## distinct\_overflow\_mode {#distinct-overflow-mode} +## distinct_overflow_mode {#distinct-overflow-mode} Veri miktarı sınırlardan birini aştığında ne yapmalı: ‘throw’ veya ‘break’. Varsayılan olarak, atın. -## max\_rows\_to\_transfer {#max-rows-to-transfer} +## max_rows_to_transfer {#max-rows-to-transfer} Uzak bir sunucuya geçirilen veya GLOBAL In kullanırken geçici bir tabloya kaydedilen satır sayısı. -## max\_bytes\_to\_transfer {#max-bytes-to-transfer} +## max_bytes_to_transfer {#max-bytes-to-transfer} Uzak bir sunucuya geçirilen veya GLOBAL In kullanırken geçici bir tabloya kaydedilen bayt sayısı (sıkıştırılmamış veri). -## transfer\_overflow\_mode {#transfer-overflow-mode} +## transfer_overflow_mode {#transfer-overflow-mode} Veri miktarı sınırlardan birini aştığında ne yapmalı: ‘throw’ veya ‘break’. Varsayılan olarak, atın. -## max\_rows\_in\_join {#settings-max_rows_in_join} +## max_rows_in_join {#settings-max_rows_in_join} Tabloları birleştirirken kullanılan karma tablodaki satır sayısını sınırlar. @@ -239,7 +239,7 @@ Bu ayarlar aşağıdakiler için geçerlidir [SELECT … JOIN](../../sql-referen Bir sorgu birden çok birleşim içeriyorsa, ClickHouse her Ara sonuç için bu ayarı denetler. -Limit ulaşıldığında ClickHouse farklı eylemlerle devam edebilirsiniz. Kullan... [join\_overflow\_mode](#settings-join_overflow_mode) eylemi seçmek için ayarlama. +Limit ulaşıldığında ClickHouse farklı eylemlerle devam edebilirsiniz. Kullan... [join_overflow_mode](#settings-join_overflow_mode) eylemi seçmek için ayarlama. Olası değerler: @@ -248,7 +248,7 @@ Olası değerler: Varsayılan değer: 0. -## max\_bytes\_in\_join {#settings-max_bytes_in_join} +## max_bytes_in_join {#settings-max_bytes_in_join} Tabloları birleştirirken kullanılan karma tablonun bayt cinsinden boyutunu sınırlar. @@ -256,7 +256,7 @@ Bu ayarlar aşağıdakiler için geçerlidir [SELECT … JOIN](../../sql-referen Sorgu birleşimler içeriyorsa, ClickHouse her Ara sonuç için bu ayarı denetler. -Limit ulaşıldığında ClickHouse farklı eylemlerle devam edebilirsiniz. Kullanmak [join\_overflow\_mode](#settings-join_overflow_mode) eylemi seçmek için ayarlar. +Limit ulaşıldığında ClickHouse farklı eylemlerle devam edebilirsiniz. Kullanmak [join_overflow_mode](#settings-join_overflow_mode) eylemi seçmek için ayarlar. Olası değerler: @@ -265,12 +265,12 @@ Olası değerler: Varsayılan değer: 0. -## join\_overflow\_mode {#settings-join_overflow_mode} +## join_overflow_mode {#settings-join_overflow_mode} Tanımlar katılın aşağıdaki sınırlar her zaman eylem ClickHouse gerçekleştirdiği ulaştı: -- [max\_bytes\_in\_join](#settings-max_bytes_in_join) -- [max\_rows\_in\_join](#settings-max_rows_in_join) +- [max_bytes_in_join](#settings-max_bytes_in_join) +- [max_rows_in_join](#settings-max_rows_in_join) Olası değerler: @@ -284,7 +284,7 @@ Varsayılan değer: `THROW`. - [Jo](../../sql-reference/statements/select/join.md#select-join) - [Jo tablein table engine](../../engines/table-engines/special/join.md) -## max\_partitions\_per\_ınsert\_block {#max-partitions-per-insert-block} +## max_partitions_per_ınsert_block {#max-partitions-per-insert-block} Eklenen tek bir bloktaki en fazla bölüm sayısını sınırlar. @@ -297,6 +297,6 @@ Varsayılan değer: 100. Veri eklerken, ClickHouse eklenen bloktaki bölüm sayısını hesaplar. Bölüm sayısı fazla ise `max_partitions_per_insert_block`, ClickHouse aşağıdaki metinle bir özel durum atar: -> “Too many partitions for single INSERT block (more than” + toString (max\_parts) + “). The limit is controlled by ‘max\_partitions\_per\_insert\_block’ setting. A large number of partitions is a common misconception. It will lead to severe negative performance impact, including slow server startup, slow INSERT queries and slow SELECT queries. Recommended total number of partitions for a table is under 1000..10000. Please note, that partitioning is not intended to speed up SELECT queries (ORDER BY key is sufficient to make range queries fast). Partitions are intended for data manipulation (DROP PARTITION, etc).” +> “Too many partitions for single INSERT block (more than” + toString (max_parts) + “). The limit is controlled by ‘max_partitions_per_insert_block’ setting. A large number of partitions is a common misconception. It will lead to severe negative performance impact, including slow server startup, slow INSERT queries and slow SELECT queries. Recommended total number of partitions for a table is under 1000..10000. Please note, that partitioning is not intended to speed up SELECT queries (ORDER BY key is sufficient to make range queries fast). Partitions are intended for data manipulation (DROP PARTITION, etc).” [Orijinal makale](https://clickhouse.tech/docs/en/operations/settings/query_complexity/) diff --git a/docs/tr/operations/settings/settings-users.md b/docs/tr/operations/settings/settings-users.md index eead71c0a77..9d792663a42 100644 --- a/docs/tr/operations/settings/settings-users.md +++ b/docs/tr/operations/settings/settings-users.md @@ -75,7 +75,7 @@ Bu yapı `users` bölme: Sonucun ilk satırı şifredir. İkinci satır karşılık gelen çift SHA1 karmasıdır. -### access\_management {#access_management-user-setting} +### access_management {#access_management-user-setting} Bu ayar, SQL-driven kullanarak devre dışı bırakır sağlar [erişim kontrolü ve hesap yönetimi](../access-rights.md#access-control) kullanıcı için. @@ -86,7 +86,7 @@ Olası değerler: Varsayılan değer: 0. -### kullanıcı\_adı / ağlar {#user-namenetworks} +### kullanıcı_adı / ağlar {#user-namenetworks} Kullanıcının ClickHouse sunucusuna bağlanabileceği ağların listesi. @@ -128,18 +128,18 @@ Erişimi yalnızca localhost'tan açmak için şunları belirtin: 127.0.0.1 ``` -### kullanıcı\_adı / profil {#user-nameprofile} +### kullanıcı_adı / profil {#user-nameprofile} Kullanıcı için bir ayarlar profili atayabilirsiniz. Ayarlar profilleri ayrı bir bölümde yapılandırılır `users.xml` Dosya. Daha fazla bilgi için, bkz. [Ayarların profilleri](settings-profiles.md). -### user\_name / kota {#user-namequota} +### user_name / kota {#user-namequota} Kotalar, belirli bir süre boyunca kaynak kullanımını izlemenize veya sınırlamanıza izin verir. Kotalar yapılandırılır `quotas` bu bölüm `users.xml` yapılandırma dosyası. Kullanıcı için ayarlanmış bir kotalar atayabilirsiniz. Kotalar yapılandırmasının ayrıntılı bir açıklaması için bkz. [Kotalar](../quotas.md#quotas). -### user\_name / veritabanları {#user-namedatabases} +### user_name / veritabanları {#user-namedatabases} Bu bölümde, ClickHouse tarafından döndürülen satırları sınırlayabilirsiniz `SELECT` geçerli kullanıcı tarafından yapılan sorgular, böylece temel satır düzeyinde güvenlik uygular. diff --git a/docs/tr/operations/settings/settings.md b/docs/tr/operations/settings/settings.md index f942da86c10..c9c7e7b898f 100644 --- a/docs/tr/operations/settings/settings.md +++ b/docs/tr/operations/settings/settings.md @@ -5,7 +5,7 @@ machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd # Ayarlar {#settings} -## distributed\_product\_mode {#distributed-product-mode} +## distributed_product_mode {#distributed-product-mode} Davranışını değiştirir [dağıtılmış alt sorgular](../../sql-reference/operators/in.md). @@ -25,7 +25,7 @@ Olası değerler: - `global` — Replaces the `IN`/`JOIN` ile sorgu `GLOBAL IN`/`GLOBAL JOIN.` - `allow` — Allows the use of these types of subqueries. -## enable\_optimize\_predicate\_expression {#enable-optimize-predicate-expression} +## enable_optimize_predicate_expression {#enable-optimize-predicate-expression} Yüklemi pushdown açar `SELECT` sorgular. @@ -49,7 +49,7 @@ Eğer `enable_optimize_predicate_expression = 1`, daha sonra bu sorguların yür Eğer `enable_optimize_predicate_expression = 0`, daha sonra ikinci sorgunun yürütme süresi çok daha uzundur, çünkü `WHERE` yan tümcesi alt sorgu tamamlandıktan sonra tüm veriler için geçerlidir. -## fallback\_to\_stale\_replicas\_for\_distributed\_queries {#settings-fallback_to_stale_replicas_for_distributed_queries} +## fallback_to_stale_replicas_for_distributed_queries {#settings-fallback_to_stale_replicas_for_distributed_queries} Güncelleştirilmiş veriler mevcut değilse, bir sorgu için güncel olmayan bir yineleme zorlar. Görmek [Çoğalma](../../engines/table-engines/mergetree-family/replication.md). @@ -59,7 +59,7 @@ Yaparken kullanılır `SELECT` çoğaltılmış tablolara işaret eden dağıtı Varsayılan olarak, 1 (etkin). -## force\_index\_by\_date {#settings-force_index_by_date} +## force_index_by_date {#settings-force_index_by_date} Dizin tarihe göre kullanılamıyorsa, sorgu yürütülmesini devre dışı bırakır. @@ -67,7 +67,7 @@ MergeTree ailesindeki tablolarla çalışır. Eğer `force_index_by_date=1`, ClickHouse sorgunun veri aralıklarını kısıtlamak için kullanılabilecek bir tarih anahtarı koşulu olup olmadığını denetler. Uygun bir koşul yoksa, bir istisna atar. Ancak, koşul okumak için veri miktarını azaltır olup olmadığını denetlemez. Örneğin, durum `Date != ' 2000-01-01 '` tablodaki tüm verilerle eşleştiğinde bile kabul edilebilir (yani, sorguyu çalıştırmak tam bir tarama gerektirir). MergeTree tablolarındaki veri aralıkları hakkında daha fazla bilgi için bkz. [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). -## force\_primary\_key {#force-primary-key} +## force_primary_key {#force-primary-key} Birincil anahtar tarafından dizin oluşturma mümkün değilse, sorgu yürütülmesini devre dışı bırakır. @@ -75,17 +75,17 @@ MergeTree ailesindeki tablolarla çalışır. Eğer `force_primary_key=1`, ClickHouse, sorgunun veri aralıklarını kısıtlamak için kullanılabilecek bir birincil anahtar koşulu olup olmadığını denetler. Uygun bir koşul yoksa, bir istisna atar. Ancak, koşul okumak için veri miktarını azaltır olup olmadığını denetlemez. MergeTree tablolarındaki veri aralıkları hakkında daha fazla bilgi için bkz. [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). -## format\_schema {#format-schema} +## format_schema {#format-schema} Bu parametre, aşağıdaki gibi bir şema tanımı gerektiren biçimler kullanırken kullanışlıdır [Cap'n Proto](https://capnproto.org/) veya [Protobuf](https://developers.google.com/protocol-buffers/). Değer biçime bağlıdır. -## fsync\_metadata {#fsync-metadata} +## fsync_metadata {#fsync-metadata} Etkinleştirir veya devre dışı bırakır [fsync](http://pubs.opengroup.org/onlinepubs/9699919799/functions/fsync.html) yazarken `.sql` eğe. Varsayılan olarak etkin. Sunucu, sürekli olarak oluşturulan ve yok edilen milyonlarca küçük tabloya sahipse, onu devre dışı bırakmak mantıklıdır. -## enable\_http\_compression {#settings-enable_http_compression} +## enable_http_compression {#settings-enable_http_compression} Bir HTTP isteğine yanıt olarak veri sıkıştırmasını etkinleştirir veya devre dışı bırakır. @@ -98,15 +98,15 @@ Olası değerler: Varsayılan değer: 0. -## http\_zlib\_compression\_level {#settings-http_zlib_compression_level} +## http_zlib_compression_level {#settings-http_zlib_compression_level} -Eğer bir HTTP isteğine yanıt veri sıkıştırma düzeyini ayarlar [enable\_http\_compression = 1](#settings-enable_http_compression). +Eğer bir HTTP isteğine yanıt veri sıkıştırma düzeyini ayarlar [enable_http_compression = 1](#settings-enable_http_compression). Olası değerler: 1'den 9'a kadar olan sayılar. Varsayılan değer: 3. -## http\_native\_compression\_disable\_checksumming\_on\_decompress {#settings-http_native_compression_disable_checksumming_on_decompress} +## http_native_compression_disable_checksumming_on_decompress {#settings-http_native_compression_disable_checksumming_on_decompress} İstemciden HTTP POST verilerini açarken sağlama toplamı doğrulamasını etkinleştirir veya devre dışı bırakır. Sadece ClickHouse yerel sıkıştırma formatı için kullanılır (ile kullanılmaz `gzip` veya `deflate`). @@ -119,7 +119,7 @@ Olası değerler: Varsayılan değer: 0. -## send\_progress\_in\_http\_headers {#settings-send_progress_in_http_headers} +## send_progress_in_http_headers {#settings-send_progress_in_http_headers} Etkinleştirir veya devre dışı bırakır `X-ClickHouse-Progress` HTTP yanıt başlıkları `clickhouse-server` yanıtlar. @@ -132,7 +132,7 @@ Olası değerler: Varsayılan değer: 0. -## max\_http\_get\_redirects {#setting-max_http_get_redirects} +## max_http_get_redirects {#setting-max_http_get_redirects} Maksimum http get yönlendirme atlama sayısını sınırlar [URL](../../engines/table-engines/special/url.md)- motor masaları. Ayarı tablolar iki tür tarafından oluşturulan bu geçerlidir: [CREATE TABLE](../../sql-reference/statements/create.md#create-table-query) sorgu ve [url](../../sql-reference/table-functions/url.md) tablo işlevi. @@ -143,7 +143,7 @@ Olası değerler: Varsayılan değer: 0. -## ınput\_format\_allow\_errors\_num {#settings-input_format_allow_errors_num} +## ınput_format_allow_errors_num {#settings-input_format_allow_errors_num} Metin biçimlerinden (CSV, TSV, vb.) okurken kabul edilebilir hataların maksimum sayısını ayarlar.). @@ -155,7 +155,7 @@ Satırları okurken bir hata oluştu, ancak hata sayacı hala daha az `input_for Eğer her ikisi de `input_format_allow_errors_num` ve `input_format_allow_errors_ratio` aşıldı, ClickHouse bir istisna atar. -## ınput\_format\_allow\_errors\_ratio {#settings-input_format_allow_errors_ratio} +## ınput_format_allow_errors_ratio {#settings-input_format_allow_errors_ratio} Metin biçimlerinden (CSV, TSV, vb.) okurken izin verilen maksimum hata yüzdesini ayarlar.). Hataların yüzdesi 0 ile 1 arasında kayan nokta sayısı olarak ayarlanır. @@ -168,7 +168,7 @@ Satırları okurken bir hata oluştu, ancak hata sayacı hala daha az `input_for Eğer her ikisi de `input_format_allow_errors_num` ve `input_format_allow_errors_ratio` aşıldı, ClickHouse bir istisna atar. -## ınput\_format\_values\_interpret\_expressions {#settings-input_format_values_interpret_expressions} +## ınput_format_values_interpret_expressions {#settings-input_format_values_interpret_expressions} Hızlı akış ayrıştırıcısı verileri ayrıştıramazsa, tam SQL ayrıştırıcısını etkinleştirir veya devre dışı bırakır. Bu ayar yalnızca için kullanılır [Değerler](../../interfaces/formats.md#data-format-values) veri ekleme sırasında biçimlendirin. Sözdizimi ayrıştırma hakkında daha fazla bilgi için bkz: [Sözdizimi](../../sql-reference/syntax.md) bölme. @@ -218,7 +218,7 @@ INSERT INTO datetime_t SELECT now() Ok. ``` -## ınput\_format\_values\_deduce\_templates\_of\_expressions {#settings-input_format_values_deduce_templates_of_expressions} +## ınput_format_values_deduce_templates_of_expressions {#settings-input_format_values_deduce_templates_of_expressions} SQL deyimleri için şablon kesintisini etkinleştirir veya devre dışı bırakır [Değerler](../../interfaces/formats.md#data-format-values) biçimli. Bu ayrıştırma ve ifadeleri yorumlama sağlar `Values` ardışık satırlardaki ifadeler aynı yapıya sahipse çok daha hızlı. ClickHouse, bir ifadenin şablonunu çıkarmaya, bu şablonu kullanarak aşağıdaki satırları ayrıştırmaya ve ifadeyi başarılı bir şekilde ayrıştırılmış satırların bir yığınında değerlendirmeye çalışır. @@ -239,7 +239,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( - Eğer `input_format_values_interpret_expressions=0` ve `format_values_deduce_templates_of_expressions=1`, birinci, ikinci ve üçüncü satırlardaki ifadeler şablon kullanılarak ayrıştırılır `lower(String)` ve birlikte yorumlanır, ileri satırdaki ifade başka bir şablonla ayrıştırılır (`upper(String)`). - Eğer `input_format_values_interpret_expressions=1` ve `format_values_deduce_templates_of_expressions=1`, önceki durumda olduğu gibi aynı, ama aynı zamanda şablon anlamak mümkün değilse ayrı ayrı ifadeleri yorumlama geri dönüş sağlar. -## ınput\_format\_values\_accurate\_types\_of\_literals {#settings-input-format-values-accurate-types-of-literals} +## ınput_format_values_accurate_types_of_literals {#settings-input-format-values-accurate-types-of-literals} Bu ayar yalnızca şu durumlarda kullanılır `input_format_values_deduce_templates_of_expressions = 1`. Bu, bazı sütunların ifadelerinin aynı yapıya sahip olması, ancak farklı türlerde sayısal değişmezler içermesi olabilir, örneğin @@ -261,7 +261,7 @@ Olası değerler: Varsayılan değer: 1. -## ınput\_format\_defaults\_for\_omitted\_fields {#session_settings-input_format_defaults_for_omitted_fields} +## ınput_format_defaults_for_omitted_fields {#session_settings-input_format_defaults_for_omitted_fields} Yaparken `INSERT` sorgular, atlanmış giriş sütun değerlerini ilgili sütunların varsayılan değerleriyle değiştirin. Bu seçenek yalnızca aşağıdakiler için geçerlidir [JSONEachRow](../../interfaces/formats.md#jsoneachrow), [CSV](../../interfaces/formats.md#csv) ve [TabSeparated](../../interfaces/formats.md#tabseparated) biçimliler. @@ -275,17 +275,17 @@ Olası değerler: Varsayılan değer: 1. -## ınput\_format\_tsv\_empty\_as\_default {#settings-input-format-tsv-empty-as-default} +## ınput_format_tsv_empty_as_default {#settings-input-format-tsv-empty-as-default} Etkinleştirildiğinde, TSV'DEKİ boş giriş alanlarını varsayılan değerlerle değiştirin. Karmaşık varsayılan ifadeler için `input_format_defaults_for_omitted_fields` de etkin olmalıdır. Varsayılan olarak devre dışı. -## ınput\_format\_null\_as\_default {#settings-input-format-null-as-default} +## ınput_format_null_as_default {#settings-input-format-null-as-default} Giriş verileri içeriyorsa, varsayılan değerleri kullanarak etkinleştirir veya devre dışı bırakır `NULL`, ancak ilgili sütunun veri türü değil `Nullable(T)` (Metin Giriş biçimleri için). -## ınput\_format\_skip\_unknown\_fields {#settings-input-format-skip-unknown-fields} +## ınput_format_skip_unknown_fields {#settings-input-format-skip-unknown-fields} Etkinleştirir veya ek veri ekleme atlama devre dışı bırakır. @@ -305,7 +305,7 @@ Olası değerler: Varsayılan değer: 0. -## ınput\_format\_ımport\_nested\_json {#settings-input_format_import_nested_json} +## ınput_format_ımport_nested_json {#settings-input_format_import_nested_json} Json verilerinin iç içe nesnelerle eklenmesini etkinleştirir veya devre dışı bırakır. @@ -324,7 +324,7 @@ Ayrıca bakınız: - [İç içe yapıların kullanımı](../../interfaces/formats.md#jsoneachrow-nested) ile... `JSONEachRow` biçimli. -## ınput\_format\_with\_names\_use\_header {#settings-input-format-with-names-use-header} +## ınput_format_with_names_use_header {#settings-input-format-with-names-use-header} Veri eklerken sütun sırasını denetlemeyi etkinleştirir veya devre dışı bırakır. @@ -342,7 +342,7 @@ Olası değerler: Varsayılan değer: 1. -## date\_time\_input\_format {#settings-date_time_input_format} +## date_time_input_format {#settings-date_time_input_format} Tarih ve saat metin gösterimi bir ayrıştırıcı seçme sağlar. @@ -365,7 +365,7 @@ Ayrıca bakınız: - [DateTime veri türü.](../../sql-reference/data-types/datetime.md) - [Tarihler ve saatler ile çalışmak için fonksiyonlar.](../../sql-reference/functions/date-time-functions.md) -## join\_default\_strictness {#settings-join_default_strictness} +## join_default_strictness {#settings-join_default_strictness} Ayarlar varsayılan strictness için [Maddeleri KATILIN ](../../sql-reference/statements/select/join.md#select-join). @@ -378,7 +378,7 @@ Olası değerler: Varsayılan değer: `ALL`. -## join\_any\_take\_last\_row {#settings-join_any_take_last_row} +## join_any_take_last_row {#settings-join_any_take_last_row} İle birleştirme işlemlerinin davranışını değiştirir `ANY` katılık. @@ -396,9 +396,9 @@ Ayrıca bakınız: - [Jo](../../sql-reference/statements/select/join.md#select-join) - [Jo tablein table engine](../../engines/table-engines/special/join.md) -- [join\_default\_strictness](#settings-join_default_strictness) +- [join_default_strictness](#settings-join_default_strictness) -## join\_use\_nulls {#join_use_nulls} +## join_use_nulls {#join_use_nulls} Türünü ayarlar [JOIN](../../sql-reference/statements/select/join.md) davranış. Tabloları birleştirirken boş hücreler görünebilir. ClickHouse bu ayara göre onları farklı şekilde doldurur. @@ -409,7 +409,7 @@ Olası değerler: Varsayılan değer: 0. -## max\_block\_size {#setting-max_block_size} +## max_block_size {#setting-max_block_size} Clickhouse'da, veriler bloklarla (sütun parçaları kümeleri) işlenir. Tek bir blok için dahili işlem döngüleri yeterince verimlidir, ancak her blokta gözle görülür harcamalar vardır. Bu `max_block_size` ayar, blokun boyutunun (satır sayımında) tablolardan yükleneceği bir öneridir. Blok boyutu çok küçük olmamalı, böylece her bloktaki harcamalar hala fark edilebilir, ancak çok büyük olmamalı, böylece ilk blok hızla işlendikten sonra tamamlanan limitli sorgu çok büyük olmamalıdır. Amaç, birden çok iş parçacığında çok sayıda sütun ayıklarken çok fazla bellek tüketmekten kaçınmak ve en azından bazı önbellek konumlarını korumaktır. @@ -417,13 +417,13 @@ Varsayılan değer: 65,536. Blok boyutu `max_block_size` her zaman tablodan yüklenmez. Daha az verinin alınması gerektiği açıksa, daha küçük bir blok işlenir. -## preferred\_block\_size\_bytes {#preferred-block-size-bytes} +## preferred_block_size_bytes {#preferred-block-size-bytes} Olarak aynı amaç için kullanılır `max_block_size`, ancak önerilen blok boyutunu bayt cinsinden, bloktaki satır sayısına uyarlayarak ayarlar. Ancak, blok boyutu daha fazla olamaz `max_block_size` satırlar. Varsayılan olarak: 1.000.000. Sadece MergeTree motorlarından okurken çalışır. -## merge\_tree\_mın\_rows\_for\_concurrent\_read {#setting-merge-tree-min-rows-for-concurrent-read} +## merge_tree_mın_rows_for_concurrent_read {#setting-merge-tree-min-rows-for-concurrent-read} Bir dosyadan okunacak satır sayısı ise [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) tablo aşıyor `merge_tree_min_rows_for_concurrent_read` daha sonra ClickHouse, bu dosyadan birkaç iş parçacığı üzerinde eşzamanlı bir okuma gerçekleştirmeye çalışır. @@ -433,7 +433,7 @@ Olası değerler: Varsayılan değer: 163840. -## merge\_tree\_min\_bytes\_for\_concurrent\_read {#setting-merge-tree-min-bytes-for-concurrent-read} +## merge_tree_min_bytes_for_concurrent_read {#setting-merge-tree-min-bytes-for-concurrent-read} Eğer bir dosyadan okunacak bayt sayısı [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md)- motor tablosu `merge_tree_min_bytes_for_concurrent_read`, daha sonra ClickHouse, bu dosyadan aynı anda birkaç iş parçacığında okumaya çalışır. @@ -443,7 +443,7 @@ Olası değer: Varsayılan değer: 251658240. -## merge\_tree\_mın\_rows\_for\_seek {#setting-merge-tree-min-rows-for-seek} +## merge_tree_mın_rows_for_seek {#setting-merge-tree-min-rows-for-seek} Bir dosyada okunacak iki veri bloğu arasındaki mesafe daha az ise `merge_tree_min_rows_for_seek` satırlar, daha sonra ClickHouse dosyayı aramaz, ancak verileri sırayla okur. @@ -453,7 +453,7 @@ Olası değerler: Varsayılan değer: 0. -## merge\_tree\_min\_bytes\_for\_seek {#setting-merge-tree-min-bytes-for-seek} +## merge_tree_min_bytes_for_seek {#setting-merge-tree-min-bytes-for-seek} Bir dosyada okunacak iki veri bloğu arasındaki mesafe daha az ise `merge_tree_min_bytes_for_seek` bayt, daha sonra ClickHouse sırayla böylece ekstra arama kaçınarak, her iki blok içeren bir dosya aralığını okur. @@ -463,7 +463,7 @@ Olası değerler: Varsayılan değer: 0. -## merge\_tree\_coarse\_index\_granularity {#setting-merge-tree-coarse-index-granularity} +## merge_tree_coarse_index_granularity {#setting-merge-tree-coarse-index-granularity} Veri ararken, ClickHouse dizin dosyasındaki veri işaretlerini denetler. ClickHouse gerekli tuşların bazı aralıklarda olduğunu bulursa, bu aralığı `merge_tree_coarse_index_granularity` subranges ve gerekli anahtarları orada yinelemeli olarak arar. @@ -473,11 +473,11 @@ Olası değerler: Varsayılan değer: 8. -## merge\_tree\_max\_rows\_to\_use\_cache {#setting-merge-tree-max-rows-to-use-cache} +## merge_tree_max_rows_to_use_cache {#setting-merge-tree-max-rows-to-use-cache} ClickHouse daha fazla okumak gerekiyorsa `merge_tree_max_rows_to_use_cache` bir sorgudaki satırlar, sıkıştırılmamış blokların önbelleğini kullanmaz. -Sıkıştırılmamış blokların önbelleği, sorgular için ayıklanan verileri depolar. ClickHouse, tekrarlanan küçük sorgulara verilen yanıtları hızlandırmak için bu önbelleği kullanır. Bu ayar, önbelleğin büyük miktarda veri okuyan sorgularla çöpe atmasını önler. Bu [uncompressed\_cache\_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) sunucu ayarı, sıkıştırılmamış blokların önbelleğinin boyutunu tanımlar. +Sıkıştırılmamış blokların önbelleği, sorgular için ayıklanan verileri depolar. ClickHouse, tekrarlanan küçük sorgulara verilen yanıtları hızlandırmak için bu önbelleği kullanır. Bu ayar, önbelleğin büyük miktarda veri okuyan sorgularla çöpe atmasını önler. Bu [uncompressed_cache_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) sunucu ayarı, sıkıştırılmamış blokların önbelleğinin boyutunu tanımlar. Olası değerler: @@ -485,11 +485,11 @@ Olası değerler: Default value: 128 ✕ 8192. -## merge\_tree\_max\_bytes\_to\_use\_cache {#setting-merge-tree-max-bytes-to-use-cache} +## merge_tree_max_bytes_to_use_cache {#setting-merge-tree-max-bytes-to-use-cache} ClickHouse daha fazla okumak gerekiyorsa `merge_tree_max_bytes_to_use_cache` bir sorguda bayt, sıkıştırılmamış blokların önbelleğini kullanmaz. -Sıkıştırılmamış blokların önbelleği, sorgular için ayıklanan verileri depolar. ClickHouse, tekrarlanan küçük sorgulara verilen yanıtları hızlandırmak için bu önbelleği kullanır. Bu ayar, önbelleğin büyük miktarda veri okuyan sorgularla çöpe atmasını önler. Bu [uncompressed\_cache\_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) sunucu ayarı, sıkıştırılmamış blokların önbelleğinin boyutunu tanımlar. +Sıkıştırılmamış blokların önbelleği, sorgular için ayıklanan verileri depolar. ClickHouse, tekrarlanan küçük sorgulara verilen yanıtları hızlandırmak için bu önbelleği kullanır. Bu ayar, önbelleğin büyük miktarda veri okuyan sorgularla çöpe atmasını önler. Bu [uncompressed_cache_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) sunucu ayarı, sıkıştırılmamış blokların önbelleğinin boyutunu tanımlar. Olası değer: @@ -497,7 +497,7 @@ Olası değer: Varsayılan değer: 2013265920. -## min\_bytes\_to\_use\_direct\_io {#settings-min-bytes-to-use-direct-io} +## min_bytes_to_use_direct_io {#settings-min-bytes-to-use-direct-io} Depolama diskine Doğrudan G/Ç erişimi kullanmak için gereken minimum veri hacmi. @@ -510,11 +510,11 @@ Olası değerler: Varsayılan değer: 0. -## log\_queries {#settings-log-queries} +## log_queries {#settings-log-queries} Sorgu günlüğü ayarlama. -Bu kurulum ile Clickhouse'a gönderilen sorgular, [query\_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query-log) sunucu yapılandırma parametresi. +Bu kurulum ile Clickhouse'a gönderilen sorgular, [query_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query-log) sunucu yapılandırma parametresi. Örnek: @@ -522,7 +522,7 @@ Bu kurulum ile Clickhouse'a gönderilen sorgular, [query\_log](../server-configu log_queries=1 ``` -## log\_queries\_min\_type {#settings-log-queries-min-type} +## log_queries_min_type {#settings-log-queries-min-type} `query_log` giriş yapmak için en az tür. @@ -540,11 +540,11 @@ Entiries gider hangi sınırlamak için kullanılabilir `query_log`, sadece hata log_queries_min_type='EXCEPTION_WHILE_PROCESSING' ``` -## log\_query\_threads {#settings-log-query-threads} +## log_query_threads {#settings-log-query-threads} Sorgu iş parçacığı günlüğü ayarlama. -Bu kurulum ile ClickHouse tarafından çalıştırılan sorguların konuları, [query\_thread\_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) sunucu yapılandırma parametresi. +Bu kurulum ile ClickHouse tarafından çalıştırılan sorguların konuları, [query_thread_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) sunucu yapılandırma parametresi. Örnek: @@ -552,19 +552,19 @@ Bu kurulum ile ClickHouse tarafından çalıştırılan sorguların konuları, [ log_query_threads=1 ``` -## max\_ınsert\_block\_size {#settings-max_insert_block_size} +## max_ınsert_block_size {#settings-max_insert_block_size} Bir tabloya eklemek için oluşturulacak blokların boyutu. Bu ayar yalnızca sunucu blokları oluşturduğu durumlarda geçerlidir. Örneğin, HTTP arabirimi üzerinden bir ekleme için sunucu veri biçimini ayrıştırır ve belirtilen boyuttaki blokları oluşturur. -Ancak, clickhouse-client kullanırken, istemci verileri kendisi ayrıştırır ve ‘max\_insert\_block\_size’ sunucudaki ayar, eklenen blokların boyutunu etkilemez. +Ancak, clickhouse-client kullanırken, istemci verileri kendisi ayrıştırır ve ‘max_insert_block_size’ sunucudaki ayar, eklenen blokların boyutunu etkilemez. Veri SELECT sonra oluşturulan aynı blokları kullanarak eklendiğinden, INSERT SELECT kullanırken ayarı da bir amacı yoktur. Varsayılan değer: 1.048,576. Varsayılan biraz daha fazla `max_block_size`. Bunun nedeni, bazı tablo motorlarının (`*MergeTree`) oldukça büyük bir varlık olan eklenen her blok için diskte bir veri parçası oluşturun. Benzer bir şekilde, `*MergeTree` tablolar ekleme sırasında verileri sıralar ve yeterince büyük bir blok boyutu RAM'de daha fazla veriyi sıralamaya izin verir. -## min\_insert\_block\_size\_rows {#min-insert-block-size-rows} +## min_insert_block_size_rows {#min-insert-block-size-rows} Bir tabloya eklenebilen blok içindeki minimum satır sayısını ayarlar. `INSERT` sorgu. Daha küçük boyutlu bloklar daha büyük olanlara ezilir. @@ -575,7 +575,7 @@ Olası değerler: Varsayılan değer: 1048576. -## min\_insert\_block\_size\_bytes {#min-insert-block-size-bytes} +## min_insert_block_size_bytes {#min-insert-block-size-bytes} Bir tabloya eklenebilen blok içindeki minimum bayt sayısını ayarlar. `INSERT` sorgu. Daha küçük boyutlu bloklar daha büyük olanlara ezilir. @@ -586,7 +586,7 @@ Olası değerler: Varsayılan değer: 268435456. -## max\_replica\_delay\_for\_distributed\_queries {#settings-max_replica_delay_for_distributed_queries} +## max_replica_delay_for_distributed_queries {#settings-max_replica_delay_for_distributed_queries} Dağıtılmış sorgular için gecikmeli yinelemeleri devre dışı bırakır. Görmek [Çoğalma](../../engines/table-engines/mergetree-family/replication.md). @@ -596,22 +596,22 @@ Varsayılan değer: 300. Yaparken kullanılır `SELECT` çoğaltılmış tablolara işaret eden dağıtılmış bir tablodan. -## max\_threads {#settings-max_threads} +## max_threads {#settings-max_threads} -Uzak sunuculardan veri almak için iş parçacıkları hariç olmak üzere sorgu işleme iş parçacıklarının maksimum sayısı (bkz. ‘max\_distributed\_connections’ parametre). +Uzak sunuculardan veri almak için iş parçacıkları hariç olmak üzere sorgu işleme iş parçacıklarının maksimum sayısı (bkz. ‘max_distributed_connections’ parametre). Bu parametre, paralel olarak sorgu işleme ardışık düzeninin aynı aşamalarını gerçekleştiren iş parçacıkları için geçerlidir. -Örneğin, bir tablodan okurken, ifadeleri işlevlerle değerlendirmek mümkün ise, en azından paralel olarak grup için where ve pre-aggregate ile filtreleyin ‘max\_threads’ konu sayısı, daha sonra ‘max\_threads’ kullanılır. +Örneğin, bir tablodan okurken, ifadeleri işlevlerle değerlendirmek mümkün ise, en azından paralel olarak grup için where ve pre-aggregate ile filtreleyin ‘max_threads’ konu sayısı, daha sonra ‘max_threads’ kullanılır. Varsayılan değer: fiziksel CPU çekirdeği sayısı. Bir kerede bir sunucuda normal olarak birden az SELECT sorgusu çalıştırılırsa, bu parametreyi gerçek işlemci çekirdeği sayısından biraz daha küçük bir değere ayarlayın. -Bir sınır nedeniyle hızlı bir şekilde tamamlanan sorgular için, daha düşük bir ‘max\_threads’. Örneğin, gerekli sayıda giriş her blokta ve max\_threads = 8'de bulunuyorsa, sadece bir tane okumak için yeterli olsa da, 8 blok alınır. +Bir sınır nedeniyle hızlı bir şekilde tamamlanan sorgular için, daha düşük bir ‘max_threads’. Örneğin, gerekli sayıda giriş her blokta ve max_threads = 8'de bulunuyorsa, sadece bir tane okumak için yeterli olsa da, 8 blok alınır. Daha küçük `max_threads` değer, daha az bellek tüketilir. -## max\_ınsert\_threads {#settings-max-insert-threads} +## max_ınsert_threads {#settings-max-insert-threads} Çalıştırılacak maksimum iş parçacığı sayısı `INSERT SELECT` sorgu. @@ -622,61 +622,61 @@ Olası değerler: Varsayılan değer: 0. -Paralellik `INSERT SELECT` etkisi vardır sadece eğer `SELECT` bölüm paralel olarak yürütülür, bkz [max\_threads](#settings-max_threads) ayar. +Paralellik `INSERT SELECT` etkisi vardır sadece eğer `SELECT` bölüm paralel olarak yürütülür, bkz [max_threads](#settings-max_threads) ayar. Daha yüksek değerler daha yüksek bellek kullanımına yol açacaktır. -## max\_compress\_block\_size {#max-compress-block-size} +## max_compress_block_size {#max-compress-block-size} Bir tabloya yazmak için sıkıştırmadan önce sıkıştırılmamış veri bloklarının en büyük boyutu. Varsayılan olarak, 1.048.576 (1 MiB). Boyut azaltılırsa, sıkıştırma oranı önemli ölçüde azalır, önbellek konumu nedeniyle sıkıştırma ve dekompresyon hızı biraz artar ve bellek tüketimi azalır. Bu ayarı değiştirmek için genellikle herhangi bir neden yoktur. Sıkıştırma için blokları (bayttan oluşan bir bellek yığını) sorgu işleme için bloklarla (bir tablodan satır kümesi) karıştırmayın. -## min\_compress\_block\_size {#min-compress-block-size} +## min_compress_block_size {#min-compress-block-size} -İçin [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md)" Tablolar. Sorguları işlerken gecikmeyi azaltmak için, boyutu en az bir sonraki işareti yazarken bir blok sıkıştırılır ‘min\_compress\_block\_size’. Varsayılan olarak, 65.536. +İçin [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md)" Tablolar. Sorguları işlerken gecikmeyi azaltmak için, boyutu en az bir sonraki işareti yazarken bir blok sıkıştırılır ‘min_compress_block_size’. Varsayılan olarak, 65.536. -Sıkıştırılmamış veriler daha az ise, bloğun gerçek boyutu ‘max\_compress\_block\_size’, bu değerden daha az değildir ve bir işaret için veri hacminden daha az değildir. +Sıkıştırılmamış veriler daha az ise, bloğun gerçek boyutu ‘max_compress_block_size’, bu değerden daha az değildir ve bir işaret için veri hacminden daha az değildir. -Bir örneğe bakalım. Varsaymak ‘index\_granularity’ tablo oluşturma sırasında 8192 olarak ayarlandı. +Bir örneğe bakalım. Varsaymak ‘index_granularity’ tablo oluşturma sırasında 8192 olarak ayarlandı. -Bir uint32 tipi sütun yazıyoruz (değer başına 4 bayt). 8192 satır yazarken, toplam 32 KB veri olacaktır. Min\_compress\_block\_size = 65.536 olduğundan, her iki işaret için sıkıştırılmış bir blok oluşturulacaktır. +Bir uint32 tipi sütun yazıyoruz (değer başına 4 bayt). 8192 satır yazarken, toplam 32 KB veri olacaktır. Min_compress_block_size = 65.536 olduğundan, her iki işaret için sıkıştırılmış bir blok oluşturulacaktır. Dize türüne sahip bir URL sütunu yazıyoruz (değer başına ortalama 60 bayt boyutu). 8192 satır yazarken, ortalama 500 KB veri biraz daha az olacaktır. Bu 65,536'dan fazla olduğu için, her işaret için sıkıştırılmış bir blok oluşturulacaktır. Bu durumda, diskteki verileri tek bir işaret aralığında okurken, ekstra veriler sıkıştırılmaz. Bu ayarı değiştirmek için genellikle herhangi bir neden yoktur. -## max\_query\_size {#settings-max_query_size} +## max_query_size {#settings-max_query_size} SQL ayrıştırıcısı ile ayrıştırmak için RAM'e alınabilecek bir sorgunun en büyük kısmı. INSERT sorgusu, bu kısıtlamaya dahil olmayan ayrı bir akış ayrıştırıcısı (o(1) RAM tüketir) tarafından işlenen INSERT için veri de içerir. Varsayılan değer: 256 KiB. -## ınteractive\_delay {#interactive-delay} +## ınteractive_delay {#interactive-delay} İstek yürütülmesinin iptal edilip edilmediğini kontrol etmek ve ilerlemeyi göndermek için mikrosaniye cinsinden Aralık. Varsayılan değer: 100.000 (iptal için denetler ve ilerleme saniyede on kez gönderir). -## connect\_timeout, receıve\_tımeout, send\_timeout {#connect-timeout-receive-timeout-send-timeout} +## connect_timeout, receıve_tımeout, send_timeout {#connect-timeout-receive-timeout-send-timeout} İstemci ile iletişim kurmak için kullanılan sokette saniye cinsinden zaman aşımları. Varsayılan değer: 10, 300, 300. -## cancel\_http\_readonly\_queries\_on\_client\_close {#cancel-http-readonly-queries-on-client-close} +## cancel_http_readonly_queries_on_client_close {#cancel-http-readonly-queries-on-client-close} Cancels HTTP read-only queries (e.g. SELECT) when a client closes the connection without waiting for the response. Varsayılan değer: 0 -## poll\_interval {#poll-interval} +## poll_interval {#poll-interval} Belirtilen saniye sayısı için bir bekleme döngüsünde kilitleyin. Varsayılan değer: 10. -## max\_distributed\_connections {#max-distributed-connections} +## max_distributed_connections {#max-distributed-connections} Tek bir dağıtılmış tabloya tek bir sorgunun dağıtılmış işlenmesi için uzak sunucularla eşzamanlı bağlantı sayısı. Kümedeki sunucu sayısından daha az bir değer ayarlamanızı öneririz. @@ -684,20 +684,20 @@ Varsayılan değer: 1024. Aşağıdaki parametreler yalnızca dağıtılmış tablolar oluştururken (ve bir sunucu başlatırken) kullanılır, bu nedenle bunları çalışma zamanında değiştirmek için hiçbir neden yoktur. -## distributed\_connections\_pool\_size {#distributed-connections-pool-size} +## distributed_connections_pool_size {#distributed-connections-pool-size} Tüm sorguların tek bir dağıtılmış tabloya dağıtılmış işlenmesi için uzak sunucularla eşzamanlı bağlantıların maksimum sayısı. Kümedeki sunucu sayısından daha az bir değer ayarlamanızı öneririz. Varsayılan değer: 1024. -## connect\_timeout\_with\_failover\_ms {#connect-timeout-with-failover-ms} +## connect_timeout_with_failover_ms {#connect-timeout-with-failover-ms} Dağıtılmış bir tablo altyapısı için uzak bir sunucuya bağlanmak için milisaniye cinsinden zaman aşımı ‘shard’ ve ‘replica’ bölümler küme tanımında kullanılır. Başarısız olursa, çeşitli yinelemelere bağlanmak için birkaç deneme yapılır. Varsayılan değer: 50. -## connections\_with\_failover\_max\_tries {#connections-with-failover-max-tries} +## connections_with_failover_max_tries {#connections-with-failover-max-tries} Dağıtılmış tablo altyapısı için her yineleme ile bağlantı girişimi sayısı. @@ -708,27 +708,27 @@ Varsayılan değer: 3. Aşırı değerleri (bir sorgu sonucunun sütunlarındaki minimum ve maksimum değerler) saymak ister. 0 veya 1 kabul eder. Varsayılan olarak, 0 (devre dışı). Daha fazla bilgi için bölüme bakın “Extreme values”. -## use\_uncompressed\_cache {#setting-use_uncompressed_cache} +## use_uncompressed_cache {#setting-use_uncompressed_cache} Sıkıştırılmamış blokların önbelleğinin kullanılıp kullanılmayacağı. 0 veya 1 kabul eder. Varsayılan olarak, 0 (devre dışı). -Sıkıştırılmamış önbelleği (yalnızca mergetree ailesindeki tablolar için) kullanmak, çok sayıda kısa Sorgu ile çalışırken gecikmeyi önemli ölçüde azaltabilir ve verimi artırabilir. Sık sık kısa istek Gönderen kullanıcılar için bu ayarı etkinleştirin. Ayrıca dikkat [uncompressed\_cache\_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuration parameter (only set in the config file) – the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed and the least-used data is automatically deleted. +Sıkıştırılmamış önbelleği (yalnızca mergetree ailesindeki tablolar için) kullanmak, çok sayıda kısa Sorgu ile çalışırken gecikmeyi önemli ölçüde azaltabilir ve verimi artırabilir. Sık sık kısa istek Gönderen kullanıcılar için bu ayarı etkinleştirin. Ayrıca dikkat [uncompressed_cache_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuration parameter (only set in the config file) – the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed and the least-used data is automatically deleted. -En azından biraz büyük bir veri hacmi (bir milyon satır veya daha fazla) okuyan sorgular için sıkıştırılmamış önbellek, gerçekten küçük sorgular için yer kazanmak için otomatik olarak devre dışı bırakılır. Bu tutmak anlamına gelir ‘use\_uncompressed\_cache’ ayar her zaman 1 olarak ayarlanır. +En azından biraz büyük bir veri hacmi (bir milyon satır veya daha fazla) okuyan sorgular için sıkıştırılmamış önbellek, gerçekten küçük sorgular için yer kazanmak için otomatik olarak devre dışı bırakılır. Bu tutmak anlamına gelir ‘use_uncompressed_cache’ ayar her zaman 1 olarak ayarlanır. -## replace\_running\_query {#replace-running-query} +## replace_running_query {#replace-running-query} -HTTP arayüzünü kullanırken, ‘query\_id’ parametre geçirilebilir. Bu, sorgu tanımlayıcısı olarak hizmet veren herhangi bir dizedir. -Aynı kullanıcıdan aynı sorgu varsa ‘query\_id’ zaten şu anda var, davranış bağlıdır ‘replace\_running\_query’ parametre. +HTTP arayüzünü kullanırken, ‘query_id’ parametre geçirilebilir. Bu, sorgu tanımlayıcısı olarak hizmet veren herhangi bir dizedir. +Aynı kullanıcıdan aynı sorgu varsa ‘query_id’ zaten şu anda var, davranış bağlıdır ‘replace_running_query’ parametre. -`0` (default) – Throw an exception (don't allow the query to run if a query with the same ‘query\_id’ zaten çalışan) var. +`0` (default) – Throw an exception (don't allow the query to run if a query with the same ‘query_id’ zaten çalışan) var. `1` – Cancel the old query and start running the new one. Üye.Metrica, segmentasyon koşulları için öneriler uygulamak için 1 olarak ayarlanmış bu parametreyi kullanır. Bir sonraki karakteri girdikten sonra, eski sorgu henüz tamamlanmamışsa, iptal edilmelidir. -## stream\_flush\_interval\_ms {#stream-flush-interval-ms} +## stream_flush_interval_ms {#stream-flush-interval-ms} -Bir zaman aşımı durumunda akışlı tablolar için çalışır veya bir iş parçacığı oluşturduğunda [max\_ınsert\_block\_size](#settings-max_insert_block_size) satırlar. +Bir zaman aşımı durumunda akışlı tablolar için çalışır veya bir iş parçacığı oluşturduğunda [max_ınsert_block_size](#settings-max_insert_block_size) satırlar. Varsayılan değer 7500'dür. @@ -787,7 +787,7 @@ load_balancing = first_or_random Bu `first_or_random` algoritma sorunu çözer `in_order` algoritma. İle `in_order`, bir çoğaltma aşağı giderse, kalan yinelemeler normal trafik miktarını işlerken bir sonraki bir çift yük alır. Kullanırken `first_or_random` algoritma, yük hala mevcut olan kopyalar arasında eşit olarak dağıtılır. -## prefer\_localhost\_replica {#settings-prefer-localhost-replica} +## prefer_localhost_replica {#settings-prefer-localhost-replica} Etkinleştirir / devre dışı bırakır tercih kullanarak localhost çoğaltma dağıtılmış sorguları işlerken. @@ -799,19 +799,19 @@ Olası değerler: Varsayılan değer: 1. !!! warning "Uyarıcı" - Kullanıyorsanız bu ayarı devre dışı bırakın [max\_parallel\_replicas](#settings-max_parallel_replicas). + Kullanıyorsanız bu ayarı devre dışı bırakın [max_parallel_replicas](#settings-max_parallel_replicas). -## totals\_mode {#totals-mode} +## totals_mode {#totals-mode} -MAX\_ROWS\_TO\_GROUP\_BY ve group\_by\_overflow\_mode = ‘any’ Bulunmak. +MAX_ROWS_TO_GROUP_BY ve group_by_overflow_mode = ‘any’ Bulunmak. Bölümüne bakınız “WITH TOTALS modifier”. -## totals\_auto\_threshold {#totals-auto-threshold} +## totals_auto_threshold {#totals-auto-threshold} İçin eşik `totals_mode = 'auto'`. Bölümüne bakınız “WITH TOTALS modifier”. -## max\_parallel\_replicas {#settings-max_parallel_replicas} +## max_parallel_replicas {#settings-max_parallel_replicas} Bir sorgu yürütülürken her parça için en fazla yineleme sayısı. Tutarlılık için (aynı veri bölünmesinin farklı bölümlerini elde etmek için), bu seçenek yalnızca örnekleme anahtarı ayarlandığında çalışır. @@ -824,7 +824,7 @@ Sorguların derlenmesini etkinleştirin. Varsayılan olarak, 0 (devre dışı). Derleme yalnızca sorgu işleme boru hattının bir parçası için kullanılır: toplamanın ilk aşaması için (GROUP BY). Potansiyel hattın bu bölümü derlenmişse, sorgu, kısa döngüleri ve inlining toplu işlev çağrılarının dağıtımı nedeniyle daha hızlı çalışabilir. Birden çok basit toplama işlevine sahip sorgular için maksimum performans artışı (nadir durumlarda dört kata kadar daha hızlı) görülür. Tipik olarak, performans kazancı önemsİzdİr. Çok nadir durumlarda, sorgu yürütülmesini yavaşlatabilir. -## min\_count\_to\_compile {#min-count-to-compile} +## min_count_to_compile {#min-count-to-compile} Derleme çalıştırmadan önce derlenmiş bir kod yığını potansiyel olarak kaç kez kullanılır. Varsayılan olarak, 3. For testing, the value can be set to 0: compilation runs synchronously and the query waits for the end of the compilation process before continuing execution. For all other cases, use values ​​starting with 1. Compilation normally takes about 5-10 seconds. @@ -833,27 +833,27 @@ Değer 1 veya daha fazla ise, derleme zaman uyumsuz olarak ayrı bir iş parçac Derlenmiş kod, sorguda kullanılan toplama işlevlerinin her farklı birleşimi ve GROUP BY yan tümcesindeki anahtarların türü için gereklidir. The results of the compilation are saved in the build directory in the form of .so files. There is no restriction on the number of compilation results since they don't use very much space. Old results will be used after server restarts, except in the case of a server upgrade – in this case, the old results are deleted. -## output\_format\_json\_quote\_64bit\_integers {#session_settings-output_format_json_quote_64bit_integers} +## output_format_json_quote_64bit_integers {#session_settings-output_format_json_quote_64bit_integers} Değer doğruysa, json\* Int64 ve Uİnt64 formatlarını kullanırken tamsayılar tırnak içinde görünür (çoğu JavaScript uygulamasıyla uyumluluk için); aksi takdirde, tamsayılar tırnak işaretleri olmadan çıktılanır. -## format\_csv\_delimiter {#settings-format_csv_delimiter} +## format_csv_delimiter {#settings-format_csv_delimiter} Karakter CSV verilerinde bir sınırlayıcı olarak yorumlanır. Varsayılan olarak, sınırlayıcı `,`. -## ınput\_format\_csv\_unquoted\_null\_literal\_as\_null {#settings-input_format_csv_unquoted_null_literal_as_null} +## ınput_format_csv_unquoted_null_literal_as_null {#settings-input_format_csv_unquoted_null_literal_as_null} CSV giriş biçimi sağlar veya unquoted ayrıştırma devre dışı bırakır için `NULL` literal olarak (eşanlamlı `\N`). -## output\_format\_csv\_crlf\_end\_of\_line {#settings-output-format-csv-crlf-end-of-line} +## output_format_csv_crlf_end_of_line {#settings-output-format-csv-crlf-end-of-line} Unix stili (LF) yerine CSV'DE DOS/Windows stili çizgi ayırıcı (CRLF) kullanın. -## output\_format\_tsv\_crlf\_end\_of\_line {#settings-output-format-tsv-crlf-end-of-line} +## output_format_tsv_crlf_end_of_line {#settings-output-format-tsv-crlf-end-of-line} Unıx stili (LF) yerine TSV'DE DOC/Windows stili çizgi ayırıcı (CRLF) kullanın. -## insert\_quorum {#settings-insert_quorum} +## insert_quorum {#settings-insert_quorum} Çekirdek yazma sağlar. @@ -868,7 +868,7 @@ Nis writesap yazar Nisaptaki tüm kopyalar tutarlıdır, yani önceki tüm verileri içerir `INSERT` sorgular. Bu `INSERT` sıra doğrusallaştırılmıştır. -Yazılan verileri okurken `insert_quorum` olabilir kullanın [select\_sequential\_consistency](#settings-select_sequential_consistency) seçenek. +Yazılan verileri okurken `insert_quorum` olabilir kullanın [select_sequential_consistency](#settings-select_sequential_consistency) seçenek. ClickHouse bir istisna oluşturur @@ -877,10 +877,10 @@ ClickHouse bir istisna oluşturur Ayrıca bakınız: -- [ınsert\_quorum\_timeout](#settings-insert_quorum_timeout) -- [select\_sequential\_consistency](#settings-select_sequential_consistency) +- [ınsert_quorum_timeout](#settings-insert_quorum_timeout) +- [select_sequential_consistency](#settings-select_sequential_consistency) -## ınsert\_quorum\_timeout {#settings-insert_quorum_timeout} +## ınsert_quorum_timeout {#settings-insert_quorum_timeout} Çekirdek zaman aşımına saniyeler içinde yazın. Zaman aşımı geçti ve yazma henüz gerçekleşmedi, ClickHouse bir özel durum oluşturur ve istemci aynı bloğu aynı veya başka bir yineleme yazmak için sorguyu yinelemeniz gerekir. @@ -888,10 +888,10 @@ Varsayılan değer: 60 saniye. Ayrıca bakınız: -- [insert\_quorum](#settings-insert_quorum) -- [select\_sequential\_consistency](#settings-select_sequential_consistency) +- [insert_quorum](#settings-insert_quorum) +- [select_sequential_consistency](#settings-select_sequential_consistency) -## select\_sequential\_consistency {#settings-select_sequential_consistency} +## select_sequential_consistency {#settings-select_sequential_consistency} İçin sıralı tutarlılığı etkinleştirir veya devre dışı bırakır `SELECT` sorgular: @@ -908,10 +908,10 @@ Sıralı tutarlılık etkinleştirildiğinde, clickhouse istemci çalıştırmak Ayrıca bakınız: -- [insert\_quorum](#settings-insert_quorum) -- [ınsert\_quorum\_timeout](#settings-insert_quorum_timeout) +- [insert_quorum](#settings-insert_quorum) +- [ınsert_quorum_timeout](#settings-insert_quorum_timeout) -## ınsert\_deduplicate {#settings-insert-deduplicate} +## ınsert_deduplicate {#settings-insert-deduplicate} Blok tekilleştirmesini etkinleştirir veya devre dışı bırakır `INSERT` (çoğaltılmış \* tablolar için). @@ -924,7 +924,7 @@ Varsayılan değer: 1. Varsayılan olarak, çoğaltılmış tablolara eklenen bloklar `INSERT` açıklama tekilleştirilmiştir (bkz [Veri Çoğaltma](../../engines/table-engines/mergetree-family/replication.md)). -## deduplicate\_blocks\_ın\_dependent\_materialized\_views {#settings-deduplicate-blocks-in-dependent-materialized-views} +## deduplicate_blocks_ın_dependent_materialized_views {#settings-deduplicate-blocks-in-dependent-materialized-views} Yinelenmiş\* tablolardan veri alan materialized görünümler için tekilleştirme denetimini etkinleştirir veya devre dışı bırakır. @@ -942,7 +942,7 @@ Eklenen bir blok, kaynak tablodaki tekilleştirme nedeniyle atlanırsa, ekli mat Aynı zamanda, bu davranış “breaks” `INSERT` idempotency. Eğer bir `INSERT` ana tabloya başarılı oldu ve `INSERT` into a materialized view failed (e.g. because of communication failure with Zookeeper) a client will get an error and can retry the operation. However, the materialized view won't receive the second insert because it will be discarded by deduplication in the main (source) table. The setting `deduplicate_blocks_in_dependent_materialized_views` bu davranışı değiştirmeye izin verir. Yeniden denemede, somutlaştırılmış bir görünüm tekrar ekleme işlemini alacak ve tekilleştirme kontrolünü kendi başına gerçekleştirecektir, kaynak tablo için onay sonucunu yoksayar ve ilk hata nedeniyle kaybedilen satırları ekler. -## max\_network\_bytes {#settings-max-network-bytes} +## max_network_bytes {#settings-max-network-bytes} Alınan veya bir sorgu yürütülürken ağ üzerinden iletilen veri birimi (bayt cinsinden) sınırlar. Bu ayar, her bir sorgu için geçerlidir. @@ -953,7 +953,7 @@ Olası değerler: Varsayılan değer: 0. -## max\_network\_bandwidth {#settings-max-network-bandwidth} +## max_network_bandwidth {#settings-max-network-bandwidth} Ağ üzerinden veri alışverişinin hızını saniyede bayt cinsinden sınırlar. Bu ayar her sorgu için geçerlidir. @@ -964,7 +964,7 @@ Olası değerler: Varsayılan değer: 0. -## max\_network\_bandwidth\_for\_user {#settings-max-network-bandwidth-for-user} +## max_network_bandwidth_for_user {#settings-max-network-bandwidth-for-user} Ağ üzerinden veri alışverişinin hızını saniyede bayt cinsinden sınırlar. Bu ayar, tek bir kullanıcı tarafından gerçekleştirilen tüm aynı anda çalışan sorgular için geçerlidir. @@ -975,7 +975,7 @@ Olası değerler: Varsayılan değer: 0. -## max\_network\_bandwidth\_for\_all\_users {#settings-max-network-bandwidth-for-all-users} +## max_network_bandwidth_for_all_users {#settings-max-network-bandwidth-for-all-users} Verilerin ağ üzerinden saniyede bayt olarak değiştirildiği hızı sınırlar. Bu ayar, sunucuda aynı anda çalışan tüm sorgular için geçerlidir. @@ -986,7 +986,7 @@ Olası değerler: Varsayılan değer: 0. -## count\_distinct\_implementation {#settings-count_distinct_implementation} +## count_distinct_implementation {#settings-count_distinct_implementation} Aşağıdakilerden hang theisinin `uniq*` işlevleri gerçekleştirmek için kullanılmalıdır [COUNT(DISTINCT …)](../../sql-reference/aggregate-functions/reference.md#agg_function-count) yapı. @@ -1000,7 +1000,7 @@ Olası değerler: Varsayılan değer: `uniqExact`. -## skip\_unavailable\_shards {#settings-skip_unavailable_shards} +## skip_unavailable_shards {#settings-skip_unavailable_shards} Etkinleştirir veya sessizce kullanılamaz kırıkları atlama devre dışı bırakır. @@ -1030,13 +1030,13 @@ Olası değerler: Varsayılan değer: 0. -## optimize\_skip\_unused\_shards {#settings-optimize_skip_unused_shards} +## optimize_skip_unused_shards {#settings-optimize_skip_unused_shards} Prewhere/WHERE (verilerin sharding anahtarı tarafından dağıtıldığını varsayar, aksi takdirde hiçbir şey yapmaz). Varsayılan değer: 0 -## force\_optimize\_skip\_unused\_shards {#settings-force_optimize_skip_unused_shards} +## force_optimize_skip_unused_shards {#settings-force_optimize_skip_unused_shards} Sorgu yürütülmesini etkinleştirir veya devre dışı bırakır [`optimize_skip_unused_shards`](#settings-optimize_skip_unused_shards) etkin ve kullanılmayan kırıkları atlama mümkün değildir. Atlama mümkün değilse ve ayar etkinse özel durum atılır. @@ -1048,7 +1048,7 @@ Olası değerler: Varsayılan değer: 0 -## optimize\_throw\_if\_noop {#setting-optimize_throw_if_noop} +## optimize_throw_if_noop {#setting-optimize_throw_if_noop} Bir özel durum atmayı etkinleştirir veya devre dışı bırakır. [OPTIMIZE](../../sql-reference/statements/misc.md#misc_operations-optimize) sorgu birleştirme gerçekleştirmedi. @@ -1061,19 +1061,19 @@ Olası değerler: Varsayılan değer: 0. -## distributed\_replica\_error\_half\_life {#settings-distributed_replica_error_half_life} +## distributed_replica_error_half_life {#settings-distributed_replica_error_half_life} - Türü: saniye - Varsayılan değer: 60 saniye -Dağıtılmış tablolardaki hataların ne kadar hızlı sıfırlandığını denetler. Bir yineleme bir süre için kullanılamıyorsa, 5 hataları biriktirir ve distributed\_replica\_error\_half\_lıfe 1 saniye olarak ayarlanır, sonra yineleme son hatadan sonra normal 3 saniye olarak kabul edilir. +Dağıtılmış tablolardaki hataların ne kadar hızlı sıfırlandığını denetler. Bir yineleme bir süre için kullanılamıyorsa, 5 hataları biriktirir ve distributed_replica_error_half_lıfe 1 saniye olarak ayarlanır, sonra yineleme son hatadan sonra normal 3 saniye olarak kabul edilir. Ayrıca bakınız: - [Masa motoru Dağıt Distributedıldı](../../engines/table-engines/special/distributed.md) -- [distributed\_replica\_error\_cap](#settings-distributed_replica_error_cap) +- [distributed_replica_error_cap](#settings-distributed_replica_error_cap) -## distributed\_replica\_error\_cap {#settings-distributed_replica_error_cap} +## distributed_replica_error_cap {#settings-distributed_replica_error_cap} - Tür: imzasız int - Varsayılan değer: 1000 @@ -1083,9 +1083,9 @@ Her yineleme hata sayısı çok fazla hata biriken tek bir yineleme engelleyerek Ayrıca bakınız: - [Masa motoru Dağıt Distributedıldı](../../engines/table-engines/special/distributed.md) -- [distributed\_replica\_error\_half\_life](#settings-distributed_replica_error_half_life) +- [distributed_replica_error_half_life](#settings-distributed_replica_error_half_life) -## distributed\_directory\_monitor\_sleep\_time\_ms {#distributed_directory_monitor_sleep_time_ms} +## distributed_directory_monitor_sleep_time_ms {#distributed_directory_monitor_sleep_time_ms} İçin taban aralığı [Dağılı](../../engines/table-engines/special/distributed.md) veri göndermek için tablo motoru. Gerçek Aralık, hatalar durumunda katlanarak büyür. @@ -1095,9 +1095,9 @@ Olası değerler: Varsayılan değer: 100 milisaniye. -## distributed\_directory\_monitor\_max\_sleep\_time\_ms {#distributed_directory_monitor_max_sleep_time_ms} +## distributed_directory_monitor_max_sleep_time_ms {#distributed_directory_monitor_max_sleep_time_ms} -İçin Maksimum Aralık [Dağılı](../../engines/table-engines/special/distributed.md) veri göndermek için tablo motoru. Sınırları içinde belirlenen Aralık üstel büyüme [distributed\_directory\_monitor\_sleep\_time\_ms](#distributed_directory_monitor_sleep_time_ms) ayar. +İçin Maksimum Aralık [Dağılı](../../engines/table-engines/special/distributed.md) veri göndermek için tablo motoru. Sınırları içinde belirlenen Aralık üstel büyüme [distributed_directory_monitor_sleep_time_ms](#distributed_directory_monitor_sleep_time_ms) ayar. Olası değerler: @@ -1105,7 +1105,7 @@ Olası değerler: Varsayılan değer: 30000 milisaniye (30 saniye). -## distributed\_directory\_monitor\_batch\_ınserts {#distributed_directory_monitor_batch_inserts} +## distributed_directory_monitor_batch_ınserts {#distributed_directory_monitor_batch_inserts} Eklenen verilerin toplu olarak gönderilmesini etkinleştirir / devre dışı bırakır. @@ -1118,7 +1118,7 @@ Olası değerler: Varsayılan değer: 0. -## os\_thread\_priority {#setting-os-thread-priority} +## os_thread_priority {#setting-os-thread-priority} Önceliği ayarlar ([güzel](https://en.wikipedia.org/wiki/Nice_(Unix))) sorguları yürüten iş parçacıkları için. İşletim sistemi Zamanlayıcısı, kullanılabilir her CPU çekirdeğinde çalışacak bir sonraki iş parçacığını seçerken bu önceliği dikkate alır. @@ -1133,7 +1133,7 @@ Daha düşük değerler daha yüksek öncelik anlamına gelir. Düşük olan ipl Varsayılan değer: 0. -## query\_profiler\_real\_time\_period\_ns {#query_profiler_real_time_period_ns} +## query_profiler_real_time_period_ns {#query_profiler_real_time_period_ns} Gerçek bir saat zamanlayıcı için süreyi ayarlar [sorgu profiler](../../operations/optimizing-performance/sampling-query-profiler.md). Gerçek saat zamanlayıcı duvar saati zaman sayar. @@ -1154,9 +1154,9 @@ Varsayılan değer: 1000000000 nanosaniye (saniyede bir kez). Ayrıca bakınız: -- Sistem tablosu [trace\_log](../../operations/system-tables.md#system_tables-trace_log) +- Sistem tablosu [trace_log](../../operations/system-tables.md#system_tables-trace_log) -## query\_profiler\_cpu\_time\_period\_ns {#query_profiler_cpu_time_period_ns} +## query_profiler_cpu_time_period_ns {#query_profiler_cpu_time_period_ns} Bir CPU saat süreölçerinin dönemini ayarlar. [sorgu profiler](../../operations/optimizing-performance/sampling-query-profiler.md). Bu zamanlayıcı sadece CPU süresini sayar. @@ -1177,9 +1177,9 @@ Varsayılan değer: 1000000000 nanosaniye. Ayrıca bakınız: -- Sistem tablosu [trace\_log](../../operations/system-tables.md#system_tables-trace_log) +- Sistem tablosu [trace_log](../../operations/system-tables.md#system_tables-trace_log) -## allow\_introspection\_functions {#settings-allow_introspection_functions} +## allow_introspection_functions {#settings-allow_introspection_functions} Devre dışı bırakmayı etkinleştirir [ıntrospections fonksiyonları](../../sql-reference/functions/introspection.md) sorgu profilleme için. @@ -1193,23 +1193,23 @@ Varsayılan değer: 0. **Ayrıca Bakınız** - [Örnekleme Sorgusu Profiler](../optimizing-performance/sampling-query-profiler.md) -- Sistem tablosu [trace\_log](../../operations/system-tables.md#system_tables-trace_log) +- Sistem tablosu [trace_log](../../operations/system-tables.md#system_tables-trace_log) -## ınput\_format\_parallel\_parsing {#input-format-parallel-parsing} +## ınput_format_parallel_parsing {#input-format-parallel-parsing} - Tipi: bool - Varsayılan değer: True Veri biçimlerinin paralel ayrıştırma sırasını koruyarak etkinleştirin. Sadece TSV, TKSV, CSV ve JSONEachRow formatları için desteklenir. -## min\_chunk\_bytes\_for\_parallel\_parsing {#min-chunk-bytes-for-parallel-parsing} +## min_chunk_bytes_for_parallel_parsing {#min-chunk-bytes-for-parallel-parsing} - Tür: imzasız int - Varsayılan değer: 1 MiB Her iş parçacığının paralel olarak ayrıştırılacağı bayt cinsinden minimum yığın boyutu. -## output\_format\_avro\_codec {#settings-output_format_avro_codec} +## output_format_avro_codec {#settings-output_format_avro_codec} Çıkış Avro dosyası için kullanılan sıkıştırma codec ayarlar. @@ -1223,7 +1223,7 @@ Olası değerler: Varsayılan değer: `snappy` (varsa) veya `deflate`. -## output\_format\_avro\_sync\_interval {#settings-output_format_avro_sync_interval} +## output_format_avro_sync_interval {#settings-output_format_avro_sync_interval} Çıkış Avro dosyası için senkronizasyon işaretçileri arasında minimum veri boyutunu (bayt cinsinden) ayarlar. @@ -1233,7 +1233,7 @@ Olası değerler: 32 (32 bayt) - 1073741824 (1 GiB) Varsayılan değer: 32768 (32 KiB) -## format\_avro\_schema\_registry\_url {#settings-format_avro_schema_registry_url} +## format_avro_schema_registry_url {#settings-format_avro_schema_registry_url} Sets Confluent Schema Registry URL to use with [AvroConfluent](../../interfaces/formats.md#data-format-avro-confluent) biçimli @@ -1241,7 +1241,7 @@ Type: URL Varsayılan değer: boş -## background\_pool\_size {#background_pool_size} +## background_pool_size {#background_pool_size} Tablo altyapılarında arka plan işlemlerini gerçekleştiren iş parçacıklarının sayısını ayarlar (örneğin, [MergeTree motoru](../../engines/table-engines/mergetree-family/index.md) Tablolar). Bu ayar ClickHouse sunucu başlangıcında uygulanır ve bir kullanıcı oturumunda değiştirilemez. Bu ayarı ayarlayarak, CPU ve disk yükünü yönetirsiniz. Daha küçük havuz boyutu daha az CPU ve disk kaynağı kullanır, ancak arka plan işlemleri daha yavaş ilerler ve bu da sorgu performansını etkileyebilir. diff --git a/docs/tr/operations/system-tables.md b/docs/tr/operations/system-tables.md index 9412669287c..315621f7e7e 100644 --- a/docs/tr/operations/system-tables.md +++ b/docs/tr/operations/system-tables.md @@ -13,7 +13,7 @@ Sistem tablolarında diskte veri bulunan dosyalar veya meta verilere sahip dosya Sistem tabloları salt okunur. Bulun theurlar. ‘system’ veritabanı. -## sistem.asynchronous\_metrics {#system_tables-asynchronous_metrics} +## sistem.asynchronous_metrics {#system_tables-asynchronous_metrics} Arka planda periyodik olarak hesaplanan metrikleri içerir. Örneğin, kullanılan RAM miktarı. @@ -48,7 +48,7 @@ SELECT * FROM system.asynchronous_metrics LIMIT 10 - [İzleme](monitoring.md) — Base concepts of ClickHouse monitoring. - [sistem.metrik](#system_tables-metrics) — Contains instantly calculated metrics. - [sistem.etkinlik](#system_tables-events) — Contains a number of events that have occurred. -- [sistem.metric\_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [sistem.metric_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. ## sistem.kümeler {#system-clusters} @@ -72,8 +72,8 @@ Lütfen unutmayın `errors_count` küme için sorgu başına bir kez güncelleş **Ayrıca bakınız** - [Masa motoru Dağıt Distributedıldı](../engines/table-engines/special/distributed.md) -- [distributed\_replica\_error\_cap ayarı](settings/settings.md#settings-distributed_replica_error_cap) -- [distributed\_replica\_error\_half\_life ayarı](settings/settings.md#settings-distributed_replica_error_half_life) +- [distributed_replica_error_cap ayarı](settings/settings.md#settings-distributed_replica_error_cap) +- [distributed_replica_error_half_life ayarı](settings/settings.md#settings-distributed_replica_error_half_life) ## sistem.sütun {#system-columns} @@ -145,7 +145,7 @@ Bu tablo, adı verilen tek bir dize sütunu içerir ‘name’ – the name of a Sunucunun bildiği her veritabanı, tabloda karşılık gelen bir girdiye sahiptir. Bu sistem tablosu uygulamak için kullanılır `SHOW DATABASES` sorgu. -## sistem.detached\_parts {#system_tables-detached_parts} +## sistem.detached_parts {#system_tables-detached_parts} Müstakil parçaları hakkında bilgiler içerir [MergeTree](../engines/table-engines/mergetree-family/mergetree.md) Tablolar. Bu `reason` sütun, parçanın neden ayrıldığını belirtir. Kullanıcı tarafından ayrılmış parçalar için sebep boştur. Bu tür parçalar ile eklenebilir [ALTER TABLE ATTACH PARTITION\|PART](../sql-reference/statements/alter.md#alter_attach-partition) komut. Diğer sütunların açıklaması için bkz. [sistem.parçalar](#system_tables-parts). Bölüm adı geçersiz ise, bazı sütunların değerleri olabilir `NULL`. Bu tür parçalar ile silinebilir [ALTER TABLE DROP DETACHED PART](../sql-reference/statements/alter.md#alter_drop-detached). @@ -239,9 +239,9 @@ SELECT * FROM system.events LIMIT 5 **Ayrıca Bakınız** -- [sistem.asynchronous\_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [sistem.asynchronous_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. - [sistem.metrik](#system_tables-metrics) — Contains instantly calculated metrics. -- [sistem.metric\_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [sistem.metric_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. - [İzleme](monitoring.md) — Base concepts of ClickHouse monitoring. ## sistem.işlevler {#system-functions} @@ -253,9 +253,9 @@ Sütun: - `name`(`String`) – The name of the function. - `is_aggregate`(`UInt8`) — Whether the function is aggregate. -## sistem.graphite\_retentions {#system-graphite-retentions} +## sistem.graphite_retentions {#system-graphite-retentions} -Parametreleri hakkında bilgi içerir [graphite\_rollup](server-configuration-parameters/settings.md#server_configuration_parameters-graphite) tablo usedlarında kullanılan [\* Graphıtemergetree](../engines/table-engines/mergetree-family/graphitemergetree.md) motorlar. +Parametreleri hakkında bilgi içerir [graphite_rollup](server-configuration-parameters/settings.md#server_configuration_parameters-graphite) tablo usedlarında kullanılan [\* Graphıtemergetree](../engines/table-engines/mergetree-family/graphitemergetree.md) motorlar. Sütun: @@ -324,12 +324,12 @@ SELECT * FROM system.metrics LIMIT 10 **Ayrıca Bakınız** -- [sistem.asynchronous\_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [sistem.asynchronous_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. - [sistem.etkinlik](#system_tables-events) — Contains a number of events that occurred. -- [sistem.metric\_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [sistem.metric_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. - [İzleme](monitoring.md) — Base concepts of ClickHouse monitoring. -## sistem.metric\_log {#system_tables-metric_log} +## sistem.metric_log {#system_tables-metric_log} Tablolardan metrik değerlerinin geçmişini içerir `system.metrics` ve `system.events`, periyodik olarak diske boşaltılır. Metrik geçmişi koleksiyonunu açmak için `system.metric_log`, oluşturmak `/etc/clickhouse-server/config.d/metric_log.xml` aşağıdaki içerik ile: @@ -380,7 +380,7 @@ CurrentMetric_ReplicatedChecks: 0 **Ayrıca bakınız** -- [sistem.asynchronous\_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [sistem.asynchronous_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. - [sistem.etkinlik](#system_tables-events) — Contains a number of events that occurred. - [sistem.metrik](#system_tables-metrics) — Contains instantly calculated metrics. - [İzleme](monitoring.md) — Base concepts of ClickHouse monitoring. @@ -391,7 +391,7 @@ Bu tablo adında tek bir uint64 sütunu içerir ‘number’ bu sıfırdan başl Bu tabloyu testler için veya kaba kuvvet araması yapmanız gerekiyorsa kullanabilirsiniz. Bu tablodan okumalar parallelized değil. -## sistem.numbers\_mt {#system-numbers-mt} +## sistem.numbers_mt {#system-numbers-mt} Olarak aynı ‘system.numbers’ ancak okumalar paralelleştirilmiştir. Sayılar herhangi bir sırayla iade edilebilir. Testler için kullanılır. @@ -483,9 +483,9 @@ Sütun: - `marks_size` (`UInt64`) – Alias for `marks_bytes`. -## sistem.part\_log {#system_tables-part-log} +## sistem.part_log {#system_tables-part-log} -Bu `system.part_log` tablo yalnızca aşağıdaki durumlarda oluşturulur: [part\_log](server-configuration-parameters/settings.md#server_configuration_parameters-part-log) sunucu ayarı belirtilir. +Bu `system.part_log` tablo yalnızca aşağıdaki durumlarda oluşturulur: [part_log](server-configuration-parameters/settings.md#server_configuration_parameters-part-log) sunucu ayarı belirtilir. Bu tablo ile oluşan olaylar hakkında bilgi içerir [veri parçaları](../engines/table-engines/mergetree-family/custom-partitioning-key.md) in the [MergeTree](../engines/table-engines/mergetree-family/mergetree.md) veri ekleme veya birleştirme gibi aile tabloları. @@ -528,11 +528,11 @@ Sütun: - `rows_read` (UInt64) – The number of rows read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. - `bytes_read` (UInt64) – The number of uncompressed bytes read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. - `total_rows_approx` (UInt64) – The approximation of the total number of rows that should be read. For distributed processing, on the requestor server, this is the total for all remote servers. It can be updated during request processing, when new sources to process become known. -- `memory_usage` (UInt64) – Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max\_memory\_usage](../operations/settings/query-complexity.md#settings_max_memory_usage) ayar. +- `memory_usage` (UInt64) – Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max_memory_usage](../operations/settings/query-complexity.md#settings_max_memory_usage) ayar. - `query` (String) – The query text. For `INSERT`, eklemek için veri içermez. - `query_id` (String) – Query ID, if defined. -## sistem.text\_log {#system_tables-text_log} +## sistem.text_log {#system_tables-text_log} Günlük girişleri içerir. Bu tabloya giden günlük seviyesi ile sınırlı olabilir `text_log.level` sunucu ayarı. @@ -559,16 +559,16 @@ Sütun: - `source_file` (`LowCardinality(String)`)- Günlüğü yapıldığı kaynak dosya. - `source_line` (`UInt64`)- Kaynak satır hangi günlüğü yapıldı. -## sistem.query\_log {#system_tables-query_log} +## sistem.query_log {#system_tables-query_log} Sorguların yürütülmesi hakkında bilgi içerir. Her sorgu için, işlem başlangıç saatini, işlem süresini, hata mesajlarını ve diğer bilgileri görebilirsiniz. !!! note "Not" Tablo için giriş verileri içermiyor `INSERT` sorgular. -ClickHouse bu tabloyu yalnızca [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) sunucu parametresi belirtilir. Bu parametre, günlük aralığı veya sorguların oturum açacağı tablonun adı gibi günlük kurallarını ayarlar. +ClickHouse bu tabloyu yalnızca [query_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) sunucu parametresi belirtilir. Bu parametre, günlük aralığı veya sorguların oturum açacağı tablonun adı gibi günlük kurallarını ayarlar. -Sorgu günlüğünü etkinleştirmek için, [log\_queries](settings/settings.md#settings-log-queries) parametre 1. Ayrıntılar için, bkz. [Ayarlar](settings/settings.md) bölme. +Sorgu günlüğünü etkinleştirmek için, [log_queries](settings/settings.md#settings-log-queries) parametre 1. Ayrıntılar için, bkz. [Ayarlar](settings/settings.md) bölme. Bu `system.query_log` tablo iki tür sorgu kaydeder: @@ -636,22 +636,22 @@ Her sorgu bir veya iki satır oluşturur `query_log` tablo, sorgunun durumuna ba 2. Sorgu işleme sırasında bir hata oluştu, iki olay türleri 1 ve 4 oluşturulur. 3. Sorguyu başlatmadan önce bir hata oluşmuşsa, 3 tipi olan tek bir olay oluşturulur. -Varsayılan olarak, günlükleri 7.5 saniye aralıklarla tabloya eklenir. Bu aralığı ayarlayabilirsiniz [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) sunucu ayarı (bkz. `flush_interval_milliseconds` parametre). Günlükleri zorla bellek arabelleğinden tabloya temizlemek için `SYSTEM FLUSH LOGS` sorgu. +Varsayılan olarak, günlükleri 7.5 saniye aralıklarla tabloya eklenir. Bu aralığı ayarlayabilirsiniz [query_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) sunucu ayarı (bkz. `flush_interval_milliseconds` parametre). Günlükleri zorla bellek arabelleğinden tabloya temizlemek için `SYSTEM FLUSH LOGS` sorgu. Tablo elle silindiğinde, otomatik olarak anında oluşturulur. Önceki tüm günlüklerin silineceğini unutmayın. !!! note "Not" Günlüklerin depolama süresi sınırsızdır. Günlükler tablodan otomatik olarak silinmez. Eski günlüklerin kaldırılmasını kendiniz düzenlemeniz gerekir. -İçin keyfi bir bölümleme anahtarı belirtebilirsiniz `system.query_log` tablo içinde [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) sunucu ayarı (bkz. `partition_by` parametre). +İçin keyfi bir bölümleme anahtarı belirtebilirsiniz `system.query_log` tablo içinde [query_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) sunucu ayarı (bkz. `partition_by` parametre). -## sistem.query\_thread\_log {#system_tables-query-thread-log} +## sistem.query_thread_log {#system_tables-query-thread-log} Tablo, her sorgu yürütme iş parçacığı hakkında bilgi içerir. -ClickHouse bu tabloyu yalnızca [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) sunucu parametresi belirtilir. Bu parametre, günlük aralığı veya sorguların oturum açacağı tablonun adı gibi günlük kurallarını ayarlar. +ClickHouse bu tabloyu yalnızca [query_thread_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) sunucu parametresi belirtilir. Bu parametre, günlük aralığı veya sorguların oturum açacağı tablonun adı gibi günlük kurallarını ayarlar. -Sorgu günlüğünü etkinleştirmek için, [log\_query\_threads](settings/settings.md#settings-log-query-threads) parametre 1. Ayrıntılar için, bkz. [Ayarlar](settings/settings.md) bölme. +Sorgu günlüğünü etkinleştirmek için, [log_query_threads](settings/settings.md#settings-log-query-threads) parametre 1. Ayrıntılar için, bkz. [Ayarlar](settings/settings.md) bölme. Sütun: @@ -701,20 +701,20 @@ Sütun: - `ProfileEvents.Names` (Array(String)) — Counters that measure different metrics for this thread. The description of them could be found in the table [sistem.etkinlik](#system_tables-events) - `ProfileEvents.Values` (Array(UInt64)) — Values of metrics for this thread that are listed in the `ProfileEvents.Names` sütun. -Varsayılan olarak, günlükleri 7.5 saniye aralıklarla tabloya eklenir. Bu aralığı ayarlayabilirsiniz [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) sunucu ayarı (bkz. `flush_interval_milliseconds` parametre). Günlükleri zorla bellek arabelleğinden tabloya temizlemek için `SYSTEM FLUSH LOGS` sorgu. +Varsayılan olarak, günlükleri 7.5 saniye aralıklarla tabloya eklenir. Bu aralığı ayarlayabilirsiniz [query_thread_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) sunucu ayarı (bkz. `flush_interval_milliseconds` parametre). Günlükleri zorla bellek arabelleğinden tabloya temizlemek için `SYSTEM FLUSH LOGS` sorgu. Tablo elle silindiğinde, otomatik olarak anında oluşturulur. Önceki tüm günlüklerin silineceğini unutmayın. !!! note "Not" Günlüklerin depolama süresi sınırsızdır. Günlükler tablodan otomatik olarak silinmez. Eski günlüklerin kaldırılmasını kendiniz düzenlemeniz gerekir. -İçin keyfi bir bölümleme anahtarı belirtebilirsiniz `system.query_thread_log` tablo içinde [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) sunucu ayarı (bkz. `partition_by` parametre). +İçin keyfi bir bölümleme anahtarı belirtebilirsiniz `system.query_thread_log` tablo içinde [query_thread_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) sunucu ayarı (bkz. `partition_by` parametre). -## sistem.trace\_log {#system_tables-trace_log} +## sistem.trace_log {#system_tables-trace_log} Örnekleme sorgusu profiler tarafından toplanan yığın izlemeleri içerir. -ClickHouse bu tabloyu oluşturduğunda [trace\_log](server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) sunucu yapılandırma bölümü ayarlanır. Ayrıca [query\_profiler\_real\_time\_period\_ns](settings/settings.md#query_profiler_real_time_period_ns) ve [query\_profiler\_cpu\_time\_period\_ns](settings/settings.md#query_profiler_cpu_time_period_ns) ayarlar ayarlan .malıdır. +ClickHouse bu tabloyu oluşturduğunda [trace_log](server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) sunucu yapılandırma bölümü ayarlanır. Ayrıca [query_profiler_real_time_period_ns](settings/settings.md#query_profiler_real_time_period_ns) ve [query_profiler_cpu_time_period_ns](settings/settings.md#query_profiler_cpu_time_period_ns) ayarlar ayarlan .malıdır. Günlükleri analiz etmek için `addressToLine`, `addressToSymbol` ve `demangle` iç gözlem fonksiyonları. @@ -737,7 +737,7 @@ Sütun: - `thread_number` ([Uİnt32](../sql-reference/data-types/int-uint.md)) — Thread identifier. -- `query_id` ([Dize](../sql-reference/data-types/string.md)) — Query identifier that can be used to get details about a query that was running from the [query\_log](#system_tables-query_log) sistem tablosu. +- `query_id` ([Dize](../sql-reference/data-types/string.md)) — Query identifier that can be used to get details about a query that was running from the [query_log](#system_tables-query_log) sistem tablosu. - `trace` ([Dizi (Uİnt64)](../sql-reference/data-types/array.md)) — Stack trace at the moment of sampling. Each element is a virtual memory address inside ClickHouse server process. @@ -824,7 +824,7 @@ Sütun: - `parts_to_check` (`UInt32`)- Doğrulama için kuyruktaki veri parçalarının sayısı. Hasar görebileceğinden şüphe varsa, bir parça doğrulama kuyruğuna konur. - `zookeeper_path` (`String`)- ZooKeeper tablo verilerine yolu. - `replica_name` (`String`)- Zookeeper çoğaltma adı. Aynı tablonun farklı kopyaları farklı adlara sahiptir. -- `replica_path` (`String`)- ZooKeeper çoğaltma veri yolu. Birleştirme ile aynı ‘zookeeper\_path/replicas/replica\_path’. +- `replica_path` (`String`)- ZooKeeper çoğaltma veri yolu. Birleştirme ile aynı ‘zookeeper_path/replicas/replica_path’. - `columns_version` (`Int32`)- Tablo yapısının sürüm numarası. ALTER kaç kez gerçekleştirildiğini gösterir. Kopyaların farklı sürümleri varsa, bazı kopyaların tüm değişiklikleri henüz yapmadığı anlamına gelir. - `queue_size` (`UInt32`)- Yapılması beklenen işlemler için sıranın büyüklüğü. İşlemler, veri bloklarını, birleştirmeleri ve diğer bazı eylemleri eklemeyi içerir. Genellikle ile çakışmaktadır `future_parts`. - `inserts_in_queue` (`UInt32`)- Yapılması gereken veri bloklarının eklerinin sayısı. Eklemeler genellikle oldukça hızlı bir şekilde çoğaltılır. Bu sayı büyükse, bir şeylerin yanlış olduğu anlamına gelir. @@ -845,7 +845,7 @@ Sonraki 4 sütun, yalnızca ZK ile aktif bir oturumun olduğu sıfır olmayan bi - `active_replicas` (`UInt8`)- ZooKeeper bir oturum var bu tablonun kopyaları sayısı (yani, işleyen kopyaları sayısı). Tüm sütunları talep ederseniz, Tablo biraz yavaş çalışabilir, çünkü ZooKeeper birkaç okuma her satır için yapılır. -Son 4 sütun (log\_max\_ındex, log\_pointer, total\_replicas, active\_replicas) istemiyorsanız, tablo hızlı bir şekilde çalışır. +Son 4 sütun (log_max_ındex, log_pointer, total_replicas, active_replicas) istemiyorsanız, tablo hızlı bir şekilde çalışır. Örneğin, her şeyin böyle düzgün çalıştığını kontrol edebilirsiniz: @@ -932,7 +932,7 @@ SELECT * FROM system.settings WHERE changed AND name='load_balancing' - [Sorgular için izinler](settings/permissions-for-queries.md#settings_readonly) - [Ayarlardaki kısıtlamalar](settings/constraints-on-settings.md) -## sistem.table\_engines {#system.table_engines} +## sistem.table_engines {#system.table_engines} ``` text ┌─name───────────────────┬─value───────┐ @@ -943,7 +943,7 @@ SELECT * FROM system.settings WHERE changed AND name='load_balancing' └────────────────────────┴─────────────┘ ``` -## sistem.merge\_tree\_settings {#system-merge_tree_settings} +## sistem.merge_tree_settings {#system-merge_tree_settings} İçin ayarlar hakkında bilgi içerir `MergeTree` Tablolar. @@ -955,7 +955,7 @@ Sütun: - `type` (String) — Setting type (implementation specific string value). - `changed` (UInt8) — Whether the setting was explicitly defined in the config or explicitly changed. -## sistem.table\_engines {#system-table-engines} +## sistem.table_engines {#system-table-engines} Sunucu tarafından desteklenen tablo motorlarının açıklamasını ve özellik destek bilgilerini içerir. @@ -1118,25 +1118,25 @@ Tablo hakkında bilgi içerir [mutasyonlar](../sql-reference/statements/alter.md **veritabanı**, **Tablo** - Mutasyonun uygulandığı veritabanı ve tablonun adı. -**mutation\_id** - Mutasyonun kimliği. Çoğaltılmış tablolar için bu kimlikler znode adlarına karşılık gelir `/mutations/` ZooKeeper dizin. Yinelenmemiş tablolar için kimlikler, tablonun veri dizinindeki dosya adlarına karşılık gelir. +**mutation_id** - Mutasyonun kimliği. Çoğaltılmış tablolar için bu kimlikler znode adlarına karşılık gelir `/mutations/` ZooKeeper dizin. Yinelenmemiş tablolar için kimlikler, tablonun veri dizinindeki dosya adlarına karşılık gelir. **komut** - Mut commandasyon komut diz (gesi (sorgu afterdan sonra `ALTER TABLE [db.]table`). -**create\_time** - Bu mutasyon komutu idam için sunulduğunda. +**create_time** - Bu mutasyon komutu idam için sunulduğunda. -**block\_numbers.partition\_id**, **block\_numbers.numara** - İç içe geçmiş bir sütun. Çoğaltılmış tabloların mutasyonları için, her bölüm için bir kayıt içerir: bölüm kimliği ve mutasyon tarafından elde edilen blok numarası (her bölümde, yalnızca bu bölümdeki mutasyon tarafından elde edilen blok sayısından daha az sayıda blok içeren parçalar mutasyona uğrayacaktır). Çoğaltılmamış tablolarda, tüm bölümlerdeki blok numaraları tek bir sıra oluşturur. Bu, çoğaltılmamış tabloların mutasyonları için, sütunun mutasyon tarafından elde edilen tek bir blok numarasına sahip bir kayıt içereceği anlamına gelir. +**block_numbers.partition_id**, **block_numbers.numara** - İç içe geçmiş bir sütun. Çoğaltılmış tabloların mutasyonları için, her bölüm için bir kayıt içerir: bölüm kimliği ve mutasyon tarafından elde edilen blok numarası (her bölümde, yalnızca bu bölümdeki mutasyon tarafından elde edilen blok sayısından daha az sayıda blok içeren parçalar mutasyona uğrayacaktır). Çoğaltılmamış tablolarda, tüm bölümlerdeki blok numaraları tek bir sıra oluşturur. Bu, çoğaltılmamış tabloların mutasyonları için, sütunun mutasyon tarafından elde edilen tek bir blok numarasına sahip bir kayıt içereceği anlamına gelir. -**parts\_to\_do** - Mutasyonun bitmesi için mutasyona uğraması gereken veri parçalarının sayısı. +**parts_to_do** - Mutasyonun bitmesi için mutasyona uğraması gereken veri parçalarının sayısı. -**is\_done** - Mutasyon bitti mi? Not bile `parts_to_do = 0` çoğaltılmış bir tablonun mutasyonu, mutasyona uğraması gereken yeni bir veri parçası yaratacak uzun süren bir ekleme nedeniyle henüz yapılmamıştır. +**is_done** - Mutasyon bitti mi? Not bile `parts_to_do = 0` çoğaltılmış bir tablonun mutasyonu, mutasyona uğraması gereken yeni bir veri parçası yaratacak uzun süren bir ekleme nedeniyle henüz yapılmamıştır. Bazı bölümleri mutasyon ile ilgili sorunlar varsa, aşağıdaki sütunlar ek bilgi içerir: -**latest\_failed\_part** - Mutasyona uğramayan en son bölümün adı. +**latest_failed_part** - Mutasyona uğramayan en son bölümün adı. -**latest\_fail\_time** - En son bölüm mutasyon başarısızlığı zamanı. +**latest_fail_time** - En son bölüm mutasyon başarısızlığı zamanı. -**latest\_fail\_reason** - En son bölüm mutasyon başarısızlığına neden olan istisna mesajı. +**latest_fail_reason** - En son bölüm mutasyon başarısızlığına neden olan istisna mesajı. ## sistem.diskler {#system_tables-disks} @@ -1150,7 +1150,7 @@ Sütun: - `total_space` ([Uİnt64](../sql-reference/data-types/int-uint.md)) — Disk volume in bytes. - `keep_free_space` ([Uİnt64](../sql-reference/data-types/int-uint.md)) — Amount of disk space that should stay free on disk in bytes. Defined in the `keep_free_space_bytes` disk yapılandırması parametresi. -## sistem.storage\_policies {#system_tables-storage_policies} +## sistem.storage_policies {#system_tables-storage_policies} Depolama ilkeleri ve birimlerinde tanımlanan bilgiler içerir. [sunucu yapılandırması](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes_configure). diff --git a/docs/tr/operations/tips.md b/docs/tr/operations/tips.md index 50e78dc772f..5b76fe26504 100644 --- a/docs/tr/operations/tips.md +++ b/docs/tr/operations/tips.md @@ -59,7 +59,7 @@ RAID-10 oluştururken, `far` düzen. Bütçeniz izin veriyorsa, RAID-10'u seçin. 4'ten fazla diskiniz varsa, RAID-5 yerine RAID-6 (tercih edilen) veya RAID-50 kullanın. -RAID-5, RAID-6 veya RAID-50 kullanırken, varsayılan değer genellikle en iyi seçenek olmadığından daima stripe\_cache\_size değerini artırın. +RAID-5, RAID-6 veya RAID-50 kullanırken, varsayılan değer genellikle en iyi seçenek olmadığından daima stripe_cache_size değerini artırın. ``` bash $ echo 4096 | sudo tee /sys/block/md2/md/stripe_cache_size diff --git a/docs/tr/operations/troubleshooting.md b/docs/tr/operations/troubleshooting.md index fb547342b2f..e9852be0ce1 100644 --- a/docs/tr/operations/troubleshooting.md +++ b/docs/tr/operations/troubleshooting.md @@ -105,7 +105,7 @@ Kontrol: - Bitiş noktası ayarları. - Kontrol [listen\_host](server-configuration-parameters/settings.md#server_configuration_parameters-listen_host) ve [tcp\_port](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) ayarlar. + Kontrol [listen_host](server-configuration-parameters/settings.md#server_configuration_parameters-listen_host) ve [tcp_port](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) ayarlar. ClickHouse server, yalnızca varsayılan olarak localhost bağlantılarını kabul eder. @@ -117,7 +117,7 @@ Kontrol: Kontrol: - - Bu [tcp\_port\_secure](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) ayar. + - Bu [tcp_port_secure](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) ayar. - İçin ayarlar [SSL sertifikaları](server-configuration-parameters/settings.md#server_configuration_parameters-openssl). Bağlanırken uygun parametreleri kullanın. Örneğin, kullanın `port_secure` parametre ile `clickhouse_client`. diff --git a/docs/tr/sql-reference/aggregate-functions/parametric-functions.md b/docs/tr/sql-reference/aggregate-functions/parametric-functions.md index 27c359c807c..3e8be673bb2 100644 --- a/docs/tr/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/tr/sql-reference/aggregate-functions/parametric-functions.md @@ -494,6 +494,6 @@ Solution: Write in the GROUP BY query SearchPhrase HAVING uniqUpTo(4)(UserID) >= [Orijinal makale](https://clickhouse.tech/docs/en/query_language/agg_functions/parametric_functions/) -## sumMapFiltered (keys\_to\_keep) (anahtarlar, değerler) {#summapfilteredkeys-to-keepkeys-values} +## sumMapFiltered (keys_to_keep) (anahtarlar, değerler) {#summapfilteredkeys-to-keepkeys-values} Aynı davranış [sumMap](reference.md#agg_functions-summap) dışında bir dizi anahtar parametre olarak geçirilir. Bu, özellikle yüksek bir Anahtarlık ile çalışırken yararlı olabilir. diff --git a/docs/tr/sql-reference/aggregate-functions/reference.md b/docs/tr/sql-reference/aggregate-functions/reference.md index 9865fa914ed..32706b325a9 100644 --- a/docs/tr/sql-reference/aggregate-functions/reference.md +++ b/docs/tr/sql-reference/aggregate-functions/reference.md @@ -31,7 +31,7 @@ Her iki durumda da döndürülen değerin türü [Uİnt64](../../sql-reference/d **Ayrıntı** -ClickHouse destekler `COUNT(DISTINCT ...)` sözdizimi. Bu yapının davranışı Aşağıdakilere bağlıdır [count\_distinct\_implementation](../../operations/settings/settings.md#settings-count_distinct_implementation) ayar. Aşağıdakilerden hang theisini tanımlar [uniq\*](#agg_function-uniq) fonksiyonlar işlemi gerçekleştirmek için kullanılır. Varsayılan değer [uniqExact](#agg_function-uniqexact) İşlev. +ClickHouse destekler `COUNT(DISTINCT ...)` sözdizimi. Bu yapının davranışı Aşağıdakilere bağlıdır [count_distinct_implementation](../../operations/settings/settings.md#settings-count_distinct_implementation) ayar. Aşağıdakilerden hang theisini tanımlar [uniq\*](#agg_function-uniq) fonksiyonlar işlemi gerçekleştirmek için kullanılır. Varsayılan değer [uniqExact](#agg_function-uniqexact) İşlev. Bu `SELECT count() FROM table` tablodaki girdi sayısı ayrı olarak depolanmadığı için sorgu en iyi duruma getirilmez. Tablodan küçük bir sütun seçer ve içindeki değerlerin sayısını sayar. @@ -721,7 +721,7 @@ Fonksiyon değişken sayıda parametre alır. Parametreler olabilir `Tuple`, `Ar - [uniqCombined](#agg_function-uniqcombined) - [uniqHLL12](#agg_function-uniqhll12) -## groupArray (x), groupArray (max\_size)(x) {#agg_function-grouparray} +## groupArray (x), groupArray (max_size)(x) {#agg_function-grouparray} Bağımsız değişken değerleri dizisi oluşturur. Değerler diziye herhangi bir (belirsiz) sırayla eklenebilir. @@ -967,7 +967,7 @@ FROM t └───────────┴──────────────────────────────────┴───────────────────────┘ ``` -## groupUniqArray (x), groupUniqArray (max\_size)(x) {#groupuniqarrayx-groupuniqarraymax-sizex} +## groupUniqArray (x), groupUniqArray (max_size)(x) {#groupuniqarrayx-groupuniqarraymax-sizex} Farklı bağımsız değişken değerlerinden bir dizi oluşturur. Bellek tüketimi için aynıdır `uniqExact` İşlev. diff --git a/docs/tr/sql-reference/ansi.md b/docs/tr/sql-reference/ansi.md index 5946da05ce2..650f32d257d 100644 --- a/docs/tr/sql-reference/ansi.md +++ b/docs/tr/sql-reference/ansi.md @@ -36,8 +36,8 @@ Aşağıdaki tabloda, sorgu özelliği ClickHouse çalışır, ancak ANSI SQL'DE | E021-01 | Karakter veri türü | Hayır{.text-danger} | | | E021-02 | Karakter değişken veri türü | Hayır{.text-danger} | `String` benzer şekilde davranır, ancak parantez içinde uzunluk sınırı olmadan | | E021-03 | Karakter değişmezleri | Kısmi{.text-warning} | Ardışık değişmezlerin ve karakter seti desteğinin otomatik olarak birleştirilmesi yok | -| E021-04 | CHARACTER\_LENGTH işlevi | Kısmi{.text-warning} | Hayır `USING` yan | -| E021-05 | OCTET\_LENGTH işlevi | Hayır{.text-danger} | `LENGTH` benzer şekilde davranır | +| E021-04 | CHARACTER_LENGTH işlevi | Kısmi{.text-warning} | Hayır `USING` yan | +| E021-05 | OCTET_LENGTH işlevi | Hayır{.text-danger} | `LENGTH` benzer şekilde davranır | | E021-06 | SUBSTRING | Kısmi{.text-warning} | İçin destek yok `SIMILAR` ve `ESCAPE` CLA ,us ,es, no `SUBSTRING_REGEX` varyant | | E021-07 | Karakter birleştirme | Kısmi{.text-warning} | Hayır `COLLATE` yan | | E021-08 | Üst ve alt fonksiyonlar | Evet{.text-success} | | @@ -144,7 +144,7 @@ Aşağıdaki tabloda, sorgu özelliği ClickHouse çalışır, ancak ANSI SQL'DE | F051-03 | Zaman damgası veri türü (zaman damgası literal desteği dahil) en az 0 ve 6 kesirli saniye hassasiyetle | Hayır{.text-danger} | `DateTime64` zaman benzer işlevsellik sağlar | | F051-04 | Tarih, Saat ve zaman damgası veri türlerinde karşılaştırma yüklemi | Kısmi{.text-warning} | Yalnızca bir veri türü kullanılabilir | | F051-05 | Datetime türleri ve karakter dizesi türleri arasında açık döküm | Evet{.text-success} | | -| F051-06 | CURRENT\_DATE | Hayır{.text-danger} | `today()` benzer mi | +| F051-06 | CURRENT_DATE | Hayır{.text-danger} | `today()` benzer mi | | F051-07 | LOCALTIME | Hayır{.text-danger} | `now()` benzer mi | | F051-08 | LOCALTIMESTAMP | Hayır{.text-danger} | | | **F081** | **Sendika ve görüş EXCEPTLERDE** | **Kısmi**{.text-warning} | | diff --git a/docs/tr/sql-reference/data-types/aggregatefunction.md b/docs/tr/sql-reference/data-types/aggregatefunction.md index 18fdb9549a4..45847352ad5 100644 --- a/docs/tr/sql-reference/data-types/aggregatefunction.md +++ b/docs/tr/sql-reference/data-types/aggregatefunction.md @@ -5,7 +5,7 @@ toc_priority: 52 toc_title: AggregateFunction (ad, types_of_arguments...) --- -# AggregateFunction(name, types\_of\_arguments…) {#data-type-aggregatefunction} +# AggregateFunction(name, types_of_arguments…) {#data-type-aggregatefunction} Aggregate functions can have an implementation-defined intermediate state that can be serialized to an AggregateFunction(…) data type and stored in a table, usually, by means of [materyalize bir görünüm](../../sql-reference/statements/create.md#create-view). Bir toplama işlevi durumu üretmek için ortak yolu ile toplama işlevi çağırarak olduğunu `-State` sonek. Gelecekte toplanmanın nihai sonucunu elde etmek için, aynı toplama işlevini `-Merge`sonek. diff --git a/docs/tr/sql-reference/data-types/datetime.md b/docs/tr/sql-reference/data-types/datetime.md index b4706c87023..52656d2afb7 100644 --- a/docs/tr/sql-reference/data-types/datetime.md +++ b/docs/tr/sql-reference/data-types/datetime.md @@ -31,7 +31,7 @@ Bu [clickhouse-müşteri](../../interfaces/cli.md) veri türünü başlatırken ClickHouse çıkış değerleri `YYYY-MM-DD hh:mm:ss` varsayılan olarak metin biçimi. Çıkış ile değiştirebilirsiniz [formatDateTime](../../sql-reference/functions/date-time-functions.md#formatdatetime) İşlev. -Clickhouse'a veri eklerken, Tarih ve saat dizelerinin farklı biçimlerini kullanabilirsiniz. [date\_time\_input\_format](../../operations/settings/settings.md#settings-date_time_input_format) ayar. +Clickhouse'a veri eklerken, Tarih ve saat dizelerinin farklı biçimlerini kullanabilirsiniz. [date_time_input_format](../../operations/settings/settings.md#settings-date_time_input_format) ayar. ## Örnekler {#examples} diff --git a/docs/tr/sql-reference/data-types/fixedstring.md b/docs/tr/sql-reference/data-types/fixedstring.md index 28f41a51eb2..e3075ebbb40 100644 --- a/docs/tr/sql-reference/data-types/fixedstring.md +++ b/docs/tr/sql-reference/data-types/fixedstring.md @@ -22,7 +22,7 @@ Bu `FixedString` veri tam olarak uzunluğa sahip olduğunda tür etkilidir `N` b Verimli bir şekilde depolan theabilen değerlere örnekler `FixedString`- yazılan sütunlar: - IP adreslerinin ikili gösterimi (`FixedString(16)` IPv6 için). -- Language codes (ru\_RU, en\_US … ). +- Language codes (ru_RU, en_US … ). - Currency codes (USD, RUB … ). - Karma ikili gösterimi (`FixedString(16)` MD5 için, `FixedString(32)` SHA256 için). diff --git a/docs/tr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md b/docs/tr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md index 8a483e01a30..f569f844ace 100644 --- a/docs/tr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md +++ b/docs/tr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md @@ -27,7 +27,7 @@ Aşağıdaki hiyerarşik yapıya bakın: Bu hiyerarşi aşağıdaki sözlük tablosu olarak ifade edilebilir. -| region\_id | parent\_region | region\_name | +| region_id | parent_region | region_name | |------------|----------------|--------------| | 1 | 0 | Rusya | | 2 | 1 | Moskova | diff --git a/docs/tr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/tr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 016da6fb71f..cc45231fe72 100644 --- a/docs/tr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/tr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -9,7 +9,7 @@ toc_title: "S\xF6zl\xFCkleri bellekte saklama" Sözlükleri bellekte saklamanın çeşitli yolları vardır. -Biz tavsiye [düzlük](#flat), [karıştırıyordu](#dicts-external_dicts_dict_layout-hashed) ve [complex\_key\_hashed](#complex-key-hashed). hangi optimum işleme hızı sağlamak. +Biz tavsiye [düzlük](#flat), [karıştırıyordu](#dicts-external_dicts_dict_layout-hashed) ve [complex_key_hashed](#complex-key-hashed). hangi optimum işleme hızı sağlamak. Önbelleğe alma, potansiyel olarak düşük performans ve en uygun parametreleri seçmede zorluklar nedeniyle önerilmez. Bölümünde devamını oku “[önbellek](#cache)”. @@ -54,13 +54,13 @@ LAYOUT(LAYOUT_TYPE(param value)) -- layout settings - [düzlük](#flat) - [karıştırıyordu](#dicts-external_dicts_dict_layout-hashed) -- [sparse\_hashed](#dicts-external_dicts_dict_layout-sparse_hashed) +- [sparse_hashed](#dicts-external_dicts_dict_layout-sparse_hashed) - [önbellek](#cache) - [direkt](#direct) -- [range\_hashed](#range-hashed) -- [complex\_key\_hashed](#complex-key-hashed) -- [complex\_key\_cache](#complex-key-cache) -- [ıp\_trie](#ip-trie) +- [range_hashed](#range-hashed) +- [complex_key_hashed](#complex-key-hashed) +- [complex_key_cache](#complex-key-cache) +- [ıp_trie](#ip-trie) ### düzlük {#flat} @@ -106,7 +106,7 @@ veya LAYOUT(HASHED()) ``` -### sparse\_hashed {#dicts-external_dicts_dict_layout-sparse_hashed} +### sparse_hashed {#dicts-external_dicts_dict_layout-sparse_hashed} Benzer `hashed`, ancak daha fazla CPU kullanımı lehine daha az bellek kullanır. @@ -122,7 +122,7 @@ Yapılandırma örneği: LAYOUT(SPARSE_HASHED()) ``` -### complex\_key\_hashed {#complex-key-hashed} +### complex_key_hashed {#complex-key-hashed} Bu tür depolama kompozit ile kullanım içindir [anahtarlar](external-dicts-dict-structure.md). Benzer `hashed`. @@ -138,7 +138,7 @@ Yapılandırma örneği: LAYOUT(COMPLEX_KEY_HASHED()) ``` -### range\_hashed {#range-hashed} +### range_hashed {#range-hashed} Sözlük, sıralı bir aralık dizisi ve bunlara karşılık gelen değerleri olan bir karma tablo şeklinde bellekte saklanır. @@ -293,7 +293,7 @@ Yeterince büyük bir önbellek boyutu ayarlayın. Sen hücre sayısını seçme !!! warning "Uyarıcı" Rasgele okuma ile sorguları işlemek için yavaş olduğundan, ClickHouse kaynak olarak kullanmayın. -### complex\_key\_cache {#complex-key-cache} +### complex_key_cache {#complex-key-cache} Bu tür depolama kompozit ile kullanım içindir [anahtarlar](external-dicts-dict-structure.md). Benzer `cache`. @@ -319,7 +319,7 @@ veya LAYOUT(DIRECT()) ``` -### ıp\_trie {#ip-trie} +### ıp_trie {#ip-trie} Bu tür depolama, ağ öneklerini (IP adresleri) asn gibi meta verilere eşlemek içindir. diff --git a/docs/tr/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/tr/sql-reference/dictionaries/external-dictionaries/external-dicts.md index 076970e6141..cfbbc1a1ad8 100644 --- a/docs/tr/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ b/docs/tr/sql-reference/dictionaries/external-dictionaries/external-dicts.md @@ -15,9 +15,9 @@ ClickHouse: - Sözlükleri periyodik olarak günceller ve eksik değerleri dinamik olarak yükler. Başka bir deyişle, sözlükler dinamik olarak yüklenebilir. - Xml dosyaları ile harici sözlükler oluşturmak için izin verir veya [DDL sorguları](../../statements/create.md#create-dictionary-query). -Dış sözlüklerin yapılandırması bir veya daha fazla xml dosyasında bulunabilir. Yapılandırma yolu belirtilen [dictionaries\_config](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config) parametre. +Dış sözlüklerin yapılandırması bir veya daha fazla xml dosyasında bulunabilir. Yapılandırma yolu belirtilen [dictionaries_config](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config) parametre. -Sözlükler sunucu başlangıçta veya ilk kullanımda, bağlı olarak yüklenebilir [dictionaries\_lazy\_load](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load) ayar. +Sözlükler sunucu başlangıçta veya ilk kullanımda, bağlı olarak yüklenebilir [dictionaries_lazy_load](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load) ayar. Bu [sözlükler](../../../operations/system-tables.md#system_tables-dictionaries) sistem tablosu sunucuda yapılandırılmış sözlükler hakkında bilgi içerir. Her sözlük için orada bulabilirsiniz: diff --git a/docs/tr/sql-reference/functions/array-functions.md b/docs/tr/sql-reference/functions/array-functions.md index 9638481db52..e1887af03bc 100644 --- a/docs/tr/sql-reference/functions/array-functions.md +++ b/docs/tr/sql-reference/functions/array-functions.md @@ -1028,7 +1028,7 @@ Sonuç: ## arrayAUC {#arrayauc} -Auc'yi hesaplayın (makine öğreniminde bir kavram olan eğrinin altındaki alan, daha fazla ayrıntıya bakın: https://en.wikipedia.org/wiki/Receiver\_operating\_characteristic\#Area\_under\_the\_curve). +Auc'yi hesaplayın (makine öğreniminde bir kavram olan eğrinin altındaki alan, daha fazla ayrıntıya bakın: https://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve). **Sözdizimi** diff --git a/docs/tr/sql-reference/functions/bitmap-functions.md b/docs/tr/sql-reference/functions/bitmap-functions.md index 8bc7f835770..8e51218da16 100644 --- a/docs/tr/sql-reference/functions/bitmap-functions.md +++ b/docs/tr/sql-reference/functions/bitmap-functions.md @@ -65,7 +65,7 @@ SELECT bitmapToArray(bitmapBuild([1, 2, 3, 4, 5])) AS res ## bitmapsubsetınrange {#bitmap-functions-bitmapsubsetinrange} -Belirtilen aralıktaki alt kümesi döndürür (range\_end içermez). +Belirtilen aralıktaki alt kümesi döndürür (range_end içermez). ``` sql bitmapSubsetInRange(bitmap, range_start, range_end) @@ -241,7 +241,7 @@ SELECT bitmapCardinality(bitmapBuild([1, 2, 3, 4, 5])) AS res ## bitmapMin {#bitmapmin} -Kümedeki uint64 türünün en küçük değerini yeniden çalıştırın, küme boşsa UİNT32\_MAX. +Kümedeki uint64 türünün en küçük değerini yeniden çalıştırın, küme boşsa UİNT32_MAX. bitmapMin(bitmap) @@ -288,8 +288,8 @@ Bitmap'teki bir değer dizisini başka bir değer dizisine dönüştürün, sonu **Parametre** - `bitmap` – bitmap object. -- `from_array` – UInt32 array. For idx in range \[0, from\_array.size()), if bitmap contains from\_array\[idx\], then replace it with to\_array\[idx\]. Note that the result depends on array ordering if there are common elements between from\_array and to\_array. -- `to_array` – UInt32 array, its size shall be the same to from\_array. +- `from_array` – UInt32 array. For idx in range \[0, from_array.size()), if bitmap contains from_array\[idx\], then replace it with to_array\[idx\]. Note that the result depends on array ordering if there are common elements between from_array and to_array. +- `to_array` – UInt32 array, its size shall be the same to from_array. **Örnek** diff --git a/docs/tr/sql-reference/functions/date-time-functions.md b/docs/tr/sql-reference/functions/date-time-functions.md index ae489986389..7dc082c3670 100644 --- a/docs/tr/sql-reference/functions/date-time-functions.md +++ b/docs/tr/sql-reference/functions/date-time-functions.md @@ -163,7 +163,7 @@ On dakikalık aralığın başlangıcına kadar bir tarih aşağı yuvarlar. On beş dakikalık aralığın başlangıcına kadar tarih aşağı yuvarlar. -## toStartOfİnterval (time\_or\_data, Aralık x birimi \[, time\_zone\]) {#tostartofintervaltime-or-data-interval-x-unit-time-zone} +## toStartOfİnterval (time_or_data, Aralık x birimi \[, time_zone\]) {#tostartofintervaltime-or-data-interval-x-unit-time-zone} Bu, diğer işlevlerin bir genellemesidir `toStartOf*`. Mesela, `toStartOfInterval(t, INTERVAL 1 year)` aynı döndürür `toStartOfYear(t)`, diff --git a/docs/tr/sql-reference/functions/geo.md b/docs/tr/sql-reference/functions/geo.md index 84db217bb81..39ffd44270c 100644 --- a/docs/tr/sql-reference/functions/geo.md +++ b/docs/tr/sql-reference/functions/geo.md @@ -209,10 +209,10 @@ Verilen kutunun içine giren ve verilen kutunun sınırlarını kesişen, temel **Giriş değerleri** -- longitude\_min - min boylam, aralıkta kayan değer `[-180°, 180°]` -- latitude\_min - min enlem, aralıkta kayan değer `[-90°, 90°]` -- longitude\_max-maksimum boylam, aralıkta kayan değer `[-180°, 180°]` -- latitude\_max-maksimum enlem, aralıkta kayan değer `[-90°, 90°]` +- longitude_min - min boylam, aralıkta kayan değer `[-180°, 180°]` +- latitude_min - min enlem, aralıkta kayan değer `[-90°, 90°]` +- longitude_max-maksimum boylam, aralıkta kayan değer `[-180°, 180°]` +- latitude_max-maksimum enlem, aralıkta kayan değer `[-90°, 90°]` - hassas-geohash hassas, `UInt8` Aralık inta `[1, 12]` Lütfen tüm koordinat parametrelerinin aynı tipte olması gerektiğini unutmayın: `Float32` veya `Float64`. diff --git a/docs/tr/sql-reference/functions/hash-functions.md b/docs/tr/sql-reference/functions/hash-functions.md index e9b64b0bd26..021ffa72ca1 100644 --- a/docs/tr/sql-reference/functions/hash-functions.md +++ b/docs/tr/sql-reference/functions/hash-functions.md @@ -317,7 +317,7 @@ Bir Uint64 Formu jumpconsistenthash hesaplar. İki bağımsız değişkeni kabul eder: bir uint64 tipi anahtar ve kova sayısı. Int32 Döndürür. Daha fazla bilgi için bağlantıya bakın: [JumpConsistentHash](https://arxiv.org/pdf/1406.2294.pdf) -## murmurHash2\_32, murmurHash2\_64 {#murmurhash2-32-murmurhash2-64} +## murmurHash2_32, murmurHash2_64 {#murmurhash2-32-murmurhash2-64} Üreten bir [MurmurHash2](https://github.com/aappleby/smhasher) karma değeri. @@ -385,7 +385,7 @@ Sonuç: └──────────────────────┴─────────────────────┘ ``` -## murmurHash3\_32, murmurHash3\_64 {#murmurhash3-32-murmurhash3-64} +## murmurHash3_32, murmurHash3_64 {#murmurhash3-32-murmurhash3-64} Üreten bir [MurmurHash3](https://github.com/aappleby/smhasher) karma değeri. @@ -415,7 +415,7 @@ SELECT murmurHash3_32(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23: └─────────────┴────────┘ ``` -## murmurHash3\_128 {#murmurhash3-128} +## murmurHash3_128 {#murmurhash3-128} 128-bit üretir [MurmurHash3](https://github.com/aappleby/smhasher) karma değeri. diff --git a/docs/tr/sql-reference/functions/introspection.md b/docs/tr/sql-reference/functions/introspection.md index a2a5b4d53be..9181463ae3f 100644 --- a/docs/tr/sql-reference/functions/introspection.md +++ b/docs/tr/sql-reference/functions/introspection.md @@ -16,11 +16,11 @@ toc_title: "\u0130\xE7g\xF6zlem" - Yüklemek `clickhouse-common-static-dbg` paket. -- Ayarla... [allow\_introspection\_functions](../../operations/settings/settings.md#settings-allow_introspection_functions) ayar 1. +- Ayarla... [allow_introspection_functions](../../operations/settings/settings.md#settings-allow_introspection_functions) ayar 1. For security reasons introspection functions are disabled by default. -ClickHouse için profiler raporları kaydeder [trace\_log](../../operations/system-tables.md#system_tables-trace_log) sistem tablosu. Tablo ve profiler düzgün yapılandırıldığından emin olun. +ClickHouse için profiler raporları kaydeder [trace_log](../../operations/system-tables.md#system_tables-trace_log) sistem tablosu. Tablo ve profiler düzgün yapılandırıldığından emin olun. ## addressToLine {#addresstoline} diff --git a/docs/tr/sql-reference/functions/json-functions.md b/docs/tr/sql-reference/functions/json-functions.md index 81d98174a1b..36e741988e3 100644 --- a/docs/tr/sql-reference/functions/json-functions.md +++ b/docs/tr/sql-reference/functions/json-functions.md @@ -75,7 +75,7 @@ SELECT isValidJSON('{"a": "hello", "b": [-100, 200.0, 300]}') = 1 SELECT isValidJSON('not a json') = 0 ``` -## JSONHas(json\[, indices\_or\_keys\]…) {#jsonhasjson-indices-or-keys} +## JSONHas(json\[, indices_or_keys\]…) {#jsonhasjson-indices-or-keys} Değer JSON belgesinde varsa, `1` iade edilecektir. @@ -108,7 +108,7 @@ SELECT JSONExtractKey('{"a": "hello", "b": [-100, 200.0, 300]}', -2) = 'a' SELECT JSONExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 1) = 'hello' ``` -## JSONLength(json\[, indices\_or\_keys\]…) {#jsonlengthjson-indices-or-keys} +## JSONLength(json\[, indices_or_keys\]…) {#jsonlengthjson-indices-or-keys} Bir json dizisinin veya bir JSON nesnesinin uzunluğunu döndürür. @@ -121,7 +121,7 @@ SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 3 SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}') = 2 ``` -## JSONType(json\[, indices\_or\_keys\]…) {#jsontypejson-indices-or-keys} +## JSONType(json\[, indices_or_keys\]…) {#jsontypejson-indices-or-keys} Bir JSON değerinin türünü döndürür. @@ -135,13 +135,13 @@ SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'a') = 'String' SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 'Array' ``` -## JSONExtractUInt(json\[, indices\_or\_keys\]…) {#jsonextractuintjson-indices-or-keys} +## JSONExtractUInt(json\[, indices_or_keys\]…) {#jsonextractuintjson-indices-or-keys} -## JSONExtractInt(json\[, indices\_or\_keys\]…) {#jsonextractintjson-indices-or-keys} +## JSONExtractInt(json\[, indices_or_keys\]…) {#jsonextractintjson-indices-or-keys} -## JSONExtractFloat(json\[, indices\_or\_keys\]…) {#jsonextractfloatjson-indices-or-keys} +## JSONExtractFloat(json\[, indices_or_keys\]…) {#jsonextractfloatjson-indices-or-keys} -## JSONExtractBool(json\[, indices\_or\_keys\]…) {#jsonextractbooljson-indices-or-keys} +## JSONExtractBool(json\[, indices_or_keys\]…) {#jsonextractbooljson-indices-or-keys} Bir JSON ayrıştırır ve bir değer ayıklayın. Bu işlevler benzer `visitParam` işlevler. @@ -155,7 +155,7 @@ SELECT JSONExtractFloat('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 2) = 200 SELECT JSONExtractUInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', -1) = 300 ``` -## JSONExtractString(json\[, indices\_or\_keys\]…) {#jsonextractstringjson-indices-or-keys} +## JSONExtractString(json\[, indices_or_keys\]…) {#jsonextractstringjson-indices-or-keys} Bir json ayrıştırır ve bir dize ayıklayın. Bu işlev benzer `visitParamExtractString` işlevler. @@ -173,7 +173,7 @@ SELECT JSONExtractString('{"abc":"\\u263"}', 'abc') = '' SELECT JSONExtractString('{"abc":"hello}', 'abc') = '' ``` -## JSONExtract(json\[, indices\_or\_keys…\], Return\_type) {#jsonextractjson-indices-or-keys-return-type} +## JSONExtract(json\[, indices_or_keys…\], Return_type) {#jsonextractjson-indices-or-keys-return-type} Bir Json ayrıştırır ve verilen ClickHouse veri türünün bir değerini çıkarır. @@ -194,7 +194,7 @@ SELECT JSONExtract('{"day": "Thursday"}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday SELECT JSONExtract('{"day": 5}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \'Tuesday\' = 2, \'Wednesday\' = 3, \'Thursday\' = 4, \'Friday\' = 5, \'Saturday\' = 6)') = 'Friday' ``` -## JSONExtractKeysAndValues(json\[, indices\_or\_keys…\], Value\_type) {#jsonextractkeysandvaluesjson-indices-or-keys-value-type} +## JSONExtractKeysAndValues(json\[, indices_or_keys…\], Value_type) {#jsonextractkeysandvaluesjson-indices-or-keys-value-type} Anahtar değer çiftlerini, değerlerin verilen ClickHouse veri türünde olduğu bir JSON'DAN ayrıştırır. @@ -204,7 +204,7 @@ Anahtar değer çiftlerini, değerlerin verilen ClickHouse veri türünde olduğ SELECT JSONExtractKeysAndValues('{"x": {"a": 5, "b": 7, "c": 11}}', 'x', 'Int8') = [('a',5),('b',7),('c',11)] ``` -## JSONExtractRaw(json\[, indices\_or\_keys\]…) {#jsonextractrawjson-indices-or-keys} +## JSONExtractRaw(json\[, indices_or_keys\]…) {#jsonextractrawjson-indices-or-keys} Json'un bir bölümünü ayrıştırılmamış dize olarak döndürür. @@ -216,7 +216,7 @@ Bölüm yoksa veya yanlış bir türe sahipse, boş bir dize döndürülür. SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = '[-100, 200.0, 300]' ``` -## JSONExtractArrayRaw(json\[, indices\_or\_keys…\]) {#jsonextractarrayrawjson-indices-or-keys} +## JSONExtractArrayRaw(json\[, indices_or_keys…\]) {#jsonextractarrayrawjson-indices-or-keys} Her biri ayrıştırılmamış dize olarak temsil edilen json dizisinin öğeleriyle bir dizi döndürür. diff --git a/docs/tr/sql-reference/functions/other-functions.md b/docs/tr/sql-reference/functions/other-functions.md index 928ef999e34..bf055abfa45 100644 --- a/docs/tr/sql-reference/functions/other-functions.md +++ b/docs/tr/sql-reference/functions/other-functions.md @@ -419,7 +419,7 @@ ORDER BY h ASC Bir değeri, bazı öğelerin açıkça tanımlanmış eşlemesine göre diğer öğelere dönüştürür. Bu fonksiyonun iki varyasyonu vardır: -### transform (x, array\_from, array\_to, varsayılan) {#transformx-array-from-array-to-default} +### transform (x, array_from, array_to, varsayılan) {#transformx-array-from-array-to-default} `x` – What to transform. @@ -439,7 +439,7 @@ Türler: Aynı harfin belirtildiği (t veya U), sayısal türler için bunlar eşleşen türler değil, ortak bir türe sahip türler olabilir. Örneğin, ilk bağımsız değişken Int64 türüne sahip olabilir, ikincisi ise Array(Uİnt16) türüne sahiptir. -Eğer... ‘x’ değer, içindeki öğelerden birine eşittir. ‘array\_from’ array, varolan öğeyi döndürür (aynı numaralandırılır) ‘array\_to’ dizi. Aksi takdirde, döner ‘default’. İçinde birden fazla eşleşen öğe varsa ‘array\_from’, maçlardan birini döndürür. +Eğer... ‘x’ değer, içindeki öğelerden birine eşittir. ‘array_from’ array, varolan öğeyi döndürür (aynı numaralandırılır) ‘array_to’ dizi. Aksi takdirde, döner ‘default’. İçinde birden fazla eşleşen öğe varsa ‘array_from’, maçlardan birini döndürür. Örnek: @@ -461,10 +461,10 @@ ORDER BY c DESC └───────────┴────────┘ ``` -### transform (x, array\_from, array\_to) {#transformx-array-from-array-to} +### transform (x, array_from, array_to) {#transformx-array-from-array-to} İlk vary thatasyon differsdan farklıdır. ‘default’ argüman atlandı. -Eğer... ‘x’ değer, içindeki öğelerden birine eşittir. ‘array\_from’ array, eşleşen öğeyi (aynı numaralandırılmış) döndürür ‘array\_to’ dizi. Aksi takdirde, döner ‘x’. +Eğer... ‘x’ değer, içindeki öğelerden birine eşittir. ‘array_from’ array, eşleşen öğeyi (aynı numaralandırılmış) döndürür ‘array_to’ dizi. Aksi takdirde, döner ‘x’. Türler: @@ -1082,7 +1082,7 @@ joinGet(join_storage_table_name, `value_column`, join_keys) Anahtarların listesine karşılık gelen değerlerin listesini döndürür. -Kaynak tabloda kesin yoksa o zaman `0` veya `null` esas alınarak iade edilecektir [join\_use\_nulls](../../operations/settings/settings.md#join_use_nulls) ayar. +Kaynak tabloda kesin yoksa o zaman `0` veya `null` esas alınarak iade edilecektir [join_use_nulls](../../operations/settings/settings.md#join_use_nulls) ayar. Hakkında daha fazla bilgi `join_use_nulls` içinde [Birleştirme işlemi](../../engines/table-engines/special/join.md). @@ -1121,15 +1121,15 @@ Sonuç: └──────────────────────────────────────────────────┘ ``` -## modelEvaluate(model\_name, …) {#function-modelevaluate} +## modelEvaluate(model_name, …) {#function-modelevaluate} Dış modeli değerlendirin. Bir model adı ve model bağımsız değişkenleri kabul eder. Float64 Döndürür. -## throwİf(x \[, custom\_message\]) {#throwifx-custom-message} +## throwİf(x \[, custom_message\]) {#throwifx-custom-message} Argüman sıfır değilse bir istisna atın. -custom\_message-isteğe bağlı bir parametredir: sabit bir dize, bir hata mesajı sağlar +custom_message-isteğe bağlı bir parametredir: sabit bir dize, bir hata mesajı sağlar ``` sql SELECT throwIf(number = 3, 'Too many') FROM numbers(10); diff --git a/docs/tr/sql-reference/functions/string-functions.md b/docs/tr/sql-reference/functions/string-functions.md index 925a0206d2d..b2a3a58f1f2 100644 --- a/docs/tr/sql-reference/functions/string-functions.md +++ b/docs/tr/sql-reference/functions/string-functions.md @@ -31,12 +31,12 @@ Sonuç türü Uint64'tür. Dizenin UTF-8 kodlanmış metni oluşturan bir bayt kümesi içerdiğini varsayarak, Unicode kod noktalarında (karakterlerde değil) bir dizenin uzunluğunu döndürür. Bu varsayım karşılanmazsa, bir sonuç döndürür (bir istisna atmaz). Sonuç türü Uint64'tür. -## char\_length, CHAR\_LENGTH {#char-length} +## char_length, CHAR_LENGTH {#char-length} Dizenin UTF-8 kodlanmış metni oluşturan bir bayt kümesi içerdiğini varsayarak, Unicode kod noktalarında (karakterlerde değil) bir dizenin uzunluğunu döndürür. Bu varsayım karşılanmazsa, bir sonuç döndürür (bir istisna atmaz). Sonuç türü Uint64'tür. -## character\_length, CHARACTER\_LENGTH {#character-length} +## character_length, CHARACTER_LENGTH {#character-length} Dizenin UTF-8 kodlanmış metni oluşturan bir bayt kümesi içerdiğini varsayarak, Unicode kod noktalarında (karakterlerde değil) bir dizenin uzunluğunu döndürür. Bu varsayım karşılanmazsa, bir sonuç döndürür (bir istisna atmaz). Sonuç türü Uint64'tür. @@ -77,7 +77,7 @@ toValidUTF8( input_string ) Parametre: -- input\_string — Any set of bytes represented as the [Dize](../../sql-reference/data-types/string.md) veri türü nesnesi. +- input_string — Any set of bytes represented as the [Dize](../../sql-reference/data-types/string.md) veri türü nesnesi. Döndürülen değer: geçerli UTF-8 dizesi. diff --git a/docs/tr/sql-reference/functions/type-conversion-functions.md b/docs/tr/sql-reference/functions/type-conversion-functions.md index 159ad169080..c767d0e8063 100644 --- a/docs/tr/sql-reference/functions/type-conversion-functions.md +++ b/docs/tr/sql-reference/functions/type-conversion-functions.md @@ -246,7 +246,7 @@ YYYY-MM-DD YYYY-MM-DD hh:mm:ss ``` -Özel durum olarak, uınt32, Int32, Uınt64 veya Int64 sayısal türlerinden bugüne dönüştürme ve sayı 65536'dan büyük veya eşitse, sayı Unıx zaman damgası (ve gün sayısı olarak değil) olarak yorumlanır ve tarihe yuvarlanır. Bu, yaygın yazı oluşumu için destek sağlar ‘toDate(unix\_timestamp)’, aksi takdirde bir hata olur ve daha hantal yazmayı gerektirir ‘toDate(toDateTime(unix\_timestamp))’. +Özel durum olarak, uınt32, Int32, Uınt64 veya Int64 sayısal türlerinden bugüne dönüştürme ve sayı 65536'dan büyük veya eşitse, sayı Unıx zaman damgası (ve gün sayısı olarak değil) olarak yorumlanır ve tarihe yuvarlanır. Bu, yaygın yazı oluşumu için destek sağlar ‘toDate(unix_timestamp)’, aksi takdirde bir hata olur ve daha hantal yazmayı gerektirir ‘toDate(toDateTime(unix_timestamp))’. Bir tarih ve tarih ile saat arasında dönüştürme doğal bir şekilde gerçekleştirilir: boş bir zaman ekleyerek veya saati bırakarak. diff --git a/docs/tr/sql-reference/functions/url-functions.md b/docs/tr/sql-reference/functions/url-functions.md index fe978a5b774..ac4f9c1430d 100644 --- a/docs/tr/sql-reference/functions/url-functions.md +++ b/docs/tr/sql-reference/functions/url-functions.md @@ -125,11 +125,11 @@ Yolu döndürür. Örnek: `/top/news.html` Yol sorgu dizesini içermez. ### pathFull {#pathfull} -Yukarıdaki ile aynı, ancak sorgu dizesi ve parça dahil. Örnek: / top / haberler.html?Sayfa = 2 \# yorumlar +Yukarıdaki ile aynı, ancak sorgu dizesi ve parça dahil. Örnek: / top / haberler.html?Sayfa = 2 # yorumlar ### queryString {#querystring} -Sorgu dizesini döndürür. Örnek: Sayfa = 1 & lr = 213. sorgu dizesi, ilk soru işaretinin yanı sıra \# ve \# sonrası her şeyi içermez. +Sorgu dizesini döndürür. Örnek: Sayfa = 1 & lr = 213. sorgu dizesi, ilk soru işaretinin yanı sıra # ve # sonrası her şeyi içermez. ### parça {#fragment} @@ -137,7 +137,7 @@ Parça tanımlayıcısını döndürür. fragment ilk karma sembolü içermez. ### queryStringAndFragment {#querystringandfragment} -Sorgu dizesini ve parça tanımlayıcısını döndürür. Örnek: Sayfa = 1\#29390. +Sorgu dizesini ve parça tanımlayıcısını döndürür. Örnek: Sayfa = 1#29390. ### extractURLParameter (URL, isim) {#extracturlparameterurl-name} diff --git a/docs/tr/sql-reference/functions/ym-dict-functions.md b/docs/tr/sql-reference/functions/ym-dict-functions.md index 662a6b5a398..f7bdefb10eb 100644 --- a/docs/tr/sql-reference/functions/ym-dict-functions.md +++ b/docs/tr/sql-reference/functions/ym-dict-functions.md @@ -17,12 +17,12 @@ ClickHouse, belirli bölgelerin hangi ülkelere ait olduğu konusunda çeşitli Bu ‘clickhouse-server’ config, dosyayı bölgesel hiyerarşi ile belirtir::`/opt/geo/regions_hierarchy.txt` -Bu dosyanın yanı sıra, yakındaki \_ sembolüne ve isme eklenen herhangi bir sonek (dosya uzantısından önce) olan dosyaları da arar. +Bu dosyanın yanı sıra, yakındaki _ sembolüne ve isme eklenen herhangi bir sonek (dosya uzantısından önce) olan dosyaları da arar. Örneğin, dosyayı da bulacaktır `/opt/geo/regions_hierarchy_ua.txt` varsa. `ua` sözlük anahtarı denir. Soneksiz bir sözlük için anahtar boş bir dizedir. -Tüm sözlükler çalışma zamanında yeniden yüklenir (buıltın\_dıctıonarıes\_reload\_ınterval yapılandırma parametresinde tanımlandığı gibi belirli sayıda saniyede bir kez veya varsayılan olarak saatte bir kez). Ancak, sunucu başladığında kullanılabilir sözlüklerin listesi bir kez tanımlanır. +Tüm sözlükler çalışma zamanında yeniden yüklenir (buıltın_dıctıonarıes_reload_ınterval yapılandırma parametresinde tanımlandığı gibi belirli sayıda saniyede bir kez veya varsayılan olarak saatte bir kez). Ancak, sunucu başladığında kullanılabilir sözlüklerin listesi bir kez tanımlanır. All functions for working with regions have an optional argument at the end – the dictionary key. It is referred to as the geobase. Örnek: @@ -107,7 +107,7 @@ Bir bölgeyi bir ülkeye dönüştürür. Diğer her şekilde, bu işlev aynıd Bir bölgeyi bir kıtaya dönüştürür. Diğer her şekilde, bu işlev aynıdır ‘regionToCity’. Örnek: `regionToContinent(toUInt32(213)) = 10001` Moskova'yı (213) Avrasya'ya (10001) dönüştürür. -### regionToTopContinent (\#regiontotopcontinent) {#regiontotopcontinent-regiontotopcontinent} +### regionToTopContinent (#regiontotopcontinent) {#regiontotopcontinent-regiontotopcontinent} Bölgenin hiyerarşisinde en yüksek kıtayı bulur. diff --git a/docs/tr/sql-reference/statements/alter.md b/docs/tr/sql-reference/statements/alter.md index 6563dca1717..5acdd1ca2c4 100644 --- a/docs/tr/sql-reference/statements/alter.md +++ b/docs/tr/sql-reference/statements/alter.md @@ -208,7 +208,7 @@ Aşağıdaki işlemler ile [bölümler](../../engines/table-engines/mergetree-fa - [ATTACH PART\|PARTITION](#alter_attach-partition) – Adds a part or partition from the `detached` tabloya dizin. - [ATTACH PARTITION FROM](#alter_attach-partition-from) – Copies the data partition from one table to another and adds. - [REPLACE PARTITION](#alter_replace-partition) - Veri bölümünü bir tablodan diğerine kopyalar ve değiştirir. -- [MOVE PARTITION TO TABLE](#alter_move_to_table-partition)(\#alter\_move\_to\_table-partition) - veri bölümünü bir tablodan diğerine taşıyın. +- [MOVE PARTITION TO TABLE](#alter_move_to_table-partition)(#alter_move_to_table-partition) - veri bölümünü bir tablodan diğerine taşıyın. - [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) - Bir bölümdeki belirtilen sütunun değerini sıfırlar. - [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) - Bir bölümde belirtilen ikincil dizini sıfırlar. - [FREEZE PARTITION](#alter_freeze-partition) – Creates a backup of a partition. @@ -436,7 +436,7 @@ Bölüm ifadesini şu şekilde belirtebilirsiniz `ALTER ... PARTITION` farklı - Bu gibi bir değer `partition` sütun `system.parts` Tablo. Mesela, `ALTER TABLE visits DETACH PARTITION 201901`. - Tablo sütunundan ifade olarak. Sabitler ve sabit ifadeler desteklenir. Mesela, `ALTER TABLE visits DETACH PARTITION toYYYYMM(toDate('2019-01-25'))`. - Bölüm kimliğini kullanma. Partition ID, dosya sistemindeki ve Zookeeper'daki bölümlerin adları olarak kullanılan bölümün (mümkünse insan tarafından okunabilir) bir dize tanımlayıcısıdır. Bölüm kimliği belirtilmelidir `PARTITION ID` fık .ra, tek tırnak içinde. Mesela, `ALTER TABLE visits DETACH PARTITION ID '201901'`. -- İn the [ALTER ATTACH PART](#alter_attach-partition) ve [DROP DETACHED PART](#alter_drop-detached) sorgu, bir parçanın adını belirtmek için, bir değer ile dize literal kullanın `name` sütun [sistem.detached\_parts](../../operations/system-tables.md#system_tables-detached_parts) Tablo. Mesela, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. +- İn the [ALTER ATTACH PART](#alter_attach-partition) ve [DROP DETACHED PART](#alter_drop-detached) sorgu, bir parçanın adını belirtmek için, bir değer ile dize literal kullanın `name` sütun [sistem.detached_parts](../../operations/system-tables.md#system_tables-detached_parts) Tablo. Mesela, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. Bölüm belirtilirken tırnak kullanımı bölüm ifadesi türüne bağlıdır. Örneğin, için `String` yazın, adını tırnak içinde belirtmeniz gerekir (`'`). İçin `Date` ve `Int*` türleri hiçbir tırnak gereklidir. diff --git a/docs/tr/sql-reference/statements/insert-into.md b/docs/tr/sql-reference/statements/insert-into.md index f25b57b08fd..dd5dec73efe 100644 --- a/docs/tr/sql-reference/statements/insert-into.md +++ b/docs/tr/sql-reference/statements/insert-into.md @@ -20,7 +20,7 @@ Sorgu eklemek için sütunların bir listesini belirtebilirsiniz `[(c1, c2, c3)] - Hesaplanan değerler `DEFAULT` tablo tanımında belirtilen ifadeler. - Sıfırlar ve boş dizeler, eğer `DEFAULT` ifadeler tanımlanmamıştır. -Eğer [strict\_ınsert\_defaults = 1](../../operations/settings/settings.md), sahip olmayan sütunlar `DEFAULT` tanımlanan sorguda listelenmelidir. +Eğer [strict_ınsert_defaults = 1](../../operations/settings/settings.md), sahip olmayan sütunlar `DEFAULT` tanımlanan sorguda listelenmelidir. Veri herhangi bir İNSERT geçirilebilir [biçimli](../../interfaces/formats.md#formats) ClickHouse tarafından desteklenmektedir. Biçim sorguda açıkça belirtilmelidir: diff --git a/docs/tr/sql-reference/statements/misc.md b/docs/tr/sql-reference/statements/misc.md index cd2a2bc24a6..328bb83a049 100644 --- a/docs/tr/sql-reference/statements/misc.md +++ b/docs/tr/sql-reference/statements/misc.md @@ -57,7 +57,7 @@ Motor fromlardan `*Log` aile başarısızlık otomatik veri kurtarma sağlamaz. Tablo bozuksa, bozuk olmayan verileri başka bir tabloya kopyalayabilirsiniz. Bunu yapmak için : 1. Bozuk tablo ile aynı yapıya sahip yeni bir tablo oluşturun. Bunu yapmak için sorguyu yürütün `CREATE TABLE AS `. -2. Ayarla... [max\_threads](../../operations/settings/settings.md#settings-max_threads) bir sonraki sorguyu tek bir iş parçacığında işlemek için 1 değeri. Bunu yapmak için sorguyu çalıştırın `SET max_threads = 1`. +2. Ayarla... [max_threads](../../operations/settings/settings.md#settings-max_threads) bir sonraki sorguyu tek bir iş parçacığında işlemek için 1 değeri. Bunu yapmak için sorguyu çalıştırın `SET max_threads = 1`. 3. Sorgu yürütme `INSERT INTO SELECT * FROM `. Bu istek bozuk olmayan verileri bozuk tablodan başka bir tabloya kopyalar. Yalnızca bozuk parçadan önceki veriler kopyalanır. 4. Yeniden Başlat `clickhouse-client` sıfırlamak için `max_threads` değer. @@ -253,7 +253,7 @@ Bu `OPTMIZE` sorgu için de desteklenmektedir [MaterializedView](../../engines/t Ne zaman `OPTIMIZE` ile kullanılır [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/replication.md) Tablo motorları ailesi, ClickHouse birleştirme için bir görev oluşturur ve tüm düğümlerde yürütülmeyi bekler (eğer `replication_alter_partitions_sync` ayar etkinse) ' dir. -- Eğer `OPTIMIZE` herhangi bir nedenle bir birleştirme gerçekleştirmez, müşteriye bildirmez. Bildirimleri etkinleştirmek için [optimize\_throw\_if\_noop](../../operations/settings/settings.md#setting-optimize_throw_if_noop) ayar. +- Eğer `OPTIMIZE` herhangi bir nedenle bir birleştirme gerçekleştirmez, müşteriye bildirmez. Bildirimleri etkinleştirmek için [optimize_throw_if_noop](../../operations/settings/settings.md#setting-optimize_throw_if_noop) ayar. - Belirtir aseniz bir `PARTITION`, sadece belirtilen bölüm optimize edilmiştir. [Bölüm ifadesi nasıl ayarlanır](alter.md#alter-how-to-specify-part-expr). - Belirtir specifyseniz `FINAL`, optimizasyon, tüm veriler zaten bir parçada olsa bile gerçekleştirilir. - Belirtir specifyseniz `DEDUPLICATE`, sonra tamamen aynı satırlar tekilleştirilecektir (tüm sütunlar karşılaştırılır), sadece MergeTree motoru için anlamlıdır. diff --git a/docs/tr/sql-reference/statements/system.md b/docs/tr/sql-reference/statements/system.md index ac2796d2539..6a81c78ac40 100644 --- a/docs/tr/sql-reference/statements/system.md +++ b/docs/tr/sql-reference/statements/system.md @@ -24,12 +24,12 @@ toc_title: SYSTEM ## RELOAD DICTIONARIES {#query_language-system-reload-dictionaries} Daha önce başarıyla yüklenen tüm sözlükleri yeniden yükler. -Varsayılan olarak, sözlükler tembel yüklenir (bkz [dictionaries\_lazy\_load](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load)), bu nedenle başlangıçta otomatik olarak yüklenmek yerine, dictGet işlevi aracılığıyla ilk erişimde başlatılır veya ENGİNE = Dictionary ile tablolardan seçim yapılır. Bu `SYSTEM RELOAD DICTIONARIES` sorgu bu sözlükleri yeniden yükler (yüklü). +Varsayılan olarak, sözlükler tembel yüklenir (bkz [dictionaries_lazy_load](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load)), bu nedenle başlangıçta otomatik olarak yüklenmek yerine, dictGet işlevi aracılığıyla ilk erişimde başlatılır veya ENGİNE = Dictionary ile tablolardan seçim yapılır. Bu `SYSTEM RELOAD DICTIONARIES` sorgu bu sözlükleri yeniden yükler (yüklü). Her zaman döner `Ok.` sözlük güncellemesinin sonucu ne olursa olsun. -## Sözlük Dictionary\_name yeniden yükle {#query_language-system-reload-dictionary} +## Sözlük Dictionary_name yeniden yükle {#query_language-system-reload-dictionary} -Tamamen bir sözlük reloads `dictionary_name`, sözlük durumuna bakılmaksızın (LOADED / NOT\_LOADED / FAİLED). +Tamamen bir sözlük reloads `dictionary_name`, sözlük durumuna bakılmaksızın (LOADED / NOT_LOADED / FAİLED). Her zaman döner `Ok.` ne olursa olsun sözlük güncelleme sonucu. Sözlüğün durumu sorgulanarak kontrol edilebilir `system.dictionaries` Tablo. @@ -41,7 +41,7 @@ SELECT name, status FROM system.dictionaries; Clickhouse'un iç DNS önbelleğini sıfırlar. Bazen (eski ClickHouse sürümleri için) altyapıyı değiştirirken (başka bir ClickHouse sunucusunun IP adresini veya sözlükler tarafından kullanılan sunucuyu değiştirirken) bu komutu kullanmak gerekir. -Daha uygun (otomatik) önbellek yönetimi için bkz: disable\_internal\_dns\_cache, dns\_cache\_update\_period parametreleri. +Daha uygun (otomatik) önbellek yönetimi için bkz: disable_internal_dns_cache, dns_cache_update_period parametreleri. ## DROP MARK CACHE {#query_language-system-drop-mark-cache} @@ -49,7 +49,7 @@ Daha uygun (otomatik) önbellek yönetimi için bkz: disable\_internal\_dns\_cac ## FLUSH LOGS {#query_language-system-flush_logs} -Flushes buffers of log messages to system tables (e.g. system.query\_log). Allows you to not wait 7.5 seconds when debugging. +Flushes buffers of log messages to system tables (e.g. system.query_log). Allows you to not wait 7.5 seconds when debugging. ## RELOAD CONFIG {#query_language-system-reload-config} diff --git a/docs/tr/sql-reference/syntax.md b/docs/tr/sql-reference/syntax.md index e6e64c7af41..c9cc6bd078f 100644 --- a/docs/tr/sql-reference/syntax.md +++ b/docs/tr/sql-reference/syntax.md @@ -15,9 +15,9 @@ Bu `INSERT` sorgu her iki ayrıştırıcıyı da kullanır: INSERT INTO t VALUES (1, 'Hello, world'), (2, 'abc'), (3, 'def') ``` -Bu `INSERT INTO t VALUES` parça tam ayrıştırıcı tarafından ayrıştırılır ve veriler `(1, 'Hello, world'), (2, 'abc'), (3, 'def')` hızlı akış ayrıştırıcısı tarafından ayrıştırılır. Ayrıca kullanarak veriler için tam ayrıştırıcı açabilirsiniz [ınput\_format\_values\_interpret\_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions) ayar. Ne zaman `input_format_values_interpret_expressions = 1`, ClickHouse önce hızlı akış ayrıştırıcısı ile değerleri ayrıştırmaya çalışır. Başarısız olursa, ClickHouse veriler için tam ayrıştırıcıyı kullanmaya çalışır ve bir SQL gibi davranır [ifade](#syntax-expressions). +Bu `INSERT INTO t VALUES` parça tam ayrıştırıcı tarafından ayrıştırılır ve veriler `(1, 'Hello, world'), (2, 'abc'), (3, 'def')` hızlı akış ayrıştırıcısı tarafından ayrıştırılır. Ayrıca kullanarak veriler için tam ayrıştırıcı açabilirsiniz [ınput_format_values_interpret_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions) ayar. Ne zaman `input_format_values_interpret_expressions = 1`, ClickHouse önce hızlı akış ayrıştırıcısı ile değerleri ayrıştırmaya çalışır. Başarısız olursa, ClickHouse veriler için tam ayrıştırıcıyı kullanmaya çalışır ve bir SQL gibi davranır [ifade](#syntax-expressions). -Veri herhangi bir biçime sahip olabilir. Bir sorgu alındığında, sunucu daha fazla hesaplar [max\_query\_size](../operations/settings/settings.md#settings-max_query_size) istek bayt RAM (varsayılan olarak, 1 MB) ve geri kalanı akış ayrıştırılır. +Veri herhangi bir biçime sahip olabilir. Bir sorgu alındığında, sunucu daha fazla hesaplar [max_query_size](../operations/settings/settings.md#settings-max_query_size) istek bayt RAM (varsayılan olarak, 1 MB) ve geri kalanı akış ayrıştırılır. Bu büyük sorunları önlemek için izin verir `INSERT` sorgular. Kullanırken `Values` biçim içinde bir `INSERT` sorgu, verilerin bir ifadedeki ifadelerle aynı şekilde ayrıştırıldığı görünebilir `SELECT` sorgu, ancak bu doğru değil. Bu `Values` biçim çok daha sınırlıdır. diff --git a/docs/tr/sql-reference/table-functions/file.md b/docs/tr/sql-reference/table-functions/file.md index 7f5606d2a69..834a2b13cb1 100644 --- a/docs/tr/sql-reference/table-functions/file.md +++ b/docs/tr/sql-reference/table-functions/file.md @@ -15,7 +15,7 @@ file(path, format, structure) **Giriş parametreleri** -- `path` — The relative path to the file from [user\_files\_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Readonly modunda glob'ları takip eden dosya desteğine giden yol: `*`, `?`, `{abc,def}` ve `{N..M}` nerede `N`, `M` — numbers, \``'abc', 'def'` — strings. +- `path` — The relative path to the file from [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Readonly modunda glob'ları takip eden dosya desteğine giden yol: `*`, `?`, `{abc,def}` ve `{N..M}` nerede `N`, `M` — numbers, \``'abc', 'def'` — strings. - `format` — The [biçimli](../../interfaces/formats.md#formats) dosya. - `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. @@ -72,12 +72,12 @@ Birden çok yol bileşenleri globs olabilir. İşlenmek için dosya var olmalı 1. Aşağıdaki göreli yollara sahip birkaç dosyamız olduğunu varsayalım: -- ‘some\_dir/some\_file\_1’ -- ‘some\_dir/some\_file\_2’ -- ‘some\_dir/some\_file\_3’ -- ‘another\_dir/some\_file\_1’ -- ‘another\_dir/some\_file\_2’ -- ‘another\_dir/some\_file\_3’ +- ‘some_dir/some_file_1’ +- ‘some_dir/some_file_2’ +- ‘some_dir/some_file_3’ +- ‘another_dir/some_file_1’ +- ‘another_dir/some_file_2’ +- ‘another_dir/some_file_3’ 1. Bu dosyalardaki satır miktarını sorgula: diff --git a/docs/tr/sql-reference/table-functions/hdfs.md b/docs/tr/sql-reference/table-functions/hdfs.md index 1dbb1abb4e4..438a2bfe00e 100644 --- a/docs/tr/sql-reference/table-functions/hdfs.md +++ b/docs/tr/sql-reference/table-functions/hdfs.md @@ -55,12 +55,12 @@ Birden çok yol bileşenleri globs olabilir. İşlenmek için dosya var olmalı 1. HDFS'DE aşağıdaki Urı'lere sahip birkaç dosyamız olduğunu varsayalım: -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_3’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_3’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_3’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_3’ 1. Bu dosyalardaki satır miktarını sorgula: diff --git a/docs/tr/sql-reference/table-functions/index.md b/docs/tr/sql-reference/table-functions/index.md index 36d4a5ed6a6..3d9715be66a 100644 --- a/docs/tr/sql-reference/table-functions/index.md +++ b/docs/tr/sql-reference/table-functions/index.md @@ -16,12 +16,12 @@ Tablo işlevlerini kullanabilirsiniz: The method for creating a temporary table that is available only in the current query. The table is deleted when the query finishes. -- [Tablo oluştur \](../statements/create.md#create-table-query) sorgu. +- [Tablo oluştur \](../statements/create.md#create-table-query) sorgu. It's one of the methods of creating a table. !!! warning "Uyarıcı" - Eğer tablo işlevlerini kullanamazsınız [allow\_ddl](../../operations/settings/permissions-for-queries.md#settings_allow_ddl) ayarı devre dışı. + Eğer tablo işlevlerini kullanamazsınız [allow_ddl](../../operations/settings/permissions-for-queries.md#settings_allow_ddl) ayarı devre dışı. | İşlev | Açıklama | |--------------------------|-----------------------------------------------------------------------------------------------------------------------------| diff --git a/docs/tr/sql-reference/table-functions/remote.md b/docs/tr/sql-reference/table-functions/remote.md index 059e8aba72b..8d5427c91e4 100644 --- a/docs/tr/sql-reference/table-functions/remote.md +++ b/docs/tr/sql-reference/table-functions/remote.md @@ -80,6 +80,6 @@ Bu `remote` tablo işlevi aşağıdaki durumlarda yararlı olabilir: Kullanıcı belirtilmemişse, `default` kullanılır. Parola belirtilmezse, boş bir parola kullanılır. -`remoteSecure` - aynı `remote` but with secured connection. Default port — [tcp\_port\_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) yapılandırma veya 9440'ten. +`remoteSecure` - aynı `remote` but with secured connection. Default port — [tcp_port_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) yapılandırma veya 9440'ten. [Orijinal makale](https://clickhouse.tech/docs/en/query_language/table_functions/remote/) diff --git a/docs/tr/whats-new/security-changelog.md b/docs/tr/whats-new/security-changelog.md index b4843bd15e8..8e93723aa71 100644 --- a/docs/tr/whats-new/security-changelog.md +++ b/docs/tr/whats-new/security-changelog.md @@ -53,7 +53,7 @@ Kredi: Yandex Bilgi Güvenliği ekibinden Andrey Krasichkov ve Evgeny Sidorov ### CVE-2018-14668 {#cve-2018-14668} -“remote” tablo fonksiyonu izin keyfi semboller “user”, “password” ve “default\_database” çapraz Protokol isteği sahtecilik saldırılarına yol açan alanlar. +“remote” tablo fonksiyonu izin keyfi semboller “user”, “password” ve “default_database” çapraz Protokol isteği sahtecilik saldırılarına yol açan alanlar. Kredi: Yandex Bilgi Güvenliği ekibinden Andrey Krasichkov diff --git a/docs/zh/changelog/index.md b/docs/zh/changelog/index.md index 2f2b7384a76..14bed16c381 100644 --- a/docs/zh/changelog/index.md +++ b/docs/zh/changelog/index.md @@ -11,259 +11,259 @@ toc_title: "\u53D8\u66F4\u65E5\u5FD7" #### 错误修复 {#bug-fix} - 此版本还包含20.1.8.41的所有错误修复 -- 修复丢失 `rows_before_limit_at_least` 用于通过http进行查询(使用处理器管道)。 这修复 [\#9730](https://github.com/ClickHouse/ClickHouse/issues/9730). [\#9757](https://github.com/ClickHouse/ClickHouse/pull/9757) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 修复丢失 `rows_before_limit_at_least` 用于通过http进行查询(使用处理器管道)。 这修复 [#9730](https://github.com/ClickHouse/ClickHouse/issues/9730). [#9757](https://github.com/ClickHouse/ClickHouse/pull/9757) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) ### ClickHouse释放v20.3.3.6,2020-03-17 {#clickhouse-release-v20-3-3-6-2020-03-17} #### 错误修复 {#bug-fix-1} - 此版本还包含20.1.7.38的所有错误修复 -- 修复复制中的错误,如果用户在以前的版本上执行了突变,则不允许复制工作。 这修复 [\#9645](https://github.com/ClickHouse/ClickHouse/issues/9645). [\#9652](https://github.com/ClickHouse/ClickHouse/pull/9652) ([阿利沙平](https://github.com/alesapin)). 它使版本20.3再次向后兼容。 -- 添加设置 `use_compact_format_in_distributed_parts_names` 它允许写文件 `INSERT` 查询到 `Distributed` 表格格式更紧凑。 这修复 [\#9647](https://github.com/ClickHouse/ClickHouse/issues/9647). [\#9653](https://github.com/ClickHouse/ClickHouse/pull/9653) ([阿利沙平](https://github.com/alesapin)). 它使版本20.3再次向后兼容。 +- 修复复制中的错误,如果用户在以前的版本上执行了突变,则不允许复制工作。 这修复 [#9645](https://github.com/ClickHouse/ClickHouse/issues/9645). [#9652](https://github.com/ClickHouse/ClickHouse/pull/9652) ([阿利沙平](https://github.com/alesapin)). 它使版本20.3再次向后兼容。 +- 添加设置 `use_compact_format_in_distributed_parts_names` 它允许写文件 `INSERT` 查询到 `Distributed` 表格格式更紧凑。 这修复 [#9647](https://github.com/ClickHouse/ClickHouse/issues/9647). [#9653](https://github.com/ClickHouse/ClickHouse/pull/9653) ([阿利沙平](https://github.com/alesapin)). 它使版本20.3再次向后兼容。 ### ClickHouse版本v20.3.2.1,2020-03-12 {#clickhouse-release-v20-3-2-1-2020-03-12} #### 向后不兼容的更改 {#backward-incompatible-change} -- 修正了这个问题 `file name too long` 当发送数据 `Distributed` 大量副本的表。 修复了服务器日志中显示副本凭据的问题。 磁盘上的目录名格式已更改为 `[shard{shard_index}[_replica{replica_index}]]`. [\#8911](https://github.com/ClickHouse/ClickHouse/pull/8911) ([米哈伊尔\*科罗托夫](https://github.com/millb))升级到新版本后,您将无法在没有人工干预的情况下降级,因为旧的服务器版本无法识别新的目录格式。 如果要降级,则必须手动将相应的目录重命名为旧格式。 仅当您使用了异步时,此更改才相关 `INSERT`s到 `Distributed` 桌子 在版本20.3.3中,我们将介绍一个设置,让您逐渐启用新格式。 +- 修正了这个问题 `file name too long` 当发送数据 `Distributed` 大量副本的表。 修复了服务器日志中显示副本凭据的问题。 磁盘上的目录名格式已更改为 `[shard{shard_index}[_replica{replica_index}]]`. [#8911](https://github.com/ClickHouse/ClickHouse/pull/8911) ([米哈伊尔\*科罗托夫](https://github.com/millb))升级到新版本后,您将无法在没有人工干预的情况下降级,因为旧的服务器版本无法识别新的目录格式。 如果要降级,则必须手动将相应的目录重命名为旧格式。 仅当您使用了异步时,此更改才相关 `INSERT`s到 `Distributed` 桌子 在版本20.3.3中,我们将介绍一个设置,让您逐渐启用新格式。 - 更改了mutation命令的复制日志条目的格式。 在安装新版本之前,您必须等待旧的突变处理。 -- 实现简单的内存分析器,将堆栈跟踪转储到 `system.trace_log` 超过软分配限制的每N个字节 [\#8765](https://github.com/ClickHouse/ClickHouse/pull/8765) ([伊万](https://github.com/abyss7)) [\#9472](https://github.com/ClickHouse/ClickHouse/pull/9472) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov))列 `system.trace_log` 从改名 `timer_type` 到 `trace_type`. 这将需要改变第三方性能分析和flamegraph处理工具。 -- 在任何地方使用操作系统线程id,而不是内部线程编号。 这修复 [\#7477](https://github.com/ClickHouse/ClickHouse/issues/7477) 老 `clickhouse-client` 无法接收从服务器发送的日志,当设置 `send_logs_level` 已启用,因为结构化日志消息的名称和类型已更改。 另一方面,不同的服务器版本可以相互发送不同类型的日志。 当你不使用 `send_logs_level` 设置,你不应该关心。 [\#8954](https://github.com/ClickHouse/ClickHouse/pull/8954) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 删除 `indexHint` 功能 [\#9542](https://github.com/ClickHouse/ClickHouse/pull/9542) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 删除 `findClusterIndex`, `findClusterValue` 功能。 这修复 [\#8641](https://github.com/ClickHouse/ClickHouse/issues/8641). 如果您正在使用这些功能,请发送电子邮件至 `clickhouse-feedback@yandex-team.com` [\#9543](https://github.com/ClickHouse/ClickHouse/pull/9543) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 现在不允许创建列或添加列 `SELECT` 子查询作为默认表达式。 [\#9481](https://github.com/ClickHouse/ClickHouse/pull/9481) ([阿利沙平](https://github.com/alesapin)) -- 需要联接中的子查询的别名。 [\#9274](https://github.com/ClickHouse/ClickHouse/pull/9274) ([Artem Zuikov](https://github.com/4ertus2)) -- 改进 `ALTER MODIFY/ADD` 查询逻辑。 现在你不能 `ADD` 不带类型的列, `MODIFY` 默认表达式不改变列的类型和 `MODIFY` type不会丢失默认表达式值。 修复 [\#8669](https://github.com/ClickHouse/ClickHouse/issues/8669). [\#9227](https://github.com/ClickHouse/ClickHouse/pull/9227) ([阿利沙平](https://github.com/alesapin)) -- 要求重新启动服务器以应用日志记录配置中的更改。 这是一种临时解决方法,可以避免服务器将日志记录到已删除的日志文件中的错误(请参阅 [\#8696](https://github.com/ClickHouse/ClickHouse/issues/8696)). [\#8707](https://github.com/ClickHouse/ClickHouse/pull/8707) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- 设置 `experimental_use_processors` 默认情况下启用。 此设置允许使用新的查询管道。 这是内部重构,我们期望没有明显的变化。 如果您将看到任何问题,请将其设置为返回零。 [\#8768](https://github.com/ClickHouse/ClickHouse/pull/8768) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 实现简单的内存分析器,将堆栈跟踪转储到 `system.trace_log` 超过软分配限制的每N个字节 [#8765](https://github.com/ClickHouse/ClickHouse/pull/8765) ([伊万](https://github.com/abyss7)) [#9472](https://github.com/ClickHouse/ClickHouse/pull/9472) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov))列 `system.trace_log` 从改名 `timer_type` 到 `trace_type`. 这将需要改变第三方性能分析和flamegraph处理工具。 +- 在任何地方使用操作系统线程id,而不是内部线程编号。 这修复 [#7477](https://github.com/ClickHouse/ClickHouse/issues/7477) 老 `clickhouse-client` 无法接收从服务器发送的日志,当设置 `send_logs_level` 已启用,因为结构化日志消息的名称和类型已更改。 另一方面,不同的服务器版本可以相互发送不同类型的日志。 当你不使用 `send_logs_level` 设置,你不应该关心。 [#8954](https://github.com/ClickHouse/ClickHouse/pull/8954) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 删除 `indexHint` 功能 [#9542](https://github.com/ClickHouse/ClickHouse/pull/9542) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 删除 `findClusterIndex`, `findClusterValue` 功能。 这修复 [#8641](https://github.com/ClickHouse/ClickHouse/issues/8641). 如果您正在使用这些功能,请发送电子邮件至 `clickhouse-feedback@yandex-team.com` [#9543](https://github.com/ClickHouse/ClickHouse/pull/9543) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 现在不允许创建列或添加列 `SELECT` 子查询作为默认表达式。 [#9481](https://github.com/ClickHouse/ClickHouse/pull/9481) ([阿利沙平](https://github.com/alesapin)) +- 需要联接中的子查询的别名。 [#9274](https://github.com/ClickHouse/ClickHouse/pull/9274) ([Artem Zuikov](https://github.com/4ertus2)) +- 改进 `ALTER MODIFY/ADD` 查询逻辑。 现在你不能 `ADD` 不带类型的列, `MODIFY` 默认表达式不改变列的类型和 `MODIFY` type不会丢失默认表达式值。 修复 [#8669](https://github.com/ClickHouse/ClickHouse/issues/8669). [#9227](https://github.com/ClickHouse/ClickHouse/pull/9227) ([阿利沙平](https://github.com/alesapin)) +- 要求重新启动服务器以应用日志记录配置中的更改。 这是一种临时解决方法,可以避免服务器将日志记录到已删除的日志文件中的错误(请参阅 [#8696](https://github.com/ClickHouse/ClickHouse/issues/8696)). [#8707](https://github.com/ClickHouse/ClickHouse/pull/8707) ([Alexander Kuzmenkov](https://github.com/akuzm)) +- 设置 `experimental_use_processors` 默认情况下启用。 此设置允许使用新的查询管道。 这是内部重构,我们期望没有明显的变化。 如果您将看到任何问题,请将其设置为返回零。 [#8768](https://github.com/ClickHouse/ClickHouse/pull/8768) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) #### 新功能 {#new-feature} -- 添加 `Avro` 和 `AvroConfluent` 输入/输出格式 [\#8571](https://github.com/ClickHouse/ClickHouse/pull/8571) ([安德鲁Onyshchuk](https://github.com/oandrew)) [\#8957](https://github.com/ClickHouse/ClickHouse/pull/8957) ([安德鲁Onyshchuk](https://github.com/oandrew)) [\#8717](https://github.com/ClickHouse/ClickHouse/pull/8717) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 过期密钥的多线程和非阻塞更新 `cache` 字典(可选的权限读取旧的)。 [\#8303](https://github.com/ClickHouse/ClickHouse/pull/8303) ([尼基塔\*米哈伊洛夫](https://github.com/nikitamikhaylov)) -- 添加查询 `ALTER ... MATERIALIZE TTL`. 它运行突变,强制通过TTL删除过期的数据,并重新计算所有部分有关ttl的元信息。 [\#8775](https://github.com/ClickHouse/ClickHouse/pull/8775) ([安东\*波波夫](https://github.com/CurtizJ)) -- 如果需要,从HashJoin切换到MergeJoin(在磁盘上 [\#9082](https://github.com/ClickHouse/ClickHouse/pull/9082) ([Artem Zuikov](https://github.com/4ertus2)) -- 已添加 `MOVE PARTITION` 命令 `ALTER TABLE` [\#4729](https://github.com/ClickHouse/ClickHouse/issues/4729) [\#6168](https://github.com/ClickHouse/ClickHouse/pull/6168) ([纪尧姆\*塔瑟里](https://github.com/YiuRULE)) -- 动态地从配置文件重新加载存储配置。 [\#8594](https://github.com/ClickHouse/ClickHouse/pull/8594) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 允许更改 `storage_policy` 为了不那么富有的人。 [\#8107](https://github.com/ClickHouse/ClickHouse/pull/8107) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 增加了对s3存储和表功能的globs/通配符的支持。 [\#8851](https://github.com/ClickHouse/ClickHouse/pull/8851) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 执行 `bitAnd`, `bitOr`, `bitXor`, `bitNot` 为 `FixedString(N)` 数据类型。 [\#9091](https://github.com/ClickHouse/ClickHouse/pull/9091) ([纪尧姆\*塔瑟里](https://github.com/YiuRULE)) -- 添加功能 `bitCount`. 这修复 [\#8702](https://github.com/ClickHouse/ClickHouse/issues/8702). [\#8708](https://github.com/ClickHouse/ClickHouse/pull/8708) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) [\#8749](https://github.com/ClickHouse/ClickHouse/pull/8749) ([ikopylov](https://github.com/ikopylov)) -- 添加 `generateRandom` 表函数生成具有给定模式的随机行。 允许用数据填充任意测试表。 [\#8994](https://github.com/ClickHouse/ClickHouse/pull/8994) ([Ilya Yatsishin](https://github.com/qoega)) -- `JSONEachRowFormat`:当对象包含在顶层数组中时,支持特殊情况。 [\#8860](https://github.com/ClickHouse/ClickHouse/pull/8860) ([克鲁格洛夫\*帕维尔](https://github.com/Avogar)) -- 现在可以创建一个列 `DEFAULT` 取决于默认列的表达式 `ALIAS` 表达。 [\#9489](https://github.com/ClickHouse/ClickHouse/pull/9489) ([阿利沙平](https://github.com/alesapin)) -- 允许指定 `--limit` 超过源数据大小 `clickhouse-obfuscator`. 数据将以不同的随机种子重复。 [\#9155](https://github.com/ClickHouse/ClickHouse/pull/9155) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 已添加 `groupArraySample` 功能(类似于 `groupArray`)与reservior采样算法。 [\#8286](https://github.com/ClickHouse/ClickHouse/pull/8286) ([阿莫斯鸟](https://github.com/amosbird)) -- 现在,您可以监视更新队列的大小 `cache`/`complex_key_cache` 通过系统指标字典。 [\#9413](https://github.com/ClickHouse/ClickHouse/pull/9413) ([尼基塔\*米哈伊洛夫](https://github.com/nikitamikhaylov)) -- 允许使用CRLF作为CSV输出格式的行分隔符与设置 `output_format_csv_crlf_end_of_line` 设置为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) ([米哈伊尔\*科罗托夫](https://github.com/millb)) -- 实现的更多功能 [H3](https://github.com/uber/h3) API: `h3GetBaseCell`, `h3HexAreaM2`, `h3IndexesAreNeighbors`, `h3ToChildren`, `h3ToString` 和 `stringToH3` [\#8938](https://github.com/ClickHouse/ClickHouse/pull/8938) ([Nico Mandery](https://github.com/nmandery)) -- 引入新设置: `max_parser_depth` 控制最大堆栈大小并允许大型复杂查询。 这修复 [\#6681](https://github.com/ClickHouse/ClickHouse/issues/6681) 和 [\#7668](https://github.com/ClickHouse/ClickHouse/issues/7668). [\#8647](https://github.com/ClickHouse/ClickHouse/pull/8647) ([马克西姆\*斯米尔诺夫](https://github.com/qMBQx8GH)) -- 添加设置 `force_optimize_skip_unused_shards` 如果无法跳过未使用的分片,则设置为抛出 [\#8805](https://github.com/ClickHouse/ClickHouse/pull/8805) ([Azat Khuzhin](https://github.com/azat)) -- 允许配置多个磁盘/卷用于存储数据发送 `Distributed` 发动机 [\#8756](https://github.com/ClickHouse/ClickHouse/pull/8756) ([Azat Khuzhin](https://github.com/azat)) -- 支持存储策略 (``)用于存储临时数据。 [\#8750](https://github.com/ClickHouse/ClickHouse/pull/8750) ([Azat Khuzhin](https://github.com/azat)) -- 已添加 `X-ClickHouse-Exception-Code` 如果在发送数据之前引发异常,则设置的HTTP头。 这实现了 [\#4971](https://github.com/ClickHouse/ClickHouse/issues/4971). [\#8786](https://github.com/ClickHouse/ClickHouse/pull/8786) ([米哈伊尔\*科罗托夫](https://github.com/millb)) -- 添加功能 `ifNotFinite`. 这只是一个句法糖: `ifNotFinite(x, y) = isFinite(x) ? x : y`. [\#8710](https://github.com/ClickHouse/ClickHouse/pull/8710) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 已添加 `last_successful_update_time` 列中 `system.dictionaries` 表 [\#9394](https://github.com/ClickHouse/ClickHouse/pull/9394) ([尼基塔\*米哈伊洛夫](https://github.com/nikitamikhaylov)) -- 添加 `blockSerializedSize` 功能(磁盘大小不压缩) [\#8952](https://github.com/ClickHouse/ClickHouse/pull/8952) ([Azat Khuzhin](https://github.com/azat)) -- 添加功能 `moduloOrZero` [\#9358](https://github.com/ClickHouse/ClickHouse/pull/9358) ([hcz](https://github.com/hczhcz)) -- 添加系统表 `system.zeros` 和 `system.zeros_mt` 以及故事功能 `zeros()` 和 `zeros_mt()`. 表(和表函数)包含具有名称的单列 `zero` 和类型 `UInt8`. 此列包含零。 为了测试目的,需要它作为生成许多行的最快方法。 这修复 [\#6604](https://github.com/ClickHouse/ClickHouse/issues/6604) [\#9593](https://github.com/ClickHouse/ClickHouse/pull/9593) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 添加 `Avro` 和 `AvroConfluent` 输入/输出格式 [#8571](https://github.com/ClickHouse/ClickHouse/pull/8571) ([安德鲁Onyshchuk](https://github.com/oandrew)) [#8957](https://github.com/ClickHouse/ClickHouse/pull/8957) ([安德鲁Onyshchuk](https://github.com/oandrew)) [#8717](https://github.com/ClickHouse/ClickHouse/pull/8717) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 过期密钥的多线程和非阻塞更新 `cache` 字典(可选的权限读取旧的)。 [#8303](https://github.com/ClickHouse/ClickHouse/pull/8303) ([尼基塔\*米哈伊洛夫](https://github.com/nikitamikhaylov)) +- 添加查询 `ALTER ... MATERIALIZE TTL`. 它运行突变,强制通过TTL删除过期的数据,并重新计算所有部分有关ttl的元信息。 [#8775](https://github.com/ClickHouse/ClickHouse/pull/8775) ([安东\*波波夫](https://github.com/CurtizJ)) +- 如果需要,从HashJoin切换到MergeJoin(在磁盘上 [#9082](https://github.com/ClickHouse/ClickHouse/pull/9082) ([Artem Zuikov](https://github.com/4ertus2)) +- 已添加 `MOVE PARTITION` 命令 `ALTER TABLE` [#4729](https://github.com/ClickHouse/ClickHouse/issues/4729) [#6168](https://github.com/ClickHouse/ClickHouse/pull/6168) ([纪尧姆\*塔瑟里](https://github.com/YiuRULE)) +- 动态地从配置文件重新加载存储配置。 [#8594](https://github.com/ClickHouse/ClickHouse/pull/8594) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 允许更改 `storage_policy` 为了不那么富有的人。 [#8107](https://github.com/ClickHouse/ClickHouse/pull/8107) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 增加了对s3存储和表功能的globs/通配符的支持。 [#8851](https://github.com/ClickHouse/ClickHouse/pull/8851) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 执行 `bitAnd`, `bitOr`, `bitXor`, `bitNot` 为 `FixedString(N)` 数据类型。 [#9091](https://github.com/ClickHouse/ClickHouse/pull/9091) ([纪尧姆\*塔瑟里](https://github.com/YiuRULE)) +- 添加功能 `bitCount`. 这修复 [#8702](https://github.com/ClickHouse/ClickHouse/issues/8702). [#8708](https://github.com/ClickHouse/ClickHouse/pull/8708) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) [#8749](https://github.com/ClickHouse/ClickHouse/pull/8749) ([ikopylov](https://github.com/ikopylov)) +- 添加 `generateRandom` 表函数生成具有给定模式的随机行。 允许用数据填充任意测试表。 [#8994](https://github.com/ClickHouse/ClickHouse/pull/8994) ([Ilya Yatsishin](https://github.com/qoega)) +- `JSONEachRowFormat`:当对象包含在顶层数组中时,支持特殊情况。 [#8860](https://github.com/ClickHouse/ClickHouse/pull/8860) ([克鲁格洛夫\*帕维尔](https://github.com/Avogar)) +- 现在可以创建一个列 `DEFAULT` 取决于默认列的表达式 `ALIAS` 表达。 [#9489](https://github.com/ClickHouse/ClickHouse/pull/9489) ([阿利沙平](https://github.com/alesapin)) +- 允许指定 `--limit` 超过源数据大小 `clickhouse-obfuscator`. 数据将以不同的随机种子重复。 [#9155](https://github.com/ClickHouse/ClickHouse/pull/9155) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 已添加 `groupArraySample` 功能(类似于 `groupArray`)与reservior采样算法。 [#8286](https://github.com/ClickHouse/ClickHouse/pull/8286) ([阿莫斯鸟](https://github.com/amosbird)) +- 现在,您可以监视更新队列的大小 `cache`/`complex_key_cache` 通过系统指标字典。 [#9413](https://github.com/ClickHouse/ClickHouse/pull/9413) ([尼基塔\*米哈伊洛夫](https://github.com/nikitamikhaylov)) +- 允许使用CRLF作为CSV输出格式的行分隔符与设置 `output_format_csv_crlf_end_of_line` 设置为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) ([米哈伊尔\*科罗托夫](https://github.com/millb)) +- 实现的更多功能 [H3](https://github.com/uber/h3) API: `h3GetBaseCell`, `h3HexAreaM2`, `h3IndexesAreNeighbors`, `h3ToChildren`, `h3ToString` 和 `stringToH3` [#8938](https://github.com/ClickHouse/ClickHouse/pull/8938) ([Nico Mandery](https://github.com/nmandery)) +- 引入新设置: `max_parser_depth` 控制最大堆栈大小并允许大型复杂查询。 这修复 [#6681](https://github.com/ClickHouse/ClickHouse/issues/6681) 和 [#7668](https://github.com/ClickHouse/ClickHouse/issues/7668). [#8647](https://github.com/ClickHouse/ClickHouse/pull/8647) ([马克西姆\*斯米尔诺夫](https://github.com/qMBQx8GH)) +- 添加设置 `force_optimize_skip_unused_shards` 如果无法跳过未使用的分片,则设置为抛出 [#8805](https://github.com/ClickHouse/ClickHouse/pull/8805) ([Azat Khuzhin](https://github.com/azat)) +- 允许配置多个磁盘/卷用于存储数据发送 `Distributed` 发动机 [#8756](https://github.com/ClickHouse/ClickHouse/pull/8756) ([Azat Khuzhin](https://github.com/azat)) +- 支持存储策略 (``)用于存储临时数据。 [#8750](https://github.com/ClickHouse/ClickHouse/pull/8750) ([Azat Khuzhin](https://github.com/azat)) +- 已添加 `X-ClickHouse-Exception-Code` 如果在发送数据之前引发异常,则设置的HTTP头。 这实现了 [#4971](https://github.com/ClickHouse/ClickHouse/issues/4971). [#8786](https://github.com/ClickHouse/ClickHouse/pull/8786) ([米哈伊尔\*科罗托夫](https://github.com/millb)) +- 添加功能 `ifNotFinite`. 这只是一个句法糖: `ifNotFinite(x, y) = isFinite(x) ? x : y`. [#8710](https://github.com/ClickHouse/ClickHouse/pull/8710) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 已添加 `last_successful_update_time` 列中 `system.dictionaries` 表 [#9394](https://github.com/ClickHouse/ClickHouse/pull/9394) ([尼基塔\*米哈伊洛夫](https://github.com/nikitamikhaylov)) +- 添加 `blockSerializedSize` 功能(磁盘大小不压缩) [#8952](https://github.com/ClickHouse/ClickHouse/pull/8952) ([Azat Khuzhin](https://github.com/azat)) +- 添加功能 `moduloOrZero` [#9358](https://github.com/ClickHouse/ClickHouse/pull/9358) ([hcz](https://github.com/hczhcz)) +- 添加系统表 `system.zeros` 和 `system.zeros_mt` 以及故事功能 `zeros()` 和 `zeros_mt()`. 表(和表函数)包含具有名称的单列 `zero` 和类型 `UInt8`. 此列包含零。 为了测试目的,需要它作为生成许多行的最快方法。 这修复 [#6604](https://github.com/ClickHouse/ClickHouse/issues/6604) [#9593](https://github.com/ClickHouse/ClickHouse/pull/9593) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) #### 实验特点 {#experimental-feature} -- 添加新的紧凑格式的部件 `MergeTree`-家庭表中的所有列都存储在一个文件中。 它有助于提高小型和频繁插入的性能。 旧的格式(每列一个文件)现在被称为wide。 数据存储格式由设置控制 `min_bytes_for_wide_part` 和 `min_rows_for_wide_part`. [\#8290](https://github.com/ClickHouse/ClickHouse/pull/8290) ([安东\*波波夫](https://github.com/CurtizJ)) -- 支持S3存储 `Log`, `TinyLog` 和 `StripeLog` 桌子 [\#8862](https://github.com/ClickHouse/ClickHouse/pull/8862) ([帕维尔\*科瓦连科](https://github.com/Jokser)) +- 添加新的紧凑格式的部件 `MergeTree`-家庭表中的所有列都存储在一个文件中。 它有助于提高小型和频繁插入的性能。 旧的格式(每列一个文件)现在被称为wide。 数据存储格式由设置控制 `min_bytes_for_wide_part` 和 `min_rows_for_wide_part`. [#8290](https://github.com/ClickHouse/ClickHouse/pull/8290) ([安东\*波波夫](https://github.com/CurtizJ)) +- 支持S3存储 `Log`, `TinyLog` 和 `StripeLog` 桌子 [#8862](https://github.com/ClickHouse/ClickHouse/pull/8862) ([帕维尔\*科瓦连科](https://github.com/Jokser)) #### 错误修复 {#bug-fix-2} -- 修正了日志消息中不一致的空格。 [\#9322](https://github.com/ClickHouse/ClickHouse/pull/9322) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修复在创建表时将未命名元组数组展平为嵌套结构的错误。 [\#8866](https://github.com/ClickHouse/ClickHouse/pull/8866) ([achulkov2](https://github.com/achulkov2)) -- 修复了以下问题 “Too many open files” 如果有太多的文件匹配glob模式可能会发生错误 `File` 表或 `file` 表功能。 现在文件懒洋洋地打开。 这修复 [\#8857](https://github.com/ClickHouse/ClickHouse/issues/8857) [\#8861](https://github.com/ClickHouse/ClickHouse/pull/8861) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 删除临时表现在只删除临时表。 [\#8907](https://github.com/ClickHouse/ClickHouse/pull/8907) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) -- 当我们关闭服务器或分离/附加表时删除过时的分区。 [\#8602](https://github.com/ClickHouse/ClickHouse/pull/8602) ([纪尧姆\*塔瑟里](https://github.com/YiuRULE)) -- 默认磁盘如何计算可用空间 `data` 子目录。 修复了可用空间量计算不正确的问题,如果 `data` 目录被安装到一个单独的设备(罕见的情况)。 这修复 [\#7441](https://github.com/ClickHouse/ClickHouse/issues/7441) [\#9257](https://github.com/ClickHouse/ClickHouse/pull/9257) ([米哈伊尔\*科罗托夫](https://github.com/millb)) -- 允许逗号(交叉)与IN()内部连接。 [\#9251](https://github.com/ClickHouse/ClickHouse/pull/9251) ([Artem Zuikov](https://github.com/4ertus2)) -- 如果在WHERE部分中有\[NOT\]LIKE运算符,则允许将CROSS重写为INNER JOIN。 [\#9229](https://github.com/ClickHouse/ClickHouse/pull/9229) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复后可能不正确的结果 `GROUP BY` 启用设置 `distributed_aggregation_memory_efficient`. 修复 [\#9134](https://github.com/ClickHouse/ClickHouse/issues/9134). [\#9289](https://github.com/ClickHouse/ClickHouse/pull/9289) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 找到的键在缓存字典的指标中被计为错过。 [\#9411](https://github.com/ClickHouse/ClickHouse/pull/9411) ([尼基塔\*米哈伊洛夫](https://github.com/nikitamikhaylov)) -- 修复引入的复制协议不兼容 [\#8598](https://github.com/ClickHouse/ClickHouse/issues/8598). [\#9412](https://github.com/ClickHouse/ClickHouse/pull/9412) ([阿利沙平](https://github.com/alesapin)) -- 在固定的竞争条件 `queue_task_handle` 在启动 `ReplicatedMergeTree` 桌子 [\#9552](https://github.com/ClickHouse/ClickHouse/pull/9552) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 令牌 `NOT` 没有工作 `SHOW TABLES NOT LIKE` 查询 [\#8727](https://github.com/ClickHouse/ClickHouse/issues/8727) [\#8940](https://github.com/ClickHouse/ClickHouse/pull/8940) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 添加范围检查功能 `h3EdgeLengthM`. 如果没有这个检查,缓冲区溢出是可能的。 [\#8945](https://github.com/ClickHouse/ClickHouse/pull/8945) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修复了多个参数(超过10)的三元逻辑运算批量计算中的错误。 [\#8718](https://github.com/ClickHouse/ClickHouse/pull/8718) ([亚历山大\*卡扎科夫](https://github.com/Akazz)) -- 修复PREWHERE优化的错误,这可能导致段错误或 `Inconsistent number of columns got from MergeTreeRangeReader` 例外。 [\#9024](https://github.com/ClickHouse/ClickHouse/pull/9024) ([安东\*波波夫](https://github.com/CurtizJ)) -- 修复意外 `Timeout exceeded while reading from socket` 异常,在实际超时之前以及启用查询探查器时,在安全连接上随机发生。 还添加 `connect_timeout_with_failover_secure_ms` 设置(默认100ms),这是类似于 `connect_timeout_with_failover_ms`,但用于安全连接(因为SSL握手比普通TCP连接慢) [\#9026](https://github.com/ClickHouse/ClickHouse/pull/9026) ([tavplubix](https://github.com/tavplubix)) -- 修复突变最终确定的错误,当突变可能处于以下状态时 `parts_to_do=0` 和 `is_done=0`. [\#9022](https://github.com/ClickHouse/ClickHouse/pull/9022) ([阿利沙平](https://github.com/alesapin)) -- 使用新的任何连接逻辑 `partial_merge_join` 设置。 有可能使 `ANY|ALL|SEMI LEFT` 和 `ALL INNER` 加入与 `partial_merge_join=1` 现在 [\#8932](https://github.com/ClickHouse/ClickHouse/pull/8932) ([Artem Zuikov](https://github.com/4ertus2)) -- Shard现在将从发起者获得的设置夹到shard的constaints,而不是抛出异常。 此修补程序允许将查询发送到具有另一个约束的分片。 [\#9447](https://github.com/ClickHouse/ClickHouse/pull/9447) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) -- 修正了内存管理问题 `MergeTreeReadPool`. [\#8791](https://github.com/ClickHouse/ClickHouse/pull/8791) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 修复 `toDecimal*OrNull()` 使用字符串调用时的函数系列 `e`. 修复 [\#8312](https://github.com/ClickHouse/ClickHouse/issues/8312) [\#8764](https://github.com/ClickHouse/ClickHouse/pull/8764) ([Artem Zuikov](https://github.com/4ertus2)) -- 请确保 `FORMAT Null` 不向客户端发送数据。 [\#8767](https://github.com/ClickHouse/ClickHouse/pull/8767) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- 修复时间戳中的错误 `LiveViewBlockInputStream` 不会更新。 `LIVE VIEW` 是一个实验特征。 [\#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)) -- 固定 `ALTER MODIFY TTL` 不允许删除旧ttl表达式的错误行为。 [\#8422](https://github.com/ClickHouse/ClickHouse/pull/8422) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 修复了MergeTreeIndexSet中的UBSan报告。 这修复 [\#9250](https://github.com/ClickHouse/ClickHouse/issues/9250) [\#9365](https://github.com/ClickHouse/ClickHouse/pull/9365) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 固定的行为 `match` 和 `extract` 当干草堆有零字节的函数。 当干草堆不变时,这种行为是错误的。 这修复 [\#9160](https://github.com/ClickHouse/ClickHouse/issues/9160) [\#9163](https://github.com/ClickHouse/ClickHouse/pull/9163) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) [\#9345](https://github.com/ClickHouse/ClickHouse/pull/9345) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 避免从apache Avro第三方库中的析构函数抛出。 [\#9066](https://github.com/ClickHouse/ClickHouse/pull/9066) ([安德鲁Onyshchuk](https://github.com/oandrew)) -- 不要提交从轮询的批次 `Kafka` 部分,因为它可能会导致数据漏洞。 [\#8876](https://github.com/ClickHouse/ClickHouse/pull/8876) ([filimonov](https://github.com/filimonov)) -- 修复 `joinGet` 使用可为空的返回类型。 https://github.com/ClickHouse/ClickHouse/issues/8919 [\#9014](https://github.com/ClickHouse/ClickHouse/pull/9014) ([阿莫斯鸟](https://github.com/amosbird)) -- 修复压缩时的数据不兼容 `T64` 编解ec [\#9016](https://github.com/ClickHouse/ClickHouse/pull/9016) ([Artem Zuikov](https://github.com/4ertus2))修复数据类型id `T64` 在受影响的版本中导致错误(de)压缩的压缩编解ec。 [\#9033](https://github.com/ClickHouse/ClickHouse/pull/9033) ([Artem Zuikov](https://github.com/4ertus2)) -- 添加设置 `enable_early_constant_folding` 并禁用它在某些情况下,导致错误。 [\#9010](https://github.com/ClickHouse/ClickHouse/pull/9010) ([Artem Zuikov](https://github.com/4ertus2)) -- 使用VIEW修复下推谓词优化器并启用测试 [\#9011](https://github.com/ClickHouse/ClickHouse/pull/9011) ([张冬](https://github.com/zhang2014)) -- 修复段错误 `Merge` 表,从读取时可能发生 `File` 储存 [\#9387](https://github.com/ClickHouse/ClickHouse/pull/9387) ([tavplubix](https://github.com/tavplubix)) -- 添加了对存储策略的检查 `ATTACH PARTITION FROM`, `REPLACE PARTITION`, `MOVE TO TABLE`. 否则,它可以使部分数据重新启动后无法访问,并阻止ClickHouse启动。 [\#9383](https://github.com/ClickHouse/ClickHouse/pull/9383) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 修复改变,如果有TTL设置表。 [\#8800](https://github.com/ClickHouse/ClickHouse/pull/8800) ([安东\*波波夫](https://github.com/CurtizJ)) -- 修复在以下情况下可能发生的竞争条件 `SYSTEM RELOAD ALL DICTIONARIES` 在某些字典被修改/添加/删除时执行。 [\#8801](https://github.com/ClickHouse/ClickHouse/pull/8801) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) -- 在以前的版本 `Memory` 数据库引擎使用空数据路径,因此在以下位置创建表 `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)) -- 修复了关于缺少默认磁盘或策略的错误日志消息。 [\#9530](https://github.com/ClickHouse/ClickHouse/pull/9530) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 修复数组类型的bloom\_filter索引的not(has())。 [\#9407](https://github.com/ClickHouse/ClickHouse/pull/9407) ([achimbab](https://github.com/achimbab)) -- 允许表中的第一列 `Log` 引擎是别名 [\#9231](https://github.com/ClickHouse/ClickHouse/pull/9231) ([伊万](https://github.com/abyss7)) -- 从读取时修复范围的顺序 `MergeTree` 表中的一个线程。 它可能会导致例外 `MergeTreeRangeReader` 或错误的查询结果。 [\#9050](https://github.com/ClickHouse/ClickHouse/pull/9050) ([安东\*波波夫](https://github.com/CurtizJ)) -- 赂眉露\>\> `reinterpretAsFixedString` 返回 `FixedString` 而不是 `String`. [\#9052](https://github.com/ClickHouse/ClickHouse/pull/9052) ([安德鲁Onyshchuk](https://github.com/oandrew)) -- 避免极少数情况下,当用户可以得到错误的错误消息 (`Success` 而不是详细的错误描述)。 [\#9457](https://github.com/ClickHouse/ClickHouse/pull/9457) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 使用时不要崩溃 `Template` 使用空行模板格式化。 [\#8785](https://github.com/ClickHouse/ClickHouse/pull/8785) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- 系统表的元数据文件可能在错误的位置创建 [\#8653](https://github.com/ClickHouse/ClickHouse/pull/8653) ([tavplubix](https://github.com/tavplubix))修复 [\#8581](https://github.com/ClickHouse/ClickHouse/issues/8581). -- 修复缓存字典中exception\_ptr上的数据竞赛 [\#8303](https://github.com/ClickHouse/ClickHouse/issues/8303). [\#9379](https://github.com/ClickHouse/ClickHouse/pull/9379) ([尼基塔\*米哈伊洛夫](https://github.com/nikitamikhaylov)) -- 不要为查询引发异常 `ATTACH TABLE IF NOT EXISTS`. 以前它是抛出,如果表已经存在,尽管 `IF NOT EXISTS` 条款 [\#8967](https://github.com/ClickHouse/ClickHouse/pull/8967) ([安东\*波波夫](https://github.com/CurtizJ)) -- 修复了异常消息中丢失的关闭paren。 [\#8811](https://github.com/ClickHouse/ClickHouse/pull/8811) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 避免消息 `Possible deadlock avoided` 在clickhouse客户端在交互模式下启动。 [\#9455](https://github.com/ClickHouse/ClickHouse/pull/9455) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修复了base64编码值末尾填充格式错误的问题。 更新base64库。 这修复 [\#9491](https://github.com/ClickHouse/ClickHouse/issues/9491),关闭 [\#9492](https://github.com/ClickHouse/ClickHouse/issues/9492) [\#9500](https://github.com/ClickHouse/ClickHouse/pull/9500) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 防止丢失数据 `Kafka` 在极少数情况下,在读取后缀之后但在提交之前发生异常。 修复 [\#9378](https://github.com/ClickHouse/ClickHouse/issues/9378) [\#9507](https://github.com/ClickHouse/ClickHouse/pull/9507) ([filimonov](https://github.com/filimonov)) -- 在固定的异常 `DROP TABLE IF EXISTS` [\#8663](https://github.com/ClickHouse/ClickHouse/pull/8663) ([尼基塔\*瓦西列夫](https://github.com/nikvas0)) -- 修复当用户尝试崩溃 `ALTER MODIFY SETTING` 对于老格式化 `MergeTree` 表引擎家族. [\#9435](https://github.com/ClickHouse/ClickHouse/pull/9435) ([阿利沙平](https://github.com/alesapin)) -- 支持在JSON相关函数中不适合Int64的UInt64号码。 更新SIMDJSON掌握。 这修复 [\#9209](https://github.com/ClickHouse/ClickHouse/issues/9209) [\#9344](https://github.com/ClickHouse/ClickHouse/pull/9344) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 当使用非严格单调函数索引时,固定执行反转谓词。 [\#9223](https://github.com/ClickHouse/ClickHouse/pull/9223) ([亚历山大\*卡扎科夫](https://github.com/Akazz)) -- 不要试图折叠 `IN` 常量在 `GROUP BY` [\#8868](https://github.com/ClickHouse/ClickHouse/pull/8868) ([阿莫斯鸟](https://github.com/amosbird)) -- 修复bug `ALTER DELETE` 突变导致索引损坏。 这修复 [\#9019](https://github.com/ClickHouse/ClickHouse/issues/9019) 和 [\#8982](https://github.com/ClickHouse/ClickHouse/issues/8982). 另外修复极其罕见的竞争条件 `ReplicatedMergeTree` `ALTER` 查询。 [\#9048](https://github.com/ClickHouse/ClickHouse/pull/9048) ([阿利沙平](https://github.com/alesapin)) -- 当设置 `compile_expressions` 被启用,你可以得到 `unexpected column` 在 `LLVMExecutableFunction` 当我们使用 `Nullable` 类型 [\#8910](https://github.com/ClickHouse/ClickHouse/pull/8910) ([纪尧姆\*塔瑟里](https://github.com/YiuRULE)) -- 多个修复 `Kafka` 引擎:1)修复在消费者组重新平衡期间出现的重复项。 2)修复罕见 ‘holes’ 当数据从一个轮询的几个分区轮询并部分提交时出现(现在我们总是处理/提交整个轮询的消息块)。 3)通过块大小修复刷新(在此之前,只有超时刷新才能正常工作)。 4)更好的订阅程序(与分配反馈)。 5)使测试工作得更快(默认时间间隔和超时)。 由于数据之前没有被块大小刷新(根据文档),pr可能会导致默认设置的性能下降(由于更频繁和更小的刷新不太理想)。 如果您在更改后遇到性能问题-请增加 `kafka_max_block_size` 在表中的更大的值(例如 `CREATE TABLE ...Engine=Kafka ... SETTINGS ... kafka_max_block_size=524288`). 修复 [\#7259](https://github.com/ClickHouse/ClickHouse/issues/7259) [\#8917](https://github.com/ClickHouse/ClickHouse/pull/8917) ([filimonov](https://github.com/filimonov)) -- 修复 `Parameter out of bound` 在PREWHERE优化之后的某些查询中出现异常。 [\#8914](https://github.com/ClickHouse/ClickHouse/pull/8914) ([Baudouin Giard](https://github.com/bgiard)) -- 修正了函数参数混合常量的情况 `arrayZip`. [\#8705](https://github.com/ClickHouse/ClickHouse/pull/8705) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 执行时 `CREATE` 查询,在存储引擎参数中折叠常量表达式。 将空数据库名称替换为当前数据库。 修复 [\#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)) -- 现在不可能创建或添加具有简单循环别名的列,如 `a DEFAULT b, b DEFAULT a`. [\#9603](https://github.com/ClickHouse/ClickHouse/pull/9603) ([阿利沙平](https://github.com/alesapin)) -- 修正了双重移动可能会损坏原始部分的错误。 这是相关的,如果你使用 `ALTER TABLE MOVE` [\#8680](https://github.com/ClickHouse/ClickHouse/pull/8680) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 允许 `interval` 用于正确解析的标识符,而无需反引号。 当一个查询不能被执行,即使固定的问题 `interval` 标识符用反引号或双引号括起来。 这修复 [\#9124](https://github.com/ClickHouse/ClickHouse/issues/9124). [\#9142](https://github.com/ClickHouse/ClickHouse/pull/9142) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修正了模糊测试和不正确的行为 `bitTestAll`/`bitTestAny` 功能。 [\#9143](https://github.com/ClickHouse/ClickHouse/pull/9143) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修复可能的崩溃/错误的行数 `LIMIT n WITH TIES` 当有很多行等于第n行时。 [\#9464](https://github.com/ClickHouse/ClickHouse/pull/9464) ([tavplubix](https://github.com/tavplubix)) -- 使用enabled编写的部件修复突变 `insert_quorum`. [\#9463](https://github.com/ClickHouse/ClickHouse/pull/9463) ([阿利沙平](https://github.com/alesapin)) -- 修复数据竞赛破坏 `Poco::HTTPServer`. 当服务器启动并立即关闭时,可能会发生这种情况。 [\#9468](https://github.com/ClickHouse/ClickHouse/pull/9468) ([安东\*波波夫](https://github.com/CurtizJ)) -- 修复运行时显示误导性错误消息的错误 `SHOW CREATE TABLE a_table_that_does_not_exist`. [\#8899](https://github.com/ClickHouse/ClickHouse/pull/8899) ([achulkov2](https://github.com/achulkov2)) -- 固定 `Parameters are out of bound` 例外在一些罕见的情况下,当我们在一个常数 `SELECT` 条款时,我们有一个 `ORDER BY` 和一个 `LIMIT` 条款 [\#8892](https://github.com/ClickHouse/ClickHouse/pull/8892) ([纪尧姆\*塔瑟里](https://github.com/YiuRULE)) -- 修复突变定稿,当已经完成突变可以有状态 `is_done=0`. [\#9217](https://github.com/ClickHouse/ClickHouse/pull/9217) ([阿利沙平](https://github.com/alesapin)) -- 防止执行 `ALTER ADD INDEX` 对于旧语法的MergeTree表,因为它不起作用。 [\#8822](https://github.com/ClickHouse/ClickHouse/pull/8822) ([米哈伊尔\*科罗托夫](https://github.com/millb)) -- 在服务器启动时不要访问表,这 `LIVE VIEW` 取决于,所以服务器将能够启动。 也删除 `LIVE VIEW` 分离时的依赖关系 `LIVE VIEW`. `LIVE VIEW` 是一个实验特征。 [\#8824](https://github.com/ClickHouse/ClickHouse/pull/8824) ([tavplubix](https://github.com/tavplubix)) -- 修复可能的段错误 `MergeTreeRangeReader`,同时执行 `PREWHERE`. [\#9106](https://github.com/ClickHouse/ClickHouse/pull/9106) ([安东\*波波夫](https://github.com/CurtizJ)) -- 修复与列Ttl可能不匹配的校验和。 [\#9451](https://github.com/ClickHouse/ClickHouse/pull/9451) ([安东\*波波夫](https://github.com/CurtizJ)) -- 修正了一个错误,当部分没有被移动的情况下,只有一个卷的TTL规则在后台。 [\#8672](https://github.com/ClickHouse/ClickHouse/pull/8672) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 修正了这个问题 `Method createColumn() is not implemented for data type Set`. 这修复 [\#7799](https://github.com/ClickHouse/ClickHouse/issues/7799). [\#8674](https://github.com/ClickHouse/ClickHouse/pull/8674) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 现在我们将尝试更频繁地完成突变。 [\#9427](https://github.com/ClickHouse/ClickHouse/pull/9427) ([阿利沙平](https://github.com/alesapin)) -- 修复 `intDiv` 减一个常数 [\#9351](https://github.com/ClickHouse/ClickHouse/pull/9351) ([hcz](https://github.com/hczhcz)) -- 修复可能的竞争条件 `BlockIO`. [\#9356](https://github.com/ClickHouse/ClickHouse/pull/9356) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 修复尝试使用/删除时导致服务器终止的错误 `Kafka` 使用错误的参数创建的表。 [\#9513](https://github.com/ClickHouse/ClickHouse/pull/9513) ([filimonov](https://github.com/filimonov)) -- 增加了解决方法,如果操作系统返回错误的结果 `timer_create` 功能。 [\#8837](https://github.com/ClickHouse/ClickHouse/pull/8837) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 在使用固定错误 `min_marks_for_seek` 参数。 修复了分布式表中没有分片键时的错误消息,并且我们尝试跳过未使用的分片。 [\#8908](https://github.com/ClickHouse/ClickHouse/pull/8908) ([Azat Khuzhin](https://github.com/azat)) +- 修正了日志消息中不一致的空格。 [#9322](https://github.com/ClickHouse/ClickHouse/pull/9322) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修复在创建表时将未命名元组数组展平为嵌套结构的错误。 [#8866](https://github.com/ClickHouse/ClickHouse/pull/8866) ([achulkov2](https://github.com/achulkov2)) +- 修复了以下问题 “Too many open files” 如果有太多的文件匹配glob模式可能会发生错误 `File` 表或 `file` 表功能。 现在文件懒洋洋地打开。 这修复 [#8857](https://github.com/ClickHouse/ClickHouse/issues/8857) [#8861](https://github.com/ClickHouse/ClickHouse/pull/8861) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 删除临时表现在只删除临时表。 [#8907](https://github.com/ClickHouse/ClickHouse/pull/8907) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) +- 当我们关闭服务器或分离/附加表时删除过时的分区。 [#8602](https://github.com/ClickHouse/ClickHouse/pull/8602) ([纪尧姆\*塔瑟里](https://github.com/YiuRULE)) +- 默认磁盘如何计算可用空间 `data` 子目录。 修复了可用空间量计算不正确的问题,如果 `data` 目录被安装到一个单独的设备(罕见的情况)。 这修复 [#7441](https://github.com/ClickHouse/ClickHouse/issues/7441) [#9257](https://github.com/ClickHouse/ClickHouse/pull/9257) ([米哈伊尔\*科罗托夫](https://github.com/millb)) +- 允许逗号(交叉)与IN()内部连接。 [#9251](https://github.com/ClickHouse/ClickHouse/pull/9251) ([Artem Zuikov](https://github.com/4ertus2)) +- 如果在WHERE部分中有\[NOT\]LIKE运算符,则允许将CROSS重写为INNER JOIN。 [#9229](https://github.com/ClickHouse/ClickHouse/pull/9229) ([Artem Zuikov](https://github.com/4ertus2)) +- 修复后可能不正确的结果 `GROUP BY` 启用设置 `distributed_aggregation_memory_efficient`. 修复 [#9134](https://github.com/ClickHouse/ClickHouse/issues/9134). [#9289](https://github.com/ClickHouse/ClickHouse/pull/9289) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 找到的键在缓存字典的指标中被计为错过。 [#9411](https://github.com/ClickHouse/ClickHouse/pull/9411) ([尼基塔\*米哈伊洛夫](https://github.com/nikitamikhaylov)) +- 修复引入的复制协议不兼容 [#8598](https://github.com/ClickHouse/ClickHouse/issues/8598). [#9412](https://github.com/ClickHouse/ClickHouse/pull/9412) ([阿利沙平](https://github.com/alesapin)) +- 在固定的竞争条件 `queue_task_handle` 在启动 `ReplicatedMergeTree` 桌子 [#9552](https://github.com/ClickHouse/ClickHouse/pull/9552) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 令牌 `NOT` 没有工作 `SHOW TABLES NOT LIKE` 查询 [#8727](https://github.com/ClickHouse/ClickHouse/issues/8727) [#8940](https://github.com/ClickHouse/ClickHouse/pull/8940) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 添加范围检查功能 `h3EdgeLengthM`. 如果没有这个检查,缓冲区溢出是可能的。 [#8945](https://github.com/ClickHouse/ClickHouse/pull/8945) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修复了多个参数(超过10)的三元逻辑运算批量计算中的错误。 [#8718](https://github.com/ClickHouse/ClickHouse/pull/8718) ([亚历山大\*卡扎科夫](https://github.com/Akazz)) +- 修复PREWHERE优化的错误,这可能导致段错误或 `Inconsistent number of columns got from MergeTreeRangeReader` 例外。 [#9024](https://github.com/ClickHouse/ClickHouse/pull/9024) ([安东\*波波夫](https://github.com/CurtizJ)) +- 修复意外 `Timeout exceeded while reading from socket` 异常,在实际超时之前以及启用查询探查器时,在安全连接上随机发生。 还添加 `connect_timeout_with_failover_secure_ms` 设置(默认100ms),这是类似于 `connect_timeout_with_failover_ms`,但用于安全连接(因为SSL握手比普通TCP连接慢) [#9026](https://github.com/ClickHouse/ClickHouse/pull/9026) ([tavplubix](https://github.com/tavplubix)) +- 修复突变最终确定的错误,当突变可能处于以下状态时 `parts_to_do=0` 和 `is_done=0`. [#9022](https://github.com/ClickHouse/ClickHouse/pull/9022) ([阿利沙平](https://github.com/alesapin)) +- 使用新的任何连接逻辑 `partial_merge_join` 设置。 有可能使 `ANY|ALL|SEMI LEFT` 和 `ALL INNER` 加入与 `partial_merge_join=1` 现在 [#8932](https://github.com/ClickHouse/ClickHouse/pull/8932) ([Artem Zuikov](https://github.com/4ertus2)) +- Shard现在将从发起者获得的设置夹到shard的constaints,而不是抛出异常。 此修补程序允许将查询发送到具有另一个约束的分片。 [#9447](https://github.com/ClickHouse/ClickHouse/pull/9447) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) +- 修正了内存管理问题 `MergeTreeReadPool`. [#8791](https://github.com/ClickHouse/ClickHouse/pull/8791) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 修复 `toDecimal*OrNull()` 使用字符串调用时的函数系列 `e`. 修复 [#8312](https://github.com/ClickHouse/ClickHouse/issues/8312) [#8764](https://github.com/ClickHouse/ClickHouse/pull/8764) ([Artem Zuikov](https://github.com/4ertus2)) +- 请确保 `FORMAT Null` 不向客户端发送数据。 [#8767](https://github.com/ClickHouse/ClickHouse/pull/8767) ([Alexander Kuzmenkov](https://github.com/akuzm)) +- 修复时间戳中的错误 `LiveViewBlockInputStream` 不会更新。 `LIVE VIEW` 是一个实验特征。 [#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)) +- 固定 `ALTER MODIFY TTL` 不允许删除旧ttl表达式的错误行为。 [#8422](https://github.com/ClickHouse/ClickHouse/pull/8422) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 修复了MergeTreeIndexSet中的UBSan报告。 这修复 [#9250](https://github.com/ClickHouse/ClickHouse/issues/9250) [#9365](https://github.com/ClickHouse/ClickHouse/pull/9365) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 固定的行为 `match` 和 `extract` 当干草堆有零字节的函数。 当干草堆不变时,这种行为是错误的。 这修复 [#9160](https://github.com/ClickHouse/ClickHouse/issues/9160) [#9163](https://github.com/ClickHouse/ClickHouse/pull/9163) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) [#9345](https://github.com/ClickHouse/ClickHouse/pull/9345) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 避免从apache Avro第三方库中的析构函数抛出。 [#9066](https://github.com/ClickHouse/ClickHouse/pull/9066) ([安德鲁Onyshchuk](https://github.com/oandrew)) +- 不要提交从轮询的批次 `Kafka` 部分,因为它可能会导致数据漏洞。 [#8876](https://github.com/ClickHouse/ClickHouse/pull/8876) ([filimonov](https://github.com/filimonov)) +- 修复 `joinGet` 使用可为空的返回类型。 https://github.com/ClickHouse/ClickHouse/issues/8919 [#9014](https://github.com/ClickHouse/ClickHouse/pull/9014) ([阿莫斯鸟](https://github.com/amosbird)) +- 修复压缩时的数据不兼容 `T64` 编解ec [#9016](https://github.com/ClickHouse/ClickHouse/pull/9016) ([Artem Zuikov](https://github.com/4ertus2))修复数据类型id `T64` 在受影响的版本中导致错误(de)压缩的压缩编解ec。 [#9033](https://github.com/ClickHouse/ClickHouse/pull/9033) ([Artem Zuikov](https://github.com/4ertus2)) +- 添加设置 `enable_early_constant_folding` 并禁用它在某些情况下,导致错误。 [#9010](https://github.com/ClickHouse/ClickHouse/pull/9010) ([Artem Zuikov](https://github.com/4ertus2)) +- 使用VIEW修复下推谓词优化器并启用测试 [#9011](https://github.com/ClickHouse/ClickHouse/pull/9011) ([张冬](https://github.com/zhang2014)) +- 修复段错误 `Merge` 表,从读取时可能发生 `File` 储存 [#9387](https://github.com/ClickHouse/ClickHouse/pull/9387) ([tavplubix](https://github.com/tavplubix)) +- 添加了对存储策略的检查 `ATTACH PARTITION FROM`, `REPLACE PARTITION`, `MOVE TO TABLE`. 否则,它可以使部分数据重新启动后无法访问,并阻止ClickHouse启动。 [#9383](https://github.com/ClickHouse/ClickHouse/pull/9383) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 修复改变,如果有TTL设置表。 [#8800](https://github.com/ClickHouse/ClickHouse/pull/8800) ([安东\*波波夫](https://github.com/CurtizJ)) +- 修复在以下情况下可能发生的竞争条件 `SYSTEM RELOAD ALL DICTIONARIES` 在某些字典被修改/添加/删除时执行。 [#8801](https://github.com/ClickHouse/ClickHouse/pull/8801) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) +- 在以前的版本 `Memory` 数据库引擎使用空数据路径,因此在以下位置创建表 `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)) +- 修复了关于缺少默认磁盘或策略的错误日志消息。 [#9530](https://github.com/ClickHouse/ClickHouse/pull/9530) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 修复数组类型的bloom_filter索引的not(has())。 [#9407](https://github.com/ClickHouse/ClickHouse/pull/9407) ([achimbab](https://github.com/achimbab)) +- 允许表中的第一列 `Log` 引擎是别名 [#9231](https://github.com/ClickHouse/ClickHouse/pull/9231) ([伊万](https://github.com/abyss7)) +- 从读取时修复范围的顺序 `MergeTree` 表中的一个线程。 它可能会导致例外 `MergeTreeRangeReader` 或错误的查询结果。 [#9050](https://github.com/ClickHouse/ClickHouse/pull/9050) ([安东\*波波夫](https://github.com/CurtizJ)) +- 赂眉露\>\> `reinterpretAsFixedString` 返回 `FixedString` 而不是 `String`. [#9052](https://github.com/ClickHouse/ClickHouse/pull/9052) ([安德鲁Onyshchuk](https://github.com/oandrew)) +- 避免极少数情况下,当用户可以得到错误的错误消息 (`Success` 而不是详细的错误描述)。 [#9457](https://github.com/ClickHouse/ClickHouse/pull/9457) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 使用时不要崩溃 `Template` 使用空行模板格式化。 [#8785](https://github.com/ClickHouse/ClickHouse/pull/8785) ([Alexander Kuzmenkov](https://github.com/akuzm)) +- 系统表的元数据文件可能在错误的位置创建 [#8653](https://github.com/ClickHouse/ClickHouse/pull/8653) ([tavplubix](https://github.com/tavplubix))修复 [#8581](https://github.com/ClickHouse/ClickHouse/issues/8581). +- 修复缓存字典中exception_ptr上的数据竞赛 [#8303](https://github.com/ClickHouse/ClickHouse/issues/8303). [#9379](https://github.com/ClickHouse/ClickHouse/pull/9379) ([尼基塔\*米哈伊洛夫](https://github.com/nikitamikhaylov)) +- 不要为查询引发异常 `ATTACH TABLE IF NOT EXISTS`. 以前它是抛出,如果表已经存在,尽管 `IF NOT EXISTS` 条款 [#8967](https://github.com/ClickHouse/ClickHouse/pull/8967) ([安东\*波波夫](https://github.com/CurtizJ)) +- 修复了异常消息中丢失的关闭paren。 [#8811](https://github.com/ClickHouse/ClickHouse/pull/8811) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 避免消息 `Possible deadlock avoided` 在clickhouse客户端在交互模式下启动。 [#9455](https://github.com/ClickHouse/ClickHouse/pull/9455) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修复了base64编码值末尾填充格式错误的问题。 更新base64库。 这修复 [#9491](https://github.com/ClickHouse/ClickHouse/issues/9491),关闭 [#9492](https://github.com/ClickHouse/ClickHouse/issues/9492) [#9500](https://github.com/ClickHouse/ClickHouse/pull/9500) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 防止丢失数据 `Kafka` 在极少数情况下,在读取后缀之后但在提交之前发生异常。 修复 [#9378](https://github.com/ClickHouse/ClickHouse/issues/9378) [#9507](https://github.com/ClickHouse/ClickHouse/pull/9507) ([filimonov](https://github.com/filimonov)) +- 在固定的异常 `DROP TABLE IF EXISTS` [#8663](https://github.com/ClickHouse/ClickHouse/pull/8663) ([尼基塔\*瓦西列夫](https://github.com/nikvas0)) +- 修复当用户尝试崩溃 `ALTER MODIFY SETTING` 对于老格式化 `MergeTree` 表引擎家族. [#9435](https://github.com/ClickHouse/ClickHouse/pull/9435) ([阿利沙平](https://github.com/alesapin)) +- 支持在JSON相关函数中不适合Int64的UInt64号码。 更新SIMDJSON掌握。 这修复 [#9209](https://github.com/ClickHouse/ClickHouse/issues/9209) [#9344](https://github.com/ClickHouse/ClickHouse/pull/9344) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 当使用非严格单调函数索引时,固定执行反转谓词。 [#9223](https://github.com/ClickHouse/ClickHouse/pull/9223) ([亚历山大\*卡扎科夫](https://github.com/Akazz)) +- 不要试图折叠 `IN` 常量在 `GROUP BY` [#8868](https://github.com/ClickHouse/ClickHouse/pull/8868) ([阿莫斯鸟](https://github.com/amosbird)) +- 修复bug `ALTER DELETE` 突变导致索引损坏。 这修复 [#9019](https://github.com/ClickHouse/ClickHouse/issues/9019) 和 [#8982](https://github.com/ClickHouse/ClickHouse/issues/8982). 另外修复极其罕见的竞争条件 `ReplicatedMergeTree` `ALTER` 查询。 [#9048](https://github.com/ClickHouse/ClickHouse/pull/9048) ([阿利沙平](https://github.com/alesapin)) +- 当设置 `compile_expressions` 被启用,你可以得到 `unexpected column` 在 `LLVMExecutableFunction` 当我们使用 `Nullable` 类型 [#8910](https://github.com/ClickHouse/ClickHouse/pull/8910) ([纪尧姆\*塔瑟里](https://github.com/YiuRULE)) +- 多个修复 `Kafka` 引擎:1)修复在消费者组重新平衡期间出现的重复项。 2)修复罕见 ‘holes’ 当数据从一个轮询的几个分区轮询并部分提交时出现(现在我们总是处理/提交整个轮询的消息块)。 3)通过块大小修复刷新(在此之前,只有超时刷新才能正常工作)。 4)更好的订阅程序(与分配反馈)。 5)使测试工作得更快(默认时间间隔和超时)。 由于数据之前没有被块大小刷新(根据文档),pr可能会导致默认设置的性能下降(由于更频繁和更小的刷新不太理想)。 如果您在更改后遇到性能问题-请增加 `kafka_max_block_size` 在表中的更大的值(例如 `CREATE TABLE ...Engine=Kafka ... SETTINGS ... kafka_max_block_size=524288`). 修复 [#7259](https://github.com/ClickHouse/ClickHouse/issues/7259) [#8917](https://github.com/ClickHouse/ClickHouse/pull/8917) ([filimonov](https://github.com/filimonov)) +- 修复 `Parameter out of bound` 在PREWHERE优化之后的某些查询中出现异常。 [#8914](https://github.com/ClickHouse/ClickHouse/pull/8914) ([Baudouin Giard](https://github.com/bgiard)) +- 修正了函数参数混合常量的情况 `arrayZip`. [#8705](https://github.com/ClickHouse/ClickHouse/pull/8705) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 执行时 `CREATE` 查询,在存储引擎参数中折叠常量表达式。 将空数据库名称替换为当前数据库。 修复 [#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)) +- 现在不可能创建或添加具有简单循环别名的列,如 `a DEFAULT b, b DEFAULT a`. [#9603](https://github.com/ClickHouse/ClickHouse/pull/9603) ([阿利沙平](https://github.com/alesapin)) +- 修正了双重移动可能会损坏原始部分的错误。 这是相关的,如果你使用 `ALTER TABLE MOVE` [#8680](https://github.com/ClickHouse/ClickHouse/pull/8680) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 允许 `interval` 用于正确解析的标识符,而无需反引号。 当一个查询不能被执行,即使固定的问题 `interval` 标识符用反引号或双引号括起来。 这修复 [#9124](https://github.com/ClickHouse/ClickHouse/issues/9124). [#9142](https://github.com/ClickHouse/ClickHouse/pull/9142) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修正了模糊测试和不正确的行为 `bitTestAll`/`bitTestAny` 功能。 [#9143](https://github.com/ClickHouse/ClickHouse/pull/9143) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修复可能的崩溃/错误的行数 `LIMIT n WITH TIES` 当有很多行等于第n行时。 [#9464](https://github.com/ClickHouse/ClickHouse/pull/9464) ([tavplubix](https://github.com/tavplubix)) +- 使用enabled编写的部件修复突变 `insert_quorum`. [#9463](https://github.com/ClickHouse/ClickHouse/pull/9463) ([阿利沙平](https://github.com/alesapin)) +- 修复数据竞赛破坏 `Poco::HTTPServer`. 当服务器启动并立即关闭时,可能会发生这种情况。 [#9468](https://github.com/ClickHouse/ClickHouse/pull/9468) ([安东\*波波夫](https://github.com/CurtizJ)) +- 修复运行时显示误导性错误消息的错误 `SHOW CREATE TABLE a_table_that_does_not_exist`. [#8899](https://github.com/ClickHouse/ClickHouse/pull/8899) ([achulkov2](https://github.com/achulkov2)) +- 固定 `Parameters are out of bound` 例外在一些罕见的情况下,当我们在一个常数 `SELECT` 条款时,我们有一个 `ORDER BY` 和一个 `LIMIT` 条款 [#8892](https://github.com/ClickHouse/ClickHouse/pull/8892) ([纪尧姆\*塔瑟里](https://github.com/YiuRULE)) +- 修复突变定稿,当已经完成突变可以有状态 `is_done=0`. [#9217](https://github.com/ClickHouse/ClickHouse/pull/9217) ([阿利沙平](https://github.com/alesapin)) +- 防止执行 `ALTER ADD INDEX` 对于旧语法的MergeTree表,因为它不起作用。 [#8822](https://github.com/ClickHouse/ClickHouse/pull/8822) ([米哈伊尔\*科罗托夫](https://github.com/millb)) +- 在服务器启动时不要访问表,这 `LIVE VIEW` 取决于,所以服务器将能够启动。 也删除 `LIVE VIEW` 分离时的依赖关系 `LIVE VIEW`. `LIVE VIEW` 是一个实验特征。 [#8824](https://github.com/ClickHouse/ClickHouse/pull/8824) ([tavplubix](https://github.com/tavplubix)) +- 修复可能的段错误 `MergeTreeRangeReader`,同时执行 `PREWHERE`. [#9106](https://github.com/ClickHouse/ClickHouse/pull/9106) ([安东\*波波夫](https://github.com/CurtizJ)) +- 修复与列Ttl可能不匹配的校验和。 [#9451](https://github.com/ClickHouse/ClickHouse/pull/9451) ([安东\*波波夫](https://github.com/CurtizJ)) +- 修正了一个错误,当部分没有被移动的情况下,只有一个卷的TTL规则在后台。 [#8672](https://github.com/ClickHouse/ClickHouse/pull/8672) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 修正了这个问题 `Method createColumn() is not implemented for data type Set`. 这修复 [#7799](https://github.com/ClickHouse/ClickHouse/issues/7799). [#8674](https://github.com/ClickHouse/ClickHouse/pull/8674) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 现在我们将尝试更频繁地完成突变。 [#9427](https://github.com/ClickHouse/ClickHouse/pull/9427) ([阿利沙平](https://github.com/alesapin)) +- 修复 `intDiv` 减一个常数 [#9351](https://github.com/ClickHouse/ClickHouse/pull/9351) ([hcz](https://github.com/hczhcz)) +- 修复可能的竞争条件 `BlockIO`. [#9356](https://github.com/ClickHouse/ClickHouse/pull/9356) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 修复尝试使用/删除时导致服务器终止的错误 `Kafka` 使用错误的参数创建的表。 [#9513](https://github.com/ClickHouse/ClickHouse/pull/9513) ([filimonov](https://github.com/filimonov)) +- 增加了解决方法,如果操作系统返回错误的结果 `timer_create` 功能。 [#8837](https://github.com/ClickHouse/ClickHouse/pull/8837) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 在使用固定错误 `min_marks_for_seek` 参数。 修复了分布式表中没有分片键时的错误消息,并且我们尝试跳过未使用的分片。 [#8908](https://github.com/ClickHouse/ClickHouse/pull/8908) ([Azat Khuzhin](https://github.com/azat)) #### 改进 {#improvement} -- 执行 `ALTER MODIFY/DROP` 对突变的顶部查询 `ReplicatedMergeTree*` 引擎家族. 现在 `ALTERS` 仅在元数据更新阶段阻止,之后不阻止。 [\#8701](https://github.com/ClickHouse/ClickHouse/pull/8701) ([阿利沙平](https://github.com/alesapin)) -- 添加重写交叉到内部连接的能力 `WHERE` 包含未编译名称的部分。 [\#9512](https://github.com/ClickHouse/ClickHouse/pull/9512) ([Artem Zuikov](https://github.com/4ertus2)) -- 赂眉露\>\> `SHOW TABLES` 和 `SHOW DATABASES` 查询支持 `WHERE` 表达式和 `FROM`/`IN` [\#9076](https://github.com/ClickHouse/ClickHouse/pull/9076) ([sundyli](https://github.com/sundy-li)) -- 添加了一个设置 `deduplicate_blocks_in_dependent_materialized_views`. [\#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) ([urykhy](https://github.com/urykhy)) -- 在最近的变化之后,MySQL客户端开始以十六进制打印二进制字符串,从而使它们不可读 ([\#9032](https://github.com/ClickHouse/ClickHouse/issues/9032)). ClickHouse中的解决方法是将字符串列标记为UTF-8,这并不总是如此,但通常是这种情况。 [\#9079](https://github.com/ClickHouse/ClickHouse/pull/9079) ([尤里\*巴拉诺夫](https://github.com/yurriy)) -- 添加对字符串和FixedString键的支持 `sumMap` [\#8903](https://github.com/ClickHouse/ClickHouse/pull/8903) ([Baudouin Giard](https://github.com/bgiard)) -- 支持SummingMergeTree地图中的字符串键 [\#8933](https://github.com/ClickHouse/ClickHouse/pull/8933) ([Baudouin Giard](https://github.com/bgiard)) -- 即使线程已抛出异常,也向线程池发送线程终止信号 [\#8736](https://github.com/ClickHouse/ClickHouse/pull/8736) ([丁香飞](https://github.com/dingxiangfei2009)) -- 允许设置 `query_id` 在 `clickhouse-benchmark` [\#9416](https://github.com/ClickHouse/ClickHouse/pull/9416) ([安东\*波波夫](https://github.com/CurtizJ)) -- 不要让奇怪的表达 `ALTER TABLE ... PARTITION partition` 查询。 这个地址 [\#7192](https://github.com/ClickHouse/ClickHouse/issues/7192) [\#8835](https://github.com/ClickHouse/ClickHouse/pull/8835) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 表 `system.table_engines` 现在提供有关功能支持的信息(如 `supports_ttl` 或 `supports_sort_order`). [\#8830](https://github.com/ClickHouse/ClickHouse/pull/8830) ([Max Akhmedov](https://github.com/zlobober)) -- 启用 `system.metric_log` 默认情况下。 它将包含具有ProfileEvents值的行,CurrentMetrics收集与 “collect\_interval\_milliseconds” 间隔(默认情况下为一秒)。 该表非常小(通常以兆字节为单位),默认情况下收集此数据是合理的。 [\#9225](https://github.com/ClickHouse/ClickHouse/pull/9225) ([阿列克谢-米洛维多夫](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) ([伊万](https://github.com/abyss7)) -- 现在是暂时的 `LIVE VIEW` 创建者 `CREATE LIVE VIEW name WITH TIMEOUT [42] ...` 而不是 `CREATE TEMPORARY LIVE VIEW ...`,因为以前的语法不符合 `CREATE TEMPORARY TABLE ...` [\#9131](https://github.com/ClickHouse/ClickHouse/pull/9131) ([tavplubix](https://github.com/tavplubix)) -- 添加text\_log。级别配置参数,以限制进入 `system.text_log` 表 [\#8809](https://github.com/ClickHouse/ClickHouse/pull/8809) ([Azat Khuzhin](https://github.com/azat)) -- 允许根据TTL规则将下载的部分放入磁盘/卷 [\#8598](https://github.com/ClickHouse/ClickHouse/pull/8598) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 对于外部MySQL字典,允许将MySQL连接池共同化为 “share” 他们在字典中。 此选项显着减少到MySQL服务器的连接数。 [\#9409](https://github.com/ClickHouse/ClickHouse/pull/9409) ([Clément Rodriguez](https://github.com/clemrodriguez)) -- 显示分位数的最近查询执行时间 `clickhouse-benchmark` 输出而不是插值值。 最好显示与某些查询的执行时间相对应的值。 [\#8712](https://github.com/ClickHouse/ClickHouse/pull/8712) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 可以在将数据插入到Kafka时为消息添加密钥和时间戳。 修复 [\#7198](https://github.com/ClickHouse/ClickHouse/issues/7198) [\#8969](https://github.com/ClickHouse/ClickHouse/pull/8969) ([filimonov](https://github.com/filimonov)) -- 如果服务器从终端运行,请按颜色突出显示线程号,查询id和日志优先级。 这是为了提高开发人员相关日志消息的可读性。 [\#8961](https://github.com/ClickHouse/ClickHouse/pull/8961) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 更好的异常消息,同时加载表 `Ordinary` 数据库。 [\#9527](https://github.com/ClickHouse/ClickHouse/pull/9527) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 执行 `arraySlice` 对于具有聚合函数状态的数组。 这修复 [\#9388](https://github.com/ClickHouse/ClickHouse/issues/9388) [\#9391](https://github.com/ClickHouse/ClickHouse/pull/9391) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 允许在in运算符的右侧使用常量函数和常量数组。 [\#8813](https://github.com/ClickHouse/ClickHouse/pull/8813) ([安东\*波波夫](https://github.com/CurtizJ)) -- 如果在获取系统数据时发生了zookeeper异常。副本,将其显示在单独的列中。 这实现了 [\#9137](https://github.com/ClickHouse/ClickHouse/issues/9137) [\#9138](https://github.com/ClickHouse/ClickHouse/pull/9138) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 原子删除destroy上的MergeTree数据部分。 [\#8402](https://github.com/ClickHouse/ClickHouse/pull/8402) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 支持分布式表的行级安全性。 [\#8926](https://github.com/ClickHouse/ClickHouse/pull/8926) ([伊万](https://github.com/abyss7)) -- Now we recognize suffix (like KB, KiB…) in settings values. [\#8072](https://github.com/ClickHouse/ClickHouse/pull/8072) ([米哈伊尔\*科罗托夫](https://github.com/millb)) -- 在构建大型连接的结果时防止内存不足。 [\#8637](https://github.com/ClickHouse/ClickHouse/pull/8637) ([Artem Zuikov](https://github.com/4ertus2)) -- 在交互模式下为建议添加群集名称 `clickhouse-client`. [\#8709](https://github.com/ClickHouse/ClickHouse/pull/8709) ([阿列克谢-米洛维多夫](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) ([伊万](https://github.com/abyss7)) -- 添加列 `exception_code` 在 `system.query_log` 桌子 [\#8770](https://github.com/ClickHouse/ClickHouse/pull/8770) ([米哈伊尔\*科罗托夫](https://github.com/millb)) -- 在端口上启用MySQL兼容服务器 `9004` 在默认服务器配置文件中。 在配置的例子固定密码生成命令。 [\#8771](https://github.com/ClickHouse/ClickHouse/pull/8771) ([尤里\*巴拉诺夫](https://github.com/yurriy)) -- 如果文件系统是只读的,请防止在关闭时中止。 这修复 [\#9094](https://github.com/ClickHouse/ClickHouse/issues/9094) [\#9100](https://github.com/ClickHouse/ClickHouse/pull/9100) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 当HTTP POST查询中需要长度时,更好的异常消息。 [\#9453](https://github.com/ClickHouse/ClickHouse/pull/9453) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 添加 `_path` 和 `_file` 虚拟列 `HDFS` 和 `File` 发动机和 `hdfs` 和 `file` 表函数 [\#8489](https://github.com/ClickHouse/ClickHouse/pull/8489) ([Olga Khvostikova](https://github.com/stavrolia)) -- 修复错误 `Cannot find column` 同时插入到 `MATERIALIZED VIEW` 在情况下,如果新列被添加到视图的内部表。 [\#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) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) [\#8803](https://github.com/ClickHouse/ClickHouse/pull/8803) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 通过最终更新后发送进度(如日志)修复本机客户端-服务器协议的进度。 这可能仅与使用本机协议的某些第三方工具相关。 [\#9495](https://github.com/ClickHouse/ClickHouse/pull/9495) ([Azat Khuzhin](https://github.com/azat)) -- 添加系统指标跟踪使用MySQL协议的客户端连接数 ([\#9013](https://github.com/ClickHouse/ClickHouse/issues/9013)). [\#9015](https://github.com/ClickHouse/ClickHouse/pull/9015) ([尤金\*克里莫夫](https://github.com/Slach)) -- 从现在开始,HTTP响应将有 `X-ClickHouse-Timezone` 标题设置为相同的时区值 `SELECT timezone()` 会报告。 [\#9493](https://github.com/ClickHouse/ClickHouse/pull/9493) ([Denis Glazachev](https://github.com/traceon)) +- 执行 `ALTER MODIFY/DROP` 对突变的顶部查询 `ReplicatedMergeTree*` 引擎家族. 现在 `ALTERS` 仅在元数据更新阶段阻止,之后不阻止。 [#8701](https://github.com/ClickHouse/ClickHouse/pull/8701) ([阿利沙平](https://github.com/alesapin)) +- 添加重写交叉到内部连接的能力 `WHERE` 包含未编译名称的部分。 [#9512](https://github.com/ClickHouse/ClickHouse/pull/9512) ([Artem Zuikov](https://github.com/4ertus2)) +- 赂眉露\>\> `SHOW TABLES` 和 `SHOW DATABASES` 查询支持 `WHERE` 表达式和 `FROM`/`IN` [#9076](https://github.com/ClickHouse/ClickHouse/pull/9076) ([sundyli](https://github.com/sundy-li)) +- 添加了一个设置 `deduplicate_blocks_in_dependent_materialized_views`. [#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) ([urykhy](https://github.com/urykhy)) +- 在最近的变化之后,MySQL客户端开始以十六进制打印二进制字符串,从而使它们不可读 ([#9032](https://github.com/ClickHouse/ClickHouse/issues/9032)). ClickHouse中的解决方法是将字符串列标记为UTF-8,这并不总是如此,但通常是这种情况。 [#9079](https://github.com/ClickHouse/ClickHouse/pull/9079) ([尤里\*巴拉诺夫](https://github.com/yurriy)) +- 添加对字符串和FixedString键的支持 `sumMap` [#8903](https://github.com/ClickHouse/ClickHouse/pull/8903) ([Baudouin Giard](https://github.com/bgiard)) +- 支持SummingMergeTree地图中的字符串键 [#8933](https://github.com/ClickHouse/ClickHouse/pull/8933) ([Baudouin Giard](https://github.com/bgiard)) +- 即使线程已抛出异常,也向线程池发送线程终止信号 [#8736](https://github.com/ClickHouse/ClickHouse/pull/8736) ([丁香飞](https://github.com/dingxiangfei2009)) +- 允许设置 `query_id` 在 `clickhouse-benchmark` [#9416](https://github.com/ClickHouse/ClickHouse/pull/9416) ([安东\*波波夫](https://github.com/CurtizJ)) +- 不要让奇怪的表达 `ALTER TABLE ... PARTITION partition` 查询。 这个地址 [#7192](https://github.com/ClickHouse/ClickHouse/issues/7192) [#8835](https://github.com/ClickHouse/ClickHouse/pull/8835) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 表 `system.table_engines` 现在提供有关功能支持的信息(如 `supports_ttl` 或 `supports_sort_order`). [#8830](https://github.com/ClickHouse/ClickHouse/pull/8830) ([Max Akhmedov](https://github.com/zlobober)) +- 启用 `system.metric_log` 默认情况下。 它将包含具有ProfileEvents值的行,CurrentMetrics收集与 “collect_interval_milliseconds” 间隔(默认情况下为一秒)。 该表非常小(通常以兆字节为单位),默认情况下收集此数据是合理的。 [#9225](https://github.com/ClickHouse/ClickHouse/pull/9225) ([阿列克谢-米洛维多夫](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) ([伊万](https://github.com/abyss7)) +- 现在是暂时的 `LIVE VIEW` 创建者 `CREATE LIVE VIEW name WITH TIMEOUT [42] ...` 而不是 `CREATE TEMPORARY LIVE VIEW ...`,因为以前的语法不符合 `CREATE TEMPORARY TABLE ...` [#9131](https://github.com/ClickHouse/ClickHouse/pull/9131) ([tavplubix](https://github.com/tavplubix)) +- 添加text_log。级别配置参数,以限制进入 `system.text_log` 表 [#8809](https://github.com/ClickHouse/ClickHouse/pull/8809) ([Azat Khuzhin](https://github.com/azat)) +- 允许根据TTL规则将下载的部分放入磁盘/卷 [#8598](https://github.com/ClickHouse/ClickHouse/pull/8598) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 对于外部MySQL字典,允许将MySQL连接池共同化为 “share” 他们在字典中。 此选项显着减少到MySQL服务器的连接数。 [#9409](https://github.com/ClickHouse/ClickHouse/pull/9409) ([Clément Rodriguez](https://github.com/clemrodriguez)) +- 显示分位数的最近查询执行时间 `clickhouse-benchmark` 输出而不是插值值。 最好显示与某些查询的执行时间相对应的值。 [#8712](https://github.com/ClickHouse/ClickHouse/pull/8712) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 可以在将数据插入到Kafka时为消息添加密钥和时间戳。 修复 [#7198](https://github.com/ClickHouse/ClickHouse/issues/7198) [#8969](https://github.com/ClickHouse/ClickHouse/pull/8969) ([filimonov](https://github.com/filimonov)) +- 如果服务器从终端运行,请按颜色突出显示线程号,查询id和日志优先级。 这是为了提高开发人员相关日志消息的可读性。 [#8961](https://github.com/ClickHouse/ClickHouse/pull/8961) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 更好的异常消息,同时加载表 `Ordinary` 数据库。 [#9527](https://github.com/ClickHouse/ClickHouse/pull/9527) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 执行 `arraySlice` 对于具有聚合函数状态的数组。 这修复 [#9388](https://github.com/ClickHouse/ClickHouse/issues/9388) [#9391](https://github.com/ClickHouse/ClickHouse/pull/9391) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 允许在in运算符的右侧使用常量函数和常量数组。 [#8813](https://github.com/ClickHouse/ClickHouse/pull/8813) ([安东\*波波夫](https://github.com/CurtizJ)) +- 如果在获取系统数据时发生了zookeeper异常。副本,将其显示在单独的列中。 这实现了 [#9137](https://github.com/ClickHouse/ClickHouse/issues/9137) [#9138](https://github.com/ClickHouse/ClickHouse/pull/9138) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 原子删除destroy上的MergeTree数据部分。 [#8402](https://github.com/ClickHouse/ClickHouse/pull/8402) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 支持分布式表的行级安全性。 [#8926](https://github.com/ClickHouse/ClickHouse/pull/8926) ([伊万](https://github.com/abyss7)) +- Now we recognize suffix (like KB, KiB…) in settings values. [#8072](https://github.com/ClickHouse/ClickHouse/pull/8072) ([米哈伊尔\*科罗托夫](https://github.com/millb)) +- 在构建大型连接的结果时防止内存不足。 [#8637](https://github.com/ClickHouse/ClickHouse/pull/8637) ([Artem Zuikov](https://github.com/4ertus2)) +- 在交互模式下为建议添加群集名称 `clickhouse-client`. [#8709](https://github.com/ClickHouse/ClickHouse/pull/8709) ([阿列克谢-米洛维多夫](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) ([伊万](https://github.com/abyss7)) +- 添加列 `exception_code` 在 `system.query_log` 桌子 [#8770](https://github.com/ClickHouse/ClickHouse/pull/8770) ([米哈伊尔\*科罗托夫](https://github.com/millb)) +- 在端口上启用MySQL兼容服务器 `9004` 在默认服务器配置文件中。 在配置的例子固定密码生成命令。 [#8771](https://github.com/ClickHouse/ClickHouse/pull/8771) ([尤里\*巴拉诺夫](https://github.com/yurriy)) +- 如果文件系统是只读的,请防止在关闭时中止。 这修复 [#9094](https://github.com/ClickHouse/ClickHouse/issues/9094) [#9100](https://github.com/ClickHouse/ClickHouse/pull/9100) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 当HTTP POST查询中需要长度时,更好的异常消息。 [#9453](https://github.com/ClickHouse/ClickHouse/pull/9453) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 添加 `_path` 和 `_file` 虚拟列 `HDFS` 和 `File` 发动机和 `hdfs` 和 `file` 表函数 [#8489](https://github.com/ClickHouse/ClickHouse/pull/8489) ([Olga Khvostikova](https://github.com/stavrolia)) +- 修复错误 `Cannot find column` 同时插入到 `MATERIALIZED VIEW` 在情况下,如果新列被添加到视图的内部表。 [#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) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) [#8803](https://github.com/ClickHouse/ClickHouse/pull/8803) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 通过最终更新后发送进度(如日志)修复本机客户端-服务器协议的进度。 这可能仅与使用本机协议的某些第三方工具相关。 [#9495](https://github.com/ClickHouse/ClickHouse/pull/9495) ([Azat Khuzhin](https://github.com/azat)) +- 添加系统指标跟踪使用MySQL协议的客户端连接数 ([#9013](https://github.com/ClickHouse/ClickHouse/issues/9013)). [#9015](https://github.com/ClickHouse/ClickHouse/pull/9015) ([尤金\*克里莫夫](https://github.com/Slach)) +- 从现在开始,HTTP响应将有 `X-ClickHouse-Timezone` 标题设置为相同的时区值 `SELECT timezone()` 会报告。 [#9493](https://github.com/ClickHouse/ClickHouse/pull/9493) ([Denis Glazachev](https://github.com/traceon)) #### 性能改进 {#performance-improvement} -- 使用IN提高分析指标的性能 [\#9261](https://github.com/ClickHouse/ClickHouse/pull/9261) ([安东\*波波夫](https://github.com/CurtizJ)) -- 逻辑函数+代码清理更简单,更有效的代码。 跟进到 [\#8718](https://github.com/ClickHouse/ClickHouse/issues/8718) [\#8728](https://github.com/ClickHouse/ClickHouse/pull/8728) ([亚历山大\*卡扎科夫](https://github.com/Akazz)) -- 整体性能改善(范围为5%。.通过确保使用C++20功能进行更严格的别名处理,对于受影响的查询来说,这是200%)。 [\#9304](https://github.com/ClickHouse/ClickHouse/pull/9304) ([阿莫斯鸟](https://github.com/amosbird)) -- 比较函数的内部循环更严格的别名。 [\#9327](https://github.com/ClickHouse/ClickHouse/pull/9327) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 对于算术函数的内部循环更严格的别名。 [\#9325](https://github.com/ClickHouse/ClickHouse/pull/9325) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- ColumnVector::replicate()的实现速度快约3倍,通过该实现ColumnConst::convertToFullColumn()。 在实现常数时,也将在测试中有用。 [\#9293](https://github.com/ClickHouse/ClickHouse/pull/9293) ([亚历山大\*卡扎科夫](https://github.com/Akazz)) -- 另一个小的性能改进 `ColumnVector::replicate()` (这加快了 `materialize` 函数和高阶函数),甚至进一步改进 [\#9293](https://github.com/ClickHouse/ClickHouse/issues/9293) [\#9442](https://github.com/ClickHouse/ClickHouse/pull/9442) ([亚历山大\*卡扎科夫](https://github.com/Akazz)) -- 改进的性能 `stochasticLinearRegression` 聚合函数。 此补丁由英特尔贡献。 [\#8652](https://github.com/ClickHouse/ClickHouse/pull/8652) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 提高性能 `reinterpretAsFixedString` 功能。 [\#9342](https://github.com/ClickHouse/ClickHouse/pull/9342) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 不要向客户端发送块 `Null` 处理器管道中的格式。 [\#8797](https://github.com/ClickHouse/ClickHouse/pull/8797) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) [\#8767](https://github.com/ClickHouse/ClickHouse/pull/8767) ([Alexander Kuzmenkov](https://github.com/akuzm)) +- 使用IN提高分析指标的性能 [#9261](https://github.com/ClickHouse/ClickHouse/pull/9261) ([安东\*波波夫](https://github.com/CurtizJ)) +- 逻辑函数+代码清理更简单,更有效的代码。 跟进到 [#8718](https://github.com/ClickHouse/ClickHouse/issues/8718) [#8728](https://github.com/ClickHouse/ClickHouse/pull/8728) ([亚历山大\*卡扎科夫](https://github.com/Akazz)) +- 整体性能改善(范围为5%。.通过确保使用C++20功能进行更严格的别名处理,对于受影响的查询来说,这是200%)。 [#9304](https://github.com/ClickHouse/ClickHouse/pull/9304) ([阿莫斯鸟](https://github.com/amosbird)) +- 比较函数的内部循环更严格的别名。 [#9327](https://github.com/ClickHouse/ClickHouse/pull/9327) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 对于算术函数的内部循环更严格的别名。 [#9325](https://github.com/ClickHouse/ClickHouse/pull/9325) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- ColumnVector::replicate()的实现速度快约3倍,通过该实现ColumnConst::convertToFullColumn()。 在实现常数时,也将在测试中有用。 [#9293](https://github.com/ClickHouse/ClickHouse/pull/9293) ([亚历山大\*卡扎科夫](https://github.com/Akazz)) +- 另一个小的性能改进 `ColumnVector::replicate()` (这加快了 `materialize` 函数和高阶函数),甚至进一步改进 [#9293](https://github.com/ClickHouse/ClickHouse/issues/9293) [#9442](https://github.com/ClickHouse/ClickHouse/pull/9442) ([亚历山大\*卡扎科夫](https://github.com/Akazz)) +- 改进的性能 `stochasticLinearRegression` 聚合函数。 此补丁由英特尔贡献。 [#8652](https://github.com/ClickHouse/ClickHouse/pull/8652) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 提高性能 `reinterpretAsFixedString` 功能。 [#9342](https://github.com/ClickHouse/ClickHouse/pull/9342) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 不要向客户端发送块 `Null` 处理器管道中的格式。 [#8797](https://github.com/ClickHouse/ClickHouse/pull/8797) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) [#8767](https://github.com/ClickHouse/ClickHouse/pull/8767) ([Alexander Kuzmenkov](https://github.com/akuzm)) #### 构建/测试/包装改进 {#buildtestingpackaging-improvement} -- 异常处理现在可以在适用于Linux的Windows子系统上正常工作。 看https://github.com/ClickHouse-Extras/libunwind/pull/3 这修复 [\#6480](https://github.com/ClickHouse/ClickHouse/issues/6480) [\#9564](https://github.com/ClickHouse/ClickHouse/pull/9564) ([sobolevsv](https://github.com/sobolevsv)) -- 替换 `readline` 与 `replxx` 对于在交互式线编辑 `clickhouse-client` [\#8416](https://github.com/ClickHouse/ClickHouse/pull/8416) ([伊万](https://github.com/abyss7)) -- 在FunctionsComparison中更好的构建时间和更少的模板实例化。 [\#9324](https://github.com/ClickHouse/ClickHouse/pull/9324) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 增加了与集成 `clang-tidy` 在线人 另请参阅 [\#6044](https://github.com/ClickHouse/ClickHouse/issues/6044) [\#9566](https://github.com/ClickHouse/ClickHouse/pull/9566) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 现在我们使用CI链接ClickHouse `lld` 即使是 `gcc`. [\#9049](https://github.com/ClickHouse/ClickHouse/pull/9049) ([阿利沙平](https://github.com/alesapin)) -- 允许随机线程调度和插入毛刺时 `THREAD_FUZZER_*` 设置环境变量。 这有助于测试。 [\#9459](https://github.com/ClickHouse/ClickHouse/pull/9459) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 在无状态测试中启用安全套接字 [\#9288](https://github.com/ClickHouse/ClickHouse/pull/9288) ([tavplubix](https://github.com/tavplubix)) -- 使SPLIT\_SHARED\_LIBRARIES=OFF更强大 [\#9156](https://github.com/ClickHouse/ClickHouse/pull/9156) ([Azat Khuzhin](https://github.com/azat)) -- 赂眉露\>\> “performance\_introspection\_and\_logging” 测试可靠的随机服务器卡住。 这可能发生在CI环境中。 另请参阅 [\#9515](https://github.com/ClickHouse/ClickHouse/issues/9515) [\#9528](https://github.com/ClickHouse/ClickHouse/pull/9528) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 在样式检查中验证XML。 [\#9550](https://github.com/ClickHouse/ClickHouse/pull/9550) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修正了测试中的竞争条件 `00738_lock_for_inner_table`. 这个测试依赖于睡眠。 [\#9555](https://github.com/ClickHouse/ClickHouse/pull/9555) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 删除类型的性能测试 `once`. 这是在统计比较模式下运行所有性能测试(更可靠)所需的。 [\#9557](https://github.com/ClickHouse/ClickHouse/pull/9557) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 增加了算术函数的性能测试。 [\#9326](https://github.com/ClickHouse/ClickHouse/pull/9326) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 增加了性能测试 `sumMap` 和 `sumMapWithOverflow` 聚合函数。 后续行动 [\#8933](https://github.com/ClickHouse/ClickHouse/issues/8933) [\#8947](https://github.com/ClickHouse/ClickHouse/pull/8947) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 通过样式检查确保错误代码的样式。 [\#9370](https://github.com/ClickHouse/ClickHouse/pull/9370) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 为测试历史添加脚本。 [\#8796](https://github.com/ClickHouse/ClickHouse/pull/8796) ([阿利沙平](https://github.com/alesapin)) -- 添加GCC警告 `-Wsuggest-override` 找到并修复所有地方 `override` 必须使用关键字。 [\#8760](https://github.com/ClickHouse/ClickHouse/pull/8760) ([kreuzerkrieg](https://github.com/kreuzerkrieg)) -- 在Mac OS X下忽略弱符号,因为它必须被定义 [\#9538](https://github.com/ClickHouse/ClickHouse/pull/9538) ([已删除用户](https://github.com/ghost)) -- 规范性能测试中某些查询的运行时间。 这是在准备在比较模式下运行所有性能测试时完成的。 [\#9565](https://github.com/ClickHouse/ClickHouse/pull/9565) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修复一些测试,以支持pytest与查询测试 [\#9062](https://github.com/ClickHouse/ClickHouse/pull/9062) ([伊万](https://github.com/abyss7)) -- 使用MSan在生成中启用SSL,因此在运行无状态测试时,服务器不会在启动时失败 [\#9531](https://github.com/ClickHouse/ClickHouse/pull/9531) ([tavplubix](https://github.com/tavplubix)) -- 修复测试结果中的数据库替换 [\#9384](https://github.com/ClickHouse/ClickHouse/pull/9384) ([Ilya Yatsishin](https://github.com/qoega)) -- 针对其他平台构建修复程序 [\#9381](https://github.com/ClickHouse/ClickHouse/pull/9381) ([proller](https://github.com/proller)) [\#8755](https://github.com/ClickHouse/ClickHouse/pull/8755) ([proller](https://github.com/proller)) [\#8631](https://github.com/ClickHouse/ClickHouse/pull/8631) ([proller](https://github.com/proller)) -- 将磁盘部分添加到无状态复盖率测试docker映像 [\#9213](https://github.com/ClickHouse/ClickHouse/pull/9213) ([帕维尔\*科瓦连科](https://github.com/Jokser)) -- 使用GRPC构建时,摆脱源代码树中的文件 [\#9588](https://github.com/ClickHouse/ClickHouse/pull/9588) ([阿莫斯鸟](https://github.com/amosbird)) -- 通过从上下文中删除SessionCleaner来缩短构建时间。 让SessionCleaner的代码更简单。 [\#9232](https://github.com/ClickHouse/ClickHouse/pull/9232) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 更新了clickhouse-test脚本中挂起查询的检查 [\#8858](https://github.com/ClickHouse/ClickHouse/pull/8858) ([亚历山大\*卡扎科夫](https://github.com/Akazz)) -- 从存储库中删除了一些无用的文件。 [\#8843](https://github.com/ClickHouse/ClickHouse/pull/8843) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 更改类型的数学perftests从 `once` 到 `loop`. [\#8783](https://github.com/ClickHouse/ClickHouse/pull/8783) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 添加码头镜像,它允许为我们的代码库构建交互式代码浏览器HTML报告。 [\#8781](https://github.com/ClickHouse/ClickHouse/pull/8781) ([阿利沙平](https://github.com/alesapin))见 [Woboq代码浏览器](https://clickhouse.tech/codebrowser/html_report///ClickHouse/dbms/index.html) -- 抑制MSan下的一些测试失败。 [\#8780](https://github.com/ClickHouse/ClickHouse/pull/8780) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- 加速 “exception while insert” 测试 此测试通常在具有复盖率的调试版本中超时。 [\#8711](https://github.com/ClickHouse/ClickHouse/pull/8711) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 更新 `libcxx` 和 `libcxxabi` 为了主人 在准备 [\#9304](https://github.com/ClickHouse/ClickHouse/issues/9304) [\#9308](https://github.com/ClickHouse/ClickHouse/pull/9308) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修复flacky测试 `00910_zookeeper_test_alter_compression_codecs`. [\#9525](https://github.com/ClickHouse/ClickHouse/pull/9525) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 清理重复的链接器标志。 确保链接器不会查找意想不到的符号。 [\#9433](https://github.com/ClickHouse/ClickHouse/pull/9433) ([阿莫斯鸟](https://github.com/amosbird)) -- 添加 `clickhouse-odbc` 驱动程序进入测试图像。 这允许通过自己的ODBC驱动程序测试ClickHouse与ClickHouse的交互。 [\#9348](https://github.com/ClickHouse/ClickHouse/pull/9348) ([filimonov](https://github.com/filimonov)) -- 修复单元测试中的几个错误。 [\#9047](https://github.com/ClickHouse/ClickHouse/pull/9047) ([阿利沙平](https://github.com/alesapin)) -- 启用 `-Wmissing-include-dirs` GCC警告消除所有不存在的包括-主要是由于CMake脚本错误 [\#8704](https://github.com/ClickHouse/ClickHouse/pull/8704) ([kreuzerkrieg](https://github.com/kreuzerkrieg)) -- 描述查询探查器无法工作的原因。 这是用于 [\#9049](https://github.com/ClickHouse/ClickHouse/issues/9049) [\#9144](https://github.com/ClickHouse/ClickHouse/pull/9144) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 将OpenSSL更新到上游主机。 修复了TLS连接可能会失败并显示消息的问题 `OpenSSL SSL_read: error:14094438:SSL routines:ssl3_read_bytes:tlsv1 alert internal error` 和 `SSL Exception: error:2400006E:random number generator::error retrieving entropy`. 该问题出现在版本20.1中。 [\#8956](https://github.com/ClickHouse/ClickHouse/pull/8956) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 更新服务器的Dockerfile [\#8893](https://github.com/ClickHouse/ClickHouse/pull/8893) ([Ilya Mazaev](https://github.com/ne-ray)) -- Build-gcc-from-sources脚本中的小修复 [\#8774](https://github.com/ClickHouse/ClickHouse/pull/8774) ([Michael Nacharov](https://github.com/mnach)) -- 替换 `numbers` 到 `zeros` 在perftests其中 `number` 不使用列。 这将导致更干净的测试结果。 [\#9600](https://github.com/ClickHouse/ClickHouse/pull/9600) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 修复列构造函数中使用initializer\_list时堆栈溢出问题。 [\#9367](https://github.com/ClickHouse/ClickHouse/pull/9367) ([已删除用户](https://github.com/ghost)) -- 将librdkafka升级到v1.3.0。 启用bund绑 `rdkafka` 和 `gsasl` mac OS X上的库 [\#9000](https://github.com/ClickHouse/ClickHouse/pull/9000) ([安德鲁Onyshchuk](https://github.com/oandrew)) -- 在GCC9.2.0上构建修复程序 [\#9306](https://github.com/ClickHouse/ClickHouse/pull/9306) ([vxider](https://github.com/Vxider)) +- 异常处理现在可以在适用于Linux的Windows子系统上正常工作。 看https://github.com/ClickHouse-Extras/libunwind/pull/3 这修复 [#6480](https://github.com/ClickHouse/ClickHouse/issues/6480) [#9564](https://github.com/ClickHouse/ClickHouse/pull/9564) ([sobolevsv](https://github.com/sobolevsv)) +- 替换 `readline` 与 `replxx` 对于在交互式线编辑 `clickhouse-client` [#8416](https://github.com/ClickHouse/ClickHouse/pull/8416) ([伊万](https://github.com/abyss7)) +- 在FunctionsComparison中更好的构建时间和更少的模板实例化。 [#9324](https://github.com/ClickHouse/ClickHouse/pull/9324) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 增加了与集成 `clang-tidy` 在线人 另请参阅 [#6044](https://github.com/ClickHouse/ClickHouse/issues/6044) [#9566](https://github.com/ClickHouse/ClickHouse/pull/9566) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 现在我们使用CI链接ClickHouse `lld` 即使是 `gcc`. [#9049](https://github.com/ClickHouse/ClickHouse/pull/9049) ([阿利沙平](https://github.com/alesapin)) +- 允许随机线程调度和插入毛刺时 `THREAD_FUZZER_*` 设置环境变量。 这有助于测试。 [#9459](https://github.com/ClickHouse/ClickHouse/pull/9459) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 在无状态测试中启用安全套接字 [#9288](https://github.com/ClickHouse/ClickHouse/pull/9288) ([tavplubix](https://github.com/tavplubix)) +- 使SPLIT_SHARED_LIBRARIES=OFF更强大 [#9156](https://github.com/ClickHouse/ClickHouse/pull/9156) ([Azat Khuzhin](https://github.com/azat)) +- 赂眉露\>\> “performance_introspection_and_logging” 测试可靠的随机服务器卡住。 这可能发生在CI环境中。 另请参阅 [#9515](https://github.com/ClickHouse/ClickHouse/issues/9515) [#9528](https://github.com/ClickHouse/ClickHouse/pull/9528) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 在样式检查中验证XML。 [#9550](https://github.com/ClickHouse/ClickHouse/pull/9550) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修正了测试中的竞争条件 `00738_lock_for_inner_table`. 这个测试依赖于睡眠。 [#9555](https://github.com/ClickHouse/ClickHouse/pull/9555) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 删除类型的性能测试 `once`. 这是在统计比较模式下运行所有性能测试(更可靠)所需的。 [#9557](https://github.com/ClickHouse/ClickHouse/pull/9557) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 增加了算术函数的性能测试。 [#9326](https://github.com/ClickHouse/ClickHouse/pull/9326) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 增加了性能测试 `sumMap` 和 `sumMapWithOverflow` 聚合函数。 后续行动 [#8933](https://github.com/ClickHouse/ClickHouse/issues/8933) [#8947](https://github.com/ClickHouse/ClickHouse/pull/8947) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 通过样式检查确保错误代码的样式。 [#9370](https://github.com/ClickHouse/ClickHouse/pull/9370) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 为测试历史添加脚本。 [#8796](https://github.com/ClickHouse/ClickHouse/pull/8796) ([阿利沙平](https://github.com/alesapin)) +- 添加GCC警告 `-Wsuggest-override` 找到并修复所有地方 `override` 必须使用关键字。 [#8760](https://github.com/ClickHouse/ClickHouse/pull/8760) ([kreuzerkrieg](https://github.com/kreuzerkrieg)) +- 在Mac OS X下忽略弱符号,因为它必须被定义 [#9538](https://github.com/ClickHouse/ClickHouse/pull/9538) ([已删除用户](https://github.com/ghost)) +- 规范性能测试中某些查询的运行时间。 这是在准备在比较模式下运行所有性能测试时完成的。 [#9565](https://github.com/ClickHouse/ClickHouse/pull/9565) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修复一些测试,以支持pytest与查询测试 [#9062](https://github.com/ClickHouse/ClickHouse/pull/9062) ([伊万](https://github.com/abyss7)) +- 使用MSan在生成中启用SSL,因此在运行无状态测试时,服务器不会在启动时失败 [#9531](https://github.com/ClickHouse/ClickHouse/pull/9531) ([tavplubix](https://github.com/tavplubix)) +- 修复测试结果中的数据库替换 [#9384](https://github.com/ClickHouse/ClickHouse/pull/9384) ([Ilya Yatsishin](https://github.com/qoega)) +- 针对其他平台构建修复程序 [#9381](https://github.com/ClickHouse/ClickHouse/pull/9381) ([proller](https://github.com/proller)) [#8755](https://github.com/ClickHouse/ClickHouse/pull/8755) ([proller](https://github.com/proller)) [#8631](https://github.com/ClickHouse/ClickHouse/pull/8631) ([proller](https://github.com/proller)) +- 将磁盘部分添加到无状态复盖率测试docker映像 [#9213](https://github.com/ClickHouse/ClickHouse/pull/9213) ([帕维尔\*科瓦连科](https://github.com/Jokser)) +- 使用GRPC构建时,摆脱源代码树中的文件 [#9588](https://github.com/ClickHouse/ClickHouse/pull/9588) ([阿莫斯鸟](https://github.com/amosbird)) +- 通过从上下文中删除SessionCleaner来缩短构建时间。 让SessionCleaner的代码更简单。 [#9232](https://github.com/ClickHouse/ClickHouse/pull/9232) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 更新了clickhouse-test脚本中挂起查询的检查 [#8858](https://github.com/ClickHouse/ClickHouse/pull/8858) ([亚历山大\*卡扎科夫](https://github.com/Akazz)) +- 从存储库中删除了一些无用的文件。 [#8843](https://github.com/ClickHouse/ClickHouse/pull/8843) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 更改类型的数学perftests从 `once` 到 `loop`. [#8783](https://github.com/ClickHouse/ClickHouse/pull/8783) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 添加码头镜像,它允许为我们的代码库构建交互式代码浏览器HTML报告。 [#8781](https://github.com/ClickHouse/ClickHouse/pull/8781) ([阿利沙平](https://github.com/alesapin))见 [Woboq代码浏览器](https://clickhouse.tech/codebrowser/html_report///ClickHouse/dbms/index.html) +- 抑制MSan下的一些测试失败。 [#8780](https://github.com/ClickHouse/ClickHouse/pull/8780) ([Alexander Kuzmenkov](https://github.com/akuzm)) +- 加速 “exception while insert” 测试 此测试通常在具有复盖率的调试版本中超时。 [#8711](https://github.com/ClickHouse/ClickHouse/pull/8711) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 更新 `libcxx` 和 `libcxxabi` 为了主人 在准备 [#9304](https://github.com/ClickHouse/ClickHouse/issues/9304) [#9308](https://github.com/ClickHouse/ClickHouse/pull/9308) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修复flacky测试 `00910_zookeeper_test_alter_compression_codecs`. [#9525](https://github.com/ClickHouse/ClickHouse/pull/9525) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 清理重复的链接器标志。 确保链接器不会查找意想不到的符号。 [#9433](https://github.com/ClickHouse/ClickHouse/pull/9433) ([阿莫斯鸟](https://github.com/amosbird)) +- 添加 `clickhouse-odbc` 驱动程序进入测试图像。 这允许通过自己的ODBC驱动程序测试ClickHouse与ClickHouse的交互。 [#9348](https://github.com/ClickHouse/ClickHouse/pull/9348) ([filimonov](https://github.com/filimonov)) +- 修复单元测试中的几个错误。 [#9047](https://github.com/ClickHouse/ClickHouse/pull/9047) ([阿利沙平](https://github.com/alesapin)) +- 启用 `-Wmissing-include-dirs` GCC警告消除所有不存在的包括-主要是由于CMake脚本错误 [#8704](https://github.com/ClickHouse/ClickHouse/pull/8704) ([kreuzerkrieg](https://github.com/kreuzerkrieg)) +- 描述查询探查器无法工作的原因。 这是用于 [#9049](https://github.com/ClickHouse/ClickHouse/issues/9049) [#9144](https://github.com/ClickHouse/ClickHouse/pull/9144) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 将OpenSSL更新到上游主机。 修复了TLS连接可能会失败并显示消息的问题 `OpenSSL SSL_read: error:14094438:SSL routines:ssl3_read_bytes:tlsv1 alert internal error` 和 `SSL Exception: error:2400006E:random number generator::error retrieving entropy`. 该问题出现在版本20.1中。 [#8956](https://github.com/ClickHouse/ClickHouse/pull/8956) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 更新服务器的Dockerfile [#8893](https://github.com/ClickHouse/ClickHouse/pull/8893) ([Ilya Mazaev](https://github.com/ne-ray)) +- Build-gcc-from-sources脚本中的小修复 [#8774](https://github.com/ClickHouse/ClickHouse/pull/8774) ([Michael Nacharov](https://github.com/mnach)) +- 替换 `numbers` 到 `zeros` 在perftests其中 `number` 不使用列。 这将导致更干净的测试结果。 [#9600](https://github.com/ClickHouse/ClickHouse/pull/9600) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 修复列构造函数中使用initializer_list时堆栈溢出问题。 [#9367](https://github.com/ClickHouse/ClickHouse/pull/9367) ([已删除用户](https://github.com/ghost)) +- 将librdkafka升级到v1.3.0。 启用bund绑 `rdkafka` 和 `gsasl` mac OS X上的库 [#9000](https://github.com/ClickHouse/ClickHouse/pull/9000) ([安德鲁Onyshchuk](https://github.com/oandrew)) +- 在GCC9.2.0上构建修复程序 [#9306](https://github.com/ClickHouse/ClickHouse/pull/9306) ([vxider](https://github.com/Vxider)) ## 碌莽禄.拢.0755-88888888 {#clickhouse-release-v20-1} @@ -271,396 +271,396 @@ toc_title: "\u53D8\u66F4\u65E5\u5FD7" #### 错误修复 {#bug-fix-3} -- 修复可能的永久性 `Cannot schedule a task` 错误(由于未处理的异常 `ParallelAggregatingBlockInputStream::Handler::onFinish/onFinishThread`). 这修复 [\#6833](https://github.com/ClickHouse/ClickHouse/issues/6833). [\#9154](https://github.com/ClickHouse/ClickHouse/pull/9154) ([Azat Khuzhin](https://github.com/azat)) -- 修复过多的内存消耗 `ALTER` 查询(突变)。 这修复 [\#9533](https://github.com/ClickHouse/ClickHouse/issues/9533) 和 [\#9670](https://github.com/ClickHouse/ClickHouse/issues/9670). [\#9754](https://github.com/ClickHouse/ClickHouse/pull/9754) ([阿利沙平](https://github.com/alesapin)) -- 修复外部字典DDL中反引用的错误。 这修复 [\#9619](https://github.com/ClickHouse/ClickHouse/issues/9619). [\#9734](https://github.com/ClickHouse/ClickHouse/pull/9734) ([阿利沙平](https://github.com/alesapin)) +- 修复可能的永久性 `Cannot schedule a task` 错误(由于未处理的异常 `ParallelAggregatingBlockInputStream::Handler::onFinish/onFinishThread`). 这修复 [#6833](https://github.com/ClickHouse/ClickHouse/issues/6833). [#9154](https://github.com/ClickHouse/ClickHouse/pull/9154) ([Azat Khuzhin](https://github.com/azat)) +- 修复过多的内存消耗 `ALTER` 查询(突变)。 这修复 [#9533](https://github.com/ClickHouse/ClickHouse/issues/9533) 和 [#9670](https://github.com/ClickHouse/ClickHouse/issues/9670). [#9754](https://github.com/ClickHouse/ClickHouse/pull/9754) ([阿利沙平](https://github.com/alesapin)) +- 修复外部字典DDL中反引用的错误。 这修复 [#9619](https://github.com/ClickHouse/ClickHouse/issues/9619). [#9734](https://github.com/ClickHouse/ClickHouse/pull/9734) ([阿利沙平](https://github.com/alesapin)) ### ClickHouse释放v20.1.7.38,2020-03-18 {#clickhouse-release-v20-1-7-38-2020-03-18} #### 错误修复 {#bug-fix-4} -- 修正了不正确的内部函数名称 `sumKahan` 和 `sumWithOverflow`. 在远程查询中使用此函数时,我会导致异常。 [\#9636](https://github.com/ClickHouse/ClickHouse/pull/9636) ([Azat Khuzhin](https://github.com/azat)). 这个问题是在所有ClickHouse版本。 -- 允许 `ALTER ON CLUSTER` 的 `Distributed` 具有内部复制的表。 这修复 [\#3268](https://github.com/ClickHouse/ClickHouse/issues/3268). [\#9617](https://github.com/ClickHouse/ClickHouse/pull/9617) ([shinoi2](https://github.com/shinoi2)). 这个问题是在所有ClickHouse版本。 -- 修复可能的异常 `Size of filter doesn't match size of column` 和 `Invalid number of rows in Chunk` 在 `MergeTreeRangeReader`. 它们可能在执行时出现 `PREWHERE` 在某些情况下。 修复 [\#9132](https://github.com/ClickHouse/ClickHouse/issues/9132). [\#9612](https://github.com/ClickHouse/ClickHouse/pull/9612) ([安东\*波波夫](https://github.com/CurtizJ)) -- 修复了这个问题:如果你编写一个简单的算术表达式,则不会保留时区 `time + 1` (与像这样的表达形成对比 `time + INTERVAL 1 SECOND`). 这修复 [\#5743](https://github.com/ClickHouse/ClickHouse/issues/5743). [\#9323](https://github.com/ClickHouse/ClickHouse/pull/9323) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)). 这个问题是在所有ClickHouse版本。 -- 现在不可能创建或添加具有简单循环别名的列,如 `a DEFAULT b, b DEFAULT a`. [\#9603](https://github.com/ClickHouse/ClickHouse/pull/9603) ([阿利沙平](https://github.com/alesapin)) -- 修复了base64编码值末尾填充格式错误的问题。 更新base64库。 这修复 [\#9491](https://github.com/ClickHouse/ClickHouse/issues/9491),关闭 [\#9492](https://github.com/ClickHouse/ClickHouse/issues/9492) [\#9500](https://github.com/ClickHouse/ClickHouse/pull/9500) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修复数据竞赛破坏 `Poco::HTTPServer`. 当服务器启动并立即关闭时,可能会发生这种情况。 [\#9468](https://github.com/ClickHouse/ClickHouse/pull/9468) ([安东\*波波夫](https://github.com/CurtizJ)) -- 修复可能的崩溃/错误的行数 `LIMIT n WITH TIES` 当有很多行等于第n行时。 [\#9464](https://github.com/ClickHouse/ClickHouse/pull/9464) ([tavplubix](https://github.com/tavplubix)) -- 修复与列Ttl可能不匹配的校验和。 [\#9451](https://github.com/ClickHouse/ClickHouse/pull/9451) ([安东\*波波夫](https://github.com/CurtizJ)) -- 修复当用户尝试崩溃 `ALTER MODIFY SETTING` 对于老格式化 `MergeTree` 表引擎家族. [\#9435](https://github.com/ClickHouse/ClickHouse/pull/9435) ([阿利沙平](https://github.com/alesapin)) -- 现在我们将尝试更频繁地完成突变。 [\#9427](https://github.com/ClickHouse/ClickHouse/pull/9427) ([阿利沙平](https://github.com/alesapin)) -- 修复引入的复制协议不兼容 [\#8598](https://github.com/ClickHouse/ClickHouse/issues/8598). [\#9412](https://github.com/ClickHouse/ClickHouse/pull/9412) ([阿利沙平](https://github.com/alesapin)) -- 修复数组类型的bloom\_filter索引的not(has())。 [\#9407](https://github.com/ClickHouse/ClickHouse/pull/9407) ([achimbab](https://github.com/achimbab)) -- 固定的行为 `match` 和 `extract` 当干草堆有零字节的函数。 当干草堆不变时,这种行为是错误的。 这修复 [\#9160](https://github.com/ClickHouse/ClickHouse/issues/9160) [\#9163](https://github.com/ClickHouse/ClickHouse/pull/9163) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) [\#9345](https://github.com/ClickHouse/ClickHouse/pull/9345) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修正了不正确的内部函数名称 `sumKahan` 和 `sumWithOverflow`. 在远程查询中使用此函数时,我会导致异常。 [#9636](https://github.com/ClickHouse/ClickHouse/pull/9636) ([Azat Khuzhin](https://github.com/azat)). 这个问题是在所有ClickHouse版本。 +- 允许 `ALTER ON CLUSTER` 的 `Distributed` 具有内部复制的表。 这修复 [#3268](https://github.com/ClickHouse/ClickHouse/issues/3268). [#9617](https://github.com/ClickHouse/ClickHouse/pull/9617) ([shinoi2](https://github.com/shinoi2)). 这个问题是在所有ClickHouse版本。 +- 修复可能的异常 `Size of filter doesn't match size of column` 和 `Invalid number of rows in Chunk` 在 `MergeTreeRangeReader`. 它们可能在执行时出现 `PREWHERE` 在某些情况下。 修复 [#9132](https://github.com/ClickHouse/ClickHouse/issues/9132). [#9612](https://github.com/ClickHouse/ClickHouse/pull/9612) ([安东\*波波夫](https://github.com/CurtizJ)) +- 修复了这个问题:如果你编写一个简单的算术表达式,则不会保留时区 `time + 1` (与像这样的表达形成对比 `time + INTERVAL 1 SECOND`). 这修复 [#5743](https://github.com/ClickHouse/ClickHouse/issues/5743). [#9323](https://github.com/ClickHouse/ClickHouse/pull/9323) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)). 这个问题是在所有ClickHouse版本。 +- 现在不可能创建或添加具有简单循环别名的列,如 `a DEFAULT b, b DEFAULT a`. [#9603](https://github.com/ClickHouse/ClickHouse/pull/9603) ([阿利沙平](https://github.com/alesapin)) +- 修复了base64编码值末尾填充格式错误的问题。 更新base64库。 这修复 [#9491](https://github.com/ClickHouse/ClickHouse/issues/9491),关闭 [#9492](https://github.com/ClickHouse/ClickHouse/issues/9492) [#9500](https://github.com/ClickHouse/ClickHouse/pull/9500) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修复数据竞赛破坏 `Poco::HTTPServer`. 当服务器启动并立即关闭时,可能会发生这种情况。 [#9468](https://github.com/ClickHouse/ClickHouse/pull/9468) ([安东\*波波夫](https://github.com/CurtizJ)) +- 修复可能的崩溃/错误的行数 `LIMIT n WITH TIES` 当有很多行等于第n行时。 [#9464](https://github.com/ClickHouse/ClickHouse/pull/9464) ([tavplubix](https://github.com/tavplubix)) +- 修复与列Ttl可能不匹配的校验和。 [#9451](https://github.com/ClickHouse/ClickHouse/pull/9451) ([安东\*波波夫](https://github.com/CurtizJ)) +- 修复当用户尝试崩溃 `ALTER MODIFY SETTING` 对于老格式化 `MergeTree` 表引擎家族. [#9435](https://github.com/ClickHouse/ClickHouse/pull/9435) ([阿利沙平](https://github.com/alesapin)) +- 现在我们将尝试更频繁地完成突变。 [#9427](https://github.com/ClickHouse/ClickHouse/pull/9427) ([阿利沙平](https://github.com/alesapin)) +- 修复引入的复制协议不兼容 [#8598](https://github.com/ClickHouse/ClickHouse/issues/8598). [#9412](https://github.com/ClickHouse/ClickHouse/pull/9412) ([阿利沙平](https://github.com/alesapin)) +- 修复数组类型的bloom_filter索引的not(has())。 [#9407](https://github.com/ClickHouse/ClickHouse/pull/9407) ([achimbab](https://github.com/achimbab)) +- 固定的行为 `match` 和 `extract` 当干草堆有零字节的函数。 当干草堆不变时,这种行为是错误的。 这修复 [#9160](https://github.com/ClickHouse/ClickHouse/issues/9160) [#9163](https://github.com/ClickHouse/ClickHouse/pull/9163) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) [#9345](https://github.com/ClickHouse/ClickHouse/pull/9345) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) #### 构建/测试/包装改进 {#buildtestingpackaging-improvement-1} -- 异常处理现在可以在适用于Linux的Windows子系统上正常工作。 看https://github.com/ClickHouse-Extras/libunwind/pull/3 这修复 [\#6480](https://github.com/ClickHouse/ClickHouse/issues/6480) [\#9564](https://github.com/ClickHouse/ClickHouse/pull/9564) ([sobolevsv](https://github.com/sobolevsv)) +- 异常处理现在可以在适用于Linux的Windows子系统上正常工作。 看https://github.com/ClickHouse-Extras/libunwind/pull/3 这修复 [#6480](https://github.com/ClickHouse/ClickHouse/issues/6480) [#9564](https://github.com/ClickHouse/ClickHouse/pull/9564) ([sobolevsv](https://github.com/sobolevsv)) ### ClickHouse释放v20.1.6.30,2020-03-05 {#clickhouse-release-v20-1-6-30-2020-03-05} #### 错误修复 {#bug-fix-5} - 修复压缩时的数据不兼容 `T64` 编解ec - [\#9039](https://github.com/ClickHouse/ClickHouse/pull/9039) [(abyss7)](https://github.com/abyss7) -- 在一个线程中从MergeTree表中读取时修复范围顺序。 修复 [\#8964](https://github.com/ClickHouse/ClickHouse/issues/8964). - [\#9050](https://github.com/ClickHouse/ClickHouse/pull/9050) [(CurtizJ))](https://github.com/CurtizJ) -- 修复可能的段错误 `MergeTreeRangeReader`,同时执行 `PREWHERE`. 修复 [\#9064](https://github.com/ClickHouse/ClickHouse/issues/9064). - [\#9106](https://github.com/ClickHouse/ClickHouse/pull/9106) [(CurtizJ))](https://github.com/CurtizJ) + [#9039](https://github.com/ClickHouse/ClickHouse/pull/9039) [(abyss7)](https://github.com/abyss7) +- 在一个线程中从MergeTree表中读取时修复范围顺序。 修复 [#8964](https://github.com/ClickHouse/ClickHouse/issues/8964). + [#9050](https://github.com/ClickHouse/ClickHouse/pull/9050) [(CurtizJ))](https://github.com/CurtizJ) +- 修复可能的段错误 `MergeTreeRangeReader`,同时执行 `PREWHERE`. 修复 [#9064](https://github.com/ClickHouse/ClickHouse/issues/9064). + [#9106](https://github.com/ClickHouse/ClickHouse/pull/9106) [(CurtizJ))](https://github.com/CurtizJ) - 修复 `reinterpretAsFixedString` 返回 `FixedString` 而不是 `String`. - [\#9052](https://github.com/ClickHouse/ClickHouse/pull/9052) [(oandrew)](https://github.com/oandrew) -- 修复 `joinGet` 使用可为空的返回类型。 修复 [\#8919](https://github.com/ClickHouse/ClickHouse/issues/8919) - [\#9014](https://github.com/ClickHouse/ClickHouse/pull/9014) [(amosbird)](https://github.com/amosbird) + [#9052](https://github.com/ClickHouse/ClickHouse/pull/9052) [(oandrew)](https://github.com/oandrew) +- 修复 `joinGet` 使用可为空的返回类型。 修复 [#8919](https://github.com/ClickHouse/ClickHouse/issues/8919) + [#9014](https://github.com/ClickHouse/ClickHouse/pull/9014) [(amosbird)](https://github.com/amosbird) - 修复bittestall/bitTestAny函数的模糊测试和不正确的行为。 - [\#9143](https://github.com/ClickHouse/ClickHouse/pull/9143) [(阿列克谢-米洛维多夫)](https://github.com/alexey-milovidov) -- 修复当干草堆有零字节时匹配和提取函数的行为。 当干草堆不变时,这种行为是错误的。 修复 [\#9160](https://github.com/ClickHouse/ClickHouse/issues/9160) - [\#9163](https://github.com/ClickHouse/ClickHouse/pull/9163) [(阿列克谢-米洛维多夫)](https://github.com/alexey-milovidov) -- 当使用非严格单调函数索引时,固定执行反转谓词。 修复 [\#9034](https://github.com/ClickHouse/ClickHouse/issues/9034) - [\#9223](https://github.com/ClickHouse/ClickHouse/pull/9223) [(Akazz)](https://github.com/Akazz) -- 允许重写 `CROSS` 到 `INNER JOIN` 如果有 `[NOT] LIKE` 操作员在 `WHERE` 科。 修复 [\#9191](https://github.com/ClickHouse/ClickHouse/issues/9191) - [\#9229](https://github.com/ClickHouse/ClickHouse/pull/9229) [(4ertus2)](https://github.com/4ertus2) + [#9143](https://github.com/ClickHouse/ClickHouse/pull/9143) [(阿列克谢-米洛维多夫)](https://github.com/alexey-milovidov) +- 修复当干草堆有零字节时匹配和提取函数的行为。 当干草堆不变时,这种行为是错误的。 修复 [#9160](https://github.com/ClickHouse/ClickHouse/issues/9160) + [#9163](https://github.com/ClickHouse/ClickHouse/pull/9163) [(阿列克谢-米洛维多夫)](https://github.com/alexey-milovidov) +- 当使用非严格单调函数索引时,固定执行反转谓词。 修复 [#9034](https://github.com/ClickHouse/ClickHouse/issues/9034) + [#9223](https://github.com/ClickHouse/ClickHouse/pull/9223) [(Akazz)](https://github.com/Akazz) +- 允许重写 `CROSS` 到 `INNER JOIN` 如果有 `[NOT] LIKE` 操作员在 `WHERE` 科。 修复 [#9191](https://github.com/ClickHouse/ClickHouse/issues/9191) + [#9229](https://github.com/ClickHouse/ClickHouse/pull/9229) [(4ertus2)](https://github.com/4ertus2) - 允许使用日志引擎的表中的第一列成为别名。 - [\#9231](https://github.com/ClickHouse/ClickHouse/pull/9231) [(abyss7)](https://github.com/abyss7) -- 允许逗号加入 `IN()` 进去 修复 [\#7314](https://github.com/ClickHouse/ClickHouse/issues/7314). - [\#9251](https://github.com/ClickHouse/ClickHouse/pull/9251) [(4ertus2)](https://github.com/4ertus2) -- 改进 `ALTER MODIFY/ADD` 查询逻辑。 现在你不能 `ADD` 不带类型的列, `MODIFY` 默认表达式不改变列的类型和 `MODIFY` type不会丢失默认表达式值。 修复 [\#8669](https://github.com/ClickHouse/ClickHouse/issues/8669). - [\#9227](https://github.com/ClickHouse/ClickHouse/pull/9227) [(alesapin)](https://github.com/alesapin) -- 修复突变最终确定,当已经完成突变时可以具有状态is\_done=0。 - [\#9217](https://github.com/ClickHouse/ClickHouse/pull/9217) [(alesapin)](https://github.com/alesapin) -- 碌莽禄Support: “Processors” 管道系统.数字和系统.numbers\_mt 这也修复了错误时 `max_execution_time` 不被尊重。 - [\#7796](https://github.com/ClickHouse/ClickHouse/pull/7796) [(KochetovNicolai)](https://github.com/KochetovNicolai) + [#9231](https://github.com/ClickHouse/ClickHouse/pull/9231) [(abyss7)](https://github.com/abyss7) +- 允许逗号加入 `IN()` 进去 修复 [#7314](https://github.com/ClickHouse/ClickHouse/issues/7314). + [#9251](https://github.com/ClickHouse/ClickHouse/pull/9251) [(4ertus2)](https://github.com/4ertus2) +- 改进 `ALTER MODIFY/ADD` 查询逻辑。 现在你不能 `ADD` 不带类型的列, `MODIFY` 默认表达式不改变列的类型和 `MODIFY` type不会丢失默认表达式值。 修复 [#8669](https://github.com/ClickHouse/ClickHouse/issues/8669). + [#9227](https://github.com/ClickHouse/ClickHouse/pull/9227) [(alesapin)](https://github.com/alesapin) +- 修复突变最终确定,当已经完成突变时可以具有状态is_done=0。 + [#9217](https://github.com/ClickHouse/ClickHouse/pull/9217) [(alesapin)](https://github.com/alesapin) +- 碌莽禄Support: “Processors” 管道系统.数字和系统.numbers_mt 这也修复了错误时 `max_execution_time` 不被尊重。 + [#7796](https://github.com/ClickHouse/ClickHouse/pull/7796) [(KochetovNicolai)](https://github.com/KochetovNicolai) - 修复错误的计数 `DictCacheKeysRequestedFound` 公制。 - [\#9411](https://github.com/ClickHouse/ClickHouse/pull/9411) [(nikitamikhaylov)](https://github.com/nikitamikhaylov) + [#9411](https://github.com/ClickHouse/ClickHouse/pull/9411) [(nikitamikhaylov)](https://github.com/nikitamikhaylov) - 添加了对存储策略的检查 `ATTACH PARTITION FROM`, `REPLACE PARTITION`, `MOVE TO TABLE` 否则可能使部分数据在重新启动后无法访问,并阻止ClickHouse启动。 - [\#9383](https://github.com/ClickHouse/ClickHouse/pull/9383) [(excitoon)](https://github.com/excitoon) -- 在固定的瑞银报告 `MergeTreeIndexSet`. 这修复 [\#9250](https://github.com/ClickHouse/ClickHouse/issues/9250) - [\#9365](https://github.com/ClickHouse/ClickHouse/pull/9365) [(阿列克谢-米洛维多夫)](https://github.com/alexey-milovidov) + [#9383](https://github.com/ClickHouse/ClickHouse/pull/9383) [(excitoon)](https://github.com/excitoon) +- 在固定的瑞银报告 `MergeTreeIndexSet`. 这修复 [#9250](https://github.com/ClickHouse/ClickHouse/issues/9250) + [#9365](https://github.com/ClickHouse/ClickHouse/pull/9365) [(阿列克谢-米洛维多夫)](https://github.com/alexey-milovidov) - 在BlockIO中修复可能的数据集。 - [\#9356](https://github.com/ClickHouse/ClickHouse/pull/9356) [(KochetovNicolai)](https://github.com/KochetovNicolai) -- 支持 `UInt64` 在JSON相关函数中不适合Int64的数字。 更新 `SIMDJSON` 为了主人 这修复 [\#9209](https://github.com/ClickHouse/ClickHouse/issues/9209) - [\#9344](https://github.com/ClickHouse/ClickHouse/pull/9344) [(阿列克谢-米洛维多夫)](https://github.com/alexey-milovidov) -- 如果将数据目录挂载到单独的设备,则修复可用空间量计算不正确时的问题。 对于默认磁盘,计算数据子目录的可用空间。 这修复 [\#7441](https://github.com/ClickHouse/ClickHouse/issues/7441) - [\#9257](https://github.com/ClickHouse/ClickHouse/pull/9257) [(米尔布)](https://github.com/millb) + [#9356](https://github.com/ClickHouse/ClickHouse/pull/9356) [(KochetovNicolai)](https://github.com/KochetovNicolai) +- 支持 `UInt64` 在JSON相关函数中不适合Int64的数字。 更新 `SIMDJSON` 为了主人 这修复 [#9209](https://github.com/ClickHouse/ClickHouse/issues/9209) + [#9344](https://github.com/ClickHouse/ClickHouse/pull/9344) [(阿列克谢-米洛维多夫)](https://github.com/alexey-milovidov) +- 如果将数据目录挂载到单独的设备,则修复可用空间量计算不正确时的问题。 对于默认磁盘,计算数据子目录的可用空间。 这修复 [#7441](https://github.com/ClickHouse/ClickHouse/issues/7441) + [#9257](https://github.com/ClickHouse/ClickHouse/pull/9257) [(米尔布)](https://github.com/millb) - 修复TLS连接可能会失败并显示消息时的问题 `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.` 将OpenSSL更新到上游主机。 - [\#8956](https://github.com/ClickHouse/ClickHouse/pull/8956) [(阿列克谢-米洛维多夫)](https://github.com/alexey-milovidov) -- 执行时 `CREATE` 查询,在存储引擎参数中折叠常量表达式。 将空数据库名称替换为当前数据库。 修复 [\#6508](https://github.com/ClickHouse/ClickHouse/issues/6508), [\#3492](https://github.com/ClickHouse/ClickHouse/issues/3492). 还修复了ClickHouseDictionarySource中检查本地地址。 - [\#9262](https://github.com/ClickHouse/ClickHouse/pull/9262) [(tabplubix)](https://github.com/tavplubix) + [#8956](https://github.com/ClickHouse/ClickHouse/pull/8956) [(阿列克谢-米洛维多夫)](https://github.com/alexey-milovidov) +- 执行时 `CREATE` 查询,在存储引擎参数中折叠常量表达式。 将空数据库名称替换为当前数据库。 修复 [#6508](https://github.com/ClickHouse/ClickHouse/issues/6508), [#3492](https://github.com/ClickHouse/ClickHouse/issues/3492). 还修复了ClickHouseDictionarySource中检查本地地址。 + [#9262](https://github.com/ClickHouse/ClickHouse/pull/9262) [(tabplubix)](https://github.com/tavplubix) - 修复段错误 `StorageMerge`,从StorageFile读取时可能发生。 - [\#9387](https://github.com/ClickHouse/ClickHouse/pull/9387) [(tabplubix)](https://github.com/tavplubix) -- 防止丢失数据 `Kafka` 在极少数情况下,在读取后缀之后但在提交之前发生异常。 修复 [\#9378](https://github.com/ClickHouse/ClickHouse/issues/9378). 相关: [\#7175](https://github.com/ClickHouse/ClickHouse/issues/7175) - [\#9507](https://github.com/ClickHouse/ClickHouse/pull/9507) [(菲利蒙诺夫)](https://github.com/filimonov) -- 修复尝试使用/删除时导致服务器终止的错误 `Kafka` 使用错误的参数创建的表。 修复 [\#9494](https://github.com/ClickHouse/ClickHouse/issues/9494). 结合 [\#9507](https://github.com/ClickHouse/ClickHouse/issues/9507). - [\#9513](https://github.com/ClickHouse/ClickHouse/pull/9513) [(菲利蒙诺夫)](https://github.com/filimonov) + [#9387](https://github.com/ClickHouse/ClickHouse/pull/9387) [(tabplubix)](https://github.com/tavplubix) +- 防止丢失数据 `Kafka` 在极少数情况下,在读取后缀之后但在提交之前发生异常。 修复 [#9378](https://github.com/ClickHouse/ClickHouse/issues/9378). 相关: [#7175](https://github.com/ClickHouse/ClickHouse/issues/7175) + [#9507](https://github.com/ClickHouse/ClickHouse/pull/9507) [(菲利蒙诺夫)](https://github.com/filimonov) +- 修复尝试使用/删除时导致服务器终止的错误 `Kafka` 使用错误的参数创建的表。 修复 [#9494](https://github.com/ClickHouse/ClickHouse/issues/9494). 结合 [#9507](https://github.com/ClickHouse/ClickHouse/issues/9507). + [#9513](https://github.com/ClickHouse/ClickHouse/pull/9513) [(菲利蒙诺夫)](https://github.com/filimonov) #### 新功能 {#new-feature-1} - 添加 `deduplicate_blocks_in_dependent_materialized_views` 用于控制具有实例化视图的表中幂等插入的行为的选项。 这个新功能是由Altinity的特殊要求添加到错误修正版本中的。 - [\#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) [(urykhy)](https://github.com/urykhy) + [#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) [(urykhy)](https://github.com/urykhy) ### ClickHouse版本v20.1.2.4,2020-01-22 {#clickhouse-release-v20-1-2-4-2020-01-22} #### 向后不兼容的更改 {#backward-incompatible-change-1} -- 使设置 `merge_tree_uniform_read_distribution` 过时了 服务器仍可识别此设置,但无效。 [\#8308](https://github.com/ClickHouse/ClickHouse/pull/8308) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 更改函数的返回类型 `greatCircleDistance` 到 `Float32` 因为现在计算的结果是 `Float32`. [\#7993](https://github.com/ClickHouse/ClickHouse/pull/7993) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 现在预计查询参数表示为 “escaped” 格式。 例如,要传递字符串 `ab` 你必须写 `a\tb` 或 `a\b` 并分别, `a%5Ctb` 或 `a%5C%09b` 在URL中。 这是需要添加传递NULL作为的可能性 `\N`. 这修复 [\#7488](https://github.com/ClickHouse/ClickHouse/issues/7488). [\#8517](https://github.com/ClickHouse/ClickHouse/pull/8517) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 启用 `use_minimalistic_part_header_in_zookeeper` 设置 `ReplicatedMergeTree` 默认情况下。 这将显着减少存储在ZooKeeper中的数据量。 自19.1版本以来支持此设置,我们已经在多个服务的生产中使用它,半年以上没有任何问题。 如果您有机会降级到19.1以前的版本,请禁用此设置。 [\#6850](https://github.com/ClickHouse/ClickHouse/pull/6850) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 数据跳过索引已准备就绪并默认启用。 设置 `allow_experimental_data_skipping_indices`, `allow_experimental_cross_to_join_conversion` 和 `allow_experimental_multiple_joins_emulation` 现在已经过时,什么也不做。 [\#7974](https://github.com/ClickHouse/ClickHouse/pull/7974) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 添加新建 `ANY JOIN` 逻辑 `StorageJoin` 符合 `JOIN` 操作。 要在不改变行为的情况下进行升级,您需要添加 `SETTINGS any_join_distinct_right_table_keys = 1` 引擎联接表元数据或在升级后重新创建这些表。 [\#8400](https://github.com/ClickHouse/ClickHouse/pull/8400) ([Artem Zuikov](https://github.com/4ertus2)) -- 要求重新启动服务器以应用日志记录配置中的更改。 这是一种临时解决方法,可以避免服务器将日志记录到已删除的日志文件中的错误(请参阅 [\#8696](https://github.com/ClickHouse/ClickHouse/issues/8696)). [\#8707](https://github.com/ClickHouse/ClickHouse/pull/8707) ([Alexander Kuzmenkov](https://github.com/akuzm)) +- 使设置 `merge_tree_uniform_read_distribution` 过时了 服务器仍可识别此设置,但无效。 [#8308](https://github.com/ClickHouse/ClickHouse/pull/8308) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 更改函数的返回类型 `greatCircleDistance` 到 `Float32` 因为现在计算的结果是 `Float32`. [#7993](https://github.com/ClickHouse/ClickHouse/pull/7993) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 现在预计查询参数表示为 “escaped” 格式。 例如,要传递字符串 `ab` 你必须写 `a\tb` 或 `a\b` 并分别, `a%5Ctb` 或 `a%5C%09b` 在URL中。 这是需要添加传递NULL作为的可能性 `\N`. 这修复 [#7488](https://github.com/ClickHouse/ClickHouse/issues/7488). [#8517](https://github.com/ClickHouse/ClickHouse/pull/8517) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 启用 `use_minimalistic_part_header_in_zookeeper` 设置 `ReplicatedMergeTree` 默认情况下。 这将显着减少存储在ZooKeeper中的数据量。 自19.1版本以来支持此设置,我们已经在多个服务的生产中使用它,半年以上没有任何问题。 如果您有机会降级到19.1以前的版本,请禁用此设置。 [#6850](https://github.com/ClickHouse/ClickHouse/pull/6850) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 数据跳过索引已准备就绪并默认启用。 设置 `allow_experimental_data_skipping_indices`, `allow_experimental_cross_to_join_conversion` 和 `allow_experimental_multiple_joins_emulation` 现在已经过时,什么也不做。 [#7974](https://github.com/ClickHouse/ClickHouse/pull/7974) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 添加新建 `ANY JOIN` 逻辑 `StorageJoin` 符合 `JOIN` 操作。 要在不改变行为的情况下进行升级,您需要添加 `SETTINGS any_join_distinct_right_table_keys = 1` 引擎联接表元数据或在升级后重新创建这些表。 [#8400](https://github.com/ClickHouse/ClickHouse/pull/8400) ([Artem Zuikov](https://github.com/4ertus2)) +- 要求重新启动服务器以应用日志记录配置中的更改。 这是一种临时解决方法,可以避免服务器将日志记录到已删除的日志文件中的错误(请参阅 [#8696](https://github.com/ClickHouse/ClickHouse/issues/8696)). [#8707](https://github.com/ClickHouse/ClickHouse/pull/8707) ([Alexander Kuzmenkov](https://github.com/akuzm)) #### 新功能 {#new-feature-2} -- 添加了有关部件路径的信息 `system.merges`. [\#8043](https://github.com/ClickHouse/ClickHouse/pull/8043) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 添加执行能力 `SYSTEM RELOAD DICTIONARY` 查询中 `ON CLUSTER` 模式 [\#8288](https://github.com/ClickHouse/ClickHouse/pull/8288) ([纪尧姆\*塔瑟里](https://github.com/YiuRULE)) -- 添加执行能力 `CREATE DICTIONARY` 查询中 `ON CLUSTER` 模式 [\#8163](https://github.com/ClickHouse/ClickHouse/pull/8163) ([阿利沙平](https://github.com/alesapin)) -- 现在用户的个人资料 `users.xml` 可以继承多个配置文件。 [\#8343](https://github.com/ClickHouse/ClickHouse/pull/8343) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) -- 已添加 `system.stack_trace` 允许查看所有服务器线程的堆栈跟踪的表。 这对于开发人员反省服务器状态非常有用。 这修复 [\#7576](https://github.com/ClickHouse/ClickHouse/issues/7576). [\#8344](https://github.com/ClickHouse/ClickHouse/pull/8344) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 添加 `DateTime64` 具有可配置子秒精度的数据类型。 [\#7170](https://github.com/ClickHouse/ClickHouse/pull/7170) ([瓦西里\*内姆科夫](https://github.com/Enmk)) -- 添加表函数 `clusterAllReplicas` 这允许查询集群中的所有节点。 [\#8493](https://github.com/ClickHouse/ClickHouse/pull/8493) ([kiran sunkari](https://github.com/kiransunkari)) -- 添加聚合函数 `categoricalInformationValue` 其计算出离散特征的信息值。 [\#8117](https://github.com/ClickHouse/ClickHouse/pull/8117) ([hcz](https://github.com/hczhcz)) -- 加快数据文件的解析 `CSV`, `TSV` 和 `JSONEachRow` 通过并行进行格式化。 [\#7780](https://github.com/ClickHouse/ClickHouse/pull/7780) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- 添加功能 `bankerRound` 它执行银行家的四舍五入。 [\#8112](https://github.com/ClickHouse/ClickHouse/pull/8112) ([hcz](https://github.com/hczhcz)) -- 支持区域名称的嵌入式字典中的更多语言: ‘ru’, ‘en’, ‘ua’, ‘uk’, ‘by’, ‘kz’, ‘tr’, ‘de’, ‘uz’, ‘lv’, ‘lt’, ‘et’, ‘pt’, ‘he’, ‘vi’. [\#8189](https://github.com/ClickHouse/ClickHouse/pull/8189) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 改进的一致性 `ANY JOIN` 逻辑 现在 `t1 ANY LEFT JOIN t2` 等于 `t2 ANY RIGHT JOIN t1`. [\#7665](https://github.com/ClickHouse/ClickHouse/pull/7665) ([Artem Zuikov](https://github.com/4ertus2)) -- 添加设置 `any_join_distinct_right_table_keys` 这使旧的行为 `ANY INNER JOIN`. [\#7665](https://github.com/ClickHouse/ClickHouse/pull/7665) ([Artem Zuikov](https://github.com/4ertus2)) -- 添加新建 `SEMI` 和 `ANTI JOIN`. 老 `ANY INNER JOIN` 行为现在可作为 `SEMI LEFT JOIN`. [\#7665](https://github.com/ClickHouse/ClickHouse/pull/7665) ([Artem Zuikov](https://github.com/4ertus2)) -- 已添加 `Distributed` 格式 `File` 发动机和 `file` 表函数,它允许从读 `.bin` 通过异步插入生成的文件 `Distributed` 桌子 [\#8535](https://github.com/ClickHouse/ClickHouse/pull/8535) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 添加可选的重置列参数 `runningAccumulate` 这允许为每个新的键值重置聚合结果。 [\#8326](https://github.com/ClickHouse/ClickHouse/pull/8326) ([谢尔盖\*科诺年科](https://github.com/kononencheg)) -- 添加使用ClickHouse作为普罗米修斯端点的能力。 [\#7900](https://github.com/ClickHouse/ClickHouse/pull/7900) ([vdimir](https://github.com/Vdimir)) -- 添加部分 `` 在 `config.xml` 这将限制允许的主机用于远程表引擎和表函数 `URL`, `S3`, `HDFS`. [\#7154](https://github.com/ClickHouse/ClickHouse/pull/7154) ([米哈伊尔\*科罗托夫](https://github.com/millb)) -- 添加功能 `greatCircleAngle` 它计算球体上的距离(以度为单位)。 [\#8105](https://github.com/ClickHouse/ClickHouse/pull/8105) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 改变地球半径与h3库一致。 [\#8105](https://github.com/ClickHouse/ClickHouse/pull/8105) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 已添加 `JSONCompactEachRow` 和 `JSONCompactEachRowWithNamesAndTypes` 输入和输出格式。 [\#7841](https://github.com/ClickHouse/ClickHouse/pull/7841) ([米哈伊尔\*科罗托夫](https://github.com/millb)) -- 增加了与文件相关的表引擎和表函数的功能 (`File`, `S3`, `URL`, `HDFS`)它允许读取和写入 `gzip` 基于附加引擎参数或文件扩展名的文件。 [\#7840](https://github.com/ClickHouse/ClickHouse/pull/7840) ([安德烈\*博德罗夫](https://github.com/apbodrov)) -- 添加了 `randomASCII(length)` 函数,生成一个字符串与一个随机集 [ASCII](https://en.wikipedia.org/wiki/ASCII#Printable_characters) 可打印字符。 [\#8401](https://github.com/ClickHouse/ClickHouse/pull/8401) ([刺刀](https://github.com/BayoNet)) -- 添加功能 `JSONExtractArrayRaw` 它返回从未解析的json数组元素上的数组 `JSON` 字符串。 [\#8081](https://github.com/ClickHouse/ClickHouse/pull/8081) ([Oleg Matrokhin](https://github.com/errx)) -- 添加 `arrayZip` 函数允许将多个长度相等的数组合成一个元组数组。 [\#8149](https://github.com/ClickHouse/ClickHouse/pull/8149) ([张冬](https://github.com/zhang2014)) -- 添加根据配置的磁盘之间移动数据的能力 `TTL`-表达式为 `*MergeTree` 表引擎家族. [\#8140](https://github.com/ClickHouse/ClickHouse/pull/8140) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 增加了新的聚合功能 `avgWeighted` 其允许计算加权平均值。 [\#7898](https://github.com/ClickHouse/ClickHouse/pull/7898) ([安德烈\*博德罗夫](https://github.com/apbodrov)) -- 现在并行解析默认启用 `TSV`, `TSKV`, `CSV` 和 `JSONEachRow` 格式。 [\#7894](https://github.com/ClickHouse/ClickHouse/pull/7894) ([尼基塔\*米哈伊洛夫](https://github.com/nikitamikhaylov)) -- 从添加几个地理功能 `H3` 图书馆: `h3GetResolution`, `h3EdgeAngle`, `h3EdgeLength`, `h3IsValid` 和 `h3kRing`. [\#8034](https://github.com/ClickHouse/ClickHouse/pull/8034) ([Konstantin Malanchev](https://github.com/hombit)) -- 增加了对brotli的支持 (`br`)压缩文件相关的存储和表函数。 这修复 [\#8156](https://github.com/ClickHouse/ClickHouse/issues/8156). [\#8526](https://github.com/ClickHouse/ClickHouse/pull/8526) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 添加 `groupBit*` 功能的 `SimpleAggregationFunction` 类型。 [\#8485](https://github.com/ClickHouse/ClickHouse/pull/8485) ([纪尧姆\*塔瑟里](https://github.com/YiuRULE)) +- 添加了有关部件路径的信息 `system.merges`. [#8043](https://github.com/ClickHouse/ClickHouse/pull/8043) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 添加执行能力 `SYSTEM RELOAD DICTIONARY` 查询中 `ON CLUSTER` 模式 [#8288](https://github.com/ClickHouse/ClickHouse/pull/8288) ([纪尧姆\*塔瑟里](https://github.com/YiuRULE)) +- 添加执行能力 `CREATE DICTIONARY` 查询中 `ON CLUSTER` 模式 [#8163](https://github.com/ClickHouse/ClickHouse/pull/8163) ([阿利沙平](https://github.com/alesapin)) +- 现在用户的个人资料 `users.xml` 可以继承多个配置文件。 [#8343](https://github.com/ClickHouse/ClickHouse/pull/8343) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) +- 已添加 `system.stack_trace` 允许查看所有服务器线程的堆栈跟踪的表。 这对于开发人员反省服务器状态非常有用。 这修复 [#7576](https://github.com/ClickHouse/ClickHouse/issues/7576). [#8344](https://github.com/ClickHouse/ClickHouse/pull/8344) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 添加 `DateTime64` 具有可配置子秒精度的数据类型。 [#7170](https://github.com/ClickHouse/ClickHouse/pull/7170) ([瓦西里\*内姆科夫](https://github.com/Enmk)) +- 添加表函数 `clusterAllReplicas` 这允许查询集群中的所有节点。 [#8493](https://github.com/ClickHouse/ClickHouse/pull/8493) ([kiran sunkari](https://github.com/kiransunkari)) +- 添加聚合函数 `categoricalInformationValue` 其计算出离散特征的信息值。 [#8117](https://github.com/ClickHouse/ClickHouse/pull/8117) ([hcz](https://github.com/hczhcz)) +- 加快数据文件的解析 `CSV`, `TSV` 和 `JSONEachRow` 通过并行进行格式化。 [#7780](https://github.com/ClickHouse/ClickHouse/pull/7780) ([Alexander Kuzmenkov](https://github.com/akuzm)) +- 添加功能 `bankerRound` 它执行银行家的四舍五入。 [#8112](https://github.com/ClickHouse/ClickHouse/pull/8112) ([hcz](https://github.com/hczhcz)) +- 支持区域名称的嵌入式字典中的更多语言: ‘ru’, ‘en’, ‘ua’, ‘uk’, ‘by’, ‘kz’, ‘tr’, ‘de’, ‘uz’, ‘lv’, ‘lt’, ‘et’, ‘pt’, ‘he’, ‘vi’. [#8189](https://github.com/ClickHouse/ClickHouse/pull/8189) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 改进的一致性 `ANY JOIN` 逻辑 现在 `t1 ANY LEFT JOIN t2` 等于 `t2 ANY RIGHT JOIN t1`. [#7665](https://github.com/ClickHouse/ClickHouse/pull/7665) ([Artem Zuikov](https://github.com/4ertus2)) +- 添加设置 `any_join_distinct_right_table_keys` 这使旧的行为 `ANY INNER JOIN`. [#7665](https://github.com/ClickHouse/ClickHouse/pull/7665) ([Artem Zuikov](https://github.com/4ertus2)) +- 添加新建 `SEMI` 和 `ANTI JOIN`. 老 `ANY INNER JOIN` 行为现在可作为 `SEMI LEFT JOIN`. [#7665](https://github.com/ClickHouse/ClickHouse/pull/7665) ([Artem Zuikov](https://github.com/4ertus2)) +- 已添加 `Distributed` 格式 `File` 发动机和 `file` 表函数,它允许从读 `.bin` 通过异步插入生成的文件 `Distributed` 桌子 [#8535](https://github.com/ClickHouse/ClickHouse/pull/8535) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 添加可选的重置列参数 `runningAccumulate` 这允许为每个新的键值重置聚合结果。 [#8326](https://github.com/ClickHouse/ClickHouse/pull/8326) ([谢尔盖\*科诺年科](https://github.com/kononencheg)) +- 添加使用ClickHouse作为普罗米修斯端点的能力。 [#7900](https://github.com/ClickHouse/ClickHouse/pull/7900) ([vdimir](https://github.com/Vdimir)) +- 添加部分 `` 在 `config.xml` 这将限制允许的主机用于远程表引擎和表函数 `URL`, `S3`, `HDFS`. [#7154](https://github.com/ClickHouse/ClickHouse/pull/7154) ([米哈伊尔\*科罗托夫](https://github.com/millb)) +- 添加功能 `greatCircleAngle` 它计算球体上的距离(以度为单位)。 [#8105](https://github.com/ClickHouse/ClickHouse/pull/8105) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 改变地球半径与h3库一致。 [#8105](https://github.com/ClickHouse/ClickHouse/pull/8105) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 已添加 `JSONCompactEachRow` 和 `JSONCompactEachRowWithNamesAndTypes` 输入和输出格式。 [#7841](https://github.com/ClickHouse/ClickHouse/pull/7841) ([米哈伊尔\*科罗托夫](https://github.com/millb)) +- 增加了与文件相关的表引擎和表函数的功能 (`File`, `S3`, `URL`, `HDFS`)它允许读取和写入 `gzip` 基于附加引擎参数或文件扩展名的文件。 [#7840](https://github.com/ClickHouse/ClickHouse/pull/7840) ([安德烈\*博德罗夫](https://github.com/apbodrov)) +- 添加了 `randomASCII(length)` 函数,生成一个字符串与一个随机集 [ASCII](https://en.wikipedia.org/wiki/ASCII#Printable_characters) 可打印字符。 [#8401](https://github.com/ClickHouse/ClickHouse/pull/8401) ([刺刀](https://github.com/BayoNet)) +- 添加功能 `JSONExtractArrayRaw` 它返回从未解析的json数组元素上的数组 `JSON` 字符串。 [#8081](https://github.com/ClickHouse/ClickHouse/pull/8081) ([Oleg Matrokhin](https://github.com/errx)) +- 添加 `arrayZip` 函数允许将多个长度相等的数组合成一个元组数组。 [#8149](https://github.com/ClickHouse/ClickHouse/pull/8149) ([张冬](https://github.com/zhang2014)) +- 添加根据配置的磁盘之间移动数据的能力 `TTL`-表达式为 `*MergeTree` 表引擎家族. [#8140](https://github.com/ClickHouse/ClickHouse/pull/8140) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 增加了新的聚合功能 `avgWeighted` 其允许计算加权平均值。 [#7898](https://github.com/ClickHouse/ClickHouse/pull/7898) ([安德烈\*博德罗夫](https://github.com/apbodrov)) +- 现在并行解析默认启用 `TSV`, `TSKV`, `CSV` 和 `JSONEachRow` 格式。 [#7894](https://github.com/ClickHouse/ClickHouse/pull/7894) ([尼基塔\*米哈伊洛夫](https://github.com/nikitamikhaylov)) +- 从添加几个地理功能 `H3` 图书馆: `h3GetResolution`, `h3EdgeAngle`, `h3EdgeLength`, `h3IsValid` 和 `h3kRing`. [#8034](https://github.com/ClickHouse/ClickHouse/pull/8034) ([Konstantin Malanchev](https://github.com/hombit)) +- 增加了对brotli的支持 (`br`)压缩文件相关的存储和表函数。 这修复 [#8156](https://github.com/ClickHouse/ClickHouse/issues/8156). [#8526](https://github.com/ClickHouse/ClickHouse/pull/8526) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 添加 `groupBit*` 功能的 `SimpleAggregationFunction` 类型。 [#8485](https://github.com/ClickHouse/ClickHouse/pull/8485) ([纪尧姆\*塔瑟里](https://github.com/YiuRULE)) #### 错误修复 {#bug-fix-6} -- 修复重命名表 `Distributed` 引擎 修复问题 [\#7868](https://github.com/ClickHouse/ClickHouse/issues/7868). [\#8306](https://github.com/ClickHouse/ClickHouse/pull/8306) ([tavplubix](https://github.com/tavplubix)) -- 现在字典支持 `EXPRESSION` 对于非ClickHouse SQL方言中任意字符串中的属性。 [\#8098](https://github.com/ClickHouse/ClickHouse/pull/8098) ([阿利沙平](https://github.com/alesapin)) -- 修复损坏 `INSERT SELECT FROM mysql(...)` 查询。 这修复 [\#8070](https://github.com/ClickHouse/ClickHouse/issues/8070) 和 [\#7960](https://github.com/ClickHouse/ClickHouse/issues/7960). [\#8234](https://github.com/ClickHouse/ClickHouse/pull/8234) ([tavplubix](https://github.com/tavplubix)) -- 修复错误 “Mismatch column sizes” 插入默认值时 `Tuple` 从 `JSONEachRow`. 这修复 [\#5653](https://github.com/ClickHouse/ClickHouse/issues/5653). [\#8606](https://github.com/ClickHouse/ClickHouse/pull/8606) ([tavplubix](https://github.com/tavplubix)) -- 现在将在使用的情况下抛出一个异常 `WITH TIES` 旁边的 `LIMIT BY`. 还增加了使用能力 `TOP` 与 `LIMIT BY`. 这修复 [\#7472](https://github.com/ClickHouse/ClickHouse/issues/7472). [\#7637](https://github.com/ClickHouse/ClickHouse/pull/7637) ([尼基塔\*米哈伊洛夫](https://github.com/nikitamikhaylov)) -- 从新鲜的glibc版本中修复unintendent依赖关系 `clickhouse-odbc-bridge` 二进制 [\#8046](https://github.com/ClickHouse/ClickHouse/pull/8046) ([阿莫斯鸟](https://github.com/amosbird)) -- 修正错误的检查功能 `*MergeTree` 引擎家族. 现在,当我们在最后一个颗粒和最后一个标记(非最终)中有相同数量的行时,它不会失败。 [\#8047](https://github.com/ClickHouse/ClickHouse/pull/8047) ([阿利沙平](https://github.com/alesapin)) -- 修复插入 `Enum*` 列后 `ALTER` 查询,当基础数值类型等于表指定类型时。 这修复 [\#7836](https://github.com/ClickHouse/ClickHouse/issues/7836). [\#7908](https://github.com/ClickHouse/ClickHouse/pull/7908) ([安东\*波波夫](https://github.com/CurtizJ)) -- 允许非常数负 “size” 函数的参数 `substring`. 这是不允许的错误。 这修复 [\#4832](https://github.com/ClickHouse/ClickHouse/issues/4832). [\#7703](https://github.com/ClickHouse/ClickHouse/pull/7703) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修复当错误数量的参数传递到解析错误 `(O|J)DBC` 表引擎。 [\#7709](https://github.com/ClickHouse/ClickHouse/pull/7709) ([阿利沙平](https://github.com/alesapin)) -- 将日志发送到syslog时使用正在运行的clickhouse进程的命令名。 在以前的版本中,使用空字符串而不是命令名称。 [\#8460](https://github.com/ClickHouse/ClickHouse/pull/8460) ([Michael Nacharov](https://github.com/mnach)) -- 修复检查允许的主机 `localhost`. 这个公关修复了在提供的解决方案 [\#8241](https://github.com/ClickHouse/ClickHouse/pull/8241). [\#8342](https://github.com/ClickHouse/ClickHouse/pull/8342) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) -- 修复罕见的崩溃 `argMin` 和 `argMax` 长字符串参数的函数,当结果被用于 `runningAccumulate` 功能。 这修复 [\#8325](https://github.com/ClickHouse/ClickHouse/issues/8325) [\#8341](https://github.com/ClickHouse/ClickHouse/pull/8341) ([恐龙](https://github.com/769344359)) -- 修复表的内存过度使用 `Buffer` 引擎 [\#8345](https://github.com/ClickHouse/ClickHouse/pull/8345) ([Azat Khuzhin](https://github.com/azat)) -- 修正了可以采取的功能中的潜在错误 `NULL` 作为参数之一,并返回非NULL。 [\#8196](https://github.com/ClickHouse/ClickHouse/pull/8196) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 在线程池中更好地计算后台进程的指标 `MergeTree` 表引擎. [\#8194](https://github.com/ClickHouse/ClickHouse/pull/8194) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 修复功能 `IN` 里面 `WHERE` 存在行级表筛选器时的语句。 修复 [\#6687](https://github.com/ClickHouse/ClickHouse/issues/6687) [\#8357](https://github.com/ClickHouse/ClickHouse/pull/8357) ([伊万](https://github.com/abyss7)) -- 现在,如果整数值没有完全解析设置值,则会引发异常。 [\#7678](https://github.com/ClickHouse/ClickHouse/pull/7678) ([米哈伊尔\*科罗托夫](https://github.com/millb)) -- 修复当聚合函数用于查询具有两个以上本地分片的分布式表时出现的异常。 [\#8164](https://github.com/ClickHouse/ClickHouse/pull/8164) ([小路](https://github.com/nicelulu)) -- 现在,bloom filter可以处理零长度数组,并且不执行冗余计算。 [\#8242](https://github.com/ClickHouse/ClickHouse/pull/8242) ([achimbab](https://github.com/achimbab)) -- 修正了通过匹配客户端主机来检查客户端主机是否允许 `host_regexp` 在指定 `users.xml`. [\#8241](https://github.com/ClickHouse/ClickHouse/pull/8241) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) -- 放松不明确的列检查,导致多个误报 `JOIN ON` 科。 [\#8385](https://github.com/ClickHouse/ClickHouse/pull/8385) ([Artem Zuikov](https://github.com/4ertus2)) -- 修正了可能的服务器崩溃 (`std::terminate`)当服务器不能发送或写入数据 `JSON` 或 `XML` 格式与值 `String` 数据类型(需要 `UTF-8` 验证)或使用Brotli算法或其他一些罕见情况下压缩结果数据时。 这修复 [\#7603](https://github.com/ClickHouse/ClickHouse/issues/7603) [\#8384](https://github.com/ClickHouse/ClickHouse/pull/8384) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修复竞争条件 `StorageDistributedDirectoryMonitor` 被线人发现 这修复 [\#8364](https://github.com/ClickHouse/ClickHouse/issues/8364). [\#8383](https://github.com/ClickHouse/ClickHouse/pull/8383) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 现在背景合并 `*MergeTree` 表引擎家族更准确地保留存储策略卷顺序。 [\#8549](https://github.com/ClickHouse/ClickHouse/pull/8549) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 现在表引擎 `Kafka` 与正常工作 `Native` 格式。 这修复 [\#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)) -- 固定格式与标题(如 `CSVWithNames`)这是抛出关于EOF表引擎的异常 `Kafka`. [\#8016](https://github.com/ClickHouse/ClickHouse/pull/8016) ([filimonov](https://github.com/filimonov)) -- 修复了从子查询右侧部分制作set的错误 `IN` 科。 这修复 [\#5767](https://github.com/ClickHouse/ClickHouse/issues/5767) 和 [\#2542](https://github.com/ClickHouse/ClickHouse/issues/2542). [\#7755](https://github.com/ClickHouse/ClickHouse/pull/7755) ([尼基塔\*米哈伊洛夫](https://github.com/nikitamikhaylov)) -- 从存储读取时修复可能的崩溃 `File`. [\#7756](https://github.com/ClickHouse/ClickHouse/pull/7756) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 在固定的文件读取 `Parquet` 包含类型列的格式 `list`. [\#8334](https://github.com/ClickHouse/ClickHouse/pull/8334) ([马苏兰](https://github.com/maxulan)) -- 修复错误 `Not found column` 对于分布式查询 `PREWHERE` 条件取决于采样键if `max_parallel_replicas > 1`. [\#7913](https://github.com/ClickHouse/ClickHouse/pull/7913) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 修复错误 `Not found column` 如果使用查询 `PREWHERE` 依赖于表的别名,结果集由于主键条件而为空。 [\#7911](https://github.com/ClickHouse/ClickHouse/pull/7911) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 函数的固定返回类型 `rand` 和 `randConstant` 在情况下 `Nullable` 争论。 现在函数总是返回 `UInt32` 而且从来没有 `Nullable(UInt32)`. [\#8204](https://github.com/ClickHouse/ClickHouse/pull/8204) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 禁用谓词下推 `WITH FILL` 表达。 这修复 [\#7784](https://github.com/ClickHouse/ClickHouse/issues/7784). [\#7789](https://github.com/ClickHouse/ClickHouse/pull/7789) ([张冬](https://github.com/zhang2014)) -- 修正错误 `count()` 结果 `SummingMergeTree` 当 `FINAL` 部分被使用。 [\#3280](https://github.com/ClickHouse/ClickHouse/issues/3280) [\#7786](https://github.com/ClickHouse/ClickHouse/pull/7786) ([尼基塔\*米哈伊洛夫](https://github.com/nikitamikhaylov)) -- 修复来自远程服务器的常量函数可能不正确的结果。 它发生在具有以下功能的查询中 `version()`, `uptime()` 等。 它为不同的服务器返回不同的常量值。 这修复 [\#7666](https://github.com/ClickHouse/ClickHouse/issues/7666). [\#7689](https://github.com/ClickHouse/ClickHouse/pull/7689) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 修复下推谓词优化中导致错误结果的复杂错误。 这解决了下推谓词优化的很多问题。 [\#8503](https://github.com/ClickHouse/ClickHouse/pull/8503) ([张冬](https://github.com/zhang2014)) -- 修复崩溃 `CREATE TABLE .. AS dictionary` 查询。 [\#8508](https://github.com/ClickHouse/ClickHouse/pull/8508) ([Azat Khuzhin](https://github.com/azat)) -- 一些改进ClickHouse语法 `.g4` 文件 [\#8294](https://github.com/ClickHouse/ClickHouse/pull/8294) ([太阳里](https://github.com/taiyang-li)) -- 修复导致崩溃的错误 `JOIN`s与表与发动机 `Join`. 这修复 [\#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)) -- 修复冗余字典重新加载 `CREATE DATABASE`. [\#7916](https://github.com/ClickHouse/ClickHouse/pull/7916) ([Azat Khuzhin](https://github.com/azat)) -- 限制从读取流的最大数量 `StorageFile` 和 `StorageHDFS`. 修复https://github.com/ClickHouse/ClickHouse/issues/7650. [\#7981](https://github.com/ClickHouse/ClickHouse/pull/7981) ([阿利沙平](https://github.com/alesapin)) -- 修复bug `ALTER ... MODIFY ... CODEC` 查询,当用户同时指定默认表达式和编解ec。 修复 [8593](https://github.com/ClickHouse/ClickHouse/issues/8593). [\#8614](https://github.com/ClickHouse/ClickHouse/pull/8614) ([阿利沙平](https://github.com/alesapin)) -- 修复列的后台合并错误 `SimpleAggregateFunction(LowCardinality)` 类型。 [\#8613](https://github.com/ClickHouse/ClickHouse/pull/8613) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 固定类型签入功能 `toDateTime64`. [\#8375](https://github.com/ClickHouse/ClickHouse/pull/8375) ([瓦西里\*内姆科夫](https://github.com/Enmk)) -- 现在服务器不崩溃 `LEFT` 或 `FULL JOIN` 与和加入引擎和不支持 `join_use_nulls` 设置。 [\#8479](https://github.com/ClickHouse/ClickHouse/pull/8479) ([Artem Zuikov](https://github.com/4ertus2)) -- 现在 `DROP DICTIONARY IF EXISTS db.dict` 查询不会抛出异常,如果 `db` 根本不存在 [\#8185](https://github.com/ClickHouse/ClickHouse/pull/8185) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) -- 修复表函数中可能出现的崩溃 (`file`, `mysql`, `remote`)引用删除引起的 `IStorage` 对象。 修复插入表函数时指定的列的不正确解析。 [\#7762](https://github.com/ClickHouse/ClickHouse/pull/7762) ([tavplubix](https://github.com/tavplubix)) -- 确保网络启动前 `clickhouse-server`. 这修复 [\#7507](https://github.com/ClickHouse/ClickHouse/issues/7507). [\#8570](https://github.com/ClickHouse/ClickHouse/pull/8570) ([余志昌](https://github.com/yuzhichang)) -- 修复安全连接的超时处理,因此查询不会无限挂起。 这修复 [\#8126](https://github.com/ClickHouse/ClickHouse/issues/8126). [\#8128](https://github.com/ClickHouse/ClickHouse/pull/8128) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修复 `clickhouse-copier`并发工人之间的冗余争用。 [\#7816](https://github.com/ClickHouse/ClickHouse/pull/7816) ([丁香飞](https://github.com/dingxiangfei2009)) -- 现在突变不会跳过附加的部分,即使它们的突变版本比当前的突变版本大。 [\#7812](https://github.com/ClickHouse/ClickHouse/pull/7812) ([余志昌](https://github.com/yuzhichang)) [\#8250](https://github.com/ClickHouse/ClickHouse/pull/8250) ([阿利沙平](https://github.com/alesapin)) -- 忽略冗余副本 `*MergeTree` 数据部分移动到另一个磁盘和服务器重新启动后。 [\#7810](https://github.com/ClickHouse/ClickHouse/pull/7810) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 修复崩溃 `FULL JOIN` 与 `LowCardinality` 在 `JOIN` 钥匙 [\#8252](https://github.com/ClickHouse/ClickHouse/pull/8252) ([Artem Zuikov](https://github.com/4ertus2)) -- 禁止在插入查询中多次使用列名,如 `INSERT INTO tbl (x, y, x)`. 这修复 [\#5465](https://github.com/ClickHouse/ClickHouse/issues/5465), [\#7681](https://github.com/ClickHouse/ClickHouse/issues/7681). [\#7685](https://github.com/ClickHouse/ClickHouse/pull/7685) ([阿利沙平](https://github.com/alesapin)) -- 增加了回退,用于检测未知Cpu的物理CPU内核数量(使用逻辑CPU内核数量)。 这修复 [\#5239](https://github.com/ClickHouse/ClickHouse/issues/5239). [\#7726](https://github.com/ClickHouse/ClickHouse/pull/7726) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修复 `There's no column` 实例化列和别名列出错。 [\#8210](https://github.com/ClickHouse/ClickHouse/pull/8210) ([Artem Zuikov](https://github.com/4ertus2)) -- 固定切断崩溃时 `EXISTS` 查询没有使用 `TABLE` 或 `DICTIONARY` 预选赛 就像 `EXISTS t`. 这修复 [\#8172](https://github.com/ClickHouse/ClickHouse/issues/8172). 此错误在版本19.17中引入。 [\#8213](https://github.com/ClickHouse/ClickHouse/pull/8213) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修复罕见错误 `"Sizes of columns doesn't match"` 使用时可能会出现 `SimpleAggregateFunction` 列。 [\#7790](https://github.com/ClickHouse/ClickHouse/pull/7790) ([Boris Granveaud](https://github.com/bgranvea)) -- 修正错误,其中用户空 `allow_databases` 可以访问所有数据库(和相同的 `allow_dictionaries`). [\#7793](https://github.com/ClickHouse/ClickHouse/pull/7793) ([DeifyTheGod](https://github.com/DeifyTheGod)) -- 修复客户端崩溃时,服务器已经从客户端断开连接。 [\#8071](https://github.com/ClickHouse/ClickHouse/pull/8071) ([Azat Khuzhin](https://github.com/azat)) -- 修复 `ORDER BY` 在按主键前缀和非主键后缀排序的情况下的行为。 [\#7759](https://github.com/ClickHouse/ClickHouse/pull/7759) ([安东\*波波夫](https://github.com/CurtizJ)) -- 检查表中是否存在合格列。 这修复 [\#6836](https://github.com/ClickHouse/ClickHouse/issues/6836). [\#7758](https://github.com/ClickHouse/ClickHouse/pull/7758) ([Artem Zuikov](https://github.com/4ertus2)) -- 固定行为 `ALTER MOVE` 合并完成后立即运行移动指定的超部分。 修复 [\#8103](https://github.com/ClickHouse/ClickHouse/issues/8103). [\#8104](https://github.com/ClickHouse/ClickHouse/pull/8104) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 使用时修复可能的服务器崩溃 `UNION` 具有不同数量的列。 修复 [\#7279](https://github.com/ClickHouse/ClickHouse/issues/7279). [\#7929](https://github.com/ClickHouse/ClickHouse/pull/7929) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 修复函数结果子字符串的大小 `substr` 负大小。 [\#8589](https://github.com/ClickHouse/ClickHouse/pull/8589) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 现在服务器不执行部分突变 `MergeTree` 如果后台池中没有足够的可用线程。 [\#8588](https://github.com/ClickHouse/ClickHouse/pull/8588) ([tavplubix](https://github.com/tavplubix)) -- 修复格式化时的小错字 `UNION ALL` AST. [\#7999](https://github.com/ClickHouse/ClickHouse/pull/7999) ([litao91](https://github.com/litao91)) -- 修正了负数不正确的布隆过滤结果。 这修复 [\#8317](https://github.com/ClickHouse/ClickHouse/issues/8317). [\#8566](https://github.com/ClickHouse/ClickHouse/pull/8566) ([张冬](https://github.com/zhang2014)) -- 在解压缩固定潜在的缓冲区溢出。 恶意用户可以传递捏造的压缩数据,这将导致缓冲区后读取。 这个问题是由Yandex信息安全团队的Eldar Zaitov发现的。 [\#8404](https://github.com/ClickHouse/ClickHouse/pull/8404) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修复因整数溢出而导致的错误结果 `arrayIntersect`. [\#7777](https://github.com/ClickHouse/ClickHouse/pull/7777) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 现在 `OPTIMIZE TABLE` query不会等待脱机副本执行该操作。 [\#8314](https://github.com/ClickHouse/ClickHouse/pull/8314) ([javi santana](https://github.com/javisantana)) -- 固定 `ALTER TTL` 解析器 `Replicated*MergeTree` 桌子 [\#8318](https://github.com/ClickHouse/ClickHouse/pull/8318) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 修复服务器和客户端之间的通信,以便服务器在查询失败后读取临时表信息。 [\#8084](https://github.com/ClickHouse/ClickHouse/pull/8084) ([Azat Khuzhin](https://github.com/azat)) -- 修复 `bitmapAnd` 在聚合位图和标量位图相交时出现函数错误。 [\#8082](https://github.com/ClickHouse/ClickHouse/pull/8082) ([黄月](https://github.com/moon03432)) -- 完善的定义 `ZXid` 根据动物园管理员的程序员指南,它修复了错误 `clickhouse-cluster-copier`. [\#8088](https://github.com/ClickHouse/ClickHouse/pull/8088) ([丁香飞](https://github.com/dingxiangfei2009)) -- `odbc` 表函数现在尊重 `external_table_functions_use_nulls` 设置。 [\#7506](https://github.com/ClickHouse/ClickHouse/pull/7506) ([瓦西里\*内姆科夫](https://github.com/Enmk)) -- 修正了导致罕见的数据竞赛的错误。 [\#8143](https://github.com/ClickHouse/ClickHouse/pull/8143) ([亚历山大\*卡扎科夫](https://github.com/Akazz)) -- 现在 `SYSTEM RELOAD DICTIONARY` 完全重新加载字典,忽略 `update_field`. 这修复 [\#7440](https://github.com/ClickHouse/ClickHouse/issues/7440). [\#8037](https://github.com/ClickHouse/ClickHouse/pull/8037) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) -- 添加检查字典是否存在于创建查询的能力。 [\#8032](https://github.com/ClickHouse/ClickHouse/pull/8032) ([阿利沙平](https://github.com/alesapin)) -- 修复 `Float*` 解析中 `Values` 格式。 这修复 [\#7817](https://github.com/ClickHouse/ClickHouse/issues/7817). [\#7870](https://github.com/ClickHouse/ClickHouse/pull/7870) ([tavplubix](https://github.com/tavplubix)) -- 修复崩溃时,我们不能在一些后台操作保留空间 `*MergeTree` 表引擎家族. [\#7873](https://github.com/ClickHouse/ClickHouse/pull/7873) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 修复表包含合并操作时的崩溃 `SimpleAggregateFunction(LowCardinality)` 列。 这修复 [\#8515](https://github.com/ClickHouse/ClickHouse/issues/8515). [\#8522](https://github.com/ClickHouse/ClickHouse/pull/8522) ([Azat Khuzhin](https://github.com/azat)) -- 恢复对所有ICU区域设置的支持,并添加对常量表达式应用排序规则的功能。 还添加语言名称 `system.collations` 桌子 [\#8051](https://github.com/ClickHouse/ClickHouse/pull/8051) ([阿利沙平](https://github.com/alesapin)) -- 修正错误时,外部字典与零最小寿命 (`LIFETIME(MIN 0 MAX N)`, `LIFETIME(N)`)不要在后台更新。 [\#7983](https://github.com/ClickHouse/ClickHouse/pull/7983) ([阿利沙平](https://github.com/alesapin)) -- 修复当clickhouse源外部字典在查询中有子查询时崩溃。 [\#8351](https://github.com/ClickHouse/ClickHouse/pull/8351) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 修复文件扩展名不正确的解析表与引擎 `URL`. 这修复 [\#8157](https://github.com/ClickHouse/ClickHouse/issues/8157). [\#8419](https://github.com/ClickHouse/ClickHouse/pull/8419) ([安德烈\*博德罗夫](https://github.com/apbodrov)) -- 修复 `CHECK TABLE` 查询为 `*MergeTree` 表没有关键. 修复 [\#7543](https://github.com/ClickHouse/ClickHouse/issues/7543). [\#7979](https://github.com/ClickHouse/ClickHouse/pull/7979) ([阿利沙平](https://github.com/alesapin)) -- 固定转换 `Float64` 到MySQL类型。 [\#8079](https://github.com/ClickHouse/ClickHouse/pull/8079) ([尤里\*巴拉诺夫](https://github.com/yurriy)) -- 现在,如果表没有完全删除,因为服务器崩溃,服务器将尝试恢复并加载它。 [\#8176](https://github.com/ClickHouse/ClickHouse/pull/8176) ([tavplubix](https://github.com/tavplubix)) -- 修复了表函数中的崩溃 `file` 同时插入到不存在的文件。 现在在这种情况下,文件将被创建,然后插入将被处理。 [\#8177](https://github.com/ClickHouse/ClickHouse/pull/8177) ([Olga Khvostikova](https://github.com/stavrolia)) -- 修复罕见的死锁时,可能发生 `trace_log` 处于启用状态。 [\#7838](https://github.com/ClickHouse/ClickHouse/pull/7838) ([filimonov](https://github.com/filimonov)) -- 添加能力与不同类型的工作,除了 `Date` 在 `RangeHashed` 从DDL查询创建的外部字典。 修复 [7899](https://github.com/ClickHouse/ClickHouse/issues/7899). [\#8275](https://github.com/ClickHouse/ClickHouse/pull/8275) ([阿利沙平](https://github.com/alesapin)) -- 修复崩溃时 `now64()` 用另一个函数的结果调用。 [\#8270](https://github.com/ClickHouse/ClickHouse/pull/8270) ([瓦西里\*内姆科夫](https://github.com/Enmk)) -- 修正了通过mysql有线协议检测客户端IP连接的错误。 [\#7743](https://github.com/ClickHouse/ClickHouse/pull/7743) ([Dmitry Muzyka](https://github.com/dmitriy-myz)) -- 修复空阵列处理 `arraySplit` 功能。 这修复 [\#7708](https://github.com/ClickHouse/ClickHouse/issues/7708). [\#7747](https://github.com/ClickHouse/ClickHouse/pull/7747) ([hcz](https://github.com/hczhcz)) -- 修复了以下问题 `pid-file` 另一个运行 `clickhouse-server` 可能会被删除。 [\#8487](https://github.com/ClickHouse/ClickHouse/pull/8487) ([徐伟清](https://github.com/weiqxu)) -- 修复字典重新加载,如果它有 `invalidate_query`,停止更新,并在以前的更新尝试一些异常。 [\#8029](https://github.com/ClickHouse/ClickHouse/pull/8029) ([阿利沙平](https://github.com/alesapin)) -- 修正了功能错误 `arrayReduce` 这可能会导致 “double free” 和聚合函数组合器中的错误 `Resample` 这可能会导致内存泄漏。 添加聚合功能 `aggThrow`. 此功能可用于测试目的。 [\#8446](https://github.com/ClickHouse/ClickHouse/pull/8446) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修复重命名表 `Distributed` 引擎 修复问题 [#7868](https://github.com/ClickHouse/ClickHouse/issues/7868). [#8306](https://github.com/ClickHouse/ClickHouse/pull/8306) ([tavplubix](https://github.com/tavplubix)) +- 现在字典支持 `EXPRESSION` 对于非ClickHouse SQL方言中任意字符串中的属性。 [#8098](https://github.com/ClickHouse/ClickHouse/pull/8098) ([阿利沙平](https://github.com/alesapin)) +- 修复损坏 `INSERT SELECT FROM mysql(...)` 查询。 这修复 [#8070](https://github.com/ClickHouse/ClickHouse/issues/8070) 和 [#7960](https://github.com/ClickHouse/ClickHouse/issues/7960). [#8234](https://github.com/ClickHouse/ClickHouse/pull/8234) ([tavplubix](https://github.com/tavplubix)) +- 修复错误 “Mismatch column sizes” 插入默认值时 `Tuple` 从 `JSONEachRow`. 这修复 [#5653](https://github.com/ClickHouse/ClickHouse/issues/5653). [#8606](https://github.com/ClickHouse/ClickHouse/pull/8606) ([tavplubix](https://github.com/tavplubix)) +- 现在将在使用的情况下抛出一个异常 `WITH TIES` 旁边的 `LIMIT BY`. 还增加了使用能力 `TOP` 与 `LIMIT BY`. 这修复 [#7472](https://github.com/ClickHouse/ClickHouse/issues/7472). [#7637](https://github.com/ClickHouse/ClickHouse/pull/7637) ([尼基塔\*米哈伊洛夫](https://github.com/nikitamikhaylov)) +- 从新鲜的glibc版本中修复unintendent依赖关系 `clickhouse-odbc-bridge` 二进制 [#8046](https://github.com/ClickHouse/ClickHouse/pull/8046) ([阿莫斯鸟](https://github.com/amosbird)) +- 修正错误的检查功能 `*MergeTree` 引擎家族. 现在,当我们在最后一个颗粒和最后一个标记(非最终)中有相同数量的行时,它不会失败。 [#8047](https://github.com/ClickHouse/ClickHouse/pull/8047) ([阿利沙平](https://github.com/alesapin)) +- 修复插入 `Enum*` 列后 `ALTER` 查询,当基础数值类型等于表指定类型时。 这修复 [#7836](https://github.com/ClickHouse/ClickHouse/issues/7836). [#7908](https://github.com/ClickHouse/ClickHouse/pull/7908) ([安东\*波波夫](https://github.com/CurtizJ)) +- 允许非常数负 “size” 函数的参数 `substring`. 这是不允许的错误。 这修复 [#4832](https://github.com/ClickHouse/ClickHouse/issues/4832). [#7703](https://github.com/ClickHouse/ClickHouse/pull/7703) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修复当错误数量的参数传递到解析错误 `(O|J)DBC` 表引擎。 [#7709](https://github.com/ClickHouse/ClickHouse/pull/7709) ([阿利沙平](https://github.com/alesapin)) +- 将日志发送到syslog时使用正在运行的clickhouse进程的命令名。 在以前的版本中,使用空字符串而不是命令名称。 [#8460](https://github.com/ClickHouse/ClickHouse/pull/8460) ([Michael Nacharov](https://github.com/mnach)) +- 修复检查允许的主机 `localhost`. 这个公关修复了在提供的解决方案 [#8241](https://github.com/ClickHouse/ClickHouse/pull/8241). [#8342](https://github.com/ClickHouse/ClickHouse/pull/8342) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) +- 修复罕见的崩溃 `argMin` 和 `argMax` 长字符串参数的函数,当结果被用于 `runningAccumulate` 功能。 这修复 [#8325](https://github.com/ClickHouse/ClickHouse/issues/8325) [#8341](https://github.com/ClickHouse/ClickHouse/pull/8341) ([恐龙](https://github.com/769344359)) +- 修复表的内存过度使用 `Buffer` 引擎 [#8345](https://github.com/ClickHouse/ClickHouse/pull/8345) ([Azat Khuzhin](https://github.com/azat)) +- 修正了可以采取的功能中的潜在错误 `NULL` 作为参数之一,并返回非NULL。 [#8196](https://github.com/ClickHouse/ClickHouse/pull/8196) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 在线程池中更好地计算后台进程的指标 `MergeTree` 表引擎. [#8194](https://github.com/ClickHouse/ClickHouse/pull/8194) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 修复功能 `IN` 里面 `WHERE` 存在行级表筛选器时的语句。 修复 [#6687](https://github.com/ClickHouse/ClickHouse/issues/6687) [#8357](https://github.com/ClickHouse/ClickHouse/pull/8357) ([伊万](https://github.com/abyss7)) +- 现在,如果整数值没有完全解析设置值,则会引发异常。 [#7678](https://github.com/ClickHouse/ClickHouse/pull/7678) ([米哈伊尔\*科罗托夫](https://github.com/millb)) +- 修复当聚合函数用于查询具有两个以上本地分片的分布式表时出现的异常。 [#8164](https://github.com/ClickHouse/ClickHouse/pull/8164) ([小路](https://github.com/nicelulu)) +- 现在,bloom filter可以处理零长度数组,并且不执行冗余计算。 [#8242](https://github.com/ClickHouse/ClickHouse/pull/8242) ([achimbab](https://github.com/achimbab)) +- 修正了通过匹配客户端主机来检查客户端主机是否允许 `host_regexp` 在指定 `users.xml`. [#8241](https://github.com/ClickHouse/ClickHouse/pull/8241) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) +- 放松不明确的列检查,导致多个误报 `JOIN ON` 科。 [#8385](https://github.com/ClickHouse/ClickHouse/pull/8385) ([Artem Zuikov](https://github.com/4ertus2)) +- 修正了可能的服务器崩溃 (`std::terminate`)当服务器不能发送或写入数据 `JSON` 或 `XML` 格式与值 `String` 数据类型(需要 `UTF-8` 验证)或使用Brotli算法或其他一些罕见情况下压缩结果数据时。 这修复 [#7603](https://github.com/ClickHouse/ClickHouse/issues/7603) [#8384](https://github.com/ClickHouse/ClickHouse/pull/8384) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修复竞争条件 `StorageDistributedDirectoryMonitor` 被线人发现 这修复 [#8364](https://github.com/ClickHouse/ClickHouse/issues/8364). [#8383](https://github.com/ClickHouse/ClickHouse/pull/8383) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 现在背景合并 `*MergeTree` 表引擎家族更准确地保留存储策略卷顺序。 [#8549](https://github.com/ClickHouse/ClickHouse/pull/8549) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 现在表引擎 `Kafka` 与正常工作 `Native` 格式。 这修复 [#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)) +- 固定格式与标题(如 `CSVWithNames`)这是抛出关于EOF表引擎的异常 `Kafka`. [#8016](https://github.com/ClickHouse/ClickHouse/pull/8016) ([filimonov](https://github.com/filimonov)) +- 修复了从子查询右侧部分制作set的错误 `IN` 科。 这修复 [#5767](https://github.com/ClickHouse/ClickHouse/issues/5767) 和 [#2542](https://github.com/ClickHouse/ClickHouse/issues/2542). [#7755](https://github.com/ClickHouse/ClickHouse/pull/7755) ([尼基塔\*米哈伊洛夫](https://github.com/nikitamikhaylov)) +- 从存储读取时修复可能的崩溃 `File`. [#7756](https://github.com/ClickHouse/ClickHouse/pull/7756) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 在固定的文件读取 `Parquet` 包含类型列的格式 `list`. [#8334](https://github.com/ClickHouse/ClickHouse/pull/8334) ([马苏兰](https://github.com/maxulan)) +- 修复错误 `Not found column` 对于分布式查询 `PREWHERE` 条件取决于采样键if `max_parallel_replicas > 1`. [#7913](https://github.com/ClickHouse/ClickHouse/pull/7913) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 修复错误 `Not found column` 如果使用查询 `PREWHERE` 依赖于表的别名,结果集由于主键条件而为空。 [#7911](https://github.com/ClickHouse/ClickHouse/pull/7911) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 函数的固定返回类型 `rand` 和 `randConstant` 在情况下 `Nullable` 争论。 现在函数总是返回 `UInt32` 而且从来没有 `Nullable(UInt32)`. [#8204](https://github.com/ClickHouse/ClickHouse/pull/8204) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 禁用谓词下推 `WITH FILL` 表达。 这修复 [#7784](https://github.com/ClickHouse/ClickHouse/issues/7784). [#7789](https://github.com/ClickHouse/ClickHouse/pull/7789) ([张冬](https://github.com/zhang2014)) +- 修正错误 `count()` 结果 `SummingMergeTree` 当 `FINAL` 部分被使用。 [#3280](https://github.com/ClickHouse/ClickHouse/issues/3280) [#7786](https://github.com/ClickHouse/ClickHouse/pull/7786) ([尼基塔\*米哈伊洛夫](https://github.com/nikitamikhaylov)) +- 修复来自远程服务器的常量函数可能不正确的结果。 它发生在具有以下功能的查询中 `version()`, `uptime()` 等。 它为不同的服务器返回不同的常量值。 这修复 [#7666](https://github.com/ClickHouse/ClickHouse/issues/7666). [#7689](https://github.com/ClickHouse/ClickHouse/pull/7689) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 修复下推谓词优化中导致错误结果的复杂错误。 这解决了下推谓词优化的很多问题。 [#8503](https://github.com/ClickHouse/ClickHouse/pull/8503) ([张冬](https://github.com/zhang2014)) +- 修复崩溃 `CREATE TABLE .. AS dictionary` 查询。 [#8508](https://github.com/ClickHouse/ClickHouse/pull/8508) ([Azat Khuzhin](https://github.com/azat)) +- 一些改进ClickHouse语法 `.g4` 文件 [#8294](https://github.com/ClickHouse/ClickHouse/pull/8294) ([太阳里](https://github.com/taiyang-li)) +- 修复导致崩溃的错误 `JOIN`s与表与发动机 `Join`. 这修复 [#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)) +- 修复冗余字典重新加载 `CREATE DATABASE`. [#7916](https://github.com/ClickHouse/ClickHouse/pull/7916) ([Azat Khuzhin](https://github.com/azat)) +- 限制从读取流的最大数量 `StorageFile` 和 `StorageHDFS`. 修复https://github.com/ClickHouse/ClickHouse/issues/7650. [#7981](https://github.com/ClickHouse/ClickHouse/pull/7981) ([阿利沙平](https://github.com/alesapin)) +- 修复bug `ALTER ... MODIFY ... CODEC` 查询,当用户同时指定默认表达式和编解ec。 修复 [8593](https://github.com/ClickHouse/ClickHouse/issues/8593). [#8614](https://github.com/ClickHouse/ClickHouse/pull/8614) ([阿利沙平](https://github.com/alesapin)) +- 修复列的后台合并错误 `SimpleAggregateFunction(LowCardinality)` 类型。 [#8613](https://github.com/ClickHouse/ClickHouse/pull/8613) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 固定类型签入功能 `toDateTime64`. [#8375](https://github.com/ClickHouse/ClickHouse/pull/8375) ([瓦西里\*内姆科夫](https://github.com/Enmk)) +- 现在服务器不崩溃 `LEFT` 或 `FULL JOIN` 与和加入引擎和不支持 `join_use_nulls` 设置。 [#8479](https://github.com/ClickHouse/ClickHouse/pull/8479) ([Artem Zuikov](https://github.com/4ertus2)) +- 现在 `DROP DICTIONARY IF EXISTS db.dict` 查询不会抛出异常,如果 `db` 根本不存在 [#8185](https://github.com/ClickHouse/ClickHouse/pull/8185) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) +- 修复表函数中可能出现的崩溃 (`file`, `mysql`, `remote`)引用删除引起的 `IStorage` 对象。 修复插入表函数时指定的列的不正确解析。 [#7762](https://github.com/ClickHouse/ClickHouse/pull/7762) ([tavplubix](https://github.com/tavplubix)) +- 确保网络启动前 `clickhouse-server`. 这修复 [#7507](https://github.com/ClickHouse/ClickHouse/issues/7507). [#8570](https://github.com/ClickHouse/ClickHouse/pull/8570) ([余志昌](https://github.com/yuzhichang)) +- 修复安全连接的超时处理,因此查询不会无限挂起。 这修复 [#8126](https://github.com/ClickHouse/ClickHouse/issues/8126). [#8128](https://github.com/ClickHouse/ClickHouse/pull/8128) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修复 `clickhouse-copier`并发工人之间的冗余争用。 [#7816](https://github.com/ClickHouse/ClickHouse/pull/7816) ([丁香飞](https://github.com/dingxiangfei2009)) +- 现在突变不会跳过附加的部分,即使它们的突变版本比当前的突变版本大。 [#7812](https://github.com/ClickHouse/ClickHouse/pull/7812) ([余志昌](https://github.com/yuzhichang)) [#8250](https://github.com/ClickHouse/ClickHouse/pull/8250) ([阿利沙平](https://github.com/alesapin)) +- 忽略冗余副本 `*MergeTree` 数据部分移动到另一个磁盘和服务器重新启动后。 [#7810](https://github.com/ClickHouse/ClickHouse/pull/7810) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 修复崩溃 `FULL JOIN` 与 `LowCardinality` 在 `JOIN` 钥匙 [#8252](https://github.com/ClickHouse/ClickHouse/pull/8252) ([Artem Zuikov](https://github.com/4ertus2)) +- 禁止在插入查询中多次使用列名,如 `INSERT INTO tbl (x, y, x)`. 这修复 [#5465](https://github.com/ClickHouse/ClickHouse/issues/5465), [#7681](https://github.com/ClickHouse/ClickHouse/issues/7681). [#7685](https://github.com/ClickHouse/ClickHouse/pull/7685) ([阿利沙平](https://github.com/alesapin)) +- 增加了回退,用于检测未知Cpu的物理CPU内核数量(使用逻辑CPU内核数量)。 这修复 [#5239](https://github.com/ClickHouse/ClickHouse/issues/5239). [#7726](https://github.com/ClickHouse/ClickHouse/pull/7726) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修复 `There's no column` 实例化列和别名列出错。 [#8210](https://github.com/ClickHouse/ClickHouse/pull/8210) ([Artem Zuikov](https://github.com/4ertus2)) +- 固定切断崩溃时 `EXISTS` 查询没有使用 `TABLE` 或 `DICTIONARY` 预选赛 就像 `EXISTS t`. 这修复 [#8172](https://github.com/ClickHouse/ClickHouse/issues/8172). 此错误在版本19.17中引入。 [#8213](https://github.com/ClickHouse/ClickHouse/pull/8213) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修复罕见错误 `"Sizes of columns doesn't match"` 使用时可能会出现 `SimpleAggregateFunction` 列。 [#7790](https://github.com/ClickHouse/ClickHouse/pull/7790) ([Boris Granveaud](https://github.com/bgranvea)) +- 修正错误,其中用户空 `allow_databases` 可以访问所有数据库(和相同的 `allow_dictionaries`). [#7793](https://github.com/ClickHouse/ClickHouse/pull/7793) ([DeifyTheGod](https://github.com/DeifyTheGod)) +- 修复客户端崩溃时,服务器已经从客户端断开连接。 [#8071](https://github.com/ClickHouse/ClickHouse/pull/8071) ([Azat Khuzhin](https://github.com/azat)) +- 修复 `ORDER BY` 在按主键前缀和非主键后缀排序的情况下的行为。 [#7759](https://github.com/ClickHouse/ClickHouse/pull/7759) ([安东\*波波夫](https://github.com/CurtizJ)) +- 检查表中是否存在合格列。 这修复 [#6836](https://github.com/ClickHouse/ClickHouse/issues/6836). [#7758](https://github.com/ClickHouse/ClickHouse/pull/7758) ([Artem Zuikov](https://github.com/4ertus2)) +- 固定行为 `ALTER MOVE` 合并完成后立即运行移动指定的超部分。 修复 [#8103](https://github.com/ClickHouse/ClickHouse/issues/8103). [#8104](https://github.com/ClickHouse/ClickHouse/pull/8104) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 使用时修复可能的服务器崩溃 `UNION` 具有不同数量的列。 修复 [#7279](https://github.com/ClickHouse/ClickHouse/issues/7279). [#7929](https://github.com/ClickHouse/ClickHouse/pull/7929) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 修复函数结果子字符串的大小 `substr` 负大小。 [#8589](https://github.com/ClickHouse/ClickHouse/pull/8589) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 现在服务器不执行部分突变 `MergeTree` 如果后台池中没有足够的可用线程。 [#8588](https://github.com/ClickHouse/ClickHouse/pull/8588) ([tavplubix](https://github.com/tavplubix)) +- 修复格式化时的小错字 `UNION ALL` AST. [#7999](https://github.com/ClickHouse/ClickHouse/pull/7999) ([litao91](https://github.com/litao91)) +- 修正了负数不正确的布隆过滤结果。 这修复 [#8317](https://github.com/ClickHouse/ClickHouse/issues/8317). [#8566](https://github.com/ClickHouse/ClickHouse/pull/8566) ([张冬](https://github.com/zhang2014)) +- 在解压缩固定潜在的缓冲区溢出。 恶意用户可以传递捏造的压缩数据,这将导致缓冲区后读取。 这个问题是由Yandex信息安全团队的Eldar Zaitov发现的。 [#8404](https://github.com/ClickHouse/ClickHouse/pull/8404) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修复因整数溢出而导致的错误结果 `arrayIntersect`. [#7777](https://github.com/ClickHouse/ClickHouse/pull/7777) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 现在 `OPTIMIZE TABLE` query不会等待脱机副本执行该操作。 [#8314](https://github.com/ClickHouse/ClickHouse/pull/8314) ([javi santana](https://github.com/javisantana)) +- 固定 `ALTER TTL` 解析器 `Replicated*MergeTree` 桌子 [#8318](https://github.com/ClickHouse/ClickHouse/pull/8318) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 修复服务器和客户端之间的通信,以便服务器在查询失败后读取临时表信息。 [#8084](https://github.com/ClickHouse/ClickHouse/pull/8084) ([Azat Khuzhin](https://github.com/azat)) +- 修复 `bitmapAnd` 在聚合位图和标量位图相交时出现函数错误。 [#8082](https://github.com/ClickHouse/ClickHouse/pull/8082) ([黄月](https://github.com/moon03432)) +- 完善的定义 `ZXid` 根据动物园管理员的程序员指南,它修复了错误 `clickhouse-cluster-copier`. [#8088](https://github.com/ClickHouse/ClickHouse/pull/8088) ([丁香飞](https://github.com/dingxiangfei2009)) +- `odbc` 表函数现在尊重 `external_table_functions_use_nulls` 设置。 [#7506](https://github.com/ClickHouse/ClickHouse/pull/7506) ([瓦西里\*内姆科夫](https://github.com/Enmk)) +- 修正了导致罕见的数据竞赛的错误。 [#8143](https://github.com/ClickHouse/ClickHouse/pull/8143) ([亚历山大\*卡扎科夫](https://github.com/Akazz)) +- 现在 `SYSTEM RELOAD DICTIONARY` 完全重新加载字典,忽略 `update_field`. 这修复 [#7440](https://github.com/ClickHouse/ClickHouse/issues/7440). [#8037](https://github.com/ClickHouse/ClickHouse/pull/8037) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) +- 添加检查字典是否存在于创建查询的能力。 [#8032](https://github.com/ClickHouse/ClickHouse/pull/8032) ([阿利沙平](https://github.com/alesapin)) +- 修复 `Float*` 解析中 `Values` 格式。 这修复 [#7817](https://github.com/ClickHouse/ClickHouse/issues/7817). [#7870](https://github.com/ClickHouse/ClickHouse/pull/7870) ([tavplubix](https://github.com/tavplubix)) +- 修复崩溃时,我们不能在一些后台操作保留空间 `*MergeTree` 表引擎家族. [#7873](https://github.com/ClickHouse/ClickHouse/pull/7873) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 修复表包含合并操作时的崩溃 `SimpleAggregateFunction(LowCardinality)` 列。 这修复 [#8515](https://github.com/ClickHouse/ClickHouse/issues/8515). [#8522](https://github.com/ClickHouse/ClickHouse/pull/8522) ([Azat Khuzhin](https://github.com/azat)) +- 恢复对所有ICU区域设置的支持,并添加对常量表达式应用排序规则的功能。 还添加语言名称 `system.collations` 桌子 [#8051](https://github.com/ClickHouse/ClickHouse/pull/8051) ([阿利沙平](https://github.com/alesapin)) +- 修正错误时,外部字典与零最小寿命 (`LIFETIME(MIN 0 MAX N)`, `LIFETIME(N)`)不要在后台更新。 [#7983](https://github.com/ClickHouse/ClickHouse/pull/7983) ([阿利沙平](https://github.com/alesapin)) +- 修复当clickhouse源外部字典在查询中有子查询时崩溃。 [#8351](https://github.com/ClickHouse/ClickHouse/pull/8351) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 修复文件扩展名不正确的解析表与引擎 `URL`. 这修复 [#8157](https://github.com/ClickHouse/ClickHouse/issues/8157). [#8419](https://github.com/ClickHouse/ClickHouse/pull/8419) ([安德烈\*博德罗夫](https://github.com/apbodrov)) +- 修复 `CHECK TABLE` 查询为 `*MergeTree` 表没有关键. 修复 [#7543](https://github.com/ClickHouse/ClickHouse/issues/7543). [#7979](https://github.com/ClickHouse/ClickHouse/pull/7979) ([阿利沙平](https://github.com/alesapin)) +- 固定转换 `Float64` 到MySQL类型。 [#8079](https://github.com/ClickHouse/ClickHouse/pull/8079) ([尤里\*巴拉诺夫](https://github.com/yurriy)) +- 现在,如果表没有完全删除,因为服务器崩溃,服务器将尝试恢复并加载它。 [#8176](https://github.com/ClickHouse/ClickHouse/pull/8176) ([tavplubix](https://github.com/tavplubix)) +- 修复了表函数中的崩溃 `file` 同时插入到不存在的文件。 现在在这种情况下,文件将被创建,然后插入将被处理。 [#8177](https://github.com/ClickHouse/ClickHouse/pull/8177) ([Olga Khvostikova](https://github.com/stavrolia)) +- 修复罕见的死锁时,可能发生 `trace_log` 处于启用状态。 [#7838](https://github.com/ClickHouse/ClickHouse/pull/7838) ([filimonov](https://github.com/filimonov)) +- 添加能力与不同类型的工作,除了 `Date` 在 `RangeHashed` 从DDL查询创建的外部字典。 修复 [7899](https://github.com/ClickHouse/ClickHouse/issues/7899). [#8275](https://github.com/ClickHouse/ClickHouse/pull/8275) ([阿利沙平](https://github.com/alesapin)) +- 修复崩溃时 `now64()` 用另一个函数的结果调用。 [#8270](https://github.com/ClickHouse/ClickHouse/pull/8270) ([瓦西里\*内姆科夫](https://github.com/Enmk)) +- 修正了通过mysql有线协议检测客户端IP连接的错误。 [#7743](https://github.com/ClickHouse/ClickHouse/pull/7743) ([Dmitry Muzyka](https://github.com/dmitriy-myz)) +- 修复空阵列处理 `arraySplit` 功能。 这修复 [#7708](https://github.com/ClickHouse/ClickHouse/issues/7708). [#7747](https://github.com/ClickHouse/ClickHouse/pull/7747) ([hcz](https://github.com/hczhcz)) +- 修复了以下问题 `pid-file` 另一个运行 `clickhouse-server` 可能会被删除。 [#8487](https://github.com/ClickHouse/ClickHouse/pull/8487) ([徐伟清](https://github.com/weiqxu)) +- 修复字典重新加载,如果它有 `invalidate_query`,停止更新,并在以前的更新尝试一些异常。 [#8029](https://github.com/ClickHouse/ClickHouse/pull/8029) ([阿利沙平](https://github.com/alesapin)) +- 修正了功能错误 `arrayReduce` 这可能会导致 “double free” 和聚合函数组合器中的错误 `Resample` 这可能会导致内存泄漏。 添加聚合功能 `aggThrow`. 此功能可用于测试目的。 [#8446](https://github.com/ClickHouse/ClickHouse/pull/8446) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) #### 改进 {#improvement-1} -- 改进了使用时的日志记录 `S3` 表引擎。 [\#8251](https://github.com/ClickHouse/ClickHouse/pull/8251) ([Grigory Pervakov](https://github.com/GrigoryPervakov)) -- 在调用时未传递任何参数时打印帮助消息 `clickhouse-local`. 这修复 [\#5335](https://github.com/ClickHouse/ClickHouse/issues/5335). [\#8230](https://github.com/ClickHouse/ClickHouse/pull/8230) ([安德烈\*纳戈尔尼](https://github.com/Melancholic)) -- 添加设置 `mutations_sync` 这允许等待 `ALTER UPDATE/DELETE` 同步查询。 [\#8237](https://github.com/ClickHouse/ClickHouse/pull/8237) ([阿利沙平](https://github.com/alesapin)) -- 允许设置相对 `user_files_path` 在 `config.xml` (在类似的方式 `format_schema_path`). [\#7632](https://github.com/ClickHouse/ClickHouse/pull/7632) ([hcz](https://github.com/hczhcz)) -- 为转换函数添加非法类型的异常 `-OrZero` 后缀 [\#7880](https://github.com/ClickHouse/ClickHouse/pull/7880) ([安德烈\*科尼亚耶夫](https://github.com/akonyaev90)) -- 简化在分布式查询中发送到分片的数据头的格式。 [\#8044](https://github.com/ClickHouse/ClickHouse/pull/8044) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) -- `Live View` 表引擎重构。 [\#8519](https://github.com/ClickHouse/ClickHouse/pull/8519) ([vzakaznikov](https://github.com/vzakaznikov)) -- 为从DDL查询创建的外部字典添加额外的检查。 [\#8127](https://github.com/ClickHouse/ClickHouse/pull/8127) ([阿利沙平](https://github.com/alesapin)) -- 修复错误 `Column ... already exists` 使用时 `FINAL` 和 `SAMPLE` together, e.g. `select count() from table final sample 1/2`. 修复 [\#5186](https://github.com/ClickHouse/ClickHouse/issues/5186). [\#7907](https://github.com/ClickHouse/ClickHouse/pull/7907) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 现在表的第一个参数 `joinGet` 函数可以是表标识符。 [\#7707](https://github.com/ClickHouse/ClickHouse/pull/7707) ([阿莫斯鸟](https://github.com/amosbird)) -- 允许使用 `MaterializedView` 与上面的子查询 `Kafka` 桌子 [\#8197](https://github.com/ClickHouse/ClickHouse/pull/8197) ([filimonov](https://github.com/filimonov)) -- 现在后台在磁盘之间移动,运行它的seprate线程池。 [\#7670](https://github.com/ClickHouse/ClickHouse/pull/7670) ([Vladimir Chebotarev](https://github.com/excitoon)) -- `SYSTEM RELOAD DICTIONARY` 现在同步执行。 [\#8240](https://github.com/ClickHouse/ClickHouse/pull/8240) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) -- 堆栈跟踪现在显示物理地址(对象文件中的偏移量),而不是虚拟内存地址(加载对象文件的位置)。 这允许使用 `addr2line` 当二进制独立于位置并且ASLR处于活动状态时。 这修复 [\#8360](https://github.com/ClickHouse/ClickHouse/issues/8360). [\#8387](https://github.com/ClickHouse/ClickHouse/pull/8387) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 支持行级安全筛选器的新语法: `
`. 修复 [\#5779](https://github.com/ClickHouse/ClickHouse/issues/5779). [\#8381](https://github.com/ClickHouse/ClickHouse/pull/8381) ([伊万](https://github.com/abyss7)) -- 现在 `cityHash` 功能可以与工作 `Decimal` 和 `UUID` 类型。 修复 [\#5184](https://github.com/ClickHouse/ClickHouse/issues/5184). [\#7693](https://github.com/ClickHouse/ClickHouse/pull/7693) ([米哈伊尔\*科罗托夫](https://github.com/millb)) -- 从系统日志中删除了固定的索引粒度(它是1024),因为它在实现自适应粒度之后已经过时。 [\#7698](https://github.com/ClickHouse/ClickHouse/pull/7698) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 当ClickHouse在没有SSL的情况下编译时,启用MySQL兼容服务器。 [\#7852](https://github.com/ClickHouse/ClickHouse/pull/7852) ([尤里\*巴拉诺夫](https://github.com/yurriy)) -- 现在服务器校验和分布式批处理,这在批处理中损坏数据的情况下提供了更多详细的错误。 [\#7914](https://github.com/ClickHouse/ClickHouse/pull/7914) ([Azat Khuzhin](https://github.com/azat)) -- 碌莽禄Support: `DROP DATABASE`, `DETACH TABLE`, `DROP TABLE` 和 `ATTACH TABLE` 为 `MySQL` 数据库引擎。 [\#8202](https://github.com/ClickHouse/ClickHouse/pull/8202) ([张冬](https://github.com/zhang2014)) -- 在S3表功能和表引擎中添加身份验证。 [\#7623](https://github.com/ClickHouse/ClickHouse/pull/7623) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 增加了检查额外的部分 `MergeTree` 在不同的磁盘上,为了不允许错过未定义磁盘上的数据部分。 [\#8118](https://github.com/ClickHouse/ClickHouse/pull/8118) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 启用Mac客户端和服务器的SSL支持。 [\#8297](https://github.com/ClickHouse/ClickHouse/pull/8297) ([伊万](https://github.com/abyss7)) -- 现在ClickHouse可以作为MySQL联合服务器(参见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` 现在只能启用 `bracketed-paste` 当多查询处于打开状态且多行处于关闭状态时。 这修复(#7757)\[https://github.com/ClickHouse/ClickHouse/issues/7757。 [\#7761](https://github.com/ClickHouse/ClickHouse/pull/7761) ([阿莫斯鸟](https://github.com/amosbird)) -- 碌莽禄Support: `Array(Decimal)` 在 `if` 功能。 [\#7721](https://github.com/ClickHouse/ClickHouse/pull/7721) ([Artem Zuikov](https://github.com/4ertus2)) -- 支持小数 `arrayDifference`, `arrayCumSum` 和 `arrayCumSumNegative` 功能。 [\#7724](https://github.com/ClickHouse/ClickHouse/pull/7724) ([Artem Zuikov](https://github.com/4ertus2)) -- 已添加 `lifetime` 列到 `system.dictionaries` 桌子 [\#6820](https://github.com/ClickHouse/ClickHouse/issues/6820) [\#7727](https://github.com/ClickHouse/ClickHouse/pull/7727) ([kekekekule](https://github.com/kekekekule)) -- 改进了检查不同磁盘上的现有部件 `*MergeTree` 表引擎. 地址 [\#7660](https://github.com/ClickHouse/ClickHouse/issues/7660). [\#8440](https://github.com/ClickHouse/ClickHouse/pull/8440) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 集成与 `AWS SDK` 为 `S3` 交互允许使用开箱即用的所有S3功能。 [\#8011](https://github.com/ClickHouse/ClickHouse/pull/8011) ([帕维尔\*科瓦连科](https://github.com/Jokser)) -- 增加了对子查询的支持 `Live View` 桌子 [\#7792](https://github.com/ClickHouse/ClickHouse/pull/7792) ([vzakaznikov](https://github.com/vzakaznikov)) -- 检查使用 `Date` 或 `DateTime` 从列 `TTL` 表达式已删除。 [\#7920](https://github.com/ClickHouse/ClickHouse/pull/7920) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 有关磁盘的信息已添加到 `system.detached_parts` 桌子 [\#7833](https://github.com/ClickHouse/ClickHouse/pull/7833) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 现在设置 `max_(table|partition)_size_to_drop` 无需重新启动即可更改。 [\#7779](https://github.com/ClickHouse/ClickHouse/pull/7779) ([Grigory Pervakov](https://github.com/GrigoryPervakov)) -- 错误消息的可用性略好。 要求用户不要删除下面的行 `Stack trace:`. [\#7897](https://github.com/ClickHouse/ClickHouse/pull/7897) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 更好地阅读消息 `Kafka` 引擎在各种格式后 [\#7935](https://github.com/ClickHouse/ClickHouse/issues/7935). [\#8035](https://github.com/ClickHouse/ClickHouse/pull/8035) ([伊万](https://github.com/abyss7)) -- 与不支持MySQL客户端更好的兼容性 `sha2_password` 验证插件。 [\#8036](https://github.com/ClickHouse/ClickHouse/pull/8036) ([尤里\*巴拉诺夫](https://github.com/yurriy)) -- 支持MySQL兼容性服务器中的更多列类型。 [\#7975](https://github.com/ClickHouse/ClickHouse/pull/7975) ([尤里\*巴拉诺夫](https://github.com/yurriy)) -- 执行 `ORDER BY` 优化 `Merge`, `Buffer` 和 `Materilized View` 存储与底层 `MergeTree` 桌子 [\#8130](https://github.com/ClickHouse/ClickHouse/pull/8130) ([安东\*波波夫](https://github.com/CurtizJ)) -- 现在我们总是使用POSIX实现 `getrandom` 与旧内核更好的兼容性(\<3.17)。 [\#7940](https://github.com/ClickHouse/ClickHouse/pull/7940) ([阿莫斯鸟](https://github.com/amosbird)) -- 更好地检查移动ttl规则中的有效目标。 [\#8410](https://github.com/ClickHouse/ClickHouse/pull/8410) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 更好地检查损坏的刀片批次 `Distributed` 表引擎。 [\#7933](https://github.com/ClickHouse/ClickHouse/pull/7933) ([Azat Khuzhin](https://github.com/azat)) -- 添加带有部件名称数组的列,这些部件将来必须处理突变 `system.mutations` 桌子 [\#8179](https://github.com/ClickHouse/ClickHouse/pull/8179) ([阿利沙平](https://github.com/alesapin)) -- 处理器的并行合并排序优化。 [\#8552](https://github.com/ClickHouse/ClickHouse/pull/8552) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 设置 `mark_cache_min_lifetime` 现在已经过时了,什么也不做。 在以前的版本中,标记缓存可以在内存中增长大于 `mark_cache_size` 以容纳内的数据 `mark_cache_min_lifetime` 秒。 这导致了混乱和比预期更高的内存使用率,这在内存受限的系统上尤其糟糕。 如果您在安装此版本后会看到性能下降,则应增加 `mark_cache_size`. [\#8484](https://github.com/ClickHouse/ClickHouse/pull/8484) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 准备使用 `tid` 到处都是 这是必要的 [\#7477](https://github.com/ClickHouse/ClickHouse/issues/7477). [\#8276](https://github.com/ClickHouse/ClickHouse/pull/8276) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 改进了使用时的日志记录 `S3` 表引擎。 [#8251](https://github.com/ClickHouse/ClickHouse/pull/8251) ([Grigory Pervakov](https://github.com/GrigoryPervakov)) +- 在调用时未传递任何参数时打印帮助消息 `clickhouse-local`. 这修复 [#5335](https://github.com/ClickHouse/ClickHouse/issues/5335). [#8230](https://github.com/ClickHouse/ClickHouse/pull/8230) ([安德烈\*纳戈尔尼](https://github.com/Melancholic)) +- 添加设置 `mutations_sync` 这允许等待 `ALTER UPDATE/DELETE` 同步查询。 [#8237](https://github.com/ClickHouse/ClickHouse/pull/8237) ([阿利沙平](https://github.com/alesapin)) +- 允许设置相对 `user_files_path` 在 `config.xml` (在类似的方式 `format_schema_path`). [#7632](https://github.com/ClickHouse/ClickHouse/pull/7632) ([hcz](https://github.com/hczhcz)) +- 为转换函数添加非法类型的异常 `-OrZero` 后缀 [#7880](https://github.com/ClickHouse/ClickHouse/pull/7880) ([安德烈\*科尼亚耶夫](https://github.com/akonyaev90)) +- 简化在分布式查询中发送到分片的数据头的格式。 [#8044](https://github.com/ClickHouse/ClickHouse/pull/8044) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) +- `Live View` 表引擎重构。 [#8519](https://github.com/ClickHouse/ClickHouse/pull/8519) ([vzakaznikov](https://github.com/vzakaznikov)) +- 为从DDL查询创建的外部字典添加额外的检查。 [#8127](https://github.com/ClickHouse/ClickHouse/pull/8127) ([阿利沙平](https://github.com/alesapin)) +- 修复错误 `Column ... already exists` 使用时 `FINAL` 和 `SAMPLE` together, e.g. `select count() from table final sample 1/2`. 修复 [#5186](https://github.com/ClickHouse/ClickHouse/issues/5186). [#7907](https://github.com/ClickHouse/ClickHouse/pull/7907) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 现在表的第一个参数 `joinGet` 函数可以是表标识符。 [#7707](https://github.com/ClickHouse/ClickHouse/pull/7707) ([阿莫斯鸟](https://github.com/amosbird)) +- 允许使用 `MaterializedView` 与上面的子查询 `Kafka` 桌子 [#8197](https://github.com/ClickHouse/ClickHouse/pull/8197) ([filimonov](https://github.com/filimonov)) +- 现在后台在磁盘之间移动,运行它的seprate线程池。 [#7670](https://github.com/ClickHouse/ClickHouse/pull/7670) ([Vladimir Chebotarev](https://github.com/excitoon)) +- `SYSTEM RELOAD DICTIONARY` 现在同步执行。 [#8240](https://github.com/ClickHouse/ClickHouse/pull/8240) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) +- 堆栈跟踪现在显示物理地址(对象文件中的偏移量),而不是虚拟内存地址(加载对象文件的位置)。 这允许使用 `addr2line` 当二进制独立于位置并且ASLR处于活动状态时。 这修复 [#8360](https://github.com/ClickHouse/ClickHouse/issues/8360). [#8387](https://github.com/ClickHouse/ClickHouse/pull/8387) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 支持行级安全筛选器的新语法: `…
`. 修复 [#5779](https://github.com/ClickHouse/ClickHouse/issues/5779). [#8381](https://github.com/ClickHouse/ClickHouse/pull/8381) ([伊万](https://github.com/abyss7)) +- 现在 `cityHash` 功能可以与工作 `Decimal` 和 `UUID` 类型。 修复 [#5184](https://github.com/ClickHouse/ClickHouse/issues/5184). [#7693](https://github.com/ClickHouse/ClickHouse/pull/7693) ([米哈伊尔\*科罗托夫](https://github.com/millb)) +- 从系统日志中删除了固定的索引粒度(它是1024),因为它在实现自适应粒度之后已经过时。 [#7698](https://github.com/ClickHouse/ClickHouse/pull/7698) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 当ClickHouse在没有SSL的情况下编译时,启用MySQL兼容服务器。 [#7852](https://github.com/ClickHouse/ClickHouse/pull/7852) ([尤里\*巴拉诺夫](https://github.com/yurriy)) +- 现在服务器校验和分布式批处理,这在批处理中损坏数据的情况下提供了更多详细的错误。 [#7914](https://github.com/ClickHouse/ClickHouse/pull/7914) ([Azat Khuzhin](https://github.com/azat)) +- 碌莽禄Support: `DROP DATABASE`, `DETACH TABLE`, `DROP TABLE` 和 `ATTACH TABLE` 为 `MySQL` 数据库引擎。 [#8202](https://github.com/ClickHouse/ClickHouse/pull/8202) ([张冬](https://github.com/zhang2014)) +- 在S3表功能和表引擎中添加身份验证。 [#7623](https://github.com/ClickHouse/ClickHouse/pull/7623) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 增加了检查额外的部分 `MergeTree` 在不同的磁盘上,为了不允许错过未定义磁盘上的数据部分。 [#8118](https://github.com/ClickHouse/ClickHouse/pull/8118) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 启用Mac客户端和服务器的SSL支持。 [#8297](https://github.com/ClickHouse/ClickHouse/pull/8297) ([伊万](https://github.com/abyss7)) +- 现在ClickHouse可以作为MySQL联合服务器(参见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` 现在只能启用 `bracketed-paste` 当多查询处于打开状态且多行处于关闭状态时。 这修复(#7757)\[https://github.com/ClickHouse/ClickHouse/issues/7757。 [#7761](https://github.com/ClickHouse/ClickHouse/pull/7761) ([阿莫斯鸟](https://github.com/amosbird)) +- 碌莽禄Support: `Array(Decimal)` 在 `if` 功能。 [#7721](https://github.com/ClickHouse/ClickHouse/pull/7721) ([Artem Zuikov](https://github.com/4ertus2)) +- 支持小数 `arrayDifference`, `arrayCumSum` 和 `arrayCumSumNegative` 功能。 [#7724](https://github.com/ClickHouse/ClickHouse/pull/7724) ([Artem Zuikov](https://github.com/4ertus2)) +- 已添加 `lifetime` 列到 `system.dictionaries` 桌子 [#6820](https://github.com/ClickHouse/ClickHouse/issues/6820) [#7727](https://github.com/ClickHouse/ClickHouse/pull/7727) ([kekekekule](https://github.com/kekekekule)) +- 改进了检查不同磁盘上的现有部件 `*MergeTree` 表引擎. 地址 [#7660](https://github.com/ClickHouse/ClickHouse/issues/7660). [#8440](https://github.com/ClickHouse/ClickHouse/pull/8440) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 集成与 `AWS SDK` 为 `S3` 交互允许使用开箱即用的所有S3功能。 [#8011](https://github.com/ClickHouse/ClickHouse/pull/8011) ([帕维尔\*科瓦连科](https://github.com/Jokser)) +- 增加了对子查询的支持 `Live View` 桌子 [#7792](https://github.com/ClickHouse/ClickHouse/pull/7792) ([vzakaznikov](https://github.com/vzakaznikov)) +- 检查使用 `Date` 或 `DateTime` 从列 `TTL` 表达式已删除。 [#7920](https://github.com/ClickHouse/ClickHouse/pull/7920) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 有关磁盘的信息已添加到 `system.detached_parts` 桌子 [#7833](https://github.com/ClickHouse/ClickHouse/pull/7833) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 现在设置 `max_(table|partition)_size_to_drop` 无需重新启动即可更改。 [#7779](https://github.com/ClickHouse/ClickHouse/pull/7779) ([Grigory Pervakov](https://github.com/GrigoryPervakov)) +- 错误消息的可用性略好。 要求用户不要删除下面的行 `Stack trace:`. [#7897](https://github.com/ClickHouse/ClickHouse/pull/7897) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 更好地阅读消息 `Kafka` 引擎在各种格式后 [#7935](https://github.com/ClickHouse/ClickHouse/issues/7935). [#8035](https://github.com/ClickHouse/ClickHouse/pull/8035) ([伊万](https://github.com/abyss7)) +- 与不支持MySQL客户端更好的兼容性 `sha2_password` 验证插件。 [#8036](https://github.com/ClickHouse/ClickHouse/pull/8036) ([尤里\*巴拉诺夫](https://github.com/yurriy)) +- 支持MySQL兼容性服务器中的更多列类型。 [#7975](https://github.com/ClickHouse/ClickHouse/pull/7975) ([尤里\*巴拉诺夫](https://github.com/yurriy)) +- 执行 `ORDER BY` 优化 `Merge`, `Buffer` 和 `Materilized View` 存储与底层 `MergeTree` 桌子 [#8130](https://github.com/ClickHouse/ClickHouse/pull/8130) ([安东\*波波夫](https://github.com/CurtizJ)) +- 现在我们总是使用POSIX实现 `getrandom` 与旧内核更好的兼容性(\<3.17)。 [#7940](https://github.com/ClickHouse/ClickHouse/pull/7940) ([阿莫斯鸟](https://github.com/amosbird)) +- 更好地检查移动ttl规则中的有效目标。 [#8410](https://github.com/ClickHouse/ClickHouse/pull/8410) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 更好地检查损坏的刀片批次 `Distributed` 表引擎。 [#7933](https://github.com/ClickHouse/ClickHouse/pull/7933) ([Azat Khuzhin](https://github.com/azat)) +- 添加带有部件名称数组的列,这些部件将来必须处理突变 `system.mutations` 桌子 [#8179](https://github.com/ClickHouse/ClickHouse/pull/8179) ([阿利沙平](https://github.com/alesapin)) +- 处理器的并行合并排序优化。 [#8552](https://github.com/ClickHouse/ClickHouse/pull/8552) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 设置 `mark_cache_min_lifetime` 现在已经过时了,什么也不做。 在以前的版本中,标记缓存可以在内存中增长大于 `mark_cache_size` 以容纳内的数据 `mark_cache_min_lifetime` 秒。 这导致了混乱和比预期更高的内存使用率,这在内存受限的系统上尤其糟糕。 如果您在安装此版本后会看到性能下降,则应增加 `mark_cache_size`. [#8484](https://github.com/ClickHouse/ClickHouse/pull/8484) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 准备使用 `tid` 到处都是 这是必要的 [#7477](https://github.com/ClickHouse/ClickHouse/issues/7477). [#8276](https://github.com/ClickHouse/ClickHouse/pull/8276) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) #### 性能改进 {#performance-improvement-1} -- 处理器管道中的性能优化。 [\#7988](https://github.com/ClickHouse/ClickHouse/pull/7988) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 缓存字典中过期密钥的非阻塞更新(具有读取旧密钥的权限)。 [\#8303](https://github.com/ClickHouse/ClickHouse/pull/8303) ([尼基塔\*米哈伊洛夫](https://github.com/nikitamikhaylov)) -- 没有编译ClickHouse `-fno-omit-frame-pointer` 在全球范围内多余一个寄存器。 [\#8097](https://github.com/ClickHouse/ClickHouse/pull/8097) ([阿莫斯鸟](https://github.com/amosbird)) -- 加速 `greatCircleDistance` 功能,并为它添加性能测试。 [\#7307](https://github.com/ClickHouse/ClickHouse/pull/7307) ([Olga Khvostikova](https://github.com/stavrolia)) -- 改进的功能性能 `roundDown`. [\#8465](https://github.com/ClickHouse/ClickHouse/pull/8465) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 改进的性能 `max`, `min`, `argMin`, `argMax` 为 `DateTime64` 数据类型。 [\#8199](https://github.com/ClickHouse/ClickHouse/pull/8199) ([瓦西里\*内姆科夫](https://github.com/Enmk)) -- 改进了无限制或大限制和外部排序的排序性能。 [\#8545](https://github.com/ClickHouse/ClickHouse/pull/8545) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 改进的性能格式化浮点数高达6倍。 [\#8542](https://github.com/ClickHouse/ClickHouse/pull/8542) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 改进的性能 `modulo` 功能。 [\#7750](https://github.com/ClickHouse/ClickHouse/pull/7750) ([阿莫斯鸟](https://github.com/amosbird)) -- 优化 `ORDER BY` 并与单列键合并。 [\#8335](https://github.com/ClickHouse/ClickHouse/pull/8335) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 更好地实施 `arrayReduce`, `-Array` 和 `-State` 组合子 [\#7710](https://github.com/ClickHouse/ClickHouse/pull/7710) ([阿莫斯鸟](https://github.com/amosbird)) -- 现在 `PREWHERE` 应优化为至少一样高效 `WHERE`. [\#7769](https://github.com/ClickHouse/ClickHouse/pull/7769) ([阿莫斯鸟](https://github.com/amosbird)) -- 改进方式 `round` 和 `roundBankers` 处理负数。 [\#8229](https://github.com/ClickHouse/ClickHouse/pull/8229) ([hcz](https://github.com/hczhcz)) -- 改进的解码性能 `DoubleDelta` 和 `Gorilla` 编解码器大约30-40%。 这修复 [\#7082](https://github.com/ClickHouse/ClickHouse/issues/7082). [\#8019](https://github.com/ClickHouse/ClickHouse/pull/8019) ([瓦西里\*内姆科夫](https://github.com/Enmk)) -- 改进的性能 `base64` 相关功能。 [\#8444](https://github.com/ClickHouse/ClickHouse/pull/8444) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 增加了一个功能 `geoDistance`. 它类似于 `greatCircleDistance` 但使用近似于WGS-84椭球模型。 两个功能的性能几乎相同。 [\#8086](https://github.com/ClickHouse/ClickHouse/pull/8086) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 更快 `min` 和 `max` 聚合函数 `Decimal` 数据类型。 [\#8144](https://github.com/ClickHouse/ClickHouse/pull/8144) ([Artem Zuikov](https://github.com/4ertus2)) -- 矢量化处理 `arrayReduce`. [\#7608](https://github.com/ClickHouse/ClickHouse/pull/7608) ([阿莫斯鸟](https://github.com/amosbird)) -- `if` 链现在优化为 `multiIf`. [\#8355](https://github.com/ClickHouse/ClickHouse/pull/8355) ([kamalov-ruslan](https://github.com/kamalov-ruslan)) -- 修复性能回归 `Kafka` 表引擎在19.15中引入。 这修复 [\#7261](https://github.com/ClickHouse/ClickHouse/issues/7261). [\#7935](https://github.com/ClickHouse/ClickHouse/pull/7935) ([filimonov](https://github.com/filimonov)) -- 已删除 “pie” 代码生成 `gcc` 从Debian软件包偶尔带来默认情况下。 [\#8483](https://github.com/ClickHouse/ClickHouse/pull/8483) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 并行解析数据格式 [\#6553](https://github.com/ClickHouse/ClickHouse/pull/6553) ([尼基塔\*米哈伊洛夫](https://github.com/nikitamikhaylov)) -- 启用优化的解析器 `Values` 默认使用表达式 (`input_format_values_deduce_templates_of_expressions=1`). [\#8231](https://github.com/ClickHouse/ClickHouse/pull/8231) ([tavplubix](https://github.com/tavplubix)) +- 处理器管道中的性能优化。 [#7988](https://github.com/ClickHouse/ClickHouse/pull/7988) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 缓存字典中过期密钥的非阻塞更新(具有读取旧密钥的权限)。 [#8303](https://github.com/ClickHouse/ClickHouse/pull/8303) ([尼基塔\*米哈伊洛夫](https://github.com/nikitamikhaylov)) +- 没有编译ClickHouse `-fno-omit-frame-pointer` 在全球范围内多余一个寄存器。 [#8097](https://github.com/ClickHouse/ClickHouse/pull/8097) ([阿莫斯鸟](https://github.com/amosbird)) +- 加速 `greatCircleDistance` 功能,并为它添加性能测试。 [#7307](https://github.com/ClickHouse/ClickHouse/pull/7307) ([Olga Khvostikova](https://github.com/stavrolia)) +- 改进的功能性能 `roundDown`. [#8465](https://github.com/ClickHouse/ClickHouse/pull/8465) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 改进的性能 `max`, `min`, `argMin`, `argMax` 为 `DateTime64` 数据类型。 [#8199](https://github.com/ClickHouse/ClickHouse/pull/8199) ([瓦西里\*内姆科夫](https://github.com/Enmk)) +- 改进了无限制或大限制和外部排序的排序性能。 [#8545](https://github.com/ClickHouse/ClickHouse/pull/8545) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 改进的性能格式化浮点数高达6倍。 [#8542](https://github.com/ClickHouse/ClickHouse/pull/8542) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 改进的性能 `modulo` 功能。 [#7750](https://github.com/ClickHouse/ClickHouse/pull/7750) ([阿莫斯鸟](https://github.com/amosbird)) +- 优化 `ORDER BY` 并与单列键合并。 [#8335](https://github.com/ClickHouse/ClickHouse/pull/8335) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 更好地实施 `arrayReduce`, `-Array` 和 `-State` 组合子 [#7710](https://github.com/ClickHouse/ClickHouse/pull/7710) ([阿莫斯鸟](https://github.com/amosbird)) +- 现在 `PREWHERE` 应优化为至少一样高效 `WHERE`. [#7769](https://github.com/ClickHouse/ClickHouse/pull/7769) ([阿莫斯鸟](https://github.com/amosbird)) +- 改进方式 `round` 和 `roundBankers` 处理负数。 [#8229](https://github.com/ClickHouse/ClickHouse/pull/8229) ([hcz](https://github.com/hczhcz)) +- 改进的解码性能 `DoubleDelta` 和 `Gorilla` 编解码器大约30-40%。 这修复 [#7082](https://github.com/ClickHouse/ClickHouse/issues/7082). [#8019](https://github.com/ClickHouse/ClickHouse/pull/8019) ([瓦西里\*内姆科夫](https://github.com/Enmk)) +- 改进的性能 `base64` 相关功能。 [#8444](https://github.com/ClickHouse/ClickHouse/pull/8444) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 增加了一个功能 `geoDistance`. 它类似于 `greatCircleDistance` 但使用近似于WGS-84椭球模型。 两个功能的性能几乎相同。 [#8086](https://github.com/ClickHouse/ClickHouse/pull/8086) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 更快 `min` 和 `max` 聚合函数 `Decimal` 数据类型。 [#8144](https://github.com/ClickHouse/ClickHouse/pull/8144) ([Artem Zuikov](https://github.com/4ertus2)) +- 矢量化处理 `arrayReduce`. [#7608](https://github.com/ClickHouse/ClickHouse/pull/7608) ([阿莫斯鸟](https://github.com/amosbird)) +- `if` 链现在优化为 `multiIf`. [#8355](https://github.com/ClickHouse/ClickHouse/pull/8355) ([kamalov-ruslan](https://github.com/kamalov-ruslan)) +- 修复性能回归 `Kafka` 表引擎在19.15中引入。 这修复 [#7261](https://github.com/ClickHouse/ClickHouse/issues/7261). [#7935](https://github.com/ClickHouse/ClickHouse/pull/7935) ([filimonov](https://github.com/filimonov)) +- 已删除 “pie” 代码生成 `gcc` 从Debian软件包偶尔带来默认情况下。 [#8483](https://github.com/ClickHouse/ClickHouse/pull/8483) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 并行解析数据格式 [#6553](https://github.com/ClickHouse/ClickHouse/pull/6553) ([尼基塔\*米哈伊洛夫](https://github.com/nikitamikhaylov)) +- 启用优化的解析器 `Values` 默认使用表达式 (`input_format_values_deduce_templates_of_expressions=1`). [#8231](https://github.com/ClickHouse/ClickHouse/pull/8231) ([tavplubix](https://github.com/tavplubix)) #### 构建/测试/包装改进 {#buildtestingpackaging-improvement-2} -- 构建修复 `ARM` 而在最小模式。 [\#8304](https://github.com/ClickHouse/ClickHouse/pull/8304) ([proller](https://github.com/proller)) -- 添加复盖文件刷新 `clickhouse-server` 当不调用std::atexit时。 还略微改进了无状态测试的复盖率日志记录。 [\#8267](https://github.com/ClickHouse/ClickHouse/pull/8267) ([阿利沙平](https://github.com/alesapin)) -- 更新contrib中的LLVM库。 避免从操作系统包中使用LLVM。 [\#8258](https://github.com/ClickHouse/ClickHouse/pull/8258) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 使bund绑 `curl` 建立完全安静。 [\#8232](https://github.com/ClickHouse/ClickHouse/pull/8232) [\#8203](https://github.com/ClickHouse/ClickHouse/pull/8203) ([帕维尔\*科瓦连科](https://github.com/Jokser)) -- 修复一些 `MemorySanitizer` 警告。 [\#8235](https://github.com/ClickHouse/ClickHouse/pull/8235) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- 使用 `add_warning` 和 `no_warning` 宏 `CMakeLists.txt`. [\#8604](https://github.com/ClickHouse/ClickHouse/pull/8604) ([伊万](https://github.com/abyss7)) -- 添加对Minio S3兼容对象的支持(https://min.io/)为了更好的集成测试。 [\#7863](https://github.com/ClickHouse/ClickHouse/pull/7863) [\#7875](https://github.com/ClickHouse/ClickHouse/pull/7875) ([帕维尔\*科瓦连科](https://github.com/Jokser)) -- 导入 `libc` 标题到contrib。 它允许在各种系统中使构建更加一致(仅适用于 `x86_64-linux-gnu`). [\#5773](https://github.com/ClickHouse/ClickHouse/pull/5773) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 删除 `-fPIC` 从一些图书馆。 [\#8464](https://github.com/ClickHouse/ClickHouse/pull/8464) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 清洁 `CMakeLists.txt` 对于卷曲。 看https://github.com/ClickHouse/ClickHouse/pull/8011\#issuecomment-569478910 [\#8459](https://github.com/ClickHouse/ClickHouse/pull/8459) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 无声警告 `CapNProto` 图书馆. [\#8220](https://github.com/ClickHouse/ClickHouse/pull/8220) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 为短字符串优化哈希表添加性能测试。 [\#7679](https://github.com/ClickHouse/ClickHouse/pull/7679) ([阿莫斯鸟](https://github.com/amosbird)) -- 现在ClickHouse将建立在 `AArch64` 即使 `MADV_FREE` 不可用。 这修复 [\#8027](https://github.com/ClickHouse/ClickHouse/issues/8027). [\#8243](https://github.com/ClickHouse/ClickHouse/pull/8243) ([阿莫斯鸟](https://github.com/amosbird)) -- 更新 `zlib-ng` 来解决记忆消毒的问题 [\#7182](https://github.com/ClickHouse/ClickHouse/pull/7182) [\#8206](https://github.com/ClickHouse/ClickHouse/pull/8206) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- 在非Linux系统上启用内部MySQL库,因为操作系统包的使用非常脆弱,通常根本不起作用。 这修复 [\#5765](https://github.com/ClickHouse/ClickHouse/issues/5765). [\#8426](https://github.com/ClickHouse/ClickHouse/pull/8426) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修复了启用后在某些系统上构建的问题 `libc++`. 这取代了 [\#8374](https://github.com/ClickHouse/ClickHouse/issues/8374). [\#8380](https://github.com/ClickHouse/ClickHouse/pull/8380) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 赂眉露\>\> `Field` 方法更类型安全,以找到更多的错误。 [\#7386](https://github.com/ClickHouse/ClickHouse/pull/7386) [\#8209](https://github.com/ClickHouse/ClickHouse/pull/8209) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- 添加丢失的文件到 `libc-headers` 子模块。 [\#8507](https://github.com/ClickHouse/ClickHouse/pull/8507) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修复错误 `JSON` 引用性能测试输出。 [\#8497](https://github.com/ClickHouse/ClickHouse/pull/8497) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 现在堆栈跟踪显示 `std::exception` 和 `Poco::Exception`. 在以前的版本中,它仅适用于 `DB::Exception`. 这改进了诊断。 [\#8501](https://github.com/ClickHouse/ClickHouse/pull/8501) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 移植 `clock_gettime` 和 `clock_nanosleep` 对于新鲜的glibc版本。 [\#8054](https://github.com/ClickHouse/ClickHouse/pull/8054) ([阿莫斯鸟](https://github.com/amosbird)) -- 启用 `part_log` 在示例配置开发人员。 [\#8609](https://github.com/ClickHouse/ClickHouse/pull/8609) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修复重新加载的异步性质 `01036_no_superfluous_dict_reload_on_create_database*`. [\#8111](https://github.com/ClickHouse/ClickHouse/pull/8111) ([Azat Khuzhin](https://github.com/azat)) -- 固定编解码器性能测试。 [\#8615](https://github.com/ClickHouse/ClickHouse/pull/8615) ([瓦西里\*内姆科夫](https://github.com/Enmk)) -- 添加安装脚本 `.tgz` 为他们构建和文档。 [\#8612](https://github.com/ClickHouse/ClickHouse/pull/8612) [\#8591](https://github.com/ClickHouse/ClickHouse/pull/8591) ([阿利沙平](https://github.com/alesapin)) -- 删除旧 `ZSTD` 测试(它是在2016年创建的,以重现zstd1.0版本之前的错误)。 这修复 [\#8618](https://github.com/ClickHouse/ClickHouse/issues/8618). [\#8619](https://github.com/ClickHouse/ClickHouse/pull/8619) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 固定构建在Mac OS卡特琳娜。 [\#8600](https://github.com/ClickHouse/ClickHouse/pull/8600) ([meo](https://github.com/meob)) -- 增加编解码器性能测试中的行数,以使结果显着。 [\#8574](https://github.com/ClickHouse/ClickHouse/pull/8574) ([瓦西里\*内姆科夫](https://github.com/Enmk)) -- 在调试版本中,处理 `LOGICAL_ERROR` 异常作为断言失败,使得它们更容易被注意到。 [\#8475](https://github.com/ClickHouse/ClickHouse/pull/8475) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- 使与格式相关的性能测试更具确定性。 [\#8477](https://github.com/ClickHouse/ClickHouse/pull/8477) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 更新 `lz4` 来修复记忆消毒器的故障 [\#8181](https://github.com/ClickHouse/ClickHouse/pull/8181) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- 在异常处理中抑制已知MemorySanitizer误报。 [\#8182](https://github.com/ClickHouse/ClickHouse/pull/8182) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- 更新 `gcc` 和 `g++` 到版本9在 `build/docker/build.sh` [\#7766](https://github.com/ClickHouse/ClickHouse/pull/7766) ([TLightSky](https://github.com/tlightsky)) -- 添加性能测试用例来测试 `PREWHERE` 比 `WHERE`. [\#7768](https://github.com/ClickHouse/ClickHouse/pull/7768) ([阿莫斯鸟](https://github.com/amosbird)) -- 在修复一个笨拙的测试方面取得了进展。 [\#8621](https://github.com/ClickHouse/ClickHouse/pull/8621) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 避免从MemorySanitizer报告数据 `libunwind`. [\#8539](https://github.com/ClickHouse/ClickHouse/pull/8539) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 更新 `libc++` 到最新版本。 [\#8324](https://github.com/ClickHouse/ClickHouse/pull/8324) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 从源头构建ICU库。 这修复 [\#6460](https://github.com/ClickHouse/ClickHouse/issues/6460). [\#8219](https://github.com/ClickHouse/ClickHouse/pull/8219) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 从切换 `libressl` 到 `openssl`. ClickHouse应在此更改后支持TLS1.3和SNI。 这修复 [\#8171](https://github.com/ClickHouse/ClickHouse/issues/8171). [\#8218](https://github.com/ClickHouse/ClickHouse/pull/8218) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 使用时固定的UBSan报告 `chacha20_poly1305` 从SSL(发生在连接到https://yandex.ru/)。 [\#8214](https://github.com/ClickHouse/ClickHouse/pull/8214) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修复默认密码文件的模式 `.deb` linux发行版。 [\#8075](https://github.com/ClickHouse/ClickHouse/pull/8075) ([proller](https://github.com/proller)) -- 改进的表达式获取 `clickhouse-server` PID输入 `clickhouse-test`. [\#8063](https://github.com/ClickHouse/ClickHouse/pull/8063) ([亚历山大\*卡扎科夫](https://github.com/Akazz)) -- 更新contrib/googletest到v1.10.0。 [\#8587](https://github.com/ClickHouse/ClickHouse/pull/8587) ([Alexander Burmak](https://github.com/Alex-Burmak)) -- 修复了ThreadSaninitizer报告 `base64` 图书馆. 还将此库更新到最新版本,但无关紧要。 这修复 [\#8397](https://github.com/ClickHouse/ClickHouse/issues/8397). [\#8403](https://github.com/ClickHouse/ClickHouse/pull/8403) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修复 `00600_replace_running_query` 对于处理器。 [\#8272](https://github.com/ClickHouse/ClickHouse/pull/8272) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 删除支持 `tcmalloc` 为了使 `CMakeLists.txt` 更简单 [\#8310](https://github.com/ClickHouse/ClickHouse/pull/8310) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 发布海湾合作委员会构建现在使用 `libc++` 而不是 `libstdc++`. 最近 `libc++` 只与叮当一起使用。 这将提高构建配置的一致性和可移植性。 [\#8311](https://github.com/ClickHouse/ClickHouse/pull/8311) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 使用MemorySanitizer启用ICU库进行构建。 [\#8222](https://github.com/ClickHouse/ClickHouse/pull/8222) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 禁止从警告 `CapNProto` 图书馆. [\#8224](https://github.com/ClickHouse/ClickHouse/pull/8224) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 删除代码的特殊情况 `tcmalloc`,因为它不再受支持。 [\#8225](https://github.com/ClickHouse/ClickHouse/pull/8225) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 在CI coverage任务中,优雅地终止服务器以允许它保存coverage报告。 这修复了我们最近看到的不完整的复盖率报告。 [\#8142](https://github.com/ClickHouse/ClickHouse/pull/8142) ([阿利沙平](https://github.com/alesapin)) -- 针对所有编解码器的性能测试 `Float64` 和 `UInt64` 值。 [\#8349](https://github.com/ClickHouse/ClickHouse/pull/8349) ([瓦西里\*内姆科夫](https://github.com/Enmk)) -- `termcap` 非常不推荐使用,并导致各种问题(f.g.missing “up” 帽和呼应 `^J` 而不是多行)。 帮个忙 `terminfo` 或bund绑 `ncurses`. [\#7737](https://github.com/ClickHouse/ClickHouse/pull/7737) ([阿莫斯鸟](https://github.com/amosbird)) -- 修复 `test_storage_s3` 集成测试。 [\#7734](https://github.com/ClickHouse/ClickHouse/pull/7734) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 碌莽禄Support: `StorageFile(, null)` 将块插入给定格式的文件而不实际写入磁盘。 这是性能测试所必需的。 [\#8455](https://github.com/ClickHouse/ClickHouse/pull/8455) ([阿莫斯鸟](https://github.com/amosbird)) -- 添加参数 `--print-time` 功能测试打印每个测试的执行时间。 [\#8001](https://github.com/ClickHouse/ClickHouse/pull/8001) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 添加断言 `KeyCondition` 同时评估RPN。 这将修复来自gcc-9的警告。 [\#8279](https://github.com/ClickHouse/ClickHouse/pull/8279) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 在CI构建中转储cmake选项。 [\#8273](https://github.com/ClickHouse/ClickHouse/pull/8273) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- 不要为某些fat库生成调试信息。 [\#8271](https://github.com/ClickHouse/ClickHouse/pull/8271) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 赂眉露\>\> `log_to_console.xml` 始终登录到stderr,无论它是否交互。 [\#8395](https://github.com/ClickHouse/ClickHouse/pull/8395) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- 删除了一些未使用的功能 `clickhouse-performance-test` 工具 [\#8555](https://github.com/ClickHouse/ClickHouse/pull/8555) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 现在我们也将搜索 `lld-X` 与相应的 `clang-X` 版本。 [\#8092](https://github.com/ClickHouse/ClickHouse/pull/8092) ([阿利沙平](https://github.com/alesapin)) -- 实木复合地板建设改善。 [\#8421](https://github.com/ClickHouse/ClickHouse/pull/8421) ([马苏兰](https://github.com/maxulan)) -- 更多海湾合作委员会警告 [\#8221](https://github.com/ClickHouse/ClickHouse/pull/8221) ([kreuzerkrieg](https://github.com/kreuzerkrieg)) -- Arch Linux的软件包现在允许运行ClickHouse服务器,而不仅仅是客户端。 [\#8534](https://github.com/ClickHouse/ClickHouse/pull/8534) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 修复与处理器的测试。 微小的性能修复。 [\#7672](https://github.com/ClickHouse/ClickHouse/pull/7672) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 更新contrib/protobuf。 [\#8256](https://github.com/ClickHouse/ClickHouse/pull/8256) ([Matwey V.Kornilov](https://github.com/matwey)) -- 在准备切换到c++20作为新年庆祝活动。 “May the C++ force be with ClickHouse.” [\#8447](https://github.com/ClickHouse/ClickHouse/pull/8447) ([阿莫斯鸟](https://github.com/amosbird)) +- 构建修复 `ARM` 而在最小模式。 [#8304](https://github.com/ClickHouse/ClickHouse/pull/8304) ([proller](https://github.com/proller)) +- 添加复盖文件刷新 `clickhouse-server` 当不调用std::atexit时。 还略微改进了无状态测试的复盖率日志记录。 [#8267](https://github.com/ClickHouse/ClickHouse/pull/8267) ([阿利沙平](https://github.com/alesapin)) +- 更新contrib中的LLVM库。 避免从操作系统包中使用LLVM。 [#8258](https://github.com/ClickHouse/ClickHouse/pull/8258) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 使bund绑 `curl` 建立完全安静。 [#8232](https://github.com/ClickHouse/ClickHouse/pull/8232) [#8203](https://github.com/ClickHouse/ClickHouse/pull/8203) ([帕维尔\*科瓦连科](https://github.com/Jokser)) +- 修复一些 `MemorySanitizer` 警告。 [#8235](https://github.com/ClickHouse/ClickHouse/pull/8235) ([Alexander Kuzmenkov](https://github.com/akuzm)) +- 使用 `add_warning` 和 `no_warning` 宏 `CMakeLists.txt`. [#8604](https://github.com/ClickHouse/ClickHouse/pull/8604) ([伊万](https://github.com/abyss7)) +- 添加对Minio S3兼容对象的支持(https://min.io/)为了更好的集成测试。 [#7863](https://github.com/ClickHouse/ClickHouse/pull/7863) [#7875](https://github.com/ClickHouse/ClickHouse/pull/7875) ([帕维尔\*科瓦连科](https://github.com/Jokser)) +- 导入 `libc` 标题到contrib。 它允许在各种系统中使构建更加一致(仅适用于 `x86_64-linux-gnu`). [#5773](https://github.com/ClickHouse/ClickHouse/pull/5773) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 删除 `-fPIC` 从一些图书馆。 [#8464](https://github.com/ClickHouse/ClickHouse/pull/8464) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 清洁 `CMakeLists.txt` 对于卷曲。 看https://github.com/ClickHouse/ClickHouse/pull/8011#issuecomment-569478910 [#8459](https://github.com/ClickHouse/ClickHouse/pull/8459) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 无声警告 `CapNProto` 图书馆. [#8220](https://github.com/ClickHouse/ClickHouse/pull/8220) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 为短字符串优化哈希表添加性能测试。 [#7679](https://github.com/ClickHouse/ClickHouse/pull/7679) ([阿莫斯鸟](https://github.com/amosbird)) +- 现在ClickHouse将建立在 `AArch64` 即使 `MADV_FREE` 不可用。 这修复 [#8027](https://github.com/ClickHouse/ClickHouse/issues/8027). [#8243](https://github.com/ClickHouse/ClickHouse/pull/8243) ([阿莫斯鸟](https://github.com/amosbird)) +- 更新 `zlib-ng` 来解决记忆消毒的问题 [#7182](https://github.com/ClickHouse/ClickHouse/pull/7182) [#8206](https://github.com/ClickHouse/ClickHouse/pull/8206) ([Alexander Kuzmenkov](https://github.com/akuzm)) +- 在非Linux系统上启用内部MySQL库,因为操作系统包的使用非常脆弱,通常根本不起作用。 这修复 [#5765](https://github.com/ClickHouse/ClickHouse/issues/5765). [#8426](https://github.com/ClickHouse/ClickHouse/pull/8426) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修复了启用后在某些系统上构建的问题 `libc++`. 这取代了 [#8374](https://github.com/ClickHouse/ClickHouse/issues/8374). [#8380](https://github.com/ClickHouse/ClickHouse/pull/8380) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 赂眉露\>\> `Field` 方法更类型安全,以找到更多的错误。 [#7386](https://github.com/ClickHouse/ClickHouse/pull/7386) [#8209](https://github.com/ClickHouse/ClickHouse/pull/8209) ([Alexander Kuzmenkov](https://github.com/akuzm)) +- 添加丢失的文件到 `libc-headers` 子模块。 [#8507](https://github.com/ClickHouse/ClickHouse/pull/8507) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修复错误 `JSON` 引用性能测试输出。 [#8497](https://github.com/ClickHouse/ClickHouse/pull/8497) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 现在堆栈跟踪显示 `std::exception` 和 `Poco::Exception`. 在以前的版本中,它仅适用于 `DB::Exception`. 这改进了诊断。 [#8501](https://github.com/ClickHouse/ClickHouse/pull/8501) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 移植 `clock_gettime` 和 `clock_nanosleep` 对于新鲜的glibc版本。 [#8054](https://github.com/ClickHouse/ClickHouse/pull/8054) ([阿莫斯鸟](https://github.com/amosbird)) +- 启用 `part_log` 在示例配置开发人员。 [#8609](https://github.com/ClickHouse/ClickHouse/pull/8609) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修复重新加载的异步性质 `01036_no_superfluous_dict_reload_on_create_database*`. [#8111](https://github.com/ClickHouse/ClickHouse/pull/8111) ([Azat Khuzhin](https://github.com/azat)) +- 固定编解码器性能测试。 [#8615](https://github.com/ClickHouse/ClickHouse/pull/8615) ([瓦西里\*内姆科夫](https://github.com/Enmk)) +- 添加安装脚本 `.tgz` 为他们构建和文档。 [#8612](https://github.com/ClickHouse/ClickHouse/pull/8612) [#8591](https://github.com/ClickHouse/ClickHouse/pull/8591) ([阿利沙平](https://github.com/alesapin)) +- 删除旧 `ZSTD` 测试(它是在2016年创建的,以重现zstd1.0版本之前的错误)。 这修复 [#8618](https://github.com/ClickHouse/ClickHouse/issues/8618). [#8619](https://github.com/ClickHouse/ClickHouse/pull/8619) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 固定构建在Mac OS卡特琳娜。 [#8600](https://github.com/ClickHouse/ClickHouse/pull/8600) ([meo](https://github.com/meob)) +- 增加编解码器性能测试中的行数,以使结果显着。 [#8574](https://github.com/ClickHouse/ClickHouse/pull/8574) ([瓦西里\*内姆科夫](https://github.com/Enmk)) +- 在调试版本中,处理 `LOGICAL_ERROR` 异常作为断言失败,使得它们更容易被注意到。 [#8475](https://github.com/ClickHouse/ClickHouse/pull/8475) ([Alexander Kuzmenkov](https://github.com/akuzm)) +- 使与格式相关的性能测试更具确定性。 [#8477](https://github.com/ClickHouse/ClickHouse/pull/8477) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 更新 `lz4` 来修复记忆消毒器的故障 [#8181](https://github.com/ClickHouse/ClickHouse/pull/8181) ([Alexander Kuzmenkov](https://github.com/akuzm)) +- 在异常处理中抑制已知MemorySanitizer误报。 [#8182](https://github.com/ClickHouse/ClickHouse/pull/8182) ([Alexander Kuzmenkov](https://github.com/akuzm)) +- 更新 `gcc` 和 `g++` 到版本9在 `build/docker/build.sh` [#7766](https://github.com/ClickHouse/ClickHouse/pull/7766) ([TLightSky](https://github.com/tlightsky)) +- 添加性能测试用例来测试 `PREWHERE` 比 `WHERE`. [#7768](https://github.com/ClickHouse/ClickHouse/pull/7768) ([阿莫斯鸟](https://github.com/amosbird)) +- 在修复一个笨拙的测试方面取得了进展。 [#8621](https://github.com/ClickHouse/ClickHouse/pull/8621) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 避免从MemorySanitizer报告数据 `libunwind`. [#8539](https://github.com/ClickHouse/ClickHouse/pull/8539) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 更新 `libc++` 到最新版本。 [#8324](https://github.com/ClickHouse/ClickHouse/pull/8324) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 从源头构建ICU库。 这修复 [#6460](https://github.com/ClickHouse/ClickHouse/issues/6460). [#8219](https://github.com/ClickHouse/ClickHouse/pull/8219) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 从切换 `libressl` 到 `openssl`. ClickHouse应在此更改后支持TLS1.3和SNI。 这修复 [#8171](https://github.com/ClickHouse/ClickHouse/issues/8171). [#8218](https://github.com/ClickHouse/ClickHouse/pull/8218) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 使用时固定的UBSan报告 `chacha20_poly1305` 从SSL(发生在连接到https://yandex.ru/)。 [#8214](https://github.com/ClickHouse/ClickHouse/pull/8214) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修复默认密码文件的模式 `.deb` linux发行版。 [#8075](https://github.com/ClickHouse/ClickHouse/pull/8075) ([proller](https://github.com/proller)) +- 改进的表达式获取 `clickhouse-server` PID输入 `clickhouse-test`. [#8063](https://github.com/ClickHouse/ClickHouse/pull/8063) ([亚历山大\*卡扎科夫](https://github.com/Akazz)) +- 更新contrib/googletest到v1.10.0。 [#8587](https://github.com/ClickHouse/ClickHouse/pull/8587) ([Alexander Burmak](https://github.com/Alex-Burmak)) +- 修复了ThreadSaninitizer报告 `base64` 图书馆. 还将此库更新到最新版本,但无关紧要。 这修复 [#8397](https://github.com/ClickHouse/ClickHouse/issues/8397). [#8403](https://github.com/ClickHouse/ClickHouse/pull/8403) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修复 `00600_replace_running_query` 对于处理器。 [#8272](https://github.com/ClickHouse/ClickHouse/pull/8272) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 删除支持 `tcmalloc` 为了使 `CMakeLists.txt` 更简单 [#8310](https://github.com/ClickHouse/ClickHouse/pull/8310) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 发布海湾合作委员会构建现在使用 `libc++` 而不是 `libstdc++`. 最近 `libc++` 只与叮当一起使用。 这将提高构建配置的一致性和可移植性。 [#8311](https://github.com/ClickHouse/ClickHouse/pull/8311) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 使用MemorySanitizer启用ICU库进行构建。 [#8222](https://github.com/ClickHouse/ClickHouse/pull/8222) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 禁止从警告 `CapNProto` 图书馆. [#8224](https://github.com/ClickHouse/ClickHouse/pull/8224) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 删除代码的特殊情况 `tcmalloc`,因为它不再受支持。 [#8225](https://github.com/ClickHouse/ClickHouse/pull/8225) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 在CI coverage任务中,优雅地终止服务器以允许它保存coverage报告。 这修复了我们最近看到的不完整的复盖率报告。 [#8142](https://github.com/ClickHouse/ClickHouse/pull/8142) ([阿利沙平](https://github.com/alesapin)) +- 针对所有编解码器的性能测试 `Float64` 和 `UInt64` 值。 [#8349](https://github.com/ClickHouse/ClickHouse/pull/8349) ([瓦西里\*内姆科夫](https://github.com/Enmk)) +- `termcap` 非常不推荐使用,并导致各种问题(f.g.missing “up” 帽和呼应 `^J` 而不是多行)。 帮个忙 `terminfo` 或bund绑 `ncurses`. [#7737](https://github.com/ClickHouse/ClickHouse/pull/7737) ([阿莫斯鸟](https://github.com/amosbird)) +- 修复 `test_storage_s3` 集成测试。 [#7734](https://github.com/ClickHouse/ClickHouse/pull/7734) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 碌莽禄Support: `StorageFile(, null)` 将块插入给定格式的文件而不实际写入磁盘。 这是性能测试所必需的。 [#8455](https://github.com/ClickHouse/ClickHouse/pull/8455) ([阿莫斯鸟](https://github.com/amosbird)) +- 添加参数 `--print-time` 功能测试打印每个测试的执行时间。 [#8001](https://github.com/ClickHouse/ClickHouse/pull/8001) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 添加断言 `KeyCondition` 同时评估RPN。 这将修复来自gcc-9的警告。 [#8279](https://github.com/ClickHouse/ClickHouse/pull/8279) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 在CI构建中转储cmake选项。 [#8273](https://github.com/ClickHouse/ClickHouse/pull/8273) ([Alexander Kuzmenkov](https://github.com/akuzm)) +- 不要为某些fat库生成调试信息。 [#8271](https://github.com/ClickHouse/ClickHouse/pull/8271) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 赂眉露\>\> `log_to_console.xml` 始终登录到stderr,无论它是否交互。 [#8395](https://github.com/ClickHouse/ClickHouse/pull/8395) ([Alexander Kuzmenkov](https://github.com/akuzm)) +- 删除了一些未使用的功能 `clickhouse-performance-test` 工具 [#8555](https://github.com/ClickHouse/ClickHouse/pull/8555) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 现在我们也将搜索 `lld-X` 与相应的 `clang-X` 版本。 [#8092](https://github.com/ClickHouse/ClickHouse/pull/8092) ([阿利沙平](https://github.com/alesapin)) +- 实木复合地板建设改善。 [#8421](https://github.com/ClickHouse/ClickHouse/pull/8421) ([马苏兰](https://github.com/maxulan)) +- 更多海湾合作委员会警告 [#8221](https://github.com/ClickHouse/ClickHouse/pull/8221) ([kreuzerkrieg](https://github.com/kreuzerkrieg)) +- Arch Linux的软件包现在允许运行ClickHouse服务器,而不仅仅是客户端。 [#8534](https://github.com/ClickHouse/ClickHouse/pull/8534) ([Vladimir Chebotarev](https://github.com/excitoon)) +- 修复与处理器的测试。 微小的性能修复。 [#7672](https://github.com/ClickHouse/ClickHouse/pull/7672) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) +- 更新contrib/protobuf。 [#8256](https://github.com/ClickHouse/ClickHouse/pull/8256) ([Matwey V.Kornilov](https://github.com/matwey)) +- 在准备切换到c++20作为新年庆祝活动。 “May the C++ force be with ClickHouse.” [#8447](https://github.com/ClickHouse/ClickHouse/pull/8447) ([阿莫斯鸟](https://github.com/amosbird)) #### 实验特点 {#experimental-feature-1} -- 增加了实验设置 `min_bytes_to_use_mmap_io`. 它允许读取大文件,而无需将数据从内核复制到用户空间。 默认情况下禁用该设置。 建议的阈值大约是64MB,因为mmap/munmap很慢。 [\#8520](https://github.com/ClickHouse/ClickHouse/pull/8520) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 返工配额作为访问控制系统的一部分。 增加了新表 `system.quotas`,新功能 `currentQuota`, `currentQuotaKey`,新的SQL语法 `CREATE QUOTA`, `ALTER QUOTA`, `DROP QUOTA`, `SHOW QUOTA`. [\#7257](https://github.com/ClickHouse/ClickHouse/pull/7257) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) -- 允许跳过带有警告的未知设置,而不是引发异常。 [\#7653](https://github.com/ClickHouse/ClickHouse/pull/7653) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) -- 重新设计的行策略作为访问控制系统的一部分。 增加了新表 `system.row_policies`,新功能 `currentRowPolicies()`,新的SQL语法 `CREATE POLICY`, `ALTER POLICY`, `DROP POLICY`, `SHOW CREATE POLICY`, `SHOW POLICIES`. [\#7808](https://github.com/ClickHouse/ClickHouse/pull/7808) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) +- 增加了实验设置 `min_bytes_to_use_mmap_io`. 它允许读取大文件,而无需将数据从内核复制到用户空间。 默认情况下禁用该设置。 建议的阈值大约是64MB,因为mmap/munmap很慢。 [#8520](https://github.com/ClickHouse/ClickHouse/pull/8520) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 返工配额作为访问控制系统的一部分。 增加了新表 `system.quotas`,新功能 `currentQuota`, `currentQuotaKey`,新的SQL语法 `CREATE QUOTA`, `ALTER QUOTA`, `DROP QUOTA`, `SHOW QUOTA`. [#7257](https://github.com/ClickHouse/ClickHouse/pull/7257) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) +- 允许跳过带有警告的未知设置,而不是引发异常。 [#7653](https://github.com/ClickHouse/ClickHouse/pull/7653) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) +- 重新设计的行策略作为访问控制系统的一部分。 增加了新表 `system.row_policies`,新功能 `currentRowPolicies()`,新的SQL语法 `CREATE POLICY`, `ALTER POLICY`, `DROP POLICY`, `SHOW CREATE POLICY`, `SHOW POLICIES`. [#7808](https://github.com/ClickHouse/ClickHouse/pull/7808) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) #### 安全修复 {#security-fix} -- 修正了读取目录结构中的表的可能性 `File` 表引擎。 这修复 [\#8536](https://github.com/ClickHouse/ClickHouse/issues/8536). [\#8537](https://github.com/ClickHouse/ClickHouse/pull/8537) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修正了读取目录结构中的表的可能性 `File` 表引擎。 这修复 [#8536](https://github.com/ClickHouse/ClickHouse/issues/8536). [#8537](https://github.com/ClickHouse/ClickHouse/pull/8537) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) ## [更新日志2019](https://github.com/ClickHouse/ClickHouse/blob/master/docs/en/changelog/2019.md) {#changelog-for-2019} diff --git a/docs/zh/development/build-cross-osx.md b/docs/zh/development/build-cross-osx.md index 124c9742b2d..6a941f4e834 100644 --- a/docs/zh/development/build-cross-osx.md +++ b/docs/zh/development/build-cross-osx.md @@ -1,6 +1,6 @@ # 如何在Linux中编译Mac OS X ClickHouse {#ru-he-zai-linuxzhong-bian-yi-mac-os-x-clickhouse} -Linux机器也可以编译运行在OS X系统的`clickhouse`二进制包,这可以用于在Linux上跑持续集成测试。如果要在Mac OS X上直接构建ClickHouse,请参考另外一篇指南: https://clickhouse.tech/docs/zh/development/build\_osx/ +Linux机器也可以编译运行在OS X系统的`clickhouse`二进制包,这可以用于在Linux上跑持续集成测试。如果要在Mac OS X上直接构建ClickHouse,请参考另外一篇指南: https://clickhouse.tech/docs/zh/development/build_osx/ Mac OS X的交叉编译基于以下构建说明,请首先遵循它们。 diff --git a/docs/zh/development/build.md b/docs/zh/development/build.md index 639b33e3ef7..8cf371b7be5 100644 --- a/docs/zh/development/build.md +++ b/docs/zh/development/build.md @@ -24,7 +24,7 @@ cd ClickHouse 以下教程是在 Ubuntu Linux 中进行编译的示例。 通过适当的更改,它应该可以适用于任何其他的 Linux 发行版。 -仅支持具有 x86\_64、AArch64。 对 Power9 的支持是实验性的。 +仅支持具有 x86_64、AArch64。 对 Power9 的支持是实验性的。 ## 安装 Git 和 CMake 和 Ninja {#an-zhuang-git-he-cmake-he-ninja} diff --git a/docs/zh/development/contrib.md b/docs/zh/development/contrib.md index d382facdcc9..0129ee62ce7 100644 --- a/docs/zh/development/contrib.md +++ b/docs/zh/development/contrib.md @@ -20,7 +20,7 @@ | libpcg-随机 | [Apache许可证2.0](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libpcg-random/LICENSE-APACHE.txt) | | libressl | [OpenSSL许可证](https://github.com/ClickHouse-Extras/ssl/blob/master/COPYING) | | librdkafka | [BSD2-条款许可](https://github.com/edenhill/librdkafka/blob/363dcad5a23dc29381cc626620e68ae418b3af19/LICENSE) | -| libwidechar\_width | [CC0 1.0通用](https://github.com/ClickHouse/ClickHouse/blob/master/libs/libwidechar_width/LICENSE) | +| libwidechar_width | [CC0 1.0通用](https://github.com/ClickHouse/ClickHouse/blob/master/libs/libwidechar_width/LICENSE) | | llvm | [BSD3-条款许可](https://github.com/ClickHouse-Extras/llvm/blob/163def217817c90fb982a6daf384744d8472b92b/llvm/LICENSE.TXT) | | lz4 | [BSD2-条款许可](https://github.com/lz4/lz4/blob/c10863b98e1503af90616ae99725ecd120265dfb/LICENSE) | | mariadb-连接器-c | [LGPL v2.1](https://github.com/ClickHouse-Extras/mariadb-connector-c/blob/3.1/COPYING.LIB) | diff --git a/docs/zh/development/developer-instruction.md b/docs/zh/development/developer-instruction.md index 35bac0322bb..faa333f8e31 100644 --- a/docs/zh/development/developer-instruction.md +++ b/docs/zh/development/developer-instruction.md @@ -32,7 +32,7 @@ ClickHose支持Linux,FreeBSD 及 Mac OS X 系统。 git clone --recursive git@guthub.com:your_github_username/ClickHouse.git cd ClickHouse -请注意,您需要将*your\_github\_username* 替换成实际使用的账户名! +请注意,您需要将*your_github_username* 替换成实际使用的账户名! 这个指令将创建一个包含项目副本的`ClickHouse`工作目录。 @@ -142,7 +142,7 @@ Yandex官方当前使用GCC构建ClickHouse,因为它生成的机器代码性 mkdir build cd build -您也可以有多个不同类型的构建目录(例如,build\_release, build\_debug等等)。 +您也可以有多个不同类型的构建目录(例如,build_release, build_debug等等)。 在`build`目录下,通过运行CMake配置构建。 在第一次运行之前,请定义用于指定编译器的环境变量(本示例中为gcc 9 编译器)。 diff --git a/docs/zh/development/style.md b/docs/zh/development/style.md index a87476d7748..85a57658c06 100644 --- a/docs/zh/development/style.md +++ b/docs/zh/development/style.md @@ -317,7 +317,7 @@ std::string getName() const override { return "Memory"; } class StorageMemory : public IStorage ``` -**4.** `using` 的命名方式与类相同,或者以\_\_t\`命名。 +**4.** `using` 的命名方式与类相同,或者以__t\`命名。 **5.** 模板类型参数的名称:在简单的情况下,使用`T`; `T`,`U`; `T1`,`T2`。 @@ -389,7 +389,7 @@ FileQueueProcessor( timer (not m_timer) ``` -**14.** 对于 `enum` 中的常量,请使用带大写字母的驼峰标识。ALL\_CAPS 也可以接受。如果 `enum` 是非本地的,请使用 `enum class`。 +**14.** 对于 `enum` 中的常量,请使用带大写字母的驼峰标识。ALL_CAPS 也可以接受。如果 `enum` 是非本地的,请使用 `enum class`。 ``` cpp enum class CompressionMethod @@ -702,7 +702,7 @@ auto s = std::string{"Hello"}; **4.** 操作系统:Linux Ubuntu,不比 Precise 早。 -**5.** 代码是为x86\_64 CPU架构编写的。 +**5.** 代码是为x86_64 CPU架构编写的。 CPU指令集是我们服务器中支持的最小集合。 目前,它是SSE 4.2。 diff --git a/docs/zh/development/tests.md b/docs/zh/development/tests.md index 5f0ea6e2c90..d2690a86887 100644 --- a/docs/zh/development/tests.md +++ b/docs/zh/development/tests.md @@ -179,7 +179,7 @@ Clang 有更多有用的警告 - 您可以使用 `-Weverything` 查找它们并 ClickHouse模糊测试可以通过[libFuzzer](https://llvm.org/docs/LibFuzzer.html)和随机SQL查询实现。 所有的模糊测试都应使用sanitizers(Address及Undefined)。 -LibFuzzer用于对库代码进行独立的模糊测试。模糊器作为测试代码的一部分实现,并具有“\_fuzzer”名称后缀。 +LibFuzzer用于对库代码进行独立的模糊测试。模糊器作为测试代码的一部分实现,并具有“_fuzzer”名称后缀。 模糊测试示例在`src/Parsers/tests/lexer_fuzzer.cpp`。LibFuzzer配置、字典及语料库存放在`tests/fuzz`。 我们鼓励您为每个处理用户输入的功能编写模糊测试。 diff --git a/docs/zh/engines/table-engines/integrations/hdfs.md b/docs/zh/engines/table-engines/integrations/hdfs.md index da77b36bdb1..8d35dfeeb95 100644 --- a/docs/zh/engines/table-engines/integrations/hdfs.md +++ b/docs/zh/engines/table-engines/integrations/hdfs.md @@ -73,12 +73,12 @@ SELECT * FROM hdfs_engine_table LIMIT 2 1. 假设我们在HDFS上有几个TSV格式的文件,其中包含以下Uri: -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_3’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_3’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_3’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_3’ 1. 有几种方法可以创建由所有六个文件组成的表: diff --git a/docs/zh/engines/table-engines/integrations/kafka.md b/docs/zh/engines/table-engines/integrations/kafka.md index 557ab6a7845..0118b6795e4 100644 --- a/docs/zh/engines/table-engines/integrations/kafka.md +++ b/docs/zh/engines/table-engines/integrations/kafka.md @@ -103,7 +103,7 @@ Kafka 特性: SELECT level, sum(total) FROM daily GROUP BY level; ``` -为了提高性能,接受的消息被分组为 [max\_insert\_block\_size](../../../operations/settings/settings.md#settings-max_insert_block_size) 大小的块。如果未在 [stream\_flush\_interval\_ms](../../../operations/settings/settings.md#stream-flush-interval-ms) 毫秒内形成块,则不关心块的完整性,都会将数据刷新到表中。 +为了提高性能,接受的消息被分组为 [max_insert_block_size](../../../operations/settings/settings.md#settings-max_insert_block_size) 大小的块。如果未在 [stream_flush_interval_ms](../../../operations/settings/settings.md#stream-flush-interval-ms) 毫秒内形成块,则不关心块的完整性,都会将数据刷新到表中。 停止接收主题数据或更改转换逻辑,请 detach 物化视图: diff --git a/docs/zh/engines/table-engines/mergetree-family/custom-partitioning-key.md b/docs/zh/engines/table-engines/mergetree-family/custom-partitioning-key.md index cf3ac76c8ce..71249e39ed2 100644 --- a/docs/zh/engines/table-engines/mergetree-family/custom-partitioning-key.md +++ b/docs/zh/engines/table-engines/mergetree-family/custom-partitioning-key.md @@ -109,7 +109,7 @@ drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 5 12:09 201902_4_6_1 drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 1 16:48 detached ``` -‘201901\_1\_1\_0’,‘201901\_1\_7\_1’ 等文件夹是数据片段的目录。每个片段都与一个对应的分区相关,并且只包含这个月的数据(本例中的表按月分区)。 +‘201901_1_1_0’,‘201901_1_7_1’ 等文件夹是数据片段的目录。每个片段都与一个对应的分区相关,并且只包含这个月的数据(本例中的表按月分区)。 `detached` 目录存放着使用 [DETACH](../../../sql-reference/statements/alter.md#alter_detach-partition) 语句从表中卸载的片段。损坏的片段不会被删除而是也会移到该目录下。服务器不会去使用`detached`目录中的数据片段。因此你可以随时添加,删除或修改此目录中的数据 – 在运行 [ATTACH](../../../sql-reference/statements/alter.md#alter_attach-partition) 语句前,服务器不会感知到。 diff --git a/docs/zh/engines/table-engines/mergetree-family/graphitemergetree.md b/docs/zh/engines/table-engines/mergetree-family/graphitemergetree.md index 9af39bcf964..7440abcc027 100644 --- a/docs/zh/engines/table-engines/mergetree-family/graphitemergetree.md +++ b/docs/zh/engines/table-engines/mergetree-family/graphitemergetree.md @@ -75,7 +75,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ## 汇总配置的参数 {#rollup-configuration} -汇总的配置参数由服务器配置的 [graphite\_rollup](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) 参数定义。参数名称可以是任意的。允许为多个不同表创建多组配置并使用。 +汇总的配置参数由服务器配置的 [graphite_rollup](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) 参数定义。参数名称可以是任意的。允许为多个不同表创建多组配置并使用。 汇总配置的结构如下: diff --git a/docs/zh/engines/table-engines/mergetree-family/mergetree.md b/docs/zh/engines/table-engines/mergetree-family/mergetree.md index 0b886547229..2fffcbe7ef3 100644 --- a/docs/zh/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/zh/engines/table-engines/mergetree-family/mergetree.md @@ -202,7 +202,7 @@ ClickHouse 不要求主键惟一,所以你可以插入多条具有相同主键 长的主键会对插入性能和内存消耗有负面影响,但主键中额外的列并不影响 `SELECT` 查询的性能。 -可以使用 `ORDER BY tuple()` 语法创建没有主键的表。在这种情况下 ClickHouse 根据数据插入的顺序存储。如果在使用 `INSERT ... SELECT` 时希望保持数据的排序,请设置 [max\_insert\_threads = 1](../../../operations/settings/settings.md#settings-max-insert-threads)。 +可以使用 `ORDER BY tuple()` 语法创建没有主键的表。在这种情况下 ClickHouse 根据数据插入的顺序存储。如果在使用 `INSERT ... SELECT` 时希望保持数据的排序,请设置 [max_insert_threads = 1](../../../operations/settings/settings.md#settings-max-insert-threads)。 想要根据初始顺序进行数据查询,使用 [单线程查询](../../../operations/settings/settings.md#settings-max_threads) @@ -245,7 +245,7 @@ ClickHouse 会依据主键索引剪掉不符合的数据,依据按月分区的 SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' ``` -要检查 ClickHouse 执行一个查询时能否使用索引,可设置 [force\_index\_by\_date](../../../operations/settings/settings.md#settings-force_index_by_date) 和 [force\_primary\_key](../../../operations/settings/settings.md) 。 +要检查 ClickHouse 执行一个查询时能否使用索引,可设置 [force_index_by_date](../../../operations/settings/settings.md#settings-force_index_by_date) 和 [force_primary_key](../../../operations/settings/settings.md) 。 按月分区的分区键是只能读取包含适当范围日期的数据块。这种情况下,数据块会包含很多天(最多整月)的数据。在块中,数据按主键排序,主键第一列可能不包含日期。因此,仅使用日期而没有带主键前几个字段作为条件的查询将会导致需要读取超过这个指定日期以外的数据。 @@ -336,7 +336,7 @@ WHERE 子句中的条件包含对列的函数调用,如果列是索引的一 `set` 索引会对所有函数生效,其他索引对函数的生效情况见下表 -| 函数 (操作符) / 索引 | primary key | minmax | ngrambf\_v1 | tokenbf\_v1 | bloom\_filter | +| 函数 (操作符) / 索引 | primary key | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter | |------------------------------------------------------------------------------------------------------------|-------------|--------|-------------|-------------|---------------| | [equals (=, ==)](../../../sql-reference/functions/comparison-functions.md#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | | [notEquals(!=, \<\>)](../../../sql-reference/functions/comparison-functions.md#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | @@ -628,7 +628,7 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' `default` 存储策略意味着只使用一个卷,这个卷只包含一个在 `` 中定义的磁盘。表创建后,它的存储策略就不能改变了。 -可以通过 [background\_move\_pool\_size](../../../operations/settings/settings.md#background_move_pool_size) 设置调整执行后台任务的线程数。 +可以通过 [background_move_pool_size](../../../operations/settings/settings.md#background_move_pool_size) 设置调整执行后台任务的线程数。 ### 详细说明 {#details} @@ -646,7 +646,7 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' 更进一步,数据变异和分区冻结使用的是 [硬链接](https://en.wikipedia.org/wiki/Hard_link)。不同磁盘之间的硬链接是不支持的,所以在这种情况下数据片段都会被存储到初始化的那一块磁盘上。 -在后台,数据片段基于剩余空间(`move_factor`参数)根据卷在配置文件中定义的顺序进行转移。数据永远不会从最后一个移出也不会从第一个移入。可以通过系统表 [system.part\_log](../../../operations/system-tables/part_log.md#system_tables-part-log) (字段 `type = MOVE_PART`) 和 [system.parts](../../../operations/system-tables/parts.md#system_tables-parts) (字段 `path` 和 `disk`) 来监控后台的移动情况。同时,具体细节可以通过服务器日志查看。 +在后台,数据片段基于剩余空间(`move_factor`参数)根据卷在配置文件中定义的顺序进行转移。数据永远不会从最后一个移出也不会从第一个移入。可以通过系统表 [system.part_log](../../../operations/system-tables/part_log.md#system_tables-part-log) (字段 `type = MOVE_PART`) 和 [system.parts](../../../operations/system-tables/parts.md#system_tables-parts) (字段 `path` 和 `disk`) 来监控后台的移动情况。同时,具体细节可以通过服务器日志查看。 用户可以通过 [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](../../../sql-reference/statements/alter.md#alter_move-partition) 强制移动一个数据片段或分区到另外一个卷,所有后台移动的限制都会被考虑在内。这个查询会自行启动,无需等待后台操作完成。如果没有足够的可用空间或任何必须条件没有被满足,用户会收到报错信息。 diff --git a/docs/zh/engines/table-engines/mergetree-family/replication.md b/docs/zh/engines/table-engines/mergetree-family/replication.md index cc327bcadf5..07ef255d6bb 100644 --- a/docs/zh/engines/table-engines/mergetree-family/replication.md +++ b/docs/zh/engines/table-engines/mergetree-family/replication.md @@ -47,7 +47,7 @@ 如果配置文件中没有设置 ZooKeeper ,则无法创建复制表,并且任何现有的复制表都将变为只读。 -`SELECT` 查询并不需要借助 ZooKeeper ,复本并不影响 `SELECT` 的性能,查询复制表与非复制表速度是一样的。查询分布式表时,ClickHouse的处理方式可通过设置 [max\_replica\_delay\_for\_distributed\_queries](../../../operations/settings/settings.md#settings-max_replica_delay_for_distributed_queries) 和 [fallback\_to\_stale\_replicas\_for\_distributed\_queries](../../../operations/settings/settings.md) 修改。 +`SELECT` 查询并不需要借助 ZooKeeper ,复本并不影响 `SELECT` 的性能,查询复制表与非复制表速度是一样的。查询分布式表时,ClickHouse的处理方式可通过设置 [max_replica_delay_for_distributed_queries](../../../operations/settings/settings.md#settings-max_replica_delay_for_distributed_queries) 和 [fallback_to_stale_replicas_for_distributed_queries](../../../operations/settings/settings.md) 修改。 对于每个 `INSERT` 语句,会通过几个事务将十来个记录添加到 ZooKeeper。(确切地说,这是针对每个插入的数据块; 每个 INSERT 语句的每 `max_insert_block_size = 1048576` 行和最后剩余的都各算作一个块。)相比非复制表,写 zk 会导致 `INSERT` 的延迟略长一些。但只要你按照建议每秒不超过一个 `INSERT` 地批量插入数据,不会有任何问题。一个 ZooKeeper 集群能给整个 ClickHouse 集群支撑协调每秒几百个 `INSERT`。数据插入的吞吐量(每秒的行数)可以跟不用复制的数据一样高。 @@ -59,7 +59,7 @@ 单个数据块写入是原子的。 INSERT 的数据按每块最多 `max_insert_block_size = 1048576` 行进行分块,换句话说,如果 `INSERT` 插入的行少于 1048576,则该 INSERT 是原子的。 -数据块会去重。对于被多次写的相同数据块(大小相同且具有相同顺序的相同行的数据块),该块仅会写入一次。这样设计的原因是万一在网络故障时客户端应用程序不知道数据是否成功写入DB,此时可以简单地重复 `INSERT` 。把相同的数据发送给多个副本 INSERT 并不会有问题。因为这些 `INSERT` 是完全相同的(会被去重)。去重参数参看服务器设置 [merge\_tree](../../../operations/server-configuration-parameters/settings.md) 。(注意:Replicated\*MergeTree 才会去重,不需要 zookeeper 的不带 MergeTree 不会去重) +数据块会去重。对于被多次写的相同数据块(大小相同且具有相同顺序的相同行的数据块),该块仅会写入一次。这样设计的原因是万一在网络故障时客户端应用程序不知道数据是否成功写入DB,此时可以简单地重复 `INSERT` 。把相同的数据发送给多个副本 INSERT 并不会有问题。因为这些 `INSERT` 是完全相同的(会被去重)。去重参数参看服务器设置 [merge_tree](../../../operations/server-configuration-parameters/settings.md) 。(注意:Replicated\*MergeTree 才会去重,不需要 zookeeper 的不带 MergeTree 不会去重) 在复制期间,只有要插入的源数据通过网络传输。进一步的数据转换(合并)会在所有副本上以相同的方式进行处理执行。这样可以最大限度地减少网络使用,这意味着即使副本在不同的数据中心,数据同步也能工作良好。(能在不同数据中心中的同步数据是副本机制的主要目标。) diff --git a/docs/zh/engines/table-engines/special/buffer.md b/docs/zh/engines/table-engines/special/buffer.md index 611813cbac4..253aba2382a 100644 --- a/docs/zh/engines/table-engines/special/buffer.md +++ b/docs/zh/engines/table-engines/special/buffer.md @@ -4,13 +4,13 @@ Buffer(database, table, num_layers, min_time, max_time, min_rows, max_rows, min_bytes, max_bytes) -引擎的参数:database,table - 要刷新数据的表。可以使用返回字符串的常量表达式而不是数据库名称。 num\_layers - 并行层数。在物理上,该表将表示为 num\_layers 个独立缓冲区。建议值为16。min\_time,max\_time,min\_rows,max\_rows,min\_bytes,max\_bytes - 从缓冲区刷新数据的条件。 +引擎的参数:database,table - 要刷新数据的表。可以使用返回字符串的常量表达式而不是数据库名称。 num_layers - 并行层数。在物理上,该表将表示为 num_layers 个独立缓冲区。建议值为16。min_time,max_time,min_rows,max_rows,min_bytes,max_bytes - 从缓冲区刷新数据的条件。 -如果满足所有 «min» 条件或至少一个 «max» 条件,则从缓冲区刷新数据并将其写入目标表。min\_time,max\_time — 从第一次写入缓冲区时起以秒为单位的时间条件。min\_rows,max\_rows - 缓冲区中行数的条件。min\_bytes,max\_bytes - 缓冲区中字节数的条件。 +如果满足所有 «min» 条件或至少一个 «max» 条件,则从缓冲区刷新数据并将其写入目标表。min_time,max_time — 从第一次写入缓冲区时起以秒为单位的时间条件。min_rows,max_rows - 缓冲区中行数的条件。min_bytes,max_bytes - 缓冲区中字节数的条件。 -写入时,数据从 num\_layers 个缓冲区中随机插入。或者,如果插入数据的大小足够大(大于 max\_rows 或 max\_bytes ),则会绕过缓冲区将其写入目标表。 +写入时,数据从 num_layers 个缓冲区中随机插入。或者,如果插入数据的大小足够大(大于 max_rows 或 max_bytes ),则会绕过缓冲区将其写入目标表。 -每个 «num\_layers» 缓冲区刷新数据的条件是分别计算。例如,如果 num\_layers = 16 且 max\_bytes = 100000000,则最大RAM消耗将为1.6 GB。 +每个 «num_layers» 缓冲区刷新数据的条件是分别计算。例如,如果 num_layers = 16 且 max_bytes = 100000000,则最大RAM消耗将为1.6 GB。 示例: @@ -18,7 +18,7 @@ CREATE TABLE merge.hits_buffer AS merge.hits ENGINE = Buffer(merge, hits, 16, 10, 100, 10000, 1000000, 10000000, 100000000) ``` -创建一个 «merge.hits\_buffer» 表,其结构与 «merge.hits» 相同,并使用 Buffer 引擎。写入此表时,数据缓冲在 RAM 中,然后写入 «merge.hits» 表。创建了16个缓冲区。如果已经过了100秒,或者已写入100万行,或者已写入100 MB数据,则刷新每个缓冲区的数据;或者如果同时已经过了10秒并且已经写入了10,000行和10 MB的数据。例如,如果只写了一行,那么在100秒之后,都会被刷新。但是如果写了很多行,数据将会更快地刷新。 +创建一个 «merge.hits_buffer» 表,其结构与 «merge.hits» 相同,并使用 Buffer 引擎。写入此表时,数据缓冲在 RAM 中,然后写入 «merge.hits» 表。创建了16个缓冲区。如果已经过了100秒,或者已写入100万行,或者已写入100 MB数据,则刷新每个缓冲区的数据;或者如果同时已经过了10秒并且已经写入了10,000行和10 MB的数据。例如,如果只写了一行,那么在100秒之后,都会被刷新。但是如果写了很多行,数据将会更快地刷新。 当服务器停止时,使用 DROP TABLE 或 DETACH TABLE,缓冲区数据也会刷新到目标表。 @@ -40,7 +40,7 @@ PREWHERE,FINAL 和 SAMPLE 对缓冲表不起作用。这些条件将传递到 将数据添加到缓冲区时,其中一个缓冲区被锁定。如果同时从表执行读操作,则会导致延迟。 -插入到 Buffer 表中的数据可能以不同的顺序和不同的块写入目标表中。因此,Buffer 表很难用于正确写入 CollapsingMergeTree。为避免出现问题,您可以将 «num\_layers» 设置为1。 +插入到 Buffer 表中的数据可能以不同的顺序和不同的块写入目标表中。因此,Buffer 表很难用于正确写入 CollapsingMergeTree。为避免出现问题,您可以将 «num_layers» 设置为1。 如果目标表是复制表,则在写入 Buffer 表时会丢失复制表的某些预期特征。数据部分的行次序和大小的随机变化导致数据不能去重,这意味着无法对复制表进行可靠的 «exactly once» 写入。 diff --git a/docs/zh/engines/table-engines/special/distributed.md b/docs/zh/engines/table-engines/special/distributed.md index ad9f7958ede..2ee17c13fae 100644 --- a/docs/zh/engines/table-engines/special/distributed.md +++ b/docs/zh/engines/table-engines/special/distributed.md @@ -59,13 +59,13 @@ logs – 服务器配置文件中的集群名称。 每个服务器需要指定 `host`,`port`,和可选的 `user`,`password`,`secure`,`compression` 的参数: - `host` – 远程服务器地址。可以域名、IPv4或IPv6。如果指定域名,则服务在启动时发起一个 DNS 请求,并且请求结果会在服务器运行期间一直被记录。如果 DNS 请求失败,则服务不会启动。如果你修改了 DNS 记录,则需要重启服务。 -- `port` – 消息传递的 TCP 端口(「tcp\_port」配置通常设为 9000)。不要跟 http\_port 混淆。 +- `port` – 消息传递的 TCP 端口(「tcp_port」配置通常设为 9000)。不要跟 http_port 混淆。 - `user` – 用于连接远程服务器的用户名。默认值:default。该用户必须有权限访问该远程服务器。访问权限配置在 users.xml 文件中。更多信息,请查看«访问权限»部分。 - `password` – 用于连接远程服务器的密码。默认值:空字符串。 - `secure` – 是否使用ssl进行连接,设为true时,通常也应该设置 `port` = 9440。服务器也要监听 `9440` 并有正确的证书。 - `compression` - 是否使用数据压缩。默认值:true。 -配置了副本,读取操作会从每个分片里选择一个可用的副本。可配置负载平衡算法(挑选副本的方式) - 请参阅«load\_balancing»设置。 +配置了副本,读取操作会从每个分片里选择一个可用的副本。可配置负载平衡算法(挑选副本的方式) - 请参阅«load_balancing»设置。 如果跟服务器的连接不可用,则在尝试短超时的重连。如果重连失败,则选择下一个副本,依此类推。如果跟所有副本的连接尝试都失败,则尝试用相同的方式再重复几次。 该机制有利于系统可用性,但不保证完全容错:如有远程服务器能够接受连接,但无法正常工作或状况不佳。 @@ -92,13 +92,13 @@ logs – 服务器配置文件中的集群名称。 每个分片都可以在配置文件中定义权重。默认情况下,权重等于1。数据依据分片权重按比例分发到分片上。例如,如果有两个分片,第一个分片的权重是9,而第二个分片的权重是10,则发送 9 / 19 的行到第一个分片, 10 / 19 的行到第二个分片。 -分片可在配置文件中定义 ‘internal\_replication’ 参数。 +分片可在配置文件中定义 ‘internal_replication’ 参数。 此参数设置为«true»时,写操作只选一个正常的副本写入数据。如果分布式表的子表是复制表(\*ReplicaMergeTree),请使用此方案。换句话说,这其实是把数据的复制工作交给实际需要写入数据的表本身而不是分布式表。 若此参数设置为«false»(默认值),写操作会将数据写入所有副本。实质上,这意味着要分布式表本身来复制数据。这种方式不如使用复制表的好,因为不会检查副本的一致性,并且随着时间的推移,副本数据可能会有些不一样。 -选择将一行数据发送到哪个分片的方法是,首先计算分片表达式,然后将这个计算结果除以所有分片的权重总和得到余数。该行会发送到那个包含该余数的从’prev\_weight’到’prev\_weights + weight’的半闭半开区间对应的分片上,其中 ‘prev\_weights’ 是该分片前面的所有分片的权重和,‘weight’ 是该分片的权重。例如,如果有两个分片,第一个分片权重为9,而第二个分片权重为10,则余数在 \[0,9) 中的行发给第一个分片,余数在 \[9,19) 中的行发给第二个分片。 +选择将一行数据发送到哪个分片的方法是,首先计算分片表达式,然后将这个计算结果除以所有分片的权重总和得到余数。该行会发送到那个包含该余数的从’prev_weight’到’prev_weights + weight’的半闭半开区间对应的分片上,其中 ‘prev_weights’ 是该分片前面的所有分片的权重和,‘weight’ 是该分片的权重。例如,如果有两个分片,第一个分片权重为9,而第二个分片权重为10,则余数在 \[0,9) 中的行发给第一个分片,余数在 \[9,19) 中的行发给第二个分片。 分片表达式可以是由常量和表列组成的任何返回整数表达式。例如,您可以使用表达式 ‘rand()’ 来随机分配数据,或者使用 ‘UserID’ 来按用户 ID 的余数分布(相同用户的数据将分配到单个分片上,这可降低带有用户信息的 IN 和 JOIN 的语句运行的复杂度)。如果该列数据分布不够均匀,可以将其包装在散列函数中:intHash64(UserID)。 @@ -115,6 +115,6 @@ SELECT 查询会被发送到所有分片,并且无论数据在分片中如何 如果在 INSERT 到分布式表时服务器节点丢失或重启(如,设备故障),则插入的数据可能会丢失。如果在表目录中检测到损坏的数据分片,则会将其转移到«broken»子目录,并不再使用。 -启用 max\_parallel\_replicas 选项后,会在分表的所有副本上并行查询处理。更多信息,请参阅«设置,max\_parallel\_replicas»部分。 +启用 max_parallel_replicas 选项后,会在分表的所有副本上并行查询处理。更多信息,请参阅«设置,max_parallel_replicas»部分。 [原始文章](https://clickhouse.tech/docs/en/operations/table_engines/distributed/) diff --git a/docs/zh/engines/table-engines/special/external-data.md b/docs/zh/engines/table-engines/special/external-data.md index 8527c6a6676..a07a9cda67e 100644 --- a/docs/zh/engines/table-engines/special/external-data.md +++ b/docs/zh/engines/table-engines/special/external-data.md @@ -20,10 +20,10 @@ ClickHouse 允许向服务器发送处理查询所需的数据以及 SELECT 查 **–file** – 带有表存储的文件的路径,或者,它指的是STDIN。 只能从 stdin 中检索单个表。 -以下的参数是可选的:**–name** – 表的名称,如果省略,则采用 \_data。 +以下的参数是可选的:**–name** – 表的名称,如果省略,则采用 _data。 **–format** – 文件中的数据格式。 如果省略,则使用 TabSeparated。 -以下的参数必选一个:**–types** – 逗号分隔列类型的列表。例如:`UInt64,String`。列将被命名为 \_1,\_2,… +以下的参数必选一个:**–types** – 逗号分隔列类型的列表。例如:`UInt64,String`。列将被命名为 _1,_2,… **–structure**– 表结构的格式 `UserID UInt64`,`URL String`。定义列的名字以及类型。 在 «file» 中指定的文件将由 «format» 中指定的格式解析,使用在 «types» 或 «structure» 中指定的数据类型。该表将被上传到服务器,并在作为名称为 «name»临时表。 @@ -41,7 +41,7 @@ cat /etc/passwd | sed 's/:/\t/g' | clickhouse-client --query="SELECT shell, coun /bin/sync 1 ``` -当使用HTTP接口时,外部数据以 multipart/form-data 格式传递。每个表作为一个单独的文件传输。表名取自文件名。«query\_string» 传递参数 «name\_format»、«name\_types»和«name\_structure»,其中 «name» 是这些参数对应的表的名称。参数的含义与使用命令行客户端时的含义相同。 +当使用HTTP接口时,外部数据以 multipart/form-data 格式传递。每个表作为一个单独的文件传输。表名取自文件名。«query_string» 传递参数 «name_format»、«name_types»和«name_structure»,其中 «name» 是这些参数对应的表的名称。参数的含义与使用命令行客户端时的含义相同。 示例: diff --git a/docs/zh/engines/table-engines/special/join.md b/docs/zh/engines/table-engines/special/join.md index 22e67ba46d5..35a50c1a54d 100644 --- a/docs/zh/engines/table-engines/special/join.md +++ b/docs/zh/engines/table-engines/special/join.md @@ -93,16 +93,16 @@ SELECT joinGet('id_val_join', 'val', toUInt32(1)) 创建表时,会应用下列设置参数: -- [join\_use\_nulls](../../../operations/settings/settings.md#join_use_nulls) -- [max\_rows\_in\_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) -- [max\_bytes\_in\_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) -- [join\_overflow\_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) -- [join\_any\_take\_last\_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) +- [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) +- [max_rows_in_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) +- [max_bytes_in_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) +- [join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) +- [join_any_take_last_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) `Join`表不能在 `GLOBAL JOIN`操作中使用 - `Join`表创建及 [查询](../../../sql-reference/statements/select/index.md)时,允许使用[join\_use\_nulls](../../../operations/settings/settings.md#join_use_nulls)参数。如果使用不同的`join_use_nulls`设置,会导致表关联异常(取决于join的类型)。当使用函数 [joinGet](../../../sql-reference/functions/other-functions.md#joinget)时,请在建表和查询语句中使用相同的 `join_use_nulls` 参数设置。 + `Join`表创建及 [查询](../../../sql-reference/statements/select/index.md)时,允许使用[join_use_nulls](../../../operations/settings/settings.md#join_use_nulls)参数。如果使用不同的`join_use_nulls`设置,会导致表关联异常(取决于join的类型)。当使用函数 [joinGet](../../../sql-reference/functions/other-functions.md#joinget)时,请在建表和查询语句中使用相同的 `join_use_nulls` 参数设置。 ## 数据存储 {#data-storage} diff --git a/docs/zh/engines/table-engines/special/merge.md b/docs/zh/engines/table-engines/special/merge.md index 236d238d211..839842e108f 100644 --- a/docs/zh/engines/table-engines/special/merge.md +++ b/docs/zh/engines/table-engines/special/merge.md @@ -22,7 +22,7 @@ 示例 2 : -我们假定你有一个旧表(WatchLog\_old),你想改变数据分区了,但又不想把旧数据转移到新表(WatchLog\_new)里,并且你需要同时能看到这两个表的数据。 +我们假定你有一个旧表(WatchLog_old),你想改变数据分区了,但又不想把旧数据转移到新表(WatchLog_new)里,并且你需要同时能看到这两个表的数据。 CREATE TABLE WatchLog_old(date Date, UserId Int64, EventType String, Cnt UInt64) ENGINE=MergeTree(date, (UserId, EventType), 8192); diff --git a/docs/zh/faq/general.md b/docs/zh/faq/general.md index 51f4e0b153f..68053330405 100644 --- a/docs/zh/faq/general.md +++ b/docs/zh/faq/general.md @@ -10,7 +10,7 @@ ## 如果我在通过ODBC使用Oracle时遇到编码问题,该怎么办? {#oracle-odbc-encodings} -如果您通过ODBC驱动程序使用Oracle作为外部字典的源,则需要为 `NLS_LANG` 在变量 `/etc/default/clickhouse`. 欲了解更多详情,请参阅 [Oracle NLS\_常见问题](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html). +如果您通过ODBC驱动程序使用Oracle作为外部字典的源,则需要为 `NLS_LANG` 在变量 `/etc/default/clickhouse`. 欲了解更多详情,请参阅 [Oracle NLS_常见问题](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html). **示例** diff --git a/docs/zh/getting-started/example-datasets/amplab-benchmark.md b/docs/zh/getting-started/example-datasets/amplab-benchmark.md index f8722df680e..11a1c34b91e 100644 --- a/docs/zh/getting-started/example-datasets/amplab-benchmark.md +++ b/docs/zh/getting-started/example-datasets/amplab-benchmark.md @@ -2,7 +2,7 @@ 参考 https://amplab.cs.berkeley.edu/benchmark/ -需要您在https://aws.amazon.com注册一个免费的账号。注册时需要您提供信用卡、邮箱、电话等信息。之后可以在https://console.aws.amazon.com/iam/home?nc2=h\_m\_sc\#security\_credential获取新的访问密钥 +需要您在https://aws.amazon.com注册一个免费的账号。注册时需要您提供信用卡、邮箱、电话等信息。之后可以在https://console.aws.amazon.com/iam/home?nc2=h_m_sc#security_credential获取新的访问密钥 在控制台运行以下命令: diff --git a/docs/zh/getting-started/example-datasets/metrica.md b/docs/zh/getting-started/example-datasets/metrica.md index 325b6231bec..22dccc26dcb 100644 --- a/docs/zh/getting-started/example-datasets/metrica.md +++ b/docs/zh/getting-started/example-datasets/metrica.md @@ -7,7 +7,7 @@ toc_title: "Yandex\u6885\u7279\u91CC\u5361\u6570\u636E" Dataset由两个表组成,其中包含有关命中的匿名数据 (`hits_v1`)和访问 (`visits_v1`)的Yandex的。梅特里卡 你可以阅读更多关于Yandex的。梅特里卡 [ClickHouse历史](../../introduction/history.md) 科。 -数据集由两个表组成,其中任何一个都可以作为压缩表下载 `tsv.xz` 文件或作为准备的分区。 除此之外,该扩展版本 `hits` 包含1亿行的表可作为TSV在https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits\_100m\_obfuscated\_v1.tsv.xz 并作为准备的分区在https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits\_100m\_obfuscated\_v1.tar.xz. +数据集由两个表组成,其中任何一个都可以作为压缩表下载 `tsv.xz` 文件或作为准备的分区。 除此之外,该扩展版本 `hits` 包含1亿行的表可作为TSV在https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits_100m_obfuscated_v1.tsv.xz 并作为准备的分区在https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits_100m_obfuscated_v1.tar.xz. ## 从准备好的分区获取表 {#obtaining-tables-from-prepared-partitions} diff --git a/docs/zh/getting-started/example-datasets/nyc-taxi.md b/docs/zh/getting-started/example-datasets/nyc-taxi.md index 31f81c1d5c8..fa146d1ca38 100644 --- a/docs/zh/getting-started/example-datasets/nyc-taxi.md +++ b/docs/zh/getting-started/example-datasets/nyc-taxi.md @@ -190,7 +190,7 @@ real 75m56.214s (也可以直接使用`COPY ... TO PROGRAM`从Postgres中导入数据) -由于数据中与天气相关的所有数据(precipitation……average\_wind\_speed)都填充了NULL。 所以,我们将从最终数据集中删除它们 +由于数据中与天气相关的所有数据(precipitation……average_wind_speed)都填充了NULL。 所以,我们将从最终数据集中删除它们 首先,我们使用单台服务器创建表,后面我们将在多台节点上创建这些表。 diff --git a/docs/zh/getting-started/install.md b/docs/zh/getting-started/install.md index bbc79c3bf78..84791197ab6 100644 --- a/docs/zh/getting-started/install.md +++ b/docs/zh/getting-started/install.md @@ -2,9 +2,9 @@ ## 系统要求 {#xi-tong-yao-qiu} -ClickHouse可以在任何具有x86\_64,AArch64或PowerPC64LE CPU架构的Linux,FreeBSD或Mac OS X上运行。 +ClickHouse可以在任何具有x86_64,AArch64或PowerPC64LE CPU架构的Linux,FreeBSD或Mac OS X上运行。 -虽然预构建的二进制文件通常是为x86  \_64编译并利用SSE 4.2指令集,但除非另有说明,否则使用支持它的CPU将成为额外的系统要求。这是检查当前CPU是否支持SSE 4.2的命令: +虽然预构建的二进制文件通常是为x86  _64编译并利用SSE 4.2指令集,但除非另有说明,否则使用支持它的CPU将成为额外的系统要求。这是检查当前CPU是否支持SSE 4.2的命令: ``` bash $ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not supported" diff --git a/docs/zh/index.md b/docs/zh/index.md index 926c4ce2fdf..5294dc6c8c7 100644 --- a/docs/zh/index.md +++ b/docs/zh/index.md @@ -6,10 +6,10 @@ ClickHouse是一个用于联机分析(OLAP)的列式数据库管理系统(DBMS) | row | watchID | JavaEnable | title | GoodEvent | EventTime | |-----|-------------|------------|------------|-----------|---------------------| -| \#0 | 89354350662 | 1 | 投资者关系 | 1 | 2016-05-18 05:19:20 | -| \#1 | 90329509958 | 0 | 联系我们 | 1 | 2016-05-18 08:10:20 | -| \#2 | 89953706054 | 1 | 任务 | 1 | 2016-05-18 07:38:00 | -| \#N | … | … | … | … | … | +| #0 | 89354350662 | 1 | 投资者关系 | 1 | 2016-05-18 05:19:20 | +| #1 | 90329509958 | 0 | 联系我们 | 1 | 2016-05-18 08:10:20 | +| #2 | 89953706054 | 1 | 任务 | 1 | 2016-05-18 07:38:00 | +| #N | … | … | … | … | … | 处于同一行中的数据总是被物理的存储在一起。 @@ -18,7 +18,7 @@ ClickHouse是一个用于联机分析(OLAP)的列式数据库管理系统(DBMS) 在列式数据库系统中,数据按如下的顺序存储: -| row: | \#0 | \#1 | \#2 | \#N | +| row: | #0 | #1 | #2 | #N | |-------------|---------------------|---------------------|---------------------|-----| | watchID: | 89354350662 | 90329509958 | 89953706054 | … | | JavaEnable: | 1 | 0 | 1 | … | diff --git a/docs/zh/interfaces/formats.md b/docs/zh/interfaces/formats.md index cdbf2108e8d..58d06916ed8 100644 --- a/docs/zh/interfaces/formats.md +++ b/docs/zh/interfaces/formats.md @@ -159,7 +159,7 @@ SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORD - `min` 是具有最小值的行 `format_schema_rows` 格式(当极值设置为1时) - `max` 是具有最大值的行 `format_schema_rows` 格式(当极值设置为1时) - `rows` 输出行总数 -- `rows_before_limit` 是没有限制的最小行数。 仅当查询包含LIMIT时输出。 如果查询包含GROUP BY,则rows\_before\_limit\_at\_least是没有限制的确切行数。 +- `rows_before_limit` 是没有限制的最小行数。 仅当查询包含LIMIT时输出。 如果查询包含GROUP BY,则rows_before_limit_at_least是没有限制的确切行数。 - `time` 请求执行时间以秒为单位 - `rows_read` 已读取的行数 - `bytes_read` 被读取的字节数(未压缩) @@ -276,7 +276,7 @@ SELECT * FROM t_null FORMAT TSKV clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV" < data.csv -\*默认情况下间隔符是 `,` ,在 [format\_csv\_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter) 中可以了解更多间隔符配置。 +\*默认情况下间隔符是 `,` ,在 [format_csv_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter) 中可以了解更多间隔符配置。 解析的时候,可以使用或不使用引号来解析所有值。支持双引号和单引号。行也可以不用引号排列。 在这种情况下,它们被解析为逗号或换行符(CR 或 LF)。在解析不带引号的行时,若违反 RFC 规则,会忽略前导和尾随的空格和制表符。 对于换行,全部支持 Unix(LF),Windows(CR LF)和 Mac OS Classic(CR LF)。 @@ -365,12 +365,12 @@ SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase WITH TOTA } ``` -JSON 与 JavaScript 兼容。为了确保这一点,一些字符被另外转义:斜线`/`被转义为`\/`; 替代的换行符 `U+2028` 和 `U+2029` 会打断一些浏览器解析,它们会被转义为 `\uXXXX`。 ASCII 控制字符被转义:退格,换页,换行,回车和水平制表符被替换为`\b`,`\f`,`\n`,`\r`,`\t` 作为使用`\uXXXX`序列的00-1F范围内的剩余字节。 无效的 UTF-8 序列更改为替换字符 ,因此输出文本将包含有效的 UTF-8 序列。 为了与 JavaScript 兼容,默认情况下,Int64 和 UInt64 整数用双引号引起来。要除去引号,可以将配置参数 output\_format\_json\_quote\_64bit\_integers 设置为0。 +JSON 与 JavaScript 兼容。为了确保这一点,一些字符被另外转义:斜线`/`被转义为`\/`; 替代的换行符 `U+2028` 和 `U+2029` 会打断一些浏览器解析,它们会被转义为 `\uXXXX`。 ASCII 控制字符被转义:退格,换页,换行,回车和水平制表符被替换为`\b`,`\f`,`\n`,`\r`,`\t` 作为使用`\uXXXX`序列的00-1F范围内的剩余字节。 无效的 UTF-8 序列更改为替换字符 ,因此输出文本将包含有效的 UTF-8 序列。 为了与 JavaScript 兼容,默认情况下,Int64 和 UInt64 整数用双引号引起来。要除去引号,可以将配置参数 output_format_json_quote_64bit_integers 设置为0。 `rows` – 结果输出的行数。 `rows_before_limit_at_least` 去掉 LIMIT 过滤后的最小行总数。 只会在查询包含 LIMIT 条件时输出。 -若查询包含 GROUP BY,rows\_before\_limit\_at\_least 就是去掉 LIMIT 后过滤后的准确行数。 +若查询包含 GROUP BY,rows_before_limit_at_least 就是去掉 LIMIT 后过滤后的准确行数。 `totals` – 总值 (当使用 TOTALS 条件时)。 @@ -451,7 +451,7 @@ ClickHouse 支持 [NULL](../sql-reference/syntax.md), 在 JSON 格式中以 `nul ### 嵌套结构的使用 {#jsoneachrow-nested} -如果你有一张桌子 [嵌套式](../sql-reference/data-types/nested-data-structures/nested.md) 数据类型列,可以插入具有相同结构的JSON数据。 启用此功能与 [input\_format\_import\_nested\_json](../operations/settings/settings.md#settings-input_format_import_nested_json) 设置。 +如果你有一张桌子 [嵌套式](../sql-reference/data-types/nested-data-structures/nested.md) 数据类型列,可以插入具有相同结构的JSON数据。 启用此功能与 [input_format_import_nested_json](../operations/settings/settings.md#settings-input_format_import_nested_json) 设置。 例如,请考虑下表: @@ -465,7 +465,7 @@ CREATE TABLE json_each_row_nested (n Nested (s String, i Int32) ) ENGINE = Memor INSERT INTO json_each_row_nested FORMAT JSONEachRow {"n.s": ["abc", "def"], "n.i": [1, 23]} ``` -将数据作为分层JSON对象集插入 [input\_format\_import\_nested\_json=1](../operations/settings/settings.md#settings-input_format_import_nested_json). +将数据作为分层JSON对象集插入 [input_format_import_nested_json=1](../operations/settings/settings.md#settings-input_format_import_nested_json). ``` json { @@ -595,7 +595,7 @@ watch -n1 "clickhouse-client --query='SELECT event, value FROM system.events FOR ## PrettySpace {#prettyspace} -与 `PrettyCompact`(\#prettycompact) 格式不一样的是,它使用空格来代替网格来显示数据。 +与 `PrettyCompact`(#prettycompact) 格式不一样的是,它使用空格来代替网格来显示数据。 ## RowBinary {#rowbinary} @@ -763,7 +763,7 @@ SELECT SearchPhrase, count() AS c FROM test.hits c @1 :Uint64; } -格式文件存储的目录可以在服务配置中的 [format\_schema\_path](../operations/server-configuration-parameters/settings.md) 指定。 +格式文件存储的目录可以在服务配置中的 [format_schema_path](../operations/server-configuration-parameters/settings.md) 指定。 Cap’n Proto 反序列化是很高效的,通常不会增加系统的负载。 @@ -840,7 +840,7 @@ ClickHouse在输入和输出protobuf消息 `length-delimited` 格式。 ClickHouse Avro格式支持读取和写入 [Avro数据文件](http://avro.apache.org/docs/current/spec.html#Object+Container+Files). -### 数据类型匹配{\#sql\_reference/data\_types-matching} {#data-types-matching-sql_referencedata_types-matching} +### 数据类型匹配{#sql_reference/data_types-matching} {#data-types-matching-sql_referencedata_types-matching} 下表显示了支持的数据类型以及它们如何匹配ClickHouse [数据类型](../sql-reference/data-types/index.md) 在 `INSERT` 和 `SELECT` 查询。 @@ -894,7 +894,7 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Avro" > file.avro - 名,名,名,名 `[A-Za-z_]` - 随后只包含 `[A-Za-z0-9_]` -输出Avro文件压缩和同步间隔可以配置 [output\_format\_avro\_codec](../operations/settings/settings.md#settings-output_format_avro_codec) 和 [output\_format\_avro\_sync\_interval](../operations/settings/settings.md#settings-output_format_avro_sync_interval) 分别。 +输出Avro文件压缩和同步间隔可以配置 [output_format_avro_codec](../operations/settings/settings.md#settings-output_format_avro_codec) 和 [output_format_avro_sync_interval](../operations/settings/settings.md#settings-output_format_avro_sync_interval) 分别。 ## AvroConfluent {#data-format-avro-confluent} @@ -904,9 +904,9 @@ AvroConfluent支持解码单对象Avro消息常用于 [卡夫卡](https://kafka. 模式解析后会进行缓存。 -架构注册表URL配置为 [format\_avro\_schema\_registry\_url](../operations/settings/settings.md#settings-format_avro_schema_registry_url) +架构注册表URL配置为 [format_avro_schema_registry_url](../operations/settings/settings.md#settings-format_avro_schema_registry_url) -### 数据类型匹配{\#sql\_reference/data\_types-matching-1} {#data-types-matching-sql_referencedata_types-matching-1} +### 数据类型匹配{#sql_reference/data_types-matching-1} {#data-types-matching-sql_referencedata_types-matching-1} 和 [Avro](#data-format-avro) @@ -948,7 +948,7 @@ SELECT * FROM topic1_stream; [阿帕奇地板](http://parquet.apache.org/) 是Hadoop生态系统中普遍存在的列式存储格式。 ClickHouse支持此格式的读写操作。 -### 数据类型匹配{\#sql\_reference/data\_types-matching-2} {#data-types-matching-sql_referencedata_types-matching-2} +### 数据类型匹配{#sql_reference/data_types-matching-2} {#data-types-matching-sql_referencedata_types-matching-2} 下表显示了支持的数据类型以及它们如何匹配ClickHouse [数据类型](../sql-reference/data-types/index.md) 在 `INSERT` 和 `SELECT` 查询。 @@ -996,7 +996,7 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Parquet" > {some_ [阿帕奇兽人](https://orc.apache.org/) 是Hadoop生态系统中普遍存在的列式存储格式。 您只能将此格式的数据插入ClickHouse。 -### 数据类型匹配{\#sql\_reference/data\_types-matching-3} {#data-types-matching-sql_referencedata_types-matching-3} +### 数据类型匹配{#sql_reference/data_types-matching-3} {#data-types-matching-sql_referencedata_types-matching-3} 下表显示了支持的数据类型以及它们如何匹配ClickHouse [数据类型](../sql-reference/data-types/index.md) 在 `INSERT` 查询。 @@ -1047,15 +1047,15 @@ e.g. `schemafile.proto:MessageType`. 如果在批处理模式下使用客户端,则由于安全原因,架构的路径必须是相对的。 如果您通过输入或输出数据 [HTTP接口](../interfaces/http.md) 格式架构中指定的文件名 -应该位于指定的目录中 [format\_schema\_path](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-format_schema_path) +应该位于指定的目录中 [format_schema_path](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-format_schema_path) 在服务器配置中。 [原始文章](https://clickhouse.tech/docs/en/interfaces/formats/) ## 跳过错误 {#skippingerrors} -一些格式,如 `CSV`, `TabSeparated`, `TSKV`, `JSONEachRow`, `Template`, `CustomSeparated` 和 `Protobuf` 如果发生解析错误,可以跳过断开的行,并从下一行开始继续解析。 看 [input\_format\_allow\_errors\_num](../operations/settings/settings.md#settings-input_format_allow_errors_num) 和 -[input\_format\_allow\_errors\_ratio](../operations/settings/settings.md#settings-input_format_allow_errors_ratio) 设置。 +一些格式,如 `CSV`, `TabSeparated`, `TSKV`, `JSONEachRow`, `Template`, `CustomSeparated` 和 `Protobuf` 如果发生解析错误,可以跳过断开的行,并从下一行开始继续解析。 看 [input_format_allow_errors_num](../operations/settings/settings.md#settings-input_format_allow_errors_num) 和 +[input_format_allow_errors_ratio](../operations/settings/settings.md#settings-input_format_allow_errors_ratio) 设置。 限制: -在解析错误的情况下 `JSONEachRow` 跳过所有数据,直到新行(或EOF),所以行必须由 `\n` 正确计算错误。 - `Template` 和 `CustomSeparated` 在最后一列之后使用分隔符,并在行之间使用分隔符来查找下一行的开头,所以跳过错误只有在其中至少有一个不为空时才有效。 diff --git a/docs/zh/interfaces/http.md b/docs/zh/interfaces/http.md index 9feb8c5d69d..67ed93b5cfb 100644 --- a/docs/zh/interfaces/http.md +++ b/docs/zh/interfaces/http.md @@ -196,7 +196,7 @@ $ echo 'SELECT number FROM system.numbers LIMIT 10' | curl 'http://localhost:812 相比起 TCP 原生接口,HTTP 接口不支持会话和会话设置的概念,不允许中止查询(准确地说,只在少数情况下允许),不显示查询处理的进展。执行解析和数据格式化都是在服务端处理,网络上会比 TCP 原生接口更低效。 -可选的 `query_id` 参数可能当做 query ID 传入(或者任何字符串)。更多信息,参见 «[设置 replace\_running\_query](../operations/settings/settings.md)» 部分。 +可选的 `query_id` 参数可能当做 query ID 传入(或者任何字符串)。更多信息,参见 «[设置 replace_running_query](../operations/settings/settings.md)» 部分。 可选的 `quota_key` 参数可能当做 quota key 传入(或者任何字符串)。更多信息,参见 «[配额](../operations/quotas.md#quotas)» 部分。 diff --git a/docs/zh/interfaces/mysql.md b/docs/zh/interfaces/mysql.md index 7e50a070f96..d15d7756520 100644 --- a/docs/zh/interfaces/mysql.md +++ b/docs/zh/interfaces/mysql.md @@ -7,7 +7,7 @@ toc_title: "MySQL\u63A5\u53E3" # MySQL接口 {#mysql-interface} -ClickHouse支持MySQL线协议。 它可以通过启用 [mysql\_port](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-mysql_port) 在配置文件中设置: +ClickHouse支持MySQL线协议。 它可以通过启用 [mysql_port](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-mysql_port) 在配置文件中设置: ``` xml 9004 diff --git a/docs/zh/interfaces/third-party/client-libraries.md b/docs/zh/interfaces/third-party/client-libraries.md index 7bc4ae0aa27..77c929b9730 100644 --- a/docs/zh/interfaces/third-party/client-libraries.md +++ b/docs/zh/interfaces/third-party/client-libraries.md @@ -4,7 +4,7 @@ Yandex不维护下面列出的库,也没有进行任何广泛的测试以确保其质量。 - Python - - [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm) + - [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm) - [ツ环板driverョツ嘉ッツ偲](https://github.com/mymarilyn/clickhouse-driver) - [ツ环板clientョツ嘉ッツ偲](https://github.com/yurial/clickhouse-client) - PHP @@ -38,7 +38,7 @@ - [掳胫client-禄脢鹿脷露胫鲁隆鹿-client酶](https://github.com/crobox/clickhouse-scala-client) - Kotlin - [AORM](https://github.com/TanVD/AORM) -- C\# +- C# - [Octonica.ClickHouseClient](https://github.com/Octonica/ClickHouseClient) - [克莱克豪斯Ado](https://github.com/killwort/ClickHouse-Net) - [ClickHouse.Net](https://github.com/ilyabreev/ClickHouse.Net) diff --git a/docs/zh/interfaces/third-party/integrations.md b/docs/zh/interfaces/third-party/integrations.md index 374cecb4c9a..2c57e23b724 100644 --- a/docs/zh/interfaces/third-party/integrations.md +++ b/docs/zh/interfaces/third-party/integrations.md @@ -12,14 +12,14 @@ - [clickhouse-mysql-data-reader](https://github.com/Altinity/clickhouse-mysql-data-reader) - [horgh-复制器](https://github.com/larsnovikov/horgh-replicator) - [PostgreSQL](https://www.postgresql.org) - - [clickhousedb\_fdw](https://github.com/Percona-Lab/clickhousedb_fdw) - - [infi.clickhouse\_fdw](https://github.com/Infinidat/infi.clickhouse_fdw) (使用 [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) + - [clickhousedb_fdw](https://github.com/Percona-Lab/clickhousedb_fdw) + - [infi.clickhouse_fdw](https://github.com/Infinidat/infi.clickhouse_fdw) (使用 [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - [pg2ch](https://github.com/mkabilov/pg2ch) - [MSSQL](https://en.wikipedia.org/wiki/Microsoft_SQL_Server) - [ClickHouseMightrator](https://github.com/zlzforever/ClickHouseMigrator) - 消息队列 - [卡夫卡](https://kafka.apache.org) - - [clickhouse\_sinker](https://github.com/housepower/clickhouse_sinker) (使用 [去客户](https://github.com/ClickHouse/clickhouse-go/)) + - [clickhouse_sinker](https://github.com/housepower/clickhouse_sinker) (使用 [去客户](https://github.com/ClickHouse/clickhouse-go/)) - [stream-loader-clickhouse](https://github.com/adform/stream-loader) - 流处理 - [Flink](https://flink.apache.org) @@ -43,12 +43,12 @@ - [Grafana](https://grafana.com/) - [clickhouse-grafana](https://github.com/Vertamedia/clickhouse-grafana) - [普罗米修斯号](https://prometheus.io/) - - [clickhouse\_exporter](https://github.com/f1yegor/clickhouse_exporter) + - [clickhouse_exporter](https://github.com/f1yegor/clickhouse_exporter) - [PromHouse](https://github.com/Percona-Lab/PromHouse) - - [clickhouse\_exporter](https://github.com/hot-wifi/clickhouse_exporter) (用途 [去客户](https://github.com/kshvakov/clickhouse/)) + - [clickhouse_exporter](https://github.com/hot-wifi/clickhouse_exporter) (用途 [去客户](https://github.com/kshvakov/clickhouse/)) - [Nagios](https://www.nagios.org/) - - [check\_clickhouse](https://github.com/exogroup/check_clickhouse/) - - [check\_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py) + - [check_clickhouse](https://github.com/exogroup/check_clickhouse/) + - [check_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py) - [Zabbix](https://www.zabbix.com) - [ツ暗ェツ氾环催ツ団ツ法ツ人](https://github.com/Altinity/clickhouse-zabbix-template) - [Sematext](https://sematext.com/) @@ -68,7 +68,7 @@ - Python - [SQLAlchemy](https://www.sqlalchemy.org) - - [ツ暗ェツ氾环催ツ団ツ法ツ人](https://github.com/cloudflare/sqlalchemy-clickhouse) (使用 [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) + - [ツ暗ェツ氾环催ツ団ツ法ツ人](https://github.com/cloudflare/sqlalchemy-clickhouse) (使用 [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - [熊猫](https://pandas.pydata.org) - [pandahouse](https://github.com/kszucs/pandahouse) - PHP @@ -83,14 +83,14 @@ - 斯卡拉 - [Akka](https://akka.io) - [掳胫client-禄脢鹿脷露胫鲁隆鹿-client酶](https://github.com/crobox/clickhouse-scala-client) -- C\# +- C# - [ADO.NET](https://docs.microsoft.com/en-us/dotnet/framework/data/adonet/ado-net-overview) - [克莱克豪斯Ado](https://github.com/killwort/ClickHouse-Net) - [ClickHouse.Net](https://github.com/ilyabreev/ClickHouse.Net) - [ClickHouse.Net.Migrations](https://github.com/ilyabreev/ClickHouse.Net.Migrations) - 仙丹 - [Ecto](https://github.com/elixir-ecto/ecto) - - [clickhouse\_ecto](https://github.com/appodeal/clickhouse_ecto) + - [clickhouse_ecto](https://github.com/appodeal/clickhouse_ecto) - Ruby - [Ruby on Rails](https://rubyonrails.org/) - [activecube](https://github.com/bitquery/activecube) diff --git a/docs/zh/operations/configuration-files.md b/docs/zh/operations/configuration-files.md index aa0da86e8d0..ae760ee2d5d 100644 --- a/docs/zh/operations/configuration-files.md +++ b/docs/zh/operations/configuration-files.md @@ -13,7 +13,7 @@ ClickHouse支持多配置文件管理。主配置文件是`/etc/clickhouse-serve 如果指定`remove`属性,则删除该元素。 -此外,配置文件还可指定"substitutions"。如果一个元素有`incl`属性,则文件中的相应替换值将被使用。默认情况下,具有替换的文件的路径为`/etc/metrika.xml`。这可以在服务配置中的[include\_from](server-configuration-parameters/settings.md#server_configuration_parameters-include_from)元素中被修改。替换值在这个文件的`/yandex/substitution_name`元素中被指定。如果`incl`中指定的替换值不存在,则将其记录在日志中。为防止ClickHouse记录丢失的替换,请指定`optional="true"`属性(例如,[宏](server-configuration-parameters/settings.md)设置)。 +此外,配置文件还可指定"substitutions"。如果一个元素有`incl`属性,则文件中的相应替换值将被使用。默认情况下,具有替换的文件的路径为`/etc/metrika.xml`。这可以在服务配置中的[include_from](server-configuration-parameters/settings.md#server_configuration_parameters-include_from)元素中被修改。替换值在这个文件的`/yandex/substitution_name`元素中被指定。如果`incl`中指定的替换值不存在,则将其记录在日志中。为防止ClickHouse记录丢失的替换,请指定`optional="true"`属性(例如,[宏](server-configuration-parameters/settings.md)设置)。 替换也可以从ZooKeeper执行。为此,请指定属性`from_zk = "/path/to/node"`。元素值被替换为ZooKeeper节点`/path/to/node`的内容。您还可以将整个XML子树放在ZooKeeper节点上,并将其完全插入到源元素中。 diff --git a/docs/zh/operations/monitoring.md b/docs/zh/operations/monitoring.md index ee913f998ca..a5c30e46f4c 100644 --- a/docs/zh/operations/monitoring.md +++ b/docs/zh/operations/monitoring.md @@ -39,4 +39,4 @@ ClickHouse 收集的指标项: 此外,您可以通过HTTP API监视服务器可用性。 将HTTP GET请求发送到 `/ping`。 如果服务器可用,它将以 `200 OK` 响应。 -要监视服务器集群的配置,应设置[max\_replica\_delay\_for\_distributed\_queries](settings/settings.md#settings-max_replica_delay_for_distributed_queries)参数并使用HTTP资源`/replicas_status`。 如果副本可用,并且不延迟在其他副本之后,则对`/replicas_status`的请求将返回200 OK。 如果副本滞后,请求将返回 `503 HTTP_SERVICE_UNAVAILABLE`,包括有关待办事项大小的信息。 +要监视服务器集群的配置,应设置[max_replica_delay_for_distributed_queries](settings/settings.md#settings-max_replica_delay_for_distributed_queries)参数并使用HTTP资源`/replicas_status`。 如果副本可用,并且不延迟在其他副本之后,则对`/replicas_status`的请求将返回200 OK。 如果副本滞后,请求将返回 `503 HTTP_SERVICE_UNAVAILABLE`,包括有关待办事项大小的信息。 diff --git a/docs/zh/operations/optimizing-performance/sampling-query-profiler.md b/docs/zh/operations/optimizing-performance/sampling-query-profiler.md index a1d75fda0c3..7b89f31e440 100644 --- a/docs/zh/operations/optimizing-performance/sampling-query-profiler.md +++ b/docs/zh/operations/optimizing-performance/sampling-query-profiler.md @@ -11,11 +11,11 @@ ClickHouse运行允许分析查询执行的采样探查器。 使用探查器, 使用概要分析器: -- 设置 [trace\_log](../server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) 服务器配置部分。 +- 设置 [trace_log](../server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) 服务器配置部分。 - 本节配置 [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) 系统表包含探查器运行的结果。 它是默认配置的。 请记住,此表中的数据仅对正在运行的服务器有效。 服务器重新启动后,ClickHouse不会清理表,所有存储的虚拟内存地址都可能无效。 + 本节配置 [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) 系统表包含探查器运行的结果。 它是默认配置的。 请记住,此表中的数据仅对正在运行的服务器有效。 服务器重新启动后,ClickHouse不会清理表,所有存储的虚拟内存地址都可能无效。 -- 设置 [query\_profiler\_cpu\_time\_period\_ns](../settings/settings.md#query_profiler_cpu_time_period_ns) 或 [query\_profiler\_real\_time\_period\_ns](../settings/settings.md#query_profiler_real_time_period_ns) 设置。 这两种设置可以同时使用。 +- 设置 [query_profiler_cpu_time_period_ns](../settings/settings.md#query_profiler_cpu_time_period_ns) 或 [query_profiler_real_time_period_ns](../settings/settings.md#query_profiler_real_time_period_ns) 设置。 这两种设置可以同时使用。 这些设置允许您配置探查器计时器。 由于这些是会话设置,您可以为整个服务器、单个用户或用户配置文件、交互式会话以及每个单个查询获取不同的采样频率。 @@ -25,7 +25,7 @@ ClickHouse运行允许分析查询执行的采样探查器。 使用探查器, - 安装 `clickhouse-common-static-dbg` 包。 看 [从DEB软件包安装](../../getting-started/install.md#install-from-deb-packages). -- 允许由内省功能 [allow\_introspection\_functions](../settings/settings.md#settings-allow_introspection_functions) 设置。 +- 允许由内省功能 [allow_introspection_functions](../settings/settings.md#settings-allow_introspection_functions) 设置。 出于安全原因,默认情况下禁用内省功能。 diff --git a/docs/zh/operations/requirements.md b/docs/zh/operations/requirements.md index 28ac93de37d..c3013f738a2 100644 --- a/docs/zh/operations/requirements.md +++ b/docs/zh/operations/requirements.md @@ -7,7 +7,7 @@ toc_title: "要求" ## CPU {#cpu} -对于从预构建的deb包进行安装,请使用具有x86\_64架构并支持SSE4.2指令的CPU。 要使用不支持SSE4.2或具有AArch64或PowerPC64LE体系结构的处理器运行ClickHouse,您应该从源代码构建ClickHouse。 +对于从预构建的deb包进行安装,请使用具有x86_64架构并支持SSE4.2指令的CPU。 要使用不支持SSE4.2或具有AArch64或PowerPC64LE体系结构的处理器运行ClickHouse,您应该从源代码构建ClickHouse。 ClickHouse实现并行数据处理并使用所有可用的硬件资源。 在选择处理器时,考虑到ClickHouse在具有大量内核但时钟速率较低的配置中的工作效率要高于具有较少内核和较高时钟速率的配置。 例如,具有2600MHz的16核心优于具有3600MHz的8核心。 diff --git a/docs/zh/operations/server-configuration-parameters/settings.md b/docs/zh/operations/server-configuration-parameters/settings.md index 252323351b2..a400696fc11 100644 --- a/docs/zh/operations/server-configuration-parameters/settings.md +++ b/docs/zh/operations/server-configuration-parameters/settings.md @@ -5,7 +5,7 @@ toc_title: "\u670D\u52A1\u5668\u8BBE\u7F6E" # 服务器配置 {#server-settings} -## builtin\_dictionaries\_reload\_interval {#builtin-dictionaries-reload-interval} +## builtin_dictionaries_reload_interval {#builtin-dictionaries-reload-interval} 重新加载内置字典的间隔时间(以秒为单位)。 @@ -66,7 +66,7 @@ ClickHouse每x秒重新加载内置字典。 这使得编辑字典 “on the fly ``` -## default\_database {#default-database} +## default_database {#default-database} 默认数据库。 @@ -78,7 +78,7 @@ ClickHouse每x秒重新加载内置字典。 这使得编辑字典 “on the fly default ``` -## default\_profile {#default-profile} +## default_profile {#default-profile} 默认配置文件。 @@ -90,7 +90,7 @@ ClickHouse每x秒重新加载内置字典。 这使得编辑字典 “on the fly default ``` -## dictionaries\_config {#server_configuration_parameters-dictionaries_config} +## dictionaries_config {#server_configuration_parameters-dictionaries_config} 外部字典的配置文件的路径。 @@ -107,7 +107,7 @@ ClickHouse每x秒重新加载内置字典。 这使得编辑字典 “on the fly *_dictionary.xml ``` -## dictionaries\_lazy\_load {#server_configuration_parameters-dictionaries_lazy_load} +## dictionaries_lazy_load {#server_configuration_parameters-dictionaries_lazy_load} 延迟加载字典。 @@ -123,7 +123,7 @@ ClickHouse每x秒重新加载内置字典。 这使得编辑字典 “on the fly true ``` -## format\_schema\_path {#server_configuration_parameters-format_schema_path} +## format_schema_path {#server_configuration_parameters-format_schema_path} 包含输入数据方案的目录路径,例如输入数据的方案 [CapnProto](../../interfaces/formats.md#capnproto) 格式。 @@ -144,11 +144,11 @@ ClickHouse每x秒重新加载内置字典。 这使得编辑字典 “on the fly - port – The port on the Graphite server. - interval – The interval for sending, in seconds. - timeout – The timeout for sending data, in seconds. -- root\_path – Prefix for keys. +- root_path – Prefix for keys. - metrics – Sending data from the [系统。指标](../../operations/system-tables/metrics.md#system_tables-metrics) 桌子 - events – Sending deltas data accumulated for the time period from the [系统。活动](../../operations/system-tables/events.md#system_tables-events) 桌子 -- events\_cumulative – Sending cumulative data from the [系统。活动](../../operations/system-tables/events.md#system_tables-events) 桌子 -- asynchronous\_metrics – Sending data from the [系统。asynchronous\_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) 桌子 +- events_cumulative – Sending cumulative data from the [系统。活动](../../operations/system-tables/events.md#system_tables-events) 桌子 +- asynchronous_metrics – Sending data from the [系统。asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) 桌子 您可以配置多个 `` 条款 例如,您可以使用它以不同的时间间隔发送不同的数据。 @@ -168,7 +168,7 @@ ClickHouse每x秒重新加载内置字典。 这使得编辑字典 “on the fly ``` -## graphite\_rollup {#server_configuration_parameters-graphite-rollup} +## graphite_rollup {#server_configuration_parameters-graphite-rollup} 石墨细化数据的设置。 @@ -196,7 +196,7 @@ ClickHouse每x秒重新加载内置字典。 这使得编辑字典 “on the fly ``` -## http\_port/https\_port {#http-porthttps-port} +## http_port/https_port {#http-porthttps-port} 通过HTTP连接到服务器的端口。 @@ -210,7 +210,7 @@ ClickHouse每x秒重新加载内置字典。 这使得编辑字典 “on the fly 9999 ``` -## http\_server\_default\_response {#server_configuration_parameters-http_server_default_response} +## http_server_default_response {#server_configuration_parameters-http_server_default_response} 访问ClickHouse HTTP(s)服务器时默认显示的页面。 默认值为 “Ok.” (最后有换行符) @@ -225,7 +225,7 @@ ClickHouse每x秒重新加载内置字典。 这使得编辑字典 “on the fly ``` -## 包括\_从 {#server_configuration_parameters-include_from} +## 包括_从 {#server_configuration_parameters-include_from} 带替换的文件的路径。 @@ -237,7 +237,7 @@ ClickHouse每x秒重新加载内置字典。 这使得编辑字典 “on the fly /etc/metrica.xml ``` -## interserver\_http\_port {#interserver-http-port} +## interserver_http_port {#interserver-http-port} 用于在ClickHouse服务器之间交换数据的端口。 @@ -247,7 +247,7 @@ ClickHouse每x秒重新加载内置字典。 这使得编辑字典 “on the fly 9009 ``` -## interserver\_http\_host {#interserver-http-host} +## interserver_http_host {#interserver-http-host} 其他服务器可用于访问此服务器的主机名。 @@ -261,7 +261,7 @@ ClickHouse每x秒重新加载内置字典。 这使得编辑字典 “on the fly example.yandex.ru ``` -## interserver\_http\_credentials {#server-settings-interserver-http-credentials} +## interserver_http_credentials {#server-settings-interserver-http-credentials} 用户名和密码用于在以下期间进行身份验证 [复制](../../engines/table-engines/mergetree-family/replication.md) 与复制\*引擎。 这些凭据仅用于副本之间的通信,与ClickHouse客户端的凭据无关。 服务器正在检查这些凭据以连接副本,并在连接到其他副本时使用相同的凭据。 因此,这些凭据应该为集群中的所有副本设置相同。 默认情况下,不使用身份验证。 @@ -280,7 +280,7 @@ ClickHouse每x秒重新加载内置字典。 这使得编辑字典 “on the fly ``` -## keep\_alive\_timeout {#keep-alive-timeout} +## keep_alive_timeout {#keep-alive-timeout} ClickHouse在关闭连接之前等待传入请求的秒数。 默认为3秒。 @@ -290,7 +290,7 @@ ClickHouse在关闭连接之前等待传入请求的秒数。 默认为3秒。 3 ``` -## listen\_host {#server_configuration_parameters-listen_host} +## listen_host {#server_configuration_parameters-listen_host} 对请求可能来自的主机的限制。 如果您希望服务器回答所有这些问题,请指定 `::`. @@ -341,10 +341,10 @@ ClickHouse在关闭连接之前等待传入请求的秒数。 默认为3秒。 键: -- use\_syslog — Required setting if you want to write to the syslog. +- use_syslog — Required setting if you want to write to the syslog. - address — The host\[:port\] of syslogd. If omitted, the local daemon is used. - hostname — Optional. The name of the host that logs are sent from. -- facility — [系统日志工具关键字](https://en.wikipedia.org/wiki/Syslog#Facility) 在大写字母与 “LOG\_” 前缀: (`LOG_USER`, `LOG_DAEMON`, `LOG_LOCAL3`,等等)。 +- facility — [系统日志工具关键字](https://en.wikipedia.org/wiki/Syslog#Facility) 在大写字母与 “LOG_” 前缀: (`LOG_USER`, `LOG_DAEMON`, `LOG_LOCAL3`,等等)。 默认值: `LOG_USER` 如果 `address` 被指定, `LOG_DAEMON otherwise.` - format – Message format. Possible values: `bsd` 和 `syslog.` @@ -362,7 +362,7 @@ ClickHouse在关闭连接之前等待传入请求的秒数。 默认为3秒。 ``` -## mark\_cache\_size {#server-mark-cache-size} +## mark_cache_size {#server-mark-cache-size} 表引擎使用的标记缓存的近似大小(以字节为单位) [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) 家人 @@ -374,7 +374,7 @@ ClickHouse在关闭连接之前等待传入请求的秒数。 默认为3秒。 5368709120 ``` -## max\_concurrent\_queries {#max-concurrent-queries} +## max_concurrent_queries {#max-concurrent-queries} 同时处理的请求的最大数量。 @@ -384,7 +384,7 @@ ClickHouse在关闭连接之前等待传入请求的秒数。 默认为3秒。 100 ``` -## max\_connections {#max-connections} +## max_connections {#max-connections} 入站连接的最大数量。 @@ -394,7 +394,7 @@ ClickHouse在关闭连接之前等待传入请求的秒数。 默认为3秒。 4096 ``` -## max\_open\_files {#max-open-files} +## max_open_files {#max-open-files} 打开文件的最大数量。 @@ -408,7 +408,7 @@ ClickHouse在关闭连接之前等待传入请求的秒数。 默认为3秒。 262144 ``` -## max\_table\_size\_to\_drop {#max-table-size-to-drop} +## max_table_size_to_drop {#max-table-size-to-drop} 限制删除表。 @@ -426,7 +426,7 @@ ClickHouse在关闭连接之前等待传入请求的秒数。 默认为3秒。 0 ``` -## merge\_tree {#server_configuration_parameters-merge_tree} +## merge_tree {#server_configuration_parameters-merge_tree} 微调中的表 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). @@ -461,7 +461,7 @@ SSL客户端/服务器配置。 - sessionTimeout – Time for caching the session on the server. - extendedVerification – Automatically extended verification of certificates after the session ends. Acceptable values: `true`, `false`. - requireTLSv1 – Require a TLSv1 connection. Acceptable values: `true`, `false`. -- requireTLSv1\_1 – Require a TLSv1.1 connection. Acceptable values: `true`, `false`. +- requireTLSv1_1 – Require a TLSv1.1 connection. Acceptable values: `true`, `false`. - requireTLSv1 – Require a TLSv1.2 connection. Acceptable values: `true`, `false`. - fips – Activates OpenSSL FIPS mode. Supported if the library’s OpenSSL version supports FIPS. - privateKeyPassphraseHandler – Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: ``, `KeyFileHandler`, `test`, ``. @@ -499,11 +499,11 @@ SSL客户端/服务器配置。 ``` -## part\_log {#server_configuration_parameters-part-log} +## part_log {#server_configuration_parameters-part-log} 记录与之关联的事件 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). 例如,添加或合并数据。 您可以使用日志来模拟合并算法并比较它们的特征。 您可以可视化合并过程。 -查询记录在 [系统。part\_log](../../operations/system-tables/part_log.md#system_tables-part-log) 表,而不是在一个单独的文件。 您可以在以下命令中配置此表的名称 `table` 参数(见下文)。 +查询记录在 [系统。part_log](../../operations/system-tables/part_log.md#system_tables-part-log) 表,而不是在一个单独的文件。 您可以在以下命令中配置此表的名称 `table` 参数(见下文)。 使用以下参数配置日志记录: @@ -536,11 +536,11 @@ SSL客户端/服务器配置。 /var/lib/clickhouse/ ``` -## query\_log {#server_configuration_parameters-query-log} +## query_log {#server_configuration_parameters-query-log} -用于记录接收到的查询的设置 [log\_queries=1](../settings/settings.md) 设置。 +用于记录接收到的查询的设置 [log_queries=1](../settings/settings.md) 设置。 -查询记录在 [系统。query\_log](../../operations/system-tables/query_log.md#system_tables-query_log) 表,而不是在一个单独的文件。 您可以更改表的名称 `table` 参数(见下文)。 +查询记录在 [系统。query_log](../../operations/system-tables/query_log.md#system_tables-query_log) 表,而不是在一个单独的文件。 您可以更改表的名称 `table` 参数(见下文)。 使用以下参数配置日志记录: @@ -562,11 +562,11 @@ SSL客户端/服务器配置。 ``` -## query\_thread\_log {#server_configuration_parameters-query-thread-log} +## query_thread_log {#server_configuration_parameters-query-thread-log} -设置用于记录接收到的查询的线程 [log\_query\_threads=1](../settings/settings.md#settings-log-query-threads) 设置。 +设置用于记录接收到的查询的线程 [log_query_threads=1](../settings/settings.md#settings-log-query-threads) 设置。 -查询记录在 [系统。query\_thread\_log](../../operations/system-tables/query_thread_log.md#system_tables-query-thread-log) 表,而不是在一个单独的文件。 您可以更改表的名称 `table` 参数(见下文)。 +查询记录在 [系统。query_thread_log](../../operations/system-tables/query_thread_log.md#system_tables-query-thread-log) 表,而不是在一个单独的文件。 您可以更改表的名称 `table` 参数(见下文)。 使用以下参数配置日志记录: @@ -588,9 +588,9 @@ SSL客户端/服务器配置。 ``` -## trace\_log {#server_configuration_parameters-trace_log} +## trace_log {#server_configuration_parameters-trace_log} -设置为 [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) 系统表操作。 +设置为 [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) 系统表操作。 参数: @@ -610,7 +610,7 @@ SSL客户端/服务器配置。 ``` -## query\_masking\_rules {#query-masking-rules} +## query_masking_rules {#query-masking-rules} 基于正则表达式的规则,在将查询以及所有日志消息存储在服务器日志中之前,这些规则将应用于查询以及所有日志消息, `system.query_log`, `system.text_log`, `system.processes` 表,并在日志中发送给客户端。 这允许防止 @@ -641,7 +641,7 @@ SSL客户端/服务器配置。 对于分布式查询,每个服务器必须单独配置,否则,子查询传递给其他 节点将被存储而不屏蔽。 -## remote\_servers {#server-settings-remote-servers} +## remote_servers {#server-settings-remote-servers} 所使用的集群的配置 [分布](../../engines/table-engines/special/distributed.md) 表引擎和由 `cluster` 表功能。 @@ -655,7 +655,7 @@ SSL客户端/服务器配置。 **另请参阅** -- [skip\_unavailable\_shards](../settings/settings.md#settings-skip_unavailable_shards) +- [skip_unavailable_shards](../settings/settings.md#settings-skip_unavailable_shards) ## 时区 {#server_configuration_parameters-timezone} @@ -671,7 +671,7 @@ SSL客户端/服务器配置。 Europe/Moscow ``` -## tcp\_port {#server_configuration_parameters-tcp_port} +## tcp_port {#server_configuration_parameters-tcp_port} 通过TCP协议与客户端通信的端口。 @@ -681,7 +681,7 @@ SSL客户端/服务器配置。 9000 ``` -## tcp\_port\_secure {#server_configuration_parameters-tcp_port_secure} +## tcp_port_secure {#server_configuration_parameters-tcp_port_secure} TCP端口,用于与客户端进行安全通信。 使用它与 [OpenSSL](#server_configuration_parameters-openssl) 设置。 @@ -695,7 +695,7 @@ TCP端口,用于与客户端进行安全通信。 使用它与 [OpenSSL](#serv 9440 ``` -## mysql\_port {#server_configuration_parameters-mysql_port} +## mysql_port {#server_configuration_parameters-mysql_port} 通过MySQL协议与客户端通信的端口。 @@ -709,7 +709,7 @@ TCP端口,用于与客户端进行安全通信。 使用它与 [OpenSSL](#serv 9004 ``` -## tmp\_path {#server-settings-tmp_path} +## tmp_path {#server-settings-tmp_path} 用于处理大型查询的临时数据的路径。 @@ -722,7 +722,7 @@ TCP端口,用于与客户端进行安全通信。 使用它与 [OpenSSL](#serv /var/lib/clickhouse/tmp/ ``` -## tmp\_policy {#server-settings-tmp-policy} +## tmp_policy {#server-settings-tmp-policy} 从政策 [`storage_configuration`](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) 存储临时文件。 如果没有设置 [`tmp_path`](#server-settings-tmp_path) 被使用,否则被忽略。 @@ -733,11 +733,11 @@ TCP端口,用于与客户端进行安全通信。 使用它与 [OpenSSL](#serv - `max_data_part_size_bytes` 被忽略 -您必须在该政策中只有一个卷 -## uncompressed\_cache\_size {#server-settings-uncompressed_cache_size} +## uncompressed_cache_size {#server-settings-uncompressed_cache_size} 表引擎使用的未压缩数据的缓存大小(以字节为单位) [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). -服务器有一个共享缓存。 内存按需分配。 如果选项使用缓存 [use\_uncompressed\_cache](../settings/settings.md#setting-use_uncompressed_cache) 被启用。 +服务器有一个共享缓存。 内存按需分配。 如果选项使用缓存 [use_uncompressed_cache](../settings/settings.md#setting-use_uncompressed_cache) 被启用。 在个别情况下,未压缩的缓存对于非常短的查询是有利的。 @@ -747,7 +747,7 @@ TCP端口,用于与客户端进行安全通信。 使用它与 [OpenSSL](#serv 8589934592 ``` -## user\_files\_path {#server_configuration_parameters-user_files_path} +## user_files_path {#server_configuration_parameters-user_files_path} 包含用户文件的目录。 在表函数中使用 [文件()](../../sql-reference/table-functions/file.md). @@ -757,7 +757,7 @@ TCP端口,用于与客户端进行安全通信。 使用它与 [OpenSSL](#serv /var/lib/clickhouse/user_files/ ``` -## users\_config {#users-config} +## users_config {#users-config} 包含文件的路径: @@ -825,13 +825,13 @@ ClickHouse使用ZooKeeper存储复制表副本的元数据。 如果未使用复 - [复制](../../engines/table-engines/mergetree-family/replication.md) - [动物园管理员程序员指南](http://zookeeper.apache.org/doc/current/zookeeperProgrammers.html) -## use\_minimalistic\_part\_header\_in\_zookeeper {#server-settings-use_minimalistic_part_header_in_zookeeper} +## use_minimalistic_part_header_in_zookeeper {#server-settings-use_minimalistic_part_header_in_zookeeper} ZooKeeper中数据部分头的存储方法。 此设置仅适用于 `MergeTree` 家人 它可以指定: -- 在全球范围内 [merge\_tree](#server_configuration_parameters-merge_tree) 一节 `config.xml` 文件 +- 在全球范围内 [merge_tree](#server_configuration_parameters-merge_tree) 一节 `config.xml` 文件 ClickHouse使用服务器上所有表的设置。 您可以随时更改设置。 当设置更改时,现有表会更改其行为。 @@ -853,14 +853,14 @@ ZooKeeper中数据部分头的存储方法。 **默认值:** 0. -## disable\_internal\_dns\_cache {#server-settings-disable-internal-dns-cache} +## disable_internal_dns_cache {#server-settings-disable-internal-dns-cache} 禁用内部DNS缓存。 推荐用于在系统中运行ClickHouse 随着频繁变化的基础设施,如Kubernetes。 **默认值:** 0. -## dns\_cache\_update\_period {#server-settings-dns-cache-update-period} +## dns_cache_update_period {#server-settings-dns-cache-update-period} 更新存储在ClickHouse内部DNS缓存中的IP地址的周期(以秒为单位)。 更新是在一个单独的系统线程中异步执行的。 diff --git a/docs/zh/operations/settings/permissions-for-queries.md b/docs/zh/operations/settings/permissions-for-queries.md index 54ed1d5822e..aa7bc8c7937 100644 --- a/docs/zh/operations/settings/permissions-for-queries.md +++ b/docs/zh/operations/settings/permissions-for-queries.md @@ -18,7 +18,7 @@ ClickHouse中的查询可以分为几种类型: 以下设置按查询类型规范用户权限: - [只读](#settings_readonly) — Restricts permissions for all types of queries except DDL queries. -- [allow\_ddl](#settings_allow_ddl) — Restricts permissions for DDL queries. +- [allow_ddl](#settings_allow_ddl) — Restricts permissions for DDL queries. `KILL QUERY` 可以与任何设置进行。 @@ -43,7 +43,7 @@ ClickHouse中的查询可以分为几种类型: 默认值:0 -## allow\_ddl {#settings_allow_ddl} +## allow_ddl {#settings_allow_ddl} 允许或拒绝 [DDL](https://en.wikipedia.org/wiki/Data_definition_language) 查询。 diff --git a/docs/zh/operations/settings/query-complexity.md b/docs/zh/operations/settings/query-complexity.md index 1e87bdf8879..1e7dc82f7e1 100644 --- a/docs/zh/operations/settings/query-complexity.md +++ b/docs/zh/operations/settings/query-complexity.md @@ -5,8 +5,8 @@ 几乎所有的限制只适用于选择。对于分布式查询处理,每个服务器上分别应用限制。 Restrictions on the «maximum amount of something» can take the value 0, which means «unrestricted». -大多数限制也有一个 ‘overflow\_mode’ 设置,这意味着超过限制时该怎么做。 -它可以采用以下两个值之一: `throw` 或 `break`. 对聚合的限制(group\_by\_overflow\_mode)也具有以下值 `any`. +大多数限制也有一个 ‘overflow_mode’ 设置,这意味着超过限制时该怎么做。 +它可以采用以下两个值之一: `throw` 或 `break`. 对聚合的限制(group_by_overflow_mode)也具有以下值 `any`. `throw` – Throw an exception (default). @@ -24,7 +24,7 @@ Restrictions on the «maximum amount of something» can take the value 0, which 在HTTP接口中使用GET方法时, ‘readonly = 1’ 自动设置。 换句话说,对于修改数据的查询,您只能使用POST方法。 您可以在POST正文或URL参数中发送查询本身。 -## max\_memory\_usage {#settings_max_memory_usage} +## max_memory_usage {#settings_max_memory_usage} 用于在单个服务器上运行查询的最大RAM量。 @@ -41,155 +41,155 @@ Restrictions on the «maximum amount of something» can take the value 0, which 内存消耗也受到参数的限制 `max_memory_usage_for_user` 和 `max_memory_usage_for_all_queries`. -## max\_memory\_usage\_for\_user {#max-memory-usage-for-user} +## max_memory_usage_for_user {#max-memory-usage-for-user} 用于在单个服务器上运行用户查询的最大RAM量。 默认值定义在 [设置。h](https://github.com/ClickHouse/ClickHouse/blob/master/src/Interpreters/Settings.h#L244). 默认情况下,金额不受限制 (`max_memory_usage_for_user = 0`). -另请参阅说明 [max\_memory\_usage](#settings_max_memory_usage). +另请参阅说明 [max_memory_usage](#settings_max_memory_usage). -## max\_memory\_usage\_for\_all\_queries {#max-memory-usage-for-all-queries} +## max_memory_usage_for_all_queries {#max-memory-usage-for-all-queries} 用于在单个服务器上运行所有查询的最大RAM数量。 默认值定义在 [设置。h](https://github.com/ClickHouse/ClickHouse/blob/master/src/Interpreters/Settings.h#L245). 默认情况下,金额不受限制 (`max_memory_usage_for_all_queries = 0`). -另请参阅说明 [max\_memory\_usage](#settings_max_memory_usage). +另请参阅说明 [max_memory_usage](#settings_max_memory_usage). -## max\_rows\_to\_read {#max-rows-to-read} +## max_rows_to_read {#max-rows-to-read} 可以在每个块(而不是每行)上检查以下限制。 也就是说,限制可以打破一点。 运行查询时可从表中读取的最大行数。 -## max\_bytes\_to\_read {#max-bytes-to-read} +## max_bytes_to_read {#max-bytes-to-read} 运行查询时可以从表中读取的最大字节数(未压缩数据)。 -## read\_overflow\_mode {#read-overflow-mode} +## read_overflow_mode {#read-overflow-mode} 读取的数据量超过其中一个限制时该怎么办: ‘throw’ 或 ‘break’. 默认情况下,扔。 -## max\_rows\_to\_group\_by {#max-rows-to-group-by} +## max_rows_to_group_by {#max-rows-to-group-by} 从聚合接收的唯一密钥的最大数量。 此设置允许您在聚合时限制内存消耗。 -## group\_by\_overflow\_mode {#group-by-overflow-mode} +## group_by_overflow_mode {#group-by-overflow-mode} 当聚合的唯一键数超过限制时该怎么办: ‘throw’, ‘break’,或 ‘any’. 默认情况下,扔。 使用 ‘any’ 值允许您运行GROUP BY的近似值。 这种近似值的质量取决于数据的统计性质。 -## max\_rows\_to\_sort {#max-rows-to-sort} +## max_rows_to_sort {#max-rows-to-sort} 排序前的最大行数。 这允许您在排序时限制内存消耗。 -## max\_bytes\_to\_sort {#max-bytes-to-sort} +## max_bytes_to_sort {#max-bytes-to-sort} 排序前的最大字节数。 -## sort\_overflow\_mode {#sort-overflow-mode} +## sort_overflow_mode {#sort-overflow-mode} 如果排序前收到的行数超过其中一个限制,该怎么办: ‘throw’ 或 ‘break’. 默认情况下,扔。 -## max\_result\_rows {#max-result-rows} +## max_result_rows {#max-result-rows} 限制结果中的行数。 还检查子查询,并在运行分布式查询的部分时在远程服务器上。 -## max\_result\_bytes {#max-result-bytes} +## max_result_bytes {#max-result-bytes} 限制结果中的字节数。 与之前的设置相同。 -## result\_overflow\_mode {#result-overflow-mode} +## result_overflow_mode {#result-overflow-mode} 如果结果的体积超过其中一个限制,该怎么办: ‘throw’ 或 ‘break’. 默认情况下,扔。 使用 ‘break’ 类似于使用限制。 -## max\_execution\_time {#max-execution-time} +## max_execution_time {#max-execution-time} 最大查询执行时间(以秒为单位)。 此时,不会检查其中一个排序阶段,也不会在合并和最终确定聚合函数时进行检查。 -## timeout\_overflow\_mode {#timeout-overflow-mode} +## timeout_overflow_mode {#timeout-overflow-mode} -如果查询的运行时间长于 ‘max\_execution\_time’: ‘throw’ 或 ‘break’. 默认情况下,扔。 +如果查询的运行时间长于 ‘max_execution_time’: ‘throw’ 或 ‘break’. 默认情况下,扔。 -## min\_execution\_speed {#min-execution-speed} +## min_execution_speed {#min-execution-speed} -以每秒行为单位的最小执行速度。 检查每个数据块时 ‘timeout\_before\_checking\_execution\_speed’ 到期。 如果执行速度较低,则会引发异常。 +以每秒行为单位的最小执行速度。 检查每个数据块时 ‘timeout_before_checking_execution_speed’ 到期。 如果执行速度较低,则会引发异常。 -## timeout\_before\_checking\_execution\_speed {#timeout-before-checking-execution-speed} +## timeout_before_checking_execution_speed {#timeout-before-checking-execution-speed} -检查执行速度是不是太慢(不低于 ‘min\_execution\_speed’),在指定的时间以秒为单位已过期之后。 +检查执行速度是不是太慢(不低于 ‘min_execution_speed’),在指定的时间以秒为单位已过期之后。 -## max\_columns\_to\_read {#max-columns-to-read} +## max_columns_to_read {#max-columns-to-read} 单个查询中可从表中读取的最大列数。 如果查询需要读取更多列,则会引发异常。 -## max\_temporary\_columns {#max-temporary-columns} +## max_temporary_columns {#max-temporary-columns} 运行查询时必须同时保留在RAM中的最大临时列数,包括常量列。 如果有比这更多的临时列,它会引发异常。 -## max\_temporary\_non\_const\_columns {#max-temporary-non-const-columns} +## max_temporary_non_const_columns {#max-temporary-non-const-columns} -同样的事情 ‘max\_temporary\_columns’,但不计数常数列。 +同样的事情 ‘max_temporary_columns’,但不计数常数列。 请注意,常量列在运行查询时经常形成,但它们需要大约零计算资源。 -## max\_subquery\_depth {#max-subquery-depth} +## max_subquery_depth {#max-subquery-depth} 子查询的最大嵌套深度。 如果子查询更深,则会引发异常。 默认情况下,100。 -## max\_pipeline\_depth {#max-pipeline-depth} +## max_pipeline_depth {#max-pipeline-depth} 最大管道深度。 对应于查询处理期间每个数据块经历的转换数。 在单个服务器的限制范围内计算。 如果管道深度较大,则会引发异常。 默认情况下,1000。 -## max\_ast\_depth {#max-ast-depth} +## max_ast_depth {#max-ast-depth} 查询语法树的最大嵌套深度。 如果超出,将引发异常。 此时,在解析过程中不会对其进行检查,而是仅在解析查询之后进行检查。 也就是说,在分析过程中可以创建一个太深的语法树,但查询将失败。 默认情况下,1000。 -## max\_ast\_elements {#max-ast-elements} +## max_ast_elements {#max-ast-elements} 查询语法树中的最大元素数。 如果超出,将引发异常。 与前面的设置相同,只有在解析查询后才会检查它。 默认情况下,50,000。 -## max\_rows\_in\_set {#max-rows-in-set} +## max_rows_in_set {#max-rows-in-set} 从子查询创建的IN子句中数据集的最大行数。 -## max\_bytes\_in\_set {#max-bytes-in-set} +## max_bytes_in_set {#max-bytes-in-set} 从子查询创建的IN子句中的集合使用的最大字节数(未压缩数据)。 -## set\_overflow\_mode {#set-overflow-mode} +## set_overflow_mode {#set-overflow-mode} 当数据量超过其中一个限制时该怎么办: ‘throw’ 或 ‘break’. 默认情况下,扔。 -## max\_rows\_in\_distinct {#max-rows-in-distinct} +## max_rows_in_distinct {#max-rows-in-distinct} 使用DISTINCT时的最大不同行数。 -## max\_bytes\_in\_distinct {#max-bytes-in-distinct} +## max_bytes_in_distinct {#max-bytes-in-distinct} 使用DISTINCT时哈希表使用的最大字节数。 -## distinct\_overflow\_mode {#distinct-overflow-mode} +## distinct_overflow_mode {#distinct-overflow-mode} 当数据量超过其中一个限制时该怎么办: ‘throw’ 或 ‘break’. 默认情况下,扔。 -## max\_rows\_to\_transfer {#max-rows-to-transfer} +## max_rows_to_transfer {#max-rows-to-transfer} 使用GLOBAL IN时,可以传递到远程服务器或保存在临时表中的最大行数。 -## max\_bytes\_to\_transfer {#max-bytes-to-transfer} +## max_bytes_to_transfer {#max-bytes-to-transfer} 使用GLOBAL IN时,可以传递到远程服务器或保存在临时表中的最大字节数(未压缩数据)。 -## transfer\_overflow\_mode {#transfer-overflow-mode} +## transfer_overflow_mode {#transfer-overflow-mode} 当数据量超过其中一个限制时该怎么办: ‘throw’ 或 ‘break’. 默认情况下,扔。 -## max\_rows\_in\_join {#settings-max_rows_in_join} +## max_rows_in_join {#settings-max_rows_in_join} Limits the number of rows in the hash table that is used when joining tables. @@ -197,7 +197,7 @@ This settings applies to [SELECT … JOIN](../../sql-reference/statements/select If a query contains multiple joins, ClickHouse checks this setting for every intermediate result. -ClickHouse can proceed with different actions when the limit is reached. Use the [join\_overflow\_mode](#settings-join_overflow_mode) setting to choose the action. +ClickHouse can proceed with different actions when the limit is reached. Use the [join_overflow_mode](#settings-join_overflow_mode) setting to choose the action. Possible values: @@ -206,7 +206,7 @@ Possible values: Default value: 0. -## max\_bytes\_in\_join {#settings-max_bytes_in_join} +## max_bytes_in_join {#settings-max_bytes_in_join} Limits the size in bytes of the hash table used when joining tables. @@ -214,7 +214,7 @@ This settings applies to [SELECT … JOIN](../../sql-reference/statements/select If the query contains joins, ClickHouse checks this setting for every intermediate result. -ClickHouse can proceed with different actions when the limit is reached. Use [join\_overflow\_mode](#settings-join_overflow_mode) settings to choose the action. +ClickHouse can proceed with different actions when the limit is reached. Use [join_overflow_mode](#settings-join_overflow_mode) settings to choose the action. Possible values: @@ -223,12 +223,12 @@ Possible values: Default value: 0. -## join\_overflow\_mode {#settings-join_overflow_mode} +## join_overflow_mode {#settings-join_overflow_mode} Defines what action ClickHouse performs when any of the following join limits is reached: -- [max\_bytes\_in\_join](#settings-max_bytes_in_join) -- [max\_rows\_in\_join](#settings-max_rows_in_join) +- [max_bytes_in_join](#settings-max_bytes_in_join) +- [max_rows_in_join](#settings-max_rows_in_join) Possible values: @@ -242,7 +242,7 @@ Default value: `THROW`. - [JOIN clause](../../sql-reference/statements/select/join.md#select-join) - [Join table engine](../../engines/table-engines/special/join.md) -## max\_bytes\_before\_external\_group\_by {#settings-max_bytes_before_external_group_by} +## max_bytes_before_external_group_by {#settings-max_bytes_before_external_group_by} Enables or disables execution of `GROUP BY` clauses in external memory. See [GROUP BY in external memory](../../sql-reference/statements/select/group-by.md#select-group-by-in-external-memory). diff --git a/docs/zh/operations/settings/settings-users.md b/docs/zh/operations/settings/settings-users.md index 6a10d66453c..bb630a43e43 100644 --- a/docs/zh/operations/settings/settings-users.md +++ b/docs/zh/operations/settings/settings-users.md @@ -128,7 +128,7 @@ DNS请求的所有结果都将被缓存,直到服务器重新启动。 127.0.0.1 ``` -### user\_name/profile {#user-nameprofile} +### user_name/profile {#user-nameprofile} 您可以为用户分配设置配置文件。 设置配置文件在单独的部分配置 `users.xml` 文件 有关详细信息,请参阅 [设置配置文件](settings-profiles.md). diff --git a/docs/zh/operations/settings/settings.md b/docs/zh/operations/settings/settings.md index 2befb7ee15d..f834ab74f5a 100644 --- a/docs/zh/operations/settings/settings.md +++ b/docs/zh/operations/settings/settings.md @@ -5,7 +5,7 @@ machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd # 设置 {#settings} -## 分布\_产品\_模式 {#distributed-product-mode} +## 分布_产品_模式 {#distributed-product-mode} 改变的行为 [分布式子查询](../../sql-reference/operators/in.md). @@ -25,7 +25,7 @@ ClickHouse applies this setting when the query contains the product of distribut - `global` — Replaces the `IN`/`JOIN` 查询与 `GLOBAL IN`/`GLOBAL JOIN.` - `allow` — Allows the use of these types of subqueries. -## enable\_optimize\_predicate\_expression {#enable-optimize-predicate-expression} +## enable_optimize_predicate_expression {#enable-optimize-predicate-expression} 打开谓词下推 `SELECT` 查询。 @@ -49,7 +49,7 @@ ClickHouse applies this setting when the query contains the product of distribut 如果 `enable_optimize_predicate_expression = 0`,那么第二个查询的执行时间要长得多,因为 `WHERE` 子句适用于子查询完成后的所有数据。 -## fallback\_to\_stale\_replicas\_for\_distributed\_queries {#settings-fallback_to_stale_replicas_for_distributed_queries} +## fallback_to_stale_replicas_for_distributed_queries {#settings-fallback_to_stale_replicas_for_distributed_queries} 如果更新的数据不可用,则强制对过期副本进行查询。 看 [复制](../../engines/table-engines/mergetree-family/replication.md). @@ -59,7 +59,7 @@ ClickHouse从表的过时副本中选择最相关的副本。 默认情况下,1(已启用)。 -## force\_index\_by\_date {#settings-force_index_by_date} +## force_index_by_date {#settings-force_index_by_date} 如果索引不能按日期使用,则禁用查询执行。 @@ -67,7 +67,7 @@ ClickHouse从表的过时副本中选择最相关的副本。 如果 `force_index_by_date=1`,ClickHouse检查查询是否具有可用于限制数据范围的date键条件。 如果没有合适的条件,则会引发异常。 但是,它不检查条件是否减少了要读取的数据量。 例如,条件 `Date != ' 2000-01-01 '` 即使它与表中的所有数据匹配(即运行查询需要完全扫描),也是可以接受的。 有关MergeTree表中数据范围的详细信息,请参阅 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). -## force\_primary\_key {#force-primary-key} +## force_primary_key {#force-primary-key} 如果无法按主键编制索引,则禁用查询执行。 @@ -75,17 +75,17 @@ ClickHouse从表的过时副本中选择最相关的副本。 如果 `force_primary_key=1`,ClickHouse检查查询是否具有可用于限制数据范围的主键条件。 如果没有合适的条件,则会引发异常。 但是,它不检查条件是否减少了要读取的数据量。 有关MergeTree表中数据范围的详细信息,请参阅 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). -## format\_schema {#format-schema} +## format_schema {#format-schema} 当您使用需要架构定义的格式时,此参数非常有用,例如 [普罗托船长](https://capnproto.org/) 或 [Protobuf](https://developers.google.com/protocol-buffers/). 该值取决于格式。 -## fsync\_metadata {#fsync-metadata} +## fsync_metadata {#fsync-metadata} 启用或禁用 [fsync](http://pubs.opengroup.org/onlinepubs/9699919799/functions/fsync.html) 写作时 `.sql` 文件 默认情况下启用。 如果服务器有数百万个不断创建和销毁的小表,那么禁用它是有意义的。 -## enable\_http\_compression {#settings-enable_http_compression} +## enable_http_compression {#settings-enable_http_compression} 在对HTTP请求的响应中启用或禁用数据压缩。 @@ -98,15 +98,15 @@ ClickHouse从表的过时副本中选择最相关的副本。 默认值:0。 -## http\_zlib\_compression\_level {#settings-http_zlib_compression_level} +## http_zlib_compression_level {#settings-http_zlib_compression_level} -在以下情况下,设置对HTTP请求的响应中的数据压缩级别 [enable\_http\_compression=1](#settings-enable_http_compression). +在以下情况下,设置对HTTP请求的响应中的数据压缩级别 [enable_http_compression=1](#settings-enable_http_compression). 可能的值:数字从1到9。 默认值:3。 -## http\_native\_compression\_disable\_checksumming\_on\_decompress {#settings-http_native_compression_disable_checksumming_on_decompress} +## http_native_compression_disable_checksumming_on_decompress {#settings-http_native_compression_disable_checksumming_on_decompress} 在从客户端解压缩HTTP POST数据时启用或禁用校验和验证。 仅用于ClickHouse原生压缩格式(不用于 `gzip` 或 `deflate`). @@ -119,7 +119,7 @@ ClickHouse从表的过时副本中选择最相关的副本。 默认值:0。 -## send\_progress\_in\_http\_headers {#settings-send_progress_in_http_headers} +## send_progress_in_http_headers {#settings-send_progress_in_http_headers} 启用或禁用 `X-ClickHouse-Progress` Http响应头 `clickhouse-server` 答复。 @@ -132,7 +132,7 @@ ClickHouse从表的过时副本中选择最相关的副本。 默认值:0。 -## max\_http\_get\_redirects {#setting-max_http_get_redirects} +## max_http_get_redirects {#setting-max_http_get_redirects} 限制HTTP GET重定向跳数的最大数量 [URL](../../engines/table-engines/special/url.md)-发动机表。 该设置适用于两种类型的表:由 [CREATE TABLE](../../sql-reference/statements/create.md#create-table-query) 查询和由 [url](../../sql-reference/table-functions/url.md) 表功能。 @@ -143,7 +143,7 @@ ClickHouse从表的过时副本中选择最相关的副本。 默认值:0。 -## input\_format\_allow\_errors\_num {#settings-input_format_allow_errors_num} +## input_format_allow_errors_num {#settings-input_format_allow_errors_num} 设置从文本格式(CSV,TSV等)读取时可接受的错误的最大数量。). @@ -155,7 +155,7 @@ ClickHouse从表的过时副本中选择最相关的副本。 如果两者 `input_format_allow_errors_num` 和 `input_format_allow_errors_ratio` 超出时,ClickHouse引发异常。 -## input\_format\_allow\_errors\_ratio {#settings-input_format_allow_errors_ratio} +## input_format_allow_errors_ratio {#settings-input_format_allow_errors_ratio} 设置从文本格式(CSV,TSV等)读取时允许的最大错误百分比。). 错误百分比设置为介于0和1之间的浮点数。 @@ -168,7 +168,7 @@ ClickHouse从表的过时副本中选择最相关的副本。 如果两者 `input_format_allow_errors_num` 和 `input_format_allow_errors_ratio` 超出时,ClickHouse引发异常。 -## input\_format\_values\_interpret\_expressions {#settings-input_format_values_interpret_expressions} +## input_format_values_interpret_expressions {#settings-input_format_values_interpret_expressions} 如果快速流解析器无法解析数据,则启用或禁用完整SQL解析器。 此设置仅用于 [值](../../interfaces/formats.md#data-format-values) 格式在数据插入。 有关语法分析的详细信息,请参阅 [语法](../../sql-reference/syntax.md) 科。 @@ -218,7 +218,7 @@ INSERT INTO datetime_t SELECT now() Ok. ``` -## input\_format\_values\_deduce\_templates\_of\_expressions {#settings-input_format_values_deduce_templates_of_expressions} +## input_format_values_deduce_templates_of_expressions {#settings-input_format_values_deduce_templates_of_expressions} 启用或禁用以下内容中的SQL表达式的模板扣除 [值](../../interfaces/formats.md#data-format-values) 格式。 它允许解析和解释表达式 `Values` 如果连续行中的表达式具有相同的结构,速度要快得多。 ClickHouse尝试推导表达式的模板,使用此模板解析以下行,并在一批成功解析的行上评估表达式。 @@ -239,7 +239,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( - 如果 `input_format_values_interpret_expressions=0` 和 `format_values_deduce_templates_of_expressions=1`,第一,第二和第三行中的表达式使用template解析 `lower(String)` 并一起解释,第四行中的表达式用另一个模板解析 (`upper(String)`). - 如果 `input_format_values_interpret_expressions=1` 和 `format_values_deduce_templates_of_expressions=1`,与前面的情况相同,但如果不可能推导出模板,也允许回退到单独解释表达式。 -## input\_format\_values\_accurate\_types\_of\_literals {#settings-input-format-values-accurate-types-of-literals} +## input_format_values_accurate_types_of_literals {#settings-input-format-values-accurate-types-of-literals} 此设置仅在以下情况下使用 `input_format_values_deduce_templates_of_expressions = 1`. 它可能发生,某些列的表达式具有相同的结构,但包含不同类型的数字文字,例如 @@ -261,7 +261,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( 默认值:1。 -## input\_format\_defaults\_for\_omitted\_fields {#session_settings-input_format_defaults_for_omitted_fields} +## input_format_defaults_for_omitted_fields {#session_settings-input_format_defaults_for_omitted_fields} 执行时 `INSERT` 查询时,将省略的输入列值替换为相应列的默认值。 此选项仅适用于 [JSONEachRow](../../interfaces/formats.md#jsoneachrow), [CSV](../../interfaces/formats.md#csv) 和 [TabSeparated](../../interfaces/formats.md#tabseparated) 格式。 @@ -275,17 +275,17 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( 默认值:1。 -## input\_format\_tsv\_empty\_as\_default {#settings-input-format-tsv-empty-as-default} +## input_format_tsv_empty_as_default {#settings-input-format-tsv-empty-as-default} 启用后,将TSV中的空输入字段替换为默认值。 对于复杂的默认表达式 `input_format_defaults_for_omitted_fields` 必须启用了。 默认情况下禁用。 -## input\_format\_null\_as\_default {#settings-input-format-null-as-default} +## input_format_null_as_default {#settings-input-format-null-as-default} 如果输入数据包含 `NULL`,但相应列的数据类型不 `Nullable(T)` (对于文本输入格式)。 -## input\_format\_skip\_unknown\_fields {#settings-input-format-skip-unknown-fields} +## input_format_skip_unknown_fields {#settings-input-format-skip-unknown-fields} 启用或禁用跳过额外数据的插入。 @@ -305,7 +305,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( 默认值:0。 -## input\_format\_import\_nested\_json {#settings-input_format_import_nested_json} +## input_format_import_nested_json {#settings-input_format_import_nested_json} 启用或禁用具有嵌套对象的JSON数据的插入。 @@ -324,7 +324,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( - [嵌套结构的使用](../../interfaces/formats.md#jsoneachrow-nested) 与 `JSONEachRow` 格式。 -## input\_format\_with\_names\_use\_header {#settings-input-format-with-names-use-header} +## input_format_with_names_use_header {#settings-input-format-with-names-use-header} 启用或禁用插入数据时检查列顺序。 @@ -342,7 +342,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( 默认值:1。 -## date\_time\_input\_format {#settings-date_time_input_format} +## date_time_input_format {#settings-date_time_input_format} 允许选择日期和时间的文本表示的解析器。 @@ -365,7 +365,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( - [日期时间数据类型。](../../sql-reference/data-types/datetime.md) - [用于处理日期和时间的函数。](../../sql-reference/functions/date-time-functions.md) -## join\_default\_strictness {#settings-join_default_strictness} +## join_default_strictness {#settings-join_default_strictness} 设置默认严格性 [加入子句](../../sql-reference/statements/select/join.md#select-join). @@ -378,7 +378,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( 默认值: `ALL`. -## join\_any\_take\_last\_row {#settings-join_any_take_last_row} +## join_any_take_last_row {#settings-join_any_take_last_row} 更改联接操作的行为 `ANY` 严格。 @@ -396,9 +396,9 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( - [JOIN子句](../../sql-reference/statements/select/join.md#select-join) - [联接表引擎](../../engines/table-engines/special/join.md) -- [join\_default\_strictness](#settings-join_default_strictness) +- [join_default_strictness](#settings-join_default_strictness) -## join\_use\_nulls {#join_use_nulls} +## join_use_nulls {#join_use_nulls} 设置类型 [JOIN](../../sql-reference/statements/select/join.md) 行为 合并表时,可能会出现空单元格。 ClickHouse根据此设置以不同的方式填充它们。 @@ -409,7 +409,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( 默认值:0。 -## max\_block\_size {#setting-max_block_size} +## max_block_size {#setting-max_block_size} 在ClickHouse中,数据由块(列部分集)处理。 单个块的内部处理周期足够高效,但每个块都有明显的支出。 该 `max_block_size` 设置是建议从表中加载块的大小(行数)。 块大小不应该太小,以便每个块上的支出仍然明显,但不能太大,以便在第一个块处理完成后快速完成限制查询。 目标是避免在多个线程中提取大量列时占用太多内存,并且至少保留一些缓存局部性。 @@ -417,13 +417,13 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( 块的大小 `max_block_size` 并不总是从表中加载。 如果显然需要检索的数据较少,则处理较小的块。 -## preferred\_block\_size\_bytes {#preferred-block-size-bytes} +## preferred_block_size_bytes {#preferred-block-size-bytes} 用于相同的目的 `max_block_size`,但它通过使其适应块中的行数来设置推荐的块大小(以字节为单位)。 但是,块大小不能超过 `max_block_size` 行。 默认情况下:1,000,000。 它只有在从MergeTree引擎读取时才有效。 -## merge\_tree\_min\_rows\_for\_concurrent\_read {#setting-merge-tree-min-rows-for-concurrent-read} +## merge_tree_min_rows_for_concurrent_read {#setting-merge-tree-min-rows-for-concurrent-read} 如果从a的文件中读取的行数 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) 表超过 `merge_tree_min_rows_for_concurrent_read` 然后ClickHouse尝试在多个线程上从该文件执行并发读取。 @@ -433,7 +433,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( 默认值:163840. -## merge\_tree\_min\_bytes\_for\_concurrent\_read {#setting-merge-tree-min-bytes-for-concurrent-read} +## merge_tree_min_bytes_for_concurrent_read {#setting-merge-tree-min-bytes-for-concurrent-read} 如果从一个文件中读取的字节数 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md)-发动机表超过 `merge_tree_min_bytes_for_concurrent_read`,然后ClickHouse尝试在多个线程中并发读取此文件。 @@ -443,7 +443,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( 默认值:251658240. -## merge\_tree\_min\_rows\_for\_seek {#setting-merge-tree-min-rows-for-seek} +## merge_tree_min_rows_for_seek {#setting-merge-tree-min-rows-for-seek} 如果要在一个文件中读取的两个数据块之间的距离小于 `merge_tree_min_rows_for_seek` 行,然后ClickHouse不查找文件,而是按顺序读取数据。 @@ -453,7 +453,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( 默认值:0。 -## merge\_tree\_min\_bytes\_for\_seek {#setting-merge-tree-min-bytes-for-seek} +## merge_tree_min_bytes_for_seek {#setting-merge-tree-min-bytes-for-seek} 如果要在一个文件中读取的两个数据块之间的距离小于 `merge_tree_min_bytes_for_seek` 字节数,然后ClickHouse依次读取包含两个块的文件范围,从而避免了额外的寻道。 @@ -463,7 +463,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( 默认值:0。 -## merge\_tree\_coarse\_index\_granularity {#setting-merge-tree-coarse-index-granularity} +## merge_tree_coarse_index_granularity {#setting-merge-tree-coarse-index-granularity} 搜索数据时,ClickHouse会检查索引文件中的数据标记。 如果ClickHouse发现所需的键在某个范围内,它将此范围划分为 `merge_tree_coarse_index_granularity` 子范围和递归地搜索所需的键。 @@ -473,11 +473,11 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( 默认值:8。 -## merge\_tree\_max\_rows\_to\_use\_cache {#setting-merge-tree-max-rows-to-use-cache} +## merge_tree_max_rows_to_use_cache {#setting-merge-tree-max-rows-to-use-cache} 如果克里克豪斯应该阅读更多 `merge_tree_max_rows_to_use_cache` 在一个查询中的行,它不使用未压缩块的缓存。 -未压缩块的缓存存储为查询提取的数据。 ClickHouse使用此缓存来加快对重复的小查询的响应。 此设置可保护缓存免受读取大量数据的查询的破坏。 该 [uncompressed\_cache\_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) 服务器设置定义未压缩块的高速缓存的大小。 +未压缩块的缓存存储为查询提取的数据。 ClickHouse使用此缓存来加快对重复的小查询的响应。 此设置可保护缓存免受读取大量数据的查询的破坏。 该 [uncompressed_cache_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) 服务器设置定义未压缩块的高速缓存的大小。 可能的值: @@ -485,11 +485,11 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( Default value: 128 ✕ 8192. -## merge\_tree\_max\_bytes\_to\_use\_cache {#setting-merge-tree-max-bytes-to-use-cache} +## merge_tree_max_bytes_to_use_cache {#setting-merge-tree-max-bytes-to-use-cache} 如果克里克豪斯应该阅读更多 `merge_tree_max_bytes_to_use_cache` 在一个查询中的字节,它不使用未压缩块的缓存。 -未压缩块的缓存存储为查询提取的数据。 ClickHouse使用此缓存来加快对重复的小查询的响应。 此设置可保护缓存免受读取大量数据的查询的破坏。 该 [uncompressed\_cache\_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) 服务器设置定义未压缩块的高速缓存的大小。 +未压缩块的缓存存储为查询提取的数据。 ClickHouse使用此缓存来加快对重复的小查询的响应。 此设置可保护缓存免受读取大量数据的查询的破坏。 该 [uncompressed_cache_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) 服务器设置定义未压缩块的高速缓存的大小。 可能的值: @@ -497,7 +497,7 @@ Default value: 128 ✕ 8192. 默认值:2013265920. -## min\_bytes\_to\_use\_direct\_io {#settings-min-bytes-to-use-direct-io} +## min_bytes_to_use_direct_io {#settings-min-bytes-to-use-direct-io} 使用直接I/O访问存储磁盘所需的最小数据量。 @@ -510,11 +510,11 @@ ClickHouse在从表中读取数据时使用此设置。 如果要读取的所有 默认值:0。 -## log\_queries {#settings-log-queries} +## log_queries {#settings-log-queries} 设置查询日志记录。 -使用此设置发送到ClickHouse的查询将根据以下内容中的规则记录 [query\_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query-log) 服务器配置参数。 +使用此设置发送到ClickHouse的查询将根据以下内容中的规则记录 [query_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query-log) 服务器配置参数。 示例: @@ -522,7 +522,7 @@ ClickHouse在从表中读取数据时使用此设置。 如果要读取的所有 log_queries=1 ``` -## log\_queries\_min\_type {#settings-log-queries-min-type} +## log_queries_min_type {#settings-log-queries-min-type} `query_log` 要记录的最小类型。 @@ -540,11 +540,11 @@ log_queries=1 log_queries_min_type='EXCEPTION_WHILE_PROCESSING' ``` -## log\_query\_threads {#settings-log-query-threads} +## log_query_threads {#settings-log-query-threads} 设置查询线程日志记录。 -ClickHouse使用此设置运行的查询线程将根据以下命令中的规则记录 [query\_thread\_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) 服务器配置参数。 +ClickHouse使用此设置运行的查询线程将根据以下命令中的规则记录 [query_thread_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) 服务器配置参数。 示例: @@ -552,19 +552,19 @@ ClickHouse使用此设置运行的查询线程将根据以下命令中的规则 log_query_threads=1 ``` -## max\_insert\_block\_size {#settings-max_insert_block_size} +## max_insert_block_size {#settings-max_insert_block_size} 要插入到表中的块的大小。 此设置仅适用于服务器形成块的情况。 例如,对于通过HTTP接口进行的插入,服务器会分析数据格式并形成指定大小的块。 -但是当使用clickhouse-client时,客户端解析数据本身,并且 ‘max\_insert\_block\_size’ 服务器上的设置不会影响插入的块的大小。 +但是当使用clickhouse-client时,客户端解析数据本身,并且 ‘max_insert_block_size’ 服务器上的设置不会影响插入的块的大小。 使用INSERT SELECT时,该设置也没有目的,因为数据是使用在SELECT之后形成的相同块插入的。 默认值:1,048,576。 默认值略高于 `max_block_size`. 这样做的原因是因为某些表引擎 (`*MergeTree`)在磁盘上为每个插入的块形成一个数据部分,这是一个相当大的实体。 同样, `*MergeTree` 表在插入过程中对数据进行排序,并且足够大的块大小允许在RAM中对更多数据进行排序。 -## min\_insert\_block\_size\_rows {#min-insert-block-size-rows} +## min_insert_block_size_rows {#min-insert-block-size-rows} 设置块中可以通过以下方式插入到表中的最小行数 `INSERT` 查询。 较小尺寸的块被压扁成较大的块。 @@ -575,7 +575,7 @@ log_query_threads=1 默认值:1048576。 -## min\_insert\_block\_size\_bytes {#min-insert-block-size-bytes} +## min_insert_block_size_bytes {#min-insert-block-size-bytes} 设置块中的最小字节数,可以通过以下方式插入到表中 `INSERT` 查询。 较小尺寸的块被压扁成较大的块。 @@ -586,7 +586,7 @@ log_query_threads=1 默认值:268435456. -## max\_replica\_delay\_for\_distributed\_queries {#settings-max_replica_delay_for_distributed_queries} +## max_replica_delay_for_distributed_queries {#settings-max_replica_delay_for_distributed_queries} 禁用分布式查询的滞后副本。 看 [复制](../../engines/table-engines/mergetree-family/replication.md). @@ -596,22 +596,22 @@ log_query_threads=1 执行时使用 `SELECT` 从指向复制表的分布式表。 -## max\_threads {#settings-max_threads} +## max_threads {#settings-max_threads} -查询处理线程的最大数量,不包括用于从远程服务器检索数据的线程(请参阅 ‘max\_distributed\_connections’ 参数)。 +查询处理线程的最大数量,不包括用于从远程服务器检索数据的线程(请参阅 ‘max_distributed_connections’ 参数)。 此参数适用于并行执行查询处理管道的相同阶段的线程。 -例如,当从表中读取时,如果可以使用函数来评估表达式,请使用WHERE进行过滤,并且至少使用并行方式对GROUP BY进行预聚合 ‘max\_threads’ 线程数,然后 ‘max\_threads’ 被使用。 +例如,当从表中读取时,如果可以使用函数来评估表达式,请使用WHERE进行过滤,并且至少使用并行方式对GROUP BY进行预聚合 ‘max_threads’ 线程数,然后 ‘max_threads’ 被使用。 默认值:物理CPU内核数。 如果一次在服务器上运行的SELECT查询通常少于一个,请将此参数设置为略小于实际处理器内核数的值。 -对于由于限制而快速完成的查询,可以设置较低的 ‘max\_threads’. 例如,如果必要数量的条目位于每个块中,并且max\_threads=8,则会检索8个块,尽管只读取一个块就足够了。 +对于由于限制而快速完成的查询,可以设置较低的 ‘max_threads’. 例如,如果必要数量的条目位于每个块中,并且max_threads=8,则会检索8个块,尽管只读取一个块就足够了。 越小 `max_threads` 值,较少的内存被消耗。 -## max\_insert\_threads {#settings-max-insert-threads} +## max_insert_threads {#settings-max-insert-threads} 要执行的最大线程数 `INSERT SELECT` 查询。 @@ -622,61 +622,61 @@ log_query_threads=1 默认值:0。 -平行 `INSERT SELECT` 只有在 `SELECT` 部分并行执行,请参阅 [max\_threads](#settings-max_threads) 设置。 +平行 `INSERT SELECT` 只有在 `SELECT` 部分并行执行,请参阅 [max_threads](#settings-max_threads) 设置。 更高的值将导致更高的内存使用率。 -## max\_compress\_block\_size {#max-compress-block-size} +## max_compress_block_size {#max-compress-block-size} 在压缩写入表之前,未压缩数据块的最大大小。 默认情况下,1,048,576(1MiB)。 如果大小减小,则压缩率显着降低,压缩和解压缩速度由于高速缓存局部性而略微增加,并且内存消耗减少。 通常没有任何理由更改此设置。 不要将用于压缩的块(由字节组成的内存块)与用于查询处理的块(表中的一组行)混淆。 -## min\_compress\_block\_size {#min-compress-block-size} +## min_compress_block_size {#min-compress-block-size} -为 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md)"表。 为了减少处理查询时的延迟,在写入下一个标记时,如果块的大小至少为 ‘min\_compress\_block\_size’. 默认情况下,65,536。 +为 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md)"表。 为了减少处理查询时的延迟,在写入下一个标记时,如果块的大小至少为 ‘min_compress_block_size’. 默认情况下,65,536。 -块的实际大小,如果未压缩的数据小于 ‘max\_compress\_block\_size’,是不小于该值且不小于一个标记的数据量。 +块的实际大小,如果未压缩的数据小于 ‘max_compress_block_size’,是不小于该值且不小于一个标记的数据量。 -让我们来看看一个例子。 假设 ‘index\_granularity’ 在表创建期间设置为8192。 +让我们来看看一个例子。 假设 ‘index_granularity’ 在表创建期间设置为8192。 -我们正在编写一个UInt32类型的列(每个值4个字节)。 当写入8192行时,总数将是32KB的数据。 由于min\_compress\_block\_size=65,536,将为每两个标记形成一个压缩块。 +我们正在编写一个UInt32类型的列(每个值4个字节)。 当写入8192行时,总数将是32KB的数据。 由于min_compress_block_size=65,536,将为每两个标记形成一个压缩块。 我们正在编写一个字符串类型的URL列(每个值的平均大小60字节)。 当写入8192行时,平均数据将略少于500KB。 由于这超过65,536,将为每个标记形成一个压缩块。 在这种情况下,当从单个标记范围内的磁盘读取数据时,额外的数据不会被解压缩。 通常没有任何理由更改此设置。 -## max\_query\_size {#settings-max_query_size} +## max_query_size {#settings-max_query_size} 查询的最大部分,可以被带到RAM用于使用SQL解析器进行解析。 插入查询还包含由单独的流解析器(消耗O(1)RAM)处理的插入数据,这些数据不包含在此限制中。 默认值:256KiB。 -## interactive\_delay {#interactive-delay} +## interactive_delay {#interactive-delay} 以微秒为单位的间隔,用于检查请求执行是否已被取消并发送进度。 默认值:100,000(检查取消并每秒发送十次进度)。 -## connect\_timeout,receive\_timeout,send\_timeout {#connect-timeout-receive-timeout-send-timeout} +## connect_timeout,receive_timeout,send_timeout {#connect-timeout-receive-timeout-send-timeout} 用于与客户端通信的套接字上的超时以秒为单位。 默认值:10,300,300。 -## cancel\_http\_readonly\_queries\_on\_client\_close {#cancel-http-readonly-queries-on-client-close} +## cancel_http_readonly_queries_on_client_close {#cancel-http-readonly-queries-on-client-close} Cancels HTTP read-only queries (e.g. SELECT) when a client closes the connection without waiting for the response. 默认值:0 -## poll\_interval {#poll-interval} +## poll_interval {#poll-interval} 锁定在指定秒数的等待循环。 默认值:10。 -## max\_distributed\_connections {#max-distributed-connections} +## max_distributed_connections {#max-distributed-connections} 与远程服务器同时连接的最大数量,用于分布式处理对单个分布式表的单个查询。 我们建议设置不小于群集中服务器数量的值。 @@ -684,20 +684,20 @@ Cancels HTTP read-only queries (e.g. SELECT) when a client closes the connectio 以下参数仅在创建分布式表(以及启动服务器时)时使用,因此没有理由在运行时更改它们。 -## distributed\_connections\_pool\_size {#distributed-connections-pool-size} +## distributed_connections_pool_size {#distributed-connections-pool-size} 与远程服务器同时连接的最大数量,用于分布式处理对单个分布式表的所有查询。 我们建议设置不小于群集中服务器数量的值。 默认值:1024。 -## connect\_timeout\_with\_failover\_ms {#connect-timeout-with-failover-ms} +## connect_timeout_with_failover_ms {#connect-timeout-with-failover-ms} 以毫秒为单位连接到分布式表引擎的远程服务器的超时,如果 ‘shard’ 和 ‘replica’ 部分用于群集定义。 如果不成功,将尝试多次连接到各种副本。 默认值:50。 -## connections\_with\_failover\_max\_tries {#connections-with-failover-max-tries} +## connections_with_failover_max_tries {#connections-with-failover-max-tries} 分布式表引擎的每个副本的最大连接尝试次数。 @@ -708,33 +708,33 @@ Cancels HTTP read-only queries (e.g. SELECT) when a client closes the connectio 是否计算极值(查询结果列中的最小值和最大值)。 接受0或1。 默认情况下,0(禁用)。 有关详细信息,请参阅部分 “Extreme values”. -## use\_uncompressed\_cache {#setting-use_uncompressed_cache} +## use_uncompressed_cache {#setting-use_uncompressed_cache} 是否使用未压缩块的缓存。 接受0或1。 默认情况下,0(禁用)。 -使用未压缩缓存(仅适用于MergeTree系列中的表)可以在处理大量短查询时显着减少延迟并提高吞吐量。 为频繁发送短请求的用户启用此设置。 还要注意 [uncompressed\_cache\_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuration parameter (only set in the config file) – the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed and the least-used data is automatically deleted. +使用未压缩缓存(仅适用于MergeTree系列中的表)可以在处理大量短查询时显着减少延迟并提高吞吐量。 为频繁发送短请求的用户启用此设置。 还要注意 [uncompressed_cache_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuration parameter (only set in the config file) – the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed and the least-used data is automatically deleted. -对于至少读取大量数据(一百万行或更多行)的查询,将自动禁用未压缩缓存,以节省真正小型查询的空间。 这意味着你可以保持 ‘use\_uncompressed\_cache’ 设置始终设置为1。 +对于至少读取大量数据(一百万行或更多行)的查询,将自动禁用未压缩缓存,以节省真正小型查询的空间。 这意味着你可以保持 ‘use_uncompressed_cache’ 设置始终设置为1。 -## replace\_running\_query {#replace-running-query} +## replace_running_query {#replace-running-query} -当使用HTTP接口时, ‘query\_id’ 参数可以传递。 这是用作查询标识符的任何字符串。 -如果来自同一用户的查询具有相同的 ‘query\_id’ 已经存在在这个时候,行为取决于 ‘replace\_running\_query’ 参数。 +当使用HTTP接口时, ‘query_id’ 参数可以传递。 这是用作查询标识符的任何字符串。 +如果来自同一用户的查询具有相同的 ‘query_id’ 已经存在在这个时候,行为取决于 ‘replace_running_query’ 参数。 -`0` (default) – Throw an exception (don't allow the query to run if a query with the same ‘query\_id’ 已经运行)。 +`0` (default) – Throw an exception (don't allow the query to run if a query with the same ‘query_id’ 已经运行)。 `1` – Cancel the old query and start running the new one. YandexMetrica使用此参数设置为1来实现分段条件的建议。 输入下一个字符后,如果旧的查询还没有完成,应该取消。 -## stream\_flush\_interval\_ms {#stream-flush-interval-ms} +## stream_flush_interval_ms {#stream-flush-interval-ms} -适用于在超时的情况下或线程生成流式传输的表 [max\_insert\_block\_size](#settings-max_insert_block_size) 行。 +适用于在超时的情况下或线程生成流式传输的表 [max_insert_block_size](#settings-max_insert_block_size) 行。 默认值为7500。 值越小,数据被刷新到表中的频率就越高。 将该值设置得太低会导致性能较差。 -## load\_balancing {#settings-load_balancing} +## load_balancing {#settings-load_balancing} 指定用于分布式查询处理的副本选择算法。 @@ -787,31 +787,31 @@ load_balancing = first_or_random 该 `first_or_random` 算法解决的问题 `in_order` 算法。 与 `in_order`,如果一个副本出现故障,下一个副本将获得双重负载,而其余副本将处理通常的流量。 使用时 `first_or_random` 算法中,负载均匀分布在仍然可用的副本之间。 -## prefer\_localhost\_replica {#settings-prefer-localhost-replica} +## prefer_localhost_replica {#settings-prefer-localhost-replica} 在处理分布式查询时,最好使用localhost副本启用/禁用该副本。 可能的值: - 1 — ClickHouse always sends a query to the localhost replica if it exists. -- 0 — ClickHouse uses the balancing strategy specified by the [load\_balancing](#settings-load_balancing) 设置。 +- 0 — ClickHouse uses the balancing strategy specified by the [load_balancing](#settings-load_balancing) 设置。 默认值:1。 !!! warning "警告" - 如果使用此设置,请禁用此设置 [max\_parallel\_replicas](#settings-max_parallel_replicas). + 如果使用此设置,请禁用此设置 [max_parallel_replicas](#settings-max_parallel_replicas). -## totals\_mode {#totals-mode} +## totals_mode {#totals-mode} -如何计算总计时有存在,以及当max\_rows\_to\_group\_by和group\_by\_overflow\_mode= ‘any’ 都在场。 +如何计算总计时有存在,以及当max_rows_to_group_by和group_by_overflow_mode= ‘any’ 都在场。 请参阅部分 “WITH TOTALS modifier”. -## totals\_auto\_threshold {#totals-auto-threshold} +## totals_auto_threshold {#totals-auto-threshold} 阈值 `totals_mode = 'auto'`. 请参阅部分 “WITH TOTALS modifier”. -## max\_parallel\_replicas {#settings-max_parallel_replicas} +## max_parallel_replicas {#settings-max_parallel_replicas} 执行查询时每个分片的最大副本数。 为了保持一致性(以获取相同数据拆分的不同部分),此选项仅在设置了采样键时有效。 @@ -824,7 +824,7 @@ load_balancing = first_or_random 编译仅用于查询处理管道的一部分:用于聚合的第一阶段(GROUP BY)。 如果编译了管道的这一部分,则由于部署周期较短和内联聚合函数调用,查询可能运行得更快。 对于具有多个简单聚合函数的查询,可以看到最大的性能改进(在极少数情况下可快四倍)。 通常,性能增益是微不足道的。 在极少数情况下,它可能会减慢查询执行速度。 -## min\_count\_to\_compile {#min-count-to-compile} +## min_count_to_compile {#min-count-to-compile} 在运行编译之前可能使用已编译代码块的次数。 默认情况下,3。 For testing, the value can be set to 0: compilation runs synchronously and the query waits for the end of the compilation process before continuing execution. For all other cases, use values ​​starting with 1. Compilation normally takes about 5-10 seconds. @@ -833,27 +833,27 @@ For testing, the value can be set to 0: compilation runs synchronously and the q 对于查询中使用的聚合函数的每个不同组合以及GROUP BY子句中的键类型,都需要编译代码。 The results of the compilation are saved in the build directory in the form of .so files. There is no restriction on the number of compilation results since they don't use very much space. Old results will be used after server restarts, except in the case of a server upgrade – in this case, the old results are deleted. -## output\_format\_json\_quote\_64bit\_integers {#session_settings-output_format_json_quote_64bit_integers} +## output_format_json_quote_64bit_integers {#session_settings-output_format_json_quote_64bit_integers} 如果该值为true,则在使用JSON\*Int64和UInt64格式时,整数将显示在引号中(为了与大多数JavaScript实现兼容);否则,整数将不带引号输出。 -## format\_csv\_delimiter {#settings-format_csv_delimiter} +## format_csv_delimiter {#settings-format_csv_delimiter} 将字符解释为CSV数据中的分隔符。 默认情况下,分隔符为 `,`. -## input\_format\_csv\_unquoted\_null\_literal\_as\_null {#settings-input_format_csv_unquoted_null_literal_as_null} +## input_format_csv_unquoted_null_literal_as_null {#settings-input_format_csv_unquoted_null_literal_as_null} 对于CSV输入格式,启用或禁用未引用的解析 `NULL` 作为文字(同义词 `\N`). -## output\_format\_csv\_crlf\_end\_of\_line {#settings-output-format-csv-crlf-end-of-line} +## output_format_csv_crlf_end_of_line {#settings-output-format-csv-crlf-end-of-line} 在CSV中使用DOS/Windows样式的行分隔符(CRLF)而不是Unix样式(LF)。 -## output\_format\_tsv\_crlf\_end\_of\_line {#settings-output-format-tsv-crlf-end-of-line} +## output_format_tsv_crlf_end_of_line {#settings-output-format-tsv-crlf-end-of-line} 在TSV中使用DOC/Windows样式的行分隔符(CRLF)而不是Unix样式(LF)。 -## insert\_quorum {#settings-insert_quorum} +## insert_quorum {#settings-insert_quorum} 启用仲裁写入。 @@ -868,7 +868,7 @@ The results of the compilation are saved in the build directory in the form of . 仲裁中的所有副本都是一致的,即它们包含来自所有以前的数据 `INSERT` 查询。 该 `INSERT` 序列线性化。 -当读取从写入的数据 `insert_quorum`,您可以使用 [select\_sequential\_consistency](#settings-select_sequential_consistency) 选项。 +当读取从写入的数据 `insert_quorum`,您可以使用 [select_sequential_consistency](#settings-select_sequential_consistency) 选项。 ClickHouse生成异常 @@ -877,10 +877,10 @@ ClickHouse生成异常 另请参阅: -- [insert\_quorum\_timeout](#settings-insert_quorum_timeout) -- [select\_sequential\_consistency](#settings-select_sequential_consistency) +- [insert_quorum_timeout](#settings-insert_quorum_timeout) +- [select_sequential_consistency](#settings-select_sequential_consistency) -## insert\_quorum\_timeout {#settings-insert_quorum_timeout} +## insert_quorum_timeout {#settings-insert_quorum_timeout} 写入仲裁超时以秒为单位。 如果超时已经过去,并且还没有发生写入,ClickHouse将生成异常,客户端必须重复查询以将相同的块写入相同的副本或任何其他副本。 @@ -888,10 +888,10 @@ ClickHouse生成异常 另请参阅: -- [insert\_quorum](#settings-insert_quorum) -- [select\_sequential\_consistency](#settings-select_sequential_consistency) +- [insert_quorum](#settings-insert_quorum) +- [select_sequential_consistency](#settings-select_sequential_consistency) -## select\_sequential\_consistency {#settings-select_sequential_consistency} +## select_sequential_consistency {#settings-select_sequential_consistency} 启用或禁用顺序一致性 `SELECT` 查询: @@ -908,10 +908,10 @@ ClickHouse生成异常 另请参阅: -- [insert\_quorum](#settings-insert_quorum) -- [insert\_quorum\_timeout](#settings-insert_quorum_timeout) +- [insert_quorum](#settings-insert_quorum) +- [insert_quorum_timeout](#settings-insert_quorum_timeout) -## insert\_deduplicate {#settings-insert-deduplicate} +## insert_deduplicate {#settings-insert-deduplicate} 启用或禁用块重复数据删除 `INSERT` (对于复制的\*表)。 @@ -924,7 +924,7 @@ ClickHouse生成异常 默认情况下,块插入到复制的表 `INSERT` 语句重复数据删除(见 [数据复制](../../engines/table-engines/mergetree-family/replication.md)). -## deduplicate\_blocks\_in\_dependent\_materialized\_views {#settings-deduplicate-blocks-in-dependent-materialized-views} +## deduplicate_blocks_in_dependent_materialized_views {#settings-deduplicate-blocks-in-dependent-materialized-views} 启用或禁用从已复制\*表接收数据的实例化视图的重复数据删除检查。 @@ -942,7 +942,7 @@ ClickHouse生成异常 与此同时,这种行为 “breaks” `INSERT` 幂等性 如果一个 `INSERT` 进入主表是成功的, `INSERT` into a materialized view failed (e.g. because of communication failure with Zookeeper) a client will get an error and can retry the operation. However, the materialized view won't receive the second insert because it will be discarded by deduplication in the main (source) table. The setting `deduplicate_blocks_in_dependent_materialized_views` 允许改变这种行为。 重试时,实例化视图将收到重复插入,并自行执行重复数据删除检查, 忽略源表的检查结果,并将插入由于第一次失败而丢失的行。 -## max\_network\_bytes {#settings-max-network-bytes} +## max_network_bytes {#settings-max-network-bytes} 限制在执行查询时通过网络接收或传输的数据量(以字节为单位)。 此设置适用于每个单独的查询。 @@ -953,7 +953,7 @@ ClickHouse生成异常 默认值:0。 -## max\_network\_bandwidth {#settings-max-network-bandwidth} +## max_network_bandwidth {#settings-max-network-bandwidth} 限制通过网络进行数据交换的速度,以每秒字节为单位。 此设置适用于每个查询。 @@ -964,7 +964,7 @@ ClickHouse生成异常 默认值:0。 -## max\_network\_bandwidth\_for\_user {#settings-max-network-bandwidth-for-user} +## max_network_bandwidth_for_user {#settings-max-network-bandwidth-for-user} 限制通过网络进行数据交换的速度,以每秒字节为单位。 此设置适用于单个用户执行的所有并发运行的查询。 @@ -975,7 +975,7 @@ ClickHouse生成异常 默认值:0。 -## max\_network\_bandwidth\_for\_all\_users {#settings-max-network-bandwidth-for-all-users} +## max_network_bandwidth_for_all_users {#settings-max-network-bandwidth-for-all-users} 限制通过网络交换数据的速度,以每秒字节为单位。 此设置适用于服务器上同时运行的所有查询。 @@ -986,7 +986,7 @@ ClickHouse生成异常 默认值:0。 -## count\_distinct\_implementation {#settings-count_distinct_implementation} +## count_distinct_implementation {#settings-count_distinct_implementation} 指定其中的 `uniq*` 函数应用于执行 [COUNT(DISTINCT …)](../../sql-reference/aggregate-functions/reference.md#agg_function-count) 建筑。 @@ -1000,7 +1000,7 @@ ClickHouse生成异常 默认值: `uniqExact`. -## skip\_unavailable\_shards {#settings-skip_unavailable_shards} +## skip_unavailable_shards {#settings-skip_unavailable_shards} 启用或禁用静默跳过不可用分片。 @@ -1030,13 +1030,13 @@ ClickHouse生成异常 默认值:0。 -## optimize\_skip\_unused\_shards {#settings-optimize_skip_unused_shards} +## optimize_skip_unused_shards {#settings-optimize_skip_unused_shards} 对于在PREWHERE/WHERE中具有分片键条件的SELECT查询,启用或禁用跳过未使用的分片(假定数据是通过分片键分发的,否则不执行任何操作)。 默认值:0 -## force\_optimize\_skip\_unused\_shards {#settings-force_optimize_skip_unused_shards} +## force_optimize_skip_unused_shards {#settings-force_optimize_skip_unused_shards} 在以下情况下启用或禁用查询执行 [`optimize_skip_unused_shards`](#settings-optimize_skip_unused_shards) 无法启用和跳过未使用的分片。 如果跳过是不可能的,并且设置为启用异常将被抛出。 @@ -1048,7 +1048,7 @@ ClickHouse生成异常 默认值:0 -## optimize\_throw\_if\_noop {#setting-optimize_throw_if_noop} +## optimize_throw_if_noop {#setting-optimize_throw_if_noop} 启用或禁用抛出异常,如果 [OPTIMIZE](../../sql-reference/statements/misc.md#misc_operations-optimize) 查询未执行合并。 @@ -1061,19 +1061,19 @@ ClickHouse生成异常 默认值:0。 -## distributed\_replica\_error\_half\_life {#settings-distributed_replica_error_half_life} +## distributed_replica_error_half_life {#settings-distributed_replica_error_half_life} - 类型:秒 - 默认值:60秒 -控制清零分布式表中的错误的速度。 如果某个副本在一段时间内不可用,累计出现5个错误,并且distributed\_replica\_error\_half\_life设置为1秒,则该副本在上一个错误发生3秒后视为正常。 +控制清零分布式表中的错误的速度。 如果某个副本在一段时间内不可用,累计出现5个错误,并且distributed_replica_error_half_life设置为1秒,则该副本在上一个错误发生3秒后视为正常。 另请参阅: - [表引擎分布式](../../engines/table-engines/special/distributed.md) -- [distributed\_replica\_error\_cap](#settings-distributed_replica_error_cap) +- [distributed_replica_error_cap](#settings-distributed_replica_error_cap) -## distributed\_replica\_error\_cap {#settings-distributed_replica_error_cap} +## distributed_replica_error_cap {#settings-distributed_replica_error_cap} - 类型:无符号int - 默认值:1000 @@ -1083,9 +1083,9 @@ ClickHouse生成异常 另请参阅: - [表引擎分布式](../../engines/table-engines/special/distributed.md) -- [distributed\_replica\_error\_half\_life](#settings-distributed_replica_error_half_life) +- [distributed_replica_error_half_life](#settings-distributed_replica_error_half_life) -## distributed\_directory\_monitor\_sleep\_time\_ms {#distributed_directory_monitor_sleep_time_ms} +## distributed_directory_monitor_sleep_time_ms {#distributed_directory_monitor_sleep_time_ms} 对于基本间隔 [分布](../../engines/table-engines/special/distributed.md) 表引擎发送数据。 在发生错误时,实际间隔呈指数级增长。 @@ -1095,9 +1095,9 @@ ClickHouse生成异常 默认值:100毫秒。 -## distributed\_directory\_monitor\_max\_sleep\_time\_ms {#distributed_directory_monitor_max_sleep_time_ms} +## distributed_directory_monitor_max_sleep_time_ms {#distributed_directory_monitor_max_sleep_time_ms} -的最大间隔 [分布](../../engines/table-engines/special/distributed.md) 表引擎发送数据。 限制在设置的区间的指数增长 [distributed\_directory\_monitor\_sleep\_time\_ms](#distributed_directory_monitor_sleep_time_ms) 设置。 +的最大间隔 [分布](../../engines/table-engines/special/distributed.md) 表引擎发送数据。 限制在设置的区间的指数增长 [distributed_directory_monitor_sleep_time_ms](#distributed_directory_monitor_sleep_time_ms) 设置。 可能的值: @@ -1105,7 +1105,7 @@ ClickHouse生成异常 默认值:30000毫秒(30秒)。 -## distributed\_directory\_monitor\_batch\_inserts {#distributed_directory_monitor_batch_inserts} +## distributed_directory_monitor_batch_inserts {#distributed_directory_monitor_batch_inserts} 启用/禁用批量发送插入的数据。 @@ -1118,7 +1118,7 @@ ClickHouse生成异常 默认值:0。 -## os\_thread\_priority {#setting-os-thread-priority} +## os_thread_priority {#setting-os-thread-priority} 设置优先级 ([不错](https://en.wikipedia.org/wiki/Nice_(Unix)))对于执行查询的线程。 当选择要在每个可用CPU内核上运行的下一个线程时,操作系统调度程序会考虑此优先级。 @@ -1133,7 +1133,7 @@ ClickHouse生成异常 默认值:0。 -## query\_profiler\_real\_time\_period\_ns {#query_profiler_real_time_period_ns} +## query_profiler_real_time_period_ns {#query_profiler_real_time_period_ns} 设置周期的实时时钟定时器 [查询探查器](../../operations/optimizing-performance/sampling-query-profiler.md). 真正的时钟计时器计数挂钟时间。 @@ -1154,9 +1154,9 @@ ClickHouse生成异常 另请参阅: -- 系统表 [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) +- 系统表 [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) -## query\_profiler\_cpu\_time\_period\_ns {#query_profiler_cpu_time_period_ns} +## query_profiler_cpu_time_period_ns {#query_profiler_cpu_time_period_ns} 设置周期的CPU时钟定时器 [查询探查器](../../operations/optimizing-performance/sampling-query-profiler.md). 此计时器仅计算CPU时间。 @@ -1177,9 +1177,9 @@ ClickHouse生成异常 另请参阅: -- 系统表 [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) +- 系统表 [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) -## allow\_introspection\_functions {#settings-allow_introspection_functions} +## allow_introspection_functions {#settings-allow_introspection_functions} 启用禁用 [反省函数](../../sql-reference/functions/introspection.md) 用于查询分析。 @@ -1193,23 +1193,23 @@ ClickHouse生成异常 **另请参阅** - [采样查询探查器](../optimizing-performance/sampling-query-profiler.md) -- 系统表 [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) +- 系统表 [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) -## input\_format\_parallel\_parsing {#input-format-parallel-parsing} +## input_format_parallel_parsing {#input-format-parallel-parsing} - 类型:布尔 - 默认值:True 启用数据格式的保序并行分析。 仅支持TSV,TKSV,CSV和JSONEachRow格式。 -## min\_chunk\_bytes\_for\_parallel\_parsing {#min-chunk-bytes-for-parallel-parsing} +## min_chunk_bytes_for_parallel_parsing {#min-chunk-bytes-for-parallel-parsing} - 类型:无符号int - 默认值:1MiB 以字节为单位的最小块大小,每个线程将并行解析。 -## output\_format\_avro\_codec {#settings-output_format_avro_codec} +## output_format_avro_codec {#settings-output_format_avro_codec} 设置用于输出Avro文件的压缩编解ec。 @@ -1223,7 +1223,7 @@ ClickHouse生成异常 默认值: `snappy` (如果可用)或 `deflate`. -## output\_format\_avro\_sync\_interval {#settings-output_format_avro_sync_interval} +## output_format_avro_sync_interval {#settings-output_format_avro_sync_interval} 设置输出Avro文件的同步标记之间的最小数据大小(以字节为单位)。 @@ -1233,7 +1233,7 @@ ClickHouse生成异常 默认值:32768(32KiB) -## format\_avro\_schema\_registry\_url {#settings-format_avro_schema_registry_url} +## format_avro_schema_registry_url {#settings-format_avro_schema_registry_url} 设置要与之一起使用的汇合架构注册表URL [AvroConfluent](../../interfaces/formats.md#data-format-avro-confluent) 格式 @@ -1241,7 +1241,7 @@ ClickHouse生成异常 默认值:空 -## background\_pool\_size {#background_pool_size} +## background_pool_size {#background_pool_size} 设置在表引擎中执行后台操作的线程数(例如,合并 [MergeTree引擎](../../engines/table-engines/mergetree-family/index.md) 表)。 此设置在ClickHouse服务器启动时应用,不能在用户会话中更改。 通过调整此设置,您可以管理CPU和磁盘负载。 较小的池大小使用较少的CPU和磁盘资源,但后台进程推进速度较慢,最终可能会影响查询性能。 @@ -1253,7 +1253,7 @@ ClickHouse生成异常 [原始文章](https://clickhouse.tech/docs/en/operations/settings/settings/) -## transform\_null\_in {#transform_null_in} +## transform_null_in {#transform_null_in} 为[IN](../../sql-reference/operators/in.md) 运算符启用[NULL](../../sql-reference/syntax.md#null-literal) 值的相等性。 diff --git a/docs/zh/operations/system-tables/asynchronous_metric_log.md b/docs/zh/operations/system-tables/asynchronous_metric_log.md index 9f6c697a18e..9fbe15b8507 100644 --- a/docs/zh/operations/system-tables/asynchronous_metric_log.md +++ b/docs/zh/operations/system-tables/asynchronous_metric_log.md @@ -3,6 +3,6 @@ machine_translated: true machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 --- -## 系统。asynchronous\_metric\_log {#system-tables-async-log} +## 系统。asynchronous_metric_log {#system-tables-async-log} -包含以下内容的历史值 `system.asynchronous_log` (见 [系统。asynchronous\_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics)) +包含以下内容的历史值 `system.asynchronous_log` (见 [系统。asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics)) diff --git a/docs/zh/operations/system-tables/asynchronous_metrics.md b/docs/zh/operations/system-tables/asynchronous_metrics.md index 2bd615085a8..805477c9f47 100644 --- a/docs/zh/operations/system-tables/asynchronous_metrics.md +++ b/docs/zh/operations/system-tables/asynchronous_metrics.md @@ -3,7 +3,7 @@ machine_translated: true machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 --- -# 系统。asynchronous\_metrics {#system_tables-asynchronous_metrics} +# 系统。asynchronous_metrics {#system_tables-asynchronous_metrics} 包含在后台定期计算的指标。 例如,在使用的RAM量。 @@ -38,4 +38,4 @@ SELECT * FROM system.asynchronous_metrics LIMIT 10 - [监测](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring. - [系统。指标](../../operations/system-tables/metrics.md#system_tables-metrics) — Contains instantly calculated metrics. - [系统。活动](../../operations/system-tables/events.md#system_tables-events) — Contains a number of events that have occurred. -- [系统。metric\_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [系统。metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. diff --git a/docs/zh/operations/system-tables/clusters.md b/docs/zh/operations/system-tables/clusters.md index 4bc8d4210ff..1e5935c276e 100644 --- a/docs/zh/operations/system-tables/clusters.md +++ b/docs/zh/operations/system-tables/clusters.md @@ -25,5 +25,5 @@ machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 **另请参阅** - [表引擎分布式](../../engines/table-engines/special/distributed.md) -- [distributed\_replica\_error\_cap设置](../../operations/settings/settings.md#settings-distributed_replica_error_cap) -- [distributed\_replica\_error\_half\_life设置](../../operations/settings/settings.md#settings-distributed_replica_error_half_life) +- [distributed_replica_error_cap设置](../../operations/settings/settings.md#settings-distributed_replica_error_cap) +- [distributed_replica_error_half_life设置](../../operations/settings/settings.md#settings-distributed_replica_error_half_life) diff --git a/docs/zh/operations/system-tables/data_type_families.md b/docs/zh/operations/system-tables/data_type_families.md index e6ec3fdbfce..21eb4785e23 100644 --- a/docs/zh/operations/system-tables/data_type_families.md +++ b/docs/zh/operations/system-tables/data_type_families.md @@ -3,7 +3,7 @@ machine_translated: true machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 --- -# 系统。data\_type\_families {#system_tables-data_type_families} +# 系统。data_type_families {#system_tables-data_type_families} 包含有关受支持的信息 [数据类型](../../sql-reference/data-types/). diff --git a/docs/zh/operations/system-tables/detached_parts.md b/docs/zh/operations/system-tables/detached_parts.md index dd561dec6f3..ba35444c551 100644 --- a/docs/zh/operations/system-tables/detached_parts.md +++ b/docs/zh/operations/system-tables/detached_parts.md @@ -3,7 +3,7 @@ machine_translated: true machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 --- -# 系统。detached\_parts {#system_tables-detached_parts} +# 系统。detached_parts {#system_tables-detached_parts} 包含有关分离部分的信息 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) 桌子 该 `reason` 列指定分离部件的原因。 diff --git a/docs/zh/operations/system-tables/disks.md b/docs/zh/operations/system-tables/disks.md index 39cacccb4db..8cd24d24550 100644 --- a/docs/zh/operations/system-tables/disks.md +++ b/docs/zh/operations/system-tables/disks.md @@ -15,7 +15,7 @@ machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 - `total_space` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Disk volume in bytes. - `keep_free_space` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Amount of disk space that should stay free on disk in bytes. Defined in the `keep_free_space_bytes` 磁盘配置参数。 -## 系统。storage\_policies {#system_tables-storage_policies} +## 系统。storage_policies {#system_tables-storage_policies} 包含有关存储策略和卷中定义的信息 [服务器配置](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes_configure). diff --git a/docs/zh/operations/system-tables/events.md b/docs/zh/operations/system-tables/events.md index 21b787c6064..e7ce47116bf 100644 --- a/docs/zh/operations/system-tables/events.md +++ b/docs/zh/operations/system-tables/events.md @@ -31,7 +31,7 @@ SELECT * FROM system.events LIMIT 5 **另请参阅** -- [系统。asynchronous\_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [系统。asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. - [系统。指标](../../operations/system-tables/metrics.md#system_tables-metrics) — Contains instantly calculated metrics. -- [系统。metric\_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [系统。metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. - [监测](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring. diff --git a/docs/zh/operations/system-tables/graphite_retentions.md b/docs/zh/operations/system-tables/graphite_retentions.md index b2579541920..e61f58496d0 100644 --- a/docs/zh/operations/system-tables/graphite_retentions.md +++ b/docs/zh/operations/system-tables/graphite_retentions.md @@ -3,9 +3,9 @@ machine_translated: true machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 --- -# 系统。graphite\_retentions {#system-graphite-retentions} +# 系统。graphite_retentions {#system-graphite-retentions} -包含有关参数的信息 [graphite\_rollup](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) 这是在表中使用 [\*GraphiteMergeTree](../../engines/table-engines/mergetree-family/graphitemergetree.md) 引擎 +包含有关参数的信息 [graphite_rollup](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) 这是在表中使用 [\*GraphiteMergeTree](../../engines/table-engines/mergetree-family/graphitemergetree.md) 引擎 列: diff --git a/docs/zh/operations/system-tables/index.md b/docs/zh/operations/system-tables/index.md index 73a57300de8..fcf6741761b 100644 --- a/docs/zh/operations/system-tables/index.md +++ b/docs/zh/operations/system-tables/index.md @@ -22,7 +22,7 @@ toc_title: "\u7CFB\u7EDF\u8868" 大多数系统表将数据存储在RAM中。 ClickHouse服务器在开始时创建此类系统表。 -与其他系统表不同,系统表 [metric\_log](../../operations/system-tables/metric_log.md#system_tables-metric_log), [query\_log](../../operations/system-tables/query_log.md#system_tables-query_log), [query\_thread\_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log), [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) 由 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) 表引擎并将其数据存储在存储文件系统中。 如果从文件系统中删除表,ClickHouse服务器会在下一次写入数据时再次创建空表。 如果系统表架构在新版本中发生更改,则ClickHouse会重命名当前表并创建一个新表。 +与其他系统表不同,系统表 [metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log), [query_log](../../operations/system-tables/query_log.md#system_tables-query_log), [query_thread_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log), [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) 由 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) 表引擎并将其数据存储在存储文件系统中。 如果从文件系统中删除表,ClickHouse服务器会在下一次写入数据时再次创建空表。 如果系统表架构在新版本中发生更改,则ClickHouse会重命名当前表并创建一个新表。 默认情况下,表增长是无限的。 要控制表的大小,可以使用 [TTL](../../sql-reference/statements/alter.md#manipulations-with-table-ttl) 删除过期日志记录的设置。 你也可以使用分区功能 `MergeTree`-发动机表。 diff --git a/docs/zh/operations/system-tables/merge_tree_settings.md b/docs/zh/operations/system-tables/merge_tree_settings.md index d2a5f64ba21..a6ad6f78f8e 100644 --- a/docs/zh/operations/system-tables/merge_tree_settings.md +++ b/docs/zh/operations/system-tables/merge_tree_settings.md @@ -3,7 +3,7 @@ machine_translated: true machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 --- -# 系统。merge\_tree\_settings {#system-merge_tree_settings} +# 系统。merge_tree_settings {#system-merge_tree_settings} 包含有关以下设置的信息 `MergeTree` 桌子 diff --git a/docs/zh/operations/system-tables/metric_log.md b/docs/zh/operations/system-tables/metric_log.md index 46b28f8d2f8..aaf04d74b0d 100644 --- a/docs/zh/operations/system-tables/metric_log.md +++ b/docs/zh/operations/system-tables/metric_log.md @@ -3,7 +3,7 @@ machine_translated: true machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 --- -# 系统。metric\_log {#system_tables-metric_log} +# 系统。metric_log {#system_tables-metric_log} 包含表中度量值的历史记录 `system.metrics` 和 `system.events`,定期刷新到磁盘。 打开指标历史记录收集 `system.metric_log`,创建 `/etc/clickhouse-server/config.d/metric_log.xml` 具有以下内容: @@ -54,7 +54,7 @@ CurrentMetric_ReplicatedChecks: 0 **另请参阅** -- [系统。asynchronous\_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [系统。asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. - [系统。活动](../../operations/system-tables/events.md#system_tables-events) — Contains a number of events that occurred. - [系统。指标](../../operations/system-tables/metrics.md#system_tables-metrics) — Contains instantly calculated metrics. - [监测](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring. diff --git a/docs/zh/operations/system-tables/metrics.md b/docs/zh/operations/system-tables/metrics.md index 1bf74524785..34b7fa35681 100644 --- a/docs/zh/operations/system-tables/metrics.md +++ b/docs/zh/operations/system-tables/metrics.md @@ -38,7 +38,7 @@ SELECT * FROM system.metrics LIMIT 10 **另请参阅** -- [系统。asynchronous\_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [系统。asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. - [系统。活动](../../operations/system-tables/events.md#system_tables-events) — Contains a number of events that occurred. -- [系统。metric\_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [系统。metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. - [监测](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring. diff --git a/docs/zh/operations/system-tables/mutations.md b/docs/zh/operations/system-tables/mutations.md index 8e2d66a42e8..17313274bd5 100644 --- a/docs/zh/operations/system-tables/mutations.md +++ b/docs/zh/operations/system-tables/mutations.md @@ -9,22 +9,22 @@ machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 **数据库**, **表** -应用突变的数据库和表的名称。 -**mutation\_id** -变异的ID 对于复制的表,这些Id对应于znode中的名称 `/mutations/` 动物园管理员的目录。 对于未复制的表,Id对应于表的数据目录中的文件名。 +**mutation_id** -变异的ID 对于复制的表,这些Id对应于znode中的名称 `/mutations/` 动物园管理员的目录。 对于未复制的表,Id对应于表的数据目录中的文件名。 **命令** -Mutation命令字符串(查询后的部分 `ALTER TABLE [db.]table`). -**create\_time** -当这个突变命令被提交执行。 +**create_time** -当这个突变命令被提交执行。 -**block\_numbers.partition\_id**, **block\_numbers.编号** -嵌套列。 对于复制表的突变,它包含每个分区的一条记录:分区ID和通过突变获取的块编号(在每个分区中,只有包含编号小于该分区中突变获取的块编号的块的 在非复制表中,所有分区中的块编号形成一个序列。 这意味着对于非复制表的突变,该列将包含一条记录,其中包含由突变获取的单个块编号。 +**block_numbers.partition_id**, **block_numbers.编号** -嵌套列。 对于复制表的突变,它包含每个分区的一条记录:分区ID和通过突变获取的块编号(在每个分区中,只有包含编号小于该分区中突变获取的块编号的块的 在非复制表中,所有分区中的块编号形成一个序列。 这意味着对于非复制表的突变,该列将包含一条记录,其中包含由突变获取的单个块编号。 -**parts\_to\_do** -为了完成突变,需要突变的数据部分的数量。 +**parts_to_do** -为了完成突变,需要突变的数据部分的数量。 -**is\_done** -变异完成了?? 请注意,即使 `parts_to_do = 0` 由于长时间运行的INSERT将创建需要突变的新数据部分,因此可能尚未完成复制表的突变。 +**is_done** -变异完成了?? 请注意,即使 `parts_to_do = 0` 由于长时间运行的INSERT将创建需要突变的新数据部分,因此可能尚未完成复制表的突变。 如果在改变某些部分时出现问题,以下列将包含其他信息: -**latest\_failed\_part** -不能变异的最新部分的名称。 +**latest_failed_part** -不能变异的最新部分的名称。 -**latest\_fail\_time** -最近的部分突变失败的时间。 +**latest_fail_time** -最近的部分突变失败的时间。 -**latest\_fail\_reason** -导致最近部件变异失败的异常消息。 +**latest_fail_reason** -导致最近部件变异失败的异常消息。 diff --git a/docs/zh/operations/system-tables/numbers_mt.md b/docs/zh/operations/system-tables/numbers_mt.md index 90addea157d..185bee95171 100644 --- a/docs/zh/operations/system-tables/numbers_mt.md +++ b/docs/zh/operations/system-tables/numbers_mt.md @@ -3,7 +3,7 @@ machine_translated: true machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 --- -# 系统。numbers\_mt {#system-numbers-mt} +# 系统。numbers_mt {#system-numbers-mt} 一样的 [系统。数字](../../operations/system-tables/numbers.md) 但读取是并行的。 这些数字可以以任何顺序返回。 diff --git a/docs/zh/operations/system-tables/part_log.md b/docs/zh/operations/system-tables/part_log.md index b8388455948..e316531dbd9 100644 --- a/docs/zh/operations/system-tables/part_log.md +++ b/docs/zh/operations/system-tables/part_log.md @@ -3,9 +3,9 @@ machine_translated: true machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 --- -# 系统。part\_log {#system_tables-part-log} +# 系统。part_log {#system_tables-part-log} -该 `system.part_log` 表只有当创建 [part\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-part-log) 指定了服务器设置。 +该 `system.part_log` 表只有当创建 [part_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-part-log) 指定了服务器设置。 此表包含与以下情况发生的事件有关的信息 [数据部分](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) 在 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) 家庭表,例如添加或合并数据。 diff --git a/docs/zh/operations/system-tables/processes.md b/docs/zh/operations/system-tables/processes.md index c42b7e59827..5ac74a29ee3 100644 --- a/docs/zh/operations/system-tables/processes.md +++ b/docs/zh/operations/system-tables/processes.md @@ -15,6 +15,6 @@ machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 - `rows_read` (UInt64) – The number of rows read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. - `bytes_read` (UInt64) – The number of uncompressed bytes read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. - `total_rows_approx` (UInt64) – The approximation of the total number of rows that should be read. For distributed processing, on the requestor server, this is the total for all remote servers. It can be updated during request processing, when new sources to process become known. -- `memory_usage` (UInt64) – Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max\_memory\_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) 设置。 +- `memory_usage` (UInt64) – Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) 设置。 - `query` (String) – The query text. For `INSERT`,它不包括要插入的数据。 - `query_id` (String) – Query ID, if defined. diff --git a/docs/zh/operations/system-tables/query_log.md b/docs/zh/operations/system-tables/query_log.md index 7658196b81b..bf3aa063a83 100644 --- a/docs/zh/operations/system-tables/query_log.md +++ b/docs/zh/operations/system-tables/query_log.md @@ -3,18 +3,18 @@ machine_translated: true machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 --- -# 系统。query\_log {#system_tables-query_log} +# 系统。query_log {#system_tables-query_log} 包含有关已执行查询的信息,例如,开始时间、处理持续时间、错误消息。 !!! note "注" 此表不包含以下内容的摄取数据 `INSERT` 查询。 -您可以更改查询日志记录的设置 [query\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query-log) 服务器配置部分。 +您可以更改查询日志记录的设置 [query_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query-log) 服务器配置部分。 -您可以通过设置禁用查询日志记录 [log\_queries=0](../../operations/settings/settings.md#settings-log-queries). 我们不建议关闭日志记录,因为此表中的信息对于解决问题很重要。 +您可以通过设置禁用查询日志记录 [log_queries=0](../../operations/settings/settings.md#settings-log-queries). 我们不建议关闭日志记录,因为此表中的信息对于解决问题很重要。 -数据的冲洗周期设置在 `flush_interval_milliseconds` 的参数 [query\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query-log) 服务器设置部分。 要强制冲洗,请使用 [SYSTEM FLUSH LOGS](../../sql-reference/statements/system.md#query_language-system-flush_logs) 查询。 +数据的冲洗周期设置在 `flush_interval_milliseconds` 的参数 [query_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query-log) 服务器设置部分。 要强制冲洗,请使用 [SYSTEM FLUSH LOGS](../../sql-reference/statements/system.md#query_language-system-flush_logs) 查询。 ClickHouse不会自动从表中删除数据。 看 [导言](../../operations/system-tables/index.md#system-tables-introduction) 欲了解更多详情。 @@ -140,4 +140,4 @@ Settings.Values: ['0','random','1','10000000000'] **另请参阅** -- [系统。query\_thread\_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log) — This table contains information about each query execution thread. +- [系统。query_thread_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log) — This table contains information about each query execution thread. diff --git a/docs/zh/operations/system-tables/query_thread_log.md b/docs/zh/operations/system-tables/query_thread_log.md index 115e69ec93c..8043be8bd75 100644 --- a/docs/zh/operations/system-tables/query_thread_log.md +++ b/docs/zh/operations/system-tables/query_thread_log.md @@ -3,16 +3,16 @@ machine_translated: true machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 --- -# 系统。query\_thread\_log {#system_tables-query_thread_log} +# 系统。query_thread_log {#system_tables-query_thread_log} 包含有关执行查询的线程的信息,例如,线程名称、线程开始时间、查询处理的持续时间。 开始记录: -1. 在配置参数 [query\_thread\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) 科。 -2. 设置 [log\_query\_threads](../../operations/settings/settings.md#settings-log-query-threads) 到1。 +1. 在配置参数 [query_thread_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) 科。 +2. 设置 [log_query_threads](../../operations/settings/settings.md#settings-log-query-threads) 到1。 -数据的冲洗周期设置在 `flush_interval_milliseconds` 的参数 [query\_thread\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) 服务器设置部分。 要强制冲洗,请使用 [SYSTEM FLUSH LOGS](../../sql-reference/statements/system.md#query_language-system-flush_logs) 查询。 +数据的冲洗周期设置在 `flush_interval_milliseconds` 的参数 [query_thread_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) 服务器设置部分。 要强制冲洗,请使用 [SYSTEM FLUSH LOGS](../../sql-reference/statements/system.md#query_language-system-flush_logs) 查询。 ClickHouse不会自动从表中删除数据。 看 [导言](../../operations/system-tables/index.md#system-tables-introduction) 欲了解更多详情。 @@ -115,4 +115,4 @@ ProfileEvents.Values: [1,97,81,5,81] **另请参阅** -- [系统。query\_log](../../operations/system-tables/query_log.md#system_tables-query_log) — Description of the `query_log` 系统表,其中包含有关查询执行的公共信息。 +- [系统。query_log](../../operations/system-tables/query_log.md#system_tables-query_log) — Description of the `query_log` 系统表,其中包含有关查询执行的公共信息。 diff --git a/docs/zh/operations/system-tables/replicas.md b/docs/zh/operations/system-tables/replicas.md index b3b905b5d5b..b5bf0c891d4 100644 --- a/docs/zh/operations/system-tables/replicas.md +++ b/docs/zh/operations/system-tables/replicas.md @@ -68,7 +68,7 @@ active_replicas: 2 - `parts_to_check` (`UInt32`)-队列中用于验证的数据部分的数量。 如果怀疑零件可能已损坏,则将其放入验证队列。 - `zookeeper_path` (`String`)-在ZooKeeper中的表数据路径。 - `replica_name` (`String`)-在动物园管理员副本名称. 同一表的不同副本具有不同的名称。 -- `replica_path` (`String`)-在ZooKeeper中的副本数据的路径。 与连接相同 ‘zookeeper\_path/replicas/replica\_path’. +- `replica_path` (`String`)-在ZooKeeper中的副本数据的路径。 与连接相同 ‘zookeeper_path/replicas/replica_path’. - `columns_version` (`Int32`)-表结构的版本号。 指示执行ALTER的次数。 如果副本有不同的版本,这意味着一些副本还没有做出所有的改变。 - `queue_size` (`UInt32`)-等待执行的操作的队列大小。 操作包括插入数据块、合并和某些其他操作。 它通常与 `future_parts`. - `inserts_in_queue` (`UInt32`)-需要插入数据块的数量。 插入通常复制得相当快。 如果这个数字很大,这意味着有什么不对劲。 @@ -89,7 +89,7 @@ active_replicas: 2 - `active_replicas` (`UInt8`)-在ZooKeeper中具有会话的此表的副本的数量(即正常运行的副本的数量)。 如果您请求所有列,表可能会工作得有点慢,因为每行都会从ZooKeeper进行几次读取。 -如果您没有请求最后4列(log\_max\_index,log\_pointer,total\_replicas,active\_replicas),表工作得很快。 +如果您没有请求最后4列(log_max_index,log_pointer,total_replicas,active_replicas),表工作得很快。 例如,您可以检查一切是否正常工作,如下所示: diff --git a/docs/zh/operations/system-tables/storage_policies.md b/docs/zh/operations/system-tables/storage_policies.md index 29347aa36c2..550af6b2b27 100644 --- a/docs/zh/operations/system-tables/storage_policies.md +++ b/docs/zh/operations/system-tables/storage_policies.md @@ -3,7 +3,7 @@ machine_translated: true machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 --- -# 系统。storage\_policies {#system_tables-storage_policies} +# 系统。storage_policies {#system_tables-storage_policies} 包含有关存储策略和卷中定义的信息 [服务器配置](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes_configure). diff --git a/docs/zh/operations/system-tables/table_engines.md b/docs/zh/operations/system-tables/table_engines.md index 401a3a8616d..4e107868aa5 100644 --- a/docs/zh/operations/system-tables/table_engines.md +++ b/docs/zh/operations/system-tables/table_engines.md @@ -3,7 +3,7 @@ machine_translated: true machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 --- -# 系统。表\_engines {#system-table-engines} +# 系统。表_engines {#system-table-engines} 包含服务器支持的表引擎的描述及其功能支持信息。 diff --git a/docs/zh/operations/system-tables/text_log.md b/docs/zh/operations/system-tables/text_log.md index 9643fbef358..3c56c8e42e1 100644 --- a/docs/zh/operations/system-tables/text_log.md +++ b/docs/zh/operations/system-tables/text_log.md @@ -3,7 +3,7 @@ machine_translated: true machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 --- -# 系统。text\_log {#system_tables-text_log} +# 系统。text_log {#system_tables-text_log} 包含日志记录条目。 进入该表的日志记录级别可以通过以下方式进行限制 `text_log.level` 服务器设置。 diff --git a/docs/zh/operations/system-tables/trace_log.md b/docs/zh/operations/system-tables/trace_log.md index 3004dca707a..6c9404d6e82 100644 --- a/docs/zh/operations/system-tables/trace_log.md +++ b/docs/zh/operations/system-tables/trace_log.md @@ -3,11 +3,11 @@ machine_translated: true machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 --- -# 系统。trace\_log {#system_tables-trace_log} +# 系统。trace_log {#system_tables-trace_log} 包含采样查询探查器收集的堆栈跟踪。 -ClickHouse创建此表时 [trace\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) 服务器配置部分被设置。 也是 [query\_profiler\_real\_time\_period\_ns](../../operations/settings/settings.md#query_profiler_real_time_period_ns) 和 [query\_profiler\_cpu\_time\_period\_ns](../../operations/settings/settings.md#query_profiler_cpu_time_period_ns) 应设置设置。 +ClickHouse创建此表时 [trace_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) 服务器配置部分被设置。 也是 [query_profiler_real_time_period_ns](../../operations/settings/settings.md#query_profiler_real_time_period_ns) 和 [query_profiler_cpu_time_period_ns](../../operations/settings/settings.md#query_profiler_cpu_time_period_ns) 应设置设置。 要分析日志,请使用 `addressToLine`, `addressToSymbol` 和 `demangle` 内省功能。 @@ -30,7 +30,7 @@ ClickHouse创建此表时 [trace\_log](../../operations/server-configuration-par - `thread_number` ([UInt32](../../sql-reference/data-types/int-uint.md)) — Thread identifier. -- `query_id` ([字符串](../../sql-reference/data-types/string.md)) — Query identifier that can be used to get details about a query that was running from the [query\_log](#system_tables-query_log) 系统表. +- `query_id` ([字符串](../../sql-reference/data-types/string.md)) — Query identifier that can be used to get details about a query that was running from the [query_log](#system_tables-query_log) 系统表. - `trace` ([数组(UInt64)](../../sql-reference/data-types/array.md)) — Stack trace at the moment of sampling. Each element is a virtual memory address inside ClickHouse server process. diff --git a/docs/zh/operations/tips.md b/docs/zh/operations/tips.md index eb3abab08c6..4a8cf2efb96 100644 --- a/docs/zh/operations/tips.md +++ b/docs/zh/operations/tips.md @@ -66,7 +66,7 @@ echo 'never' | sudo tee /sys/kernel/mm/transparent_hugepage/enabled 如果您的预算允许,请选择RAID-10。 如果您有超过4个磁盘,请使用RAID-6(首选)或RAID-50,而不是RAID-5。 -当使用RAID-5、RAID-6或RAID-50时,始终增加stripe\_cache\_size,因为默认值通常不是最佳选择。 +当使用RAID-5、RAID-6或RAID-50时,始终增加stripe_cache_size,因为默认值通常不是最佳选择。 ``` bash echo 4096 | sudo tee /sys/block/md2/md/stripe_cache_size diff --git a/docs/zh/operations/troubleshooting.md b/docs/zh/operations/troubleshooting.md index d9a00717d7b..56b18aa1307 100644 --- a/docs/zh/operations/troubleshooting.md +++ b/docs/zh/operations/troubleshooting.md @@ -103,7 +103,7 @@ $ sudo -u clickhouse /usr/bin/clickhouse-server --config-file /etc/clickhouse-se - 端点设置。 - 检查 [listen\_host](server-configuration-parameters/settings.md#server_configuration_parameters-listen_host) 和 [tcp\_port](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) 设置。 + 检查 [listen_host](server-configuration-parameters/settings.md#server_configuration_parameters-listen_host) 和 [tcp_port](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) 设置。 ClickHouse服务器默认情况下仅接受本地主机连接。 @@ -115,7 +115,7 @@ $ sudo -u clickhouse /usr/bin/clickhouse-server --config-file /etc/clickhouse-se 检查: - - [tcp\_port\_secure](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) 设置。 + - [tcp_port_secure](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) 设置。 - [SSL证书](server-configuration-parameters/settings.md#server_configuration_parameters-openssl) 设置. 连接时使用正确的参数。 例如,使用 `clickhouse_client` 的时候使用 `port_secure` 参数 . diff --git a/docs/zh/sql-reference/aggregate-functions/parametric-functions.md b/docs/zh/sql-reference/aggregate-functions/parametric-functions.md index 69572086549..d151bbc3957 100644 --- a/docs/zh/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/zh/sql-reference/aggregate-functions/parametric-functions.md @@ -491,7 +491,7 @@ FROM 解决方案: 写group by查询语句 HAVING uniqUpTo(4)(UserID) >= 5 ``` -## sumMapFiltered(keys\_to\_keep)(keys, values) {#summapfilteredkeys-to-keepkeys-values} +## sumMapFiltered(keys_to_keep)(keys, values) {#summapfilteredkeys-to-keepkeys-values} 和 [sumMap](reference.md#agg_functions-summap) 基本一致, 除了一个键数组作为参数传递。这在使用高基数key时尤其有用。 diff --git a/docs/zh/sql-reference/aggregate-functions/reference.md b/docs/zh/sql-reference/aggregate-functions/reference.md index 7d5ecda7bb4..8d1dcda4d83 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference.md +++ b/docs/zh/sql-reference/aggregate-functions/reference.md @@ -29,7 +29,7 @@ ClickHouse支持以下语法 `count`: **详细信息** -ClickHouse支持 `COUNT(DISTINCT ...)` 语法 这种结构的行为取决于 [count\_distinct\_implementation](../../operations/settings/settings.md#settings-count_distinct_implementation) 设置。 它定义了其中的 [uniq\*](#agg_function-uniq) 函数用于执行操作。 默认值为 [uniqExact](#agg_function-uniqexact) 功能。 +ClickHouse支持 `COUNT(DISTINCT ...)` 语法 这种结构的行为取决于 [count_distinct_implementation](../../operations/settings/settings.md#settings-count_distinct_implementation) 设置。 它定义了其中的 [uniq\*](#agg_function-uniq) 函数用于执行操作。 默认值为 [uniqExact](#agg_function-uniqexact) 功能。 该 `SELECT count() FROM table` 查询未被优化,因为表中的条目数没有单独存储。 它从表中选择一个小列并计算其中的值数。 @@ -719,7 +719,7 @@ uniqExact(x[, ...]) - [uniqCombined](#agg_function-uniqcombined) - [uniqHLL12](#agg_function-uniqhll12) -## groupArray(x), groupArray(max\_size)(x) {#agg_function-grouparray} +## groupArray(x), groupArray(max_size)(x) {#agg_function-grouparray} 创建参数值的数组。 值可以按任何(不确定)顺序添加到数组中。 @@ -965,7 +965,7 @@ FROM t └───────────┴──────────────────────────────────┴───────────────────────┘ ``` -## groupUniqArray(x), groupUniqArray(max\_size)(x) {#groupuniqarrayx-groupuniqarraymax-sizex} +## groupUniqArray(x), groupUniqArray(max_size)(x) {#groupuniqarrayx-groupuniqarraymax-sizex} 从不同的参数值创建一个数组。 内存消耗是一样的 `uniqExact` 功能。 diff --git a/docs/zh/sql-reference/ansi.md b/docs/zh/sql-reference/ansi.md index 8c80bc20398..8e8590079e2 100644 --- a/docs/zh/sql-reference/ansi.md +++ b/docs/zh/sql-reference/ansi.md @@ -37,7 +37,7 @@ toc_title: "ANSI\u517C\u5BB9\u6027" | E021-02 | 字符变化数据类型 | 非也。{.text-danger} | `String` 行为类似,但括号中没有长度限制 | | E021-03 | 字符文字 | 部分{.text-warning} | 不自动连接连续文字和字符集支持 | | E021-04 | 字符长度函数 | 部分{.text-warning} | 非也。 `USING` 条款 | -| E021-05 | OCTET\_LENGTH函数 | 非也。{.text-danger} | `LENGTH` 表现类似 | +| E021-05 | OCTET_LENGTH函数 | 非也。{.text-danger} | `LENGTH` 表现类似 | | E021-06 | SUBSTRING | 部分{.text-warning} | 不支持 `SIMILAR` 和 `ESCAPE` 条款,否 `SUBSTRING_REGEX` 备选案文 | | E021-07 | 字符串联 | 部分{.text-warning} | 非也。 `COLLATE` 条款 | | E021-08 | 上下功能 | 是{.text-success} | | @@ -144,7 +144,7 @@ toc_title: "ANSI\u517C\u5BB9\u6027" | F051-03 | 时间戳数据类型(包括对时间戳文字的支持),小数秒精度至少为0和6 | 非也。{.text-danger} | `DateTime64` 时间提供了类似的功能 | | F051-04 | 日期、时间和时间戳数据类型的比较谓词 | 部分{.text-warning} | 只有一种数据类型可用 | | F051-05 | Datetime类型和字符串类型之间的显式转换 | 是{.text-success} | | -| F051-06 | CURRENT\_DATE | 非也。{.text-danger} | `today()` 是相似的 | +| F051-06 | CURRENT_DATE | 非也。{.text-danger} | `today()` 是相似的 | | F051-07 | LOCALTIME | 非也。{.text-danger} | `now()` 是相似的 | | F051-08 | LOCALTIMESTAMP | 非也。{.text-danger} | | | **F081** | **联盟和视图除外** | **部分**{.text-warning} | | diff --git a/docs/zh/sql-reference/data-types/aggregatefunction.md b/docs/zh/sql-reference/data-types/aggregatefunction.md index a7e1ee35c16..522ecf1b42d 100644 --- a/docs/zh/sql-reference/data-types/aggregatefunction.md +++ b/docs/zh/sql-reference/data-types/aggregatefunction.md @@ -1,4 +1,4 @@ -# AggregateFunction(name, types\_of\_arguments…) {#data-type-aggregatefunction} +# AggregateFunction(name, types_of_arguments…) {#data-type-aggregatefunction} 聚合函数的中间状态,可以通过聚合函数名称加`-State`后缀的形式得到它。与此同时,当您需要访问该类型的最终状态数据时,您需要以相同的聚合函数名加`-Merge`后缀的形式来得到最终状态数据。 diff --git a/docs/zh/sql-reference/data-types/domains/index.md b/docs/zh/sql-reference/data-types/domains/index.md index 6ef788b0650..1bcf8d31998 100644 --- a/docs/zh/sql-reference/data-types/domains/index.md +++ b/docs/zh/sql-reference/data-types/domains/index.md @@ -19,9 +19,9 @@ Domain类型是特定实现的类型,它总是与某个现存的基础类型 ### Domains的额外特性 {#domainsde-e-wai-te-xing} - 在执行SHOW CREATE TABLE 或 DESCRIBE TABLE时,其对应的列总是展示为Domain类型的名称 -- 在INSERT INTO domain\_table(domain\_column) VALUES(…)中输入数据总是以更人性化的格式进行输入 -- 在SELECT domain\_column FROM domain\_table中数据总是以更人性化的格式输出 -- 在INSERT INTO domain\_table FORMAT CSV …中,实现外部源数据以更人性化的格式载入 +- 在INSERT INTO domain_table(domain_column) VALUES(…)中输入数据总是以更人性化的格式进行输入 +- 在SELECT domain_column FROM domain_table中数据总是以更人性化的格式输出 +- 在INSERT INTO domain_table FORMAT CSV …中,实现外部源数据以更人性化的格式载入 ### Domains类型的限制 {#domainslei-xing-de-xian-zhi} diff --git a/docs/zh/sql-reference/data-types/fixedstring.md b/docs/zh/sql-reference/data-types/fixedstring.md index 45550e12b4e..f73259197c8 100644 --- a/docs/zh/sql-reference/data-types/fixedstring.md +++ b/docs/zh/sql-reference/data-types/fixedstring.md @@ -15,7 +15,7 @@ 可以有效存储在`FixedString`类型的列中的值的示例: - 二进制表示的IP地址(IPv6使用`FixedString(16)`) -- 语言代码(ru\_RU, en\_US … ) +- 语言代码(ru_RU, en_US … ) - 货币代码(USD, RUB … ) - 二进制表示的哈希值(MD5使用`FixedString(16)`,SHA256使用`FixedString(32)`) diff --git a/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md b/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md index 0b473f0c816..2a890441085 100644 --- a/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md +++ b/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md @@ -27,7 +27,7 @@ ClickHouse支持分层字典与 [数字键](external-dicts-dict-structure.md#ext 这种层次结构可以表示为下面的字典表。 -| region\_id | parent\_region | region\_name | +| region_id | parent_region | region_name | |------------|----------------|--------------| | 1 | 0 | 俄罗斯 | | 2 | 1 | 莫斯科 | diff --git a/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 0843a6a535a..176859d3633 100644 --- a/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -9,7 +9,7 @@ toc_title: "\u5728\u5185\u5B58\u4E2D\u5B58\u50A8\u5B57\u5178" 有多种方法可以将字典存储在内存中。 -我们建议 [平](#flat), [散列](#dicts-external_dicts_dict_layout-hashed) 和 [complex\_key\_hashed](#complex-key-hashed). 其提供最佳的处理速度。 +我们建议 [平](#flat), [散列](#dicts-external_dicts_dict_layout-hashed) 和 [complex_key_hashed](#complex-key-hashed). 其提供最佳的处理速度。 不建议使用缓存,因为性能可能较差,并且难以选择最佳参数。 阅读更多的部分 “[缓存](#cache)”. @@ -54,13 +54,13 @@ LAYOUT(LAYOUT_TYPE(param value)) -- layout settings - [平](#flat) - [散列](#dicts-external_dicts_dict_layout-hashed) -- [sparse\_hashed](#dicts-external_dicts_dict_layout-sparse_hashed) +- [sparse_hashed](#dicts-external_dicts_dict_layout-sparse_hashed) - [缓存](#cache) - [直接](#direct) -- [range\_hashed](#range-hashed) -- [complex\_key\_hashed](#complex-key-hashed) -- [complex\_key\_cache](#complex-key-cache) -- [ip\_trie](#ip-trie) +- [range_hashed](#range-hashed) +- [complex_key_hashed](#complex-key-hashed) +- [complex_key_cache](#complex-key-cache) +- [ip_trie](#ip-trie) ### 平 {#flat} @@ -106,7 +106,7 @@ LAYOUT(FLAT()) LAYOUT(HASHED()) ``` -### sparse\_hashed {#dicts-external_dicts_dict_layout-sparse_hashed} +### sparse_hashed {#dicts-external_dicts_dict_layout-sparse_hashed} 类似于 `hashed`,但使用更少的内存,有利于更多的CPU使用率。 @@ -122,7 +122,7 @@ LAYOUT(HASHED()) LAYOUT(SPARSE_HASHED()) ``` -### complex\_key\_hashed {#complex-key-hashed} +### complex_key_hashed {#complex-key-hashed} 这种类型的存储是用于复合 [键](external-dicts-dict-structure.md). 类似于 `hashed`. @@ -138,7 +138,7 @@ LAYOUT(SPARSE_HASHED()) LAYOUT(COMPLEX_KEY_HASHED()) ``` -### range\_hashed {#range-hashed} +### range_hashed {#range-hashed} 字典以哈希表的形式存储在内存中,其中包含有序范围及其相应值的数组。 @@ -293,7 +293,7 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000)) !!! warning "警告" 不要使用ClickHouse作为源,因为处理随机读取的查询速度很慢。 -### complex\_key\_cache {#complex-key-cache} +### complex_key_cache {#complex-key-cache} 这种类型的存储是用于复合 [键](external-dicts-dict-structure.md). 类似于 `cache`. @@ -319,7 +319,7 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000)) LAYOUT(DIRECT()) ``` -### ip\_trie {#ip-trie} +### ip_trie {#ip-trie} 这种类型的存储用于将网络前缀(IP地址)映射到ASN等元数据。 diff --git a/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts.md index 756eee31026..1dfea8d21be 100644 --- a/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ b/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts.md @@ -15,9 +15,9 @@ ClickHouse: - 定期更新字典并动态加载缺失的值。 换句话说,字典可以动态加载。 - 允许创建外部字典与xml文件或 [DDL查询](../../statements/create.md#create-dictionary-query). -外部字典的配置可以位于一个或多个xml文件中。 配置的路径在指定 [dictionaries\_config](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config) 参数。 +外部字典的配置可以位于一个或多个xml文件中。 配置的路径在指定 [dictionaries_config](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config) 参数。 -字典可以在服务器启动或首次使用时加载,具体取决于 [dictionaries\_lazy\_load](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load) 设置。 +字典可以在服务器启动或首次使用时加载,具体取决于 [dictionaries_lazy_load](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load) 设置。 该 [字典](../../../operations/system-tables/dictionaries.md#system_tables-dictionaries) 系统表包含有关在服务器上配置的字典的信息。 对于每个字典,你可以在那里找到: diff --git a/docs/zh/sql-reference/functions/array-functions.md b/docs/zh/sql-reference/functions/array-functions.md index 858a97940c6..ac5dae3a97e 100644 --- a/docs/zh/sql-reference/functions/array-functions.md +++ b/docs/zh/sql-reference/functions/array-functions.md @@ -652,11 +652,11 @@ SELECT │ [] │ [1] │ └──────────────┴───────────┘ -## arrayReduce(agg\_func, arr1, …) {#arrayreduceagg-func-arr1} +## arrayReduce(agg_func, arr1, …) {#arrayreduceagg-func-arr1} 将聚合函数应用于数组并返回其结果。如果聚合函数具有多个参数,则此函数可应用于相同大小的多个数组。 -arrayReduce(‘agg\_func’,arr1,…) - 将聚合函数`agg_func`应用于数组`arr1 ...`。如果传递了多个数组,则相应位置上的元素将作为多个参数传递给聚合函数。例如:SELECT arrayReduce(‘max’,\[1,2,3\])= 3 +arrayReduce(‘agg_func’,arr1,…) - 将聚合函数`agg_func`应用于数组`arr1 ...`。如果传递了多个数组,则相应位置上的元素将作为多个参数传递给聚合函数。例如:SELECT arrayReduce(‘max’,\[1,2,3\])= 3 ## arrayReverse(arr) {#arrayreversearr} diff --git a/docs/zh/sql-reference/functions/bitmap-functions.md b/docs/zh/sql-reference/functions/bitmap-functions.md index 239cd8b7d2d..d2018f5d9c1 100644 --- a/docs/zh/sql-reference/functions/bitmap-functions.md +++ b/docs/zh/sql-reference/functions/bitmap-functions.md @@ -46,7 +46,7 @@ SELECT bitmapToArray(bitmapBuild([1, 2, 3, 4, 5])) AS res ## bitmapSubsetInRange {#bitmapsubsetinrange} -将位图指定范围(不包含range\_end)转换为另一个位图。 +将位图指定范围(不包含range_end)转换为另一个位图。 bitmapSubsetInRange(bitmap, range_start, range_end) @@ -259,7 +259,7 @@ SELECT bitmapCardinality(bitmapBuild([1, 2, 3, 4, 5])) AS res ## bitmapMin {#bitmapmin} -返回一个UInt64类型的数值,表示位图中的最小值。如果位图为空则返回UINT32\_MAX。 +返回一个UInt64类型的数值,表示位图中的最小值。如果位图为空则返回UINT32_MAX。 bitmapMin(bitmap) diff --git a/docs/zh/sql-reference/functions/conditional-functions.md b/docs/zh/sql-reference/functions/conditional-functions.md index 265c4387cb1..b8e96620014 100644 --- a/docs/zh/sql-reference/functions/conditional-functions.md +++ b/docs/zh/sql-reference/functions/conditional-functions.md @@ -23,7 +23,7 @@ **返回值** -该函数返回值«then\_N»或«else»之一,具体取决于条件`cond_N`。 +该函数返回值«then_N»或«else»之一,具体取决于条件`cond_N`。 **示例** diff --git a/docs/zh/sql-reference/functions/date-time-functions.md b/docs/zh/sql-reference/functions/date-time-functions.md index a418379d4ec..65d331a7846 100644 --- a/docs/zh/sql-reference/functions/date-time-functions.md +++ b/docs/zh/sql-reference/functions/date-time-functions.md @@ -116,7 +116,7 @@ SELECT 将DateTime以十五分钟为单位向前取整到最接近的时间点。 -## toStartOfInterval(time\_or\_data,间隔x单位\[,time\_zone\]) {#tostartofintervaltime-or-data-interval-x-unit-time-zone} +## toStartOfInterval(time_or_data,间隔x单位\[,time_zone\]) {#tostartofintervaltime-or-data-interval-x-unit-time-zone} 这是名为`toStartOf*`的所有函数的通用函数。例如, `toStartOfInterval(t,INTERVAL 1 year)`返回与`toStartOfYear(t)`相同的结果, diff --git a/docs/zh/sql-reference/functions/ext-dict-functions.md b/docs/zh/sql-reference/functions/ext-dict-functions.md index cae193b2e22..22ee9641b55 100644 --- a/docs/zh/sql-reference/functions/ext-dict-functions.md +++ b/docs/zh/sql-reference/functions/ext-dict-functions.md @@ -16,7 +16,7 @@ `dictGetT('dict_name', 'attr_name', id)` -- 使用’id’键获取dict\_name字典中attr\_name属性的值。`dict_name`和`attr_name`是常量字符串。`id`必须是UInt64。 +- 使用’id’键获取dict_name字典中attr_name属性的值。`dict_name`和`attr_name`是常量字符串。`id`必须是UInt64。 如果字典中没有`id`键,则返回字典描述中指定的默认值。 ## dictGetTOrDefault {#ext_dict_functions-dictgettordefault} @@ -29,13 +29,13 @@ `dictIsIn ('dict_name', child_id, ancestor_id)` -- 对于’dict\_name’分层字典,查找’child\_id’键是否位于’ancestor\_id’内(或匹配’ancestor\_id’)。返回UInt8。 +- 对于’dict_name’分层字典,查找’child_id’键是否位于’ancestor_id’内(或匹配’ancestor_id’)。返回UInt8。 ## 独裁主义 {#dictgethierarchy} `dictGetHierarchy('dict_name', id)` -- 对于’dict\_name’分层字典,返回从’id’开始并沿父元素链继续的字典键数组。返回Array(UInt64) +- 对于’dict_name’分层字典,返回从’id’开始并沿父元素链继续的字典键数组。返回Array(UInt64) ## dictHas {#dicthas} diff --git a/docs/zh/sql-reference/functions/geo.md b/docs/zh/sql-reference/functions/geo.md index 7cc094932fb..d3e7ba0619e 100644 --- a/docs/zh/sql-reference/functions/geo.md +++ b/docs/zh/sql-reference/functions/geo.md @@ -192,10 +192,10 @@ SELECT geoToH3(37.79506683, 55.71290588, 15) as h3Index **输入值** -- longitude\_min - 最小经度。其值应在`[-180°,180°]`范围内 -- latitude\_min - 最小纬度。其值应在`[-90°,90°]`范围内 -- longitude\_max - 最大经度。其值应在`[-180°,180°]`范围内 -- latitude\_max - 最大纬度。其值应在`[-90°,90°]`范围内 +- longitude_min - 最小经度。其值应在`[-180°,180°]`范围内 +- latitude_min - 最小纬度。其值应在`[-90°,90°]`范围内 +- longitude_max - 最大经度。其值应在`[-180°,180°]`范围内 +- latitude_max - 最大纬度。其值应在`[-90°,90°]`范围内 - precision - geohash的精度。其值应在`[1, 12]`内的`UInt8`类型的数字 请注意,上述所有的坐标参数必须同为`Float32`或`Float64`中的一种类型。 diff --git a/docs/zh/sql-reference/functions/hash-functions.md b/docs/zh/sql-reference/functions/hash-functions.md index 5646eee6af8..b64456fe644 100644 --- a/docs/zh/sql-reference/functions/hash-functions.md +++ b/docs/zh/sql-reference/functions/hash-functions.md @@ -93,13 +93,13 @@ URL的层级与URLHierarchy中的层级相同。 此函数被用于Yandex.Metric 接受UInt64类型的参数。返回Int32。 有关更多信息,请参见链接:[JumpConsistentHash](https://arxiv.org/pdf/1406.2294.pdf) -## murmurHash2\_32,murmurHash2\_64 {#murmurhash2-32-murmurhash2-64} +## murmurHash2_32,murmurHash2_64 {#murmurhash2-32-murmurhash2-64} 计算字符串的MurmurHash2。 接受一个String类型的参数。返回UInt64或UInt32。 有关更多信息,请参阅链接:[MurmurHash2](https://github.com/aappleby/smhasher) -## murmurHash3\_32,murmurHash3\_64,murmurHash3\_128 {#murmurhash3-32-murmurhash3-64-murmurhash3-128} +## murmurHash3_32,murmurHash3_64,murmurHash3_128 {#murmurhash3-32-murmurhash3-64-murmurhash3-128} 计算字符串的MurmurHash3。 接受一个String类型的参数。返回UInt64或UInt32或FixedString(16)。 diff --git a/docs/zh/sql-reference/functions/introspection.md b/docs/zh/sql-reference/functions/introspection.md index 4b4367f3dc8..8d3b909b199 100644 --- a/docs/zh/sql-reference/functions/introspection.md +++ b/docs/zh/sql-reference/functions/introspection.md @@ -16,11 +16,11 @@ toc_title: "\u81EA\u7701" - 安装 `clickhouse-common-static-dbg` 包。 -- 设置 [allow\_introspection\_functions](../../operations/settings/settings.md#settings-allow_introspection_functions) 设置为1。 +- 设置 [allow_introspection_functions](../../operations/settings/settings.md#settings-allow_introspection_functions) 设置为1。 For security reasons introspection functions are disabled by default. -ClickHouse将探查器报告保存到 [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) 系统表. 确保正确配置了表和探查器。 +ClickHouse将探查器报告保存到 [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) 系统表. 确保正确配置了表和探查器。 ## addressToLine {#addresstoline} diff --git a/docs/zh/sql-reference/functions/json-functions.md b/docs/zh/sql-reference/functions/json-functions.md index 137e60e5430..441ec29a289 100644 --- a/docs/zh/sql-reference/functions/json-functions.md +++ b/docs/zh/sql-reference/functions/json-functions.md @@ -53,7 +53,7 @@ 以下函数基于[simdjson](https://github.com/lemire/simdjson),专为更复杂的JSON解析要求而设计。但上述假设2仍然适用。 -## JSONHas(json\[, indices\_or\_keys\]…) {#jsonhasjson-indices-or-keys} +## JSONHas(json\[, indices_or_keys\]…) {#jsonhasjson-indices-or-keys} 如果JSON中存在该值,则返回`1`。 @@ -80,7 +80,7 @@ select JSONExtractKey('{"a": "hello", "b": [-100, 200.0, 300]}', -2) = 'a' select JSONExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 1) = 'hello' -## JSONLength(json\[, indices\_or\_keys\]…) {#jsonlengthjson-indices-or-keys} +## JSONLength(json\[, indices_or_keys\]…) {#jsonlengthjson-indices-or-keys} 返回JSON数组或JSON对象的长度。 @@ -91,7 +91,7 @@ select JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 3 select JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}') = 2 -## JSONType(json\[, indices\_or\_keys\]…) {#jsontypejson-indices-or-keys} +## JSONType(json\[, indices_or_keys\]…) {#jsontypejson-indices-or-keys} 返回JSON值的类型。 @@ -103,13 +103,13 @@ select JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'a') = 'String' select JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 'Array' -## JSONExtractUInt(json\[, indices\_or\_keys\]…) {#jsonextractuintjson-indices-or-keys} +## JSONExtractUInt(json\[, indices_or_keys\]…) {#jsonextractuintjson-indices-or-keys} -## JSONExtractInt(json\[, indices\_or\_keys\]…) {#jsonextractintjson-indices-or-keys} +## JSONExtractInt(json\[, indices_or_keys\]…) {#jsonextractintjson-indices-or-keys} -## JSONExtractFloat(json\[, indices\_or\_keys\]…) {#jsonextractfloatjson-indices-or-keys} +## JSONExtractFloat(json\[, indices_or_keys\]…) {#jsonextractfloatjson-indices-or-keys} -## JSONExtractBool(json\[, indices\_or\_keys\]…) {#jsonextractbooljson-indices-or-keys} +## JSONExtractBool(json\[, indices_or_keys\]…) {#jsonextractbooljson-indices-or-keys} 解析JSON并提取值。这些函数类似于`visitParam*`函数。 @@ -121,7 +121,7 @@ select JSONExtractFloat('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 2) = 200.0 select JSONExtractUInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', -1) = 300 -## JSONExtractString(json\[, indices\_or\_keys\]…) {#jsonextractstringjson-indices-or-keys} +## JSONExtractString(json\[, indices_or_keys\]…) {#jsonextractstringjson-indices-or-keys} 解析JSON并提取字符串。此函数类似于`visitParamExtractString`函数。 @@ -137,7 +137,7 @@ select JSONExtractString('{"abc":"\\u263"}', 'abc') = '' select JSONExtractString('{"abc":"hello}', 'abc') = '' -## JSONExtract(json\[, indices\_or\_keys…\], Return\_type) {#jsonextractjson-indices-or-keys-return-type} +## JSONExtract(json\[, indices_or_keys…\], Return_type) {#jsonextractjson-indices-or-keys-return-type} 解析JSON并提取给定ClickHouse数据类型的值。 @@ -153,7 +153,7 @@ SELECT JSONExtract('{"day": "Thursday"}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \'Tuesday\' = 2, \'Wednesday\' = 3, \'Thursday\' = 4, \'Friday\' = 5, \'Saturday\' = 6)') = 'Thursday' SELECT JSONExtract('{"day": 5}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \'Tuesday\' = 2, \'Wednesday\' = 3, \'Thursday\' = 4, \'Friday\' = 5, \'Saturday\' = 6)') = 'Friday' -## JSONExtractKeysAndValues(json\[, indices\_or\_keys…\], Value\_type) {#jsonextractkeysandvaluesjson-indices-or-keys-value-type} +## JSONExtractKeysAndValues(json\[, indices_or_keys…\], Value_type) {#jsonextractkeysandvaluesjson-indices-or-keys-value-type} 从JSON中解析键值对,其中值是给定的ClickHouse数据类型。 @@ -161,7 +161,7 @@ SELECT JSONExtractKeysAndValues('{"x": {"a": 5, "b": 7, "c": 11}}', 'x', 'Int8') = [('a',5),('b',7),('c',11)]; -## JSONExtractRaw(json\[, indices\_or\_keys\]…) {#jsonextractrawjson-indices-or-keys} +## JSONExtractRaw(json\[, indices_or_keys\]…) {#jsonextractrawjson-indices-or-keys} 返回JSON的部分。 diff --git a/docs/zh/sql-reference/functions/other-functions.md b/docs/zh/sql-reference/functions/other-functions.md index 16ce90e6ced..b17a5e89332 100644 --- a/docs/zh/sql-reference/functions/other-functions.md +++ b/docs/zh/sql-reference/functions/other-functions.md @@ -201,7 +201,7 @@ ORDER BY h ASC 对于相同的字母(T或U),如果数值类型,那么它们不可不完全匹配的,只需要具备共同的类型即可。 例如,第一个参数是Int64类型,第二个参数是Array(UInt16)类型。 -如果’x’值等于’array\_from’数组中的一个元素,它将从’array\_to’数组返回一个对应的元素(下标相同)。否则,它返回’default’。如果’array\_from’匹配到了多个元素,则返回第一个匹配的元素。 +如果’x’值等于’array_from’数组中的一个元素,它将从’array_to’数组返回一个对应的元素(下标相同)。否则,它返回’default’。如果’array_from’匹配到了多个元素,则返回第一个匹配的元素。 示例: @@ -224,7 +224,7 @@ ORDER BY c DESC 1. `transform(x, array_from, array_to)` 与第一种不同在于省略了’default’参数。 -如果’x’值等于’array\_from’数组中的一个元素,它将从’array\_to’数组返回相应的元素(下标相同)。 否则,它返回’x’。 +如果’x’值等于’array_from’数组中的一个元素,它将从’array_to’数组返回相应的元素(下标相同)。 否则,它返回’x’。 类型约束: @@ -521,11 +521,11 @@ FROM 例如,获取聚合函数的状态(示例runningAccumulate(uniqState(UserID))),对于数据块的每一行,返回所有先前行和当前行的状态合并后的聚合函数的结果。 因此,函数的结果取决于分区中数据块的顺序以及数据块中行的顺序。 -## joinGet(‘join\_storage\_table\_name’, ‘get\_column’,join\_key) {#joingetjoin-storage-table-name-get-column-join-key} +## joinGet(‘join_storage_table_name’, ‘get_column’,join_key) {#joingetjoin-storage-table-name-get-column-join-key} 使用指定的连接键从Join类型引擎的表中获取数据。 -## modelEvaluate(model\_name, …) {#function-modelevaluate} +## modelEvaluate(model_name, …) {#function-modelevaluate} 使用外部模型计算。 接受模型的名称以及模型的参数。返回Float64类型的值。 diff --git a/docs/zh/sql-reference/functions/string-functions.md b/docs/zh/sql-reference/functions/string-functions.md index 70b361ebf1b..0dbcc031d56 100644 --- a/docs/zh/sql-reference/functions/string-functions.md +++ b/docs/zh/sql-reference/functions/string-functions.md @@ -24,7 +24,7 @@ 假定字符串以UTF-8编码组成的文本,返回此字符串的Unicode字符长度。如果传入的字符串不是UTF-8编码,则函数可能返回一个预期外的值(不会抛出异常)。 结果类型是UInt64。 -## char\_length,CHAR\_LENGTH {#char-length-char-length} +## char_length,CHAR_LENGTH {#char-length-char-length} 假定字符串以UTF-8编码组成的文本,返回此字符串的Unicode字符长度。如果传入的字符串不是UTF-8编码,则函数可能返回一个预期外的值(不会抛出异常)。 结果类型是UInt64。 @@ -68,7 +68,7 @@ 参数: -- input\_string — 任何一个[字符串](../../sql-reference/functions/string-functions.md)类型的对象。 +- input_string — 任何一个[字符串](../../sql-reference/functions/string-functions.md)类型的对象。 返回值: 有效的UTF-8字符串。 diff --git a/docs/zh/sql-reference/functions/string-replace-functions.md b/docs/zh/sql-reference/functions/string-replace-functions.md index 67346f5c156..6e0745ba5b1 100644 --- a/docs/zh/sql-reference/functions/string-replace-functions.md +++ b/docs/zh/sql-reference/functions/string-replace-functions.md @@ -74,6 +74,6 @@ SELECT replaceRegexpAll('Hello, World!', '^', 'here: ') AS res 该函数用于在字符串中的某些预定义字符之前添加反斜杠。 预定义字符:‘0’,‘\\’,‘\|’,‘(’,‘)’,‘^’,‘$’,‘。’,‘\[’,‘\]’,‘?’,‘\*’,‘+’,‘{’,‘:’,’ - ’。 这个实现与re2 :: RE2 :: QuoteMeta略有不同。它以\\0而不是00转义零字节,它只转义所需的字符。 -有关详细信息,请参阅链接:\[RE2\](https://github.com/google/re2/blob/master/re2/re2.cc\#L473) +有关详细信息,请参阅链接:\[RE2\](https://github.com/google/re2/blob/master/re2/re2.cc#L473) [来源文章](https://clickhouse.tech/docs/en/query_language/functions/string_replace_functions/) diff --git a/docs/zh/sql-reference/functions/type-conversion-functions.md b/docs/zh/sql-reference/functions/type-conversion-functions.md index 6e3ed9afb78..4c94b2b2f9a 100644 --- a/docs/zh/sql-reference/functions/type-conversion-functions.md +++ b/docs/zh/sql-reference/functions/type-conversion-functions.md @@ -241,7 +241,7 @@ toDate/toDateTime函数的日期和日期时间格式定义如下: YYYY-MM-DD YYYY-MM-DD hh:mm:ss -例外的是,如果将UInt32、Int32、UInt64或Int64类型的数值转换为Date类型,并且其对应的值大于等于65536,则该数值将被解析成unix时间戳(而不是对应的天数)。这意味着允许写入’toDate(unix\_timestamp)‘这种常见情况,否则这将是错误的,并且需要便携更加繁琐的’toDate(toDateTime(unix\_timestamp))’。 +例外的是,如果将UInt32、Int32、UInt64或Int64类型的数值转换为Date类型,并且其对应的值大于等于65536,则该数值将被解析成unix时间戳(而不是对应的天数)。这意味着允许写入’toDate(unix_timestamp)‘这种常见情况,否则这将是错误的,并且需要便携更加繁琐的’toDate(toDateTime(unix_timestamp))’。 Date与DateTime之间的转换以更为自然的方式进行:通过添加空的time或删除time。 diff --git a/docs/zh/sql-reference/functions/url-functions.md b/docs/zh/sql-reference/functions/url-functions.md index f967ccd4964..a7b91c773c8 100644 --- a/docs/zh/sql-reference/functions/url-functions.md +++ b/docs/zh/sql-reference/functions/url-functions.md @@ -38,19 +38,19 @@ ### pathFull {#pathfull} -与上面相同,但包括请求参数和fragment。例如:/top/news.html?page=2\#comments +与上面相同,但包括请求参数和fragment。例如:/top/news.html?page=2#comments ### 查询字符串 {#querystring} -返回请求参数。例如:page=1&lr=213。请求参数不包含问号已经\# 以及\# 之后所有的内容。 +返回请求参数。例如:page=1&lr=213。请求参数不包含问号已经# 以及# 之后所有的内容。 ### 片段 {#fragment} -返回URL的fragment标识。fragment不包含\#。 +返回URL的fragment标识。fragment不包含#。 ### querystring andfragment {#querystringandfragment} -返回请求参数和fragment标识。例如:page=1\#29390。 +返回请求参数和fragment标识。例如:page=1#29390。 ### extractURLParameter(URL,name) {#extracturlparameterurl-name} @@ -105,11 +105,11 @@ SELECT decodeURLComponent('http://127.0.0.1:8123/?query=SELECT%201%3B') AS Decod ### cutFragment {#cutfragment} -删除fragment标识。\#同样也会被删除。 +删除fragment标识。#同样也会被删除。 ### cutquerystring andfragment {#cutquerystringandfragment} -删除请求参数以及fragment标识。问号以及\#也会被删除。 +删除请求参数以及fragment标识。问号以及#也会被删除。 ### cutURLParameter(URL,name) {#cuturlparameterurl-name} diff --git a/docs/zh/sql-reference/functions/ym-dict-functions.md b/docs/zh/sql-reference/functions/ym-dict-functions.md index c04405d81b4..429105084dd 100644 --- a/docs/zh/sql-reference/functions/ym-dict-functions.md +++ b/docs/zh/sql-reference/functions/ym-dict-functions.md @@ -10,12 +10,12 @@ ClickHouse支持同时使用多个备选地理基(区域层次结构),以 该 ‘clickhouse-server’ config指定具有区域层次结构的文件::`/opt/geo/regions_hierarchy.txt` -除了这个文件,它还搜索附近有\_符号和任何后缀附加到名称(文件扩展名之前)的文件。 +除了这个文件,它还搜索附近有_符号和任何后缀附加到名称(文件扩展名之前)的文件。 例如,它还会找到该文件 `/opt/geo/regions_hierarchy_ua.txt`,如果存在。 `ua` 被称为字典键。 对于没有后缀的字典,键是空字符串。 -所有字典都在运行时重新加载(每隔一定数量的秒重新加载一次,如builtin\_dictionaries\_reload\_interval config参数中定义,或默认情况下每小时一次)。 但是,可用字典列表在服务器启动时定义一次。 +所有字典都在运行时重新加载(每隔一定数量的秒重新加载一次,如builtin_dictionaries_reload_interval config参数中定义,或默认情况下每小时一次)。 但是,可用字典列表在服务器启动时定义一次。 All functions for working with regions have an optional argument at the end – the dictionary key. It is referred to as the geobase. 示例: diff --git a/docs/zh/sql-reference/operators/in.md b/docs/zh/sql-reference/operators/in.md index bcd3ca1fa18..a16b75fe555 100644 --- a/docs/zh/sql-reference/operators/in.md +++ b/docs/zh/sql-reference/operators/in.md @@ -69,7 +69,7 @@ IN子句中的子查询始终只在单个服务器上运行一次。 没有依 ## 空处理 {#in-null-processing} -在请求处理过程中, `IN` 运算符假定运算的结果 [NULL](../../sql-reference/syntax.md#null-literal) 总是等于 `0`,无论是否 `NULL` 位于操作员的右侧或左侧。 `NULL` 值不包含在任何数据集中,彼此不对应,并且在以下情况下无法进行比较 [transform\_null\_in=0](../../operations/settings/settings.md#transform_null_in). +在请求处理过程中, `IN` 运算符假定运算的结果 [NULL](../../sql-reference/syntax.md#null-literal) 总是等于 `0`,无论是否 `NULL` 位于操作员的右侧或左侧。 `NULL` 值不包含在任何数据集中,彼此不对应,并且在以下情况下无法进行比较 [transform_null_in=0](../../operations/settings/settings.md#transform_null_in). 下面是一个例子 `t_null` 表: @@ -117,9 +117,9 @@ FROM t_null 在使用子查询时要小心 `IN` / `JOIN` 用于分布式查询处理的子句。 -让我们来看看一些例子。 假设集群中的每个服务器都有一个正常的 **local\_table**. 每个服务器还具有 **distributed\_table** 表与 **分布** 类型,它查看群集中的所有服务器。 +让我们来看看一些例子。 假设集群中的每个服务器都有一个正常的 **local_table**. 每个服务器还具有 **distributed_table** 表与 **分布** 类型,它查看群集中的所有服务器。 -对于查询 **distributed\_table**,查询将被发送到所有远程服务器,并使用以下命令在其上运行 **local\_table**. +对于查询 **distributed_table**,查询将被发送到所有远程服务器,并使用以下命令在其上运行 **local_table**. 例如,查询 @@ -153,7 +153,7 @@ SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID IN (SEL 如果您已经为此情况做好准备,并且已经将数据分散到群集服务器上,以便单个用户Id的数据完全驻留在单个服务器上,则这将正常和最佳地工作。 在这种情况下,所有必要的数据将在每台服务器上本地提供。 否则,结果将是不准确的。 我们将查询的这种变体称为 “local IN”. -若要更正数据在群集服务器上随机传播时查询的工作方式,可以指定 **distributed\_table** 在子查询中。 查询如下所示: +若要更正数据在群集服务器上随机传播时查询的工作方式,可以指定 **distributed_table** 在子查询中。 查询如下所示: ``` sql SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM distributed_table WHERE CounterID = 34) diff --git a/docs/zh/sql-reference/statements/alter.md b/docs/zh/sql-reference/statements/alter.md index d9e9bebf339..446feac96ce 100644 --- a/docs/zh/sql-reference/statements/alter.md +++ b/docs/zh/sql-reference/statements/alter.md @@ -426,7 +426,7 @@ ALTER TABLE hits MOVE PARTITION '2019-09-01' TO DISK 'fast_ssd' - `system.parts`表 `partition`列的某个值,例如, `ALTER TABLE visits DETACH PARTITION 201901` - 表的列表达式。支持常量及常量表达式。例如, `ALTER TABLE visits DETACH PARTITION toYYYYMM(toDate('2019-01-25'))` - 使用分区ID。分区ID是字符串变量(可能的话有较好的可读性),在文件系统和ZooKeeper中作为分区名称。分区ID必须配置在 `PARTITION ID`中,用单引号包含,例如, `ALTER TABLE visits DETACH PARTITION ID '201901'` -- 在 [ALTER ATTACH PART](#alter_attach-partition) 和 [DROP DETACHED PART](#alter_drop-detached) 操作中,要配置块的名称,使用 [system.detached\_parts](../../operations/system-tables/detached_parts.md#system_tables-detached_parts)表中 `name`列的字符串值,例如: `ALTER TABLE visits ATTACH PART '201901_1_1_0'` +- 在 [ALTER ATTACH PART](#alter_attach-partition) 和 [DROP DETACHED PART](#alter_drop-detached) 操作中,要配置块的名称,使用 [system.detached_parts](../../operations/system-tables/detached_parts.md#system_tables-detached_parts)表中 `name`列的字符串值,例如: `ALTER TABLE visits ATTACH PART '201901_1_1_0'` 设置分区时,引号使用要看分区表达式的类型。例如,对于 `String`类型,需要设置用引号(`'`)包含的名称。对于 `Date` 和 `Int*`引号就不需要了。 diff --git a/docs/zh/sql-reference/statements/insert-into.md b/docs/zh/sql-reference/statements/insert-into.md index e8b5edfdb37..b71e84ef0f6 100644 --- a/docs/zh/sql-reference/statements/insert-into.md +++ b/docs/zh/sql-reference/statements/insert-into.md @@ -13,7 +13,7 @@ INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), . - 如果存在`DEFAULT`表达式,根据`DEFAULT`表达式计算被填充的值。 - 如果没有定义`DEFAULT`表达式,则填充零或空字符串。 -如果 [strict\_insert\_defaults=1](../../operations/settings/settings.md),你必须在查询中列出所有没有定义`DEFAULT`表达式的列。 +如果 [strict_insert_defaults=1](../../operations/settings/settings.md),你必须在查询中列出所有没有定义`DEFAULT`表达式的列。 数据可以以ClickHouse支持的任何 [输入输出格式](../../interfaces/formats.md#formats) 传递给INSERT。格式的名称必须显示的指定在查询中: diff --git a/docs/zh/sql-reference/statements/misc.md b/docs/zh/sql-reference/statements/misc.md index 5320fbd0869..fd3eea9796e 100644 --- a/docs/zh/sql-reference/statements/misc.md +++ b/docs/zh/sql-reference/statements/misc.md @@ -57,7 +57,7 @@ CHECK TABLE [db.]name 如果表已损坏,则可以将未损坏的数据复制到另一个表。 要做到这一点: 1. 创建具有与损坏的表相同结构的新表。 要执行此操作,请执行查询 `CREATE TABLE AS `. -2. 设置 [max\_threads](../../operations/settings/settings.md#settings-max_threads) 值为1以在单个线程中处理下一个查询。 要执行此操作,请运行查询 `SET max_threads = 1`. +2. 设置 [max_threads](../../operations/settings/settings.md#settings-max_threads) 值为1以在单个线程中处理下一个查询。 要执行此操作,请运行查询 `SET max_threads = 1`. 3. 执行查询 `INSERT INTO SELECT * FROM `. 此请求将未损坏的数据从损坏的表复制到另一个表。 只有损坏部分之前的数据才会被复制。 4. 重新启动 `clickhouse-client` 要重置 `max_threads` 价值。 @@ -253,7 +253,7 @@ OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition | PARTITION I 当 `OPTIMIZE` 与使用 [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/replication.md) 表引擎的家族,ClickHouse创建合并任务,并等待在所有节点上执行(如果 `replication_alter_partitions_sync` 设置已启用)。 -- 如果 `OPTIMIZE` 出于任何原因不执行合并,它不通知客户端。 要启用通知,请使用 [optimize\_throw\_if\_noop](../../operations/settings/settings.md#setting-optimize_throw_if_noop) 设置。 +- 如果 `OPTIMIZE` 出于任何原因不执行合并,它不通知客户端。 要启用通知,请使用 [optimize_throw_if_noop](../../operations/settings/settings.md#setting-optimize_throw_if_noop) 设置。 - 如果您指定 `PARTITION`,仅优化指定的分区。 [如何设置分区表达式](alter.md#alter-how-to-specify-part-expr). - 如果您指定 `FINAL`,即使所有数据已经在一个部分中,也会执行优化。 - 如果您指定 `DEDUPLICATE`,然后完全相同的行将被重复数据删除(所有列进行比较),这仅适用于MergeTree引擎。 diff --git a/docs/zh/sql-reference/statements/select/group-by.md b/docs/zh/sql-reference/statements/select/group-by.md index ecb2f4683fe..f4ead11036b 100644 --- a/docs/zh/sql-reference/statements/select/group-by.md +++ b/docs/zh/sql-reference/statements/select/group-by.md @@ -66,9 +66,9 @@ toc_title: GROUP BY `after_having_exclusive` – Don't include rows that didn't pass through `max_rows_to_group_by`. 换句话说, ‘totals’ 将有少于或相同数量的行,因为它会 `max_rows_to_group_by` 被省略。 -`after_having_inclusive` – Include all the rows that didn't pass through ‘max\_rows\_to\_group\_by’ 在 ‘totals’. 换句话说, ‘totals’ 将有多个或相同数量的行,因为它会 `max_rows_to_group_by` 被省略。 +`after_having_inclusive` – Include all the rows that didn't pass through ‘max_rows_to_group_by’ 在 ‘totals’. 换句话说, ‘totals’ 将有多个或相同数量的行,因为它会 `max_rows_to_group_by` 被省略。 -`after_having_auto` – Count the number of rows that passed through HAVING. If it is more than a certain amount (by default, 50%), include all the rows that didn't pass through ‘max\_rows\_to\_group\_by’ 在 ‘totals’. 否则,不包括它们。 +`after_having_auto` – Count the number of rows that passed through HAVING. If it is more than a certain amount (by default, 50%), include all the rows that didn't pass through ‘max_rows_to_group_by’ 在 ‘totals’. 否则,不包括它们。 `totals_auto_threshold` – By default, 0.5. The coefficient for `after_having_auto`. @@ -116,11 +116,11 @@ GROUP BY domain ### 在外部存储器中分组 {#select-group-by-in-external-memory} 您可以启用将临时数据转储到磁盘以限制内存使用期间 `GROUP BY`. -该 [max\_bytes\_before\_external\_group\_by](../../../operations/settings/settings.md#settings-max_bytes_before_external_group_by) 设置确定倾销的阈值RAM消耗 `GROUP BY` 临时数据到文件系统。 如果设置为0(默认值),它将被禁用。 +该 [max_bytes_before_external_group_by](../../../operations/settings/settings.md#settings-max_bytes_before_external_group_by) 设置确定倾销的阈值RAM消耗 `GROUP BY` 临时数据到文件系统。 如果设置为0(默认值),它将被禁用。 使用时 `max_bytes_before_external_group_by`,我们建议您设置 `max_memory_usage` 大约两倍高。 这是必要的,因为聚合有两个阶段:读取数据和形成中间数据(1)和合并中间数据(2)。 将数据转储到文件系统只能在阶段1中发生。 如果未转储临时数据,则阶段2可能需要与阶段1相同的内存量。 -例如,如果 [max\_memory\_usage](../../../operations/settings/settings.md#settings_max_memory_usage) 设置为10000000000,你想使用外部聚合,这是有意义的设置 `max_bytes_before_external_group_by` 到10000000000,和 `max_memory_usage` 到20000000000。 当触发外部聚合(如果至少有一个临时数据转储)时,RAM的最大消耗仅略高于 `max_bytes_before_external_group_by`. +例如,如果 [max_memory_usage](../../../operations/settings/settings.md#settings_max_memory_usage) 设置为10000000000,你想使用外部聚合,这是有意义的设置 `max_bytes_before_external_group_by` 到10000000000,和 `max_memory_usage` 到20000000000。 当触发外部聚合(如果至少有一个临时数据转储)时,RAM的最大消耗仅略高于 `max_bytes_before_external_group_by`. 通过分布式查询处理,在远程服务器上执行外部聚合。 为了使请求者服务器只使用少量的RAM,设置 `distributed_aggregation_memory_efficient` 到1。 diff --git a/docs/zh/sql-reference/statements/select/join.md b/docs/zh/sql-reference/statements/select/join.md index 48395933665..2976484e09a 100644 --- a/docs/zh/sql-reference/statements/select/join.md +++ b/docs/zh/sql-reference/statements/select/join.md @@ -39,7 +39,7 @@ ClickHouse中提供的其他联接类型: ## 严格 {#join-settings} !!! note "注" - 可以使用以下方式复盖默认的严格性值 [join\_default\_strictness](../../../operations/settings/settings.md#settings-join_default_strictness) 设置。 + 可以使用以下方式复盖默认的严格性值 [join_default_strictness](../../../operations/settings/settings.md#settings-join_default_strictness) 设置。 Also the behavior of ClickHouse server for `ANY JOIN` operations depends on the [any_join_distinct_right_table_keys](../../../operations/settings/settings.md#any_join_distinct_right_table_keys) setting. @@ -106,7 +106,7 @@ USING (equi_column1, ... equi_columnN, asof_column) ### 处理空单元格或空单元格 {#processing-of-empty-or-null-cells} -在连接表时,可能会出现空单元格。 设置 [join\_use\_nulls](../../../operations/settings/settings.md#join_use_nulls) 定义ClickHouse如何填充这些单元格。 +在连接表时,可能会出现空单元格。 设置 [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) 定义ClickHouse如何填充这些单元格。 如果 `JOIN` 键是 [可为空](../../../sql-reference/data-types/nullable.md) 字段,其中至少有一个键具有值的行 [NULL](../../../sql-reference/syntax.md#null-literal) 没有加入。 @@ -143,10 +143,10 @@ USING (equi_column1, ... equi_columnN, asof_column) 如果需要限制联接操作内存消耗,请使用以下设置: -- [max\_rows\_in\_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) — Limits number of rows in the hash table. -- [max\_bytes\_in\_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) — Limits size of the hash table. +- [max_rows_in_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) — Limits number of rows in the hash table. +- [max_bytes_in_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) — Limits size of the hash table. -当任何这些限制达到,ClickHouse作为 [join\_overflow\_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) 设置指示。 +当任何这些限制达到,ClickHouse作为 [join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) 设置指示。 ## 例子 {#examples} diff --git a/docs/zh/sql-reference/statements/system.md b/docs/zh/sql-reference/statements/system.md index c46d1745125..9952f383236 100644 --- a/docs/zh/sql-reference/statements/system.md +++ b/docs/zh/sql-reference/statements/system.md @@ -45,12 +45,12 @@ toc_title: SYSTEM ## RELOAD DICTIONARIES {#query_language-system-reload-dictionaries} 重载已经被成功加载过的所有字典。 -默认情况下,字典是延时加载的( [dictionaries\_lazy\_load](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load)),不是在服务启动时自动加载,而是在第一次使用dictGet函数或通过 `SELECT from tables with ENGINE = Dictionary` 进行访问时被初始化。这个命令 `SYSTEM RELOAD DICTIONARIES` 就是针对这类表进行重新加载的。 +默认情况下,字典是延时加载的( [dictionaries_lazy_load](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load)),不是在服务启动时自动加载,而是在第一次使用dictGet函数或通过 `SELECT from tables with ENGINE = Dictionary` 进行访问时被初始化。这个命令 `SYSTEM RELOAD DICTIONARIES` 就是针对这类表进行重新加载的。 -## RELOAD DICTIONARY Dictionary\_name {#query_language-system-reload-dictionary} +## RELOAD DICTIONARY Dictionary_name {#query_language-system-reload-dictionary} -完全重新加载指定字典 `dictionary_name`,不管该字典的状态如何(LOADED / NOT\_LOADED / FAILED)。不管字典的更新结果如何,总是返回 `OK.` +完全重新加载指定字典 `dictionary_name`,不管该字典的状态如何(LOADED / NOT_LOADED / FAILED)。不管字典的更新结果如何,总是返回 `OK.` 字典的状态可以通过查询 `system.dictionaries`表来检查。 @@ -61,7 +61,7 @@ SELECT name, status FROM system.dictionaries; ## DROP DNS CACHE {#query_language-system-drop-dns-cache} 重置CH的dns缓存。有时候(对于旧的ClickHouse版本)当某些底层环境发生变化时(修改其它Clickhouse服务器的ip或字典所在服务器的ip),需要使用该命令。 -更多自动化的缓存管理相关信息,参见disable\_internal\_dns\_cache, dns\_cache\_update\_period这些参数。 +更多自动化的缓存管理相关信息,参见disable_internal_dns_cache, dns_cache_update_period这些参数。 ## DROP MARK CACHE {#query_language-system-drop-mark-cache} @@ -89,7 +89,7 @@ SYSTEM DROP REPLICA 'replica_name' FROM ZKPATH '/path/to/table/in/zk'; ## DROP UNCOMPRESSED CACHE {#query_language-system-drop-uncompressed-cache} 重置未压缩数据的缓存。用于ClickHouse开发和性能测试。 -管理未压缩数据缓存的参数,使用以下的服务器级别设置 [uncompressed\_cache\_size](../../operations/server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size)以及 `query/user/profile`级别设置 [use\_uncompressed\_cache](../../operations/settings/settings.md#setting-use_uncompressed_cache) +管理未压缩数据缓存的参数,使用以下的服务器级别设置 [uncompressed_cache_size](../../operations/server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size)以及 `query/user/profile`级别设置 [use_uncompressed_cache](../../operations/settings/settings.md#setting-use_uncompressed_cache) ## DROP COMPILED EXPRESSION CACHE {#query_language-system-drop-compiled-expression-cache} @@ -99,7 +99,7 @@ SYSTEM DROP REPLICA 'replica_name' FROM ZKPATH '/path/to/table/in/zk'; ## FLUSH LOGS {#query_language-system-flush_logs} -将日志信息缓冲数据刷入系统表(例如system.query\_log)。调试时允许等待不超过7.5秒。当信息队列为空时,会创建系统表。 +将日志信息缓冲数据刷入系统表(例如system.query_log)。调试时允许等待不超过7.5秒。当信息队列为空时,会创建系统表。 ## RELOAD CONFIG {#query_language-system-reload-config} diff --git a/docs/zh/sql-reference/syntax.md b/docs/zh/sql-reference/syntax.md index 78cbd5660ac..8c331db1139 100644 --- a/docs/zh/sql-reference/syntax.md +++ b/docs/zh/sql-reference/syntax.md @@ -12,9 +12,9 @@ CH有2类解析器:完整SQL解析器(递归式解析器),以及数据 INSERT INTO t VALUES (1, 'Hello, world'), (2, 'abc'), (3, 'def') ``` -含`INSERT INTO t VALUES` 的部分由完整SQL解析器处理,包含数据的部分 `(1, 'Hello, world'), (2, 'abc'), (3, 'def')` 交给快速流式解析器解析。通过设置参数 [input\_format\_values\_interpret\_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions),你也可以对数据部分开启完整SQL解析器。当 `input_format_values_interpret_expressions = 1` 时,CH优先采用快速流式解析器来解析数据。如果失败,CH再尝试用完整SQL解析器来处理,就像处理SQL [expression](#syntax-expressions) 一样。 +含`INSERT INTO t VALUES` 的部分由完整SQL解析器处理,包含数据的部分 `(1, 'Hello, world'), (2, 'abc'), (3, 'def')` 交给快速流式解析器解析。通过设置参数 [input_format_values_interpret_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions),你也可以对数据部分开启完整SQL解析器。当 `input_format_values_interpret_expressions = 1` 时,CH优先采用快速流式解析器来解析数据。如果失败,CH再尝试用完整SQL解析器来处理,就像处理SQL [expression](#syntax-expressions) 一样。 -数据可以采用任何格式。当CH接受到请求时,服务端先在内存中计算不超过 [max\_query\_size](../operations/settings/settings.md#settings-max_query_size) 字节的请求数据(默认1 mb),然后剩下部分交给快速流式解析器。 +数据可以采用任何格式。当CH接受到请求时,服务端先在内存中计算不超过 [max_query_size](../operations/settings/settings.md#settings-max_query_size) 字节的请求数据(默认1 mb),然后剩下部分交给快速流式解析器。 这将避免在处理大型的 `INSERT`语句时出现问题。 diff --git a/docs/zh/sql-reference/table-functions/file.md b/docs/zh/sql-reference/table-functions/file.md index 71f84d65f21..4d694cb6729 100644 --- a/docs/zh/sql-reference/table-functions/file.md +++ b/docs/zh/sql-reference/table-functions/file.md @@ -15,7 +15,7 @@ file(path, format, structure) **输入参数** -- `path` — The relative path to the file from [user\_files\_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). 只读模式下的globs后的文件支持路径: `*`, `?`, `{abc,def}` 和 `{N..M}` 哪里 `N`, `M` — numbers, \``'abc', 'def'` — strings. +- `path` — The relative path to the file from [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). 只读模式下的globs后的文件支持路径: `*`, `?`, `{abc,def}` 和 `{N..M}` 哪里 `N`, `M` — numbers, \``'abc', 'def'` — strings. - `format` — The [格式](../../interfaces/formats.md#formats) 的文件。 - `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. @@ -72,12 +72,12 @@ SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 U 1. 假设我们有几个具有以下相对路径的文件: -- ‘some\_dir/some\_file\_1’ -- ‘some\_dir/some\_file\_2’ -- ‘some\_dir/some\_file\_3’ -- ‘another\_dir/some\_file\_1’ -- ‘another\_dir/some\_file\_2’ -- ‘another\_dir/some\_file\_3’ +- ‘some_dir/some_file_1’ +- ‘some_dir/some_file_2’ +- ‘some_dir/some_file_3’ +- ‘another_dir/some_file_1’ +- ‘another_dir/some_file_2’ +- ‘another_dir/some_file_3’ 1. 查询这些文件中的行数: diff --git a/docs/zh/sql-reference/table-functions/hdfs.md b/docs/zh/sql-reference/table-functions/hdfs.md index 4ea4e71f8fc..112c88450e2 100644 --- a/docs/zh/sql-reference/table-functions/hdfs.md +++ b/docs/zh/sql-reference/table-functions/hdfs.md @@ -55,12 +55,12 @@ LIMIT 2 1. 假设我们在HDFS上有几个具有以下Uri的文件: -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/some\_dir/some\_file\_3’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_1’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_2’ -- ‘hdfs://hdfs1:9000/another\_dir/some\_file\_3’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_3’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_3’ 1. 查询这些文件中的行数: diff --git a/docs/zh/sql-reference/table-functions/index.md b/docs/zh/sql-reference/table-functions/index.md index 1fa985a529a..d9eadb9c592 100644 --- a/docs/zh/sql-reference/table-functions/index.md +++ b/docs/zh/sql-reference/table-functions/index.md @@ -16,12 +16,12 @@ toc_title: "\u5BFC\u8A00" The method for creating a temporary table that is available only in the current query. The table is deleted when the query finishes. -- [创建表为\](../statements/create.md#create-table-query) 查询。 +- [创建表为\](../statements/create.md#create-table-query) 查询。 It's one of the methods of creating a table. !!! warning "警告" - 你不能使用表函数,如果 [allow\_ddl](../../operations/settings/permissions-for-queries.md#settings_allow_ddl) 设置被禁用。 + 你不能使用表函数,如果 [allow_ddl](../../operations/settings/permissions-for-queries.md#settings_allow_ddl) 设置被禁用。 | 功能 | 产品描述 | |--------------------|--------------------------------------------------------------------------------------------------------| diff --git a/docs/zh/sql-reference/table-functions/remote.md b/docs/zh/sql-reference/table-functions/remote.md index a7fa228cbbd..b7bd494609b 100644 --- a/docs/zh/sql-reference/table-functions/remote.md +++ b/docs/zh/sql-reference/table-functions/remote.md @@ -49,7 +49,7 @@ example01-{01..02}-1 如果您有多对大括号,它会生成相应集合的直接乘积。 -大括号中的地址和部分地址可以用管道符号(\|)分隔。 在这种情况下,相应的地址集被解释为副本,并且查询将被发送到第一个正常副本。 但是,副本将按照当前[load\_balancing](../../operations/settings/settings.md)设置的顺序进行迭代。 +大括号中的地址和部分地址可以用管道符号(\|)分隔。 在这种情况下,相应的地址集被解释为副本,并且查询将被发送到第一个正常副本。 但是,副本将按照当前[load_balancing](../../operations/settings/settings.md)设置的顺序进行迭代。 示例: @@ -73,6 +73,6 @@ example01-{01..02}-{1|2} 如果未指定用户, 将会使用`default`。 如果未指定密码,则使用空密码。 -`remoteSecure` - 与 `remote` 相同,但是会使用加密链接。默认端口为配置文件中的[tcp\_port\_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure),或9440。 +`remoteSecure` - 与 `remote` 相同,但是会使用加密链接。默认端口为配置文件中的[tcp_port_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure),或9440。 [原始文章](https://clickhouse.tech/docs/en/query_language/table_functions/remote/) diff --git a/docs/zh/whats-new/security-changelog.md b/docs/zh/whats-new/security-changelog.md index e35d6a7c632..f21158c7aed 100644 --- a/docs/zh/whats-new/security-changelog.md +++ b/docs/zh/whats-new/security-changelog.md @@ -18,7 +18,7 @@ unixODBC允许从文件系统加载任意共享对象,从而导致«远程执 ### CVE-2018-14668 {#cve-2018-14668} -远程表函数功能允许在 «user», «password» 及 «default\_database» 字段中使用任意符号,从而导致跨协议请求伪造攻击。 +远程表函数功能允许在 «user», «password» 及 «default_database» 字段中使用任意符号,从而导致跨协议请求伪造攻击。 来源:Yandex信息安全团队的Andrey Krasichkov From b4701c948875d6dd790575f587181789d9e5f610 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 13 Oct 2020 23:02:51 +0300 Subject: [PATCH 36/42] Update adopters.md --- docs/en/introduction/adopters.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 0cbfdcb7d81..e30ff0bbdfc 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -60,6 +60,7 @@ toc_title: Adopters | LifeStreet | Ad network | Main product | 75 servers (3 replicas) | 5.27 PiB | [Blog post in Russian, February 2017](https://habr.com/en/post/322620/) | | Mail.ru Cloud Solutions | Cloud services | Main product | — | — | [Article in Russian](https://mcs.mail.ru/help/db-create/clickhouse#) | | Marilyn | Advertising | Statistics | — | — | [Talk in Russian, June 2017](https://www.youtube.com/watch?v=iXlIgx2khwc) | +| Mello | Marketing | Analytics | 1 server | — | [Article, Oct 2020](https://vc.ru/marketing/166180-razrabotka-tipovogo-otcheta-skvoznoy-analitiki) | | MessageBird | Telecommunications | Statistics | — | — | [Slides in English, November 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup20/messagebird.pdf) | | MindsDB | Machine Learning | Main Product | — | — | [Official Website](https://www.mindsdb.com/blog/machine-learning-models-as-tables-in-ch) | | MGID | Ad network | Web-analytics | — | — | [Blog post in Russian, April 2020](http://gs-studio.com/news-about-it/32777----clickhouse---c) | From 628e2b472c68277e8fecf0b58789c145ce80b0a1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 14 Oct 2020 01:21:51 +0300 Subject: [PATCH 37/42] Fix flackiness of LIVE VIEW tests by increasing timeout for HTTPConnection Trace from [1]: 2020-10-13 01:37:06 00966_live_view_watch_events_http: [ FAIL ] - return code 1 2020-10-13 01:37:06 Exception in thread Thread-2: 2020-10-13 01:37:06 Traceback (most recent call last): 2020-10-13 01:37:06 File "/usr/lib/python3.7/threading.py", line 926, in _bootstrap_inner 2020-10-13 01:37:06 self.run() 2020-10-13 01:37:06 File "/usr/lib/python3.7/threading.py", line 870, in run 2020-10-13 01:37:06 self._target(*self._args, **self._kwargs) 2020-10-13 01:37:06 File "/usr/share/clickhouse-test/queries/0_stateless/helpers/httpexpect.py", line 48, in reader 2020-10-13 01:37:06 data = response.read(1).decode() 2020-10-13 01:37:06 File "/usr/lib/python3.7/http/client.py", line 457, in read 2020-10-13 01:37:06 n = self.readinto(b) 2020-10-13 01:37:06 File "/usr/lib/python3.7/http/client.py", line 491, in readinto 2020-10-13 01:37:06 return self._readinto_chunked(b) 2020-10-13 01:37:06 File "/usr/lib/python3.7/http/client.py", line 586, in _readinto_chunked 2020-10-13 01:37:06 chunk_left = self._get_chunk_left() 2020-10-13 01:37:06 File "/usr/lib/python3.7/http/client.py", line 554, in _get_chunk_left 2020-10-13 01:37:06 chunk_left = self._read_next_chunk_size() 2020-10-13 01:37:06 File "/usr/lib/python3.7/http/client.py", line 514, in _read_next_chunk_size 2020-10-13 01:37:06 line = self.fp.readline(_MAXLINE + 1) 2020-10-13 01:37:06 File "/usr/lib/python3.7/socket.py", line 589, in readinto 2020-10-13 01:37:06 return self._sock.recv_into(b) 2020-10-13 01:37:06 socket.timeout: timed out [1]: https://clickhouse-test-reports.s3.yandex.net/15879/f7bf94b5d361d0a54e85673c14e630cfdf8dbe01/functional_stateless_tests_(release,_polymorphic_parts_enabled)/test_run.txt.out.log --- tests/queries/0_stateless/helpers/httpclient.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/helpers/httpclient.py b/tests/queries/0_stateless/helpers/httpclient.py index 6fb6edff142..adbfbc7d287 100644 --- a/tests/queries/0_stateless/helpers/httpclient.py +++ b/tests/queries/0_stateless/helpers/httpclient.py @@ -8,7 +8,7 @@ sys.path.insert(0, os.path.join(CURDIR)) import httpexpect def client(request, name='', log=None): - client = httpexpect.spawn({'host':'localhost','port':8123,'timeout':1}, request) + client = httpexpect.spawn({'host':'localhost','port':8123,'timeout':30}, request) client.logger(log, prefix=name) client.timeout(20) return client From a00bcbb8ba45719dc178170a89d143aa3757351d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 14 Oct 2020 13:26:28 +0800 Subject: [PATCH 38/42] ISSUES-15265 try fix crash when create failure --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 08062b6e6ea..ddb1d738031 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -218,7 +218,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) { if (renamed) { - [[maybe_unused]] bool removed = fs::remove(metadata_file_tmp_path); + [[maybe_unused]] bool removed = fs::remove(metadata_file_path); assert(removed); } if (added) From 2c72cdef4434121880db81e20a07a324740f9213 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Wed, 14 Oct 2020 15:05:02 +0800 Subject: [PATCH 39/42] Fix error for offset_fetch without only/with ties --- src/Parsers/ParserSelectQuery.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index 8f0c70308ff..91c48fc362d 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -305,6 +305,10 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { select_query->limit_with_ties = false; } + else + { + return false; + } } } From d51deaeecf2eff06c70ee0d825acfd95f1f25aed Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 14 Oct 2020 10:49:50 +0300 Subject: [PATCH 40/42] Fix flaky 01509_parallel_quorum_insert_no_replicas --- .../0_stateless/01509_parallel_quorum_insert_no_replicas.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas.sql b/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas.sql index 7607a4e9012..1b680cf26c1 100644 --- a/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas.sql +++ b/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas.sql @@ -35,6 +35,8 @@ INSERT INTO r1 VALUES(2, '2'); ATTACH TABLE r2; +SYSTEM SYNC REPLICA r2; + SET insert_quorum=2, insert_quorum_parallel=1; INSERT INTO r1 VALUES(3, '3'); From c65d1e5c704f390fd902ac869865ec34f728a524 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 14 Oct 2020 13:59:29 +0300 Subject: [PATCH 41/42] Revert "Write structure of table functions to metadata" --- docker/test/base/Dockerfile | 1 - src/Databases/DatabaseOnDisk.cpp | 7 +- src/Databases/DatabaseOrdinary.cpp | 7 +- .../ClusterProxy/SelectStreamFactory.cpp | 3 +- src/Interpreters/Context.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 18 +- src/Interpreters/InterpreterDescribeQuery.cpp | 23 ++- src/Interpreters/InterpreterInsertQuery.cpp | 3 +- .../getHeaderForProcessingStage.cpp | 69 -------- .../getHeaderForProcessingStage.h | 27 --- src/Interpreters/ya.make | 1 - src/Parsers/ASTCreateQuery.cpp | 12 +- src/Parsers/ParserCreateQuery.cpp | 7 +- src/Storages/StorageDistributed.cpp | 36 +++- src/Storages/StorageDistributed.h | 21 ++- src/Storages/StorageFile.cpp | 57 +++---- src/Storages/StorageFile.h | 2 - src/Storages/StorageMaterializeMySQL.cpp | 2 +- src/Storages/StorageMaterializeMySQL.h | 20 +-- src/Storages/StorageMerge.cpp | 57 ++++++- src/Storages/StorageMerge.h | 7 + src/Storages/StorageProxy.h | 158 ------------------ src/Storages/StorageTableFunction.h | 137 --------------- src/Storages/StorageXDBC.h | 3 +- src/Storages/getStructureOfRemoteTable.cpp | 6 +- src/TableFunctions/CMakeLists.txt | 1 - src/TableFunctions/ITableFunction.cpp | 17 +- src/TableFunctions/ITableFunction.h | 27 +-- src/TableFunctions/ITableFunctionFileLike.cpp | 39 ++--- src/TableFunctions/ITableFunctionFileLike.h | 17 +- src/TableFunctions/ITableFunctionXDBC.cpp | 33 ++-- src/TableFunctions/ITableFunctionXDBC.h | 11 +- src/TableFunctions/TableFunctionFactory.cpp | 13 +- src/TableFunctions/TableFunctionFactory.h | 2 +- src/TableFunctions/TableFunctionFile.cpp | 5 +- src/TableFunctions/TableFunctionFile.h | 3 +- .../TableFunctionGenerateRandom.cpp | 18 +- .../TableFunctionGenerateRandom.h | 12 +- src/TableFunctions/TableFunctionHDFS.cpp | 10 +- src/TableFunctions/TableFunctionHDFS.h | 3 +- src/TableFunctions/TableFunctionInput.cpp | 16 +- src/TableFunctions/TableFunctionInput.h | 8 +- src/TableFunctions/TableFunctionMerge.cpp | 18 +- src/TableFunctions/TableFunctionMerge.h | 8 +- src/TableFunctions/TableFunctionMySQL.cpp | 46 ++--- src/TableFunctions/TableFunctionMySQL.h | 22 +-- src/TableFunctions/TableFunctionNull.cpp | 33 ++-- src/TableFunctions/TableFunctionNull.h | 7 +- src/TableFunctions/TableFunctionNumbers.cpp | 12 +- src/TableFunctions/TableFunctionNumbers.h | 5 +- src/TableFunctions/TableFunctionRemote.cpp | 53 ++---- src/TableFunctions/TableFunctionRemote.h | 14 +- src/TableFunctions/TableFunctionS3.cpp | 61 ++++--- src/TableFunctions/TableFunctionS3.h | 22 ++- src/TableFunctions/TableFunctionURL.cpp | 6 +- src/TableFunctions/TableFunctionURL.h | 3 +- src/TableFunctions/TableFunctionValues.cpp | 19 +-- src/TableFunctions/TableFunctionValues.h | 8 +- src/TableFunctions/TableFunctionView.cpp | 32 ++-- src/TableFunctions/TableFunctionView.h | 9 +- src/TableFunctions/TableFunctionZeros.cpp | 12 +- src/TableFunctions/TableFunctionZeros.h | 5 +- tests/config/lsan_suppressions.txt | 2 - .../test_distributed_format/test.py | 6 +- ..._expressions_in_engine_arguments.reference | 6 +- ...eate_as_table_function_structure.reference | 10 -- ...457_create_as_table_function_structure.sql | 33 ---- .../01461_alter_table_function.reference | 9 +- .../01461_alter_table_function.sql | 4 +- .../queries/0_stateless/arcadia_skip_list.txt | 1 - 70 files changed, 378 insertions(+), 1009 deletions(-) delete mode 100644 src/Interpreters/getHeaderForProcessingStage.cpp delete mode 100644 src/Interpreters/getHeaderForProcessingStage.h delete mode 100644 src/Storages/StorageProxy.h delete mode 100644 src/Storages/StorageTableFunction.h delete mode 100644 tests/config/lsan_suppressions.txt delete mode 100644 tests/queries/0_stateless/01457_create_as_table_function_structure.reference delete mode 100644 tests/queries/0_stateless/01457_create_as_table_function_structure.sql diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index 61a40673a96..506e32c18b3 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -52,7 +52,6 @@ RUN apt-get update \ RUN echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7'" >> /etc/environment; \ echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; \ echo "MSAN_OPTIONS='abort_on_error=1'" >> /etc/environment; \ - echo "LSAN_OPTIONS='suppressions=/usr/share/clickhouse-test/config/lsan_suppressions.txt'" >> /etc/environment; \ ln -s /usr/lib/llvm-${LLVM_VERSION}/bin/llvm-symbolizer /usr/bin/llvm-symbolizer; # Sanitizer options for current shell (not current, but the one that will be spawned on "docker run") # (but w/o verbosity for TSAN, otherwise test.reference will not match) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 8d9f222bf69..a2a2dd992ed 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -54,12 +54,9 @@ std::pair createTableFromAST( if (ast_create_query.as_table_function) { + const auto & table_function = ast_create_query.as_table_function->as(); const auto & factory = TableFunctionFactory::instance(); - auto table_function = factory.get(ast_create_query.as_table_function, context); - ColumnsDescription columns; - if (ast_create_query.columns_list && ast_create_query.columns_list->columns) - columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context, false); - StoragePtr storage = table_function->execute(ast_create_query.as_table_function, context, ast_create_query.table, std::move(columns)); + StoragePtr storage = factory.get(table_function.name, context)->execute(ast_create_query.as_table_function, context, ast_create_query.table); storage->renameInMemory(ast_create_query); return {ast_create_query.table, storage}; } diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index a1d24226444..13aeb7de148 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -254,12 +254,9 @@ void DatabaseOrdinary::alterTable(const Context & context, const StorageID & tab auto & ast_create_query = ast->as(); - bool has_structure = ast_create_query.columns_list && ast_create_query.columns_list->columns; - if (ast_create_query.as_table_function && !has_structure) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot alter table {} because it was created AS table function" - " and doesn't have structure in metadata", backQuote(table_name)); + if (ast_create_query.as_table_function) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot alter table {} because it was created AS table function", backQuote(table_name)); - assert(has_structure); ASTPtr new_columns = InterpreterCreateQuery::formatColumns(metadata.columns); ASTPtr new_indices = InterpreterCreateQuery::formatIndices(metadata.secondary_indices); ASTPtr new_constraints = InterpreterCreateQuery::formatConstraints(metadata.constraints); diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 7d99cbd1d43..c642b84afca 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -159,7 +159,8 @@ void SelectStreamFactory::createForShard( if (table_func_ptr) { - TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_func_ptr, context); + const auto * table_function = table_func_ptr->as(); + TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_function->name, context); main_table_storage = table_function_ptr->execute(table_func_ptr, context, table_function_ptr->getName()); } else diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 4295972ef80..e7614a66761 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -933,7 +933,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression) if (!res) { - TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression, *this); + TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression->as()->name, *this); /// Run it and remember the result res = table_function_ptr->execute(table_expression, *this, table_function_ptr->getName()); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ddb1d738031..665c34c45b8 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -453,9 +453,6 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(AS if (create.columns_list) { - if (create.as_table_function && (create.columns_list->indices || create.columns_list->constraints)) - throw Exception("Indexes and constraints are not supported for table functions", ErrorCodes::INCORRECT_QUERY); - if (create.columns_list->columns) { bool sanity_check_compression_codecs = !create.attach && !context.getSettingsRef().allow_suspicious_codecs; @@ -492,12 +489,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(AS properties.columns = ColumnsDescription(as_select_sample.getNamesAndTypesList()); } else if (create.as_table_function) - { - /// Table function without columns list. - auto table_function = TableFunctionFactory::instance().get(create.as_table_function, context); - properties.columns = table_function->getActualTableStructure(context); - assert(!properties.columns.empty()); - } + return {}; else throw Exception("Incorrect CREATE query: required list of column descriptions or AS section or SELECT.", ErrorCodes::INCORRECT_QUERY); @@ -583,12 +575,9 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const { - if (create.as_table_function) - return; - if (create.storage || create.is_view || create.is_materialized_view || create.is_live_view || create.is_dictionary) { - if (create.temporary && create.storage && create.storage->engine && create.storage->engine->name != "Memory") + if (create.temporary && create.storage->engine->name != "Memory") throw Exception( "Temporary tables can only be created with ENGINE = Memory, not " + create.storage->engine->name, ErrorCodes::INCORRECT_QUERY); @@ -768,8 +757,9 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, /// NOTE: CREATE query may be rewritten by Storage creator or table function if (create.as_table_function) { + const auto & table_function = create.as_table_function->as(); const auto & factory = TableFunctionFactory::instance(); - res = factory.get(create.as_table_function, context)->execute(create.as_table_function, context, create.table, properties.columns); + res = factory.get(table_function.name, context)->execute(create.as_table_function, context, create.table); res->renameInMemory({create.database, create.table, create.uuid}); } else diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 27d8ac48bc8..9c11bf147d3 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -72,16 +72,23 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() table_expression.subquery->children.at(0), context).getNamesAndTypesList(); columns = ColumnsDescription(std::move(names_and_types)); } - else if (table_expression.table_function) - { - TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression.table_function, context); - columns = table_function_ptr->getActualTableStructure(context); - } else { - auto table_id = context.resolveStorageID(table_expression.database_and_table_name); - context.checkAccess(AccessType::SHOW_COLUMNS, table_id); - auto table = DatabaseCatalog::instance().getTable(table_id, context); + StoragePtr table; + if (table_expression.table_function) + { + const auto & table_function = table_expression.table_function->as(); + TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_function.name, context); + /// Run the table function and remember the result + table = table_function_ptr->execute(table_expression.table_function, context, table_function_ptr->getName()); + } + else + { + auto table_id = context.resolveStorageID(table_expression.database_and_table_name); + context.checkAccess(AccessType::SHOW_COLUMNS, table_id); + table = DatabaseCatalog::instance().getTable(table_id, context); + } + auto table_lock = table->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout); auto metadata_snapshot = table->getInMemoryMetadataPtr(); columns = metadata_snapshot->getColumns(); diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 1a0ee21b99b..05707b66c0b 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -67,8 +67,9 @@ StoragePtr InterpreterInsertQuery::getTable(ASTInsertQuery & query) { if (query.table_function) { + const auto * table_function = query.table_function->as(); const auto & factory = TableFunctionFactory::instance(); - TableFunctionPtr table_function_ptr = factory.get(query.table_function, context); + TableFunctionPtr table_function_ptr = factory.get(table_function->name, context); return table_function_ptr->execute(query.table_function, context, table_function_ptr->getName()); } diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp deleted file mode 100644 index db7008a1779..00000000000 --- a/src/Interpreters/getHeaderForProcessingStage.cpp +++ /dev/null @@ -1,69 +0,0 @@ -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -/// Rewrite original query removing joined tables from it -bool removeJoin(ASTSelectQuery & select) -{ - const auto & tables = select.tables(); - if (!tables || tables->children.size() < 2) - return false; - - const auto & joined_table = tables->children[1]->as(); - if (!joined_table.table_join) - return false; - - /// The most simple temporary solution: leave only the first table in query. - /// TODO: we also need to remove joined columns and related functions (taking in account aliases if any). - tables->children.resize(1); - return true; -} - -Block getHeaderForProcessingStage( - const IStorage & storage, - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum processed_stage) -{ - switch (processed_stage) - { - case QueryProcessingStage::FetchColumns: - { - Block header = metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID()); - if (query_info.prewhere_info) - { - query_info.prewhere_info->prewhere_actions->execute(header); - if (query_info.prewhere_info->remove_prewhere_column) - header.erase(query_info.prewhere_info->prewhere_column_name); - } - return header; - } - case QueryProcessingStage::WithMergeableState: - case QueryProcessingStage::Complete: - case QueryProcessingStage::WithMergeableStateAfterAggregation: - case QueryProcessingStage::MAX: - { - auto query = query_info.query->clone(); - removeJoin(*query->as()); - - auto stream = std::make_shared( - metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID())); - return InterpreterSelectQuery(query, context, stream, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); - } - } - throw Exception("Logical Error: unknown processed stage.", ErrorCodes::LOGICAL_ERROR); -} - -} - diff --git a/src/Interpreters/getHeaderForProcessingStage.h b/src/Interpreters/getHeaderForProcessingStage.h deleted file mode 100644 index ec238edf774..00000000000 --- a/src/Interpreters/getHeaderForProcessingStage.h +++ /dev/null @@ -1,27 +0,0 @@ -#pragma once -#include -#include -#include - - -namespace DB -{ - -class IStorage; -struct StorageInMemoryMetadata; -using StorageMetadataPtr = std::shared_ptr; -struct SelectQueryInfo; -class Context; -class ASTSelectQuery; - -bool removeJoin(ASTSelectQuery & select); - -Block getHeaderForProcessingStage( - const IStorage & storage, - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum processed_stage); - -} diff --git a/src/Interpreters/ya.make b/src/Interpreters/ya.make index 4c0b64934c7..411100b0b9d 100644 --- a/src/Interpreters/ya.make +++ b/src/Interpreters/ya.make @@ -63,7 +63,6 @@ SRCS( ExtractExpressionInfoVisitor.cpp FillingRow.cpp getClusterName.cpp - getHeaderForProcessingStage.cpp getTableExpressions.cpp HashJoin.cpp IdentifierSemantic.cpp diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index 8c5681c3a9c..73903e28f84 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -282,23 +282,13 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat if (as_table_function) { - if (columns_list) - { - frame.expression_list_always_start_on_new_line = true; - settings.ostr << (settings.one_line ? " (" : "\n("); - FormatStateStacked frame_nested = frame; - columns_list->formatImpl(settings, state, frame_nested); - settings.ostr << (settings.one_line ? ")" : "\n)"); - frame.expression_list_always_start_on_new_line = false; - } - settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : ""); as_table_function->formatImpl(settings, state, frame); } frame.expression_list_always_start_on_new_line = true; - if (columns_list && !as_table_function) + if (columns_list) { settings.ostr << (settings.one_line ? " (" : "\n("); FormatStateStacked frame_nested = frame; diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index fb0fd20b4ed..55208ca4133 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -416,12 +416,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe return false; if (!storage_p.parse(pos, storage, expected) && !is_temporary) - { - if (!s_as.ignore(pos, expected)) - return false; - if (!table_function_p.parse(pos, as_table_function, expected)) - return false; - } + return false; } else { diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index afac3f97d68..81815be7afb 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -360,14 +360,12 @@ StorageDistributed::StorageDistributed( const ASTPtr & sharding_key_, const String & storage_policy_name_, const String & relative_data_path_, - bool attach_, - ClusterPtr owned_cluster_) + bool attach_) : IStorage(id_) , remote_database(remote_database_) , remote_table(remote_table_) , global_context(std::make_unique(context_)) , log(&Poco::Logger::get("StorageDistributed (" + id_.table_name + ")")) - , owned_cluster(std::move(owned_cluster_)) , cluster_name(global_context->getMacros()->expand(cluster_name_)) , has_sharding_key(sharding_key_) , relative_data_path(relative_data_path_) @@ -413,13 +411,39 @@ StorageDistributed::StorageDistributed( const ASTPtr & sharding_key_, const String & storage_policy_name_, const String & relative_data_path_, - bool attach, - ClusterPtr owned_cluster_) - : StorageDistributed(id_, columns_, constraints_, String{}, String{}, cluster_name_, context_, sharding_key_, storage_policy_name_, relative_data_path_, attach, std::move(owned_cluster_)) + bool attach) + : StorageDistributed(id_, columns_, constraints_, String{}, String{}, cluster_name_, context_, sharding_key_, storage_policy_name_, relative_data_path_, attach) { remote_table_function_ptr = std::move(remote_table_function_ptr_); } + +StoragePtr StorageDistributed::createWithOwnCluster( + const StorageID & table_id_, + const ColumnsDescription & columns_, + const String & remote_database_, /// database on remote servers. + const String & remote_table_, /// The name of the table on the remote servers. + ClusterPtr owned_cluster_, + const Context & context_) +{ + auto res = create(table_id_, columns_, ConstraintsDescription{}, remote_database_, remote_table_, String{}, context_, ASTPtr(), String(), String(), false); + res->owned_cluster = std::move(owned_cluster_); + return res; +} + + +StoragePtr StorageDistributed::createWithOwnCluster( + const StorageID & table_id_, + const ColumnsDescription & columns_, + ASTPtr & remote_table_function_ptr_, + ClusterPtr & owned_cluster_, + const Context & context_) +{ + auto res = create(table_id_, columns_, ConstraintsDescription{}, remote_table_function_ptr_, String{}, context_, ASTPtr(), String(), String(), false); + res->owned_cluster = owned_cluster_; + return res; +} + QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Context &context, QueryProcessingStage::Enum to_stage, const ASTPtr & query_ptr) const { const auto & settings = context.getSettingsRef(); diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index dd043e5feeb..7e4e9f56ab4 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -42,6 +42,21 @@ class StorageDistributed final : public ext::shared_ptr_helpergetHeader(); - auto & first_path = paths[0]; - Block header = StorageDistributedDirectoryMonitor::createStreamFromFile(first_path)->getHeader(); - StorageInMemoryMetadata storage_metadata; - auto columns = ColumnsDescription(header.getNamesAndTypesList()); - if (!args.columns.empty() && columns != args.columns) - throw Exception("Table structure and file structure are different", ErrorCodes::INCOMPATIBLE_COLUMNS); - storage_metadata.setColumns(columns); - setInMemoryMetadata(storage_metadata); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(ColumnsDescription(header.getNamesAndTypesList())); + setInMemoryMetadata(storage_metadata); + } } } @@ -191,8 +178,6 @@ StorageFile::StorageFile(const std::string & relative_table_dir_path, CommonArgu { if (relative_table_dir_path.empty()) throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); - if (args.format_name == "Distributed") - throw Exception("Distributed format is allowed only with explicit file path", ErrorCodes::INCORRECT_FILE_NAME); String table_dir_path = base_path + relative_table_dir_path + "/"; Poco::File(table_dir_path).createDirectories(); diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index f331538b4c7..ea70dcd5311 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -60,8 +60,6 @@ public: NamesAndTypesList getVirtuals() const override; - static Strings getPathsList(const String & table_path, const String & user_files_path, const Context & context); - protected: friend class StorageFileSource; friend class StorageFileBlockOutputStream; diff --git a/src/Storages/StorageMaterializeMySQL.cpp b/src/Storages/StorageMaterializeMySQL.cpp index 99c573e575e..c56d0f610e7 100644 --- a/src/Storages/StorageMaterializeMySQL.cpp +++ b/src/Storages/StorageMaterializeMySQL.cpp @@ -23,7 +23,7 @@ namespace DB { StorageMaterializeMySQL::StorageMaterializeMySQL(const StoragePtr & nested_storage_, const DatabaseMaterializeMySQL * database_) - : StorageProxy(nested_storage_->getStorageID()), nested_storage(nested_storage_), database(database_) + : IStorage(nested_storage_->getStorageID()), nested_storage(nested_storage_), database(database_) { auto nested_memory_metadata = nested_storage->getInMemoryMetadata(); StorageInMemoryMetadata in_memory_metadata; diff --git a/src/Storages/StorageMaterializeMySQL.h b/src/Storages/StorageMaterializeMySQL.h index 236fbedb421..ba79fd6bca4 100644 --- a/src/Storages/StorageMaterializeMySQL.h +++ b/src/Storages/StorageMaterializeMySQL.h @@ -4,41 +4,31 @@ #if USE_MYSQL -#include +#include #include namespace DB { -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - -class StorageMaterializeMySQL final : public ext::shared_ptr_helper, public StorageProxy +class StorageMaterializeMySQL final : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; public: String getName() const override { return "MaterializeMySQL"; } + bool supportsFinal() const override { return nested_storage->supportsFinal(); } + bool supportsSampling() const override { return nested_storage->supportsSampling(); } + StorageMaterializeMySQL(const StoragePtr & nested_storage_, const DatabaseMaterializeMySQL * database_); Pipe read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; - BlockOutputStreamPtr write(const ASTPtr &, const StorageMetadataPtr &, const Context &) override { throwNotAllowed(); } - NamesAndTypesList getVirtuals() const override; ColumnSizeByName getColumnSizes() const override; private: - StoragePtr getNested() const override { return nested_storage; } - [[noreturn]] void throwNotAllowed() const - { - throw Exception("This method is not allowed for MaterializeMySQ", ErrorCodes::NOT_IMPLEMENTED); - } - StoragePtr nested_storage; const DatabaseMaterializeMySQL * database; }; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 0ca89f42670..5c68d2dd047 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include @@ -43,6 +42,23 @@ namespace ErrorCodes namespace { +/// Rewrite original query removing joined tables from it +bool removeJoin(ASTSelectQuery & select) +{ + const auto & tables = select.tables(); + if (!tables || tables->children.size() < 2) + return false; + + const auto & joined_table = tables->children[1]->as(); + if (!joined_table.table_join) + return false; + + /// The most simple temporary solution: leave only the first table in query. + /// TODO: we also need to remove joined columns and related functions (taking in account aliases if any). + tables->children.resize(1); + return true; +} + void modifySelect(ASTSelectQuery & select, const TreeRewriterResult & rewriter_result) { if (removeJoin(select)) @@ -67,6 +83,7 @@ void modifySelect(ASTSelectQuery & select, const TreeRewriterResult & rewriter_r } + StorageMerge::StorageMerge( const StorageID & table_id_, const ColumnsDescription & columns_, @@ -186,7 +203,7 @@ Pipe StorageMerge::read( modified_context->setSetting("optimize_move_to_prewhere", false); /// What will be result structure depending on query processed stage in source tables? - Block header = getHeaderForProcessingStage(*this, column_names, metadata_snapshot, query_info, context, processed_stage); + Block header = getQueryHeader(column_names, metadata_snapshot, query_info, context, processed_stage); /** First we make list of selected tables to find out its size. * This is necessary to correctly pass the recommended number of threads to each table. @@ -439,6 +456,42 @@ void StorageMerge::alter( setInMemoryMetadata(storage_metadata); } +Block StorageMerge::getQueryHeader( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage) +{ + switch (processed_stage) + { + case QueryProcessingStage::FetchColumns: + { + Block header = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); + if (query_info.prewhere_info) + { + query_info.prewhere_info->prewhere_actions->execute(header); + if (query_info.prewhere_info->remove_prewhere_column) + header.erase(query_info.prewhere_info->prewhere_column_name); + } + return header; + } + case QueryProcessingStage::WithMergeableState: + case QueryProcessingStage::Complete: + case QueryProcessingStage::WithMergeableStateAfterAggregation: + case QueryProcessingStage::MAX: + { + auto query = query_info.query->clone(); + removeJoin(*query->as()); + + auto stream = std::make_shared( + metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID())); + return InterpreterSelectQuery(query, context, stream, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + } + } + throw Exception("Logical Error: unknown processed stage.", ErrorCodes::LOGICAL_ERROR); +} + void StorageMerge::convertingSourceStream( const Block & header, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 9da71745e2c..78debf194f0 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -76,6 +76,13 @@ protected: const String & table_name_regexp_, const Context & context_); + Block getQueryHeader( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage); + Pipe createSources( const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h deleted file mode 100644 index 7b010476b22..00000000000 --- a/src/Storages/StorageProxy.h +++ /dev/null @@ -1,158 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -class StorageProxy : public IStorage -{ -public: - - StorageProxy(const StorageID & table_id_) : IStorage(table_id_) {} - - virtual StoragePtr getNested() const = 0; - - String getName() const override { return "StorageProxy"; } - - bool isRemote() const override { return getNested()->isRemote(); } - bool isView() const override { return getNested()->isView(); } - bool supportsSampling() const override { return getNested()->supportsSampling(); } - bool supportsFinal() const override { return getNested()->supportsFinal(); } - bool supportsPrewhere() const override { return getNested()->supportsPrewhere(); } - bool supportsReplication() const override { return getNested()->supportsReplication(); } - bool supportsParallelInsert() const override { return getNested()->supportsParallelInsert(); } - bool supportsDeduplication() const override { return getNested()->supportsDeduplication(); } - bool supportsSettings() const override { return getNested()->supportsSettings(); } - bool noPushingToViews() const override { return getNested()->noPushingToViews(); } - bool hasEvenlyDistributedRead() const override { return getNested()->hasEvenlyDistributedRead(); } - - ColumnSizeByName getColumnSizes() const override { return getNested()->getColumnSizes(); } - NamesAndTypesList getVirtuals() const override { return getNested()->getVirtuals(); } - QueryProcessingStage::Enum getQueryProcessingStage(const Context & context, QueryProcessingStage::Enum to_stage, const ASTPtr & ast) const override - { - return getNested()->getQueryProcessingStage(context, to_stage, ast); - } - - BlockInputStreams watch( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size, - unsigned num_streams) override - { - return getNested()->watch(column_names, query_info, context, processed_stage, max_block_size, num_streams); - } - - Pipe read( - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - unsigned num_streams) override - { - return getNested()->read(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); - } - - BlockOutputStreamPtr write( - const ASTPtr & query, - const StorageMetadataPtr & metadata_snapshot, - const Context & context) override - { - return getNested()->write(query, metadata_snapshot, context); - } - - void drop() override { getNested()->drop(); } - - void truncate( - const ASTPtr & query, - const StorageMetadataPtr & metadata_snapshot, - const Context & context, - TableExclusiveLockHolder & lock) override - { - getNested()->truncate(query, metadata_snapshot, context, lock); - } - - void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override - { - getNested()->rename(new_path_to_table_data, new_table_id); - IStorage::renameInMemory(new_table_id); - } - - void renameInMemory(const StorageID & new_table_id) override - { - getNested()->renameInMemory(new_table_id); - IStorage::renameInMemory(new_table_id); - } - - void alter(const AlterCommands & params, const Context & context, TableLockHolder & alter_lock_holder) override - { - getNested()->alter(params, context, alter_lock_holder); - IStorage::setInMemoryMetadata(getNested()->getInMemoryMetadata()); - } - - void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override - { - getNested()->checkAlterIsPossible(commands, settings); - } - - Pipe alterPartition( - const ASTPtr & query, - const StorageMetadataPtr & metadata_snapshot, - const PartitionCommands & commands, - const Context & context) override - { - return getNested()->alterPartition(query, metadata_snapshot, commands, context); - } - - void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const override - { - getNested()->checkAlterPartitionIsPossible(commands, metadata_snapshot, settings); - } - - bool optimize( - const ASTPtr & query, - const StorageMetadataPtr & metadata_snapshot, - const ASTPtr & partition, - bool final, - bool deduplicate, - const Context & context) override - { - return getNested()->optimize(query, metadata_snapshot, partition, final, deduplicate, context); - } - - void mutate(const MutationCommands & commands, const Context & context) override { getNested()->mutate(commands, context); } - - CancellationCode killMutation(const String & mutation_id) override { return getNested()->killMutation(mutation_id); } - - void startup() override { getNested()->startup(); } - void shutdown() override { getNested()->shutdown(); } - - ActionLock getActionLock(StorageActionBlockType action_type) override { return getNested()->getActionLock(action_type); } - - bool supportsIndexForIn() const override { return getNested()->supportsIndexForIn(); } - bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context, const StorageMetadataPtr & metadata_snapshot) const override - { - return getNested()->mayBenefitFromIndexForIn(left_in_operand, query_context, metadata_snapshot); - } - - CheckResults checkData(const ASTPtr & query , const Context & context) override { return getNested()->checkData(query, context); } - void checkTableCanBeDropped() const override { getNested()->checkTableCanBeDropped(); } - void checkPartitionCanBeDropped(const ASTPtr & partition) override { getNested()->checkPartitionCanBeDropped(partition); } - Strings getDataPaths() const override { return getNested()->getDataPaths(); } - StoragePolicyPtr getStoragePolicy() const override { return getNested()->getStoragePolicy(); } - std::optional totalRows() const override { return getNested()->totalRows(); } - std::optional totalBytes() const override { return getNested()->totalBytes(); } - std::optional lifetimeRows() const override { return getNested()->lifetimeRows(); } - std::optional lifetimeBytes() const override { return getNested()->lifetimeBytes(); } - -}; - - -} - diff --git a/src/Storages/StorageTableFunction.h b/src/Storages/StorageTableFunction.h deleted file mode 100644 index 9b698cb3954..00000000000 --- a/src/Storages/StorageTableFunction.h +++ /dev/null @@ -1,137 +0,0 @@ -#pragma once -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int INCOMPATIBLE_COLUMNS; -} - -using GetNestedStorageFunc = std::function; - -/// Lazily creates underlying storage. -/// Adds ConversionTransform in case of structure mismatch. -class StorageTableFunctionProxy final : public StorageProxy -{ -public: - StorageTableFunctionProxy(const StorageID & table_id_, GetNestedStorageFunc get_nested_, - ColumnsDescription cached_columns, bool add_conversion_ = true) - : StorageProxy(table_id_), get_nested(std::move(get_nested_)), add_conversion(add_conversion_) - { - StorageInMemoryMetadata cached_metadata; - cached_metadata.setColumns(std::move(cached_columns)); - setInMemoryMetadata(cached_metadata); - } - - StoragePtr getNested() const override - { - std::lock_guard lock{nested_mutex}; - if (nested) - return nested; - - auto nested_storage = get_nested(); - nested_storage->startup(); - nested_storage->renameInMemory(getStorageID()); - nested = nested_storage; - get_nested = {}; - return nested; - } - - String getName() const override - { - std::lock_guard lock{nested_mutex}; - if (nested) - return nested->getName(); - return StorageProxy::getName(); - } - - void startup() override { } - void shutdown() override - { - std::lock_guard lock{nested_mutex}; - if (nested) - nested->shutdown(); - } - - void drop() override - { - std::lock_guard lock{nested_mutex}; - if (nested) - nested->drop(); - } - - Pipe read( - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - unsigned num_streams) override - { - String cnames; - for (const auto & c : column_names) - cnames += c + " "; - auto storage = getNested(); - auto nested_metadata = storage->getInMemoryMetadataPtr(); - auto pipe = storage->read(column_names, nested_metadata, query_info, context, - processed_stage, max_block_size, num_streams); - if (!pipe.empty() && add_conversion) - { - auto to_header = getHeaderForProcessingStage(*this, column_names, metadata_snapshot, - query_info, context, processed_stage); - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared( - header, - to_header, - ConvertingTransform::MatchColumnsMode::Name); - }); - } - return pipe; - } - - BlockOutputStreamPtr write( - const ASTPtr & query, - const StorageMetadataPtr & metadata_snapshot, - const Context & context) override - { - auto storage = getNested(); - auto cached_structure = metadata_snapshot->getSampleBlock(); - auto actual_structure = storage->getInMemoryMetadataPtr()->getSampleBlock(); - if (!blocksHaveEqualStructure(actual_structure, cached_structure) && add_conversion) - { - throw Exception("Source storage and table function have different structure", ErrorCodes::INCOMPATIBLE_COLUMNS); - } - return storage->write(query, metadata_snapshot, context); - } - - void renameInMemory(const StorageID & new_table_id) override - { - std::lock_guard lock{nested_mutex}; - if (nested) - StorageProxy::renameInMemory(new_table_id); - else - IStorage::renameInMemory(new_table_id); - } - - bool isView() const override { return false; } - void checkTableCanBeDropped() const override {} - -private: - mutable std::mutex nested_mutex; - mutable GetNestedStorageFunc get_nested; - mutable StoragePtr nested; - const bool add_conversion; -}; - -} diff --git a/src/Storages/StorageXDBC.h b/src/Storages/StorageXDBC.h index 7f93cbcd320..508edf22684 100644 --- a/src/Storages/StorageXDBC.h +++ b/src/Storages/StorageXDBC.h @@ -34,7 +34,6 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; - std::string getName() const override; private: BridgeHelperPtr bridge_helper; @@ -62,6 +61,8 @@ private: size_t max_block_size) const override; Block getHeaderBlock(const Names & column_names, const StorageMetadataPtr & metadata_snapshot) const override; + + std::string getName() const override; }; } diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index a987e3d4e8a..6a5ef58a326 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -38,8 +38,10 @@ ColumnsDescription getStructureOfRemoteTableInShard( { if (shard_info.isLocal()) { - TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_func_ptr, context); - return table_function_ptr->getActualTableStructure(context); + const auto * table_function = table_func_ptr->as(); + TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_function->name, context); + auto storage_ptr = table_function_ptr->execute(table_func_ptr, context, table_function_ptr->getName()); + return storage_ptr->getInMemoryMetadataPtr()->getColumns(); } auto table_func_name = queryToString(table_func_ptr); diff --git a/src/TableFunctions/CMakeLists.txt b/src/TableFunctions/CMakeLists.txt index 8e9eedadf53..33e900231e5 100644 --- a/src/TableFunctions/CMakeLists.txt +++ b/src/TableFunctions/CMakeLists.txt @@ -6,4 +6,3 @@ list(REMOVE_ITEM clickhouse_table_functions_headers ITableFunction.h TableFuncti add_library(clickhouse_table_functions ${clickhouse_table_functions_sources}) target_link_libraries(clickhouse_table_functions PRIVATE clickhouse_parsers clickhouse_storages_system dbms) - diff --git a/src/TableFunctions/ITableFunction.cpp b/src/TableFunctions/ITableFunction.cpp index 804a5b232ec..6a784c062da 100644 --- a/src/TableFunctions/ITableFunction.cpp +++ b/src/TableFunctions/ITableFunction.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include #include @@ -14,23 +13,11 @@ namespace ProfileEvents namespace DB { -StoragePtr ITableFunction::execute(const ASTPtr & ast_function, const Context & context, const std::string & table_name, - ColumnsDescription cached_columns) const +StoragePtr ITableFunction::execute(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { ProfileEvents::increment(ProfileEvents::TableFunctionExecute); context.checkAccess(AccessType::CREATE_TEMPORARY_TABLE | StorageFactory::instance().getSourceAccessType(getStorageTypeName())); - - if (cached_columns.empty() || (hasStaticStructure() && cached_columns == getActualTableStructure(context))) - return executeImpl(ast_function, context, table_name, std::move(cached_columns)); - - auto get_storage = [=, tf = shared_from_this()]() -> StoragePtr - { - return tf->executeImpl(ast_function, context, table_name, cached_columns); - }; - - /// It will request actual table structure and create underlying storage lazily - return std::make_shared(StorageID(getDatabaseName(), table_name), std::move(get_storage), - std::move(cached_columns), needStructureConversion()); + return executeImpl(ast_function, context, table_name); } } diff --git a/src/TableFunctions/ITableFunction.h b/src/TableFunctions/ITableFunction.h index 4a73adbdf80..0bbd7e787a5 100644 --- a/src/TableFunctions/ITableFunction.h +++ b/src/TableFunctions/ITableFunction.h @@ -2,7 +2,6 @@ #include #include -#include #include #include @@ -22,18 +21,9 @@ class Context; * Example: * SELECT count() FROM remote('example01-01-1', merge, hits) * - go to `example01-01-1`, in `merge` database, `hits` table. - * - * - * When creating table AS table_function(...) we probably don't know structure of the table - * and have to request if from remote server, because structure is required to create a Storage. - * To avoid failures on server startup, we write obtained structure to metadata file. - * So, table function may have two different columns lists: - * - cached_columns written to metadata - * - the list returned from getActualTableStructure(...) - * See StorageTableFunctionProxy. */ -class ITableFunction : public std::enable_shared_from_this +class ITableFunction { public: static inline std::string getDatabaseName() { return "_table_function"; } @@ -41,24 +31,13 @@ public: /// Get the main function name. virtual std::string getName() const = 0; - /// Returns true if we always know table structure when executing table function - /// (e.g. structure is specified in table function arguments) - virtual bool hasStaticStructure() const { return false; } - /// Returns false if storage returned by table function supports type conversion (e.g. StorageDistributed) - virtual bool needStructureConversion() const { return true; } - - virtual void parseArguments(const ASTPtr & /*ast_function*/, const Context & /*context*/) {} - - /// Returns actual table structure probably requested from remote server, may fail - virtual ColumnsDescription getActualTableStructure(const Context & /*context*/) const = 0; - /// Create storage according to the query. - StoragePtr execute(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns_ = {}) const; + StoragePtr execute(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const; virtual ~ITableFunction() {} private: - virtual StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const = 0; + virtual StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const = 0; virtual const char * getStorageTypeName() const = 0; }; diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index f876da02fd1..46a64cef785 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -9,12 +9,11 @@ #include #include -#include -#include #include #include + namespace DB { @@ -22,10 +21,9 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int INCORRECT_FILE_NAME; } -void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, const Context & context) +StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { /// Parse args ASTs & args_func = ast_function->children; @@ -41,8 +39,8 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, const C for (auto & arg : args) arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); - filename = args[0]->as().value.safeGet(); - format = args[1]->as().value.safeGet(); + std::string filename = args[0]->as().value.safeGet(); + std::string format = args[1]->as().value.safeGet(); if (args.size() == 2 && getName() == "file") { @@ -53,33 +51,24 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, const C throw Exception("Table function '" + getName() + "' requires 3 or 4 arguments: filename, format, structure and compression method (default auto).", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + ColumnsDescription columns; + std::string compression_method = "auto"; + if (args.size() > 2) - structure = args[2]->as().value.safeGet(); + { + auto structure = args[2]->as().value.safeGet(); + columns = parseColumnsListFromString(structure, context); + } if (args.size() == 4) compression_method = args[3]->as().value.safeGet(); -} -StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const -{ - auto columns = getActualTableStructure(context); + /// Create table StoragePtr storage = getStorage(filename, format, columns, const_cast(context), table_name, compression_method); + storage->startup(); + return storage; } -ColumnsDescription ITableFunctionFileLike::getActualTableStructure(const Context & context) const -{ - if (structure.empty()) - { - assert(getName() == "file" && format == "Distributed"); - Strings paths = StorageFile::getPathsList(filename, context.getUserFilesPath(), context); - if (paths.empty()) - throw Exception("Cannot get table structure from file, because no files match specified name", ErrorCodes::INCORRECT_FILE_NAME); - auto read_stream = StorageDistributedDirectoryMonitor::createStreamFromFile(paths[0]); - return ColumnsDescription{read_stream->getHeader().getNamesAndTypesList()}; - } - return parseColumnsListFromString(structure, context); -} - } diff --git a/src/TableFunctions/ITableFunctionFileLike.h b/src/TableFunctions/ITableFunctionFileLike.h index f1c648ac0aa..101f4104cb8 100644 --- a/src/TableFunctions/ITableFunctionFileLike.h +++ b/src/TableFunctions/ITableFunctionFileLike.h @@ -13,21 +13,8 @@ class Context; class ITableFunctionFileLike : public ITableFunction { private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override; - + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; virtual StoragePtr getStorage( - const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, - const std::string & table_name, const String & compression_method) const = 0; - - ColumnsDescription getActualTableStructure(const Context & context) const override; - - void parseArguments(const ASTPtr & ast_function, const Context & context) override; - - bool hasStaticStructure() const override { return true; } - - String filename; - String format; - String structure; - String compression_method = "auto"; + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const = 0; }; } diff --git a/src/TableFunctions/ITableFunctionXDBC.cpp b/src/TableFunctions/ITableFunctionXDBC.cpp index 67d1257fe4c..e3b36e97525 100644 --- a/src/TableFunctions/ITableFunctionXDBC.cpp +++ b/src/TableFunctions/ITableFunctionXDBC.cpp @@ -24,10 +24,11 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int UNKNOWN_EXCEPTION; extern const int LOGICAL_ERROR; } -void ITableFunctionXDBC::parseArguments(const ASTPtr & ast_function, const Context & context) +StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { const auto & args_func = ast_function->as(); @@ -43,6 +44,10 @@ void ITableFunctionXDBC::parseArguments(const ASTPtr & ast_function, const Conte for (auto & arg : args) arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); + std::string connection_string; + std::string schema_name; + std::string remote_table_name; + if (args.size() == 3) { connection_string = args[0]->as().value.safeGet(); @@ -55,16 +60,11 @@ void ITableFunctionXDBC::parseArguments(const ASTPtr & ast_function, const Conte remote_table_name = args[1]->as().value.safeGet(); } - /// Have to const_cast, because bridges store their commands inside context - helper = createBridgeHelper(const_cast(context), context.getSettingsRef().http_receive_timeout.value, connection_string); - helper->startBridgeSync(); -} - -ColumnsDescription ITableFunctionXDBC::getActualTableStructure(const Context & context) const -{ - assert(helper); - /* Infer external table structure */ + /// Have to const_cast, because bridges store their commands inside context + BridgeHelperPtr helper = createBridgeHelper(const_cast(context), context.getSettingsRef().http_receive_timeout.value, connection_string); + helper->startBridgeSync(); + Poco::URI columns_info_uri = helper->getColumnsInfoURI(); columns_info_uri.addQueryParameter("connection_string", connection_string); if (!schema_name.empty()) @@ -73,7 +73,7 @@ ColumnsDescription ITableFunctionXDBC::getActualTableStructure(const Context & c const auto use_nulls = context.getSettingsRef().external_table_functions_use_nulls; columns_info_uri.addQueryParameter("external_table_functions_use_nulls", - Poco::NumberFormatter::format(use_nulls)); + Poco::NumberFormatter::format(use_nulls)); ReadWriteBufferFromHTTP buf(columns_info_uri, Poco::Net::HTTPRequest::HTTP_POST, {}, ConnectionTimeouts::getHTTPTimeouts(context)); @@ -81,14 +81,11 @@ ColumnsDescription ITableFunctionXDBC::getActualTableStructure(const Context & c readStringBinary(columns_info, buf); NamesAndTypesList columns = NamesAndTypesList::parse(columns_info); - return ColumnsDescription{columns}; -} + auto result = std::make_shared(StorageID(getDatabaseName(), table_name), schema_name, remote_table_name, ColumnsDescription{columns}, context, helper); + + if (!result) + throw Exception("Failed to instantiate storage from table function " + getName(), ErrorCodes::UNKNOWN_EXCEPTION); -StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const -{ - assert(helper); - auto columns = getActualTableStructure(context); - auto result = std::make_shared(StorageID(getDatabaseName(), table_name), schema_name, remote_table_name, columns, context, helper); result->startup(); return result; } diff --git a/src/TableFunctions/ITableFunctionXDBC.h b/src/TableFunctions/ITableFunctionXDBC.h index fb0a0fd1185..547fda3f734 100644 --- a/src/TableFunctions/ITableFunctionXDBC.h +++ b/src/TableFunctions/ITableFunctionXDBC.h @@ -18,21 +18,12 @@ namespace DB class ITableFunctionXDBC : public ITableFunction { private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; /* A factory method to create bridge helper, that will assist in remote interaction */ virtual BridgeHelperPtr createBridgeHelper(Context & context, const Poco::Timespan & http_timeout_, const std::string & connection_string_) const = 0; - - ColumnsDescription getActualTableStructure(const Context & context) const override; - - void parseArguments(const ASTPtr & ast_function, const Context & context) override; - - String connection_string; - String schema_name; - String remote_table_name; - BridgeHelperPtr helper; }; class TableFunctionJDBC : public ITableFunctionXDBC diff --git a/src/TableFunctions/TableFunctionFactory.cpp b/src/TableFunctions/TableFunctionFactory.cpp index 0399d83fe4d..bc139edfb73 100644 --- a/src/TableFunctions/TableFunctionFactory.cpp +++ b/src/TableFunctions/TableFunctionFactory.cpp @@ -3,7 +3,6 @@ #include #include #include -#include namespace DB @@ -29,21 +28,19 @@ void TableFunctionFactory::registerFunction(const std::string & name, Value crea } TableFunctionPtr TableFunctionFactory::get( - const ASTPtr & ast_function, + const std::string & name, const Context & context) const { - const auto * table_function = ast_function->as(); - auto res = tryGet(table_function->name, context); + auto res = tryGet(name, context); if (!res) { - auto hints = getHints(table_function->name); + auto hints = getHints(name); if (!hints.empty()) - throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}. Maybe you meant: {}", table_function->name , toString(hints)); + throw Exception("Unknown table function " + name + ". Maybe you meant: " + toString(hints), ErrorCodes::UNKNOWN_FUNCTION); else - throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}", table_function->name); + throw Exception("Unknown table function " + name, ErrorCodes::UNKNOWN_FUNCTION); } - res->parseArguments(ast_function, context); return res; } diff --git a/src/TableFunctions/TableFunctionFactory.h b/src/TableFunctions/TableFunctionFactory.h index 820b5eb1c7b..6d0302a64ff 100644 --- a/src/TableFunctions/TableFunctionFactory.h +++ b/src/TableFunctions/TableFunctionFactory.h @@ -41,7 +41,7 @@ public: } /// Throws an exception if not found. - TableFunctionPtr get(const ASTPtr & ast_function, const Context & context) const; + TableFunctionPtr get(const std::string & name, const Context & context) const; /// Returns nullptr if not found. TableFunctionPtr tryGet(const std::string & name, const Context & context) const; diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index 39de6dce92c..4f67f4cfd10 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -9,10 +9,9 @@ namespace DB { StoragePtr TableFunctionFile::getStorage( - const String & source, const String & format_, const ColumnsDescription & columns, Context & global_context, - const std::string & table_name, const std::string & compression_method_) const + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const std::string & compression_method) const { - StorageFile::CommonArguments args{StorageID(getDatabaseName(), table_name), format_, compression_method_, columns, ConstraintsDescription{}, global_context}; + StorageFile::CommonArguments args{StorageID(getDatabaseName(), table_name), format, compression_method, columns, ConstraintsDescription{}, global_context}; return StorageFile::create(source, global_context.getUserFilesPath(), args); } diff --git a/src/TableFunctions/TableFunctionFile.h b/src/TableFunctions/TableFunctionFile.h index 91e2225a6d0..e0d8c20ac61 100644 --- a/src/TableFunctions/TableFunctionFile.h +++ b/src/TableFunctions/TableFunctionFile.h @@ -23,7 +23,6 @@ public: private: StoragePtr getStorage( - const String & source, const String & format_, const ColumnsDescription & columns, Context & global_context, - const std::string & table_name, const std::string & compression_method_) const override; + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const std::string & compression_method) const override; const char * getStorageTypeName() const override { return "File"; } };} diff --git a/src/TableFunctions/TableFunctionGenerateRandom.cpp b/src/TableFunctions/TableFunctionGenerateRandom.cpp index 15c2c2bfa1f..548db38515c 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.cpp +++ b/src/TableFunctions/TableFunctionGenerateRandom.cpp @@ -26,7 +26,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, const Context & /*context*/) +StoragePtr TableFunctionGenerateRandom::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { ASTs & args_func = ast_function->children; @@ -58,7 +58,11 @@ void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, co } /// Parsing first argument as table structure and creating a sample block - structure = args[0]->as().value.safeGet(); + std::string structure = args[0]->as().value.safeGet(); + + UInt64 max_string_length = 10; + UInt64 max_array_length = 10; + std::optional random_seed; if (args.size() >= 2) { @@ -72,16 +76,10 @@ void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, co if (args.size() == 4) max_array_length = args[3]->as().value.safeGet(); -} -ColumnsDescription TableFunctionGenerateRandom::getActualTableStructure(const Context & context) const -{ - return parseColumnsListFromString(structure, context); -} -StoragePtr TableFunctionGenerateRandom::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const -{ - auto columns = getActualTableStructure(context); + ColumnsDescription columns = parseColumnsListFromString(structure, context); + auto res = StorageGenerateRandom::create(StorageID(getDatabaseName(), table_name), columns, max_array_length, max_string_length, random_seed); res->startup(); return res; diff --git a/src/TableFunctions/TableFunctionGenerateRandom.h b/src/TableFunctions/TableFunctionGenerateRandom.h index 1d8839ba6d4..b0919608737 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.h +++ b/src/TableFunctions/TableFunctionGenerateRandom.h @@ -13,19 +13,9 @@ class TableFunctionGenerateRandom : public ITableFunction public: static constexpr auto name = "generateRandom"; std::string getName() const override { return name; } - bool hasStaticStructure() const override { return true; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; const char * getStorageTypeName() const override { return "GenerateRandom"; } - - ColumnsDescription getActualTableStructure(const Context & context) const override; - void parseArguments(const ASTPtr & ast_function, const Context & context) override; - - String structure; - UInt64 max_string_length = 10; - UInt64 max_array_length = 10; - std::optional random_seed; - }; diff --git a/src/TableFunctions/TableFunctionHDFS.cpp b/src/TableFunctions/TableFunctionHDFS.cpp index e2f227ef7b5..3bd6cd3ed76 100644 --- a/src/TableFunctions/TableFunctionHDFS.cpp +++ b/src/TableFunctions/TableFunctionHDFS.cpp @@ -10,17 +10,15 @@ namespace DB { StoragePtr TableFunctionHDFS::getStorage( - const String & source, const String & format_, const ColumnsDescription & columns, Context & global_context, - const std::string & table_name, const String & compression_method_) const + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const { - return StorageHDFS::create( - source, + return StorageHDFS::create(source, StorageID(getDatabaseName(), table_name), - format_, + format, columns, ConstraintsDescription{}, global_context, - compression_method_); + compression_method); } diff --git a/src/TableFunctions/TableFunctionHDFS.h b/src/TableFunctions/TableFunctionHDFS.h index 47e040f7593..5e8503b318e 100644 --- a/src/TableFunctions/TableFunctionHDFS.h +++ b/src/TableFunctions/TableFunctionHDFS.h @@ -26,8 +26,7 @@ public: private: StoragePtr getStorage( - const String & source, const String & format_, const ColumnsDescription & columns, Context & global_context, - const std::string & table_name, const String & compression_method_) const override; + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const override; const char * getStorageTypeName() const override { return "HDFS"; } }; diff --git a/src/TableFunctions/TableFunctionInput.cpp b/src/TableFunctions/TableFunctionInput.cpp index 41c41835434..5b4a3d97ee4 100644 --- a/src/TableFunctions/TableFunctionInput.cpp +++ b/src/TableFunctions/TableFunctionInput.cpp @@ -22,7 +22,7 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -void TableFunctionInput::parseArguments(const ASTPtr & ast_function, const Context & context) +StoragePtr TableFunctionInput::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { const auto * function = ast_function->as(); @@ -35,18 +35,12 @@ void TableFunctionInput::parseArguments(const ASTPtr & ast_function, const Conte throw Exception("Table function '" + getName() + "' requires exactly 1 argument: structure", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - structure = evaluateConstantExpressionOrIdentifierAsLiteral(args[0], context)->as().value.safeGet(); -} + String structure = evaluateConstantExpressionOrIdentifierAsLiteral(args[0], context)->as().value.safeGet(); + auto columns = parseColumnsListFromString(structure, context); + StoragePtr storage = StorageInput::create(StorageID(getDatabaseName(), table_name), columns); -ColumnsDescription TableFunctionInput::getActualTableStructure(const Context & context) const -{ - return parseColumnsListFromString(structure, context); -} - -StoragePtr TableFunctionInput::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const -{ - auto storage = StorageInput::create(StorageID(getDatabaseName(), table_name), getActualTableStructure(context)); storage->startup(); + return storage; } diff --git a/src/TableFunctions/TableFunctionInput.h b/src/TableFunctions/TableFunctionInput.h index 5809d48a77c..a2791533c5d 100644 --- a/src/TableFunctions/TableFunctionInput.h +++ b/src/TableFunctions/TableFunctionInput.h @@ -15,16 +15,10 @@ class TableFunctionInput : public ITableFunction public: static constexpr auto name = "input"; std::string getName() const override { return name; } - bool hasStaticStructure() const override { return true; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; const char * getStorageTypeName() const override { return "Input"; } - - ColumnsDescription getActualTableStructure(const Context & context) const override; - void parseArguments(const ASTPtr & ast_function, const Context & context) override; - - String structure; }; } diff --git a/src/TableFunctions/TableFunctionMerge.cpp b/src/TableFunctions/TableFunctionMerge.cpp index a878909e29d..7c0c1fb233f 100644 --- a/src/TableFunctions/TableFunctionMerge.cpp +++ b/src/TableFunctions/TableFunctionMerge.cpp @@ -45,7 +45,8 @@ static NamesAndTypesList chooseColumns(const String & source_database, const Str return any_table->getInMemoryMetadataPtr()->getColumns().getAllPhysical(); } -void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, const Context & context) + +StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { ASTs & args_func = ast_function->children; @@ -64,24 +65,15 @@ void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, const Conte args[0] = evaluateConstantExpressionForDatabaseName(args[0], context); args[1] = evaluateConstantExpressionAsLiteral(args[1], context); - source_database = args[0]->as().value.safeGet(); - table_name_regexp = args[1]->as().value.safeGet(); -} + String source_database = args[0]->as().value.safeGet(); + String table_name_regexp = args[1]->as().value.safeGet(); -ColumnsDescription TableFunctionMerge::getActualTableStructure(const Context & context) const -{ - return ColumnsDescription{chooseColumns(source_database, table_name_regexp, context)}; -} - -StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const -{ auto res = StorageMerge::create( StorageID(getDatabaseName(), table_name), - getActualTableStructure(context), + ColumnsDescription{chooseColumns(source_database, table_name_regexp, context)}, source_database, table_name_regexp, context); - res->startup(); return res; } diff --git a/src/TableFunctions/TableFunctionMerge.h b/src/TableFunctions/TableFunctionMerge.h index a9c4dd6b038..b11a9551d34 100644 --- a/src/TableFunctions/TableFunctionMerge.h +++ b/src/TableFunctions/TableFunctionMerge.h @@ -16,14 +16,8 @@ public: static constexpr auto name = "merge"; std::string getName() const override { return name; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; const char * getStorageTypeName() const override { return "Merge"; } - - ColumnsDescription getActualTableStructure(const Context & context) const override; - void parseArguments(const ASTPtr & ast_function, const Context & context) override; - - String source_database; - String table_name_regexp; }; diff --git a/src/TableFunctions/TableFunctionMySQL.cpp b/src/TableFunctions/TableFunctionMySQL.cpp index 7281ae434e5..05315d18f80 100644 --- a/src/TableFunctions/TableFunctionMySQL.cpp +++ b/src/TableFunctions/TableFunctionMySQL.cpp @@ -24,6 +24,8 @@ # include // for fetchTablesColumnsList +# include + namespace DB { @@ -36,7 +38,8 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; } -void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, const Context & context) + +StoragePtr TableFunctionMySQL::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { const auto & args_func = ast_function->as(); @@ -52,12 +55,14 @@ void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, const Conte for (auto & arg : args) arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); - String host_port = args[0]->as().value.safeGet(); - remote_database_name = args[1]->as().value.safeGet(); - remote_table_name = args[2]->as().value.safeGet(); - user_name = args[3]->as().value.safeGet(); - password = args[4]->as().value.safeGet(); + std::string host_port = args[0]->as().value.safeGet(); + std::string remote_database_name = args[1]->as().value.safeGet(); + std::string remote_table_name = args[2]->as().value.safeGet(); + std::string user_name = args[3]->as().value.safeGet(); + std::string password = args[4]->as().value.safeGet(); + bool replace_query = false; + std::string on_duplicate_clause; if (args.size() >= 6) replace_query = args[5]->as().value.safeGet() > 0; if (args.size() == 7) @@ -69,46 +74,27 @@ void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, const Conte ErrorCodes::BAD_ARGUMENTS); /// 3306 is the default MySQL port number - parsed_host_port = parseAddress(host_port, 3306); -} - -ColumnsDescription TableFunctionMySQL::getActualTableStructure(const Context & context) const -{ - assert(!parsed_host_port.first.empty()); - if (!pool) - pool.emplace(remote_database_name, parsed_host_port.first, user_name, password, parsed_host_port.second); + auto parsed_host_port = parseAddress(host_port, 3306); + mysqlxx::Pool pool(remote_database_name, parsed_host_port.first, user_name, password, parsed_host_port.second); const auto & settings = context.getSettingsRef(); - const auto tables_and_columns = fetchTablesColumnsList(*pool, remote_database_name, {remote_table_name}, settings.external_table_functions_use_nulls, settings.mysql_datatypes_support_level); + const auto tables_and_columns = fetchTablesColumnsList(pool, remote_database_name, {remote_table_name}, settings.external_table_functions_use_nulls, settings.mysql_datatypes_support_level); const auto columns = tables_and_columns.find(remote_table_name); if (columns == tables_and_columns.end()) throw Exception("MySQL table " + backQuoteIfNeed(remote_database_name) + "." + backQuoteIfNeed(remote_table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); - return ColumnsDescription{columns->second}; -} - -StoragePtr TableFunctionMySQL::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const -{ - assert(!parsed_host_port.first.empty()); - if (!pool) - pool.emplace(remote_database_name, parsed_host_port.first, user_name, password, parsed_host_port.second); - - auto columns = getActualTableStructure(context); - auto res = StorageMySQL::create( StorageID(getDatabaseName(), table_name), - std::move(*pool), + std::move(pool), remote_database_name, remote_table_name, replace_query, on_duplicate_clause, - columns, + ColumnsDescription{columns->second}, ConstraintsDescription{}, context); - pool.reset(); - res->startup(); return res; } diff --git a/src/TableFunctions/TableFunctionMySQL.h b/src/TableFunctions/TableFunctionMySQL.h index 1fe5a4aa4ac..850affc5887 100644 --- a/src/TableFunctions/TableFunctionMySQL.h +++ b/src/TableFunctions/TableFunctionMySQL.h @@ -1,11 +1,6 @@ #pragma once -#if !defined(ARCADIA_BUILD) -# include "config_core.h" -#endif -#if USE_MYSQL #include -#include namespace DB @@ -24,23 +19,8 @@ public: return name; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; const char * getStorageTypeName() const override { return "MySQL"; } - - ColumnsDescription getActualTableStructure(const Context & context) const override; - void parseArguments(const ASTPtr & ast_function, const Context & context) override; - - std::pair parsed_host_port; - String remote_database_name; - String remote_table_name; - String user_name; - String password; - bool replace_query = false; - String on_duplicate_clause; - - mutable std::optional pool; }; } - -#endif diff --git a/src/TableFunctions/TableFunctionNull.cpp b/src/TableFunctions/TableFunctionNull.cpp index 6abe0319394..00283554041 100644 --- a/src/TableFunctions/TableFunctionNull.cpp +++ b/src/TableFunctions/TableFunctionNull.cpp @@ -17,30 +17,23 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -void TableFunctionNull::parseArguments(const ASTPtr & ast_function, const Context & context) +StoragePtr TableFunctionNull::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { - const auto * function = ast_function->as(); - if (!function || !function->arguments) - throw Exception("Table function '" + getName() + "' requires 'structure'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + if (const auto * function = ast_function->as()) + { + auto arguments = function->arguments->children; - const auto & arguments = function->arguments->children; - if (arguments.size() != 1) - throw Exception("Table function '" + getName() + "' requires 'structure'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + if (arguments.size() != 1) + throw Exception("Table function '" + getName() + "' requires 'structure'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - structure = evaluateConstantExpressionOrIdentifierAsLiteral(arguments[0], context)->as()->value.safeGet(); -} + auto structure = evaluateConstantExpressionOrIdentifierAsLiteral(arguments[0], context)->as()->value.safeGet(); + ColumnsDescription columns = parseColumnsListFromString(structure, context); -ColumnsDescription TableFunctionNull::getActualTableStructure(const Context & context) const -{ - return parseColumnsListFromString(structure, context); -} - -StoragePtr TableFunctionNull::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const -{ - auto columns = getActualTableStructure(context); - auto res = StorageNull::create(StorageID(getDatabaseName(), table_name), columns, ConstraintsDescription()); - res->startup(); - return res; + auto res = StorageNull::create(StorageID(getDatabaseName(), table_name), columns, ConstraintsDescription()); + res->startup(); + return res; + } + throw Exception("Table function '" + getName() + "' requires 'structure'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } void registerTableFunctionNull(TableFunctionFactory & factory) diff --git a/src/TableFunctions/TableFunctionNull.h b/src/TableFunctions/TableFunctionNull.h index 4d4cecb0292..48617352b25 100644 --- a/src/TableFunctions/TableFunctionNull.h +++ b/src/TableFunctions/TableFunctionNull.h @@ -17,13 +17,8 @@ public: static constexpr auto name = "null"; std::string getName() const override { return name; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const String & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; const char * getStorageTypeName() const override { return "Null"; } - - void parseArguments(const ASTPtr & ast_function, const Context & context) override; - ColumnsDescription getActualTableStructure(const Context & context) const override; - - String structure; }; } diff --git a/src/TableFunctions/TableFunctionNumbers.cpp b/src/TableFunctions/TableFunctionNumbers.cpp index 4658165735a..c8c0fe96092 100644 --- a/src/TableFunctions/TableFunctionNumbers.cpp +++ b/src/TableFunctions/TableFunctionNumbers.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include "registerTableFunctions.h" @@ -19,16 +18,8 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } - template -ColumnsDescription TableFunctionNumbers::getActualTableStructure(const Context & /*context*/) const -{ - /// NOTE: https://bugs.llvm.org/show_bug.cgi?id=47418 - return ColumnsDescription{{{"number", std::make_shared()}}}; -} - -template -StoragePtr TableFunctionNumbers::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const +StoragePtr TableFunctionNumbers::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { if (const auto * function = ast_function->as()) { @@ -37,6 +28,7 @@ StoragePtr TableFunctionNumbers::executeImpl(const ASTPtr & ast_f if (arguments.size() != 1 && arguments.size() != 2) throw Exception("Table function '" + getName() + "' requires 'length' or 'offset, length'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + UInt64 offset = arguments.size() == 2 ? evaluateArgument(context, arguments[0]) : 0; UInt64 length = arguments.size() == 2 ? evaluateArgument(context, arguments[1]) : evaluateArgument(context, arguments[0]); diff --git a/src/TableFunctions/TableFunctionNumbers.h b/src/TableFunctions/TableFunctionNumbers.h index c27bb7319ba..13064a3309e 100644 --- a/src/TableFunctions/TableFunctionNumbers.h +++ b/src/TableFunctions/TableFunctionNumbers.h @@ -17,14 +17,11 @@ class TableFunctionNumbers : public ITableFunction public: static constexpr auto name = multithreaded ? "numbers_mt" : "numbers"; std::string getName() const override { return name; } - bool hasStaticStructure() const override { return true; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; const char * getStorageTypeName() const override { return "SystemNumbers"; } UInt64 evaluateArgument(const Context & context, ASTPtr & argument) const; - - ColumnsDescription getActualTableStructure(const Context & context) const override; }; diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 2e34e82ce36..7fa3868d142 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -27,8 +27,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } - -void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, const Context & context) +StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { ASTs & args_func = ast_function->children; @@ -45,6 +44,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, const Cont String cluster_description; String remote_database; String remote_table; + ASTPtr remote_table_function_ptr; String username; String password; @@ -136,6 +136,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, const Cont for (auto ast : args) setIdentifierSpecial(ast); + ClusterPtr cluster; if (!cluster_name.empty()) { /// Use an existing cluster from the main config @@ -188,54 +189,30 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, const Cont if (!remote_table_function_ptr && remote_table.empty()) throw Exception("The name of remote table cannot be empty", ErrorCodes::BAD_ARGUMENTS); + auto remote_table_id = StorageID::createEmpty(); remote_table_id.database_name = remote_database; remote_table_id.table_name = remote_table; -} + auto structure_remote_table = getStructureOfRemoteTable(*cluster, remote_table_id, context, remote_table_function_ptr); -StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const -{ - /// StorageDistributed supports mismatching structure of remote table, so we can use outdated structure for CREATE ... AS remote(...) - /// without additional conversion in StorageTableFunctionProxy - if (cached_columns.empty()) - cached_columns = getActualTableStructure(context); - - assert(cluster); StoragePtr res = remote_table_function_ptr - ? StorageDistributed::create( + ? StorageDistributed::createWithOwnCluster( StorageID(getDatabaseName(), table_name), - cached_columns, - ConstraintsDescription{}, + structure_remote_table, remote_table_function_ptr, - String{}, - context, - ASTPtr{}, - String{}, - String{}, - false, - cluster) - : StorageDistributed::create( + cluster, + context) + : StorageDistributed::createWithOwnCluster( StorageID(getDatabaseName(), table_name), - cached_columns, - ConstraintsDescription{}, - remote_table_id.database_name, - remote_table_id.table_name, - String{}, - context, - ASTPtr{}, - String{}, - String{}, - false, - cluster); + structure_remote_table, + remote_database, + remote_table, + cluster, + context); res->startup(); return res; } -ColumnsDescription TableFunctionRemote::getActualTableStructure(const Context & context) const -{ - assert(cluster); - return getStructureOfRemoteTable(*cluster, remote_table_id, context, remote_table_function_ptr); -} TableFunctionRemote::TableFunctionRemote(const std::string & name_, bool secure_) : name{name_}, secure{secure_} diff --git a/src/TableFunctions/TableFunctionRemote.h b/src/TableFunctions/TableFunctionRemote.h index 6ec591a34ac..2dd58a8a6a7 100644 --- a/src/TableFunctions/TableFunctionRemote.h +++ b/src/TableFunctions/TableFunctionRemote.h @@ -1,8 +1,6 @@ #pragma once #include -#include -#include namespace DB @@ -22,24 +20,14 @@ public: std::string getName() const override { return name; } - ColumnsDescription getActualTableStructure(const Context & context) const override; - - bool needStructureConversion() const override { return false; } - private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; const char * getStorageTypeName() const override { return "Distributed"; } - void parseArguments(const ASTPtr & ast_function, const Context & context) override; - std::string name; bool is_cluster_function; std::string help_message; bool secure; - - ClusterPtr cluster; - StorageID remote_table_id = StorageID::createEmpty(); - ASTPtr remote_table_function_ptr; }; } diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index dfe1cf6e792..1a0bff12ba2 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -21,7 +21,7 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -void TableFunctionS3::parseArguments(const ASTPtr & ast_function, const Context & context) +StoragePtr TableFunctionS3::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { /// Parse args ASTs & args_func = ast_function->children; @@ -38,7 +38,11 @@ void TableFunctionS3::parseArguments(const ASTPtr & ast_function, const Context for (auto & arg : args) arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); - filename = args[0]->as().value.safeGet(); + String filename = args[0]->as().value.safeGet(); + String format; + String structure; + String access_key_id; + String secret_access_key; if (args.size() < 5) { @@ -53,38 +57,47 @@ void TableFunctionS3::parseArguments(const ASTPtr & ast_function, const Context structure = args[4]->as().value.safeGet(); } + String compression_method; if (args.size() == 4 || args.size() == 6) compression_method = args.back()->as().value.safeGet(); -} + else + compression_method = "auto"; -ColumnsDescription TableFunctionS3::getActualTableStructure(const Context & context) const -{ - return parseColumnsListFromString(structure, context); -} + ColumnsDescription columns = parseColumnsListFromString(structure, context); -StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const -{ - Poco::URI uri (filename); - S3::URI s3_uri (uri); - UInt64 min_upload_part_size = context.getSettingsRef().s3_min_upload_part_size; - - StoragePtr storage = StorageS3::create( - s3_uri, - access_key_id, - secret_access_key, - StorageID(getDatabaseName(), table_name), - format, - min_upload_part_size, - getActualTableStructure(context), - ConstraintsDescription{}, - const_cast(context), - compression_method); + /// Create table + StoragePtr storage = getStorage(filename, access_key_id, secret_access_key, format, columns, const_cast(context), table_name, compression_method); storage->startup(); return storage; } +StoragePtr TableFunctionS3::getStorage( + const String & source, + const String & access_key_id, + const String & secret_access_key, + const String & format, + const ColumnsDescription & columns, + Context & global_context, + const std::string & table_name, + const String & compression_method) +{ + Poco::URI uri (source); + S3::URI s3_uri (uri); + UInt64 min_upload_part_size = global_context.getSettingsRef().s3_min_upload_part_size; + return StorageS3::create( + s3_uri, + access_key_id, + secret_access_key, + StorageID(getDatabaseName(), table_name), + format, + min_upload_part_size, + columns, + ConstraintsDescription{}, + global_context, + compression_method); +} void registerTableFunctionS3(TableFunctionFactory & factory) { diff --git a/src/TableFunctions/TableFunctionS3.h b/src/TableFunctions/TableFunctionS3.h index 722fb9eb23c..b66da57d4fa 100644 --- a/src/TableFunctions/TableFunctionS3.h +++ b/src/TableFunctions/TableFunctionS3.h @@ -22,26 +22,24 @@ public: { return name; } - bool hasStaticStructure() const override { return true; } protected: StoragePtr executeImpl( const ASTPtr & ast_function, const Context & context, + const std::string & table_name) const override; + + static StoragePtr getStorage( + const String & source, + const String & access_key_id, + const String & secret_access_key, + const String & format, + const ColumnsDescription & columns, + Context & global_context, const std::string & table_name, - ColumnsDescription cached_columns) const override; + const String & compression_method); const char * getStorageTypeName() const override { return "S3"; } - - ColumnsDescription getActualTableStructure(const Context & context) const override; - void parseArguments(const ASTPtr & ast_function, const Context & context) override; - - String filename; - String format; - String structure; - String access_key_id; - String secret_access_key; - String compression_method = "auto"; }; class TableFunctionCOS : public TableFunctionS3 diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index 6139e6ffecb..59978ae08b1 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -10,12 +10,10 @@ namespace DB { StoragePtr TableFunctionURL::getStorage( - const String & source, const String & format_, const ColumnsDescription & columns, Context & global_context, - const std::string & table_name, const String & compression_method_) const + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const { Poco::URI uri(source); - return StorageURL::create(uri, StorageID(getDatabaseName(), table_name), format_, columns, ConstraintsDescription{}, - global_context, compression_method_); + return StorageURL::create(uri, StorageID(getDatabaseName(), table_name), format, columns, ConstraintsDescription{}, global_context, compression_method); } void registerTableFunctionURL(TableFunctionFactory & factory) diff --git a/src/TableFunctions/TableFunctionURL.h b/src/TableFunctions/TableFunctionURL.h index 5eb027e2b8a..366d573bcf4 100644 --- a/src/TableFunctions/TableFunctionURL.h +++ b/src/TableFunctions/TableFunctionURL.h @@ -21,8 +21,7 @@ public: private: StoragePtr getStorage( - const String & source, const String & format_, const ColumnsDescription & columns, Context & global_context, - const std::string & table_name, const String & compression_method_) const override; + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const override; const char * getStorageTypeName() const override { return "URL"; } }; diff --git a/src/TableFunctions/TableFunctionValues.cpp b/src/TableFunctions/TableFunctionValues.cpp index 2d01d581fda..b4b243416f2 100644 --- a/src/TableFunctions/TableFunctionValues.cpp +++ b/src/TableFunctions/TableFunctionValues.cpp @@ -62,10 +62,8 @@ static void parseAndInsertValues(MutableColumns & res_columns, const ASTs & args } } -void TableFunctionValues::parseArguments(const ASTPtr & ast_function, const Context & /*context*/) +StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { - - ASTs & args_func = ast_function->children; if (args_func.size() != 1) @@ -85,18 +83,9 @@ void TableFunctionValues::parseArguments(const ASTPtr & ast_function, const Cont "Got '{}' instead", getName(), args[0]->formatForErrorMessage()), ErrorCodes::BAD_ARGUMENTS); } + std::string structure = args[0]->as().value.safeGet(); - structure = args[0]->as().value.safeGet(); -} - -ColumnsDescription TableFunctionValues::getActualTableStructure(const Context & context) const -{ - return parseColumnsListFromString(structure, context); -} - -StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const -{ - auto columns = getActualTableStructure(context); + ColumnsDescription columns = parseColumnsListFromString(structure, context); Block sample_block; for (const auto & name_type : columns.getOrdinary()) @@ -104,8 +93,6 @@ StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, const C MutableColumns res_columns = sample_block.cloneEmptyColumns(); - ASTs & args = ast_function->children.at(0)->children; - /// Parsing other arguments as values and inserting them into columns parseAndInsertValues(res_columns, args, sample_block, context); diff --git a/src/TableFunctions/TableFunctionValues.h b/src/TableFunctions/TableFunctionValues.h index 549fa2de507..3cc3687dab5 100644 --- a/src/TableFunctions/TableFunctionValues.h +++ b/src/TableFunctions/TableFunctionValues.h @@ -12,15 +12,9 @@ class TableFunctionValues : public ITableFunction public: static constexpr auto name = "values"; std::string getName() const override { return name; } - bool hasStaticStructure() const override { return true; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; const char * getStorageTypeName() const override { return "Values"; } - - ColumnsDescription getActualTableStructure(const Context & context) const override; - void parseArguments(const ASTPtr & ast_function, const Context & context) override; - - String structure; }; diff --git a/src/TableFunctions/TableFunctionView.cpp b/src/TableFunctions/TableFunctionView.cpp index 5ec06592103..8d3f7b06fa3 100644 --- a/src/TableFunctions/TableFunctionView.cpp +++ b/src/TableFunctions/TableFunctionView.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -15,37 +16,24 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -void TableFunctionView::parseArguments(const ASTPtr & ast_function, const Context & /*context*/) +StoragePtr TableFunctionView::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { - const auto * function = ast_function->as(); - if (function) + if (const auto * function = ast_function->as()) { if (auto * select = function->tryGetQueryArgument()) { - create.set(create.select, select->clone()); - return; + auto sample = InterpreterSelectWithUnionQuery::getSampleBlock(function->arguments->children[0] /* ASTPtr */, context); + auto columns = ColumnsDescription(sample.getNamesAndTypesList()); + ASTCreateQuery create; + create.select = select; + auto res = StorageView::create(StorageID(getDatabaseName(), table_name), create, columns); + res->startup(); + return res; } } throw Exception("Table function '" + getName() + "' requires a query argument.", ErrorCodes::BAD_ARGUMENTS); } -ColumnsDescription TableFunctionView::getActualTableStructure(const Context & context) const -{ - assert(create.select); - assert(create.children.size() == 1); - assert(create.children[0]->as()); - auto sample = InterpreterSelectWithUnionQuery::getSampleBlock(create.children[0], context); - return ColumnsDescription(sample.getNamesAndTypesList()); -} - -StoragePtr TableFunctionView::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const -{ - auto columns = getActualTableStructure(context); - auto res = StorageView::create(StorageID(getDatabaseName(), table_name), create, columns); - res->startup(); - return res; -} - void registerTableFunctionView(TableFunctionFactory & factory) { factory.registerFunction(); diff --git a/src/TableFunctions/TableFunctionView.h b/src/TableFunctions/TableFunctionView.h index 0ed66ff712c..43d55a7dcb8 100644 --- a/src/TableFunctions/TableFunctionView.h +++ b/src/TableFunctions/TableFunctionView.h @@ -1,9 +1,9 @@ #pragma once #include -#include #include + namespace DB { @@ -17,13 +17,10 @@ public: static constexpr auto name = "view"; std::string getName() const override { return name; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const String & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; const char * getStorageTypeName() const override { return "View"; } - void parseArguments(const ASTPtr & ast_function, const Context & context) override; - ColumnsDescription getActualTableStructure(const Context & context) const override; - - ASTCreateQuery create; + UInt64 evaluateArgument(const Context & context, ASTPtr & argument) const; }; diff --git a/src/TableFunctions/TableFunctionZeros.cpp b/src/TableFunctions/TableFunctionZeros.cpp index 9b0c6c6e78b..13436f04e1c 100644 --- a/src/TableFunctions/TableFunctionZeros.cpp +++ b/src/TableFunctions/TableFunctionZeros.cpp @@ -3,8 +3,8 @@ #include #include #include +#include #include -#include #include #include #include "registerTableFunctions.h" @@ -18,16 +18,8 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } - template -ColumnsDescription TableFunctionZeros::getActualTableStructure(const Context & /*context*/) const -{ - /// NOTE: https://bugs.llvm.org/show_bug.cgi?id=47418 - return ColumnsDescription{{{"zero", std::make_shared()}}}; -} - -template -StoragePtr TableFunctionZeros::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const +StoragePtr TableFunctionZeros::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { if (const auto * function = ast_function->as()) { diff --git a/src/TableFunctions/TableFunctionZeros.h b/src/TableFunctions/TableFunctionZeros.h index 48a2d8019f6..cae4604b1d7 100644 --- a/src/TableFunctions/TableFunctionZeros.h +++ b/src/TableFunctions/TableFunctionZeros.h @@ -17,14 +17,11 @@ class TableFunctionZeros : public ITableFunction public: static constexpr auto name = multithreaded ? "zeros_mt" : "zeros"; std::string getName() const override { return name; } - bool hasStaticStructure() const override { return true; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; const char * getStorageTypeName() const override { return "SystemZeros"; } UInt64 evaluateArgument(const Context & context, ASTPtr & argument) const; - - ColumnsDescription getActualTableStructure(const Context & context) const override; }; diff --git a/tests/config/lsan_suppressions.txt b/tests/config/lsan_suppressions.txt deleted file mode 100644 index 39eb40560d7..00000000000 --- a/tests/config/lsan_suppressions.txt +++ /dev/null @@ -1,2 +0,0 @@ -# See https://bugs.llvm.org/show_bug.cgi?id=47418 -# leak:getActualTableStructure diff --git a/tests/integration/test_distributed_format/test.py b/tests/integration/test_distributed_format/test.py index 607154e37f1..7e9d740c171 100644 --- a/tests/integration/test_distributed_format/test.py +++ b/tests/integration/test_distributed_format/test.py @@ -34,7 +34,7 @@ def test_single_file(started_cluster, cluster): assert out == '1\ta\n2\tbb\n3\tccc\n' - query = "create table t (x UInt64, s String) engine = File('Distributed', '/var/lib/clickhouse/data/test/distr_1/shard1_replica1/1.bin');" \ + query = "create table t (dummy UInt32) engine = File('Distributed', '/var/lib/clickhouse/data/test/distr_1/shard1_replica1/1.bin');" \ "select * from t" out = node.exec_in_container(['/usr/bin/clickhouse', 'local', '--stacktrace', '-q', query]) @@ -57,7 +57,7 @@ def test_two_files(started_cluster, cluster): assert out == '0\t_\n1\ta\n2\tbb\n3\tccc\n' - query = "create table t (x UInt64, s String) engine = File('Distributed', '/var/lib/clickhouse/data/test/distr_2/shard1_replica1/{1,2,3,4}.bin');" \ + query = "create table t (dummy UInt32) engine = File('Distributed', '/var/lib/clickhouse/data/test/distr_2/shard1_replica1/{1,2,3,4}.bin');" \ "select * from t order by x" out = node.exec_in_container(['/usr/bin/clickhouse', 'local', '--stacktrace', '-q', query]) @@ -77,7 +77,7 @@ def test_single_file_old(started_cluster, cluster): assert out == '1\ta\n2\tbb\n3\tccc\n' - query = "create table t (x UInt64, s String) engine = File('Distributed', '/var/lib/clickhouse/data/test/distr_3/default@not_existing:9000/1.bin');" \ + query = "create table t (dummy UInt32) engine = File('Distributed', '/var/lib/clickhouse/data/test/distr_3/default@not_existing:9000/1.bin');" \ "select * from t" out = node.exec_in_container(['/usr/bin/clickhouse', 'local', '--stacktrace', '-q', query]) diff --git a/tests/queries/0_stateless/01083_expressions_in_engine_arguments.reference b/tests/queries/0_stateless/01083_expressions_in_engine_arguments.reference index 2a5d7e6da32..933fda604d7 100644 --- a/tests/queries/0_stateless/01083_expressions_in_engine_arguments.reference +++ b/tests/queries/0_stateless/01083_expressions_in_engine_arguments.reference @@ -1,11 +1,11 @@ CREATE TABLE default.file\n(\n `n` Int8\n)\nENGINE = File(\'TSVWithNamesAndTypes\') CREATE TABLE default.buffer\n(\n `n` Int8\n)\nENGINE = Buffer(\'default\', \'file\', 16, 10, 200, 10000, 1000000, 10000000, 1000000000) CREATE TABLE default.merge\n(\n `n` Int8\n)\nENGINE = Merge(\'default\', \'distributed\') -CREATE TABLE default.merge_tf\n(\n `n` Int8\n) AS merge(\'default\', \'.*\') +CREATE TABLE default.merge_tf AS merge(\'default\', \'.*\') CREATE TABLE default.distributed\n(\n `n` Int8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'default\', \'file\') -CREATE TABLE default.distributed_tf\n(\n `n` Int8\n) AS cluster(\'test_shard_localhost\', \'default\', \'buffer\') +CREATE TABLE default.distributed_tf AS cluster(\'test_shard_localhost\', \'default\', \'buffer\') CREATE TABLE default.url\n(\n `n` UInt64,\n `col` String\n)\nENGINE = URL(\'https://localhost:8443/?query=select+n,+_table+from+default.merge+format+CSV\', \'CSV\') -CREATE TABLE default.rich_syntax\n(\n `n` Int64\n) AS remote(\'localhos{x|y|t}\', cluster(\'test_shard_localhost\', remote(\'127.0.0.{1..4}\', \'default\', \'view\'))) +CREATE TABLE default.rich_syntax AS remote(\'localhos{x|y|t}\', cluster(\'test_shard_localhost\', remote(\'127.0.0.{1..4}\', \'default\', \'view\'))) CREATE VIEW default.view\n(\n `n` Int64\n) AS\nSELECT toInt64(n) AS n\nFROM \n(\n SELECT toString(n) AS n\n FROM default.merge\n WHERE _table != \'qwerty\'\n ORDER BY _table ASC\n)\nUNION ALL\nSELECT *\nFROM default.file CREATE DICTIONARY default.dict\n(\n `n` UInt64,\n `col` String DEFAULT \'42\'\n)\nPRIMARY KEY n\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9440 SECURE 1 USER \'default\' TABLE \'url\'))\nLIFETIME(MIN 0 MAX 1)\nLAYOUT(CACHE(SIZE_IN_CELLS 1)) 16 diff --git a/tests/queries/0_stateless/01457_create_as_table_function_structure.reference b/tests/queries/0_stateless/01457_create_as_table_function_structure.reference deleted file mode 100644 index 0deaa20171a..00000000000 --- a/tests/queries/0_stateless/01457_create_as_table_function_structure.reference +++ /dev/null @@ -1,10 +0,0 @@ -CREATE TABLE test_01457.tf_remote\n(\n `n` Int8\n) AS remote(\'localhost\', \'default\', \'tmp\') -CREATE TABLE test_01457.tf_remote_explicit_structure\n(\n `n` UInt64\n) AS remote(\'localhost\', \'default\', \'tmp\') -CREATE TABLE test_01457.tf_numbers\n(\n `number` String\n) AS numbers(1) -CREATE TABLE test_01457.tf_merge\n(\n `n` Int8\n) AS merge(\'default\', \'tmp\') -42 -0 Int8 -0 Int8 -0 UInt64 -0 String -0 Int8 diff --git a/tests/queries/0_stateless/01457_create_as_table_function_structure.sql b/tests/queries/0_stateless/01457_create_as_table_function_structure.sql deleted file mode 100644 index edfab747fc4..00000000000 --- a/tests/queries/0_stateless/01457_create_as_table_function_structure.sql +++ /dev/null @@ -1,33 +0,0 @@ -DROP DATABASE IF EXISTS test_01457; - -CREATE DATABASE test_01457; - -CREATE TABLE tmp (n Int8) ENGINE=Memory; - -CREATE TABLE test_01457.tf_remote AS remote('localhost', currentDatabase(), 'tmp'); -SHOW CREATE TABLE test_01457.tf_remote; -CREATE TABLE test_01457.tf_remote_explicit_structure (n UInt64) AS remote('localhost', currentDatabase(), 'tmp'); -SHOW CREATE TABLE test_01457.tf_remote_explicit_structure; -CREATE TABLE test_01457.tf_numbers (number String) AS numbers(1); -SHOW CREATE TABLE test_01457.tf_numbers; -CREATE TABLE test_01457.tf_merge AS merge(currentDatabase(), 'tmp'); -SHOW CREATE TABLE test_01457.tf_merge; - -DROP TABLE tmp; - -DETACH DATABASE test_01457; -ATTACH DATABASE test_01457; - -CREATE TABLE tmp (n Int8) ENGINE=Memory; -INSERT INTO test_01457.tf_remote_explicit_structure VALUES ('42'); -SELECT * FROM tmp; -TRUNCATE TABLE tmp; -INSERT INTO test_01457.tf_remote VALUES (0); - -SELECT (*,).1 AS c, toTypeName(c) FROM tmp; -SELECT (*,).1 AS c, toTypeName(c) FROM test_01457.tf_remote; -SELECT (*,).1 AS c, toTypeName(c) FROM test_01457.tf_remote_explicit_structure; -SELECT (*,).1 AS c, toTypeName(c) FROM test_01457.tf_numbers; -SELECT (*,).1 AS c, toTypeName(c) FROM test_01457.tf_merge; - -DROP DATABASE test_01457; diff --git a/tests/queries/0_stateless/01461_alter_table_function.reference b/tests/queries/0_stateless/01461_alter_table_function.reference index b552dd81b77..395155967a9 100644 --- a/tests/queries/0_stateless/01461_alter_table_function.reference +++ b/tests/queries/0_stateless/01461_alter_table_function.reference @@ -1,6 +1,7 @@ -CREATE TABLE default.table_from_remote\n(\n `number` UInt64\n) AS remote(\'localhost\', \'system\', \'numbers\') -CREATE TABLE default.table_from_remote\n(\n `number` UInt64,\n `col` UInt8\n) AS remote(\'localhost\', \'system\', \'numbers\') -CREATE TABLE default.table_from_numbers\n(\n `number` UInt64\n) AS numbers(1000) -CREATE TABLE default.table_from_numbers\n(\n `number` UInt64\n) AS numbers(1000) +CREATE TABLE default.table_from_remote AS remote(\'localhost\', \'system\', \'numbers\') +CREATE TABLE default.table_from_remote AS remote(\'localhost\', \'system\', \'numbers\') +CREATE TABLE default.table_from_numbers AS numbers(1000) +CREATE TABLE default.table_from_numbers AS numbers(1000) CREATE TABLE default.table_from_select\n(\n `number` UInt64\n)\nENGINE = MergeTree()\nORDER BY tuple()\nSETTINGS index_granularity = 8192 CREATE TABLE default.table_from_select\n(\n `number` UInt64,\n `col` UInt8\n)\nENGINE = MergeTree()\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +1 diff --git a/tests/queries/0_stateless/01461_alter_table_function.sql b/tests/queries/0_stateless/01461_alter_table_function.sql index 11f643f1e3e..e242d1f0b7b 100644 --- a/tests/queries/0_stateless/01461_alter_table_function.sql +++ b/tests/queries/0_stateless/01461_alter_table_function.sql @@ -6,7 +6,7 @@ CREATE TABLE table_from_remote AS remote('localhost', 'system', 'numbers'); SHOW CREATE TABLE table_from_remote; -ALTER TABLE table_from_remote ADD COLUMN col UInt8; +ALTER TABLE table_from_remote ADD COLUMN col UInt8; --{serverError 48} SHOW CREATE TABLE table_from_remote; @@ -26,6 +26,8 @@ ALTER TABLE table_from_select ADD COLUMN col UInt8; SHOW CREATE TABLE table_from_select; +SELECT 1; + DROP TABLE IF EXISTS table_from_remote; DROP TABLE IF EXISTS table_from_select; DROP TABLE IF EXISTS table_from_numbers; diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index ec8dbf7a6d4..78dfbb2e9e1 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -141,7 +141,6 @@ 01460_DistributedFilesToInsert 01474_executable_dictionary 01474_bad_global_join -01457_create_as_table_function_structure 01473_event_time_microseconds 01461_query_start_time_microseconds 01455_shard_leaf_max_rows_bytes_to_read From b1cdcf2d31deba51052948d5b59dbbdfd450dcb3 Mon Sep 17 00:00:00 2001 From: AnaUvarova <64017504+AnaUvarova@users.noreply.github.com> Date: Wed, 14 Oct 2020 14:33:23 +0300 Subject: [PATCH 42/42] DOCSUP-2018-formatDateTime_description_update: added two modificators into the table of supported modificators (#15614) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Added two modificators into the table of supported modificators * Clarified description of added modifiers and started to add info according to the template of function description * fixed link * fix link again * edits * Update docs/en/sql-reference/functions/date-time-functions.md Co-authored-by: BayoNet * Update docs/en/sql-reference/functions/date-time-functions.md Co-authored-by: BayoNet * Update docs/en/sql-reference/functions/date-time-functions.md Co-authored-by: BayoNet * comments from ticket * + * + * polishing * Update docs/en/sql-reference/functions/date-time-functions.md Co-authored-by: BayoNet * external link for understanding * Перевод на русский * Доработала русский перевод * Причесала перевод * Уточнение части перевода * Шлифовка части перевода Co-authored-by: BayoNet --- .../functions/date-time-functions.md | 36 +++++++++++++++-- .../functions/date-time-functions.md | 40 ++++++++++++++++--- 2 files changed, 66 insertions(+), 10 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index c4186c47f45..15214cc4c00 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -486,20 +486,32 @@ For a time interval starting at ‘StartTime’ and continuing for ‘Duration For example, `timeSlots(toDateTime('2012-01-01 12:20:00'), 600) = [toDateTime('2012-01-01 12:00:00'), toDateTime('2012-01-01 12:30:00')]`. This is necessary for searching for pageviews in the corresponding session. -## formatDateTime(Time, Format\[, Timezone\]) {#formatdatetime} +## formatDateTime {#formatdatetime} Function formats a Time according given Format string. N.B.: Format is a constant expression, e.g. you can not have multiple formats for single result column. -Supported modifiers for Format: -(“Example” column shows formatting result for time `2018-01-02 22:33:44`) +**Syntax** -| Modifier | Description | Example | +``` sql +formatDateTime(Time, Format\[, Timezone\]) +``` + +**Returned value(s)** + +Returnes time and date values according to the determined format. + +**Replacement fields** +Using replacement fields, you can define a pattern for the resulting string. “Example” column shows formatting result for `2018-01-02 22:33:44`. + +| Placeholder | Description | Example | |----------|---------------------------------------------------------|------------| | %C | year divided by 100 and truncated to integer (00-99) | 20 | | %d | day of the month, zero-padded (01-31) | 02 | | %D | Short MM/DD/YY date, equivalent to %m/%d/%y | 01/02/18 | | %e | day of the month, space-padded ( 1-31) | 2 | | %F | short YYYY-MM-DD date, equivalent to %Y-%m-%d | 2018-01-02 | +| %G | four-digit year format for ISO week number, calculated from the week-based year [defined by the ISO 8601](https://en.wikipedia.org/wiki/ISO_8601#Week_dates) standard, normally useful only with %V | 2018 | +| %g | two-digit year format, aligned to ISO 8601, abbreviated from four-digit notation | 18 | | %H | hour in 24h format (00-23) | 22 | | %I | hour in 12h format (01-12) | 10 | | %j | day of the year (001-366) | 002 | @@ -518,6 +530,22 @@ Supported modifiers for Format: | %Y | Year | 2018 | | %% | a % sign | % | +**Example** + +Query: + +``` sql +SELECT formatDateTime(toDate('2010-01-04'), '%g') +``` + +Result: + +``` +┌─formatDateTime(toDate('2010-01-04'), '%g')─┐ +│ 10 │ +└────────────────────────────────────────────┘ +``` + [Original article](https://clickhouse.tech/docs/en/query_language/functions/date_time_functions/) ## FROM_UNIXTIME diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index cdc869190e4..aa9fdee478d 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -311,20 +311,32 @@ SELECT dateDiff('hour', toDateTime('2018-01-01 22:00:00'), toDateTime('2018-01-0 Например, `timeSlots(toDateTime('2012-01-01 12:20:00'), toUInt32(600)) = [toDateTime('2012-01-01 12:00:00'), toDateTime('2012-01-01 12:30:00')]`. Это нужно для поиска хитов, входящих в соответствующий визит. -## formatDateTime(Time, Format\[, Timezone\]) {#formatdatetime} +## formatDateTime {#formatdatetime} -Функция преобразования даты-с-временем в String согласно заданному шаблону. Важно - шаблон является константным выражением, т.е. невозможно использование разных шаблонов в одной колонке. +Функция преобразует дату-и-время в строку по заданному шаблону. Важно: шаблон — константное выражение, поэтому использовать разные шаблоны в одной колонке не получится. -Поддерживаемые модификаторы в шаблоне Format: -(колонка «Пример» показана для времени `2018-01-02 22:33:44`) +**Синтаксис** -| Модификатор | Описание | Пример | +``` sql +formatDateTime(Time, Format\[, Timezone\]) +``` + +**Возвращаемое значение** + +Возвращает значение времени и даты в определенном вами формате. + +**Поля подстановки** +Используйте поля подстановки для того, чтобы определить шаблон для выводимой строки. В колонке «Пример» результат работы функции для времени `2018-01-02 22:33:44`. + +| Поле | Описание | Пример | |-------------|----------------------------------------------------------------------|------------| | %C | номер года, поделённый на 100 (00-99) | 20 | | %d | день месяца, с ведущим нулём (01-31) | 02 | | %D | короткая запись %m/%d/%y | 01/02/18 | | %e | день месяца, с ведущим пробелом ( 1-31) | 2 | | %F | короткая запись %Y-%m-%d | 2018-01-02 | +| %G | четырехзначный формат вывода ISO-года, который основывается на особом подсчете номера недели согласно [стандарту ISO 8601](https://ru.wikipedia.org/wiki/ISO_8601), обычно используется вместе с %V | 2018 | +| %g | двузначный формат вывода года по стандарту ISO 8601 | 18 | | %H | час в 24-часовом формате (00-23) | 22 | | %I | час в 12-часовом формате (01-12) | 10 | | %j | номер дня в году, с ведущими нулями (001-366) | 002 | @@ -343,4 +355,20 @@ SELECT dateDiff('hour', toDateTime('2018-01-01 22:00:00'), toDateTime('2018-01-0 | %Y | год, 4 цифры | 2018 | | %% | символ % | % | -[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/functions/date_time_functions/) +**Пример** + +Запрос: + +``` sql +SELECT formatDateTime(toDate('2010-01-04'), '%g') +``` + +Ответ: + +``` +┌─formatDateTime(toDate('2010-01-04'), '%g')─┐ +│ 10 │ +└────────────────────────────────────────────┘ +``` + +[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/functions/date_time_functions/) \ No newline at end of file