From 9120d5f38e81188be9e81719ac64e1dea530f5d2 Mon Sep 17 00:00:00 2001 From: chertus Date: Fri, 12 Oct 2018 18:41:28 +0300 Subject: [PATCH 01/28] extract visitor from ExpressionAnalyser [CLICKHOUSE-3996] --- dbms/src/Interpreters/ActionsVisitor.cpp | 622 +++++++++++++++ dbms/src/Interpreters/ActionsVisitor.h | 128 +++ dbms/src/Interpreters/ExpressionAnalyzer.cpp | 733 +----------------- dbms/src/Interpreters/ExpressionAnalyzer.h | 80 +- .../Interpreters/ProjectionManipulation.cpp | 2 +- .../src/Interpreters/ProjectionManipulation.h | 7 +- dbms/src/Interpreters/interpretSubquery.cpp | 130 ++++ dbms/src/Interpreters/interpretSubquery.h | 14 + 8 files changed, 934 insertions(+), 782 deletions(-) create mode 100644 dbms/src/Interpreters/ActionsVisitor.cpp create mode 100644 dbms/src/Interpreters/ActionsVisitor.h create mode 100644 dbms/src/Interpreters/interpretSubquery.cpp create mode 100644 dbms/src/Interpreters/interpretSubquery.h diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp new file mode 100644 index 00000000000..bde379fa69b --- /dev/null +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -0,0 +1,622 @@ +#include +#include + +#include + +#include +#include +#include +#include +#include +#include + +#include + +#include +#include +#include + +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_IDENTIFIER; + extern const int NOT_AN_AGGREGATE; + extern const int UNEXPECTED_EXPRESSION; + extern const int TYPE_MISMATCH; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +/// defined in ExpressionAnalyser.cpp +NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypesList & cols); + + +void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool create_ordered_set, + const Context & context, const SizeLimits & size_limits, PreparedSets & prepared_sets) +{ + const IAST & args = *node->arguments; + + if (args.children.size() != 2) + throw Exception("Wrong number of arguments passed to function in", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const ASTPtr & left_arg = args.children.at(0); + const ASTPtr & right_arg = args.children.at(1); + + auto getTupleTypeFromAst = [&context](const ASTPtr & tuple_ast) -> DataTypePtr + { + auto ast_function = typeid_cast(tuple_ast.get()); + if (ast_function && ast_function->name == "tuple" && !ast_function->arguments->children.empty()) + { + /// Won't parse all values of outer tuple. + auto element = ast_function->arguments->children.at(0); + std::pair value_raw = evaluateConstantExpression(element, context); + return std::make_shared(DataTypes({value_raw.second})); + } + + return evaluateConstantExpression(tuple_ast, context).second; + }; + + const DataTypePtr & left_arg_type = sample_block.getByName(left_arg->getColumnName()).type; + const DataTypePtr & right_arg_type = getTupleTypeFromAst(right_arg); + + std::function getTupleDepth; + getTupleDepth = [&getTupleDepth](const DataTypePtr & type) -> size_t + { + if (auto tuple_type = typeid_cast(type.get())) + return 1 + (tuple_type->getElements().empty() ? 0 : getTupleDepth(tuple_type->getElements().at(0))); + + return 0; + }; + + size_t left_tuple_depth = getTupleDepth(left_arg_type); + size_t right_tuple_depth = getTupleDepth(right_arg_type); + + DataTypes set_element_types = {left_arg_type}; + auto left_tuple_type = typeid_cast(left_arg_type.get()); + if (left_tuple_type && left_tuple_type->getElements().size() != 1) + set_element_types = left_tuple_type->getElements(); + + for (auto & element_type : set_element_types) + if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + element_type = low_cardinality_type->getDictionaryType(); + + ASTPtr elements_ast = nullptr; + + /// 1 in 1; (1, 2) in (1, 2); identity(tuple(tuple(tuple(1)))) in tuple(tuple(tuple(1))); etc. + if (left_tuple_depth == right_tuple_depth) + { + ASTPtr exp_list = std::make_shared(); + exp_list->children.push_back(right_arg); + elements_ast = exp_list; + } + /// 1 in (1, 2); (1, 2) in ((1, 2), (3, 4)); etc. + else if (left_tuple_depth + 1 == right_tuple_depth) + { + ASTFunction * set_func = typeid_cast(right_arg.get()); + + if (!set_func || set_func->name != "tuple") + throw Exception("Incorrect type of 2nd argument for function " + node->name + + ". Must be subquery or set of elements with type " + left_arg_type->getName() + ".", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + elements_ast = set_func->arguments; + } + else + throw Exception("Invalid types for IN function: " + + left_arg_type->getName() + " and " + right_arg_type->getName() + ".", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + SetPtr set = std::make_shared(size_limits, create_ordered_set); + set->createFromAST(set_element_types, elements_ast, context); + prepared_sets[right_arg->range] = std::move(set); +} + +static String getUniqueName(const Block & block, const String & prefix) +{ + int i = 1; + while (block.has(prefix + toString(i))) + ++i; + return prefix + toString(i); +} + +ScopeStack::ScopeStack(const ExpressionActionsPtr & actions, const Context & context_) + : context(context_) +{ + stack.emplace_back(); + stack.back().actions = actions; + + const Block & sample_block = actions->getSampleBlock(); + for (size_t i = 0, size = sample_block.columns(); i < size; ++i) + stack.back().new_columns.insert(sample_block.getByPosition(i).name); +} + +void ScopeStack::pushLevel(const NamesAndTypesList & input_columns) +{ + stack.emplace_back(); + Level & prev = stack[stack.size() - 2]; + + ColumnsWithTypeAndName all_columns; + NameSet new_names; + + for (NamesAndTypesList::const_iterator it = input_columns.begin(); it != input_columns.end(); ++it) + { + all_columns.emplace_back(nullptr, it->type, it->name); + new_names.insert(it->name); + stack.back().new_columns.insert(it->name); + } + + const Block & prev_sample_block = prev.actions->getSampleBlock(); + for (size_t i = 0, size = prev_sample_block.columns(); i < size; ++i) + { + const ColumnWithTypeAndName & col = prev_sample_block.getByPosition(i); + if (!new_names.count(col.name)) + all_columns.push_back(col); + } + + stack.back().actions = std::make_shared(all_columns, context); +} + +size_t ScopeStack::getColumnLevel(const std::string & name) +{ + for (int i = static_cast(stack.size()) - 1; i >= 0; --i) + if (stack[i].new_columns.count(name)) + return i; + + throw Exception("Unknown identifier: " + name, ErrorCodes::UNKNOWN_IDENTIFIER); +} + +void ScopeStack::addAction(const ExpressionAction & action) +{ + size_t level = 0; + Names required = action.getNeededColumns(); + for (size_t i = 0; i < required.size(); ++i) + level = std::max(level, getColumnLevel(required[i])); + + Names added; + stack[level].actions->add(action, added); + + stack[level].new_columns.insert(added.begin(), added.end()); + + for (size_t i = 0; i < added.size(); ++i) + { + const ColumnWithTypeAndName & col = stack[level].actions->getSampleBlock().getByName(added[i]); + for (size_t j = level + 1; j < stack.size(); ++j) + stack[j].actions->addInput(col); + } +} + +ExpressionActionsPtr ScopeStack::popLevel() +{ + ExpressionActionsPtr res = stack.back().actions; + stack.pop_back(); + return res; +} + +const Block & ScopeStack::getSampleBlock() const +{ + return stack.back().actions->getSampleBlock(); +} + + +void ActionsVisitor::visit(const ASTPtr & ast, ScopeStack & actions_stack, ProjectionManipulatorPtr projection_manipulator) +{ + DumpASTNode dump(*ast, ostr, visit_depth, "getActions"); + + String ast_column_name; + auto getColumnName = [&ast, &ast_column_name]() + { + if (ast_column_name.empty()) + ast_column_name = ast->getColumnName(); + + return ast_column_name; + }; + + /// If the result of the calculation already exists in the block. + if ((typeid_cast(ast.get()) || typeid_cast(ast.get())) + && projection_manipulator->tryToGetFromUpperProjection(getColumnName())) + return; + + if (typeid_cast(ast.get())) + { + if (!only_consts && !projection_manipulator->tryToGetFromUpperProjection(getColumnName())) + { + /// The requested column is not in the block. + /// If such a column exists in the table, then the user probably forgot to surround it with an aggregate function or add it to GROUP BY. + + bool found = false; + for (const auto & column_name_type : source_columns) + if (column_name_type.name == getColumnName()) + found = true; + + if (found) + throw Exception("Column " + getColumnName() + " is not under aggregate function and not in GROUP BY.", + ErrorCodes::NOT_AN_AGGREGATE); + } + } + else if (ASTFunction * node = typeid_cast(ast.get())) + { + if (node->name == "lambda") + throw Exception("Unexpected lambda expression", ErrorCodes::UNEXPECTED_EXPRESSION); + + /// Function arrayJoin. + if (node->name == "arrayJoin") + { + if (node->arguments->children.size() != 1) + throw Exception("arrayJoin requires exactly 1 argument", ErrorCodes::TYPE_MISMATCH); + + ASTPtr arg = node->arguments->children.at(0); + visit(arg, actions_stack, projection_manipulator); + if (!only_consts) + { + String result_name = projection_manipulator->getColumnName(getColumnName()); + actions_stack.addAction(ExpressionAction::copyColumn(projection_manipulator->getColumnName(arg->getColumnName()), result_name)); + NameSet joined_columns; + joined_columns.insert(result_name); + actions_stack.addAction(ExpressionAction::arrayJoin(joined_columns, false, context)); + } + + return; + } + + if (functionIsInOrGlobalInOperator(node->name)) + { + /// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything). + visit(node->arguments->children.at(0), actions_stack, projection_manipulator); + + if (!no_subqueries) + { + /// Transform tuple or subquery into a set. + makeSet(node, actions_stack.getSampleBlock()); + } + else + { + if (!only_consts) + { + /// We are in the part of the tree that we are not going to compute. You just need to define types. + /// Do not subquery and create sets. We treat "IN" as "ignore" function. + + actions_stack.addAction(ExpressionAction::applyFunction( + FunctionFactory::instance().get("ignore", context), + { node->arguments->children.at(0)->getColumnName() }, + projection_manipulator->getColumnName(getColumnName()), + projection_manipulator->getProjectionSourceColumn())); + } + return; + } + } + + /// A special function `indexHint`. Everything that is inside it is not calculated + /// (and is used only for index analysis, see KeyCondition). + if (node->name == "indexHint") + { + actions_stack.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName( + ColumnConst::create(ColumnUInt8::create(1, 1), 1), std::make_shared(), + projection_manipulator->getColumnName(getColumnName())), projection_manipulator->getProjectionSourceColumn(), false)); + return; + } + + if (AggregateFunctionFactory::instance().isAggregateFunctionName(node->name)) + return; + + /// Context object that we pass to function should live during query. + const Context & function_context = context.hasQueryContext() + ? context.getQueryContext() + : context; + + const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get(node->name, function_context); + auto projection_action = getProjectionAction(node->name, actions_stack, projection_manipulator, getColumnName(), function_context); + + Names argument_names; + DataTypes argument_types; + bool arguments_present = true; + + /// If the function has an argument-lambda expression, you need to determine its type before the recursive call. + bool has_lambda_arguments = false; + + for (size_t arg = 0; arg < node->arguments->children.size(); ++arg) + { + auto & child = node->arguments->children[arg]; + auto child_column_name = child->getColumnName(); + + ASTFunction * lambda = typeid_cast(child.get()); + if (lambda && lambda->name == "lambda") + { + /// If the argument is a lambda expression, just remember its approximate type. + if (lambda->arguments->children.size() != 2) + throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + ASTFunction * lambda_args_tuple = typeid_cast(lambda->arguments->children.at(0).get()); + + if (!lambda_args_tuple || lambda_args_tuple->name != "tuple") + throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH); + + has_lambda_arguments = true; + argument_types.emplace_back(std::make_shared(DataTypes(lambda_args_tuple->arguments->children.size()))); + /// Select the name in the next cycle. + argument_names.emplace_back(); + } + else if (prepared_sets.count(child->range) && functionIsInOrGlobalInOperator(node->name) && arg == 1) + { + ColumnWithTypeAndName column; + column.type = std::make_shared(); + + const SetPtr & set = prepared_sets[child->range]; + + /// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name, + /// so that sets with the same literal representation do not fuse together (they can have different types). + if (!set->empty()) + column.name = getUniqueName(actions_stack.getSampleBlock(), "__set"); + else + column.name = child_column_name; + + column.name = projection_manipulator->getColumnName(column.name); + + if (!actions_stack.getSampleBlock().has(column.name)) + { + column.column = ColumnSet::create(1, set); + + actions_stack.addAction(ExpressionAction::addColumn(column, projection_manipulator->getProjectionSourceColumn(), false)); + } + + argument_types.push_back(column.type); + argument_names.push_back(column.name); + } + else + { + /// If the argument is not a lambda expression, call it recursively and find out its type. + projection_action->preArgumentAction(); + visit(child, actions_stack, projection_manipulator); + std::string name = projection_manipulator->getColumnName(child_column_name); + projection_action->postArgumentAction(child_column_name); + if (actions_stack.getSampleBlock().has(name)) + { + argument_types.push_back(actions_stack.getSampleBlock().getByName(name).type); + argument_names.push_back(name); + } + else + { + if (only_consts) + { + arguments_present = false; + } + else + { + throw Exception("Unknown identifier: " + name + ", projection layer " + projection_manipulator->getProjectionExpression() , ErrorCodes::UNKNOWN_IDENTIFIER); + } + } + } + } + + if (only_consts && !arguments_present) + return; + + if (has_lambda_arguments && !only_consts) + { + function_builder->getLambdaArgumentTypes(argument_types); + + /// Call recursively for lambda expressions. + for (size_t i = 0; i < node->arguments->children.size(); ++i) + { + ASTPtr child = node->arguments->children[i]; + + ASTFunction * lambda = typeid_cast(child.get()); + if (lambda && lambda->name == "lambda") + { + const DataTypeFunction * lambda_type = typeid_cast(argument_types[i].get()); + ASTFunction * lambda_args_tuple = typeid_cast(lambda->arguments->children.at(0).get()); + ASTs lambda_arg_asts = lambda_args_tuple->arguments->children; + NamesAndTypesList lambda_arguments; + + for (size_t j = 0; j < lambda_arg_asts.size(); ++j) + { + ASTIdentifier * identifier = typeid_cast(lambda_arg_asts[j].get()); + if (!identifier) + throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH); + + String arg_name = identifier->name; + + lambda_arguments.emplace_back(arg_name, lambda_type->getArgumentTypes()[j]); + } + + projection_action->preArgumentAction(); + actions_stack.pushLevel(lambda_arguments); + visit(lambda->arguments->children.at(1), actions_stack, projection_manipulator); + ExpressionActionsPtr lambda_actions = actions_stack.popLevel(); + + String result_name = projection_manipulator->getColumnName(lambda->arguments->children.at(1)->getColumnName()); + lambda_actions->finalize(Names(1, result_name)); + DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type; + + Names captured; + Names required = lambda_actions->getRequiredColumns(); + for (const auto & required_arg : required) + if (findColumn(required_arg, lambda_arguments) == lambda_arguments.end()) + captured.push_back(required_arg); + + /// We can not name `getColumnName()`, + /// because it does not uniquely define the expression (the types of arguments can be different). + String lambda_name = getUniqueName(actions_stack.getSampleBlock(), "__lambda"); + + auto function_capture = std::make_shared( + lambda_actions, captured, lambda_arguments, result_type, result_name); + actions_stack.addAction(ExpressionAction::applyFunction(function_capture, captured, lambda_name, + projection_manipulator->getProjectionSourceColumn())); + + argument_types[i] = std::make_shared(lambda_type->getArgumentTypes(), result_type); + argument_names[i] = lambda_name; + projection_action->postArgumentAction(lambda_name); + } + } + } + + if (only_consts) + { + for (const auto & argument_name : argument_names) + { + if (!actions_stack.getSampleBlock().has(argument_name)) + { + arguments_present = false; + break; + } + } + } + + if (arguments_present) + { + projection_action->preCalculation(); + if (projection_action->isCalculationRequired()) + { + actions_stack.addAction( + ExpressionAction::applyFunction(function_builder, + argument_names, + projection_manipulator->getColumnName(getColumnName()), + projection_manipulator->getProjectionSourceColumn())); + } + } + } + else if (ASTLiteral * literal = typeid_cast(ast.get())) + { + DataTypePtr type = applyVisitor(FieldToDataType(), literal->value); + + ColumnWithTypeAndName column; + column.column = type->createColumnConst(1, convertFieldToType(literal->value, *type)); + column.type = type; + column.name = getColumnName(); + + actions_stack.addAction(ExpressionAction::addColumn(column, "", false)); + projection_manipulator->tryToGetFromUpperProjection(column.name); + } + else + { + for (auto & child : ast->children) + { + /// Do not go to FROM, JOIN, UNION. + if (!typeid_cast(child.get()) + && !typeid_cast(child.get())) + visit(child, actions_stack, projection_manipulator); + } + } +} + +void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_block) +{ + /** You need to convert the right argument to a set. + * This can be a table name, a value, a value enumeration, or a subquery. + * The enumeration of values is parsed as a function `tuple`. + */ + const IAST & args = *node->arguments; + const ASTPtr & arg = args.children.at(1); + + /// Already converted. + if (prepared_sets.count(arg->range)) + return; + + /// If the subquery or table name for SELECT. + const ASTIdentifier * identifier = typeid_cast(arg.get()); + if (typeid_cast(arg.get()) || identifier) + { + /// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery. + String set_id = arg->getColumnName(); + + /// A special case is if the name of the table is specified on the right side of the IN statement, + /// and the table has the type Set (a previously prepared set). + if (identifier) + { + auto database_table = getDatabaseAndTableNameFromIdentifier(*identifier); + StoragePtr table = context.tryGetTable(database_table.first, database_table.second); + + if (table) + { + StorageSet * storage_set = dynamic_cast(table.get()); + + if (storage_set) + { + prepared_sets[arg->range] = storage_set->getSet(); + return; + } + } + } + + SubqueryForSet & subquery_for_set = subqueries_for_sets[set_id]; + + /// If you already created a Set with the same subquery / table. + if (subquery_for_set.set) + { + prepared_sets[arg->range] = subquery_for_set.set; + return; + } + + SetPtr set = std::make_shared(set_size_limit, false); + + /** The following happens for GLOBAL INs: + * - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1, + * in the subquery_for_set object, this subquery is set as source and the temporary table _data1 as the table. + * - this function shows the expression IN_data1. + */ + if (!subquery_for_set.source && no_storage_or_local) + { + auto interpreter = interpretSubquery(arg, context, subquery_depth, {}); + subquery_for_set.source = std::make_shared( + interpreter->getSampleBlock(), [interpreter]() mutable { return interpreter->execute().in; }); + + /** Why is LazyBlockInputStream used? + * + * The fact is that when processing a query of the form + * SELECT ... FROM remote_test WHERE column GLOBAL IN (subquery), + * if the distributed remote_test table contains localhost as one of the servers, + * the query will be interpreted locally again (and not sent over TCP, as in the case of a remote server). + * + * The query execution pipeline will be: + * CreatingSets + * subquery execution, filling the temporary table with _data1 (1) + * CreatingSets + * reading from the table _data1, creating the set (2) + * read from the table subordinate to remote_test. + * + * (The second part of the pipeline under CreateSets is a reinterpretation of the query inside StorageDistributed, + * the query differs in that the database name and tables are replaced with subordinates, and the subquery is replaced with _data1.) + * + * But when creating the pipeline, when creating the source (2), it will be found that the _data1 table is empty + * (because the query has not started yet), and empty source will be returned as the source. + * And then, when the query is executed, an empty set will be created in step (2). + * + * Therefore, we make the initialization of step (2) lazy + * - so that it does not occur until step (1) is completed, on which the table will be populated. + * + * Note: this solution is not very good, you need to think better. + */ + } + + subquery_for_set.set = set; + prepared_sets[arg->range] = set; + } + else + { + /// An explicit enumeration of values in parentheses. + makeExplicitSet(node, sample_block, false, context, set_size_limit, prepared_sets); + } +} + +} diff --git a/dbms/src/Interpreters/ActionsVisitor.h b/dbms/src/Interpreters/ActionsVisitor.h new file mode 100644 index 00000000000..78b91b64afc --- /dev/null +++ b/dbms/src/Interpreters/ActionsVisitor.h @@ -0,0 +1,128 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; +class ASTFunction; + +class ExpressionActions; +using ExpressionActionsPtr = std::shared_ptr; + +struct ProjectionManipulatorBase; +using ProjectionManipulatorPtr = std::shared_ptr; + +class Set; +using SetPtr = std::shared_ptr; +/// Will compare sets by their position in query string. It's possible because IAST::clone() doesn't chane IAST::range. +/// It should be taken into account when we want to change AST part which contains sets. +using PreparedSets = std::unordered_map; + +class Join; +using JoinPtr = std::shared_ptr; + +/// Information on what to do when executing a subquery in the [GLOBAL] IN/JOIN section. +struct SubqueryForSet +{ + /// The source is obtained using the InterpreterSelectQuery subquery. + BlockInputStreamPtr source; + + /// If set, build it from result. + SetPtr set; + JoinPtr join; + /// Apply this actions to joined block. + ExpressionActionsPtr joined_block_actions; + /// Rename column from joined block from this list. + NamesWithAliases joined_block_aliases; + + /// If set, put the result into the table. + /// This is a temporary table for transferring to remote servers for distributed query processing. + StoragePtr table; +}; + +/// ID of subquery -> what to do with it. +using SubqueriesForSets = std::unordered_map; + + +/// The case of an explicit enumeration of values. +void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool create_ordered_set, + const Context & context, const SizeLimits & limits, PreparedSets & prepared_sets); + + +/** For ActionsVisitor + * A stack of ExpressionActions corresponding to nested lambda expressions. + * The new action should be added to the highest possible level. + * For example, in the expression "select arrayMap(x -> x + column1 * column2, array1)" + * calculation of the product must be done outside the lambda expression (it does not depend on x), + * and the calculation of the sum is inside (depends on x). + */ +struct ScopeStack +{ + struct Level + { + ExpressionActionsPtr actions; + NameSet new_columns; + }; + + using Levels = std::vector; + + Levels stack; + + const Context & context; + + ScopeStack(const ExpressionActionsPtr & actions, const Context & context_); + + void pushLevel(const NamesAndTypesList & input_columns); + + size_t getColumnLevel(const std::string & name); + + void addAction(const ExpressionAction & action); + + ExpressionActionsPtr popLevel(); + + const Block & getSampleBlock() const; +}; + + +/// TODO: There sould be some description, but... +class ActionsVisitor +{ +public: + ActionsVisitor(const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, + const NamesAndTypesList & source_columns_, PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, + bool no_subqueries_, bool only_consts_, bool no_storage_or_local_, std::ostream * ostr_ = nullptr) + : context(context_), + set_size_limit(set_size_limit_), + subquery_depth(subquery_depth_), + source_columns(source_columns_), + prepared_sets(prepared_sets_), + subqueries_for_sets(subqueries_for_sets_), + no_subqueries(no_subqueries_), + only_consts(only_consts_), + no_storage_or_local(no_storage_or_local_), + visit_depth(0), + ostr(ostr_) + {} + + void visit(const ASTPtr & ast, ScopeStack & actions_stack, ProjectionManipulatorPtr projection_manipulator); + +private: + const Context & context; + SizeLimits set_size_limit; + size_t subquery_depth; + const NamesAndTypesList & source_columns; + PreparedSets & prepared_sets; + SubqueriesForSets & subqueries_for_sets; + const bool no_subqueries; + const bool only_consts; + const bool no_storage_or_local; + mutable size_t visit_depth; + std::ostream * ostr; + + void makeSet(const ASTFunction * node, const Block & sample_block); +}; + +} diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 7aa946cad39..15938a3f52a 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -1,8 +1,6 @@ #include #include -#include - #include #include #include @@ -16,13 +14,11 @@ #include #include -#include #include #include #include -#include -#include +#include #include #include @@ -31,11 +27,9 @@ #include #include #include -#include #include #include #include -#include #include #include @@ -44,7 +38,6 @@ #include #include -#include #include #include @@ -57,22 +50,18 @@ #include -#include -#include - #include #include -#include #include -#include #include #include #include #include +#include #include #include #include -#include +#include namespace DB @@ -91,8 +80,6 @@ namespace ErrorCodes extern const int INCORRECT_ELEMENT_OF_SET; extern const int ALIAS_REQUIRED; extern const int EMPTY_NESTED_TABLE; - extern const int NOT_AN_AGGREGATE; - extern const int UNEXPECTED_EXPRESSION; extern const int DUPLICATE_COLUMN; extern const int FUNCTION_CANNOT_HAVE_PARAMETERS; extern const int ILLEGAL_AGGREGATION; @@ -163,6 +150,7 @@ void removeDuplicateColumns(NamesAndTypesList & columns) } + ExpressionAnalyzer::ExpressionAnalyzer( const ASTPtr & query_, const Context & context_, @@ -279,6 +267,12 @@ ExpressionAnalyzer::ExpressionAnalyzer( analyzeAggregation(); } +bool ExpressionAnalyzer::noStorageOrLocal() const +{ + return !storage || !storage->isRemote(); +} + + static std::vector getTableExpressions(const ASTPtr & query) { ASTSelectQuery * select_query = typeid_cast(query.get()); @@ -596,118 +590,6 @@ void ExpressionAnalyzer::findExternalTables(ASTPtr & ast) external_tables[node->name] = external_storage; } -static std::shared_ptr interpretSubquery( - const ASTPtr & table_expression, const Context & context, size_t subquery_depth, const Names & required_source_columns) -{ - /// Subquery or table name. The name of the table is similar to the subquery `SELECT * FROM t`. - const ASTSubquery * subquery = typeid_cast(table_expression.get()); - const ASTFunction * function = typeid_cast(table_expression.get()); - const ASTIdentifier * table = typeid_cast(table_expression.get()); - - if (!subquery && !table && !function) - throw Exception("Table expression is undefined, Method: ExpressionAnalyzer::interpretSubquery." , ErrorCodes::LOGICAL_ERROR); - - /** The subquery in the IN / JOIN section does not have any restrictions on the maximum size of the result. - * Because the result of this query is not the result of the entire query. - * Constraints work instead - * max_rows_in_set, max_bytes_in_set, set_overflow_mode, - * max_rows_in_join, max_bytes_in_join, join_overflow_mode, - * which are checked separately (in the Set, Join objects). - */ - Context subquery_context = context; - Settings subquery_settings = context.getSettings(); - subquery_settings.max_result_rows = 0; - subquery_settings.max_result_bytes = 0; - /// The calculation of `extremes` does not make sense and is not necessary (if you do it, then the `extremes` of the subquery can be taken instead of the whole query). - subquery_settings.extremes = 0; - subquery_context.setSettings(subquery_settings); - - ASTPtr query; - if (table || function) - { - /// create ASTSelectQuery for "SELECT * FROM table" as if written by hand - const auto select_with_union_query = std::make_shared(); - query = select_with_union_query; - - select_with_union_query->list_of_selects = std::make_shared(); - - const auto select_query = std::make_shared(); - select_with_union_query->list_of_selects->children.push_back(select_query); - - const auto select_expression_list = std::make_shared(); - select_query->select_expression_list = select_expression_list; - select_query->children.emplace_back(select_query->select_expression_list); - - NamesAndTypesList columns; - - /// get columns list for target table - if (function) - { - auto query_context = const_cast(&context.getQueryContext()); - const auto & storage = query_context->executeTableFunction(table_expression); - columns = storage->getColumns().ordinary; - select_query->addTableFunction(*const_cast(&table_expression)); - } - else - { - auto database_table = getDatabaseAndTableNameFromIdentifier(*table); - const auto & storage = context.getTable(database_table.first, database_table.second); - columns = storage->getColumns().ordinary; - select_query->replaceDatabaseAndTable(database_table.first, database_table.second); - } - - select_expression_list->children.reserve(columns.size()); - /// manually substitute column names in place of asterisk - for (const auto & column : columns) - select_expression_list->children.emplace_back(std::make_shared(column.name)); - } - else - { - query = subquery->children.at(0); - - /** Columns with the same name can be specified in a subquery. For example, SELECT x, x FROM t - * This is bad, because the result of such a query can not be saved to the table, because the table can not have the same name columns. - * Saving to the table is required for GLOBAL subqueries. - * - * To avoid this situation, we will rename the same columns. - */ - - std::set all_column_names; - std::set assigned_column_names; - - if (ASTSelectWithUnionQuery * select_with_union = typeid_cast(query.get())) - { - if (ASTSelectQuery * select = typeid_cast(select_with_union->list_of_selects->children.at(0).get())) - { - for (auto & expr : select->select_expression_list->children) - all_column_names.insert(expr->getAliasOrColumnName()); - - for (auto & expr : select->select_expression_list->children) - { - auto name = expr->getAliasOrColumnName(); - - if (!assigned_column_names.insert(name).second) - { - size_t i = 1; - while (all_column_names.end() != all_column_names.find(name + "_" + toString(i))) - ++i; - - name = name + "_" + toString(i); - expr = expr->clone(); /// Cancels fuse of the same expressions in the tree. - expr->setAlias(name); - - all_column_names.insert(name); - assigned_column_names.insert(name); - } - } - } - } - } - - return std::make_shared( - query, subquery_context, required_source_columns, QueryProcessingStage::Complete, subquery_depth + 1); -} - void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name_or_table_expression) { @@ -800,7 +682,7 @@ void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name_or_t } -static NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypesList & cols) +NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypesList & cols) { return std::find_if(cols.begin(), cols.end(), [&](const NamesAndTypesList::value_type & val) { return val.name == name; }); @@ -1118,8 +1000,7 @@ void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_ { BlockIO res = interpretSubquery(subquery_or_table_name, context, subquery_depth + 1, {})->execute(); - SizeLimits set_for_index_size_limits = SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode); - SetPtr set = std::make_shared(set_for_index_size_limits, true); + SetPtr set = std::make_shared(getSetSizeLimits(settings), true); set->setHeader(res.in->getHeader()); while (Block block = res.in->read()) @@ -1176,7 +1057,7 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block & Block sample_block_with_calculated_columns = temp_actions->getSampleBlock(); if (sample_block_with_calculated_columns.has(args.children.at(0)->getColumnName())) - makeExplicitSet(func, sample_block_with_calculated_columns, true); + makeExplicitSet(func, sample_block_with_calculated_columns, true, context, getSetSizeLimits(settings), prepared_sets); } } } @@ -1184,278 +1065,6 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block & } -void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_block) -{ - /** You need to convert the right argument to a set. - * This can be a table name, a value, a value enumeration, or a subquery. - * The enumeration of values is parsed as a function `tuple`. - */ - const IAST & args = *node->arguments; - const ASTPtr & arg = args.children.at(1); - - /// Already converted. - if (prepared_sets.count(arg->range)) - return; - - /// If the subquery or table name for SELECT. - const ASTIdentifier * identifier = typeid_cast(arg.get()); - if (typeid_cast(arg.get()) || identifier) - { - /// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery. - String set_id = arg->getColumnName(); - - /// A special case is if the name of the table is specified on the right side of the IN statement, - /// and the table has the type Set (a previously prepared set). - if (identifier) - { - auto database_table = getDatabaseAndTableNameFromIdentifier(*identifier); - StoragePtr table = context.tryGetTable(database_table.first, database_table.second); - - if (table) - { - StorageSet * storage_set = dynamic_cast(table.get()); - - if (storage_set) - { - prepared_sets[arg->range] = storage_set->getSet(); - return; - } - } - } - - SubqueryForSet & subquery_for_set = subqueries_for_sets[set_id]; - - /// If you already created a Set with the same subquery / table. - if (subquery_for_set.set) - { - prepared_sets[arg->range] = subquery_for_set.set; - return; - } - - SetPtr set = std::make_shared(SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode), false); - - /** The following happens for GLOBAL INs: - * - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1, - * in the subquery_for_set object, this subquery is set as source and the temporary table _data1 as the table. - * - this function shows the expression IN_data1. - */ - if (!subquery_for_set.source && (!storage || !storage->isRemote())) - { - auto interpreter = interpretSubquery(arg, context, subquery_depth, {}); - subquery_for_set.source = std::make_shared( - interpreter->getSampleBlock(), [interpreter]() mutable { return interpreter->execute().in; }); - - /** Why is LazyBlockInputStream used? - * - * The fact is that when processing a query of the form - * SELECT ... FROM remote_test WHERE column GLOBAL IN (subquery), - * if the distributed remote_test table contains localhost as one of the servers, - * the query will be interpreted locally again (and not sent over TCP, as in the case of a remote server). - * - * The query execution pipeline will be: - * CreatingSets - * subquery execution, filling the temporary table with _data1 (1) - * CreatingSets - * reading from the table _data1, creating the set (2) - * read from the table subordinate to remote_test. - * - * (The second part of the pipeline under CreateSets is a reinterpretation of the query inside StorageDistributed, - * the query differs in that the database name and tables are replaced with subordinates, and the subquery is replaced with _data1.) - * - * But when creating the pipeline, when creating the source (2), it will be found that the _data1 table is empty - * (because the query has not started yet), and empty source will be returned as the source. - * And then, when the query is executed, an empty set will be created in step (2). - * - * Therefore, we make the initialization of step (2) lazy - * - so that it does not occur until step (1) is completed, on which the table will be populated. - * - * Note: this solution is not very good, you need to think better. - */ - } - - subquery_for_set.set = set; - prepared_sets[arg->range] = set; - } - else - { - /// An explicit enumeration of values in parentheses. - makeExplicitSet(node, sample_block, false); - } -} - -/// The case of an explicit enumeration of values. -void ExpressionAnalyzer::makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool create_ordered_set) -{ - const IAST & args = *node->arguments; - - if (args.children.size() != 2) - throw Exception("Wrong number of arguments passed to function in", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - const ASTPtr & left_arg = args.children.at(0); - const ASTPtr & right_arg = args.children.at(1); - - auto getTupleTypeFromAst = [this](const ASTPtr & tuple_ast) -> DataTypePtr - { - auto ast_function = typeid_cast(tuple_ast.get()); - if (ast_function && ast_function->name == "tuple" && !ast_function->arguments->children.empty()) - { - /// Won't parse all values of outer tuple. - auto element = ast_function->arguments->children.at(0); - std::pair value_raw = evaluateConstantExpression(element, context); - return std::make_shared(DataTypes({value_raw.second})); - } - - return evaluateConstantExpression(tuple_ast, context).second; - }; - - const DataTypePtr & left_arg_type = sample_block.getByName(left_arg->getColumnName()).type; - const DataTypePtr & right_arg_type = getTupleTypeFromAst(right_arg); - - std::function getTupleDepth; - getTupleDepth = [&getTupleDepth](const DataTypePtr & type) -> size_t - { - if (auto tuple_type = typeid_cast(type.get())) - return 1 + (tuple_type->getElements().empty() ? 0 : getTupleDepth(tuple_type->getElements().at(0))); - - return 0; - }; - - size_t left_tuple_depth = getTupleDepth(left_arg_type); - size_t right_tuple_depth = getTupleDepth(right_arg_type); - - DataTypes set_element_types = {left_arg_type}; - auto left_tuple_type = typeid_cast(left_arg_type.get()); - if (left_tuple_type && left_tuple_type->getElements().size() != 1) - set_element_types = left_tuple_type->getElements(); - - for (auto & element_type : set_element_types) - if (const auto * low_cardinality_type = typeid_cast(element_type.get())) - element_type = low_cardinality_type->getDictionaryType(); - - ASTPtr elements_ast = nullptr; - - /// 1 in 1; (1, 2) in (1, 2); identity(tuple(tuple(tuple(1)))) in tuple(tuple(tuple(1))); etc. - if (left_tuple_depth == right_tuple_depth) - { - ASTPtr exp_list = std::make_shared(); - exp_list->children.push_back(right_arg); - elements_ast = exp_list; - } - /// 1 in (1, 2); (1, 2) in ((1, 2), (3, 4)); etc. - else if (left_tuple_depth + 1 == right_tuple_depth) - { - ASTFunction * set_func = typeid_cast(right_arg.get()); - - if (!set_func || set_func->name != "tuple") - throw Exception("Incorrect type of 2nd argument for function " + node->name - + ". Must be subquery or set of elements with type " + left_arg_type->getName() + ".", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - elements_ast = set_func->arguments; - } - else - throw Exception("Invalid types for IN function: " - + left_arg_type->getName() + " and " + right_arg_type->getName() + ".", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - SetPtr set = std::make_shared(SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode), create_ordered_set); - set->createFromAST(set_element_types, elements_ast, context); - prepared_sets[right_arg->range] = std::move(set); -} - - -static String getUniqueName(const Block & block, const String & prefix) -{ - int i = 1; - while (block.has(prefix + toString(i))) - ++i; - return prefix + toString(i); -} - -/** For getActionsImpl. - * A stack of ExpressionActions corresponding to nested lambda expressions. - * The new action should be added to the highest possible level. - * For example, in the expression "select arrayMap(x -> x + column1 * column2, array1)" - * calculation of the product must be done outside the lambda expression (it does not depend on x), and the calculation of the sum is inside (depends on x). - */ -ScopeStack::ScopeStack(const ExpressionActionsPtr & actions, const Context & context_) - : context(context_) -{ - stack.emplace_back(); - stack.back().actions = actions; - - const Block & sample_block = actions->getSampleBlock(); - for (size_t i = 0, size = sample_block.columns(); i < size; ++i) - stack.back().new_columns.insert(sample_block.getByPosition(i).name); -} - -void ScopeStack::pushLevel(const NamesAndTypesList & input_columns) -{ - stack.emplace_back(); - Level & prev = stack[stack.size() - 2]; - - ColumnsWithTypeAndName all_columns; - NameSet new_names; - - for (NamesAndTypesList::const_iterator it = input_columns.begin(); it != input_columns.end(); ++it) - { - all_columns.emplace_back(nullptr, it->type, it->name); - new_names.insert(it->name); - stack.back().new_columns.insert(it->name); - } - - const Block & prev_sample_block = prev.actions->getSampleBlock(); - for (size_t i = 0, size = prev_sample_block.columns(); i < size; ++i) - { - const ColumnWithTypeAndName & col = prev_sample_block.getByPosition(i); - if (!new_names.count(col.name)) - all_columns.push_back(col); - } - - stack.back().actions = std::make_shared(all_columns, context); -} - -size_t ScopeStack::getColumnLevel(const std::string & name) -{ - for (int i = static_cast(stack.size()) - 1; i >= 0; --i) - if (stack[i].new_columns.count(name)) - return i; - - throw Exception("Unknown identifier: " + name, ErrorCodes::UNKNOWN_IDENTIFIER); -} - -void ScopeStack::addAction(const ExpressionAction & action) -{ - size_t level = 0; - Names required = action.getNeededColumns(); - for (size_t i = 0; i < required.size(); ++i) - level = std::max(level, getColumnLevel(required[i])); - - Names added; - stack[level].actions->add(action, added); - - stack[level].new_columns.insert(added.begin(), added.end()); - - for (size_t i = 0; i < added.size(); ++i) - { - const ColumnWithTypeAndName & col = stack[level].actions->getSampleBlock().getByName(added[i]); - for (size_t j = level + 1; j < stack.size(); ++j) - stack[j].actions->addInput(col); - } -} - -ExpressionActionsPtr ScopeStack::popLevel() -{ - ExpressionActionsPtr res = stack.back().actions; - stack.pop_back(); - return res; -} - -const Block & ScopeStack::getSampleBlock() const -{ - return stack.back().actions->getSampleBlock(); -} - void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, bool only_consts, ExpressionActionsPtr & actions) { ScopeStack scopes(actions, context); @@ -1466,7 +1075,11 @@ void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, else projection_manipulator = std::make_shared(scopes); - getActionsImpl(ast, no_subqueries, only_consts, scopes, projection_manipulator); + LogAST log; + ActionsVisitor actions_visitor(context, getSetSizeLimits(settings), subquery_depth, + source_columns, prepared_sets, subqueries_for_sets, + no_subqueries, only_consts, noStorageOrLocal(), log.stream()); + actions_visitor.visit(ast, scopes, projection_manipulator); actions = scopes.popLevel(); } @@ -1637,318 +1250,22 @@ void ExpressionAnalyzer::getActionsFromJoinKeys(const ASTTableJoin & table_join, else projection_manipulator = std::make_shared(scopes); + LogAST log; + ActionsVisitor actions_visitor(context, getSetSizeLimits(settings), subquery_depth, + source_columns, prepared_sets, subqueries_for_sets, + no_subqueries, only_consts, noStorageOrLocal(), log.stream()); + if (table_join.using_expression_list) - getActionsImpl(table_join.using_expression_list, no_subqueries, only_consts, scopes, projection_manipulator); + actions_visitor.visit(table_join.using_expression_list, scopes, projection_manipulator); else if (table_join.on_expression) { for (const auto & ast : analyzed_join.key_asts_left) - getActionsImpl(ast, no_subqueries, only_consts, scopes, projection_manipulator); + actions_visitor.visit(ast, scopes, projection_manipulator); } actions = scopes.popLevel(); } -void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries, bool only_consts, ScopeStack & actions_stack, - ProjectionManipulatorPtr projection_manipulator) -{ - String ast_column_name; - auto getColumnName = [&ast, &ast_column_name]() - { - if (ast_column_name.empty()) - ast_column_name = ast->getColumnName(); - - return ast_column_name; - }; - - /// If the result of the calculation already exists in the block. - if ((typeid_cast(ast.get()) || typeid_cast(ast.get())) - && projection_manipulator->tryToGetFromUpperProjection(getColumnName())) - return; - - if (typeid_cast(ast.get())) - { - if (!only_consts && !projection_manipulator->tryToGetFromUpperProjection(getColumnName())) - { - /// The requested column is not in the block. - /// If such a column exists in the table, then the user probably forgot to surround it with an aggregate function or add it to GROUP BY. - - bool found = false; - for (const auto & column_name_type : source_columns) - if (column_name_type.name == getColumnName()) - found = true; - - if (found) - throw Exception("Column " + getColumnName() + " is not under aggregate function and not in GROUP BY.", - ErrorCodes::NOT_AN_AGGREGATE); - } - } - else if (ASTFunction * node = typeid_cast(ast.get())) - { - if (node->name == "lambda") - throw Exception("Unexpected lambda expression", ErrorCodes::UNEXPECTED_EXPRESSION); - - /// Function arrayJoin. - if (node->name == "arrayJoin") - { - if (node->arguments->children.size() != 1) - throw Exception("arrayJoin requires exactly 1 argument", ErrorCodes::TYPE_MISMATCH); - - ASTPtr arg = node->arguments->children.at(0); - getActionsImpl(arg, no_subqueries, only_consts, actions_stack, projection_manipulator); - if (!only_consts) - { - String result_name = projection_manipulator->getColumnName(getColumnName()); - actions_stack.addAction(ExpressionAction::copyColumn(projection_manipulator->getColumnName(arg->getColumnName()), result_name)); - NameSet joined_columns; - joined_columns.insert(result_name); - actions_stack.addAction(ExpressionAction::arrayJoin(joined_columns, false, context)); - } - - return; - } - - if (functionIsInOrGlobalInOperator(node->name)) - { - /// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything). - getActionsImpl(node->arguments->children.at(0), no_subqueries, only_consts, actions_stack, projection_manipulator); - - if (!no_subqueries) - { - /// Transform tuple or subquery into a set. - makeSet(node, actions_stack.getSampleBlock()); - } - else - { - if (!only_consts) - { - /// We are in the part of the tree that we are not going to compute. You just need to define types. - /// Do not subquery and create sets. We treat "IN" as "ignore" function. - - actions_stack.addAction(ExpressionAction::applyFunction( - FunctionFactory::instance().get("ignore", context), - { node->arguments->children.at(0)->getColumnName() }, - projection_manipulator->getColumnName(getColumnName()), - projection_manipulator->getProjectionSourceColumn())); - } - return; - } - } - - /// A special function `indexHint`. Everything that is inside it is not calculated - /// (and is used only for index analysis, see KeyCondition). - if (node->name == "indexHint") - { - actions_stack.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName( - ColumnConst::create(ColumnUInt8::create(1, 1), 1), std::make_shared(), - projection_manipulator->getColumnName(getColumnName())), projection_manipulator->getProjectionSourceColumn(), false)); - return; - } - - if (AggregateFunctionFactory::instance().isAggregateFunctionName(node->name)) - return; - - /// Context object that we pass to function should live during query. - const Context & function_context = context.hasQueryContext() - ? context.getQueryContext() - : context; - - const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get(node->name, function_context); - auto projection_action = getProjectionAction(node->name, actions_stack, projection_manipulator, getColumnName(), function_context); - - Names argument_names; - DataTypes argument_types; - bool arguments_present = true; - - /// If the function has an argument-lambda expression, you need to determine its type before the recursive call. - bool has_lambda_arguments = false; - - for (size_t arg = 0; arg < node->arguments->children.size(); ++arg) - { - auto & child = node->arguments->children[arg]; - auto child_column_name = child->getColumnName(); - - ASTFunction * lambda = typeid_cast(child.get()); - if (lambda && lambda->name == "lambda") - { - /// If the argument is a lambda expression, just remember its approximate type. - if (lambda->arguments->children.size() != 2) - throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - ASTFunction * lambda_args_tuple = typeid_cast(lambda->arguments->children.at(0).get()); - - if (!lambda_args_tuple || lambda_args_tuple->name != "tuple") - throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH); - - has_lambda_arguments = true; - argument_types.emplace_back(std::make_shared(DataTypes(lambda_args_tuple->arguments->children.size()))); - /// Select the name in the next cycle. - argument_names.emplace_back(); - } - else if (prepared_sets.count(child->range) && functionIsInOrGlobalInOperator(node->name) && arg == 1) - { - ColumnWithTypeAndName column; - column.type = std::make_shared(); - - const SetPtr & set = prepared_sets[child->range]; - - /// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name, - /// so that sets with the same literal representation do not fuse together (they can have different types). - if (!set->empty()) - column.name = getUniqueName(actions_stack.getSampleBlock(), "__set"); - else - column.name = child_column_name; - - column.name = projection_manipulator->getColumnName(column.name); - - if (!actions_stack.getSampleBlock().has(column.name)) - { - column.column = ColumnSet::create(1, set); - - actions_stack.addAction(ExpressionAction::addColumn(column, projection_manipulator->getProjectionSourceColumn(), false)); - } - - argument_types.push_back(column.type); - argument_names.push_back(column.name); - } - else - { - /// If the argument is not a lambda expression, call it recursively and find out its type. - projection_action->preArgumentAction(); - getActionsImpl(child, no_subqueries, only_consts, actions_stack, - projection_manipulator); - std::string name = projection_manipulator->getColumnName(child_column_name); - projection_action->postArgumentAction(child_column_name); - if (actions_stack.getSampleBlock().has(name)) - { - argument_types.push_back(actions_stack.getSampleBlock().getByName(name).type); - argument_names.push_back(name); - } - else - { - if (only_consts) - { - arguments_present = false; - } - else - { - throw Exception("Unknown identifier: " + name + ", projection layer " + projection_manipulator->getProjectionExpression() , ErrorCodes::UNKNOWN_IDENTIFIER); - } - } - } - } - - if (only_consts && !arguments_present) - return; - - if (has_lambda_arguments && !only_consts) - { - function_builder->getLambdaArgumentTypes(argument_types); - - /// Call recursively for lambda expressions. - for (size_t i = 0; i < node->arguments->children.size(); ++i) - { - ASTPtr child = node->arguments->children[i]; - - ASTFunction * lambda = typeid_cast(child.get()); - if (lambda && lambda->name == "lambda") - { - const DataTypeFunction * lambda_type = typeid_cast(argument_types[i].get()); - ASTFunction * lambda_args_tuple = typeid_cast(lambda->arguments->children.at(0).get()); - ASTs lambda_arg_asts = lambda_args_tuple->arguments->children; - NamesAndTypesList lambda_arguments; - - for (size_t j = 0; j < lambda_arg_asts.size(); ++j) - { - ASTIdentifier * identifier = typeid_cast(lambda_arg_asts[j].get()); - if (!identifier) - throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH); - - String arg_name = identifier->name; - - lambda_arguments.emplace_back(arg_name, lambda_type->getArgumentTypes()[j]); - } - - projection_action->preArgumentAction(); - actions_stack.pushLevel(lambda_arguments); - getActionsImpl(lambda->arguments->children.at(1), no_subqueries, only_consts, actions_stack, - projection_manipulator); - ExpressionActionsPtr lambda_actions = actions_stack.popLevel(); - - String result_name = projection_manipulator->getColumnName(lambda->arguments->children.at(1)->getColumnName()); - lambda_actions->finalize(Names(1, result_name)); - DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type; - - Names captured; - Names required = lambda_actions->getRequiredColumns(); - for (const auto & required_arg : required) - if (findColumn(required_arg, lambda_arguments) == lambda_arguments.end()) - captured.push_back(required_arg); - - /// We can not name `getColumnName()`, - /// because it does not uniquely define the expression (the types of arguments can be different). - String lambda_name = getUniqueName(actions_stack.getSampleBlock(), "__lambda"); - - auto function_capture = std::make_shared( - lambda_actions, captured, lambda_arguments, result_type, result_name); - actions_stack.addAction(ExpressionAction::applyFunction(function_capture, captured, lambda_name, - projection_manipulator->getProjectionSourceColumn())); - - argument_types[i] = std::make_shared(lambda_type->getArgumentTypes(), result_type); - argument_names[i] = lambda_name; - projection_action->postArgumentAction(lambda_name); - } - } - } - - if (only_consts) - { - for (const auto & argument_name : argument_names) - { - if (!actions_stack.getSampleBlock().has(argument_name)) - { - arguments_present = false; - break; - } - } - } - - if (arguments_present) - { - projection_action->preCalculation(); - if (projection_action->isCalculationRequired()) - { - actions_stack.addAction( - ExpressionAction::applyFunction(function_builder, - argument_names, - projection_manipulator->getColumnName(getColumnName()), - projection_manipulator->getProjectionSourceColumn())); - } - } - } - else if (ASTLiteral * literal = typeid_cast(ast.get())) - { - DataTypePtr type = applyVisitor(FieldToDataType(), literal->value); - - ColumnWithTypeAndName column; - column.column = type->createColumnConst(1, convertFieldToType(literal->value, *type)); - column.type = type; - column.name = getColumnName(); - - actions_stack.addAction(ExpressionAction::addColumn(column, "", false)); - projection_manipulator->tryToGetFromUpperProjection(column.name); - } - else - { - for (auto & child : ast->children) - { - /// Do not go to FROM, JOIN, UNION. - if (!typeid_cast(child.get()) - && !typeid_cast(child.get())) - getActionsImpl(child, no_subqueries, only_consts, actions_stack, projection_manipulator); - } - } -} - void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr & actions) { diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index f49d306e26f..c4619bba078 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -2,10 +2,10 @@ #include #include -#include #include -#include -#include +#include + +#include #include namespace DB @@ -16,18 +16,9 @@ class Context; class ExpressionActions; struct ExpressionActionsChain; -class Join; -using JoinPtr = std::shared_ptr; - class IAST; using ASTPtr = std::shared_ptr; -class Set; -using SetPtr = std::shared_ptr; -/// Will compare sets by their position in query string. It's possible because IAST::clone() doesn't chane IAST::range. -/// It should be taken into account when we want to change AST part which contains sets. -using PreparedSets = std::unordered_map; - class IBlockInputStream; using BlockInputStreamPtr = std::shared_ptr; @@ -39,58 +30,12 @@ class ASTFunction; class ASTExpressionList; class ASTSelectQuery; -struct ProjectionManipulatorBase; -using ProjectionManipulatorPtr = std::shared_ptr; -/** Information on what to do when executing a subquery in the [GLOBAL] IN/JOIN section. - */ -struct SubqueryForSet +inline SizeLimits getSetSizeLimits(const Settings & settings) { - /// The source is obtained using the InterpreterSelectQuery subquery. - BlockInputStreamPtr source; + return SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode); +} - /// If set, build it from result. - SetPtr set; - JoinPtr join; - /// Apply this actions to joined block. - ExpressionActionsPtr joined_block_actions; - /// Rename column from joined block from this list. - NamesWithAliases joined_block_aliases; - - /// If set, put the result into the table. - /// This is a temporary table for transferring to remote servers for distributed query processing. - StoragePtr table; -}; - -/// ID of subquery -> what to do with it. -using SubqueriesForSets = std::unordered_map; - -struct ScopeStack -{ - struct Level - { - ExpressionActionsPtr actions; - NameSet new_columns; - }; - - using Levels = std::vector; - - Levels stack; - - const Context & context; - - ScopeStack(const ExpressionActionsPtr & actions, const Context & context_); - - void pushLevel(const NamesAndTypesList & input_columns); - - size_t getColumnLevel(const std::string & name); - - void addAction(const ExpressionAction & action); - - ExpressionActionsPtr popLevel(); - - const Block & getSampleBlock() const; -}; /** Transforms an expression from a syntax tree into a sequence of actions to execute it. * @@ -341,14 +286,11 @@ private: void optimizeIfWithConstantConditionImpl(ASTPtr & current_ast); bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & value) const; - void makeSet(const ASTFunction * node, const Block & sample_block); - /// Adds a list of ALIAS columns from the table. void addAliasColumns(); /// Replacing scalar subqueries with constant values. void executeScalarSubqueries(); - void executeScalarSubqueriesImpl(ASTPtr & ast); /// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables. void initGlobalSubqueriesAndExternalTables(); @@ -370,9 +312,6 @@ private: bool isThereArrayJoin(const ASTPtr & ast); - void getActionsImpl(const ASTPtr & ast, bool no_subqueries, bool only_consts, ScopeStack & actions_stack, - ProjectionManipulatorPtr projection_manipulator); - /// If ast is ASTSelectQuery with JOIN, add actions for JOIN key columns. void getActionsFromJoinKeys(const ASTTableJoin & table_join, bool no_subqueries, bool only_consts, ExpressionActionsPtr & actions); @@ -404,11 +343,6 @@ private: void assertSelect() const; void assertAggregation() const; - /** Create Set from an explicit enumeration of values in the query. - * If create_ordered_set = true - create a data structure suitable for using the index. - */ - void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool create_ordered_set); - /** * Create Set from a subuqery or a table expression in the query. The created set is suitable for using the index. * The set will not be created if its size hits the limit. @@ -427,6 +361,8 @@ private: * This is the case when we have DISTINCT or arrayJoin: we require more columns in SELECT even if we need less columns in result. */ void removeUnneededColumnsFromSelectClause(); + + bool noStorageOrLocal() const; }; } diff --git a/dbms/src/Interpreters/ProjectionManipulation.cpp b/dbms/src/Interpreters/ProjectionManipulation.cpp index 1fb3bb071fb..afdc289d2fa 100644 --- a/dbms/src/Interpreters/ProjectionManipulation.cpp +++ b/dbms/src/Interpreters/ProjectionManipulation.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Interpreters/ProjectionManipulation.h b/dbms/src/Interpreters/ProjectionManipulation.h index 5c707908af3..b5529b5ee5c 100644 --- a/dbms/src/Interpreters/ProjectionManipulation.h +++ b/dbms/src/Interpreters/ProjectionManipulation.h @@ -1,13 +1,18 @@ #pragma once #include +#include +#include +#include namespace DB { -class ExpressionAnalyzer; +class ExpressionAnalyzer; +class Context; struct ScopeStack; + namespace ErrorCodes { extern const int CONDITIONAL_TREE_PARENT_NOT_FOUND; diff --git a/dbms/src/Interpreters/interpretSubquery.cpp b/dbms/src/Interpreters/interpretSubquery.cpp new file mode 100644 index 00000000000..305e76f7d4a --- /dev/null +++ b/dbms/src/Interpreters/interpretSubquery.cpp @@ -0,0 +1,130 @@ +#include +#include + +#include + +#include +#include +#include +#include +#include + +#include +#include + +namespace DB +{ + +std::shared_ptr interpretSubquery( + const ASTPtr & table_expression, const Context & context, size_t subquery_depth, const Names & required_source_columns) +{ + /// Subquery or table name. The name of the table is similar to the subquery `SELECT * FROM t`. + const ASTSubquery * subquery = typeid_cast(table_expression.get()); + const ASTFunction * function = typeid_cast(table_expression.get()); + const ASTIdentifier * table = typeid_cast(table_expression.get()); + + if (!subquery && !table && !function) + throw Exception("Table expression is undefined, Method: ExpressionAnalyzer::interpretSubquery." , ErrorCodes::LOGICAL_ERROR); + + /** The subquery in the IN / JOIN section does not have any restrictions on the maximum size of the result. + * Because the result of this query is not the result of the entire query. + * Constraints work instead + * max_rows_in_set, max_bytes_in_set, set_overflow_mode, + * max_rows_in_join, max_bytes_in_join, join_overflow_mode, + * which are checked separately (in the Set, Join objects). + */ + Context subquery_context = context; + Settings subquery_settings = context.getSettings(); + subquery_settings.max_result_rows = 0; + subquery_settings.max_result_bytes = 0; + /// The calculation of `extremes` does not make sense and is not necessary (if you do it, then the `extremes` of the subquery can be taken instead of the whole query). + subquery_settings.extremes = 0; + subquery_context.setSettings(subquery_settings); + + ASTPtr query; + if (table || function) + { + /// create ASTSelectQuery for "SELECT * FROM table" as if written by hand + const auto select_with_union_query = std::make_shared(); + query = select_with_union_query; + + select_with_union_query->list_of_selects = std::make_shared(); + + const auto select_query = std::make_shared(); + select_with_union_query->list_of_selects->children.push_back(select_query); + + const auto select_expression_list = std::make_shared(); + select_query->select_expression_list = select_expression_list; + select_query->children.emplace_back(select_query->select_expression_list); + + NamesAndTypesList columns; + + /// get columns list for target table + if (function) + { + auto query_context = const_cast(&context.getQueryContext()); + const auto & storage = query_context->executeTableFunction(table_expression); + columns = storage->getColumns().ordinary; + select_query->addTableFunction(*const_cast(&table_expression)); + } + else + { + auto database_table = getDatabaseAndTableNameFromIdentifier(*table); + const auto & storage = context.getTable(database_table.first, database_table.second); + columns = storage->getColumns().ordinary; + select_query->replaceDatabaseAndTable(database_table.first, database_table.second); + } + + select_expression_list->children.reserve(columns.size()); + /// manually substitute column names in place of asterisk + for (const auto & column : columns) + select_expression_list->children.emplace_back(std::make_shared(column.name)); + } + else + { + query = subquery->children.at(0); + + /** Columns with the same name can be specified in a subquery. For example, SELECT x, x FROM t + * This is bad, because the result of such a query can not be saved to the table, because the table can not have the same name columns. + * Saving to the table is required for GLOBAL subqueries. + * + * To avoid this situation, we will rename the same columns. + */ + + std::set all_column_names; + std::set assigned_column_names; + + if (ASTSelectWithUnionQuery * select_with_union = typeid_cast(query.get())) + { + if (ASTSelectQuery * select = typeid_cast(select_with_union->list_of_selects->children.at(0).get())) + { + for (auto & expr : select->select_expression_list->children) + all_column_names.insert(expr->getAliasOrColumnName()); + + for (auto & expr : select->select_expression_list->children) + { + auto name = expr->getAliasOrColumnName(); + + if (!assigned_column_names.insert(name).second) + { + size_t i = 1; + while (all_column_names.end() != all_column_names.find(name + "_" + toString(i))) + ++i; + + name = name + "_" + toString(i); + expr = expr->clone(); /// Cancels fuse of the same expressions in the tree. + expr->setAlias(name); + + all_column_names.insert(name); + assigned_column_names.insert(name); + } + } + } + } + } + + return std::make_shared( + query, subquery_context, required_source_columns, QueryProcessingStage::Complete, subquery_depth + 1); +} + +} diff --git a/dbms/src/Interpreters/interpretSubquery.h b/dbms/src/Interpreters/interpretSubquery.h new file mode 100644 index 00000000000..2af72eea6a9 --- /dev/null +++ b/dbms/src/Interpreters/interpretSubquery.h @@ -0,0 +1,14 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class Context; + +std::shared_ptr interpretSubquery( + const ASTPtr & table_expression, const Context & context, size_t subquery_depth, const Names & required_source_columns); + +} From 4071155043f8a8adcba13c622e39087aa1701620 Mon Sep 17 00:00:00 2001 From: chertus Date: Tue, 16 Oct 2018 15:34:20 +0300 Subject: [PATCH 02/28] better ActionsVisitor extraction [CLICKHOUSE-3996] --- dbms/src/Interpreters/ActionsVisitor.cpp | 36 ++++++-- dbms/src/Interpreters/ActionsVisitor.h | 31 +++---- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 97 +++++++++----------- dbms/src/Interpreters/ExpressionAnalyzer.h | 4 +- 4 files changed, 86 insertions(+), 82 deletions(-) diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index bde379fa69b..29d8f190fbf 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -220,7 +220,31 @@ const Block & ScopeStack::getSampleBlock() const } -void ActionsVisitor::visit(const ASTPtr & ast, ScopeStack & actions_stack, ProjectionManipulatorPtr projection_manipulator) +ActionsVisitor::ActionsVisitor( + const Context & context_, SizeLimits set_size_limit_, bool is_conditional_tree, size_t subquery_depth_, + const NamesAndTypesList & source_columns_, const ExpressionActionsPtr & actions, + PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, + bool no_subqueries_, bool only_consts_, bool no_storage_or_local_, std::ostream * ostr_) +: context(context_), + set_size_limit(set_size_limit_), + subquery_depth(subquery_depth_), + source_columns(source_columns_), + prepared_sets(prepared_sets_), + subqueries_for_sets(subqueries_for_sets_), + no_subqueries(no_subqueries_), + only_consts(only_consts_), + no_storage_or_local(no_storage_or_local_), + visit_depth(0), + ostr(ostr_), + actions_stack(actions, context) +{ + if (is_conditional_tree) + projection_manipulator = std::make_shared(actions_stack, context); + else + projection_manipulator = std::make_shared(actions_stack); +} + +void ActionsVisitor::visit(const ASTPtr & ast) { DumpASTNode dump(*ast, ostr, visit_depth, "getActions"); @@ -267,7 +291,7 @@ void ActionsVisitor::visit(const ASTPtr & ast, ScopeStack & actions_stack, Proje throw Exception("arrayJoin requires exactly 1 argument", ErrorCodes::TYPE_MISMATCH); ASTPtr arg = node->arguments->children.at(0); - visit(arg, actions_stack, projection_manipulator); + visit(arg); if (!only_consts) { String result_name = projection_manipulator->getColumnName(getColumnName()); @@ -283,7 +307,7 @@ void ActionsVisitor::visit(const ASTPtr & ast, ScopeStack & actions_stack, Proje if (functionIsInOrGlobalInOperator(node->name)) { /// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything). - visit(node->arguments->children.at(0), actions_stack, projection_manipulator); + visit(node->arguments->children.at(0)); if (!no_subqueries) { @@ -387,7 +411,7 @@ void ActionsVisitor::visit(const ASTPtr & ast, ScopeStack & actions_stack, Proje { /// If the argument is not a lambda expression, call it recursively and find out its type. projection_action->preArgumentAction(); - visit(child, actions_stack, projection_manipulator); + visit(child); std::string name = projection_manipulator->getColumnName(child_column_name); projection_action->postArgumentAction(child_column_name); if (actions_stack.getSampleBlock().has(name)) @@ -442,7 +466,7 @@ void ActionsVisitor::visit(const ASTPtr & ast, ScopeStack & actions_stack, Proje projection_action->preArgumentAction(); actions_stack.pushLevel(lambda_arguments); - visit(lambda->arguments->children.at(1), actions_stack, projection_manipulator); + visit(lambda->arguments->children.at(1)); ExpressionActionsPtr lambda_actions = actions_stack.popLevel(); String result_name = projection_manipulator->getColumnName(lambda->arguments->children.at(1)->getColumnName()); @@ -515,7 +539,7 @@ void ActionsVisitor::visit(const ASTPtr & ast, ScopeStack & actions_stack, Proje /// Do not go to FROM, JOIN, UNION. if (!typeid_cast(child.get()) && !typeid_cast(child.get())) - visit(child, actions_stack, projection_manipulator); + visit(child); } } } diff --git a/dbms/src/Interpreters/ActionsVisitor.h b/dbms/src/Interpreters/ActionsVisitor.h index 78b91b64afc..41560c55c7c 100644 --- a/dbms/src/Interpreters/ActionsVisitor.h +++ b/dbms/src/Interpreters/ActionsVisitor.h @@ -8,12 +8,11 @@ namespace DB class Context; class ASTFunction; +struct ProjectionManipulatorBase; class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; -struct ProjectionManipulatorBase; -using ProjectionManipulatorPtr = std::shared_ptr; class Set; using SetPtr = std::shared_ptr; @@ -87,27 +86,19 @@ struct ScopeStack }; -/// TODO: There sould be some description, but... +/// Collect ExpressionAction from AST. Returns PreparedSets and SubqueriesForSets too. +/// After AST is visited source ExpressionActions should be updated with popActionsLevel() method. class ActionsVisitor { public: - ActionsVisitor(const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, - const NamesAndTypesList & source_columns_, PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, - bool no_subqueries_, bool only_consts_, bool no_storage_or_local_, std::ostream * ostr_ = nullptr) - : context(context_), - set_size_limit(set_size_limit_), - subquery_depth(subquery_depth_), - source_columns(source_columns_), - prepared_sets(prepared_sets_), - subqueries_for_sets(subqueries_for_sets_), - no_subqueries(no_subqueries_), - only_consts(only_consts_), - no_storage_or_local(no_storage_or_local_), - visit_depth(0), - ostr(ostr_) - {} + ActionsVisitor(const Context & context_, SizeLimits set_size_limit_, bool is_conditional_tree, size_t subquery_depth_, + const NamesAndTypesList & source_columns_, const ExpressionActionsPtr & actions, + PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, + bool no_subqueries_, bool only_consts_, bool no_storage_or_local_, std::ostream * ostr_ = nullptr); - void visit(const ASTPtr & ast, ScopeStack & actions_stack, ProjectionManipulatorPtr projection_manipulator); + void visit(const ASTPtr & ast); + + ExpressionActionsPtr popActionsLevel() { return actions_stack.popLevel(); } private: const Context & context; @@ -121,6 +112,8 @@ private: const bool no_storage_or_local; mutable size_t visit_depth; std::ostream * ostr; + ScopeStack actions_stack; + std::shared_ptr projection_manipulator; void makeSet(const ASTFunction * node, const Block & sample_block); }; diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 15938a3f52a..d04f251cb43 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -29,7 +29,6 @@ #include #include #include -#include #include #include @@ -439,7 +438,7 @@ void ExpressionAnalyzer::analyzeAggregation() if (select_query && select_query->array_join_expression_list()) { - getRootActions(select_query->array_join_expression_list(), true, false, temp_actions); + getRootActions(select_query->array_join_expression_list(), true, temp_actions); addMultipleArrayJoinAction(temp_actions); array_join_columns = temp_actions->getSampleBlock().getNamesAndTypesList(); } @@ -451,10 +450,10 @@ void ExpressionAnalyzer::analyzeAggregation() { const auto table_join = static_cast(*join->table_join); if (table_join.using_expression_list) - getRootActions(table_join.using_expression_list, true, false, temp_actions); + getRootActions(table_join.using_expression_list, true, temp_actions); if (table_join.on_expression) for (const auto & key_ast : analyzed_join.key_asts_left) - getRootActions(key_ast, true, false, temp_actions); + getRootActions(key_ast, true, temp_actions); addJoinAction(temp_actions, true); } @@ -474,7 +473,7 @@ void ExpressionAnalyzer::analyzeAggregation() for (ssize_t i = 0; i < ssize_t(group_asts.size()); ++i) { ssize_t size = group_asts.size(); - getRootActions(group_asts[i], true, false, temp_actions); + getRootActions(group_asts[i], true, temp_actions); const auto & column_name = group_asts[i]->getColumnName(); const auto & block = temp_actions->getSampleBlock(); @@ -1053,7 +1052,7 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block & for (const auto & joined_column : analyzed_join.columns_added_by_join) temp_columns.push_back(joined_column.name_and_type); ExpressionActionsPtr temp_actions = std::make_shared(temp_columns, context); - getRootActions(func->arguments->children.at(0), true, false, temp_actions); + getRootActions(func->arguments->children.at(0), true, temp_actions); Block sample_block_with_calculated_columns = temp_actions->getSampleBlock(); if (sample_block_with_calculated_columns.has(args.children.at(0)->getColumnName())) @@ -1065,25 +1064,6 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block & } -void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, bool only_consts, ExpressionActionsPtr & actions) -{ - ScopeStack scopes(actions, context); - - ProjectionManipulatorPtr projection_manipulator; - if (!isThereArrayJoin(ast) && settings.enable_conditional_computation && !only_consts) - projection_manipulator = std::make_shared(scopes, context); - else - projection_manipulator = std::make_shared(scopes); - - LogAST log; - ActionsVisitor actions_visitor(context, getSetSizeLimits(settings), subquery_depth, - source_columns, prepared_sets, subqueries_for_sets, - no_subqueries, only_consts, noStorageOrLocal(), log.stream()); - actions_visitor.visit(ast, scopes, projection_manipulator); - - actions = scopes.popLevel(); -} - void ExpressionAnalyzer::getArrayJoinedColumns() { if (select_query && select_query->array_join_expression_list()) @@ -1239,31 +1219,39 @@ bool ExpressionAnalyzer::isThereArrayJoin(const ASTPtr & ast) } } -void ExpressionAnalyzer::getActionsFromJoinKeys(const ASTTableJoin & table_join, bool no_subqueries, bool only_consts, - ExpressionActionsPtr & actions) -{ - ScopeStack scopes(actions, context); - ProjectionManipulatorPtr projection_manipulator; - if (!isThereArrayJoin(query) && settings.enable_conditional_computation && !only_consts) - projection_manipulator = std::make_shared(scopes, context); - else - projection_manipulator = std::make_shared(scopes); +void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts) +{ + bool is_conditional_tree = !isThereArrayJoin(ast) && settings.enable_conditional_computation && !only_consts; LogAST log; - ActionsVisitor actions_visitor(context, getSetSizeLimits(settings), subquery_depth, - source_columns, prepared_sets, subqueries_for_sets, + ActionsVisitor actions_visitor(context, getSetSizeLimits(settings), is_conditional_tree, subquery_depth, + source_columns, actions, prepared_sets, subqueries_for_sets, + no_subqueries, only_consts, noStorageOrLocal(), log.stream()); + actions_visitor.visit(ast); + actions = actions_visitor.popActionsLevel(); +} + + +void ExpressionAnalyzer::getActionsFromJoinKeys(const ASTTableJoin & table_join, bool no_subqueries, ExpressionActionsPtr & actions) +{ + bool only_consts = false; + bool is_conditional_tree = !isThereArrayJoin(query) && settings.enable_conditional_computation && !only_consts; + + LogAST log; + ActionsVisitor actions_visitor(context, getSetSizeLimits(settings), is_conditional_tree, subquery_depth, + source_columns, actions, prepared_sets, subqueries_for_sets, no_subqueries, only_consts, noStorageOrLocal(), log.stream()); if (table_join.using_expression_list) - actions_visitor.visit(table_join.using_expression_list, scopes, projection_manipulator); + actions_visitor.visit(table_join.using_expression_list); else if (table_join.on_expression) { for (const auto & ast : analyzed_join.key_asts_left) - actions_visitor.visit(ast, scopes, projection_manipulator); + actions_visitor.visit(ast); } - actions = scopes.popLevel(); + actions = actions_visitor.popActionsLevel(); } @@ -1304,7 +1292,7 @@ void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr /// There can not be other aggregate functions within the aggregate functions. assertNoAggregates(arguments[i], "inside another aggregate function"); - getRootActions(arguments[i], true, false, actions); + getRootActions(arguments[i], true, actions); const std::string & name = arguments[i]->getColumnName(); types[i] = actions->getSampleBlock().getByName(name).type; aggregate.argument_names[i] = name; @@ -1387,7 +1375,7 @@ bool ExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, bool on initChain(chain, source_columns); ExpressionActionsChain::Step & step = chain.steps.back(); - getRootActions(select_query->array_join_expression_list(), only_types, false, step.actions); + getRootActions(select_query->array_join_expression_list(), only_types, step.actions); addMultipleArrayJoinAction(step.actions); @@ -1519,7 +1507,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty const auto & table_to_join = static_cast(*join_element.table_expression); - getActionsFromJoinKeys(join_params, only_types, false, step.actions); + getActionsFromJoinKeys(join_params, only_types, step.actions); /// Two JOINs are not supported with the same subquery, but different USINGs. auto join_hash = join_element.getTreeHash(); @@ -1634,7 +1622,7 @@ bool ExpressionAnalyzer::appendPrewhere(ExpressionActionsChain & chain, bool onl initChain(chain, source_columns); auto & step = chain.getLastStep(); - getRootActions(select_query->prewhere_expression, only_types, false, step.actions); + getRootActions(select_query->prewhere_expression, only_types, step.actions); String prewhere_column_name = select_query->prewhere_expression->getColumnName(); step.required_output.push_back(prewhere_column_name); step.can_remove_required_output.push_back(true); @@ -1642,7 +1630,7 @@ bool ExpressionAnalyzer::appendPrewhere(ExpressionActionsChain & chain, bool onl { /// Remove unused source_columns from prewhere actions. auto tmp_actions = std::make_shared(source_columns, context); - getRootActions(select_query->prewhere_expression, only_types, false, tmp_actions); + getRootActions(select_query->prewhere_expression, only_types, tmp_actions); tmp_actions->finalize({prewhere_column_name}); auto required_columns = tmp_actions->getRequiredColumns(); NameSet required_source_columns(required_columns.begin(), required_columns.end()); @@ -1711,7 +1699,7 @@ bool ExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, bool only_t step.required_output.push_back(select_query->where_expression->getColumnName()); step.can_remove_required_output = {true}; - getRootActions(select_query->where_expression, only_types, false, step.actions); + getRootActions(select_query->where_expression, only_types, step.actions); return true; } @@ -1730,7 +1718,7 @@ bool ExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain, bool only for (size_t i = 0; i < asts.size(); ++i) { step.required_output.push_back(asts[i]->getColumnName()); - getRootActions(asts[i], only_types, false, step.actions); + getRootActions(asts[i], only_types, step.actions); } return true; @@ -1771,7 +1759,7 @@ bool ExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_ ExpressionActionsChain::Step & step = chain.steps.back(); step.required_output.push_back(select_query->having_expression->getColumnName()); - getRootActions(select_query->having_expression, only_types, false, step.actions); + getRootActions(select_query->having_expression, only_types, step.actions); return true; } @@ -1783,7 +1771,7 @@ void ExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, bool only_ initChain(chain, aggregated_columns); ExpressionActionsChain::Step & step = chain.steps.back(); - getRootActions(select_query->select_expression_list, only_types, false, step.actions); + getRootActions(select_query->select_expression_list, only_types, step.actions); for (const auto & child : select_query->select_expression_list->children) step.required_output.push_back(child->getColumnName()); @@ -1799,7 +1787,7 @@ bool ExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only initChain(chain, aggregated_columns); ExpressionActionsChain::Step & step = chain.steps.back(); - getRootActions(select_query->order_expression_list, only_types, false, step.actions); + getRootActions(select_query->order_expression_list, only_types, step.actions); ASTs asts = select_query->order_expression_list->children; for (size_t i = 0; i < asts.size(); ++i) @@ -1824,7 +1812,7 @@ bool ExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain, bool only initChain(chain, aggregated_columns); ExpressionActionsChain::Step & step = chain.steps.back(); - getRootActions(select_query->limit_by_expression_list, only_types, false, step.actions); + getRootActions(select_query->limit_by_expression_list, only_types, step.actions); for (const auto & child : select_query->limit_by_expression_list->children) step.required_output.push_back(child->getColumnName()); @@ -1861,7 +1849,7 @@ void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const { initChain(chain, source_columns); ExpressionActionsChain::Step & step = chain.steps.back(); - getRootActions(expr, only_types, false, step.actions); + getRootActions(expr, only_types, step.actions); step.required_output.push_back(expr->getColumnName()); } @@ -1872,7 +1860,7 @@ void ExpressionAnalyzer::getActionsBeforeAggregation(const ASTPtr & ast, Express if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name)) for (auto & argument : node->arguments->children) - getRootActions(argument, no_subqueries, false, actions); + getRootActions(argument, no_subqueries, actions); else for (auto & child : ast->children) getActionsBeforeAggregation(child, actions, no_subqueries); @@ -1902,7 +1890,7 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool proje alias = name; result_columns.emplace_back(name, alias); result_names.push_back(alias); - getRootActions(asts[i], false, false, actions); + getRootActions(asts[i], false, actions); } if (add_aliases) @@ -1930,8 +1918,7 @@ ExpressionActionsPtr ExpressionAnalyzer::getConstActions() { ExpressionActionsPtr actions = std::make_shared(NamesAndTypesList(), context); - getRootActions(query, true, true, actions); - + getRootActions(query, true, actions, true); return actions; } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index c4619bba078..e816b947431 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -313,9 +313,9 @@ private: bool isThereArrayJoin(const ASTPtr & ast); /// If ast is ASTSelectQuery with JOIN, add actions for JOIN key columns. - void getActionsFromJoinKeys(const ASTTableJoin & table_join, bool no_subqueries, bool only_consts, ExpressionActionsPtr & actions); + void getActionsFromJoinKeys(const ASTTableJoin & table_join, bool no_subqueries, ExpressionActionsPtr & actions); - void getRootActions(const ASTPtr & ast, bool no_subqueries, bool only_consts, ExpressionActionsPtr & actions); + void getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts = false); void getActionsBeforeAggregation(const ASTPtr & ast, ExpressionActionsPtr & actions, bool no_subqueries); From 27d66a45c38479116aa88da4ac400bcd9ce86cb4 Mon Sep 17 00:00:00 2001 From: chertus Date: Tue, 16 Oct 2018 22:00:05 +0300 Subject: [PATCH 03/28] extract more visitors from ExpressionAnalyzer [CLICKHOUSE-3996] --- .../Interpreters/ArrayJoinedColumnsVisitor.h | 94 ++++++ .../ExecuteScalarSubqueriesVisitor.cpp | 6 +- .../ExecuteScalarSubqueriesVisitor.h | 8 +- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 317 ++---------------- dbms/src/Interpreters/ExpressionAnalyzer.h | 15 - dbms/src/Interpreters/ExternalTablesVisitor.h | 47 +++ .../Interpreters/GlobalSubqueriesVisitor.h | 172 ++++++++++ .../RequiredSourceColumnsVisitor.h | 136 ++++++++ 8 files changed, 483 insertions(+), 312 deletions(-) create mode 100644 dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h create mode 100644 dbms/src/Interpreters/ExternalTablesVisitor.h create mode 100644 dbms/src/Interpreters/GlobalSubqueriesVisitor.h create mode 100644 dbms/src/Interpreters/RequiredSourceColumnsVisitor.h diff --git a/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h b/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h new file mode 100644 index 00000000000..427811e2c18 --- /dev/null +++ b/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h @@ -0,0 +1,94 @@ +#pragma once + +namespace DB +{ + +/// Fills the array_join_result_to_source: on which columns-arrays to replicate, and how to call them after that. +class ArrayJoinedColumnsVisitor +{ +public: + ArrayJoinedColumnsVisitor(NameToNameMap & array_join_name_to_alias_, + NameToNameMap & array_join_alias_to_name_, + NameToNameMap & array_join_result_to_source_) + : array_join_name_to_alias(array_join_name_to_alias_), + array_join_alias_to_name(array_join_alias_to_name_), + array_join_result_to_source(array_join_result_to_source_) + {} + + void visit(ASTPtr & ast) const + { + if (!tryVisit(ast) && + !tryVisit(ast)) + visitChildren(ast); + } + +private: + NameToNameMap & array_join_name_to_alias; + NameToNameMap & array_join_alias_to_name; + NameToNameMap & array_join_result_to_source; + + void visit(const ASTTablesInSelectQuery *, ASTPtr &) const + {} + + void visit(const ASTIdentifier * node, ASTPtr &) const + { + if (node->general()) + { + auto splitted = Nested::splitName(node->name); /// ParsedParams, Key1 + + if (array_join_alias_to_name.count(node->name)) + { + /// ARRAY JOIN was written with an array column. Example: SELECT K1 FROM ... ARRAY JOIN ParsedParams.Key1 AS K1 + array_join_result_to_source[node->name] = array_join_alias_to_name[node->name]; /// K1 -> ParsedParams.Key1 + } + else if (array_join_alias_to_name.count(splitted.first) && !splitted.second.empty()) + { + /// ARRAY JOIN was written with a nested table. Example: SELECT PP.KEY1 FROM ... ARRAY JOIN ParsedParams AS PP + array_join_result_to_source[node->name] /// PP.Key1 -> ParsedParams.Key1 + = Nested::concatenateName(array_join_alias_to_name[splitted.first], splitted.second); + } + else if (array_join_name_to_alias.count(node->name)) + { + /** Example: SELECT ParsedParams.Key1 FROM ... ARRAY JOIN ParsedParams.Key1 AS PP.Key1. + * That is, the query uses the original array, replicated by itself. + */ + array_join_result_to_source[ /// PP.Key1 -> ParsedParams.Key1 + array_join_name_to_alias[node->name]] = node->name; + } + else if (array_join_name_to_alias.count(splitted.first) && !splitted.second.empty()) + { + /** Example: SELECT ParsedParams.Key1 FROM ... ARRAY JOIN ParsedParams AS PP. + */ + array_join_result_to_source[ /// PP.Key1 -> ParsedParams.Key1 + Nested::concatenateName(array_join_name_to_alias[splitted.first], splitted.second)] = node->name; + } + } + } + + void visit(const ASTSubquery *, ASTPtr &) const + {} + + void visit(const ASTSelectQuery *, ASTPtr &) const + {} + + void visitChildren(ASTPtr & ast) const + { + for (auto & child : ast->children) + if (!tryVisit(child) && + !tryVisit(child)) + visit(child); + } + + template + bool tryVisit(ASTPtr & ast) const + { + if (const T * t = typeid_cast(ast.get())) + { + visit(t, ast); + return true; + } + return false; + } +}; + +} diff --git a/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index 4716ac09517..a269cd00867 100644 --- a/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -35,7 +35,7 @@ static ASTPtr addTypeConversion(std::unique_ptr && ast, const String return res; } -void ExecuteScalarSubqueriesVisitor::visit(ASTSubquery * subquery, ASTPtr & ast, const DumpASTNode &) const +void ExecuteScalarSubqueriesVisitor::visit(const ASTSubquery * subquery, ASTPtr & ast, const DumpASTNode &) const { Context subquery_context = context; Settings subquery_settings = context.getSettings(); @@ -101,12 +101,12 @@ void ExecuteScalarSubqueriesVisitor::visit(ASTSubquery * subquery, ASTPtr & ast, } -void ExecuteScalarSubqueriesVisitor::visit(ASTTableExpression *, ASTPtr &, const DumpASTNode &) const +void ExecuteScalarSubqueriesVisitor::visit(const ASTTableExpression *, ASTPtr &, const DumpASTNode &) const { /// Don't descend into subqueries in FROM section. } -void ExecuteScalarSubqueriesVisitor::visit(ASTFunction * func, ASTPtr & ast, const DumpASTNode &) const +void ExecuteScalarSubqueriesVisitor::visit(const ASTFunction * func, ASTPtr & ast, const DumpASTNode &) const { /// Don't descend into subqueries in arguments of IN operator. /// But if an argument is not subquery, than deeper may be scalar subqueries and we need to descend in them. diff --git a/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.h b/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.h index 9688ac253cc..163be9a1292 100644 --- a/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.h +++ b/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.h @@ -54,9 +54,9 @@ private: mutable size_t visit_depth; std::ostream * ostr; - void visit(ASTSubquery * subquery, ASTPtr & ast, const DumpASTNode & dump) const; - void visit(ASTFunction * func, ASTPtr & ast, const DumpASTNode &) const; - void visit(ASTTableExpression *, ASTPtr &, const DumpASTNode &) const; + void visit(const ASTSubquery * subquery, ASTPtr & ast, const DumpASTNode & dump) const; + void visit(const ASTFunction * func, ASTPtr & ast, const DumpASTNode &) const; + void visit(const ASTTableExpression *, ASTPtr &, const DumpASTNode &) const; void visitChildren(ASTPtr & ast) const { @@ -67,7 +67,7 @@ private: template bool tryVisit(ASTPtr & ast, const DumpASTNode & dump) const { - if (T * t = typeid_cast(ast.get())) + if (const T * t = typeid_cast(ast.get())) { visit(t, ast, dump); return true; diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index d04f251cb43..b5f81047a10 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -59,9 +59,13 @@ #include #include #include + #include #include - +#include +#include +#include +#include namespace DB { @@ -87,7 +91,6 @@ namespace ErrorCodes extern const int TOO_BIG_AST; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int CONDITIONAL_TREE_PARENT_NOT_FOUND; - extern const int TYPE_MISMATCH; extern const int INVALID_JOIN_ON_EXPRESSION; extern const int EXPECTED_ALL_OR_ANY; } @@ -535,149 +538,13 @@ void ExpressionAnalyzer::analyzeAggregation() void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables() { /// Adds existing external tables (not subqueries) to the external_tables dictionary. - findExternalTables(query); + ExternalTablesVisitor tables_visitor(context, external_tables); + tables_visitor.visit(query); - /// Converts GLOBAL subqueries to external tables; Puts them into the external_tables dictionary: name -> StoragePtr. - initGlobalSubqueries(query); -} - - -void ExpressionAnalyzer::initGlobalSubqueries(ASTPtr & ast) -{ - /// Recursive calls. We do not go into subqueries. - - for (auto & child : ast->children) - if (!typeid_cast(child.get())) - initGlobalSubqueries(child); - - /// Bottom-up actions. - - if (ASTFunction * func = typeid_cast(ast.get())) - { - /// For GLOBAL IN. - if (do_global && (func->name == "globalIn" || func->name == "globalNotIn")) - { - addExternalStorage(func->arguments->children.at(1)); - has_global_subqueries = true; - } - } - else if (ASTTablesInSelectQueryElement * table_elem = typeid_cast(ast.get())) - { - /// For GLOBAL JOIN. - if (do_global && table_elem->table_join - && static_cast(*table_elem->table_join).locality == ASTTableJoin::Locality::Global) - { - addExternalStorage(table_elem->table_expression); - has_global_subqueries = true; - } - } -} - - -void ExpressionAnalyzer::findExternalTables(ASTPtr & ast) -{ - /// Traverse from the bottom. Intentionally go into subqueries. - for (auto & child : ast->children) - findExternalTables(child); - - /// If table type identifier - StoragePtr external_storage; - - if (ASTIdentifier * node = typeid_cast(ast.get())) - if (node->special()) - if ((external_storage = context.tryGetExternalTable(node->name))) - external_tables[node->name] = external_storage; -} - - -void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name_or_table_expression) -{ - /// With nondistributed queries, creating temporary tables does not make sense. - if (!(storage && storage->isRemote())) - return; - - ASTPtr subquery; - ASTPtr table_name; - ASTPtr subquery_or_table_name; - - if (typeid_cast(subquery_or_table_name_or_table_expression.get())) - { - table_name = subquery_or_table_name_or_table_expression; - subquery_or_table_name = table_name; - } - else if (auto ast_table_expr = typeid_cast(subquery_or_table_name_or_table_expression.get())) - { - if (ast_table_expr->database_and_table_name) - { - table_name = ast_table_expr->database_and_table_name; - subquery_or_table_name = table_name; - } - else if (ast_table_expr->subquery) - { - subquery = ast_table_expr->subquery; - subquery_or_table_name = subquery; - } - } - else if (typeid_cast(subquery_or_table_name_or_table_expression.get())) - { - subquery = subquery_or_table_name_or_table_expression; - subquery_or_table_name = subquery; - } - - if (!subquery_or_table_name) - throw Exception("Logical error: unknown AST element passed to ExpressionAnalyzer::addExternalStorage method", ErrorCodes::LOGICAL_ERROR); - - if (table_name) - { - /// If this is already an external table, you do not need to add anything. Just remember its presence. - if (external_tables.end() != external_tables.find(static_cast(*table_name).name)) - return; - } - - /// Generate the name for the external table. - String external_table_name = "_data" + toString(external_table_id); - while (external_tables.count(external_table_name)) - { - ++external_table_id; - external_table_name = "_data" + toString(external_table_id); - } - - auto interpreter = interpretSubquery(subquery_or_table_name, context, subquery_depth, {}); - - Block sample = interpreter->getSampleBlock(); - NamesAndTypesList columns = sample.getNamesAndTypesList(); - - StoragePtr external_storage = StorageMemory::create(external_table_name, ColumnsDescription{columns}); - external_storage->startup(); - - /** We replace the subquery with the name of the temporary table. - * It is in this form, the request will go to the remote server. - * This temporary table will go to the remote server, and on its side, - * instead of doing a subquery, you just need to read it. - */ - - auto database_and_table_name = ASTIdentifier::createSpecial(external_table_name); - - if (auto ast_table_expr = typeid_cast(subquery_or_table_name_or_table_expression.get())) - { - ast_table_expr->subquery.reset(); - ast_table_expr->database_and_table_name = database_and_table_name; - - ast_table_expr->children.clear(); - ast_table_expr->children.emplace_back(database_and_table_name); - } - else - subquery_or_table_name_or_table_expression = database_and_table_name; - - external_tables[external_table_name] = external_storage; - subqueries_for_sets[external_table_name].source = interpreter->execute().in; - subqueries_for_sets[external_table_name].table = external_storage; - - /** NOTE If it was written IN tmp_table - the existing temporary (but not external) table, - * then a new temporary table will be created (for example, _data1), - * and the data will then be copied to it. - * Maybe this can be avoided. - */ + bool is_remote = storage && storage->isRemote(); + GlobalSubqueriesVisitor subqueries_visitor(context, subquery_depth, do_global, is_remote, + external_tables, subqueries_for_sets, has_global_subqueries); + subqueries_visitor.visit(query); } @@ -778,8 +645,8 @@ void ExpressionAnalyzer::executeScalarSubqueries() if (!select_query) { - ExecuteScalarSubqueriesVisitor execute_scalar_subqueries_visitor(context, subquery_depth, log.stream()); - execute_scalar_subqueries_visitor.visit(query); + ExecuteScalarSubqueriesVisitor visitor(context, subquery_depth, log.stream()); + visitor.visit(query); } else { @@ -789,8 +656,8 @@ void ExpressionAnalyzer::executeScalarSubqueries() if (!typeid_cast(child.get()) && !typeid_cast(child.get())) { - ExecuteScalarSubqueriesVisitor execute_scalar_subqueries_visitor(context, subquery_depth, log.stream()); - execute_scalar_subqueries_visitor.visit(child); + ExecuteScalarSubqueriesVisitor visitor(context, subquery_depth, log.stream()); + visitor.visit(child); } } } @@ -1084,7 +951,10 @@ void ExpressionAnalyzer::getArrayJoinedColumns() array_join_name_to_alias[nested_table_name] = nested_table_alias; } - getArrayJoinedColumnsImpl(query); + { + ArrayJoinedColumnsVisitor visitor(array_join_name_to_alias, array_join_alias_to_name, array_join_result_to_source); + visitor.visit(query); + } /// If the result of ARRAY JOIN is not used, it is necessary to ARRAY-JOIN any column, /// to get the correct number of rows. @@ -1120,55 +990,6 @@ void ExpressionAnalyzer::getArrayJoinedColumns() } -/// Fills the array_join_result_to_source: on which columns-arrays to replicate, and how to call them after that. -void ExpressionAnalyzer::getArrayJoinedColumnsImpl(const ASTPtr & ast) -{ - if (typeid_cast(ast.get())) - return; - - if (ASTIdentifier * node = typeid_cast(ast.get())) - { - if (node->general()) - { - auto splitted = Nested::splitName(node->name); /// ParsedParams, Key1 - - if (array_join_alias_to_name.count(node->name)) - { - /// ARRAY JOIN was written with an array column. Example: SELECT K1 FROM ... ARRAY JOIN ParsedParams.Key1 AS K1 - array_join_result_to_source[node->name] = array_join_alias_to_name[node->name]; /// K1 -> ParsedParams.Key1 - } - else if (array_join_alias_to_name.count(splitted.first) && !splitted.second.empty()) - { - /// ARRAY JOIN was written with a nested table. Example: SELECT PP.KEY1 FROM ... ARRAY JOIN ParsedParams AS PP - array_join_result_to_source[node->name] /// PP.Key1 -> ParsedParams.Key1 - = Nested::concatenateName(array_join_alias_to_name[splitted.first], splitted.second); - } - else if (array_join_name_to_alias.count(node->name)) - { - /** Example: SELECT ParsedParams.Key1 FROM ... ARRAY JOIN ParsedParams.Key1 AS PP.Key1. - * That is, the query uses the original array, replicated by itself. - */ - array_join_result_to_source[ /// PP.Key1 -> ParsedParams.Key1 - array_join_name_to_alias[node->name]] = node->name; - } - else if (array_join_name_to_alias.count(splitted.first) && !splitted.second.empty()) - { - /** Example: SELECT ParsedParams.Key1 FROM ... ARRAY JOIN ParsedParams AS PP. - */ - array_join_result_to_source[ /// PP.Key1 -> ParsedParams.Key1 - Nested::concatenateName(array_join_name_to_alias[splitted.first], splitted.second)] = node->name; - } - } - } - else - { - for (auto & child : ast->children) - if (!typeid_cast(child.get()) - && !typeid_cast(child.get())) - getArrayJoinedColumnsImpl(child); - } -} - bool ExpressionAnalyzer::isThereArrayJoin(const ASTPtr & ast) { if (typeid_cast(ast.get())) @@ -1959,7 +1780,8 @@ void ExpressionAnalyzer::collectUsedColumns() { /// Nothing needs to be ignored for expressions in ARRAY JOIN. NameSet empty; - getRequiredSourceColumnsImpl(expressions[i], available_columns, required, empty, empty, empty); + RequiredSourceColumnsVisitor visitor(available_columns, required, empty, empty, empty); + visitor.visit(expressions[i]); } ignored.insert(expressions[i]->getAliasOrColumnName()); @@ -1975,9 +1797,13 @@ void ExpressionAnalyzer::collectUsedColumns() NameSet required_joined_columns; for (const auto & left_key_ast : analyzed_join.key_asts_left) - getRequiredSourceColumnsImpl(left_key_ast, available_columns, required, ignored, {}, required_joined_columns); + { + RequiredSourceColumnsVisitor columns_visitor(available_columns, required, ignored, {}, required_joined_columns); + columns_visitor.visit(left_key_ast); + } - getRequiredSourceColumnsImpl(query, available_columns, required, ignored, available_joined_columns, required_joined_columns); + RequiredSourceColumnsVisitor columns_visitor(available_columns, required, ignored, available_joined_columns, required_joined_columns); + columns_visitor.visit(query); for (auto it = analyzed_join.columns_added_by_join.begin(); it != analyzed_join.columns_added_by_join.end();) { @@ -2276,95 +2102,6 @@ Names ExpressionAnalyzer::getRequiredSourceColumns() const } -void ExpressionAnalyzer::getRequiredSourceColumnsImpl(const ASTPtr & ast, - const NameSet & available_columns, NameSet & required_source_columns, NameSet & ignored_names, - const NameSet & available_joined_columns, NameSet & required_joined_columns) -{ - /** Find all the identifiers in the query. - * We will use depth first search in AST. - * In this case - * - for lambda functions we will not take formal parameters; - * - do not go into subqueries (they have their own identifiers); - * - there is some exception for the ARRAY JOIN clause (it has a slightly different identifiers); - * - we put identifiers available from JOIN in required_joined_columns. - */ - - if (ASTIdentifier * node = typeid_cast(ast.get())) - { - if (node->general() - && !ignored_names.count(node->name) - && !ignored_names.count(Nested::extractTableName(node->name))) - { - if (!available_joined_columns.count(node->name) - || available_columns.count(node->name)) /// Read column from left table if has. - required_source_columns.insert(node->name); - else - required_joined_columns.insert(node->name); - } - - return; - } - - if (ASTFunction * node = typeid_cast(ast.get())) - { - if (node->name == "lambda") - { - if (node->arguments->children.size() != 2) - throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - ASTFunction * lambda_args_tuple = typeid_cast(node->arguments->children.at(0).get()); - - if (!lambda_args_tuple || lambda_args_tuple->name != "tuple") - throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH); - - /// You do not need to add formal parameters of the lambda expression in required_source_columns. - Names added_ignored; - for (auto & child : lambda_args_tuple->arguments->children) - { - ASTIdentifier * identifier = typeid_cast(child.get()); - if (!identifier) - throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH); - - String & name = identifier->name; - if (!ignored_names.count(name)) - { - ignored_names.insert(name); - added_ignored.push_back(name); - } - } - - getRequiredSourceColumnsImpl(node->arguments->children.at(1), - available_columns, required_source_columns, ignored_names, - available_joined_columns, required_joined_columns); - - for (size_t i = 0; i < added_ignored.size(); ++i) - ignored_names.erase(added_ignored[i]); - - return; - } - - /// A special function `indexHint`. Everything that is inside it is not calculated - /// (and is used only for index analysis, see KeyCondition). - if (node->name == "indexHint") - return; - } - - /// Recursively traverses an expression. - for (auto & child : ast->children) - { - /** We will not go to the ARRAY JOIN section, because we need to look at the names of non-ARRAY-JOIN columns. - * There, `collectUsedColumns` will send us separately. - */ - if (!typeid_cast(child.get()) - && !typeid_cast(child.get()) - && !typeid_cast(child.get()) - && !typeid_cast(child.get())) - getRequiredSourceColumnsImpl(child, available_columns, required_source_columns, - ignored_names, available_joined_columns, required_joined_columns); - } -} - - static bool hasArrayJoin(const ASTPtr & ast) { if (const ASTFunction * function = typeid_cast(&*ast)) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index e816b947431..59e6268c0cc 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -248,7 +248,6 @@ private: /// All new temporary tables obtained by performing the GLOBAL IN/JOIN subqueries. Tables external_tables; - size_t external_table_id = 1; /// Predicate optimizer overrides the sub queries bool rewrite_subqueries = false; @@ -294,10 +293,6 @@ private: /// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables. void initGlobalSubqueriesAndExternalTables(); - void initGlobalSubqueries(ASTPtr & ast); - - /// Finds in the query the usage of external tables (as table identifiers). Fills in external_tables. - void findExternalTables(ASTPtr & ast); /** Initialize InterpreterSelectQuery for a subquery in the GLOBAL IN/JOIN section, * create a temporary table of type Memory and store it in the external_tables dictionary. @@ -305,7 +300,6 @@ private: void addExternalStorage(ASTPtr & subquery_or_table_name); void getArrayJoinedColumns(); - void getArrayJoinedColumnsImpl(const ASTPtr & ast); void addMultipleArrayJoinAction(ExpressionActionsPtr & actions) const; void addJoinAction(ExpressionActionsPtr & actions, bool only_types) const; @@ -328,15 +322,6 @@ private: void getAggregates(const ASTPtr & ast, ExpressionActionsPtr & actions); void assertNoAggregates(const ASTPtr & ast, const char * description); - /** Get a set of necessary columns to read from the table. - * In this case, the columns specified in ignored_names are considered unnecessary. And the ignored_names parameter can be modified. - * The set of columns available_joined_columns are the columns available from JOIN, they are not needed for reading from the main table. - * Put in required_joined_columns the set of columns available from JOIN and needed. - */ - void getRequiredSourceColumnsImpl(const ASTPtr & ast, - const NameSet & available_columns, NameSet & required_source_columns, NameSet & ignored_names, - const NameSet & available_joined_columns, NameSet & required_joined_columns); - /// columns - the columns that are present before the transformations begin. void initChain(ExpressionActionsChain & chain, const NamesAndTypesList & columns) const; diff --git a/dbms/src/Interpreters/ExternalTablesVisitor.h b/dbms/src/Interpreters/ExternalTablesVisitor.h new file mode 100644 index 00000000000..6bba88009c5 --- /dev/null +++ b/dbms/src/Interpreters/ExternalTablesVisitor.h @@ -0,0 +1,47 @@ +#pragma once + +namespace DB +{ + +/// Finds in the query the usage of external tables (as table identifiers). Fills in external_tables. +class ExternalTablesVisitor +{ +public: + ExternalTablesVisitor(const Context & context_, Tables & tables) + : context(context_), + external_tables(tables) + {} + + void visit(ASTPtr & ast) const + { + /// Traverse from the bottom. Intentionally go into subqueries. + for (auto & child : ast->children) + visit(child); + + tryVisit(ast); + } + +private: + const Context & context; + Tables & external_tables; + + void visit(const ASTIdentifier * node, ASTPtr &) const + { + if (node->special()) + if (StoragePtr external_storage = context.tryGetExternalTable(node->name)) + external_tables[node->name] = external_storage; + } + + template + bool tryVisit(ASTPtr & ast) const + { + if (const T * t = typeid_cast(ast.get())) + { + visit(t, ast); + return true; + } + return false; + } +}; + +} diff --git a/dbms/src/Interpreters/GlobalSubqueriesVisitor.h b/dbms/src/Interpreters/GlobalSubqueriesVisitor.h new file mode 100644 index 00000000000..a2d83c70405 --- /dev/null +++ b/dbms/src/Interpreters/GlobalSubqueriesVisitor.h @@ -0,0 +1,172 @@ +#pragma once + +namespace DB +{ + +/// Converts GLOBAL subqueries to external tables; Puts them into the external_tables dictionary: name -> StoragePtr. +class GlobalSubqueriesVisitor +{ +public: + GlobalSubqueriesVisitor(const Context & context_, size_t subquery_depth_, bool do_global_, bool is_remote_, + Tables & tables, SubqueriesForSets & subqueries_for_sets_, bool & has_global_subqueries_) + : context(context_), + subquery_depth(subquery_depth_), + do_global(do_global_), + is_remote(is_remote_), + external_table_id(1), + external_tables(tables), + subqueries_for_sets(subqueries_for_sets_), + has_global_subqueries(has_global_subqueries_) + {} + + void visit(ASTPtr & ast) const + { + if (!do_global) + return; + + /// Recursive calls. We do not go into subqueries. + for (auto & child : ast->children) + if (!typeid_cast(child.get())) + visit(child); + + /// Bottom-up actions. + if (tryVisit(ast) || + tryVisit(ast)) + ; + } + +private: + const Context & context; + size_t subquery_depth; + bool do_global; + bool is_remote; + mutable size_t external_table_id = 1; + Tables & external_tables; + SubqueriesForSets & subqueries_for_sets; + bool & has_global_subqueries; + + /// GLOBAL IN + void visit(ASTFunction * func, ASTPtr &) const + { + if (func->name == "globalIn" || func->name == "globalNotIn") + { + addExternalStorage(func->arguments->children.at(1)); + has_global_subqueries = true; + } + } + + /// GLOBAL JOIN + void visit(ASTTablesInSelectQueryElement * table_elem, ASTPtr &) const + { + if (table_elem->table_join + && static_cast(*table_elem->table_join).locality == ASTTableJoin::Locality::Global) + { + addExternalStorage(table_elem->table_expression); + has_global_subqueries = true; + } + } + + template + bool tryVisit(ASTPtr & ast) const + { + if (T * t = typeid_cast(ast.get())) + { + visit(t, ast); + return true; + } + return false; + } + + void addExternalStorage(ASTPtr & subquery_or_table_name_or_table_expression) const + { + /// With nondistributed queries, creating temporary tables does not make sense. + if (!is_remote) + return; + + ASTPtr subquery; + ASTPtr table_name; + ASTPtr subquery_or_table_name; + + if (typeid_cast(subquery_or_table_name_or_table_expression.get())) + { + table_name = subquery_or_table_name_or_table_expression; + subquery_or_table_name = table_name; + } + else if (auto ast_table_expr = typeid_cast(subquery_or_table_name_or_table_expression.get())) + { + if (ast_table_expr->database_and_table_name) + { + table_name = ast_table_expr->database_and_table_name; + subquery_or_table_name = table_name; + } + else if (ast_table_expr->subquery) + { + subquery = ast_table_expr->subquery; + subquery_or_table_name = subquery; + } + } + else if (typeid_cast(subquery_or_table_name_or_table_expression.get())) + { + subquery = subquery_or_table_name_or_table_expression; + subquery_or_table_name = subquery; + } + + if (!subquery_or_table_name) + throw Exception("Logical error: unknown AST element passed to ExpressionAnalyzer::addExternalStorage method", + ErrorCodes::LOGICAL_ERROR); + + if (table_name) + { + /// If this is already an external table, you do not need to add anything. Just remember its presence. + if (external_tables.end() != external_tables.find(static_cast(*table_name).name)) + return; + } + + /// Generate the name for the external table. + String external_table_name = "_data" + toString(external_table_id); + while (external_tables.count(external_table_name)) + { + ++external_table_id; + external_table_name = "_data" + toString(external_table_id); + } + + auto interpreter = interpretSubquery(subquery_or_table_name, context, subquery_depth, {}); + + Block sample = interpreter->getSampleBlock(); + NamesAndTypesList columns = sample.getNamesAndTypesList(); + + StoragePtr external_storage = StorageMemory::create(external_table_name, ColumnsDescription{columns}); + external_storage->startup(); + + /** We replace the subquery with the name of the temporary table. + * It is in this form, the request will go to the remote server. + * This temporary table will go to the remote server, and on its side, + * instead of doing a subquery, you just need to read it. + */ + + auto database_and_table_name = ASTIdentifier::createSpecial(external_table_name); + + if (auto ast_table_expr = typeid_cast(subquery_or_table_name_or_table_expression.get())) + { + ast_table_expr->subquery.reset(); + ast_table_expr->database_and_table_name = database_and_table_name; + + ast_table_expr->children.clear(); + ast_table_expr->children.emplace_back(database_and_table_name); + } + else + subquery_or_table_name_or_table_expression = database_and_table_name; + + external_tables[external_table_name] = external_storage; + subqueries_for_sets[external_table_name].source = interpreter->execute().in; + subqueries_for_sets[external_table_name].table = external_storage; + + /** NOTE If it was written IN tmp_table - the existing temporary (but not external) table, + * then a new temporary table will be created (for example, _data1), + * and the data will then be copied to it. + * Maybe this can be avoided. + */ + } +}; + +} diff --git a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h new file mode 100644 index 00000000000..a1d09c10307 --- /dev/null +++ b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h @@ -0,0 +1,136 @@ +#pragma once + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int TYPE_MISMATCH; +} + + +/** Get a set of necessary columns to read from the table. + * In this case, the columns specified in ignored_names are considered unnecessary. And the ignored_names parameter can be modified. + * The set of columns available_joined_columns are the columns available from JOIN, they are not needed for reading from the main table. + * Put in required_joined_columns the set of columns available from JOIN and needed. + */ +class RequiredSourceColumnsVisitor +{ +public: + RequiredSourceColumnsVisitor(const NameSet & available_columns_, NameSet & required_source_columns_, NameSet & ignored_names_, + const NameSet & available_joined_columns_, NameSet & required_joined_columns_) + : available_columns(available_columns_), + required_source_columns(required_source_columns_), + ignored_names(ignored_names_), + available_joined_columns(available_joined_columns_), + required_joined_columns(required_joined_columns_) + {} + + /** Find all the identifiers in the query. + * We will use depth first search in AST. + * In this case + * - for lambda functions we will not take formal parameters; + * - do not go into subqueries (they have their own identifiers); + * - there is some exception for the ARRAY JOIN clause (it has a slightly different identifiers); + * - we put identifiers available from JOIN in required_joined_columns. + */ + void visit(const ASTPtr & ast) const + { + if (!tryVisit(ast) && + !tryVisit(ast)) + visitChildren(ast); + } + +private: + const NameSet & available_columns; + NameSet & required_source_columns; + NameSet & ignored_names; + const NameSet & available_joined_columns; + NameSet & required_joined_columns; + + void visit(const ASTIdentifier * node, const ASTPtr &) const + { + if (node->general() + && !ignored_names.count(node->name) + && !ignored_names.count(Nested::extractTableName(node->name))) + { + if (!available_joined_columns.count(node->name) + || available_columns.count(node->name)) /// Read column from left table if has. + required_source_columns.insert(node->name); + else + required_joined_columns.insert(node->name); + } + } + + void visit(const ASTFunction * node, const ASTPtr & ast) const + { + if (node->name == "lambda") + { + if (node->arguments->children.size() != 2) + throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + ASTFunction * lambda_args_tuple = typeid_cast(node->arguments->children.at(0).get()); + + if (!lambda_args_tuple || lambda_args_tuple->name != "tuple") + throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH); + + /// You do not need to add formal parameters of the lambda expression in required_source_columns. + Names added_ignored; + for (auto & child : lambda_args_tuple->arguments->children) + { + ASTIdentifier * identifier = typeid_cast(child.get()); + if (!identifier) + throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH); + + String & name = identifier->name; + if (!ignored_names.count(name)) + { + ignored_names.insert(name); + added_ignored.push_back(name); + } + } + + visit(node->arguments->children.at(1)); + + for (size_t i = 0; i < added_ignored.size(); ++i) + ignored_names.erase(added_ignored[i]); + + return; + } + + /// A special function `indexHint`. Everything that is inside it is not calculated + /// (and is used only for index analysis, see KeyCondition). + if (node->name == "indexHint") + return; + + visitChildren(ast); + } + + void visitChildren(const ASTPtr & ast) const + { + for (auto & child : ast->children) + { + /** We will not go to the ARRAY JOIN section, because we need to look at the names of non-ARRAY-JOIN columns. + * There, `collectUsedColumns` will send us separately. + */ + if (!typeid_cast(child.get()) + && !typeid_cast(child.get()) + && !typeid_cast(child.get()) + && !typeid_cast(child.get())) + visit(child); + } + } + + template + bool tryVisit(const ASTPtr & ast) const + { + if (const T * t = typeid_cast(ast.get())) + { + visit(t, ast); + return true; + } + return false; + } +}; + +} From 96435f68801fb61de369712a7e15a8e0af7cb8b0 Mon Sep 17 00:00:00 2001 From: Winter Zhang Date: Wed, 17 Oct 2018 16:19:33 +0800 Subject: [PATCH 04/28] Add Chinese Getting started docs (#3399) --- docs/toc_zh.yml | 16 +- .../example_datasets/amplab_benchmark.md | 124 +++++- .../example_datasets/criteo.md | 76 +++- .../example_datasets/nyc_taxi.md | 369 +++++++++++++++++- .../example_datasets/ontime.md | 319 ++++++++++++++- .../example_datasets/star_schema.md | 88 ++++- .../example_datasets/wikistat.md | 30 +- docs/zh/getting_started/index.md | 142 ++++++- 8 files changed, 1149 insertions(+), 15 deletions(-) mode change 120000 => 100644 docs/zh/getting_started/example_datasets/amplab_benchmark.md mode change 120000 => 100644 docs/zh/getting_started/example_datasets/criteo.md mode change 120000 => 100644 docs/zh/getting_started/example_datasets/nyc_taxi.md mode change 120000 => 100644 docs/zh/getting_started/example_datasets/ontime.md mode change 120000 => 100644 docs/zh/getting_started/example_datasets/star_schema.md mode change 120000 => 100644 docs/zh/getting_started/example_datasets/wikistat.md mode change 120000 => 100644 docs/zh/getting_started/index.md diff --git a/docs/toc_zh.yml b/docs/toc_zh.yml index 7a3bf5de7f0..4c6ce34f192 100644 --- a/docs/toc_zh.yml +++ b/docs/toc_zh.yml @@ -3,19 +3,19 @@ nav: - '介绍': - '概貌': 'index.md' - 'ClickHouse的独特功能': 'introduction/distinctive_features.md' - - 'ClickHouse功能可被视为缺点': 'introduction/features_considered_disadvantages.md' + - 'ClickHouse不擅长做什么': 'introduction/features_considered_disadvantages.md' - '性能': 'introduction/performance.md' - 'Yandex.Metrica使用案例': 'introduction/ya_metrika_task.md' -- '起步': +- '入门指南': - '部署运行': 'getting_started/index.md' - '示例数据集': - - 'OnTime': 'getting_started/example_datasets/ontime.md' - - 'New York Taxi data': 'getting_started/example_datasets/nyc_taxi.md' - - 'AMPLab Big Data Benchmark': 'getting_started/example_datasets/amplab_benchmark.md' - - 'WikiStat': 'getting_started/example_datasets/wikistat.md' - - 'Terabyte click logs from Criteo': 'getting_started/example_datasets/criteo.md' - - 'Star Schema Benchmark': 'getting_started/example_datasets/star_schema.md' + - '航班飞行数据': 'getting_started/example_datasets/ontime.md' + - '纽约市出租车数据': 'getting_started/example_datasets/nyc_taxi.md' + - 'AMPLab大数据基准测试': 'getting_started/example_datasets/amplab_benchmark.md' + - '维基访问数据': 'getting_started/example_datasets/wikistat.md' + - 'Criteo TB级别点击日志': 'getting_started/example_datasets/criteo.md' + - 'Star Schema基准测试': 'getting_started/example_datasets/star_schema.md' - '客户端': - '介绍': 'interfaces/index.md' diff --git a/docs/zh/getting_started/example_datasets/amplab_benchmark.md b/docs/zh/getting_started/example_datasets/amplab_benchmark.md deleted file mode 120000 index 78c93906bb0..00000000000 --- a/docs/zh/getting_started/example_datasets/amplab_benchmark.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/getting_started/example_datasets/amplab_benchmark.md \ No newline at end of file diff --git a/docs/zh/getting_started/example_datasets/amplab_benchmark.md b/docs/zh/getting_started/example_datasets/amplab_benchmark.md new file mode 100644 index 00000000000..415457c9403 --- /dev/null +++ b/docs/zh/getting_started/example_datasets/amplab_benchmark.md @@ -0,0 +1,123 @@ +# AMPLab 大数据基准测试 + +参考 + +需要您在注册一个免费的账号。注册时需要您提供信用卡、邮箱、电话等信息。之后可以在获取新的访问密钥 + +在控制台运行以下命令: + +```bash +sudo apt-get install s3cmd +mkdir tiny; cd tiny; +s3cmd sync s3://big-data-benchmark/pavlo/text-deflate/tiny/ . +cd .. +mkdir 1node; cd 1node; +s3cmd sync s3://big-data-benchmark/pavlo/text-deflate/1node/ . +cd .. +mkdir 5nodes; cd 5nodes; +s3cmd sync s3://big-data-benchmark/pavlo/text-deflate/5nodes/ . +cd .. +``` + +在ClickHouse运行如下查询: + +``` sql +CREATE TABLE rankings_tiny +( + pageURL String, + pageRank UInt32, + avgDuration UInt32 +) ENGINE = Log; + +CREATE TABLE uservisits_tiny +( + sourceIP String, + destinationURL String, + visitDate Date, + adRevenue Float32, + UserAgent String, + cCode FixedString(3), + lCode FixedString(6), + searchWord String, + duration UInt32 +) ENGINE = MergeTree(visitDate, visitDate, 8192); + +CREATE TABLE rankings_1node +( + pageURL String, + pageRank UInt32, + avgDuration UInt32 +) ENGINE = Log; + +CREATE TABLE uservisits_1node +( + sourceIP String, + destinationURL String, + visitDate Date, + adRevenue Float32, + UserAgent String, + cCode FixedString(3), + lCode FixedString(6), + searchWord String, + duration UInt32 +) ENGINE = MergeTree(visitDate, visitDate, 8192); + +CREATE TABLE rankings_5nodes_on_single +( + pageURL String, + pageRank UInt32, + avgDuration UInt32 +) ENGINE = Log; + +CREATE TABLE uservisits_5nodes_on_single +( + sourceIP String, + destinationURL String, + visitDate Date, + adRevenue Float32, + UserAgent String, + cCode FixedString(3), + lCode FixedString(6), + searchWord String, + duration UInt32 +) ENGINE = MergeTree(visitDate, visitDate, 8192); +``` + +回到控制台运行如下命令: + +```bash +for i in tiny/rankings/*.deflate; do echo $i; zlib-flate -uncompress < $i | clickhouse-client --host=example-perftest01j --query="INSERT INTO rankings_tiny FORMAT CSV"; done +for i in tiny/uservisits/*.deflate; do echo $i; zlib-flate -uncompress < $i | clickhouse-client --host=example-perftest01j --query="INSERT INTO uservisits_tiny FORMAT CSV"; done +for i in 1node/rankings/*.deflate; do echo $i; zlib-flate -uncompress < $i | clickhouse-client --host=example-perftest01j --query="INSERT INTO rankings_1node FORMAT CSV"; done +for i in 1node/uservisits/*.deflate; do echo $i; zlib-flate -uncompress < $i | clickhouse-client --host=example-perftest01j --query="INSERT INTO uservisits_1node FORMAT CSV"; done +for i in 5nodes/rankings/*.deflate; do echo $i; zlib-flate -uncompress < $i | clickhouse-client --host=example-perftest01j --query="INSERT INTO rankings_5nodes_on_single FORMAT CSV"; done +for i in 5nodes/uservisits/*.deflate; do echo $i; zlib-flate -uncompress < $i | clickhouse-client --host=example-perftest01j --query="INSERT INTO uservisits_5nodes_on_single FORMAT CSV"; done +``` + +简单的查询示例: + +``` sql +SELECT pageURL, pageRank FROM rankings_1node WHERE pageRank > 1000 + +SELECT substring(sourceIP, 1, 8), sum(adRevenue) FROM uservisits_1node GROUP BY substring(sourceIP, 1, 8) + +SELECT + sourceIP, + sum(adRevenue) AS totalRevenue, + avg(pageRank) AS pageRank +FROM rankings_1node ALL INNER JOIN +( + SELECT + sourceIP, + destinationURL AS pageURL, + adRevenue + FROM uservisits_1node + WHERE (visitDate > '1980-01-01') AND (visitDate < '1980-04-01') +) USING pageURL +GROUP BY sourceIP +ORDER BY totalRevenue DESC +LIMIT 1 +``` + + +[Original article](https://clickhouse.yandex/docs/en/getting_started/example_datasets/amplab_benchmark/) diff --git a/docs/zh/getting_started/example_datasets/criteo.md b/docs/zh/getting_started/example_datasets/criteo.md deleted file mode 120000 index 507dc68cd62..00000000000 --- a/docs/zh/getting_started/example_datasets/criteo.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/getting_started/example_datasets/criteo.md \ No newline at end of file diff --git a/docs/zh/getting_started/example_datasets/criteo.md b/docs/zh/getting_started/example_datasets/criteo.md new file mode 100644 index 00000000000..9914bb8720c --- /dev/null +++ b/docs/zh/getting_started/example_datasets/criteo.md @@ -0,0 +1,75 @@ +# Criteo TB级别点击日志 + +可以从上下载数据 + +创建原始数据对应的表结构: + +``` sql +CREATE TABLE criteo_log (date Date, clicked UInt8, int1 Int32, int2 Int32, int3 Int32, int4 Int32, int5 Int32, int6 Int32, int7 Int32, int8 Int32, int9 Int32, int10 Int32, int11 Int32, int12 Int32, int13 Int32, cat1 String, cat2 String, cat3 String, cat4 String, cat5 String, cat6 String, cat7 String, cat8 String, cat9 String, cat10 String, cat11 String, cat12 String, cat13 String, cat14 String, cat15 String, cat16 String, cat17 String, cat18 String, cat19 String, cat20 String, cat21 String, cat22 String, cat23 String, cat24 String, cat25 String, cat26 String) ENGINE = Log +``` + +下载数据: + +```bash +for i in {00..23}; do echo $i; zcat datasets/criteo/day_${i#0}.gz | sed -r 's/^/2000-01-'${i/00/24}'\t/' | clickhouse-client --host=example-perftest01j --query="INSERT INTO criteo_log FORMAT TabSeparated"; done +``` + +创建转换后的数据对应的表结构: + +``` sql +CREATE TABLE criteo +( + date Date, + clicked UInt8, + int1 Int32, + int2 Int32, + int3 Int32, + int4 Int32, + int5 Int32, + int6 Int32, + int7 Int32, + int8 Int32, + int9 Int32, + int10 Int32, + int11 Int32, + int12 Int32, + int13 Int32, + icat1 UInt32, + icat2 UInt32, + icat3 UInt32, + icat4 UInt32, + icat5 UInt32, + icat6 UInt32, + icat7 UInt32, + icat8 UInt32, + icat9 UInt32, + icat10 UInt32, + icat11 UInt32, + icat12 UInt32, + icat13 UInt32, + icat14 UInt32, + icat15 UInt32, + icat16 UInt32, + icat17 UInt32, + icat18 UInt32, + icat19 UInt32, + icat20 UInt32, + icat21 UInt32, + icat22 UInt32, + icat23 UInt32, + icat24 UInt32, + icat25 UInt32, + icat26 UInt32 +) ENGINE = MergeTree(date, intHash32(icat1), (date, intHash32(icat1)), 8192) +``` + +将第一张表中的原始数据转化写入到第二张表中去: + +``` sql +INSERT INTO criteo SELECT date, clicked, int1, int2, int3, int4, int5, int6, int7, int8, int9, int10, int11, int12, int13, reinterpretAsUInt32(unhex(cat1)) AS icat1, reinterpretAsUInt32(unhex(cat2)) AS icat2, reinterpretAsUInt32(unhex(cat3)) AS icat3, reinterpretAsUInt32(unhex(cat4)) AS icat4, reinterpretAsUInt32(unhex(cat5)) AS icat5, reinterpretAsUInt32(unhex(cat6)) AS icat6, reinterpretAsUInt32(unhex(cat7)) AS icat7, reinterpretAsUInt32(unhex(cat8)) AS icat8, reinterpretAsUInt32(unhex(cat9)) AS icat9, reinterpretAsUInt32(unhex(cat10)) AS icat10, reinterpretAsUInt32(unhex(cat11)) AS icat11, reinterpretAsUInt32(unhex(cat12)) AS icat12, reinterpretAsUInt32(unhex(cat13)) AS icat13, reinterpretAsUInt32(unhex(cat14)) AS icat14, reinterpretAsUInt32(unhex(cat15)) AS icat15, reinterpretAsUInt32(unhex(cat16)) AS icat16, reinterpretAsUInt32(unhex(cat17)) AS icat17, reinterpretAsUInt32(unhex(cat18)) AS icat18, reinterpretAsUInt32(unhex(cat19)) AS icat19, reinterpretAsUInt32(unhex(cat20)) AS icat20, reinterpretAsUInt32(unhex(cat21)) AS icat21, reinterpretAsUInt32(unhex(cat22)) AS icat22, reinterpretAsUInt32(unhex(cat23)) AS icat23, reinterpretAsUInt32(unhex(cat24)) AS icat24, reinterpretAsUInt32(unhex(cat25)) AS icat25, reinterpretAsUInt32(unhex(cat26)) AS icat26 FROM criteo_log; + +DROP TABLE criteo_log; +``` + + +[Original article](https://clickhouse.yandex/docs/en/getting_started/example_datasets/criteo/) diff --git a/docs/zh/getting_started/example_datasets/nyc_taxi.md b/docs/zh/getting_started/example_datasets/nyc_taxi.md deleted file mode 120000 index c47fc83a293..00000000000 --- a/docs/zh/getting_started/example_datasets/nyc_taxi.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/getting_started/example_datasets/nyc_taxi.md \ No newline at end of file diff --git a/docs/zh/getting_started/example_datasets/nyc_taxi.md b/docs/zh/getting_started/example_datasets/nyc_taxi.md new file mode 100644 index 00000000000..a5b2bd82dea --- /dev/null +++ b/docs/zh/getting_started/example_datasets/nyc_taxi.md @@ -0,0 +1,368 @@ +# 纽约市出租车数据 + +## 怎样导入原始数据 + +可以参考中的关于数据集结构描述与数据下载指令说明。 + +数据集包含227GB的CSV文件。这大约需要一个小时的下载时间(1Gbit带宽下,并行下载大概是一半时间)。 +下载时注意损坏的文件。可以检查文件大小并重新下载损坏的文件。 + +有些文件中包含一些无效的行,您可以使用如下语句修复他们: + +```bash +sed -E '/(.*,){18,}/d' data/yellow_tripdata_2010-02.csv > data/yellow_tripdata_2010-02.csv_ +sed -E '/(.*,){18,}/d' data/yellow_tripdata_2010-03.csv > data/yellow_tripdata_2010-03.csv_ +mv data/yellow_tripdata_2010-02.csv_ data/yellow_tripdata_2010-02.csv +mv data/yellow_tripdata_2010-03.csv_ data/yellow_tripdata_2010-03.csv +``` + +然后您必须在PostgreSQL中预处理这些数据。这将创建多边形中的点(以匹配在地图中纽约市中范围),然后通过使用JOIN查询将数据关联组合到一个规范的表中。为了完成这部分操作,您需要安装PostgreSQL的同时安装PostGIS插件。 + +运行`initialize_database.sh`时要小心,并手动重新检查是否正确创建了所有表。 + +在PostgreSQL中处理每个月的数据大约需要20-30分钟,总共大约需要48小时。 + +您可以按如下方式检查下载的行数: + +``` +time psql nyc-taxi-data -c "SELECT count(*) FROM trips;" +## Count + 1298979494 +(1 row) + +real 7m9.164s +``` + +(根据Mark Litwintschik的系列博客报道数据略多余11亿行) + +PostgreSQL处理这些数据大概需要370GB的磁盘空间。 + +从PostgreSQL中导出数据: + +``` sql +COPY +( + SELECT trips.id, + trips.vendor_id, + trips.pickup_datetime, + trips.dropoff_datetime, + trips.store_and_fwd_flag, + trips.rate_code_id, + trips.pickup_longitude, + trips.pickup_latitude, + trips.dropoff_longitude, + trips.dropoff_latitude, + trips.passenger_count, + trips.trip_distance, + trips.fare_amount, + trips.extra, + trips.mta_tax, + trips.tip_amount, + trips.tolls_amount, + trips.ehail_fee, + trips.improvement_surcharge, + trips.total_amount, + trips.payment_type, + trips.trip_type, + trips.pickup, + trips.dropoff, + + cab_types.type cab_type, + + weather.precipitation_tenths_of_mm rain, + weather.snow_depth_mm, + weather.snowfall_mm, + weather.max_temperature_tenths_degrees_celsius max_temp, + weather.min_temperature_tenths_degrees_celsius min_temp, + weather.average_wind_speed_tenths_of_meters_per_second wind, + + pick_up.gid pickup_nyct2010_gid, + pick_up.ctlabel pickup_ctlabel, + pick_up.borocode pickup_borocode, + pick_up.boroname pickup_boroname, + pick_up.ct2010 pickup_ct2010, + pick_up.boroct2010 pickup_boroct2010, + pick_up.cdeligibil pickup_cdeligibil, + pick_up.ntacode pickup_ntacode, + pick_up.ntaname pickup_ntaname, + pick_up.puma pickup_puma, + + drop_off.gid dropoff_nyct2010_gid, + drop_off.ctlabel dropoff_ctlabel, + drop_off.borocode dropoff_borocode, + drop_off.boroname dropoff_boroname, + drop_off.ct2010 dropoff_ct2010, + drop_off.boroct2010 dropoff_boroct2010, + drop_off.cdeligibil dropoff_cdeligibil, + drop_off.ntacode dropoff_ntacode, + drop_off.ntaname dropoff_ntaname, + drop_off.puma dropoff_puma + FROM trips + LEFT JOIN cab_types + ON trips.cab_type_id = cab_types.id + LEFT JOIN central_park_weather_observations_raw weather + ON weather.date = trips.pickup_datetime::date + LEFT JOIN nyct2010 pick_up + ON pick_up.gid = trips.pickup_nyct2010_gid + LEFT JOIN nyct2010 drop_off + ON drop_off.gid = trips.dropoff_nyct2010_gid +) TO '/opt/milovidov/nyc-taxi-data/trips.tsv'; +``` + +数据快照的创建速度约为每秒50 MB。 在创建快照时,PostgreSQL以每秒约28 MB的速度从磁盘读取数据。 +这大约需要5个小时。 最终生成的TSV文件为590612904969 bytes。 + +在ClickHouse中创建临时表: + +``` sql +CREATE TABLE trips +( +trip_id UInt32, +vendor_id String, +pickup_datetime DateTime, +dropoff_datetime Nullable(DateTime), +store_and_fwd_flag Nullable(FixedString(1)), +rate_code_id Nullable(UInt8), +pickup_longitude Nullable(Float64), +pickup_latitude Nullable(Float64), +dropoff_longitude Nullable(Float64), +dropoff_latitude Nullable(Float64), +passenger_count Nullable(UInt8), +trip_distance Nullable(Float64), +fare_amount Nullable(Float32), +extra Nullable(Float32), +mta_tax Nullable(Float32), +tip_amount Nullable(Float32), +tolls_amount Nullable(Float32), +ehail_fee Nullable(Float32), +improvement_surcharge Nullable(Float32), +total_amount Nullable(Float32), +payment_type Nullable(String), +trip_type Nullable(UInt8), +pickup Nullable(String), +dropoff Nullable(String), +cab_type Nullable(String), +precipitation Nullable(UInt8), +snow_depth Nullable(UInt8), +snowfall Nullable(UInt8), +max_temperature Nullable(UInt8), +min_temperature Nullable(UInt8), +average_wind_speed Nullable(UInt8), +pickup_nyct2010_gid Nullable(UInt8), +pickup_ctlabel Nullable(String), +pickup_borocode Nullable(UInt8), +pickup_boroname Nullable(String), +pickup_ct2010 Nullable(String), +pickup_boroct2010 Nullable(String), +pickup_cdeligibil Nullable(FixedString(1)), +pickup_ntacode Nullable(String), +pickup_ntaname Nullable(String), +pickup_puma Nullable(String), +dropoff_nyct2010_gid Nullable(UInt8), +dropoff_ctlabel Nullable(String), +dropoff_borocode Nullable(UInt8), +dropoff_boroname Nullable(String), +dropoff_ct2010 Nullable(String), +dropoff_boroct2010 Nullable(String), +dropoff_cdeligibil Nullable(String), +dropoff_ntacode Nullable(String), +dropoff_ntaname Nullable(String), +dropoff_puma Nullable(String) +) ENGINE = Log; +``` + +接下来,需要将字段转换为更正确的数据类型,并且在可能的情况下,消除NULL。 + +``` +time clickhouse-client --query="INSERT INTO trips FORMAT TabSeparated" < trips.tsv + +real 75m56.214s +``` + +数据的读取速度为112-140 Mb/秒。 +通过这种方式将数据加载到Log表中需要76分钟。 +这个表中的数据需要使用142 GB的磁盘空间. + +(也可以直接使用`COPY ... TO PROGRAM`从Postgres中导入数据) + +由于数据中与天气相关的所有数据(precipitation......average_wind_speed)都填充了NULL。 所以,我们将从最终数据集中删除它们 + +首先,我们使用单台服务器创建表,后面我们将在多台节点上创建这些表。 + +创建表结构并写入数据: + +``` +CREATE TABLE trips_mergetree +ENGINE = MergeTree(pickup_date, pickup_datetime, 8192) +AS SELECT + +trip_id, +CAST(vendor_id AS Enum8('1' = 1, '2' = 2, 'CMT' = 3, 'VTS' = 4, 'DDS' = 5, 'B02512' = 10, 'B02598' = 11, 'B02617' = 12, 'B02682' = 13, 'B02764' = 14)) AS vendor_id, +toDate(pickup_datetime) AS pickup_date, +ifNull(pickup_datetime, toDateTime(0)) AS pickup_datetime, +toDate(dropoff_datetime) AS dropoff_date, +ifNull(dropoff_datetime, toDateTime(0)) AS dropoff_datetime, +assumeNotNull(store_and_fwd_flag) IN ('Y', '1', '2') AS store_and_fwd_flag, +assumeNotNull(rate_code_id) AS rate_code_id, +assumeNotNull(pickup_longitude) AS pickup_longitude, +assumeNotNull(pickup_latitude) AS pickup_latitude, +assumeNotNull(dropoff_longitude) AS dropoff_longitude, +assumeNotNull(dropoff_latitude) AS dropoff_latitude, +assumeNotNull(passenger_count) AS passenger_count, +assumeNotNull(trip_distance) AS trip_distance, +assumeNotNull(fare_amount) AS fare_amount, +assumeNotNull(extra) AS extra, +assumeNotNull(mta_tax) AS mta_tax, +assumeNotNull(tip_amount) AS tip_amount, +assumeNotNull(tolls_amount) AS tolls_amount, +assumeNotNull(ehail_fee) AS ehail_fee, +assumeNotNull(improvement_surcharge) AS improvement_surcharge, +assumeNotNull(total_amount) AS total_amount, +CAST((assumeNotNull(payment_type) AS pt) IN ('CSH', 'CASH', 'Cash', 'CAS', 'Cas', '1') ? 'CSH' : (pt IN ('CRD', 'Credit', 'Cre', 'CRE', 'CREDIT', '2') ? 'CRE' : (pt IN ('NOC', 'No Charge', 'No', '3') ? 'NOC' : (pt IN ('DIS', 'Dispute', 'Dis', '4') ? 'DIS' : 'UNK'))) AS Enum8('CSH' = 1, 'CRE' = 2, 'UNK' = 0, 'NOC' = 3, 'DIS' = 4)) AS payment_type_, +assumeNotNull(trip_type) AS trip_type, +ifNull(toFixedString(unhex(pickup), 25), toFixedString('', 25)) AS pickup, +ifNull(toFixedString(unhex(dropoff), 25), toFixedString('', 25)) AS dropoff, +CAST(assumeNotNull(cab_type) AS Enum8('yellow' = 1, 'green' = 2, 'uber' = 3)) AS cab_type, + +assumeNotNull(pickup_nyct2010_gid) AS pickup_nyct2010_gid, +toFloat32(ifNull(pickup_ctlabel, '0')) AS pickup_ctlabel, +assumeNotNull(pickup_borocode) AS pickup_borocode, +CAST(assumeNotNull(pickup_boroname) AS Enum8('Manhattan' = 1, 'Queens' = 4, 'Brooklyn' = 3, '' = 0, 'Bronx' = 2, 'Staten Island' = 5)) AS pickup_boroname, +toFixedString(ifNull(pickup_ct2010, '000000'), 6) AS pickup_ct2010, +toFixedString(ifNull(pickup_boroct2010, '0000000'), 7) AS pickup_boroct2010, +CAST(assumeNotNull(ifNull(pickup_cdeligibil, ' ')) AS Enum8(' ' = 0, 'E' = 1, 'I' = 2)) AS pickup_cdeligibil, +toFixedString(ifNull(pickup_ntacode, '0000'), 4) AS pickup_ntacode, + +CAST(assumeNotNull(pickup_ntaname) AS Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195)) AS pickup_ntaname, + +toUInt16(ifNull(pickup_puma, '0')) AS pickup_puma, + +assumeNotNull(dropoff_nyct2010_gid) AS dropoff_nyct2010_gid, +toFloat32(ifNull(dropoff_ctlabel, '0')) AS dropoff_ctlabel, +assumeNotNull(dropoff_borocode) AS dropoff_borocode, +CAST(assumeNotNull(dropoff_boroname) AS Enum8('Manhattan' = 1, 'Queens' = 4, 'Brooklyn' = 3, '' = 0, 'Bronx' = 2, 'Staten Island' = 5)) AS dropoff_boroname, +toFixedString(ifNull(dropoff_ct2010, '000000'), 6) AS dropoff_ct2010, +toFixedString(ifNull(dropoff_boroct2010, '0000000'), 7) AS dropoff_boroct2010, +CAST(assumeNotNull(ifNull(dropoff_cdeligibil, ' ')) AS Enum8(' ' = 0, 'E' = 1, 'I' = 2)) AS dropoff_cdeligibil, +toFixedString(ifNull(dropoff_ntacode, '0000'), 4) AS dropoff_ntacode, + +CAST(assumeNotNull(dropoff_ntaname) AS Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195)) AS dropoff_ntaname, + +toUInt16(ifNull(dropoff_puma, '0')) AS dropoff_puma + +FROM trips +``` + +这需要3030秒,速度约为每秒428,000行。 +要加快速度,可以使用`Log`引擎替换'MergeTree`引擎来创建表。 在这种情况下,下载速度超过200秒。 + +这个表需要使用126GB的磁盘空间。 + +``` +:) SELECT formatReadableSize(sum(bytes)) FROM system.parts WHERE table = 'trips_mergetree' AND active + +SELECT formatReadableSize(sum(bytes)) +FROM system.parts +WHERE (table = 'trips_mergetree') AND active + +┌─formatReadableSize(sum(bytes))─┐ +│ 126.18 GiB │ +└────────────────────────────────┘ +``` + +除此之外,你还可以在MergeTree上运行OPTIMIZE查询来进行优化。但这不是必须的,因为即使在没有进行优化的情况下它的表现依然是很好的。 + +## 单台服务器运行结果 + +Q1: + +``` sql +SELECT cab_type, count(*) FROM trips_mergetree GROUP BY cab_type +``` + +0.490 seconds. + +Q2: + +``` sql +SELECT passenger_count, avg(total_amount) FROM trips_mergetree GROUP BY passenger_count +``` + +1.224 seconds. + +Q3: + +``` sql +SELECT passenger_count, toYear(pickup_date) AS year, count(*) FROM trips_mergetree GROUP BY passenger_count, year +``` + +2.104 seconds. + +Q4: + +``` sql +SELECT passenger_count, toYear(pickup_date) AS year, round(trip_distance) AS distance, count(*) +FROM trips_mergetree +GROUP BY passenger_count, year, distance +ORDER BY year, count(*) DESC +``` + +3.593 seconds. + +我们使用的是如下配置的服务器: + +Two Intel(R) Xeon(R) CPU E5-2650 v2 @ 2.60GHz, 16 physical kernels total,128 GiB RAM,8x6 TB HD on hardware RAID-5 + +执行时间是取三次运行中最好的值,但是从第二次查询开始,查询就讲从文件系统的缓存中读取数据。同时在每次读取和处理后不在进行缓存。 + +在三台服务器中创建表结构: + +在每台服务器中运行: + +``` +CREATE TABLE default.trips_mergetree_third ( trip_id UInt32, vendor_id Enum8('1' = 1, '2' = 2, 'CMT' = 3, 'VTS' = 4, 'DDS' = 5, 'B02512' = 10, 'B02598' = 11, 'B02617' = 12, 'B02682' = 13, 'B02764' = 14), pickup_date Date, pickup_datetime DateTime, dropoff_date Date, dropoff_datetime DateTime, store_and_fwd_flag UInt8, rate_code_id UInt8, pickup_longitude Float64, pickup_latitude Float64, dropoff_longitude Float64, dropoff_latitude Float64, passenger_count UInt8, trip_distance Float64, fare_amount Float32, extra Float32, mta_tax Float32, tip_amount Float32, tolls_amount Float32, ehail_fee Float32, improvement_surcharge Float32, total_amount Float32, payment_type_ Enum8('UNK' = 0, 'CSH' = 1, 'CRE' = 2, 'NOC' = 3, 'DIS' = 4), trip_type UInt8, pickup FixedString(25), dropoff FixedString(25), cab_type Enum8('yellow' = 1, 'green' = 2, 'uber' = 3), pickup_nyct2010_gid UInt8, pickup_ctlabel Float32, pickup_borocode UInt8, pickup_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), pickup_ct2010 FixedString(6), pickup_boroct2010 FixedString(7), pickup_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), pickup_ntacode FixedString(4), pickup_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), pickup_puma UInt16, dropoff_nyct2010_gid UInt8, dropoff_ctlabel Float32, dropoff_borocode UInt8, dropoff_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), dropoff_ct2010 FixedString(6), dropoff_boroct2010 FixedString(7), dropoff_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), dropoff_ntacode FixedString(4), dropoff_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), dropoff_puma UInt16) ENGINE = MergeTree(pickup_date, pickup_datetime, 8192) +``` + +在之前的服务器中运行: + +``` sql +CREATE TABLE trips_mergetree_x3 AS trips_mergetree_third ENGINE = Distributed(perftest, default, trips_mergetree_third, rand()) +``` + +运行如下查询重新分布数据: + +``` sql +INSERT INTO trips_mergetree_x3 SELECT * FROM trips_mergetree +``` + +这个查询需要运行2454秒。 + +在三台服务器集群中运行的结果: + +Q1: 0.212 seconds. +Q2: 0.438 seconds. +Q3: 0.733 seconds. +Q4: 1.241 seconds. + +不出意料,查询是线性扩展的。 + +我们同时在140台服务器的集群中运行的结果: + +Q1: 0.028 sec. +Q2: 0.043 sec. +Q3: 0.051 sec. +Q4: 0.072 sec. + +在这种情况下,查询处理时间首先由网络延迟确定。 +我们使用位于芬兰的Yandex数据中心中的客户端去位于俄罗斯的集群上运行查询,这增加了大约20毫秒的延迟。 + +## 总结 + +| servers | Q1 | Q2 | Q3 | Q4 | +| ------- | ----- | ----- | ----- | ----- | +| 1 | 0.490 | 1.224 | 2.104 | 3.593 | +| 3 | 0.212 | 0.438 | 0.733 | 1.241 | +| 140 | 0.028 | 0.043 | 0.051 | 0.072 | + + +[Original article](https://clickhouse.yandex/docs/en/getting_started/example_datasets/nyc_taxi/) diff --git a/docs/zh/getting_started/example_datasets/ontime.md b/docs/zh/getting_started/example_datasets/ontime.md deleted file mode 120000 index 87cfbb8be91..00000000000 --- a/docs/zh/getting_started/example_datasets/ontime.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/getting_started/example_datasets/ontime.md \ No newline at end of file diff --git a/docs/zh/getting_started/example_datasets/ontime.md b/docs/zh/getting_started/example_datasets/ontime.md new file mode 100644 index 00000000000..37f673e2559 --- /dev/null +++ b/docs/zh/getting_started/example_datasets/ontime.md @@ -0,0 +1,318 @@ + + +# 航班飞行数据 + +下载数据: + +```bash +for s in `seq 1987 2017` +do +for m in `seq 1 12` +do +wget http://transtats.bts.gov/PREZIP/On_Time_On_Time_Performance_${s}_${m}.zip +done +done +``` + +(引用 ) + +创建表结构: + +```sql +CREATE TABLE `ontime` ( + `Year` UInt16, + `Quarter` UInt8, + `Month` UInt8, + `DayofMonth` UInt8, + `DayOfWeek` UInt8, + `FlightDate` Date, + `UniqueCarrier` FixedString(7), + `AirlineID` Int32, + `Carrier` FixedString(2), + `TailNum` String, + `FlightNum` String, + `OriginAirportID` Int32, + `OriginAirportSeqID` Int32, + `OriginCityMarketID` Int32, + `Origin` FixedString(5), + `OriginCityName` String, + `OriginState` FixedString(2), + `OriginStateFips` String, + `OriginStateName` String, + `OriginWac` Int32, + `DestAirportID` Int32, + `DestAirportSeqID` Int32, + `DestCityMarketID` Int32, + `Dest` FixedString(5), + `DestCityName` String, + `DestState` FixedString(2), + `DestStateFips` String, + `DestStateName` String, + `DestWac` Int32, + `CRSDepTime` Int32, + `DepTime` Int32, + `DepDelay` Int32, + `DepDelayMinutes` Int32, + `DepDel15` Int32, + `DepartureDelayGroups` String, + `DepTimeBlk` String, + `TaxiOut` Int32, + `WheelsOff` Int32, + `WheelsOn` Int32, + `TaxiIn` Int32, + `CRSArrTime` Int32, + `ArrTime` Int32, + `ArrDelay` Int32, + `ArrDelayMinutes` Int32, + `ArrDel15` Int32, + `ArrivalDelayGroups` Int32, + `ArrTimeBlk` String, + `Cancelled` UInt8, + `CancellationCode` FixedString(1), + `Diverted` UInt8, + `CRSElapsedTime` Int32, + `ActualElapsedTime` Int32, + `AirTime` Int32, + `Flights` Int32, + `Distance` Int32, + `DistanceGroup` UInt8, + `CarrierDelay` Int32, + `WeatherDelay` Int32, + `NASDelay` Int32, + `SecurityDelay` Int32, + `LateAircraftDelay` Int32, + `FirstDepTime` String, + `TotalAddGTime` String, + `LongestAddGTime` String, + `DivAirportLandings` String, + `DivReachedDest` String, + `DivActualElapsedTime` String, + `DivArrDelay` String, + `DivDistance` String, + `Div1Airport` String, + `Div1AirportID` Int32, + `Div1AirportSeqID` Int32, + `Div1WheelsOn` String, + `Div1TotalGTime` String, + `Div1LongestGTime` String, + `Div1WheelsOff` String, + `Div1TailNum` String, + `Div2Airport` String, + `Div2AirportID` Int32, + `Div2AirportSeqID` Int32, + `Div2WheelsOn` String, + `Div2TotalGTime` String, + `Div2LongestGTime` String, + `Div2WheelsOff` String, + `Div2TailNum` String, + `Div3Airport` String, + `Div3AirportID` Int32, + `Div3AirportSeqID` Int32, + `Div3WheelsOn` String, + `Div3TotalGTime` String, + `Div3LongestGTime` String, + `Div3WheelsOff` String, + `Div3TailNum` String, + `Div4Airport` String, + `Div4AirportID` Int32, + `Div4AirportSeqID` Int32, + `Div4WheelsOn` String, + `Div4TotalGTime` String, + `Div4LongestGTime` String, + `Div4WheelsOff` String, + `Div4TailNum` String, + `Div5Airport` String, + `Div5AirportID` Int32, + `Div5AirportSeqID` Int32, + `Div5WheelsOn` String, + `Div5TotalGTime` String, + `Div5LongestGTime` String, + `Div5WheelsOff` String, + `Div5TailNum` String +) ENGINE = MergeTree(FlightDate, (Year, FlightDate), 8192) +``` + +加载数据: + +```bash +for i in *.zip; do echo $i; unzip -cq $i '*.csv' | sed 's/\.00//g' | clickhouse-client --host=example-perftest01j --query="INSERT INTO ontime FORMAT CSVWithNames"; done +``` + +查询: + +Q0. + +```sql +select avg(c1) from (select Year, Month, count(*) as c1 from ontime group by Year, Month); +``` + +Q1. 查询从2000年到2008年每天的航班数 + +```sql +SELECT DayOfWeek, count(*) AS c FROM ontime WHERE Year >= 2000 AND Year <= 2008 GROUP BY DayOfWeek ORDER BY c DESC; +``` + +Q2. 查询从2000年到2008年每周延误超过10分钟的航班数。 + +```sql +SELECT DayOfWeek, count(*) AS c FROM ontime WHERE DepDelay>10 AND Year >= 2000 AND Year <= 2008 GROUP BY DayOfWeek ORDER BY c DESC +``` + +Q3. 查询2000年到2008年每个机场延误超过10分钟以上的次数 + +```sql +SELECT Origin, count(*) AS c FROM ontime WHERE DepDelay>10 AND Year >= 2000 AND Year <= 2008 GROUP BY Origin ORDER BY c DESC LIMIT 10 +``` + +Q4. 查询2007年各航空公司延误超过10分钟以上的次数 + +```sql +SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC +``` + +Q5. 查询2007年各航空公司延误超过10分钟以上的百分比 + +```sql +SELECT Carrier, c, c2, c*1000/c2 as c3 +FROM +( + SELECT + Carrier, + count(*) AS c + FROM ontime + WHERE DepDelay>10 + AND Year=2007 + GROUP BY Carrier +) +ANY INNER JOIN +( + SELECT + Carrier, + count(*) AS c2 + FROM ontime + WHERE Year=2007 + GROUP BY Carrier +) USING Carrier +ORDER BY c3 DESC; +``` + +更好的查询版本: + +```sql +SELECT Carrier, avg(DepDelay > 10) * 1000 AS c3 FROM ontime WHERE Year = 2007 GROUP BY Carrier ORDER BY Carrier +``` + +Q6. 同上一个查询一致,只是查询范围扩大到2000年到2008年 + +```sql +SELECT Carrier, c, c2, c*1000/c2 as c3 +FROM +( + SELECT + Carrier, + count(*) AS c + FROM ontime + WHERE DepDelay>10 + AND Year >= 2000 AND Year <= 2008 + GROUP BY Carrier +) +ANY INNER JOIN +( + SELECT + Carrier, + count(*) AS c2 + FROM ontime + WHERE Year >= 2000 AND Year <= 2008 + GROUP BY Carrier +) USING Carrier +ORDER BY c3 DESC; +``` + +更好的查询版本: + +```sql +SELECT Carrier, avg(DepDelay > 10) * 1000 AS c3 FROM ontime WHERE Year >= 2000 AND Year <= 2008 GROUP BY Carrier ORDER BY Carrier +``` + +Q7. 每年航班延误超过10分钟的百分比 + +```sql +SELECT Year, c1/c2 +FROM +( + select + Year, + count(*)*1000 as c1 + from ontime + WHERE DepDelay>10 + GROUP BY Year +) +ANY INNER JOIN +( + select + Year, + count(*) as c2 + from ontime + GROUP BY Year +) USING (Year) +ORDER BY Year +``` + +更好的查询版本: + +```sql +SELECT Year, avg(DepDelay > 10) FROM ontime GROUP BY Year ORDER BY Year +``` + +Q8. 每年更受人们喜爱的目的地 + +```sql +SELECT DestCityName, uniqExact(OriginCityName) AS u FROM ontime WHERE Year >= 2000 and Year <= 2010 GROUP BY DestCityName ORDER BY u DESC LIMIT 10; +``` + +Q9. + +```sql +select Year, count(*) as c1 from ontime group by Year; +``` + +Q10. + +```sql +select + min(Year), max(Year), Carrier, count(*) as cnt, + sum(ArrDelayMinutes>30) as flights_delayed, + round(sum(ArrDelayMinutes>30)/count(*),2) as rate +FROM ontime +WHERE + DayOfWeek not in (6,7) and OriginState not in ('AK', 'HI', 'PR', 'VI') + and DestState not in ('AK', 'HI', 'PR', 'VI') + and FlightDate < '2010-01-01' +GROUP by Carrier +HAVING cnt > 100000 and max(Year) > 1990 +ORDER by rate DESC +LIMIT 1000; +``` + +Bonus: + +```sql +SELECT avg(cnt) FROM (SELECT Year,Month,count(*) AS cnt FROM ontime WHERE DepDel15=1 GROUP BY Year,Month) + +select avg(c1) from (select Year,Month,count(*) as c1 from ontime group by Year,Month) + +SELECT DestCityName, uniqExact(OriginCityName) AS u FROM ontime GROUP BY DestCityName ORDER BY u DESC LIMIT 10; + +SELECT OriginCityName, DestCityName, count() AS c FROM ontime GROUP BY OriginCityName, DestCityName ORDER BY c DESC LIMIT 10; + +SELECT OriginCityName, count() AS c FROM ontime GROUP BY OriginCityName ORDER BY c DESC LIMIT 10; +``` + +这个性能测试由Vadim Tkachenko提供。参考: + +- +- +- +- +- +- diff --git a/docs/zh/getting_started/example_datasets/star_schema.md b/docs/zh/getting_started/example_datasets/star_schema.md deleted file mode 120000 index 1c26392dd23..00000000000 --- a/docs/zh/getting_started/example_datasets/star_schema.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/getting_started/example_datasets/star_schema.md \ No newline at end of file diff --git a/docs/zh/getting_started/example_datasets/star_schema.md b/docs/zh/getting_started/example_datasets/star_schema.md new file mode 100644 index 00000000000..197a8fd7c30 --- /dev/null +++ b/docs/zh/getting_started/example_datasets/star_schema.md @@ -0,0 +1,87 @@ +# Star Schema 基准测试 + +编译 dbgen: + +```bash +git clone git@github.com:vadimtk/ssb-dbgen.git +cd ssb-dbgen +make +``` + +在编译过程中可能会有一些警告,这是正常的。 + +将`dbgen`和`dists.dss`放在一个可用容量大于800GB的磁盘中。 + +开始生成数据: + +```bash +./dbgen -s 1000 -T c +./dbgen -s 1000 -T l +``` + +在ClickHouse中创建表结构: + +``` sql +CREATE TABLE lineorder ( + LO_ORDERKEY UInt32, + LO_LINENUMBER UInt8, + LO_CUSTKEY UInt32, + LO_PARTKEY UInt32, + LO_SUPPKEY UInt32, + LO_ORDERDATE Date, + LO_ORDERPRIORITY String, + LO_SHIPPRIORITY UInt8, + LO_QUANTITY UInt8, + LO_EXTENDEDPRICE UInt32, + LO_ORDTOTALPRICE UInt32, + LO_DISCOUNT UInt8, + LO_REVENUE UInt32, + LO_SUPPLYCOST UInt32, + LO_TAX UInt8, + LO_COMMITDATE Date, + LO_SHIPMODE String +)Engine=MergeTree(LO_ORDERDATE,(LO_ORDERKEY,LO_LINENUMBER,LO_ORDERDATE),8192); + +CREATE TABLE customer ( + C_CUSTKEY UInt32, + C_NAME String, + C_ADDRESS String, + C_CITY String, + C_NATION String, + C_REGION String, + C_PHONE String, + C_MKTSEGMENT String, + C_FAKEDATE Date +)Engine=MergeTree(C_FAKEDATE,(C_CUSTKEY,C_FAKEDATE),8192); + +CREATE TABLE part ( + P_PARTKEY UInt32, + P_NAME String, + P_MFGR String, + P_CATEGORY String, + P_BRAND String, + P_COLOR String, + P_TYPE String, + P_SIZE UInt8, + P_CONTAINER String, + P_FAKEDATE Date +)Engine=MergeTree(P_FAKEDATE,(P_PARTKEY,P_FAKEDATE),8192); + +CREATE TABLE lineorderd AS lineorder ENGINE = Distributed(perftest_3shards_1replicas, default, lineorder, rand()); +CREATE TABLE customerd AS customer ENGINE = Distributed(perftest_3shards_1replicas, default, customer, rand()); +CREATE TABLE partd AS part ENGINE = Distributed(perftest_3shards_1replicas, default, part, rand()); +``` + +如果是在单节点中进行的测试,那么只需要创建对应的MergeTree表。 +如果是在多节点中进行的测试,您需要在配置文件中配置`perftest_3shards_1replicas`集群的信息。 +然后在每个节点中同时创建MergeTree表和Distributed表。 + +下载数据(如果您是分布式测试的话将'customer'更改为'customerd'): + +```bash +cat customer.tbl | sed 's/$/2000-01-01/' | clickhouse-client --query "INSERT INTO customer FORMAT CSV" +cat lineorder.tbl | clickhouse-client --query "INSERT INTO lineorder FORMAT CSV" +``` + + +[Original article](https://clickhouse.yandex/docs/en/getting_started/example_datasets/star_schema/) diff --git a/docs/zh/getting_started/example_datasets/wikistat.md b/docs/zh/getting_started/example_datasets/wikistat.md deleted file mode 120000 index bf6e780fb27..00000000000 --- a/docs/zh/getting_started/example_datasets/wikistat.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/getting_started/example_datasets/wikistat.md \ No newline at end of file diff --git a/docs/zh/getting_started/example_datasets/wikistat.md b/docs/zh/getting_started/example_datasets/wikistat.md new file mode 100644 index 00000000000..c306c644551 --- /dev/null +++ b/docs/zh/getting_started/example_datasets/wikistat.md @@ -0,0 +1,29 @@ +# 维基访问数据 + +参考: + +创建表结构: + +``` sql +CREATE TABLE wikistat +( + date Date, + time DateTime, + project String, + subproject String, + path String, + hits UInt64, + size UInt64 +) ENGINE = MergeTree(date, (path, time), 8192); +``` + +加载数据: + +```bash +for i in {2007..2016}; do for j in {01..12}; do echo $i-$j >&2; curl -sSL "http://dumps.wikimedia.org/other/pagecounts-raw/$i/$i-$j/" | grep -oE 'pagecounts-[0-9]+-[0-9]+\.gz'; done; done | sort | uniq | tee links.txt +cat links.txt | while read link; do wget http://dumps.wikimedia.org/other/pagecounts-raw/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1/')/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1-\2/')/$link; done +ls -1 /opt/wikistat/ | grep gz | while read i; do echo $i; gzip -cd /opt/wikistat/$i | ./wikistat-loader --time="$(echo -n $i | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})([0-9]{2})-([0-9]{2})([0-9]{2})([0-9]{2})\.gz/\1-\2-\3 \4-00-00/')" | clickhouse-client --query="INSERT INTO wikistat FORMAT TabSeparated"; done +``` + + +[Original article](https://clickhouse.yandex/docs/en/getting_started/example_datasets/wikistat/) diff --git a/docs/zh/getting_started/index.md b/docs/zh/getting_started/index.md deleted file mode 120000 index 1acedb0f03e..00000000000 --- a/docs/zh/getting_started/index.md +++ /dev/null @@ -1 +0,0 @@ -../../en/getting_started/index.md \ No newline at end of file diff --git a/docs/zh/getting_started/index.md b/docs/zh/getting_started/index.md new file mode 100644 index 00000000000..2820719990a --- /dev/null +++ b/docs/zh/getting_started/index.md @@ -0,0 +1,141 @@ +# 入门指南 + +## 系统要求 + +如果从官方仓库安装,需要确保您使用的是x86_64处理器构架的Linux并且支持SSE 4.2指令集 + +检查是否支持SSE 4.2: + +```bash +grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not supported" +``` + +我们推荐使用Ubuntu或者Debian。终端必须使用UTF-8编码。 + +基于rpm的系统,你可以使用第三方的安装包:https://packagecloud.io/altinity/clickhouse 或者直接安装debian安装包。 + +ClickHouse还可以在FreeBSD与Mac OS X上工作。同时它可以在不支持SSE 4.2的x86_64构架和AArch64 CPUs上编译。 + +## 安装 + +为了测试和开发,系统可以安装在单个服务器或普通PC机上。 + +### 为Debian/Ubuntu安装 + +在`/etc/apt/sources.list` (或创建`/etc/apt/sources.list.d/clickhouse.list`文件)中添加仓库: + +```text +deb http://repo.yandex.ru/clickhouse/deb/stable/ main/ +``` + +如果你想使用最新的测试版本,请使用'testing'替换'stable'。 + +然后运行: + +```bash +sudo apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 # optional +sudo apt-get update +sudo apt-get install clickhouse-client clickhouse-server +``` + +你也可以从这里手动下载安装包:。 + +ClickHouse包含访问控制配置,它们位于`users.xml`文件中(与'config.xml'同目录)。 +默认情况下,允许从任何地方使用默认的‘default’用户无密码的访问ClickHouse。参考‘user/default/networks’。 +有关更多信息,请参考"Configuration files"部分。 + +### 使用源码安装 + +具体编译方式可以参考build.md。 + +你可以编译并安装它们。 +你也可以直接使用而不进行安装。 + +```text +Client: dbms/programs/clickhouse-client +Server: dbms/programs/clickhouse-server +``` + +在服务器中为数据创建如下目录: + +```text +/opt/clickhouse/data/default/ +/opt/clickhouse/metadata/default/ +``` + +(它们可以在server config中配置。) +为需要的用户运行‘chown’ + +日志的路径可以在server config (src/dbms/programs/server/config.xml)中配置。 + +### 其他的安装方法 + +Docker image: + +CentOS或RHEL安装包: + +Gentoo:`emerge clickhouse` + +## 启动 + +可以运行如下命令在后台启动服务: + +```bash +sudo service clickhouse-server start +``` + +可以在`/var/log/clickhouse-server/`目录中查看日志。 + +如果服务没有启动,请检查配置文件 `/etc/clickhouse-server/config.xml`。 + +你也可以在控制台中直接启动服务: + +```bash +clickhouse-server --config-file=/etc/clickhouse-server/config.xml +``` + +在这种情况下,日志将被打印到控制台中,这在开发过程中很方便。 +如果配置文件在当前目录中,你可以不指定‘--config-file’参数。它默认使用‘./config.xml’。 + +你可以使用命令行客户端连接到服务: + +```bash +clickhouse-client +``` + +默认情况下它使用‘default’用户无密码的与localhost:9000服务建立连接。 +客户端也可以用于连接远程服务,例如: + +```bash +clickhouse-client --host=example.com +``` + +有关更多信息,请参考"Command-line client"部分。 + +检查系统是否工作: + +```bash +milovidov@hostname:~/work/metrica/src/dbms/src/Client$ ./clickhouse-client +ClickHouse client version 0.0.18749. +Connecting to localhost:9000. +Connected to ClickHouse server version 0.0.18749. + +:) SELECT 1 + +SELECT 1 + +┌─1─┐ +│ 1 │ +└───┘ + +1 rows in set. Elapsed: 0.003 sec. + +:) +``` + +**恭喜,系统已经工作了!** + +为了继续进行实验,你可以尝试下载测试数据集。 + + +[Original article](https://clickhouse.yandex/docs/en/getting_started/) From 40f1805bf416fff9081cefd9a589c506b5a4fa62 Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 17 Oct 2018 13:59:05 +0300 Subject: [PATCH 05/28] fix gcc build --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 11 +++++++---- dbms/src/Interpreters/GlobalSubqueriesVisitor.h | 9 ++------- 2 files changed, 9 insertions(+), 11 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index b5f81047a10..d65d570547e 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -541,10 +541,13 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables() ExternalTablesVisitor tables_visitor(context, external_tables); tables_visitor.visit(query); - bool is_remote = storage && storage->isRemote(); - GlobalSubqueriesVisitor subqueries_visitor(context, subquery_depth, do_global, is_remote, - external_tables, subqueries_for_sets, has_global_subqueries); - subqueries_visitor.visit(query); + if (do_global) + { + bool is_remote = storage && storage->isRemote(); + GlobalSubqueriesVisitor subqueries_visitor(context, subquery_depth, is_remote, + external_tables, subqueries_for_sets, has_global_subqueries); + subqueries_visitor.visit(query); + } } diff --git a/dbms/src/Interpreters/GlobalSubqueriesVisitor.h b/dbms/src/Interpreters/GlobalSubqueriesVisitor.h index a2d83c70405..f645059a03c 100644 --- a/dbms/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/dbms/src/Interpreters/GlobalSubqueriesVisitor.h @@ -7,11 +7,10 @@ namespace DB class GlobalSubqueriesVisitor { public: - GlobalSubqueriesVisitor(const Context & context_, size_t subquery_depth_, bool do_global_, bool is_remote_, + GlobalSubqueriesVisitor(const Context & context_, size_t subquery_depth_, bool is_remote_, Tables & tables, SubqueriesForSets & subqueries_for_sets_, bool & has_global_subqueries_) : context(context_), subquery_depth(subquery_depth_), - do_global(do_global_), is_remote(is_remote_), external_table_id(1), external_tables(tables), @@ -21,9 +20,6 @@ public: void visit(ASTPtr & ast) const { - if (!do_global) - return; - /// Recursive calls. We do not go into subqueries. for (auto & child : ast->children) if (!typeid_cast(child.get())) @@ -32,13 +28,12 @@ public: /// Bottom-up actions. if (tryVisit(ast) || tryVisit(ast)) - ; + {} } private: const Context & context; size_t subquery_depth; - bool do_global; bool is_remote; mutable size_t external_table_id = 1; Tables & external_tables; From 5d3f03a6d2b2bf2e8035cf5c37a9e9ab40cb4f2c Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 17 Oct 2018 14:06:46 +0300 Subject: [PATCH 06/28] minor refactoring (function naming) --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 11 +++++------ dbms/src/Interpreters/ExpressionAnalyzer.h | 2 +- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index d65d570547e..013bb09e4be 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -269,9 +269,9 @@ ExpressionAnalyzer::ExpressionAnalyzer( analyzeAggregation(); } -bool ExpressionAnalyzer::noStorageOrLocal() const +bool ExpressionAnalyzer::isRemoteStorage() const { - return !storage || !storage->isRemote(); + return storage && storage->isRemote(); } @@ -543,8 +543,7 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables() if (do_global) { - bool is_remote = storage && storage->isRemote(); - GlobalSubqueriesVisitor subqueries_visitor(context, subquery_depth, is_remote, + GlobalSubqueriesVisitor subqueries_visitor(context, subquery_depth, isRemoteStorage(), external_tables, subqueries_for_sets, has_global_subqueries); subqueries_visitor.visit(query); } @@ -1051,7 +1050,7 @@ void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, LogAST log; ActionsVisitor actions_visitor(context, getSetSizeLimits(settings), is_conditional_tree, subquery_depth, source_columns, actions, prepared_sets, subqueries_for_sets, - no_subqueries, only_consts, noStorageOrLocal(), log.stream()); + no_subqueries, only_consts, !isRemoteStorage(), log.stream()); actions_visitor.visit(ast); actions = actions_visitor.popActionsLevel(); } @@ -1065,7 +1064,7 @@ void ExpressionAnalyzer::getActionsFromJoinKeys(const ASTTableJoin & table_join, LogAST log; ActionsVisitor actions_visitor(context, getSetSizeLimits(settings), is_conditional_tree, subquery_depth, source_columns, actions, prepared_sets, subqueries_for_sets, - no_subqueries, only_consts, noStorageOrLocal(), log.stream()); + no_subqueries, only_consts, !isRemoteStorage(), log.stream()); if (table_join.using_expression_list) actions_visitor.visit(table_join.using_expression_list); diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index 59e6268c0cc..b0a3948def2 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -347,7 +347,7 @@ private: */ void removeUnneededColumnsFromSelectClause(); - bool noStorageOrLocal() const; + bool isRemoteStorage() const; }; } From 80d8f4bd6043457e5d35fd83319713fad2688934 Mon Sep 17 00:00:00 2001 From: sundyli <543950155@qq.com> Date: Wed, 17 Oct 2018 06:43:36 -0500 Subject: [PATCH 07/28] add datatypes translate (#3403) * add datatypes translate * fixed some left English --- docs/zh/data_types/array.md | 9 +- docs/zh/data_types/boolean.md | 6 +- docs/zh/data_types/date.md | 9 +- docs/zh/data_types/datetime.md | 17 ++- docs/zh/data_types/decimal.md | 101 +++++++++++++++++- docs/zh/data_types/enum.md | 35 +++++- docs/zh/data_types/fixedstring.md | 10 +- docs/zh/data_types/float.md | 73 ++++++++++++- docs/zh/data_types/index.md | 10 +- docs/zh/data_types/int_uint.md | 23 +++- .../aggregatefunction.md | 6 +- .../nested_data_structures/index.md | 5 +- .../nested_data_structures/nested.md | 100 ++++++++++++++++- docs/zh/data_types/nullable.md | 60 ++++++++++- .../special_data_types/expression.md | 6 +- .../zh/data_types/special_data_types/index.md | 6 +- .../data_types/special_data_types/nothing.md | 25 ++++- docs/zh/data_types/special_data_types/set.md | 6 +- docs/zh/data_types/string.md | 14 ++- docs/zh/data_types/tuple.md | 54 +++++++++- docs/zh/interfaces/formats.md | 4 +- 21 files changed, 557 insertions(+), 22 deletions(-) mode change 120000 => 100644 docs/zh/data_types/array.md mode change 120000 => 100644 docs/zh/data_types/boolean.md mode change 120000 => 100644 docs/zh/data_types/date.md mode change 120000 => 100644 docs/zh/data_types/datetime.md mode change 120000 => 100644 docs/zh/data_types/decimal.md mode change 120000 => 100644 docs/zh/data_types/enum.md mode change 120000 => 100644 docs/zh/data_types/fixedstring.md mode change 120000 => 100644 docs/zh/data_types/float.md mode change 120000 => 100644 docs/zh/data_types/index.md mode change 120000 => 100644 docs/zh/data_types/int_uint.md mode change 120000 => 100644 docs/zh/data_types/nested_data_structures/aggregatefunction.md mode change 120000 => 100644 docs/zh/data_types/nested_data_structures/index.md mode change 120000 => 100644 docs/zh/data_types/nested_data_structures/nested.md mode change 120000 => 100644 docs/zh/data_types/nullable.md mode change 120000 => 100644 docs/zh/data_types/special_data_types/expression.md mode change 120000 => 100644 docs/zh/data_types/special_data_types/index.md mode change 120000 => 100644 docs/zh/data_types/special_data_types/nothing.md mode change 120000 => 100644 docs/zh/data_types/special_data_types/set.md mode change 120000 => 100644 docs/zh/data_types/string.md mode change 120000 => 100644 docs/zh/data_types/tuple.md diff --git a/docs/zh/data_types/array.md b/docs/zh/data_types/array.md deleted file mode 120000 index 808c98bf91a..00000000000 --- a/docs/zh/data_types/array.md +++ /dev/null @@ -1 +0,0 @@ -../../en/data_types/array.md \ No newline at end of file diff --git a/docs/zh/data_types/array.md b/docs/zh/data_types/array.md new file mode 100644 index 00000000000..155d32500da --- /dev/null +++ b/docs/zh/data_types/array.md @@ -0,0 +1,8 @@ + + +# Array(T) + +一个包含类型 T 的 数组(Array)。T 类型可以是任意类型,包括数组类型。 +我们不推荐使用多维数组,因为多维数组并没有得到很好支持(比如,不能在 MergeTree 引擎的表中存储多维数组)。 + +[来源文章](https://clickhouse.yandex/docs/en/data_types/array/) \ No newline at end of file diff --git a/docs/zh/data_types/boolean.md b/docs/zh/data_types/boolean.md deleted file mode 120000 index 42e84f1e52a..00000000000 --- a/docs/zh/data_types/boolean.md +++ /dev/null @@ -1 +0,0 @@ -../../en/data_types/boolean.md \ No newline at end of file diff --git a/docs/zh/data_types/boolean.md b/docs/zh/data_types/boolean.md new file mode 100644 index 00000000000..26485028358 --- /dev/null +++ b/docs/zh/data_types/boolean.md @@ -0,0 +1,5 @@ +# Boolean values + +没有单独的类型来存储 boolean 值。可以使用 UInt8 类型,取值限制为 0 或 1。 + +[来源文章](https://clickhouse.yandex/docs/en/data_types/boolean/) diff --git a/docs/zh/data_types/date.md b/docs/zh/data_types/date.md deleted file mode 120000 index d1ebc137e8f..00000000000 --- a/docs/zh/data_types/date.md +++ /dev/null @@ -1 +0,0 @@ -../../en/data_types/date.md \ No newline at end of file diff --git a/docs/zh/data_types/date.md b/docs/zh/data_types/date.md new file mode 100644 index 00000000000..39475a6558a --- /dev/null +++ b/docs/zh/data_types/date.md @@ -0,0 +1,8 @@ +# Date + +Date 用两个字节来存储从 1970-01-01 到现在的日期值(无符号的值)。Date 允许存储的值 UNIX 纪元开始后的时间值,这个值上限会在编译阶段作为一个常量存储(当前只能到 2038 年,但可以拓展到 2106 年)。 +Date 最小的值为 0000-00-00 00:00:00。 + +Date 中没有存储时区信息。 + +[来源文章](https://clickhouse.yandex/docs/en/data_types/date/) diff --git a/docs/zh/data_types/datetime.md b/docs/zh/data_types/datetime.md deleted file mode 120000 index 2eb9f44e6eb..00000000000 --- a/docs/zh/data_types/datetime.md +++ /dev/null @@ -1 +0,0 @@ -../../en/data_types/datetime.md \ No newline at end of file diff --git a/docs/zh/data_types/datetime.md b/docs/zh/data_types/datetime.md new file mode 100644 index 00000000000..4cf8661e218 --- /dev/null +++ b/docs/zh/data_types/datetime.md @@ -0,0 +1,16 @@ +# DateTime + +包含时间和日期。用4个字节来存储 Unix 时间戳(无符号的值)。允许存储的时间范围同 Date 类型。最小的值为 0000-00-00 00:00:00。 +DateTime 类型值精确到秒(不考虑闰秒) + +## Time zones + +在客户端或服务器启动时,会使用系统的时区来将时间日期从文本(划分成多个部分)到二进制之间相互转化。在文本格式中,有关夏令时信息不作存储。 + +默认情况下,客户端连接到服务的时候会使用服务端时区。可以通过客户端命令行参数 `--use_client_time_zone` 来设置使用客户端时区。 + +在所有的时区内,只支持部分与UTC相差整数小时(不考虑闰秒)的时区。 + +因此,在处理文本日期时(例如,在保存文本转储时),请记住在夏时制期间可能会出现歧义,如果时区发生变化,可能会出现与数据匹配的问题。 + +[来源文章](https://clickhouse.yandex/docs/en/data_types/datetime/) diff --git a/docs/zh/data_types/decimal.md b/docs/zh/data_types/decimal.md deleted file mode 120000 index ccea440adfa..00000000000 --- a/docs/zh/data_types/decimal.md +++ /dev/null @@ -1 +0,0 @@ -../../en/data_types/decimal.md \ No newline at end of file diff --git a/docs/zh/data_types/decimal.md b/docs/zh/data_types/decimal.md new file mode 100644 index 00000000000..a18394409fd --- /dev/null +++ b/docs/zh/data_types/decimal.md @@ -0,0 +1,100 @@ + + +# Decimal(P, S), Decimal32(S), Decimal64(S), Decimal128(S) + +有符号的定点数,在进行加减乘操作的过程中都会保留精度。对于除法,最低有效数字被丢弃(不舍入)。 + +## 参数 + +- P - 精度。范围值: [ 1 : 38 ],决定可以有多少个十进制数字(包括分数)。 +- S - 范围。范围值: [ 0 : P ],决定小数的位数。 + +对于不同的 P 参数值 Decimal 表示,以下例子都是同义的: +- [ 1 : 9 ] 的 P - for Decimal32(S) +- [ 10 : 18 ] 的 P - for Decimal64(S) +- [ 19 : 38 ] 的 P - for Decimal128(S) + +## 十进制值范围 + +- Decimal32(S) - ( -1 * 10^(9 - S), 1 * 10^(9 - S) ) +- Decimal64(S) - ( -1 * 10^(18 - S), 1 * 10^(18 - S) ) +- Decimal128(S) - ( -1 * 10^(38 - S), 1 * 10^(38 - S) ) + +例如, Decimal32(4) 可以表示 -99999.9999 到 99999.9999 范围内步数为 0.0001 的值。 + +## 内部表示方式 + +数据采用与自身位宽相同的有符号整数存储。这个数在内存中实际范围会高于上述范围,从 String 转换到十进制数的时候会做对应的检查。 + +Decimal32/Decimal64 通常处理速度要高于Decimal128,这是因为当前通用CPU不支持128位的操作导致的。 + +## 运算以及结果的类型 + +对Decimal的二进制运算导致更宽的结果类型(具有任何参数顺序)。 + +- Decimal64(S1) Decimal32(S2) -> Decimal64(S) +- Decimal128(S1) Decimal32(S2) -> Decimal128(S) +- Decimal128(S1) Decimal64(S2) -> Decimal128(S) + +精度变化的规则: + +- 加,减: S = max(S1, S2). +- 相乘: S = S1 + S2. +- 相除:S = S1. + +对于 Decimal 和整数之间的类似操作,结果为一样参数值的 Decimal。 + +没有定义 Decimal 和 Float32/Float64 的操作。如果你真的需要他们,你可以某一个参数明确地转换为 toDecimal32,toDecimal64, toDecimal128 或 toFloat32, toFloat64。注意这个操作会丢失精度,并且类型转换是一个代价昂贵的操作。 + +有一些函数对 Decimal 进行操作后是返回 Float64 的(例如,var 或 stddev)。计算的结果可能仍在 Decimal 中执行,这可能导致 Float64 和具有相同值的 Decimal 输入计算后的结果不同。 + + +## 溢出检查 + +在对 Decimal 计算的过程中,数值会有可能溢出。分数中的过多数字被丢弃(不是舍入的)。 整数中的过多数字将导致异常。 + +``` +SELECT toDecimal32(2, 4) AS x, x / 3 +``` +``` +┌──────x─┬─divide(toDecimal32(2, 4), 3)─┐ +│ 2.0000 │ 0.6666 │ +└────────┴──────────────────────────────┘ +``` + +``` +SELECT toDecimal32(4.2, 8) AS x, x * x +``` +``` +DB::Exception: Scale is out of bounds. +``` + +``` +SELECT toDecimal32(4.2, 8) AS x, 6 * x +``` +``` +DB::Exception: Decimal math overflow. +``` + +溢出检查会导致操作减慢。 如果已知溢出不可能,则使用`decimal_check_overflow`设置禁用检查是有意义的。 禁用检查并发生溢出时,结果将不正确: + +``` +SET decimal_check_overflow = 0; +SELECT toDecimal32(4.2, 8) AS x, 6 * x +``` +``` +┌──────────x─┬─multiply(6, toDecimal32(4.2, 8))─┐ +│ 4.20000000 │ -17.74967296 │ +└────────────┴──────────────────────────────────┘ +``` + +溢出检查不仅会在数学运算中进行,还会在值比较中进行: + +``` +SELECT toDecimal32(1, 8) < 100 +``` +``` +DB::Exception: Can't compare. +``` + +[来源文章](https://clickhouse.yandex/docs/en/data_types/decimal/) diff --git a/docs/zh/data_types/enum.md b/docs/zh/data_types/enum.md deleted file mode 120000 index 23ebe64773e..00000000000 --- a/docs/zh/data_types/enum.md +++ /dev/null @@ -1 +0,0 @@ -../../en/data_types/enum.md \ No newline at end of file diff --git a/docs/zh/data_types/enum.md b/docs/zh/data_types/enum.md new file mode 100644 index 00000000000..48ab4e0d130 --- /dev/null +++ b/docs/zh/data_types/enum.md @@ -0,0 +1,34 @@ +# Enum + +`Enum8` 或者 `Enum16`。 一组有限的字符串值,比 `String` 类型的存储更加有效。 + +示例: + +```text +Enum8('hello' = 1, 'world' = 2) +``` + +- 一个类型可以表示两个值: 'hello' and 'world'。 + +`Enum8` 类型的每个值范围是 `-128 ... 127`,`Enum16` 类型的每个值范围是 `-32768 ... 32767`。所有的字符串或者数字都必须是不一样的。允许存在空字符串。如果某个 Enum 类型被指定了(在表定义的时候),数字可以是任意顺序。然而,顺序并不重要。 +(译者注:如 `Enum8('he o' = 3, 'wld' = 1)` 也是合法的) + +在内存中,Enum 类型当做 `Int8` or `Int16` 对应的数值来存储。 +当以文本方式读取的时候,ClickHouse 将值解析成字符串然后去 Enum 值的集合中搜索对应字符串。如果没有找到,会抛出异常。当读取文本格式的时候,会根据读取到的字符串去找对应的数值。如果没有找到,会抛出异常。 + +当以文本方式写入的时候,ClickHouse 将值解析成字符串写入。如果 column 数据包含垃圾(不是从有用集合含有的数值),会抛弃异常。Enum 类型以二进制读取和写入的方式与 Int8 和 Int16 类型一样的。 +隐式默认值是对应类型的最小值。 + +在 `ORDER BY`, `GROUP BY`, `IN`, `DISTINCT` 中,Enums 和对应数值是一样的工作方式。比如, ORDER BY 会将它们按数值排序。对 Enums 类型使用相同和比较操作符都与操作它们隐含的数值是一样的。 + +Enum 值不能和数值比较大小。Enums 可以和一个常量字符串比较大小。如果字符串不是一个可用的 Enum 值,会抛出异常。可以使用 IN 运算符来判断一个 Enum 是否存在于某个 Enum 集合中,其中集合中的 Enum 需要用字符串表示。 + +大部分数字运算和字符串运算都没有给 Enum 类型定义,比如,Enum 类型不能和一个数相加,或进行字符串连接的操作,但是可以通过 toString 方法返回它对应的字符串。 + +Enum 值使用 `toT` 函数可以转换成数值类型,其中 T 是一个数值类型。若 `T` 恰好对应 Enum 的底层数值类型,这个转换是零消耗的。 + +Enum 类型可以被 `ALTER` 无成本地修改对应集合的值。可以通过 `ALTER` 操作来增加或删除 Enum 的成员(只要表没有用到该值,删除都是安全的)。作为安全保障,改变之前使用过的 Enum 成员将抛出异常。 + +通过 `ALTER` 操作,可以将 `Enum8` 转成 `Enum16`,反之亦然,就像 `Int8` 转 `Int16`一样。 + +[来源文章](https://clickhouse.yandex/docs/en/data_types/enum/) diff --git a/docs/zh/data_types/fixedstring.md b/docs/zh/data_types/fixedstring.md deleted file mode 120000 index 53092fcb884..00000000000 --- a/docs/zh/data_types/fixedstring.md +++ /dev/null @@ -1 +0,0 @@ -../../en/data_types/fixedstring.md \ No newline at end of file diff --git a/docs/zh/data_types/fixedstring.md b/docs/zh/data_types/fixedstring.md new file mode 100644 index 00000000000..1d8a4982e48 --- /dev/null +++ b/docs/zh/data_types/fixedstring.md @@ -0,0 +1,9 @@ +# FixedString(N) + +固定长度 N 的字符串。N 必须是严格的正自然数。 +当服务端读取长度小于 N 的字符串时候(譬如解析插入的数据),字符串通过在末尾添加空字节来达到 N 字节长度。 +当服务端读取长度大于 N 的字符串时候,会返回一个错误。 +当服务端写入一个字符串的时候(譬如写入数据到 SELECT 查询结果中),末尾的空字节会被修剪掉。 +注意这种方式与 MYSQL 的 CHAR 类型是不一样的(MYSQL 的字符串会以空格填充,然后输出的时候空格会被修剪)。 + +很少函数会使用 `FixedString(N)` 来代替 `String`,因此它并不是很方便。 diff --git a/docs/zh/data_types/float.md b/docs/zh/data_types/float.md deleted file mode 120000 index d2ae6bd11de..00000000000 --- a/docs/zh/data_types/float.md +++ /dev/null @@ -1 +0,0 @@ -../../en/data_types/float.md \ No newline at end of file diff --git a/docs/zh/data_types/float.md b/docs/zh/data_types/float.md new file mode 100644 index 00000000000..67a237ba5a5 --- /dev/null +++ b/docs/zh/data_types/float.md @@ -0,0 +1,72 @@ +# Float32, Float64 + +[浮点数](https://en.wikipedia.org/wiki/IEEE_754)。 + +类型与以下 C 类型是相同的: + +- `Float32` - `float` +- `Float64` - ` double` + +我们建议,如果可能的话尽量用整形来存储数据。比如,将一定精度的浮点数转换成整形,譬如货币金额或者毫秒单位的加载时间。 + +## 使用浮点数 + +- 对浮点数进行计算可能引起四舍五入的误差。 + + ```sql + SELECT 1 - 0.9 + ``` + + ``` + ┌───────minus(1, 0.9)─┐ + │ 0.09999999999999998 │ + └─────────────────────┘ + ``` + +- 计算的结果取决于计算方式(处理器类型和计算机系统架构) + +- 浮点数计算可能出现这样的结果,比如 "infinity" (`Inf`) 和 "not-a-number" (`NaN`)。对浮点数计算的时候应该考虑到这点。 + +- 当一行行阅读浮点数的时候,浮点数的结果可能不是机器最近显示的数值。 + +## NaN 和 Inf + +相比于 SQL,ClickHouse 支持以下几种浮点数分类: + +- `Inf` – 正无穷。 + + ```sql + SELECT 0.5 / 0 + ``` + + ``` + ┌─divide(0.5, 0)─┐ + │ inf │ + └────────────────┘ + ``` +- `-Inf` – 负无穷。 + + ```sql + SELECT -0.5 / 0 + ``` + + ``` + ┌─divide(-0.5, 0)─┐ + │ -inf │ + └─────────────────┘ + ``` +- `NaN` – 非数字。 + + ``` + SELECT 0 / 0 + ``` + + ``` + ┌─divide(0, 0)─┐ + │ nan │ + └──────────────┘ + ``` + +可以在[ORDER BY 子句](../query_language/queries.md#query_language-queries-order_by) 查看更多关于 ` NaN` 排序的规则。 + +[来源文章](https://clickhouse.yandex/docs/en/data_types/float/) diff --git a/docs/zh/data_types/index.md b/docs/zh/data_types/index.md deleted file mode 120000 index c9f29d637f3..00000000000 --- a/docs/zh/data_types/index.md +++ /dev/null @@ -1 +0,0 @@ -../../en/data_types/index.md \ No newline at end of file diff --git a/docs/zh/data_types/index.md b/docs/zh/data_types/index.md new file mode 100644 index 00000000000..785216a22a9 --- /dev/null +++ b/docs/zh/data_types/index.md @@ -0,0 +1,9 @@ + + +# 数据类型 + +ClickHouse 可以在数据表中存储多种数据类型。 + +本节将描述 ClickHouse 支持的数据类型以及在使用 与/或 运算实现他们时候的特殊考虑(如果有)。 + +[来源文章](https://clickhouse.yandex/docs/en/data_types/) diff --git a/docs/zh/data_types/int_uint.md b/docs/zh/data_types/int_uint.md deleted file mode 120000 index 3a913c9328e..00000000000 --- a/docs/zh/data_types/int_uint.md +++ /dev/null @@ -1 +0,0 @@ -../../en/data_types/int_uint.md \ No newline at end of file diff --git a/docs/zh/data_types/int_uint.md b/docs/zh/data_types/int_uint.md new file mode 100644 index 00000000000..c142cf50106 --- /dev/null +++ b/docs/zh/data_types/int_uint.md @@ -0,0 +1,22 @@ + + +# UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64 + +固定长度的整型,包括有符号整型或无符号整型。 + +## 整型范围 + +- Int8 - [-128 : 127] +- Int16 - [-32768 : 32767] +- Int32 - [-2147483648 : 2147483647] +- Int64 - [-9223372036854775808 : 9223372036854775807] + +## 无符号整型范围 + +- UInt8 - [0 : 255] +- UInt16 - [0 : 65535] +- UInt32 - [0 : 4294967295] +- UInt64 - [0 : 18446744073709551615] + + +[来源文章](https://clickhouse.yandex/docs/en/data_types/int_uint/) diff --git a/docs/zh/data_types/nested_data_structures/aggregatefunction.md b/docs/zh/data_types/nested_data_structures/aggregatefunction.md deleted file mode 120000 index 36544324d2b..00000000000 --- a/docs/zh/data_types/nested_data_structures/aggregatefunction.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/data_types/nested_data_structures/aggregatefunction.md \ No newline at end of file diff --git a/docs/zh/data_types/nested_data_structures/aggregatefunction.md b/docs/zh/data_types/nested_data_structures/aggregatefunction.md new file mode 100644 index 00000000000..bb9d57274ec --- /dev/null +++ b/docs/zh/data_types/nested_data_structures/aggregatefunction.md @@ -0,0 +1,5 @@ +# AggregateFunction(name, types_of_arguments...) + +表示聚合函数中的中间状态。可以在聚合函数中通过 '-State' 后缀来访问它。更多信息,参考 "AggregatingMergeTree"。 + +[来源文章](https://clickhouse.yandex/docs/en/data_types/nested_data_structures/aggregatefunction/) \ No newline at end of file diff --git a/docs/zh/data_types/nested_data_structures/index.md b/docs/zh/data_types/nested_data_structures/index.md deleted file mode 120000 index a5659a9c5cd..00000000000 --- a/docs/zh/data_types/nested_data_structures/index.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/data_types/nested_data_structures/index.md \ No newline at end of file diff --git a/docs/zh/data_types/nested_data_structures/index.md b/docs/zh/data_types/nested_data_structures/index.md new file mode 100644 index 00000000000..fcb01993389 --- /dev/null +++ b/docs/zh/data_types/nested_data_structures/index.md @@ -0,0 +1,4 @@ +# 嵌套数据类型 + + +[来源文章](https://clickhouse.yandex/docs/en/data_types/nested_data_structures/) diff --git a/docs/zh/data_types/nested_data_structures/nested.md b/docs/zh/data_types/nested_data_structures/nested.md deleted file mode 120000 index 653a1ce31c3..00000000000 --- a/docs/zh/data_types/nested_data_structures/nested.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/data_types/nested_data_structures/nested.md \ No newline at end of file diff --git a/docs/zh/data_types/nested_data_structures/nested.md b/docs/zh/data_types/nested_data_structures/nested.md new file mode 100644 index 00000000000..2ab6d06cbdc --- /dev/null +++ b/docs/zh/data_types/nested_data_structures/nested.md @@ -0,0 +1,99 @@ +# Nested(Name1 Type1, Name2 Type2, ...) + +类似嵌套表的嵌套数据结构。嵌套数据结构的参数(列名和列类型)与 CREATE 查询类似。每个表可以对应任意多行嵌套数据结构。 + +示例: + +```sql +CREATE TABLE test.visits +( + CounterID UInt32, + StartDate Date, + Sign Int8, + IsNew UInt8, + VisitID UInt64, + UserID UInt64, + ... + Goals Nested + ( + ID UInt32, + Serial UInt32, + EventTime DateTime, + Price Int64, + OrderID String, + CurrencyID UInt32 + ), + ... +) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192, Sign) +``` + +上述示例声明了 `Goals` 这种嵌套数据结构,它包含访客转化相关的数据(访客达到的目标)。在 'visits' 表中每一行都可以对应零个或者任意个转化数据。 + +只支持一级嵌套。嵌套结构的列中,若列的类型是数组类型,那么该列其实和多维数组是相同的,所以目前嵌套层级的支持很局限(MergeTree 引擎中不支持存储这样的列) + +大多数情况下,处理嵌套数据结构时,会指定一个单独的列。为了这样实现,列的名称会与点号连接起来。这些列构成了一组匹配类型。在同一条嵌套数据中,所有的列都具有相同的长度。 + +示例: + +```sql +SELECT + Goals.ID, + Goals.EventTime +FROM test.visits +WHERE CounterID = 101500 AND length(Goals.ID) < 5 +LIMIT 10 +``` + +```text +┌─Goals.ID───────────────────────┬─Goals.EventTime───────────────────────────────────────────────────────────────────────────┐ +│ [1073752,591325,591325] │ ['2014-03-17 16:38:10','2014-03-17 16:38:48','2014-03-17 16:42:27'] │ +│ [1073752] │ ['2014-03-17 00:28:25'] │ +│ [1073752] │ ['2014-03-17 10:46:20'] │ +│ [1073752,591325,591325,591325] │ ['2014-03-17 13:59:20','2014-03-17 22:17:55','2014-03-17 22:18:07','2014-03-17 22:18:51'] │ +│ [] │ [] │ +│ [1073752,591325,591325] │ ['2014-03-17 11:37:06','2014-03-17 14:07:47','2014-03-17 14:36:21'] │ +│ [] │ [] │ +│ [] │ [] │ +│ [591325,1073752] │ ['2014-03-17 00:46:05','2014-03-17 00:46:05'] │ +│ [1073752,591325,591325,591325] │ ['2014-03-17 13:28:33','2014-03-17 13:30:26','2014-03-17 18:51:21','2014-03-17 18:51:45'] │ +└────────────────────────────────┴───────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +所以可以简单地把嵌套数据结构当做是所有列都是相同长度的多列数组。 + +SELECT 查询只有在使用 ARRAY JOIN 的时候才可以指定整个嵌套数据结构的名称。更多信息,参考 "ARRAY JOIN 子句"。示例: + +```sql +SELECT + Goal.ID, + Goal.EventTime +FROM test.visits +ARRAY JOIN Goals AS Goal +WHERE CounterID = 101500 AND length(Goals.ID) < 5 +LIMIT 10 +``` + +```text +┌─Goal.ID─┬──────Goal.EventTime─┐ +│ 1073752 │ 2014-03-17 16:38:10 │ +│ 591325 │ 2014-03-17 16:38:48 │ +│ 591325 │ 2014-03-17 16:42:27 │ +│ 1073752 │ 2014-03-17 00:28:25 │ +│ 1073752 │ 2014-03-17 10:46:20 │ +│ 1073752 │ 2014-03-17 13:59:20 │ +│ 591325 │ 2014-03-17 22:17:55 │ +│ 591325 │ 2014-03-17 22:18:07 │ +│ 591325 │ 2014-03-17 22:18:51 │ +│ 1073752 │ 2014-03-17 11:37:06 │ +└─────────┴─────────────────────┘ +``` + +不能对整个嵌套数据结构进行 SELECT 。只能显式地列出它的一部分列。 + +对于 INSERT 查询,可以单独地传入所有嵌套数据结构中的列数组(假如它们是单独的列数组)。在插入过程中,系统会检查它们是否有相同的长度。 + +对于 DESCRIBE 查询,嵌套数据结构中的列会以相同的方式分别列出来。 + +ALTER 查询对嵌套数据结构的操作非常局限。 + +[来源文章](https://clickhouse.yandex/docs/en/data_types/nested_data_structures/nested/) diff --git a/docs/zh/data_types/nullable.md b/docs/zh/data_types/nullable.md deleted file mode 120000 index 0233f91d954..00000000000 --- a/docs/zh/data_types/nullable.md +++ /dev/null @@ -1 +0,0 @@ -../../en/data_types/nullable.md \ No newline at end of file diff --git a/docs/zh/data_types/nullable.md b/docs/zh/data_types/nullable.md new file mode 100644 index 00000000000..00c24c26c3a --- /dev/null +++ b/docs/zh/data_types/nullable.md @@ -0,0 +1,59 @@ + + +# Nullable(TypeName) + +允许加上一个符号 ([NULL](../query_language/syntax.md#null-literal)) 表示“缺失值”和 `TypeName` 允许的正常值。 例如,`Nullable(Int8)` 类型列可以存储 `Int8` 类型值,而没有值的行将存储 `NULL` 。 + +对于`TypeName`,您不能使用复合数据类型 [Array](array.md#data_type is array) 和 [Tuple](tuple.md#data_type-tuple)。 复合数据类型可以包含 `Nullable` 类型值,例如 `Array (Nullable(Int8))`。 + + +`Nullable` 类型字段不能包含在表索引中。 + +除非在 ClickHouse 服务器配置中另有说明,否则 `NULL` 是任何 `Nullable` 类型的默认值。 + +## 存储特性 + +要在表的列中存储 `Nullable` 类型值,ClickHouse 除了使用带有值的普通文件外,还使用带有 `NULL` 掩码的单独文件。 掩码文件中的条目允许 ClickHouse 区分每个表行的 `NULL` 和相应数据类型的默认值。 由于附加了新文件,`Nullable` 列与类似的普通文件相比消耗额外的存储空间。 + +!!! info "Note" + 使用 `Nullable` 几乎总是会对性能产生负面影响,在设计数据库时请记住这一点。 + + +## 使用案例 + +``` +:) CREATE TABLE t_null(x Int8, y Nullable(Int8)) ENGINE TinyLog + +CREATE TABLE t_null +( + x Int8, + y Nullable(Int8) +) +ENGINE = TinyLog + +Ok. + +0 rows in set. Elapsed: 0.012 sec. + +:) INSERT INTO t_null VALUES (1, NULL) + +INSERT INTO t_null VALUES + +Ok. + +1 rows in set. Elapsed: 0.007 sec. + +:) SELECT x + y FROM t_null + +SELECT x + y +FROM t_null + +┌─plus(x, y)─┐ +│ ᴺᵁᴸᴸ │ +│ 5 │ +└────────────┘ + +2 rows in set. Elapsed: 0.144 sec. +``` + +[来源文章](https://clickhouse.yandex/docs/en/data_types/nullable/) diff --git a/docs/zh/data_types/special_data_types/expression.md b/docs/zh/data_types/special_data_types/expression.md deleted file mode 120000 index 4cec632b416..00000000000 --- a/docs/zh/data_types/special_data_types/expression.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/data_types/special_data_types/expression.md \ No newline at end of file diff --git a/docs/zh/data_types/special_data_types/expression.md b/docs/zh/data_types/special_data_types/expression.md new file mode 100644 index 00000000000..14e8aaf5362 --- /dev/null +++ b/docs/zh/data_types/special_data_types/expression.md @@ -0,0 +1,5 @@ +# Expression + +用在高阶函数中的 lambda 表达式中。 + +[来源文章](https://clickhouse.yandex/docs/en/data_types/special_data_types/expression/) diff --git a/docs/zh/data_types/special_data_types/index.md b/docs/zh/data_types/special_data_types/index.md deleted file mode 120000 index f3ca4a47f98..00000000000 --- a/docs/zh/data_types/special_data_types/index.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/data_types/special_data_types/index.md \ No newline at end of file diff --git a/docs/zh/data_types/special_data_types/index.md b/docs/zh/data_types/special_data_types/index.md new file mode 100644 index 00000000000..03f66e36189 --- /dev/null +++ b/docs/zh/data_types/special_data_types/index.md @@ -0,0 +1,5 @@ +# Special Data Types + +特殊数据类型的值既不能存在表中也不能在结果中输出,但是可以被用于表示查询过程中的中间结果。 + +[来源文章](https://clickhouse.yandex/docs/en/data_types/special_data_types/) diff --git a/docs/zh/data_types/special_data_types/nothing.md b/docs/zh/data_types/special_data_types/nothing.md deleted file mode 120000 index 197a752ce9c..00000000000 --- a/docs/zh/data_types/special_data_types/nothing.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/data_types/special_data_types/nothing.md \ No newline at end of file diff --git a/docs/zh/data_types/special_data_types/nothing.md b/docs/zh/data_types/special_data_types/nothing.md new file mode 100644 index 00000000000..dd7f51cca0f --- /dev/null +++ b/docs/zh/data_types/special_data_types/nothing.md @@ -0,0 +1,24 @@ + + +# Nothing + +此数据类型的唯一目的是表示不是期望值的情况。 所以你不能创建一个` Nothing` 类型的值。 + +例如, [NULL](../../query_language/syntax.md#null-literal) 有 `Nullable(Nothing)`。更多参见[Nullable](../../data_types/nullable.md#data_type-nullable). + +`Nothing` 类型也可用于表示空数组: + +```bash +:) SELECT toTypeName(array()) + +SELECT toTypeName([]) + +┌─toTypeName(array())─┐ +│ Array(Nothing) │ +└─────────────────────┘ + +1 rows in set. Elapsed: 0.062 sec. +``` + + +[来源文章](https://clickhouse.yandex/docs/en/data_types/special_data_types/nothing/) diff --git a/docs/zh/data_types/special_data_types/set.md b/docs/zh/data_types/special_data_types/set.md deleted file mode 120000 index 5beb14114d3..00000000000 --- a/docs/zh/data_types/special_data_types/set.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/data_types/special_data_types/set.md \ No newline at end of file diff --git a/docs/zh/data_types/special_data_types/set.md b/docs/zh/data_types/special_data_types/set.md new file mode 100644 index 00000000000..0e62f64f540 --- /dev/null +++ b/docs/zh/data_types/special_data_types/set.md @@ -0,0 +1,5 @@ +# Set + +可以用在 IN 表达式的右半边。 + +[来源文章](https://clickhouse.yandex/docs/en/data_types/special_data_types/set/) diff --git a/docs/zh/data_types/string.md b/docs/zh/data_types/string.md deleted file mode 120000 index 7bdd739398f..00000000000 --- a/docs/zh/data_types/string.md +++ /dev/null @@ -1 +0,0 @@ -../../en/data_types/string.md \ No newline at end of file diff --git a/docs/zh/data_types/string.md b/docs/zh/data_types/string.md new file mode 100644 index 00000000000..6d45b8f4741 --- /dev/null +++ b/docs/zh/data_types/string.md @@ -0,0 +1,13 @@ +# String + +字符串可以任意长度的。它可以包含任意的字节组合,包括空字符。 +String 类型替代了其他 DBMSs 中的 VARCHAR, BLOB, CLOB 等类型。 + +## 编码 + +ClickHouse 没有编码的概念。字符串可以是任意的字节组合,按它们原本的方式进行存储和输出。 +若需存储文本,我们建议使用 UTF-8 编码。至少,如果你的终端使用UTF-8(推荐),这样读写就不需要进行任何的转换了。 +同样,对不同的编码文本 ClickHouse 会有不同处理字符串的函数。 +比如,`length` 函数可以计算字符串包含的字节数组的长度,然而 `lengthUTF8` 函数是假设字符串以 UTF-8 编码,计算的是字符串包含的 Unicode 字符的长度。 + +[来源文章](https://clickhouse.yandex/docs/en/data_types/string/) diff --git a/docs/zh/data_types/tuple.md b/docs/zh/data_types/tuple.md deleted file mode 120000 index d30a8463aeb..00000000000 --- a/docs/zh/data_types/tuple.md +++ /dev/null @@ -1 +0,0 @@ -../../en/data_types/tuple.md \ No newline at end of file diff --git a/docs/zh/data_types/tuple.md b/docs/zh/data_types/tuple.md new file mode 100644 index 00000000000..02320a4be73 --- /dev/null +++ b/docs/zh/data_types/tuple.md @@ -0,0 +1,53 @@ +# Tuple(T1, T2, ...) + +元组,元组中的每个元素都有独立的[类型](index.md#data_types)。 + +元组不能写入表中(除 Memory 表外)。它们会用于临时列分组。在某个查询中,IN 表达式和带特定参数的 lambda 函数可以来对临时列进行分组。更多信息,参见 [IN 操作符] (../query_language/select.md#in_operators) 和 [高阶函数](../query_language/functions/higher_order_functions.md#higher_order_functions)。 + +## 创建元组 + +可以用下面函数来创建元组: + +``` +tuple(T1, T2, ...) +``` + +创建元组的示例: + +``` +:) SELECT tuple(1,'a') AS x, toTypeName(x) + +SELECT + (1, 'a') AS x, + toTypeName(x) + +┌─x───────┬─toTypeName(tuple(1, 'a'))─┐ +│ (1,'a') │ Tuple(UInt8, String) │ +└─────────┴───────────────────────────┘ + +1 rows in set. Elapsed: 0.021 sec. +``` + +## 元组中的数据类型 + +当动态创建元组时,ClickHouse 会自动为元组的每一个参数赋予最小可表达的类型。如果该参数是 [NULL](../query_language/syntax.md#null-literal),那么这个元组对应元素就是 [Nullable](nullable.md#data_type-nullable)。 + +自动识别数据类型的示例: + +``` +SELECT tuple(1, NULL) AS x, toTypeName(x) + +SELECT + (1, NULL) AS x, + toTypeName(x) + +┌─x────────┬─toTypeName(tuple(1, NULL))──────┐ +│ (1,NULL) │ Tuple(UInt8, Nullable(Nothing)) │ +└──────────┴─────────────────────────────────┘ + +1 rows in set. Elapsed: 0.002 sec. +``` + + +[来源文章](https://clickhouse.yandex/docs/en/data_types/tuple/) + diff --git a/docs/zh/interfaces/formats.md b/docs/zh/interfaces/formats.md index ec905713b1c..fdfb2ccb0e7 100644 --- a/docs/zh/interfaces/formats.md +++ b/docs/zh/interfaces/formats.md @@ -173,7 +173,7 @@ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMA 解析的时候,可以使用或不使用引号来解析所有值。支持双引号和单引号。行也可以不用引号排列。 在这种情况下,它们被解析为逗号或换行符(CR 或 LF)。在解析不带引号的行时,若违反 RFC 规则,会忽略前导和尾随的空格和制表符。 对于换行,全部支持 Unix(LF),Windows(CR LF)和 Mac OS Classic(CR LF)。 -`NULL` is formatted as `\N`. +`NULL` 将输出为 `\N`。 CSV 格式是和 TabSeparated 一样的方式输出总数和极值。 @@ -511,7 +511,7 @@ test: string with 'quotes' and with some special characters ``` -Compare with the Vertical format: +和 Vertical 格式相比: ``` :) SELECT 'string with \'quotes\' and \t with some special \n characters' AS test FORMAT Vertical; From 01d7d05847c761da9a99fed92b18c6becfc36e14 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Oct 2018 16:42:39 +0300 Subject: [PATCH 08/28] Added russian changelog for version 18.12.17 (draft) [#CLICKHOUSE-2] --- CHANGELOG_RU.md | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/CHANGELOG_RU.md b/CHANGELOG_RU.md index 9816ad4ea18..6c698595558 100644 --- a/CHANGELOG_RU.md +++ b/CHANGELOG_RU.md @@ -1,3 +1,25 @@ +## ClickHouse release 18.12.17, 2018-09-16 + +### Новые возможности: + +* `invalidate_query` (возможность задать запрос для проверки необходимости обновления внешнего словаря) реализована для источника `clickhouse`. +* Добавлена возможность использования типов данных `UInt*`, `Int*`, `DateTime` (наравне с типом `Date`) в качестве ключа внешнего словаря типа `range_hashed`, определяющего границы диапазонов. Возможность использования NULL в качестве обозначения открытого диапазона. [Vasily Nemkov]() +* Для типа `Decimal` добавлена поддержка агрегатных функций `var*`, `stddev*`. +* Для типа `Decimal` добавлена поддержка математических функций (`exp`, `sin` и т. п.) +* В таблицу `system.part_log` добавлен столбец `partition_id`. + +### Исправление ошибок: + +* Исправлена работа `Merge` таблицы поверх `Distributed` таблиц. [Winter Zhang]() +* Исправлена несовместимость (лишняя зависимость от версии `glibc`), приводящая к невозможности запуска ClickHouse на `Ubuntu Precise` и более старых. Несовместимость возникла в версии 18.12.13. +* Исправлены ошибки в работе настройки `enable_optimize_predicate_expression`. [Winter Zhang]() +* Исправлено незначительное нарушение обратной совместимости, проявляющееся при одновременной работе на кластере реплик версий до 18.12.13 и создании новой реплики таблицы на сервере более новой версии (выдаётся сообщение `Can not clone replica, because the ... updated to new ClickHouse version`, что полностью логично, но не должно было происходить). + +### Обратно несовместимые изменения: + +* Настройка `enable_optimize_predicate_expression` включена по-умолчанию, что конечно очень оптимистично. При возникновении ошибок анализа запроса, связанных с поиском имён столбцов, следует выставить `enable_optimize_predicate_expression` в 0. [Winter Zhang]() + + ## ClickHouse release 18.12.14, 2018-09-13 ### Новые возможности: From 22f51f103cc8b690677d3031eded0ffde592cac4 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 17 Oct 2018 18:53:06 +0300 Subject: [PATCH 09/28] WIP on docs/website (#3409) * CLICKHOUSE-4063: less manual html @ index.md * CLICKHOUSE-4063: recommend markdown="1" in README.md * CLICKHOUSE-4003: manually purge custom.css for now * CLICKHOUSE-4064: expand
before any print (including to pdf) * CLICKHOUSE-3927: rearrange interfaces/formats.md a bit * CLICKHOUSE-3306: add few http headers * Remove copy-paste introduced in #3392 * Hopefully better chinese fonts #3392 * get rid of tabs @ custom.css * Apply comments and patch from #3384 * Add jdbc.md to ToC and some translation, though it still looks badly incomplete * minor punctuation * Add some backlinks to official website from mirrors that just blindly take markdown sources * Do not make fonts extra light * find . -name '*.md' -type f | xargs -I{} perl -pi -e 's//g' {} * find . -name '*.md' -type f | xargs -I{} perl -pi -e 's/ sql/g' {} * Remove outdated stuff from roadmap.md * Not so light font on front page too * Refactor Chinese formats.md to match recent changes in other languages * Update some links on front page * Remove some outdated comment * Add twitter link to front page * More front page links tuning * Add Amsterdam meetup link * Smaller font to avoid second line * Add Amsterdam link to README.md * Proper docs nav translation * Back to 300 font-weight except Chinese * fix docs build * Update Amsterdam link --- README.md | 1 + docs/toc_zh.yml | 2 +- .../assets/stylesheets/custom.css | 2 +- .../partials/language/en.html | 4 +++ .../partials/language/fa.html | 4 +++ .../partials/language/ru.html | 4 +++ .../partials/language/zh.html | 4 +++ .../mkdocs-material-theme/partials/nav.html | 24 +++------------ docs/zh/data_types/float.md | 2 +- website/index.css | 4 +-- website/index.html | 29 ++++++++++--------- 11 files changed, 41 insertions(+), 39 deletions(-) diff --git a/README.md b/README.md index dec43532bc1..e5ca7ce2ec5 100644 --- a/README.md +++ b/README.md @@ -12,3 +12,4 @@ ClickHouse is an open-source column-oriented database management system that all ## Upcoming Meetups * [Beijing on October 28](http://www.clickhouse.com.cn/topic/5ba0e3f99d28dfde2ddc62a1) +* [Amsterdam on November 15](https://events.yandex.com/events/meetings/15-11-2018/) diff --git a/docs/toc_zh.yml b/docs/toc_zh.yml index 4c6ce34f192..3ae4f5de3c9 100644 --- a/docs/toc_zh.yml +++ b/docs/toc_zh.yml @@ -3,7 +3,7 @@ nav: - '介绍': - '概貌': 'index.md' - 'ClickHouse的独特功能': 'introduction/distinctive_features.md' - - 'ClickHouse不擅长做什么': 'introduction/features_considered_disadvantages.md' + - 'ClickHouse功能可被视为缺点': 'introduction/features_considered_disadvantages.md' - '性能': 'introduction/performance.md' - 'Yandex.Metrica使用案例': 'introduction/ya_metrika_task.md' diff --git a/docs/tools/mkdocs-material-theme/assets/stylesheets/custom.css b/docs/tools/mkdocs-material-theme/assets/stylesheets/custom.css index 0024345554b..8569b1d033d 100644 --- a/docs/tools/mkdocs-material-theme/assets/stylesheets/custom.css +++ b/docs/tools/mkdocs-material-theme/assets/stylesheets/custom.css @@ -64,7 +64,7 @@ } body { - font: 400 14pt/200% 'Yandex Sans Text Web', Arial, sans-serif; + font: 300 14pt/200% 'Yandex Sans Text Web', Arial, sans-serif; } body.md-lang-zh { diff --git a/docs/tools/mkdocs-material-theme/partials/language/en.html b/docs/tools/mkdocs-material-theme/partials/language/en.html index 2068c97dace..e84e566446a 100644 --- a/docs/tools/mkdocs-material-theme/partials/language/en.html +++ b/docs/tools/mkdocs-material-theme/partials/language/en.html @@ -7,6 +7,10 @@ "footer.next": "Next", "meta.comments": "Comments", "meta.source": "Source", + "nav.multi_page": "Multi page version", + "nav.pdf": "PDF version", + "nav.single_page": "Single page version", + "nav.source": "ClickHouse source code", "search.placeholder": "Search", "search.result.placeholder": "Type to start searching", "search.result.none": "No matching documents", diff --git a/docs/tools/mkdocs-material-theme/partials/language/fa.html b/docs/tools/mkdocs-material-theme/partials/language/fa.html index 54a7d2860d3..71f50043932 100644 --- a/docs/tools/mkdocs-material-theme/partials/language/fa.html +++ b/docs/tools/mkdocs-material-theme/partials/language/fa.html @@ -8,6 +8,10 @@ "footer.next": "بعدی", "meta.comments": "نظرات", "meta.source": "منبع", + "nav.multi_page": "نسخه چند صفحه ای", + "nav.pdf": "نسخه PDF", + "nav.single_page": "نسخه تک صفحه", + "nav.source": "کد منبع کلیک", "search.language": "", "search.pipeline.stopwords": false, "search.pipeline.trimmer": false, diff --git a/docs/tools/mkdocs-material-theme/partials/language/ru.html b/docs/tools/mkdocs-material-theme/partials/language/ru.html index d762438bb60..b8baaea5eff 100644 --- a/docs/tools/mkdocs-material-theme/partials/language/ru.html +++ b/docs/tools/mkdocs-material-theme/partials/language/ru.html @@ -7,6 +7,10 @@ "footer.next": "Вперед", "meta.comments": "Комментарии", "meta.source": "Исходный код", + "nav.multi_page": "Многостраничная версия", + "nav.pdf": "PDF версия", + "nav.single_page": "Одностраничная версия", + "nav.source": "Исходный код ClickHouse", "search.placeholder": "Поиск", "search.result.placeholder": "Начните печатать для поиска", "search.result.none": "Совпадений не найдено", diff --git a/docs/tools/mkdocs-material-theme/partials/language/zh.html b/docs/tools/mkdocs-material-theme/partials/language/zh.html index 84c77d43c8e..89c94ee2a98 100644 --- a/docs/tools/mkdocs-material-theme/partials/language/zh.html +++ b/docs/tools/mkdocs-material-theme/partials/language/zh.html @@ -7,6 +7,10 @@ "footer.next": "前进", "meta.comments": "评论", "meta.source": "来源", + "nav.multi_page": "多页版本", + "nav.pdf": "PDF版本", + "nav.single_page": "单页版本", + "nav.source": "ClickHouse源代码", "search.placeholder": "搜索", "search.result.placeholder": "键入以开始搜索", "search.result.none": "没有找到符合条件的结果", diff --git a/docs/tools/mkdocs-material-theme/partials/nav.html b/docs/tools/mkdocs-material-theme/partials/nav.html index 14f5d507eea..cf58c0c4130 100644 --- a/docs/tools/mkdocs-material-theme/partials/nav.html +++ b/docs/tools/mkdocs-material-theme/partials/nav.html @@ -15,35 +15,19 @@ @@ -306,15 +309,20 @@

Success Stories

@@ -428,6 +436,9 @@ clickhouse-client rel="external nofollow" target="_blank">English or in Russian. +
  • Follow official Twitter account.
  • Or email ClickHouse team at Yandex directly: @@ -485,16 +496,6 @@ clickhouse-client if (hostParts.length > 2 && hostParts[0] != 'test') { window.location.host = hostParts[0] + '.' + hostParts[1]; } - -/* var available_distributives = ['xenial', 'trusty', 'precise']; - available_distributives.forEach(function (name) { - $('#ubuntu_' + name).on('click', function () { - $('#distributive').html(name); - available_distributives.forEach(function (distr) { - $('#ubuntu_' + distr).attr('class', (name == distr) ? 'distributive_selected' : 'distributive_not_selected'); - }); - }); - });*/ }); From 9547e2debb58d594733df48893ae64b4c8b39e05 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B0=A2=E7=A3=8A?= <2217232293@qq.com> Date: Thu, 18 Oct 2018 16:06:28 +0800 Subject: [PATCH 10/28] fix translate about decimal doc (#3412) --- docs/en/data_types/decimal.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/data_types/decimal.md b/docs/en/data_types/decimal.md index 101b59adaec..cd7273c0441 100644 --- a/docs/en/data_types/decimal.md +++ b/docs/en/data_types/decimal.md @@ -4,7 +4,7 @@ Signed fixed point numbers that keep precision during add, subtract and multiply operations. For division least significant digits are discarded (not rounded). -## Параметры +## Parameters - P - precision. Valid range: [ 1 : 38 ]. Determines how many decimal digits number can have (including fraction). - S - scale. Valid range: [ 0 : P ]. Determines how many decimal digits fraction can have. From 1d7ef9133ebec28d833fee23a204f85eff386fe7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 18 Oct 2018 15:56:48 +0300 Subject: [PATCH 11/28] CLICKHOUSE-4056: Add ability to run build with ccache or distcc --- docker/packager/binary/Dockerfile | 2 ++ docker/packager/deb/Dockerfile | 2 ++ docker/packager/packager | 17 ++++++++++++++--- 3 files changed, 18 insertions(+), 3 deletions(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 49b757e1b00..6e6395e1b9d 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -5,6 +5,8 @@ RUN apt-get update -y \ apt-get install --yes --no-install-recommends \ bash \ cmake \ + ccache \ + distcc \ curl \ gcc-7 \ g++-7 \ diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index 9e0abd3afd4..b019d291080 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -5,6 +5,8 @@ RUN apt-get update -y \ apt-get install --yes --no-install-recommends \ bash \ fakeroot \ + ccache \ + distcc \ cmake \ curl \ gcc-7 \ diff --git a/docker/packager/packager b/docker/packager/packager index f101f908683..db12ed0077a 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -52,7 +52,7 @@ def run_image_with_env(image_name, output, env_variables, ch_root): subprocess.check_call(cmd, shell=True) -def parse_env_variables(build_type, compiler, sanitizer, package_type): +def parse_env_variables(build_type, compiler, sanitizer, package_type, cache, distcc_hosts): result = [] if package_type == "deb": result.append("DEB_CC={}".format(compiler)) @@ -66,18 +66,28 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type): if build_type: result.append("BUILD_TYPE={}".format(build_type)) + if cache: + result.append("CCACHE_PREFIX={}".format(cache)) + + if distcc_hosts: + hosts_with_params = ["{}/24,lz4".format(host) for host in distcc_hosts] + ["localhost/`nproc`"] + result.append('DISTCC_HOSTS="{}"'.format(" ".join(hosts_with_params))) + elif cache == "distcc": + result.append('DISTCC_HOSTS="{}"'.format("localhost/`nproc`")) return result if __name__ == "__main__": logging.basicConfig(level=logging.INFO, format='%(asctime)s %(message)s') parser = argparse.ArgumentParser(description="ClickHouse building script via docker") - parser.add_argument("--package-type", choices=IMAGE_MAP.keys()) + parser.add_argument("--package-type", choices=IMAGE_MAP.keys(), required=True) parser.add_argument("--clickhouse-repo-path", default="../../") parser.add_argument("--output-dir", required=True) parser.add_argument("--build-type", choices=("debug", ""), default="") parser.add_argument("--compiler", choices=("clang-6.0", "gcc-7", "gcc-8"), default="gcc-7") parser.add_argument("--sanitizer", choices=("address", "thread", "memory", "undefined", ""), default="") + parser.add_argument("--cache", choices=("ccache", "distcc")) + parser.add_argument("--distcc-hosts", nargs="+") parser.add_argument("--force-build-image", action="store_true") args = parser.parse_args() @@ -95,5 +105,6 @@ if __name__ == "__main__": if not check_image_exists_locally(image_name) or args.force_build_image: if not pull_image(image_name) or args.force_build_image: build_image(image_name, dockerfile) - run_image_with_env(image_name, args.output_dir, parse_env_variables(args.build_type, args.compiler, args.sanitizer, args.package_type), ch_root) + env_prepared = parse_env_variables(args.build_type, args.compiler, args.sanitizer, args.package_type, args.cache, args.distcc_hosts) + run_image_with_env(image_name, args.output_dir, env_prepared, ch_root) logging.info("Output placed into {}".format(args.output_dir)) From 935efe81b7825521f85f25407b208b0bcf714105 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 18 Oct 2018 19:49:00 +0300 Subject: [PATCH 12/28] CLICKHOUSE-4056: Fix build params --- docker/packager/binary/Dockerfile | 1 + docker/packager/deb/Dockerfile | 1 + docker/packager/packager | 2 +- 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 6e6395e1b9d..2206bcc7e49 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -7,6 +7,7 @@ RUN apt-get update -y \ cmake \ ccache \ distcc \ + distcc-pump \ curl \ gcc-7 \ g++-7 \ diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index b019d291080..08eeb3f3578 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -7,6 +7,7 @@ RUN apt-get update -y \ fakeroot \ ccache \ distcc \ + distcc-pump \ cmake \ curl \ gcc-7 \ diff --git a/docker/packager/packager b/docker/packager/packager index db12ed0077a..fed0c54fae5 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -70,7 +70,7 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, cache, di result.append("CCACHE_PREFIX={}".format(cache)) if distcc_hosts: - hosts_with_params = ["{}/24,lz4".format(host) for host in distcc_hosts] + ["localhost/`nproc`"] + hosts_with_params = ["{}/24,lzo".format(host) for host in distcc_hosts] + ["localhost/`nproc`"] result.append('DISTCC_HOSTS="{}"'.format(" ".join(hosts_with_params))) elif cache == "distcc": result.append('DISTCC_HOSTS="{}"'.format("localhost/`nproc`")) From 5aedca40c26463f5cbbd6ee8a163a49cf07774bf Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 18 Oct 2018 22:27:28 +0300 Subject: [PATCH 13/28] fix asan --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 013bb09e4be..69ab1d04302 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -1800,7 +1800,8 @@ void ExpressionAnalyzer::collectUsedColumns() for (const auto & left_key_ast : analyzed_join.key_asts_left) { - RequiredSourceColumnsVisitor columns_visitor(available_columns, required, ignored, {}, required_joined_columns); + NameSet empty; + RequiredSourceColumnsVisitor columns_visitor(available_columns, required, ignored, empty, required_joined_columns); columns_visitor.visit(left_key_ast); } From abfe6750975083525962c51a811574a45800cfab Mon Sep 17 00:00:00 2001 From: proller Date: Thu, 18 Oct 2018 22:28:05 +0300 Subject: [PATCH 14/28] Build fixes (#3419) --- CMakeLists.txt | 4 ++-- dbms/programs/odbc-bridge/IdentifierQuoteHandler.cpp | 6 +++--- dbms/programs/odbc-bridge/getIdentifierQuote.cpp | 6 +++--- dbms/programs/odbc-bridge/getIdentifierQuote.h | 2 +- dbms/src/AggregateFunctions/CMakeLists.txt | 2 +- dbms/src/Functions/CMakeLists.txt | 2 +- dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- dbms/src/Storages/System/CMakeLists.txt | 2 +- dbms/src/TableFunctions/CMakeLists.txt | 2 +- 9 files changed, 14 insertions(+), 14 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 4dbc414e079..8c954ddad93 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -147,7 +147,7 @@ set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${COMPILER_FLAGS} -fn set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -O3 ${CMAKE_C_FLAGS_ADD}") set (CMAKE_C_FLAGS_DEBUG "${CMAKE_C_FLAGS_DEBUG} -O0 -g3 -ggdb3 -fno-inline ${CMAKE_C_FLAGS_ADD}") -if (MAKE_STATIC_LIBRARIES AND NOT APPLE AND NOT (CMAKE_CXX_COMPILER_ID STREQUAL "Clang" AND OS_FREEBSD)) +if (MAKE_STATIC_LIBRARIES AND NOT APPLE AND NOT (COMPILER_CLANG AND OS_FREEBSD)) set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libgcc -static-libstdc++") # Along with executables, we also build example of shared library for "library dictionary source"; and it also should be self-contained. @@ -158,7 +158,7 @@ set(THREADS_PREFER_PTHREAD_FLAG ON) include (cmake/test_compiler.cmake) -if (OS_LINUX AND CMAKE_CXX_COMPILER_ID STREQUAL "Clang") +if (OS_LINUX AND COMPILER_CLANG) set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS}") option (USE_LIBCXX "Use libc++ and libc++abi instead of libstdc++ (only make sense on Linux with Clang)" ${HAVE_LIBCXX}) diff --git a/dbms/programs/odbc-bridge/IdentifierQuoteHandler.cpp b/dbms/programs/odbc-bridge/IdentifierQuoteHandler.cpp index 303165a4d45..77fd32d80b4 100644 --- a/dbms/programs/odbc-bridge/IdentifierQuoteHandler.cpp +++ b/dbms/programs/odbc-bridge/IdentifierQuoteHandler.cpp @@ -2,9 +2,9 @@ #if USE_POCO_SQLODBC || USE_POCO_DATAODBC #if USE_POCO_SQLODBC -#include -#include -#include +#include // Y_IGNORE +#include // Y_IGNORE +#include // Y_IGNORE #define POCO_SQL_ODBC_CLASS Poco::SQL::ODBC #endif #if USE_POCO_DATAODBC diff --git a/dbms/programs/odbc-bridge/getIdentifierQuote.cpp b/dbms/programs/odbc-bridge/getIdentifierQuote.cpp index 9ac0a56bfc6..055b211e99d 100644 --- a/dbms/programs/odbc-bridge/getIdentifierQuote.cpp +++ b/dbms/programs/odbc-bridge/getIdentifierQuote.cpp @@ -2,9 +2,9 @@ #if USE_POCO_SQLODBC || USE_POCO_DATAODBC #if USE_POCO_SQLODBC -#include -#include -#include +#include // Y_IGNORE +#include // Y_IGNORE +#include // Y_IGNORE #define POCO_SQL_ODBC_CLASS Poco::SQL::ODBC #endif #if USE_POCO_DATAODBC diff --git a/dbms/programs/odbc-bridge/getIdentifierQuote.h b/dbms/programs/odbc-bridge/getIdentifierQuote.h index 7ad271e7788..1a361dedea3 100644 --- a/dbms/programs/odbc-bridge/getIdentifierQuote.h +++ b/dbms/programs/odbc-bridge/getIdentifierQuote.h @@ -8,7 +8,7 @@ #if USE_POCO_SQLODBC || USE_POCO_DATAODBC #if USE_POCO_SQLODBC -#include +#include // Y_IGNORE #endif #if USE_POCO_DATAODBC #include diff --git a/dbms/src/AggregateFunctions/CMakeLists.txt b/dbms/src/AggregateFunctions/CMakeLists.txt index d81f3e668b5..ef2665d0d3f 100644 --- a/dbms/src/AggregateFunctions/CMakeLists.txt +++ b/dbms/src/AggregateFunctions/CMakeLists.txt @@ -19,6 +19,6 @@ list(REMOVE_ITEM clickhouse_aggregate_functions_headers FactoryHelpers.h ) -add_library(clickhouse_aggregate_functions ${clickhouse_aggregate_functions_sources}) +add_library(clickhouse_aggregate_functions ${LINK_MODE} ${clickhouse_aggregate_functions_sources}) target_link_libraries(clickhouse_aggregate_functions dbms) target_include_directories (clickhouse_aggregate_functions BEFORE PRIVATE ${COMMON_INCLUDE_DIR}) diff --git a/dbms/src/Functions/CMakeLists.txt b/dbms/src/Functions/CMakeLists.txt index 2e0c6615676..d57e3990901 100644 --- a/dbms/src/Functions/CMakeLists.txt +++ b/dbms/src/Functions/CMakeLists.txt @@ -50,7 +50,7 @@ add_headers_and_sources(clickhouse_functions ${FUNCTIONS_GENERATED_DIR}) list(REMOVE_ITEM clickhouse_functions_sources IFunction.cpp FunctionFactory.cpp FunctionHelpers.cpp) list(REMOVE_ITEM clickhouse_functions_headers IFunction.h FunctionFactory.h FunctionHelpers.h) -add_library(clickhouse_functions ${clickhouse_functions_sources}) +add_library(clickhouse_functions ${LINK_MODE} ${clickhouse_functions_sources}) target_link_libraries(clickhouse_functions PUBLIC dbms PRIVATE ${CONSISTENT_HASHING_LIBRARY} consistent-hashing-sumbur ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES} murmurhash) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 20042b528fd..50900021dec 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -651,7 +651,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( /// Let's estimate total number of rows for progress bar. const size_t total_rows = data.index_granularity * sum_marks; - LOG_TRACE(log, "Reading approx. " << total_rows << " rows"); + LOG_TRACE(log, "Reading approx. " << total_rows << " rows with " << num_streams << " streams"); for (size_t i = 0; i < num_streams; ++i) { diff --git a/dbms/src/Storages/System/CMakeLists.txt b/dbms/src/Storages/System/CMakeLists.txt index a561ebe1583..907fbc2907a 100644 --- a/dbms/src/Storages/System/CMakeLists.txt +++ b/dbms/src/Storages/System/CMakeLists.txt @@ -1,5 +1,5 @@ include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) add_headers_and_sources(storages_system .) -add_library(clickhouse_storages_system ${storages_system_headers} ${storages_system_sources}) +add_library(clickhouse_storages_system ${LINK_MODE} ${storages_system_headers} ${storages_system_sources}) target_link_libraries(clickhouse_storages_system dbms) diff --git a/dbms/src/TableFunctions/CMakeLists.txt b/dbms/src/TableFunctions/CMakeLists.txt index 08112e0b6cc..3eea07fd8e6 100644 --- a/dbms/src/TableFunctions/CMakeLists.txt +++ b/dbms/src/TableFunctions/CMakeLists.txt @@ -4,5 +4,5 @@ add_headers_and_sources(clickhouse_table_functions .) list(REMOVE_ITEM clickhouse_table_functions_sources ITableFunction.cpp TableFunctionFactory.cpp) list(REMOVE_ITEM clickhouse_table_functions_headers ITableFunction.h TableFunctionFactory.h) -add_library(clickhouse_table_functions ${clickhouse_table_functions_sources}) +add_library(clickhouse_table_functions ${LINK_MODE} ${clickhouse_table_functions_sources}) target_link_libraries(clickhouse_table_functions clickhouse_storages_system dbms ${Poco_Foundation_LIBRARY}) From 9aa6c7053f71df9d393d67153a094b0117f60d90 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B0=A2=E7=A3=8A?= <2217232293@qq.com> Date: Fri, 19 Oct 2018 16:02:26 +0800 Subject: [PATCH 15/28] finish dataTypes zh translate (#3417) * add dataTypes zh translate --> init docs * add dataTypes zh translate * finish dataTypes zh translate * fix punctuation in dataTypes zh translate --- docs/zh/data_types/array.md | 84 ++++++++++++- docs/zh/data_types/boolean.md | 6 +- docs/zh/data_types/date.md | 7 +- docs/zh/data_types/datetime.md | 17 ++- docs/zh/data_types/decimal.md | 43 +++---- docs/zh/data_types/enum.md | 114 +++++++++++++++--- docs/zh/data_types/fixedstring.md | 8 +- docs/zh/data_types/float.md | 88 +++++++------- docs/zh/data_types/index.md | 4 +- docs/zh/data_types/int_uint.md | 3 - .../aggregatefunction.md | 2 - .../nested_data_structures/index.md | 5 +- .../nested_data_structures/nested.md | 8 +- docs/zh/data_types/nullable.md | 16 ++- .../special_data_types/expression.md | 4 +- .../zh/data_types/special_data_types/index.md | 4 +- .../data_types/special_data_types/nothing.md | 9 +- docs/zh/data_types/special_data_types/set.md | 3 +- docs/zh/data_types/string.md | 8 +- docs/zh/data_types/tuple.md | 18 +-- 20 files changed, 288 insertions(+), 163 deletions(-) diff --git a/docs/zh/data_types/array.md b/docs/zh/data_types/array.md index 155d32500da..0dc89baf3c7 100644 --- a/docs/zh/data_types/array.md +++ b/docs/zh/data_types/array.md @@ -1,8 +1,84 @@ -# Array(T) +# Array(T) -一个包含类型 T 的 数组(Array)。T 类型可以是任意类型,包括数组类型。 -我们不推荐使用多维数组,因为多维数组并没有得到很好支持(比如,不能在 MergeTree 引擎的表中存储多维数组)。 +由 `T` 类型元素组成的数组。 -[来源文章](https://clickhouse.yandex/docs/en/data_types/array/) \ No newline at end of file +`T` 可以是任意类型,包含数组类型。不推荐使用多维数组,ClickHouse 对多维数组的支持有限。例如,不能存储在 `MergeTree` 表中存储多维数组。 + +## 创建数组 + +您可以使用一个函数创建数组: + +``` +array(T) +``` + +您也可以使用方括号: + +``` +[] +``` + +创建数组示例: + +``` +:) SELECT array(1, 2) AS x, toTypeName(x) + +SELECT + [1, 2] AS x, + toTypeName(x) + +┌─x─────┬─toTypeName(array(1, 2))─┐ +│ [1,2] │ Array(UInt8) │ +└───────┴─────────────────────────┘ + +1 rows in set. Elapsed: 0.002 sec. + +:) SELECT [1, 2] AS x, toTypeName(x) + +SELECT + [1, 2] AS x, + toTypeName(x) + +┌─x─────┬─toTypeName([1, 2])─┐ +│ [1,2] │ Array(UInt8) │ +└───────┴────────────────────┘ + +1 rows in set. Elapsed: 0.002 sec. +``` + +## 使用数据类型 + +当动态创建数组时,ClickHouse 自动将参数类型定义为可以存储所有列出的参数的最窄的数据类型。如果存在任何 [NULL](../query_language/syntax.md#null-literal) 或者 [Nullable](nullable.md#data_type-nullable) 类型参数,那么数组元素的类型是 [Nullable](nullable.md#data_type-nullable)。 + +如果 ClickHouse 无法确定数据类型,它将产生异常。当尝试同时创建一个包含字符串和数字的数组时会发生这种情况 (`SELECT array(1, 'a')`)。 + +自动数据类型检测示例: + +``` +:) SELECT array(1, 2, NULL) AS x, toTypeName(x) + +SELECT + [1, 2, NULL] AS x, + toTypeName(x) + +┌─x──────────┬─toTypeName(array(1, 2, NULL))─┐ +│ [1,2,NULL] │ Array(Nullable(UInt8)) │ +└────────────┴───────────────────────────────┘ + +1 rows in set. Elapsed: 0.002 sec. +``` + +如果您尝试创建不兼容的数据类型数组,ClickHouse 将引发异常: + +``` +:) SELECT array(1, 'a') + +SELECT [1, 'a'] + +Received exception from server (version 1.1.54388): +Code: 386. DB::Exception: Received from localhost:9000, 127.0.0.1. DB::Exception: There is no supertype for types UInt8, String because some of them are String/FixedString and some of them are not. + +0 rows in set. Elapsed: 0.246 sec. +``` diff --git a/docs/zh/data_types/boolean.md b/docs/zh/data_types/boolean.md index 26485028358..3a71691cdf4 100644 --- a/docs/zh/data_types/boolean.md +++ b/docs/zh/data_types/boolean.md @@ -1,5 +1,3 @@ -# Boolean values +# Boolean Values -没有单独的类型来存储 boolean 值。可以使用 UInt8 类型,取值限制为 0 或 1。 - -[来源文章](https://clickhouse.yandex/docs/en/data_types/boolean/) +没有单独的类型来存储布尔值。可以使用 UInt8 类型,取值限制为 0 或 1。 diff --git a/docs/zh/data_types/date.md b/docs/zh/data_types/date.md index 39475a6558a..42fb124af9c 100644 --- a/docs/zh/data_types/date.md +++ b/docs/zh/data_types/date.md @@ -1,8 +1,5 @@ # Date -Date 用两个字节来存储从 1970-01-01 到现在的日期值(无符号的值)。Date 允许存储的值 UNIX 纪元开始后的时间值,这个值上限会在编译阶段作为一个常量存储(当前只能到 2038 年,但可以拓展到 2106 年)。 -Date 最小的值为 0000-00-00 00:00:00。 +日期类型,用两个字节存储,表示从 1970-01-01 (无符号) 到当前的日期值。允许存储从 Unix 纪元开始到编译阶段定义的上限阈值常量(目前上限是2106年,但最终完全支持的年份为2105)。最小值输出为0000-00-00。 -Date 中没有存储时区信息。 - -[来源文章](https://clickhouse.yandex/docs/en/data_types/date/) +日期中没有存储时区信息。 diff --git a/docs/zh/data_types/datetime.md b/docs/zh/data_types/datetime.md index 4cf8661e218..4a5a8095f0a 100644 --- a/docs/zh/data_types/datetime.md +++ b/docs/zh/data_types/datetime.md @@ -1,16 +1,13 @@ + + # DateTime -包含时间和日期。用4个字节来存储 Unix 时间戳(无符号的值)。允许存储的时间范围同 Date 类型。最小的值为 0000-00-00 00:00:00。 -DateTime 类型值精确到秒(不考虑闰秒) +时间戳类型。用四个字节存储 Unix 时间戳(无符号的)。允许存储与日期类型相同的范围内的值。最小值为 0000-00-00 00:00:00。时间戳类型值精确到秒(没有闰秒)。 -## Time zones +## 时区 -在客户端或服务器启动时,会使用系统的时区来将时间日期从文本(划分成多个部分)到二进制之间相互转化。在文本格式中,有关夏令时信息不作存储。 +使用启动客户端或服务器时的系统时区,时间戳是从文本(分解为组件)转换为二进制并返回。在文本格式中,有关夏令时的信息会丢失。 -默认情况下,客户端连接到服务的时候会使用服务端时区。可以通过客户端命令行参数 `--use_client_time_zone` 来设置使用客户端时区。 +默认情况下,客户端连接到服务的时候会使用服务端时区。您可以通过启用客户端命令行选项 `--use_client_time_zone` 来设置使用客户端时间。 -在所有的时区内,只支持部分与UTC相差整数小时(不考虑闰秒)的时区。 - -因此,在处理文本日期时(例如,在保存文本转储时),请记住在夏时制期间可能会出现歧义,如果时区发生变化,可能会出现与数据匹配的问题。 - -[来源文章](https://clickhouse.yandex/docs/en/data_types/datetime/) +因此,在处理文本日期时(例如,在保存文本转储时),请记住在夏令时更改期间可能存在歧义,如果时区发生更改,则可能存在匹配数据的问题。 diff --git a/docs/zh/data_types/decimal.md b/docs/zh/data_types/decimal.md index a18394409fd..373543c2c2e 100644 --- a/docs/zh/data_types/decimal.md +++ b/docs/zh/data_types/decimal.md @@ -2,17 +2,17 @@ # Decimal(P, S), Decimal32(S), Decimal64(S), Decimal128(S) -有符号的定点数,在进行加减乘操作的过程中都会保留精度。对于除法,最低有效数字被丢弃(不舍入)。 +有符号的定点数,可在加、减和乘法运算过程中保持精度。对于除法,最低有效数字会被丢弃(不舍入)。 ## 参数 -- P - 精度。范围值: [ 1 : 38 ],决定可以有多少个十进制数字(包括分数)。 -- S - 范围。范围值: [ 0 : P ],决定小数的位数。 +- P - 精度。有效范围:[1:38],决定可以有多少个十进制数字(包括分数)。 +- S - 规模。有效范围:[0:P],决定数字的小数部分中包含的小数位数。 对于不同的 P 参数值 Decimal 表示,以下例子都是同义的: -- [ 1 : 9 ] 的 P - for Decimal32(S) -- [ 10 : 18 ] 的 P - for Decimal64(S) -- [ 19 : 38 ] 的 P - for Decimal128(S) +- P from [ 1 : 9 ] - for Decimal32(S) +- P from [ 10 : 18 ] - for Decimal64(S) +- P from [ 19 : 38 ] - for Decimal128(S) ## 十进制值范围 @@ -20,38 +20,37 @@ - Decimal64(S) - ( -1 * 10^(18 - S), 1 * 10^(18 - S) ) - Decimal128(S) - ( -1 * 10^(38 - S), 1 * 10^(38 - S) ) -例如, Decimal32(4) 可以表示 -99999.9999 到 99999.9999 范围内步数为 0.0001 的值。 +例如,Decimal32(4) 可以表示 -99999.9999 至 99999.9999 的数值,步长为0.0001。 ## 内部表示方式 数据采用与自身位宽相同的有符号整数存储。这个数在内存中实际范围会高于上述范围,从 String 转换到十进制数的时候会做对应的检查。 -Decimal32/Decimal64 通常处理速度要高于Decimal128,这是因为当前通用CPU不支持128位的操作导致的。 +由于现代CPU不支持128位数字,因此 Decimal128 上的操作由软件模拟。所以 Decimal128 的运算速度明显慢于 Decimal32/Decimal64。 -## 运算以及结果的类型 +## 运算和结果类型 -对Decimal的二进制运算导致更宽的结果类型(具有任何参数顺序)。 +对Decimal的二进制运算导致更宽的结果类型(无论参数的顺序如何)。 - Decimal64(S1) Decimal32(S2) -> Decimal64(S) - Decimal128(S1) Decimal32(S2) -> Decimal128(S) - Decimal128(S1) Decimal64(S2) -> Decimal128(S) -精度变化的规则: +精度变化的规则: -- 加,减: S = max(S1, S2). -- 相乘: S = S1 + S2. -- 相除:S = S1. +- 加法,减法:S = max(S1, S2)。 +- 乘法:S = S1 + S2。 +- 除法:S = S1。 -对于 Decimal 和整数之间的类似操作,结果为一样参数值的 Decimal。 +对于 Decimal 和整数之间的类似操作,结果是与参数大小相同的十进制。 -没有定义 Decimal 和 Float32/Float64 的操作。如果你真的需要他们,你可以某一个参数明确地转换为 toDecimal32,toDecimal64, toDecimal128 或 toFloat32, toFloat64。注意这个操作会丢失精度,并且类型转换是一个代价昂贵的操作。 - -有一些函数对 Decimal 进行操作后是返回 Float64 的(例如,var 或 stddev)。计算的结果可能仍在 Decimal 中执行,这可能导致 Float64 和具有相同值的 Decimal 输入计算后的结果不同。 +未定义Decimal和Float32/Float64之间的函数。要执行此类操作,您可以使用:toDecimal32、toDecimal64、toDecimal128 或 toFloat32,toFloat64,需要显式地转换其中一个参数。注意,结果将失去精度,类型转换是昂贵的操作。 +Decimal上的一些函数返回结果为Float64(例如,var或stddev)。对于其中一些,中间计算发生在Decimal中。对于此类函数,尽管结果类型相同,但Float64和Decimal中相同数据的结果可能不同。 ## 溢出检查 -在对 Decimal 计算的过程中,数值会有可能溢出。分数中的过多数字被丢弃(不是舍入的)。 整数中的过多数字将导致异常。 +在对 Decimal 类型执行操作时,数值可能会发生溢出。分数中的过多数字被丢弃(不是舍入的)。整数中的过多数字将导致异常。 ``` SELECT toDecimal32(2, 4) AS x, x / 3 @@ -76,7 +75,7 @@ SELECT toDecimal32(4.2, 8) AS x, 6 * x DB::Exception: Decimal math overflow. ``` -溢出检查会导致操作减慢。 如果已知溢出不可能,则使用`decimal_check_overflow`设置禁用检查是有意义的。 禁用检查并发生溢出时,结果将不正确: +检查溢出会导致计算变慢。如果已知溢出不可能,则可以通过设置`decimal_check_overflow`来禁用溢出检查,在这种情况下,溢出将导致结果不正确: ``` SET decimal_check_overflow = 0; @@ -88,7 +87,7 @@ SELECT toDecimal32(4.2, 8) AS x, 6 * x └────────────┴──────────────────────────────────┘ ``` -溢出检查不仅会在数学运算中进行,还会在值比较中进行: +溢出检查不仅发生在算术运算上,还发生在比较运算上: ``` SELECT toDecimal32(1, 8) < 100 @@ -96,5 +95,3 @@ SELECT toDecimal32(1, 8) < 100 ``` DB::Exception: Can't compare. ``` - -[来源文章](https://clickhouse.yandex/docs/en/data_types/decimal/) diff --git a/docs/zh/data_types/enum.md b/docs/zh/data_types/enum.md index 48ab4e0d130..5a4347aeb9a 100644 --- a/docs/zh/data_types/enum.md +++ b/docs/zh/data_types/enum.md @@ -1,34 +1,116 @@ -# Enum + -`Enum8` 或者 `Enum16`。 一组有限的字符串值,比 `String` 类型的存储更加有效。 +# Enum8, Enum16 -示例: +包括 `Enum8` 和 `Enum16` 类型。`Enum` 保存 `'string'= integer` 的对应关系。在 ClickHouse 中,尽管用户使用的是字符串常量,但所有含有 `Enum` 数据类型的操作都是按照包含整数的值来执行。这在性能方面比使用 `String` 数据类型更有效。 -```text -Enum8('hello' = 1, 'world' = 2) +- `Enum8` 用 `'String'= Int8` 对描述。 +- `Enum16` 用 `'String'= Int16` 对描述。 + +## 用法示例 + +创建一个带有一个枚举 `Enum8('hello' = 1, 'world' = 2)` 类型的列: + +``` +CREATE TABLE t_enum +( + x Enum8('hello' = 1, 'world' = 2) +) +ENGINE = TinyLog ``` -- 一个类型可以表示两个值: 'hello' and 'world'。 +这个 `x` 列只能存储类型定义中列出的值:`'hello'`或`'world'`。如果您尝试保存任何其他值,ClickHouse 抛出异常。 + +``` +:) INSERT INTO t_enum VALUES ('hello'), ('world'), ('hello') + +INSERT INTO t_enum VALUES + +Ok. + +3 rows in set. Elapsed: 0.002 sec. + +:) insert into t_enum values('a') + +INSERT INTO t_enum VALUES + + +Exception on client: +Code: 49. DB::Exception: Unknown element 'a' for type Enum8('hello' = 1, 'world' = 2) +``` + +当您从表中查询数据时,ClickHouse 从 `Enum` 中输出字符串值。 + +``` +SELECT * FROM t_enum + +┌─x─────┐ +│ hello │ +│ world │ +│ hello │ +└───────┘ +``` + +如果需要看到对应行的数值,则必须将 `Enum` 值转换为整数类型。 + +``` +SELECT CAST(x, 'Int8') FROM t_enum + +┌─CAST(x, 'Int8')─┐ +│ 1 │ +│ 2 │ +│ 1 │ +└─────────────────┘ +``` + +在查询中创建枚举值,您还需要使用 `CAST`。 + +``` +SELECT toTypeName(CAST('a', 'Enum8(\'a\' = 1, \'b\' = 2)')) + +┌─toTypeName(CAST('a', 'Enum8(\'a\' = 1, \'b\' = 2)'))─┐ +│ Enum8('a' = 1, 'b' = 2) │ +└──────────────────────────────────────────────────────┘ +``` + +## 规则及用法 `Enum8` 类型的每个值范围是 `-128 ... 127`,`Enum16` 类型的每个值范围是 `-32768 ... 32767`。所有的字符串或者数字都必须是不一样的。允许存在空字符串。如果某个 Enum 类型被指定了(在表定义的时候),数字可以是任意顺序。然而,顺序并不重要。 -(译者注:如 `Enum8('he o' = 3, 'wld' = 1)` 也是合法的) -在内存中,Enum 类型当做 `Int8` or `Int16` 对应的数值来存储。 -当以文本方式读取的时候,ClickHouse 将值解析成字符串然后去 Enum 值的集合中搜索对应字符串。如果没有找到,会抛出异常。当读取文本格式的时候,会根据读取到的字符串去找对应的数值。如果没有找到,会抛出异常。 +`Enum` 中的字符串和数值都不能是 [NULL](../query_language/syntax.md#null-literal)。 -当以文本方式写入的时候,ClickHouse 将值解析成字符串写入。如果 column 数据包含垃圾(不是从有用集合含有的数值),会抛弃异常。Enum 类型以二进制读取和写入的方式与 Int8 和 Int16 类型一样的。 -隐式默认值是对应类型的最小值。 +`Enum` 包含在 [Nullable](nullable.md#data_type-nullable) 类型中。因此,如果您使用此查询创建一个表 -在 `ORDER BY`, `GROUP BY`, `IN`, `DISTINCT` 中,Enums 和对应数值是一样的工作方式。比如, ORDER BY 会将它们按数值排序。对 Enums 类型使用相同和比较操作符都与操作它们隐含的数值是一样的。 +``` +CREATE TABLE t_enum_nullable +( + x Nullable( Enum8('hello' = 1, 'world' = 2) ) +) +ENGINE = TinyLog +``` -Enum 值不能和数值比较大小。Enums 可以和一个常量字符串比较大小。如果字符串不是一个可用的 Enum 值,会抛出异常。可以使用 IN 运算符来判断一个 Enum 是否存在于某个 Enum 集合中,其中集合中的 Enum 需要用字符串表示。 +不仅可以存储 `'hello'` 和 `'world'` ,还可以存储 `NULL`。 -大部分数字运算和字符串运算都没有给 Enum 类型定义,比如,Enum 类型不能和一个数相加,或进行字符串连接的操作,但是可以通过 toString 方法返回它对应的字符串。 +``` +INSERT INTO t_enum_null Values('hello'),('world'),(NULL) +``` + +在内存中,`Enum` 列的存储方式与相应数值的 `Int8` 或 `Int16` 相同。 + +当以文本方式读取的时候,ClickHouse 将值解析成字符串然后去枚举值的集合中搜索对应字符串。如果没有找到,会抛出异常。当读取文本格式的时候,会根据读取到的字符串去找对应的数值。如果没有找到,会抛出异常。 + +当以文本形式写入时,ClickHouse 将值解析成字符串写入。如果列数据包含垃圾数据(不是来自有效集合的数字),则抛出异常。Enum 类型以二进制读取和写入的方式与 `Int8` 和 `Int16` 类型一样的。 + +隐式默认值是数值最小的值。 + +在 `ORDER BY`,`GROUP BY`,`IN`,`DISTINCT` 等等中,Enum 的行为与相应的数字相同。例如,按数字排序。对于等式运算符和比较运算符,Enum 的工作机制与它们在底层数值上的工作机制相同。 + +枚举值不能与数字进行比较。枚举可以与常量字符串进行比较。如果与之比较的字符串不是有效Enum值,则将引发异常。可以使用 IN 运算符来判断一个 Enum 是否存在于某个 Enum 集合中,其中集合中的 Enum 需要用字符串表示。 + +大多数具有数字和字符串的运算并不适用于Enums;例如,Enum 类型不能和一个数值相加。但是,Enum有一个原生的 `toString` 函数,它返回它的字符串值。 Enum 值使用 `toT` 函数可以转换成数值类型,其中 T 是一个数值类型。若 `T` 恰好对应 Enum 的底层数值类型,这个转换是零消耗的。 Enum 类型可以被 `ALTER` 无成本地修改对应集合的值。可以通过 `ALTER` 操作来增加或删除 Enum 的成员(只要表没有用到该值,删除都是安全的)。作为安全保障,改变之前使用过的 Enum 成员将抛出异常。 通过 `ALTER` 操作,可以将 `Enum8` 转成 `Enum16`,反之亦然,就像 `Int8` 转 `Int16`一样。 - -[来源文章](https://clickhouse.yandex/docs/en/data_types/enum/) diff --git a/docs/zh/data_types/fixedstring.md b/docs/zh/data_types/fixedstring.md index 1d8a4982e48..050f0a582a2 100644 --- a/docs/zh/data_types/fixedstring.md +++ b/docs/zh/data_types/fixedstring.md @@ -1,9 +1,9 @@ # FixedString(N) 固定长度 N 的字符串。N 必须是严格的正自然数。 -当服务端读取长度小于 N 的字符串时候(譬如解析插入的数据),字符串通过在末尾添加空字节来达到 N 字节长度。 -当服务端读取长度大于 N 的字符串时候,会返回一个错误。 -当服务端写入一个字符串的时候(譬如写入数据到 SELECT 查询结果中),末尾的空字节会被修剪掉。 +当服务端读取长度小于 N 的字符串时候(例如解析 INSERT 数据时),通过在字符串末尾添加空字节来达到 N 字节长度。 +当服务端读取长度大于 N 的字符串时候,将返回错误消息。 +当服务器写入一个字符串(例如,当输出 SELECT 查询的结果)时,NULL字节不会从字符串的末尾被移除,而是被输出。 注意这种方式与 MYSQL 的 CHAR 类型是不一样的(MYSQL 的字符串会以空格填充,然后输出的时候空格会被修剪)。 -很少函数会使用 `FixedString(N)` 来代替 `String`,因此它并不是很方便。 +与 `String` 类型相比,极少的函数会使用 `FixedString(N)`,因此使用起来不太方便。 diff --git a/docs/zh/data_types/float.md b/docs/zh/data_types/float.md index 6fb6b9eec40..f66cb698365 100644 --- a/docs/zh/data_types/float.md +++ b/docs/zh/data_types/float.md @@ -2,71 +2,71 @@ [浮点数](https://en.wikipedia.org/wiki/IEEE_754)。 -类型与以下 C 类型是相同的: +类型与以下 C 语言中类型是相同的: - `Float32` - `float` -- `Float64` - ` double` +- `Float64` - `double` -我们建议,如果可能的话尽量用整形来存储数据。比如,将一定精度的浮点数转换成整形,譬如货币金额或者毫秒单位的加载时间。 +我们建议您尽可能以整数形式存储数据。例如,将固定精度的数字转换为整数值,例如货币数量或页面加载时间用毫秒为单位表示 ## 使用浮点数 - 对浮点数进行计算可能引起四舍五入的误差。 - ```sql - SELECT 1 - 0.9 - ``` +```sql +SELECT 1 - 0.9 +``` - ``` - ┌───────minus(1, 0.9)─┐ - │ 0.09999999999999998 │ - └─────────────────────┘ - ``` +``` +┌───────minus(1, 0.9)─┐ +│ 0.09999999999999998 │ +└─────────────────────┘ +``` -- 计算的结果取决于计算方式(处理器类型和计算机系统架构) +- 计算的结果取决于计算方法(计算机系统的处理器类型和体系结构) -- 浮点数计算可能出现这样的结果,比如 "infinity" (`Inf`) 和 "not-a-number" (`NaN`)。对浮点数计算的时候应该考虑到这点。 +- 浮点计算结果可能是诸如无穷大(`INF`)和"非数字"(`NaN`)。对浮点数计算的时候应该考虑到这点。 - 当一行行阅读浮点数的时候,浮点数的结果可能不是机器最近显示的数值。 -## NaN 和 Inf +## NaN and Inf -相比于 SQL,ClickHouse 支持以下几种浮点数分类: +与标准SQL相比,ClickHouse 支持以下类别的浮点数: -- `Inf` – 正无穷。 +- `Inf` – 正无穷 - ```sql - SELECT 0.5 / 0 - ``` +```sql +SELECT 0.5 / 0 +``` - ``` - ┌─divide(0.5, 0)─┐ - │ inf │ - └────────────────┘ - ``` -- `-Inf` – 负无穷。 +``` +┌─divide(0.5, 0)─┐ +│ inf │ +└────────────────┘ +``` - ```sql - SELECT -0.5 / 0 - ``` +- `-Inf` – 负无穷 - ``` - ┌─divide(-0.5, 0)─┐ - │ -inf │ - └─────────────────┘ - ``` -- `NaN` – 非数字。 +```sql +SELECT -0.5 / 0 +``` - ``` - SELECT 0 / 0 - ``` +``` +┌─divide(-0.5, 0)─┐ +│ -inf │ +└─────────────────┘ +``` - ``` - ┌─divide(0, 0)─┐ - │ nan │ - └──────────────┘ - ``` +- `NaN` – 非数字 + +``` +SELECT 0 / 0 +``` + +``` +┌─divide(0, 0)─┐ +│ nan │ +└──────────────┘ +``` 可以在[ORDER BY 子句](../query_language/select.md#query_language-queries-order_by) 查看更多关于 ` NaN` 排序的规则。 - -[来源文章](https://clickhouse.yandex/docs/en/data_types/float/) diff --git a/docs/zh/data_types/index.md b/docs/zh/data_types/index.md index 785216a22a9..42fc574aa2e 100644 --- a/docs/zh/data_types/index.md +++ b/docs/zh/data_types/index.md @@ -4,6 +4,4 @@ ClickHouse 可以在数据表中存储多种数据类型。 -本节将描述 ClickHouse 支持的数据类型以及在使用 与/或 运算实现他们时候的特殊考虑(如果有)。 - -[来源文章](https://clickhouse.yandex/docs/en/data_types/) +本节描述 ClickHouse 支持的数据类型,以及使用或者实现它们时(如果有的话)的注意事项。 diff --git a/docs/zh/data_types/int_uint.md b/docs/zh/data_types/int_uint.md index c142cf50106..a74d11cbc20 100644 --- a/docs/zh/data_types/int_uint.md +++ b/docs/zh/data_types/int_uint.md @@ -17,6 +17,3 @@ - UInt16 - [0 : 65535] - UInt32 - [0 : 4294967295] - UInt64 - [0 : 18446744073709551615] - - -[来源文章](https://clickhouse.yandex/docs/en/data_types/int_uint/) diff --git a/docs/zh/data_types/nested_data_structures/aggregatefunction.md b/docs/zh/data_types/nested_data_structures/aggregatefunction.md index bb9d57274ec..fb453fb9a62 100644 --- a/docs/zh/data_types/nested_data_structures/aggregatefunction.md +++ b/docs/zh/data_types/nested_data_structures/aggregatefunction.md @@ -1,5 +1,3 @@ # AggregateFunction(name, types_of_arguments...) 表示聚合函数中的中间状态。可以在聚合函数中通过 '-State' 后缀来访问它。更多信息,参考 "AggregatingMergeTree"。 - -[来源文章](https://clickhouse.yandex/docs/en/data_types/nested_data_structures/aggregatefunction/) \ No newline at end of file diff --git a/docs/zh/data_types/nested_data_structures/index.md b/docs/zh/data_types/nested_data_structures/index.md index fcb01993389..5f8d9839943 100644 --- a/docs/zh/data_types/nested_data_structures/index.md +++ b/docs/zh/data_types/nested_data_structures/index.md @@ -1,4 +1 @@ -# 嵌套数据类型 - - -[来源文章](https://clickhouse.yandex/docs/en/data_types/nested_data_structures/) +# 嵌套数据结构 diff --git a/docs/zh/data_types/nested_data_structures/nested.md b/docs/zh/data_types/nested_data_structures/nested.md index 2ab6d06cbdc..e1a882ecfe1 100644 --- a/docs/zh/data_types/nested_data_structures/nested.md +++ b/docs/zh/data_types/nested_data_structures/nested.md @@ -1,6 +1,6 @@ # Nested(Name1 Type1, Name2 Type2, ...) -类似嵌套表的嵌套数据结构。嵌套数据结构的参数(列名和列类型)与 CREATE 查询类似。每个表可以对应任意多行嵌套数据结构。 +嵌套数据结构类似于嵌套表。嵌套数据结构的参数(列名和类型)与 CREATE 查询类似。每个表可以包含任意多行嵌套数据结构。 示例: @@ -88,12 +88,10 @@ LIMIT 10 └─────────┴─────────────────────┘ ``` -不能对整个嵌套数据结构进行 SELECT 。只能显式地列出它的一部分列。 +不能对整个嵌套数据结构执行 SELECT。只能明确列出属于它一部分列。 对于 INSERT 查询,可以单独地传入所有嵌套数据结构中的列数组(假如它们是单独的列数组)。在插入过程中,系统会检查它们是否有相同的长度。 对于 DESCRIBE 查询,嵌套数据结构中的列会以相同的方式分别列出来。 -ALTER 查询对嵌套数据结构的操作非常局限。 - -[来源文章](https://clickhouse.yandex/docs/en/data_types/nested_data_structures/nested/) +ALTER 查询对嵌套数据结构的操作非常有限。 diff --git a/docs/zh/data_types/nullable.md b/docs/zh/data_types/nullable.md index 00c24c26c3a..95db5ead679 100644 --- a/docs/zh/data_types/nullable.md +++ b/docs/zh/data_types/nullable.md @@ -2,10 +2,9 @@ # Nullable(TypeName) -允许加上一个符号 ([NULL](../query_language/syntax.md#null-literal)) 表示“缺失值”和 `TypeName` 允许的正常值。 例如,`Nullable(Int8)` 类型列可以存储 `Int8` 类型值,而没有值的行将存储 `NULL` 。 - -对于`TypeName`,您不能使用复合数据类型 [Array](array.md#data_type is array) 和 [Tuple](tuple.md#data_type-tuple)。 复合数据类型可以包含 `Nullable` 类型值,例如 `Array (Nullable(Int8))`。 +允许用特殊标记 ([NULL](../query_language/syntax.md#null-literal)) 表示"缺失值",可以与 `TypeName` 的正常值存放一起。例如,`Nullable(Int8)` 类型的列可以存储 `Int8` 类型值,而没有值的行将存储 `NULL`。 +对于 `TypeName`,不能使用复合数据类型 [Array](array.md#data_type is array) 和 [Tuple](tuple.md#data_type-tuple)。复合数据类型可以包含 `Nullable` 类型值,例如`Array(Nullable(Int8))`。 `Nullable` 类型字段不能包含在表索引中。 @@ -15,11 +14,12 @@ 要在表的列中存储 `Nullable` 类型值,ClickHouse 除了使用带有值的普通文件外,还使用带有 `NULL` 掩码的单独文件。 掩码文件中的条目允许 ClickHouse 区分每个表行的 `NULL` 和相应数据类型的默认值。 由于附加了新文件,`Nullable` 列与类似的普通文件相比消耗额外的存储空间。 -!!! info "Note" - 使用 `Nullable` 几乎总是会对性能产生负面影响,在设计数据库时请记住这一点。 - +!!! 注意点 + 使用 `Nullable` 几乎总是对性能产生负面影响,在设计数据库时请记住这一点 -## 使用案例 +掩码文件中的条目允许ClickHouse区分每个表行的对应数据类型的"NULL"和默认值由于有额外的文件,"Nullable"列比普通列消耗更多的存储空间 + +## 用法示例 ``` :) CREATE TABLE t_null(x Int8, y Nullable(Int8)) ENGINE TinyLog @@ -55,5 +55,3 @@ FROM t_null 2 rows in set. Elapsed: 0.144 sec. ``` - -[来源文章](https://clickhouse.yandex/docs/en/data_types/nullable/) diff --git a/docs/zh/data_types/special_data_types/expression.md b/docs/zh/data_types/special_data_types/expression.md index 14e8aaf5362..d135b2478ff 100644 --- a/docs/zh/data_types/special_data_types/expression.md +++ b/docs/zh/data_types/special_data_types/expression.md @@ -1,5 +1,3 @@ # Expression -用在高阶函数中的 lambda 表达式中。 - -[来源文章](https://clickhouse.yandex/docs/en/data_types/special_data_types/expression/) +用于表示高阶函数中的Lambd表达式。 \ No newline at end of file diff --git a/docs/zh/data_types/special_data_types/index.md b/docs/zh/data_types/special_data_types/index.md index 03f66e36189..efcd141e2e1 100644 --- a/docs/zh/data_types/special_data_types/index.md +++ b/docs/zh/data_types/special_data_types/index.md @@ -1,5 +1,3 @@ # Special Data Types -特殊数据类型的值既不能存在表中也不能在结果中输出,但是可以被用于表示查询过程中的中间结果。 - -[来源文章](https://clickhouse.yandex/docs/en/data_types/special_data_types/) +特殊数据类型的值既不能存在表中也不能在结果中输出,但可用于查询的中间结果。 diff --git a/docs/zh/data_types/special_data_types/nothing.md b/docs/zh/data_types/special_data_types/nothing.md index dd7f51cca0f..5f282df761e 100644 --- a/docs/zh/data_types/special_data_types/nothing.md +++ b/docs/zh/data_types/special_data_types/nothing.md @@ -2,11 +2,11 @@ # Nothing -此数据类型的唯一目的是表示不是期望值的情况。 所以你不能创建一个` Nothing` 类型的值。 +此数据类型的唯一目的是表示不是期望值的情况。 所以不能创建一个 `Nothing` 类型的值。 -例如, [NULL](../../query_language/syntax.md#null-literal) 有 `Nullable(Nothing)`。更多参见[Nullable](../../data_types/nullable.md#data_type-nullable). +例如,文本 [NULL](../../query_language/syntax.md#null-literal) 的类型为 `Nullable(Nothing)`。详情请见 [Nullable](../../data_types/nullable.md#data_type-nullable)。 -`Nothing` 类型也可用于表示空数组: +`Nothing` 类型也可以用来表示空数组: ```bash :) SELECT toTypeName(array()) @@ -19,6 +19,3 @@ SELECT toTypeName([]) 1 rows in set. Elapsed: 0.062 sec. ``` - - -[来源文章](https://clickhouse.yandex/docs/en/data_types/special_data_types/nothing/) diff --git a/docs/zh/data_types/special_data_types/set.md b/docs/zh/data_types/special_data_types/set.md index 0e62f64f540..5c8264b0aea 100644 --- a/docs/zh/data_types/special_data_types/set.md +++ b/docs/zh/data_types/special_data_types/set.md @@ -1,5 +1,4 @@ # Set -可以用在 IN 表达式的右半边。 +可以用在 IN 表达式的右半部分。 -[来源文章](https://clickhouse.yandex/docs/en/data_types/special_data_types/set/) diff --git a/docs/zh/data_types/string.md b/docs/zh/data_types/string.md index 6d45b8f4741..5d7374eb56f 100644 --- a/docs/zh/data_types/string.md +++ b/docs/zh/data_types/string.md @@ -1,13 +1,13 @@ + + # String -字符串可以任意长度的。它可以包含任意的字节组合,包括空字符。 -String 类型替代了其他 DBMSs 中的 VARCHAR, BLOB, CLOB 等类型。 +字符串可以任意长度的。它可以包含任意的字节集,包含空字节。因此,字符串类型可以代替其他 DBMSs 中的 VARCHAR、BLOB、CLOB 等类型。 ## 编码 -ClickHouse 没有编码的概念。字符串可以是任意的字节组合,按它们原本的方式进行存储和输出。 +ClickHouse 没有编码的概念。字符串可以是任意的字节集,按它们原本的方式进行存储和输出。 若需存储文本,我们建议使用 UTF-8 编码。至少,如果你的终端使用UTF-8(推荐),这样读写就不需要进行任何的转换了。 同样,对不同的编码文本 ClickHouse 会有不同处理字符串的函数。 比如,`length` 函数可以计算字符串包含的字节数组的长度,然而 `lengthUTF8` 函数是假设字符串以 UTF-8 编码,计算的是字符串包含的 Unicode 字符的长度。 -[来源文章](https://clickhouse.yandex/docs/en/data_types/string/) diff --git a/docs/zh/data_types/tuple.md b/docs/zh/data_types/tuple.md index 02320a4be73..0eb3b9a9259 100644 --- a/docs/zh/data_types/tuple.md +++ b/docs/zh/data_types/tuple.md @@ -1,12 +1,16 @@ + + # Tuple(T1, T2, ...) -元组,元组中的每个元素都有独立的[类型](index.md#data_types)。 +元组,其中每个元素都有单独的 [类型](index.md#data_types)。 -元组不能写入表中(除 Memory 表外)。它们会用于临时列分组。在某个查询中,IN 表达式和带特定参数的 lambda 函数可以来对临时列进行分组。更多信息,参见 [IN 操作符] (../query_language/select.md#in_operators) 和 [高阶函数](../query_language/functions/higher_order_functions.md#higher_order_functions)。 +不能在表中存储元组(除了内存表)。它们可以用于临时列分组。在查询中,IN 表达式和带特定参数的 lambda 函数可以来对临时列进行分组。更多信息,请参阅 [IN 操作符](../query_language/select.md#in_operators) and [Higher order functions](../query_language/functions/higher_order_functions.md#higher_order_functions)。 + +元组可以是查询的结果。在这种情况下,对于JSON以外的文本格式,括号中的值是逗号分隔的。在JSON格式中,元组作为数组输出(在方括号中)。 ## 创建元组 -可以用下面函数来创建元组: +可以使用函数来创建元组: ``` tuple(T1, T2, ...) @@ -30,9 +34,9 @@ SELECT ## 元组中的数据类型 -当动态创建元组时,ClickHouse 会自动为元组的每一个参数赋予最小可表达的类型。如果该参数是 [NULL](../query_language/syntax.md#null-literal),那么这个元组对应元素就是 [Nullable](nullable.md#data_type-nullable)。 +在动态创建元组时,ClickHouse 会自动为元组的每一个参数赋予最小可表达的类型。如果参数为 [NULL](../query_language/syntax.md#null-literal),那这个元组对应元素是 [Nullable](nullable.md#data_type-nullable)。 -自动识别数据类型的示例: +自动数据类型检测示例: ``` SELECT tuple(1, NULL) AS x, toTypeName(x) @@ -47,7 +51,3 @@ SELECT 1 rows in set. Elapsed: 0.002 sec. ``` - - -[来源文章](https://clickhouse.yandex/docs/en/data_types/tuple/) - From 035a00589d198747ed92534f1e7954a83694aa2a Mon Sep 17 00:00:00 2001 From: Bolinov <41979522+Bolinov@users.noreply.github.com> Date: Fri, 19 Oct 2018 16:03:27 +0800 Subject: [PATCH 16/28] finish create translate (#3408) * commit create section translation commit create section translation * finish create section translation finish create section translation * add insert into translate * finish insert into translate finish insert into translate * Update create.md --- docs/zh/query_language/create.md | 157 +++++++++++++++++++++++++- docs/zh/query_language/insert_into.md | 69 ++++++++++- 2 files changed, 224 insertions(+), 2 deletions(-) diff --git a/docs/zh/query_language/create.md b/docs/zh/query_language/create.md index a13304d176e..2271e2dbe68 120000 --- a/docs/zh/query_language/create.md +++ b/docs/zh/query_language/create.md @@ -1 +1,156 @@ -../../en/query_language/create.md \ No newline at end of file +## 创建数据库 + +创建 'db_name' 数据库. + +```sql +CREATE DATABASE [IF NOT EXISTS] db_name +``` + +一个数据库是表的一个目录. 如果包含'IF NOT EXISTS', 如果数据库已经存在,则查询不返回错误. + + + +## 创建表 + +'CREATE TABLE' 语句有几种形式. + +```sql +CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE = engine +``` + +如果'db'没有设置, 在数据库'db'中或者当前数据库中, 创建一个表名为'name'的表, 在括号和'engine' 引擎中指定结构. 表的结构是一个列描述的列表. 如果引擎支持索引, 则他们将是表引擎的参数. + +表结构是一个列描述的列表. 如果引擎支持索引, 他们以表引擎的参数表示. + +在最简单的情况, 一个列描述是'命名类型'. 例如: RegionID UInt32. 对于默认值, 表达式也能够被定义. + +```sql +CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name AS [db2.]name2 [ENGINE = engine] +``` + +创建一个表, 其结构与另一个表相同. 你能够为此表指定一个不同的引擎. 如果引擎没有被指定, 相同的引擎将被用于'db2.name2'表上. + +```sql +CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name ENGINE = engine AS SELECT ... +``` + +创建一个表,其结构类似于 SELECT 查询后的结果, 带有'engine' 引擎, 从 SELECT查询数据填充它. + +在所有情况下,如果'IF NOT EXISTS'被指定, 如果表已经存在, 查询并不返回一个错误. 在这种情况下, 查询并不做任何事情. + +### 默认值 + +列描述能够为默认值指定一个表达式, 其中一个方法是:DEFAULT expr, MATERIALIZED expr, ALIAS expr. +例如: URLDomain String DEFAULT domain(URL). + +如果默认值的一个表达式没有定义, 如果字段是数字类型, 默认值是将设置为0, 如果是字符类型, 则设置为空字符串, 日期类型则设置为 0000-00-00 或者 0000-00-00 00:00:00(时间戳). NULLs 则不支持. + +如果默认表达式被定义, 字段类型是可选的. 如果没有明确的定义类型, 则将使用默认表达式. 例如: EventDate DEFAULT toDate(EventTime) – 'Date' 类型将用于 'EventDate' 字段. + +如果数据类型和默认表达式被明确定义, 此表达式将使用函数被转换为特定的类型. 例如: Hits UInt32 DEFAULT 0 与 Hits UInt32 DEFAULT toUInt32(0)是等价的. + +默认表达是可能被定义为一个任意的表达式,如表的常量和字段. 当创建和更改表结构时, 它将检查表达式是否包含循环. 对于 INSERT操作来说, 它将检查表达式是否可解析 – 所有的字段通过传参后进行计算. + +`DEFAULT expr` + +正常的默认值. 如果 INSERT 查询并没有指定对应的字段, 它将通过计算对应的表达式来填充. + +`物化表达式` + +物化表达式. 此类型字段并没有指定插入操作, 因为它经常执行计算任务. 对一个插入操作, 无字段列表, 那么这些字段将不考虑. 另外, 当在一个SELECT查询语句中使用星号时, 此字段并不被替换. 这将保证INSERT INTO SELECT * FROM 的不可变性. + +`别名表达式` + +别名. 此字段不存储在表中. +此列的值不插入到表中, 当在一个SELECT查询语句中使用星号时,此字段并不被替换. +它能够用在 SELECTs中,如果别名在查询解析时被扩展. + +当使用更新查询添加一个新的字段, 这些列的旧值不被写入. 相反, 新字段没有值,当读取旧值时, 表达式将被计算. 然而,如果运行表达式需要不同的字段, 这些字段将被读取 , 但是仅读取相关的数据块. + +如果你添加一个新的字段到表中, 然后改变它的默认表达式, 对于使用的旧值将更改(对于此数据, 值不保存在磁盘上). 当运行背景线程时, 缺少合并数据块的字段数据写入到合并数据块中. + +在嵌套数据结构中设置默认值是不允许的. + + + +### 临时表 + +在任何情况下, 如果临时表被指定, 一个临时表将被创建. 临时表有如下的特性: + +- 当会话结束后, 临时表将删除,或者连接丢失. +- 一个临时表使用内存表引擎创建. 其他的表引擎不支持临时表. +- 数据库不能为一个临时表指定. 它将创建在数据库之外. +- 如果一个临时表与另外的表有相同的名称 ,一个查询指定了表名并没有指定数据库, 将使用临时表. +- 对于分布式查询处理, 查询中的临时表将被传递给远程服务器. + +在大多数情况下, 临时表并不能手工创建, 但当查询外部数据或使用分布式全局(GLOBAL)IN时,可以创建临时表. + +分布式 DDL 查询 (ON CLUSTER clause) +---------------------------------------------- + +'CREATE', 'DROP', 'ALTER', 和 'RENAME' 查询支持在集群上分布式执行. 例如, 如下的查询在集群中的每个机器节点上创建了 all_hits Distributed 表: + +```sql +CREATE TABLE IF NOT EXISTS all_hits ON CLUSTER cluster (p Date, i Int32) ENGINE = Distributed(cluster, default, hits) +``` + +为了正确执行这些语句,每个节点必须有相同的集群设置(为了简化同步配置,可以使用 zookeeper 来替换). 这些节点也可以连接到ZooKeeper 服务器. +查询语句会在每个节点上执行, 而'ALTER'查询目前暂不支持在同步表(replicated table)上执行. + + + +## CREATE VIEW + +```sql +CREATE [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]name [TO[db.]name] [ENGINE = engine] [POPULATE] AS SELECT ... +``` + +创建一个视图. 有两种类型的视图: 正常视图和物化(MATERIALIZED)视图. + +当创建一个物化视图时, 你必须指定表引擎 – 此表引擎用于存储数据 + +一个物化视图工作流程如下所示: 当插入数据到SELECT 查询指定的表中时, 插入数据部分通过SELECT查询部分来转换, 结果插入到视图中. + +正常视图不保存任何数据, 但是可以从任意表中读取数据. 换句话说,正常视图可以看作是查询结果的一个结果缓存. 当从一个视图中读取数据时, 此查询可以看做是 FROM语句的子查询. + +例如, 假设你已经创建了一个视图: + +```sql +CREATE VIEW view AS SELECT ... +``` + +写了一个查询语句: + +```sql +SELECT a, b, c FROM view +``` +此查询完全等价于子查询: + +```sql +SELECT a, b, c FROM (SELECT ...) +``` + +物化视图保存由SELECT语句查询转换的数据. + +当创建一个物化视图时,你必须指定一个引擎 – 存储数据的目标引擎. + +一个物化视图使用流程如下: 当插入数据到 SELECT 指定的表时, 插入数据部分通过SELECT 来转换, 同时结果被插入到视图中. + + +如果你指定了 POPULATE, 当创建时, 现有的表数据被插入到了视图中, 类似于 CREATE TABLE ... AS SELECT ... . 否则, 在创建视图之后,查询仅包含表中插入的数据. 我们不建议使用 POPULATE, 在视图创建过程中,插入到表中的数据不插入到其中. + +一个'SELECT'查询可以包含 'DISTINCT', 'GROUP BY', 'ORDER BY', 'LIMIT'... 对应的转换在每个数据块上独立执行. 例如, 如果 GROUP BY 被设置, 数据将在插入过程中进行聚合, 但仅是在一个插入数据包中.数据不再进一步聚合. 当使用一个引擎时, 如SummingMergeTree,它将独立执行数据聚合. + +视图看起来和正常表相同. 例如, 你可以使用 SHOW TABLES来列出视图表的相关信息. + +物化视图的'ALTER'查询执行还没有完全开发出来, 因此使用上可能不方便. 如果物化视图使用 ``TO [db.]name``, 你能够 ``DETACH`` 视图, 在目标表运行 ``ALTER``, 然后 ``ATTACH`` 之前的 ``DETACH``视图. + +视图看起来和正常表相同. 例如, 你可以使用 `SHOW TABLES` 来列出视图表的相关信息. + +因此并没有一个单独的SQL语句来删除视图. 为了删除一个视图, 可以使用 `DROP TABLE`. + diff --git a/docs/zh/query_language/insert_into.md b/docs/zh/query_language/insert_into.md index 29b47662b0d..517af5aa33b 120000 --- a/docs/zh/query_language/insert_into.md +++ b/docs/zh/query_language/insert_into.md @@ -1 +1,68 @@ -../../en/query_language/insert_into.md \ No newline at end of file + + +## INSERT + +正在添加数据. + +基本查询格式: + +```sql +INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... +``` + +此查询能够指定字段的列表来插入 `[(c1, c2, c3)]`. 在这种情况下, 剩下的字段用如下来填充: + +- 从表定义中指定的 `DEFAULT` 表达式中计算出值. +- 空字符串, 如果 `DEFAULT` 表达式没有定义. + +如果 [strict_insert_defaults=1](../operations/settings/settings.md#settings-strict_insert_defaults), 没有 `DEFAULT` 定义的字段必须在查询中列出. + +在任何ClickHouse所支持的格式上 [format](../interfaces/formats.md#formats) 数据被传入到 INSERT中. 此格式必须被显式地指定在查询中: + +```sql +INSERT INTO [db.]table [(c1, c2, c3)] FORMAT format_name data_set +``` + +例如, 如下的查询格式与基本的 INSERT ... VALUES 版本相同: + +```sql +INSERT INTO [db.]table [(c1, c2, c3)] FORMAT Values (v11, v12, v13), (v21, v22, v23), ... +``` + +ClickHouse 在数据之前, 删除所有空格和换行(如果有). 当形成一个查询时, 我们推荐在查询操作符之后将数据放入新行(如果数据以空格开始, 这是重要的). + +示例: + +```sql +INSERT INTO t FORMAT TabSeparated +11 Hello, world! +22 Qwerty +``` + +你能够单独从查询中插入数据,通过命令行或 HTTP 接口. 进一步信息, 参见 "[Interfaces](../interfaces/index.md#interfaces)". + +### Inserting The Results of `SELECT` + +```sql +INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... +``` + +在 SELECT语句中, 根据字段的位置来映射. 然而, 在SELECT表达式中的名称和表名可能不同. 如果必要, 可以进行类型转换. + +除了值以外没有其他数据类型允许设置值到表达式中, 例如 `now()`, `1 + 2`, 等. 值格式允许使用有限制的表达式, 但是它并不推荐, 因为在这种情况下, 执行了低效的代码. + +不支持修改数据分区的查询如下: `UPDATE`, `DELETE`, `REPLACE`, `MERGE`, `UPSERT`, `INSERT UPDATE`. +然而, 你能够使用 `ALTER TABLE ... DROP PARTITION`来删除旧数据. + +### Performance Considerations + +`INSERT` 通过主键来排序数据, 并通过月份来拆分数据到每个分区中. 如果插入的数据有混合的月份, 会显著降低`INSERT` 插入的性能. 应该避免此类操作: + +- 大批量地添加数据, 如每次 100,000 行. +- 在上传数据之前, 通过月份分组数据. + +下面操作性能不会下降: + +- 数据实时插入. +- 上传的数据通过时间来排序. + From 7449c2555672785204229e034297184dd71722df Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 19 Oct 2018 11:09:57 +0300 Subject: [PATCH 17/28] Revert "finish create translate (#3408)" (#3422) This reverts commit 035a00589d198747ed92534f1e7954a83694aa2a. --- docs/zh/query_language/create.md | 157 +------------------------- docs/zh/query_language/insert_into.md | 69 +---------- 2 files changed, 2 insertions(+), 224 deletions(-) diff --git a/docs/zh/query_language/create.md b/docs/zh/query_language/create.md index 2271e2dbe68..a13304d176e 120000 --- a/docs/zh/query_language/create.md +++ b/docs/zh/query_language/create.md @@ -1,156 +1 @@ -## 创建数据库 - -创建 'db_name' 数据库. - -```sql -CREATE DATABASE [IF NOT EXISTS] db_name -``` - -一个数据库是表的一个目录. 如果包含'IF NOT EXISTS', 如果数据库已经存在,则查询不返回错误. - - - -## 创建表 - -'CREATE TABLE' 语句有几种形式. - -```sql -CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], - ... -) ENGINE = engine -``` - -如果'db'没有设置, 在数据库'db'中或者当前数据库中, 创建一个表名为'name'的表, 在括号和'engine' 引擎中指定结构. 表的结构是一个列描述的列表. 如果引擎支持索引, 则他们将是表引擎的参数. - -表结构是一个列描述的列表. 如果引擎支持索引, 他们以表引擎的参数表示. - -在最简单的情况, 一个列描述是'命名类型'. 例如: RegionID UInt32. 对于默认值, 表达式也能够被定义. - -```sql -CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name AS [db2.]name2 [ENGINE = engine] -``` - -创建一个表, 其结构与另一个表相同. 你能够为此表指定一个不同的引擎. 如果引擎没有被指定, 相同的引擎将被用于'db2.name2'表上. - -```sql -CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name ENGINE = engine AS SELECT ... -``` - -创建一个表,其结构类似于 SELECT 查询后的结果, 带有'engine' 引擎, 从 SELECT查询数据填充它. - -在所有情况下,如果'IF NOT EXISTS'被指定, 如果表已经存在, 查询并不返回一个错误. 在这种情况下, 查询并不做任何事情. - -### 默认值 - -列描述能够为默认值指定一个表达式, 其中一个方法是:DEFAULT expr, MATERIALIZED expr, ALIAS expr. -例如: URLDomain String DEFAULT domain(URL). - -如果默认值的一个表达式没有定义, 如果字段是数字类型, 默认值是将设置为0, 如果是字符类型, 则设置为空字符串, 日期类型则设置为 0000-00-00 或者 0000-00-00 00:00:00(时间戳). NULLs 则不支持. - -如果默认表达式被定义, 字段类型是可选的. 如果没有明确的定义类型, 则将使用默认表达式. 例如: EventDate DEFAULT toDate(EventTime) – 'Date' 类型将用于 'EventDate' 字段. - -如果数据类型和默认表达式被明确定义, 此表达式将使用函数被转换为特定的类型. 例如: Hits UInt32 DEFAULT 0 与 Hits UInt32 DEFAULT toUInt32(0)是等价的. - -默认表达是可能被定义为一个任意的表达式,如表的常量和字段. 当创建和更改表结构时, 它将检查表达式是否包含循环. 对于 INSERT操作来说, 它将检查表达式是否可解析 – 所有的字段通过传参后进行计算. - -`DEFAULT expr` - -正常的默认值. 如果 INSERT 查询并没有指定对应的字段, 它将通过计算对应的表达式来填充. - -`物化表达式` - -物化表达式. 此类型字段并没有指定插入操作, 因为它经常执行计算任务. 对一个插入操作, 无字段列表, 那么这些字段将不考虑. 另外, 当在一个SELECT查询语句中使用星号时, 此字段并不被替换. 这将保证INSERT INTO SELECT * FROM 的不可变性. - -`别名表达式` - -别名. 此字段不存储在表中. -此列的值不插入到表中, 当在一个SELECT查询语句中使用星号时,此字段并不被替换. -它能够用在 SELECTs中,如果别名在查询解析时被扩展. - -当使用更新查询添加一个新的字段, 这些列的旧值不被写入. 相反, 新字段没有值,当读取旧值时, 表达式将被计算. 然而,如果运行表达式需要不同的字段, 这些字段将被读取 , 但是仅读取相关的数据块. - -如果你添加一个新的字段到表中, 然后改变它的默认表达式, 对于使用的旧值将更改(对于此数据, 值不保存在磁盘上). 当运行背景线程时, 缺少合并数据块的字段数据写入到合并数据块中. - -在嵌套数据结构中设置默认值是不允许的. - - - -### 临时表 - -在任何情况下, 如果临时表被指定, 一个临时表将被创建. 临时表有如下的特性: - -- 当会话结束后, 临时表将删除,或者连接丢失. -- 一个临时表使用内存表引擎创建. 其他的表引擎不支持临时表. -- 数据库不能为一个临时表指定. 它将创建在数据库之外. -- 如果一个临时表与另外的表有相同的名称 ,一个查询指定了表名并没有指定数据库, 将使用临时表. -- 对于分布式查询处理, 查询中的临时表将被传递给远程服务器. - -在大多数情况下, 临时表并不能手工创建, 但当查询外部数据或使用分布式全局(GLOBAL)IN时,可以创建临时表. - -分布式 DDL 查询 (ON CLUSTER clause) ----------------------------------------------- - -'CREATE', 'DROP', 'ALTER', 和 'RENAME' 查询支持在集群上分布式执行. 例如, 如下的查询在集群中的每个机器节点上创建了 all_hits Distributed 表: - -```sql -CREATE TABLE IF NOT EXISTS all_hits ON CLUSTER cluster (p Date, i Int32) ENGINE = Distributed(cluster, default, hits) -``` - -为了正确执行这些语句,每个节点必须有相同的集群设置(为了简化同步配置,可以使用 zookeeper 来替换). 这些节点也可以连接到ZooKeeper 服务器. -查询语句会在每个节点上执行, 而'ALTER'查询目前暂不支持在同步表(replicated table)上执行. - - - -## CREATE VIEW - -```sql -CREATE [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]name [TO[db.]name] [ENGINE = engine] [POPULATE] AS SELECT ... -``` - -创建一个视图. 有两种类型的视图: 正常视图和物化(MATERIALIZED)视图. - -当创建一个物化视图时, 你必须指定表引擎 – 此表引擎用于存储数据 - -一个物化视图工作流程如下所示: 当插入数据到SELECT 查询指定的表中时, 插入数据部分通过SELECT查询部分来转换, 结果插入到视图中. - -正常视图不保存任何数据, 但是可以从任意表中读取数据. 换句话说,正常视图可以看作是查询结果的一个结果缓存. 当从一个视图中读取数据时, 此查询可以看做是 FROM语句的子查询. - -例如, 假设你已经创建了一个视图: - -```sql -CREATE VIEW view AS SELECT ... -``` - -写了一个查询语句: - -```sql -SELECT a, b, c FROM view -``` -此查询完全等价于子查询: - -```sql -SELECT a, b, c FROM (SELECT ...) -``` - -物化视图保存由SELECT语句查询转换的数据. - -当创建一个物化视图时,你必须指定一个引擎 – 存储数据的目标引擎. - -一个物化视图使用流程如下: 当插入数据到 SELECT 指定的表时, 插入数据部分通过SELECT 来转换, 同时结果被插入到视图中. - - -如果你指定了 POPULATE, 当创建时, 现有的表数据被插入到了视图中, 类似于 CREATE TABLE ... AS SELECT ... . 否则, 在创建视图之后,查询仅包含表中插入的数据. 我们不建议使用 POPULATE, 在视图创建过程中,插入到表中的数据不插入到其中. - -一个'SELECT'查询可以包含 'DISTINCT', 'GROUP BY', 'ORDER BY', 'LIMIT'... 对应的转换在每个数据块上独立执行. 例如, 如果 GROUP BY 被设置, 数据将在插入过程中进行聚合, 但仅是在一个插入数据包中.数据不再进一步聚合. 当使用一个引擎时, 如SummingMergeTree,它将独立执行数据聚合. - -视图看起来和正常表相同. 例如, 你可以使用 SHOW TABLES来列出视图表的相关信息. - -物化视图的'ALTER'查询执行还没有完全开发出来, 因此使用上可能不方便. 如果物化视图使用 ``TO [db.]name``, 你能够 ``DETACH`` 视图, 在目标表运行 ``ALTER``, 然后 ``ATTACH`` 之前的 ``DETACH``视图. - -视图看起来和正常表相同. 例如, 你可以使用 `SHOW TABLES` 来列出视图表的相关信息. - -因此并没有一个单独的SQL语句来删除视图. 为了删除一个视图, 可以使用 `DROP TABLE`. - +../../en/query_language/create.md \ No newline at end of file diff --git a/docs/zh/query_language/insert_into.md b/docs/zh/query_language/insert_into.md index 517af5aa33b..29b47662b0d 120000 --- a/docs/zh/query_language/insert_into.md +++ b/docs/zh/query_language/insert_into.md @@ -1,68 +1 @@ - - -## INSERT - -正在添加数据. - -基本查询格式: - -```sql -INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... -``` - -此查询能够指定字段的列表来插入 `[(c1, c2, c3)]`. 在这种情况下, 剩下的字段用如下来填充: - -- 从表定义中指定的 `DEFAULT` 表达式中计算出值. -- 空字符串, 如果 `DEFAULT` 表达式没有定义. - -如果 [strict_insert_defaults=1](../operations/settings/settings.md#settings-strict_insert_defaults), 没有 `DEFAULT` 定义的字段必须在查询中列出. - -在任何ClickHouse所支持的格式上 [format](../interfaces/formats.md#formats) 数据被传入到 INSERT中. 此格式必须被显式地指定在查询中: - -```sql -INSERT INTO [db.]table [(c1, c2, c3)] FORMAT format_name data_set -``` - -例如, 如下的查询格式与基本的 INSERT ... VALUES 版本相同: - -```sql -INSERT INTO [db.]table [(c1, c2, c3)] FORMAT Values (v11, v12, v13), (v21, v22, v23), ... -``` - -ClickHouse 在数据之前, 删除所有空格和换行(如果有). 当形成一个查询时, 我们推荐在查询操作符之后将数据放入新行(如果数据以空格开始, 这是重要的). - -示例: - -```sql -INSERT INTO t FORMAT TabSeparated -11 Hello, world! -22 Qwerty -``` - -你能够单独从查询中插入数据,通过命令行或 HTTP 接口. 进一步信息, 参见 "[Interfaces](../interfaces/index.md#interfaces)". - -### Inserting The Results of `SELECT` - -```sql -INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... -``` - -在 SELECT语句中, 根据字段的位置来映射. 然而, 在SELECT表达式中的名称和表名可能不同. 如果必要, 可以进行类型转换. - -除了值以外没有其他数据类型允许设置值到表达式中, 例如 `now()`, `1 + 2`, 等. 值格式允许使用有限制的表达式, 但是它并不推荐, 因为在这种情况下, 执行了低效的代码. - -不支持修改数据分区的查询如下: `UPDATE`, `DELETE`, `REPLACE`, `MERGE`, `UPSERT`, `INSERT UPDATE`. -然而, 你能够使用 `ALTER TABLE ... DROP PARTITION`来删除旧数据. - -### Performance Considerations - -`INSERT` 通过主键来排序数据, 并通过月份来拆分数据到每个分区中. 如果插入的数据有混合的月份, 会显著降低`INSERT` 插入的性能. 应该避免此类操作: - -- 大批量地添加数据, 如每次 100,000 行. -- 在上传数据之前, 通过月份分组数据. - -下面操作性能不会下降: - -- 数据实时插入. -- 上传的数据通过时间来排序. - +../../en/query_language/insert_into.md \ No newline at end of file From 2eff364e63f7732e4ef52f3be2ff4a4dcaa7ac58 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 19 Oct 2018 13:03:34 +0300 Subject: [PATCH 18/28] fixes for zh docs (#3423) * CLICKHOUSE-4063: less manual html @ index.md * CLICKHOUSE-4063: recommend markdown="1" in README.md * CLICKHOUSE-4003: manually purge custom.css for now * CLICKHOUSE-4064: expand

    before any print (including to pdf) * CLICKHOUSE-3927: rearrange interfaces/formats.md a bit * CLICKHOUSE-3306: add few http headers * Remove copy-paste introduced in #3392 * Hopefully better chinese fonts #3392 * get rid of tabs @ custom.css * Apply comments and patch from #3384 * Add jdbc.md to ToC and some translation, though it still looks badly incomplete * minor punctuation * Add some backlinks to official website from mirrors that just blindly take markdown sources * Do not make fonts extra light * find . -name '*.md' -type f | xargs -I{} perl -pi -e 's//g' {} * find . -name '*.md' -type f | xargs -I{} perl -pi -e 's/ sql/g' {} * Remove outdated stuff from roadmap.md * Not so light font on front page too * Refactor Chinese formats.md to match recent changes in other languages * Update some links on front page * Remove some outdated comment * Add twitter link to front page * More front page links tuning * Add Amsterdam meetup link * Smaller font to avoid second line * Add Amsterdam link to README.md * Proper docs nav translation * Back to 300 font-weight except Chinese * fix docs build * Update Amsterdam link * remove symlinks * more zh punctuation * apply lost comment by @zhang2014 * Apply comments by @zhang2014 from #3417 --- docs/zh/data_types/array.md | 6 +- docs/zh/data_types/datetime.md | 2 +- docs/zh/query_language/create.md | 156 +++++++++++++++++++++++++- docs/zh/query_language/insert_into.md | 69 +++++++++++- 4 files changed, 227 insertions(+), 6 deletions(-) mode change 120000 => 100644 docs/zh/query_language/create.md mode change 120000 => 100644 docs/zh/query_language/insert_into.md diff --git a/docs/zh/data_types/array.md b/docs/zh/data_types/array.md index 0dc89baf3c7..00e07d38be0 100644 --- a/docs/zh/data_types/array.md +++ b/docs/zh/data_types/array.md @@ -4,11 +4,11 @@ 由 `T` 类型元素组成的数组。 -`T` 可以是任意类型,包含数组类型。不推荐使用多维数组,ClickHouse 对多维数组的支持有限。例如,不能存储在 `MergeTree` 表中存储多维数组。 +`T` 可以是任意类型,包含数组类型。 但不推荐使用多维数组,ClickHouse 对多维数组的支持有限。例如,不能存储在 `MergeTree` 表中存储多维数组。 ## 创建数组 -您可以使用一个函数创建数组: +您可以使用array函数来创建数组: ``` array(T) @@ -50,7 +50,7 @@ SELECT ## 使用数据类型 -当动态创建数组时,ClickHouse 自动将参数类型定义为可以存储所有列出的参数的最窄的数据类型。如果存在任何 [NULL](../query_language/syntax.md#null-literal) 或者 [Nullable](nullable.md#data_type-nullable) 类型参数,那么数组元素的类型是 [Nullable](nullable.md#data_type-nullable)。 +ClickHouse会自动检测数组元素,并根据元素计算出存储这些元素最小的数据类型。如果在元素中存在[NULL](../query_language/syntax.md#null-literal)或存在[Nullable](nullable.md#data_type-nullable)类型元素,那么数组的元素类型将会变成[Nullable](nullable.md#data_type-nullable)。 如果 ClickHouse 无法确定数据类型,它将产生异常。当尝试同时创建一个包含字符串和数字的数组时会发生这种情况 (`SELECT array(1, 'a')`)。 diff --git a/docs/zh/data_types/datetime.md b/docs/zh/data_types/datetime.md index 4a5a8095f0a..6988d7da283 100644 --- a/docs/zh/data_types/datetime.md +++ b/docs/zh/data_types/datetime.md @@ -2,7 +2,7 @@ # DateTime -时间戳类型。用四个字节存储 Unix 时间戳(无符号的)。允许存储与日期类型相同的范围内的值。最小值为 0000-00-00 00:00:00。时间戳类型值精确到秒(没有闰秒)。 +时间戳类型。用四个字节(无符号的)存储 Unix 时间戳)。允许存储与日期类型相同的范围内的值。最小值为 0000-00-00 00:00:00。时间戳类型值精确到秒(没有闰秒)。 ## 时区 diff --git a/docs/zh/query_language/create.md b/docs/zh/query_language/create.md deleted file mode 120000 index a13304d176e..00000000000 --- a/docs/zh/query_language/create.md +++ /dev/null @@ -1 +0,0 @@ -../../en/query_language/create.md \ No newline at end of file diff --git a/docs/zh/query_language/create.md b/docs/zh/query_language/create.md new file mode 100644 index 00000000000..d554dfeeb5b --- /dev/null +++ b/docs/zh/query_language/create.md @@ -0,0 +1,155 @@ +## 创建数据库 + +创建 `db_name` 数据库。 + +```sql +CREATE DATABASE [IF NOT EXISTS] db_name +``` + +数据库是一个包含多个表的目录,如果在CREATE DATABASE语句中包含`IF NOT EXISTS`,则在数据库已经存在的情况下查询也不会返回错误。 + + + +## 创建表 + +`CREATE TABLE` 语句有几种形式. + +```sql +CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE = engine +``` + +如果`db`没有设置, 在数据库`db`中或者当前数据库中, 创建一个表名为`name`的表, 在括号和`engine` 引擎中指定结构。 表的结构是一个列描述的列表。 如果引擎支持索引, 则他们将是表引擎的参数。 + +表结构是一个列描述的列表。 如果引擎支持索引, 他们以表引擎的参数表示。 + +在最简单的情况, 一个列描述是'命名类型'。 例如: RegionID UInt32。 对于默认值, 表达式也能够被定义。 + +```sql +CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name AS [db2.]name2 [ENGINE = engine] +``` + +创建一个表, 其结构与另一个表相同。 你能够为此表指定一个不同的引擎。 如果引擎没有被指定, 相同的引擎将被用于`db2。name2`表上。 + +```sql +CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name ENGINE = engine AS SELECT ... +``` + +创建一个表,其结构类似于 SELECT 查询后的结果, 带有`engine` 引擎, 从 SELECT查询数据填充它。 + +在所有情况下,如果`IF NOT EXISTS`被指定, 如果表已经存在, 查询并不返回一个错误。 在这种情况下, 查询并不做任何事情。 + +### 默认值 + +列描述能够为默认值指定一个表达式, 其中一个方法是:DEFAULT expr, MATERIALIZED expr, ALIAS expr。 +例如: URLDomain String DEFAULT domain(URL)。 + +如果默认值的一个表达式没有定义, 如果字段是数字类型, 默认值是将设置为0, 如果是字符类型, 则设置为空字符串, 日期类型则设置为 0000-00-00 或者 0000-00-00 00:00:00(时间戳)。 NULLs 则不支持。 + +如果默认表达式被定义, 字段类型是可选的。 如果没有明确的定义类型, 则将使用默认表达式。 例如: EventDate DEFAULT toDate(EventTime) – `Date` 类型将用于 `EventDate` 字段。 + +如果数据类型和默认表达式被明确定义, 此表达式将使用函数被转换为特定的类型。 例如: Hits UInt32 DEFAULT 0 与 Hits UInt32 DEFAULT toUInt32(0)是等价的。 + +默认表达是可能被定义为一个任意的表达式,如表的常量和字段。 当创建和更改表结构时, 它将检查表达式是否包含循环。 对于 INSERT操作来说, 它将检查表达式是否可解析 – 所有的字段通过传参后进行计算。 + +`DEFAULT expr` + +正常的默认值。 如果 INSERT 查询并没有指定对应的字段, 它将通过计算对应的表达式来填充。 + +`物化表达式` + +物化表达式。 此类型字段并没有指定插入操作, 因为它经常执行计算任务。 对一个插入操作, 无字段列表, 那么这些字段将不考虑。 另外, 当在一个SELECT查询语句中使用星号时, 此字段并不被替换。 这将保证INSERT INTO SELECT * FROM 的不可变性。 + +`别名表达式` + +别名。 此字段不存储在表中。 +此列的值不插入到表中, 当在一个SELECT查询语句中使用星号时,此字段并不被替换。 +它能够用在 SELECTs中,如果别名在查询解析时被扩展。 + +当使用更新查询添加一个新的字段, 这些列的旧值不被写入。 相反, 新字段没有值,当读取旧值时, 表达式将被计算。 然而,如果运行表达式需要不同的字段, 这些字段将被读取 , 但是仅读取相关的数据块。 + +如果你添加一个新的字段到表中, 然后改变它的默认表达式, 对于使用的旧值将更改(对于此数据, 值不保存在磁盘上)。 当运行背景线程时, 缺少合并数据块的字段数据写入到合并数据块中。 + +在嵌套数据结构中设置默认值是不允许的。 + + +### 临时表 + +在任何情况下, 如果临时表被指定, 一个临时表将被创建。 临时表有如下的特性: + +- 当会话结束后, 临时表将删除,或者连接丢失。 +- 一个临时表使用内存表引擎创建。 其他的表引擎不支持临时表。 +- 数据库不能为一个临时表指定。 它将创建在数据库之外。 +- 如果一个临时表与另外的表有相同的名称 ,一个查询指定了表名并没有指定数据库, 将使用临时表。 +- 对于分布式查询处理, 查询中的临时表将被传递给远程服务器。 + +在大多数情况下, 临时表并不能手工创建, 但当查询外部数据或使用分布式全局(GLOBAL)IN时,可以创建临时表。 + +分布式 DDL 查询 (ON CLUSTER clause) +---------------------------------------------- + +`CREATE`, `DROP`, `ALTER`, 和 `RENAME` 查询支持在集群上分布式执行。 例如, 如下的查询在集群中的每个机器节点上创建了 all_hits Distributed 表: + +```sql +CREATE TABLE IF NOT EXISTS all_hits ON CLUSTER cluster (p Date, i Int32) ENGINE = Distributed(cluster, default, hits) +``` + +为了正确执行这些语句,每个节点必须有相同的集群设置(为了简化同步配置,可以使用 zookeeper 来替换)。 这些节点也可以连接到ZooKeeper 服务器。 +查询语句会在每个节点上执行, 而`ALTER`查询目前暂不支持在同步表(replicated table)上执行。 + + + +## CREATE VIEW + +```sql +CREATE [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]name [TO[db.]name] [ENGINE = engine] [POPULATE] AS SELECT ... +``` + +创建一个视图。 有两种类型的视图: 正常视图和物化(MATERIALIZED)视图。 + +当创建一个物化视图时, 你必须指定表引擎 – 此表引擎用于存储数据 + +一个物化视图工作流程如下所示: 当插入数据到SELECT 查询指定的表中时, 插入数据部分通过SELECT查询部分来转换, 结果插入到视图中。 + +正常视图不保存任何数据, 但是可以从任意表中读取数据。 换句话说,正常视图可以看作是查询结果的一个结果缓存。 当从一个视图中读取数据时, 此查询可以看做是 FROM语句的子查询。 + +例如, 假设你已经创建了一个视图: + +```sql +CREATE VIEW view AS SELECT ... +``` + +写了一个查询语句: + +```sql +SELECT a, b, c FROM view +``` +此查询完全等价于子查询: + +```sql +SELECT a, b, c FROM (SELECT ...) +``` + +物化视图保存由SELECT语句查询转换的数据。 + +当创建一个物化视图时,你必须指定一个引擎 – 存储数据的目标引擎。 + +一个物化视图使用流程如下: 当插入数据到 SELECT 指定的表时, 插入数据部分通过SELECT 来转换, 同时结果被插入到视图中。 + + +如果你指定了 POPULATE, 当创建时, 现有的表数据被插入到了视图中, 类似于 `CREATE TABLE ... AS SELECT ...` . 否则, 在创建视图之后,查询仅包含表中插入的数据. 我们不建议使用 POPULATE, 在视图创建过程中,插入到表中的数据不插入到其中. + +一个`SELECT`查询可以包含 `DISTINCT`, `GROUP BY`, `ORDER BY`, `LIMIT`。。。 对应的转换在每个数据块上独立执行。 例如, 如果 GROUP BY 被设置, 数据将在插入过程中进行聚合, 但仅是在一个插入数据包中。数据不再进一步聚合。 当使用一个引擎时, 如SummingMergeTree,它将独立执行数据聚合。 + +视图看起来和正常表相同。 例如, 你可以使用 SHOW TABLES来列出视图表的相关信息。 + +物化视图的`ALTER`查询执行还没有完全开发出来, 因此使用上可能不方便。 如果物化视图使用 `TO [db。]name`, 你能够 `DETACH` 视图, 在目标表运行 `ALTER`, 然后 `ATTACH` 之前的 `DETACH`视图。 + +视图看起来和正常表相同。 例如, 你可以使用 `SHOW TABLES` 来列出视图表的相关信息。 + +因此并没有一个单独的SQL语句来删除视图。 为了删除一个视图, 可以使用 `DROP TABLE`。 + diff --git a/docs/zh/query_language/insert_into.md b/docs/zh/query_language/insert_into.md deleted file mode 120000 index 29b47662b0d..00000000000 --- a/docs/zh/query_language/insert_into.md +++ /dev/null @@ -1 +0,0 @@ -../../en/query_language/insert_into.md \ No newline at end of file diff --git a/docs/zh/query_language/insert_into.md b/docs/zh/query_language/insert_into.md new file mode 100644 index 00000000000..7bea8a0b361 --- /dev/null +++ b/docs/zh/query_language/insert_into.md @@ -0,0 +1,68 @@ + + +## INSERT + +正在添加数据。 + +基本查询格式: + +```sql +INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... +``` + +此查询能够指定字段的列表来插入 `[(c1, c2, c3)]`。 在这种情况下, 剩下的字段用如下来填充: + +- 从表定义中指定的 `DEFAULT` 表达式中计算出值。 +- 空字符串, 如果 `DEFAULT` 表达式没有定义。 + +如果 [strict_insert_defaults=1](../operations/settings/settings.md#settings-strict_insert_defaults), 没有 `DEFAULT` 定义的字段必须在查询中列出. + +在任何ClickHouse所支持的格式上 [format](../interfaces/formats.md#formats) 数据被传入到 INSERT中. 此格式必须被显式地指定在查询中: + +```sql +INSERT INTO [db.]table [(c1, c2, c3)] FORMAT format_name data_set +``` + +例如, 如下的查询格式与基本的 `INSERT ... VALUES` 版本相同: + +```sql +INSERT INTO [db.]table [(c1, c2, c3)] FORMAT Values (v11, v12, v13), (v21, v22, v23), ... +``` + +ClickHouse 在数据之前, 删除所有空格和换行(如果有)。 当形成一个查询时, 我们推荐在查询操作符之后将数据放入新行(如果数据以空格开始, 这是重要的)。 + +示例: + +```sql +INSERT INTO t FORMAT TabSeparated +11 Hello, world! +22 Qwerty +``` + +你能够单独从查询中插入数据,通过命令行或 HTTP 接口. 进一步信息, 参见 "[Interfaces](../interfaces/index.md#interfaces)". + +### Inserting The Results of `SELECT` + +```sql +INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... +``` + +在 SELECT语句中, 根据字段的位置来映射。 然而, 在SELECT表达式中的名称和表名可能不同。 如果必要, 可以进行类型转换。 + +除了值以外没有其他数据类型允许设置值到表达式中, 例如 `now()`, `1 + 2`, 等。 值格式允许使用有限制的表达式, 但是它并不推荐, 因为在这种情况下, 执行了低效的代码。 + +不支持修改数据分区的查询如下: `UPDATE`, `DELETE`, `REPLACE`, `MERGE`, `UPSERT`, `INSERT UPDATE`。 +然而, 你能够使用 `ALTER TABLE ... DROP PARTITION`来删除旧数据。 + +### Performance Considerations + +`INSERT` 通过主键来排序数据, 并通过月份来拆分数据到每个分区中。 如果插入的数据有混合的月份, 会显著降低`INSERT` 插入的性能。 应该避免此类操作: + +- 大批量地添加数据, 如每次 100,000 行。 +- 在上传数据之前, 通过月份分组数据。 + +下面操作性能不会下降: + +- 数据实时插入。 +- 上传的数据通过时间来排序。 + From b5edce0e32a68157fa7f5e449bcce217b40dd23a Mon Sep 17 00:00:00 2001 From: BayoNet Date: Fri, 19 Oct 2018 14:25:22 +0300 Subject: [PATCH 19/28] Updates for Aggregating-,Collapsing-, Replacing- and SummingMergeTree. (#3346) * Update of english version of descriprion of the table function `file`. * New syntax for ReplacingMergeTree. Some improvements in text. * Significantly change article about SummingMergeTree. Article is restructured, text is changed in many places of the document. New syntax for table creation is described. * Descriptions of AggregateFunction and AggregatingMergeTree are updated. Russian version. * New syntax for new syntax of CREATE TABLE * Added english docs on Aggregating, Replacing and SummingMergeTree. * CollapsingMergeTree docs. English version. * 1. Update of CollapsingMergeTree. 2. Minor changes in markup * Update aggregatefunction.md * Update aggregatefunction.md * Update aggregatefunction.md * Update aggregatingmergetree.md * GraphiteMergeTree docs update. New syntax for creation of Replicated* tables. Minor changes in *MergeTree tables creation syntax. * Markup fix * Markup and language fixes * Clarification in the CollapsingMergeTree article --- .../aggregatefunction.md | 62 ++++- .../table_engines/aggregatingmergetree.md | 114 ++++----- .../table_engines/collapsingmergetree.md | 222 ++++++++++++++++-- .../table_engines/graphitemergetree.md | 103 ++++++-- docs/en/operations/table_engines/mergetree.md | 53 +++-- .../table_engines/replacingmergetree.md | 60 ++++- .../operations/table_engines/replication.md | 30 ++- .../table_engines/summingmergetree.md | 126 ++++++++-- .../agg_functions/combinators.md | 4 +- .../query_language/agg_functions/reference.md | 10 +- docs/en/query_language/create.md | 30 ++- .../functions/functions_for_nulls.md | 2 +- docs/en/query_language/insert_into.md | 4 +- .../en/query_language/table_functions/file.md | 43 +++- .../aggregatefunction.md | 62 ++++- docs/ru/operations/configuration_files.md | 2 +- .../table_engines/aggregatingmergetree.md | 113 ++++----- docs/ru/operations/table_engines/mergetree.md | 50 ++-- .../table_engines/replacingmergetree.md | 55 ++++- .../table_engines/summingmergetree.md | 124 ++++++++-- .../agg_functions/combinators.md | 4 +- .../query_language/agg_functions/reference.md | 8 +- docs/ru/query_language/create.md | 2 + 23 files changed, 1020 insertions(+), 263 deletions(-) diff --git a/docs/en/data_types/nested_data_structures/aggregatefunction.md b/docs/en/data_types/nested_data_structures/aggregatefunction.md index 1db3bb9e079..377c165a103 100644 --- a/docs/en/data_types/nested_data_structures/aggregatefunction.md +++ b/docs/en/data_types/nested_data_structures/aggregatefunction.md @@ -1,6 +1,66 @@ + + # AggregateFunction(name, types_of_arguments...) -The intermediate state of an aggregate function. To get it, use aggregate functions with the '-State' suffix. For more information, see "AggregatingMergeTree". +The intermediate state of an aggregate function. To get it, use aggregate functions with the `-State` suffix. To get aggregated data in the future, you must use the same aggregate functions with the `-Merge`suffix. + +`AggregateFunction` — parametric data type. + +**Parameters** + +- Name of the aggregate function. + + If the function is parametric specify its parameters too. + +- Types of the aggregate function arguments. + +**Example** + +```sql +CREATE TABLE t +( + column1 AggregateFunction(uniq, UInt64), + column2 AggregateFunction(anyIf, String, UInt8), + column3 AggregateFunction(quantiles(0.5, 0.9), UInt64) +) ENGINE = ... +``` + +[uniq](../../query_language/agg_functions/reference.md#agg_function-uniq), anyIf ([any](../../query_language/agg_functions/reference.md#agg_function-any)+[If](../../query_language/agg_functions/combinators.md#agg-functions-combine-if)) and [quantiles](../../query_language/agg_functions/reference.md#agg_function-quantiles) are the aggregate functions supported in ClickHouse. + +## Usage + +### Data Insertion + +To insert data, use `INSERT SELECT` with aggregate `-State`- functions. + +**Function examples** + +``` +uniqState(UserID) +quantilesState(0.5, 0.9)(SendTiming) +``` + +In contrast to the corresponding functions `uniq` and `quantiles`, `-State`- functions return the state, instead the final value. In other words, they return a value of `AggregateFunction` type. + +In the results of `SELECT` query the values of `AggregateFunction` type have implementation-specific binary representation for all of the ClickHouse output formats. If dump data into, for example, `TabSeparated` format with `SELECT` query then this dump can be loaded back using `INSERT` query. + +### Data Selection + +When selecting data from `AggregatingMergeTree` table, use `GROUP BY` clause and the same aggregate functions as when inserting data, but using `-Merge`suffix. + +An aggregate function with `-Merge` suffix takes a set of states, combines them, and returns the result of complete data aggregation. + +For example, the following two queries return the same result: + +```sql +SELECT uniq(UserID) FROM table + +SELECT uniqMerge(state) FROM (SELECT uniqState(UserID) AS state FROM table GROUP BY RegionID) +``` + +## Usage Example + +See [AggregatingMergeTree](../../operations/table_engines/aggregatingmergetree.md#table_engine-aggregatingmergetree) engine description. [Original article](https://clickhouse.yandex/docs/en/data_types/nested_data_structures/aggregatefunction/) diff --git a/docs/en/operations/table_engines/aggregatingmergetree.md b/docs/en/operations/table_engines/aggregatingmergetree.md index 44bc03a8bd5..4ebb707a980 100644 --- a/docs/en/operations/table_engines/aggregatingmergetree.md +++ b/docs/en/operations/table_engines/aggregatingmergetree.md @@ -1,59 +1,68 @@ + + # AggregatingMergeTree -This engine differs from `MergeTree` in that the merge combines the states of aggregate functions stored in the table for rows with the same primary key value. - -For this to work, it uses the `AggregateFunction` data type, as well as `-State` and `-Merge` modifiers for aggregate functions. Let's examine it more closely. - -There is an `AggregateFunction` data type. It is a parametric data type. As parameters, the name of the aggregate function is passed, then the types of its arguments. - -Examples: - -``` sql -CREATE TABLE t -( - column1 AggregateFunction(uniq, UInt64), - column2 AggregateFunction(anyIf, String, UInt8), - column3 AggregateFunction(quantiles(0.5, 0.9), UInt64) -) ENGINE = ... -``` - -This type of column stores the state of an aggregate function. - -To get this type of value, use aggregate functions with the `State` suffix. - -Example:`uniqState(UserID), quantilesState(0.5, 0.9)(SendTiming)` - -In contrast to the corresponding `uniq` and `quantiles` functions, these functions return the state, rather than the prepared value. In other words, they return an `AggregateFunction` type value. - -An `AggregateFunction` type value can't be output in Pretty formats. In other formats, these types of values are output as implementation-specific binary data. The `AggregateFunction` type values are not intended for output or saving in a dump. - -The only useful thing you can do with `AggregateFunction` type values is to combine the states and get a result, which essentially means to finish aggregation. Aggregate functions with the 'Merge' suffix are used for this purpose. -Example: `uniqMerge(UserIDState)`, where `UserIDState` has the `AggregateFunction` type. - -In other words, an aggregate function with the 'Merge' suffix takes a set of states, combines them, and returns the result. -As an example, these two queries return the same result: - -``` sql -SELECT uniq(UserID) FROM table - -SELECT uniqMerge(state) FROM (SELECT uniqState(UserID) AS state FROM table GROUP BY RegionID) -``` - -There is an `AggregatingMergeTree` engine. Its job during a merge is to combine the states of aggregate functions from different table rows with the same primary key value. - -You can't use a normal INSERT to insert a row in a table containing `AggregateFunction` columns, because you can't explicitly define the `AggregateFunction` value. Instead, use `INSERT SELECT` with `-State` aggregate functions for inserting data. - -With SELECT from an `AggregatingMergeTree` table, use GROUP BY and aggregate functions with the '-Merge' modifier in order to complete data aggregation. +The engine inherits from [MergeTree](mergetree.md#table_engines-mergetree), altering the logic for data parts merging. ClickHouse replaces all rows with the same primary key with a single row (within a one data part) that stores a combination of states of aggregate functions. You can use `AggregatingMergeTree` tables for incremental data aggregation, including for aggregated materialized views. -Example: +The engine processes all columns with [AggregateFunction](../../data_types/nested_data_structures/aggregatefunction.md#data_type-aggregatefunction) type. -Create an `AggregatingMergeTree` materialized view that watches the `test.visits` table: +It is appropriate to use `AggregatingMergeTree` if it reduces the number of rows by orders. + +## Creating a Table + +``` sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE = AggregatingMergeTree() +[PARTITION BY expr] +[ORDER BY expr] +[SAMPLE BY expr] +[SETTINGS name=value, ...] +``` + +For a description of request parameters, see [request description](../../query_language/create.md#query_language-queries-create_table). + +**Query clauses** + +When creating a `ReplacingMergeTree` table the same [clauses](mergetree.md#table_engines-mergetree-configuring) are required, as when creating a `MergeTree` table. + +
    Deprecated Method for Creating a Table + +!!! attention + Do not use this method in new projects and, if possible, switch the old projects to the method described above. + +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE [=] AggregatingMergeTree(date-column [, sampling_expression], (primary, key), index_granularity) +``` + +All of the parameters have the same meaning as in `MergeTree`. +
    + +## SELECT and INSERT + +To insert data, use [INSERT SELECT](../../query_language/insert_into.md#queries-insert-select) query with aggregate `-State`- functions. + +When selecting data from `AggregatingMergeTree` table, use `GROUP BY` clause and the same aggregate functions as when inserting data, but using `-Merge` suffix. + +In the results of `SELECT` query the values of `AggregateFunction` type have implementation-specific binary representation for all of the ClickHouse output formats. If dump data into, for example, `TabSeparated` format with `SELECT` query then this dump can be loaded back using `INSERT` query. + +## Example of an Aggregated Materialized View + +`AggregatingMergeTree` materialized view that watches the `test.visits` table: ``` sql CREATE MATERIALIZED VIEW test.basic -ENGINE = AggregatingMergeTree(StartDate, (CounterID, StartDate), 8192) +ENGINE = AggregatingMergeTree() PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate) AS SELECT CounterID, StartDate, @@ -63,13 +72,15 @@ FROM test.visits GROUP BY CounterID, StartDate; ``` -Insert data in the `test.visits` table. Data will also be inserted in the view, where it will be aggregated: +Inserting of data into the `test.visits` table. ``` sql INSERT INTO test.visits ... ``` -Perform `SELECT` from the view using `GROUP BY` in order to complete data aggregation: +The data are inserted in both the table and view `test.basic` that will perform the aggregation. + +To get the aggregated data, we need to execute a query such as `SELECT ... GROUP BY ...` from the view `test.basic`: ``` sql SELECT @@ -81,9 +92,4 @@ GROUP BY StartDate ORDER BY StartDate; ``` -You can create a materialized view like this and assign a normal view to it that finishes data aggregation. - -Note that in most cases, using `AggregatingMergeTree` is not justified, since queries can be run efficiently enough on non-aggregated data. - - [Original article](https://clickhouse.yandex/docs/en/operations/table_engines/aggregatingmergetree/) diff --git a/docs/en/operations/table_engines/collapsingmergetree.md b/docs/en/operations/table_engines/collapsingmergetree.md index 193984c92a5..45106cb25e8 100644 --- a/docs/en/operations/table_engines/collapsingmergetree.md +++ b/docs/en/operations/table_engines/collapsingmergetree.md @@ -2,39 +2,219 @@ # CollapsingMergeTree -*This engine is used specifically for Yandex.Metrica.* +The engine inherits from [MergeTree](mergetree.md#table_engines-mergetree) and adds the logic of rows collapsing to data parts merge algorithm. -It differs from `MergeTree` in that it allows automatic deletion, or "collapsing" certain pairs of rows when merging. +`CollapsingMergeTree` asynchronously deletes (collapses) pairs of rows if all of the fields in a row are equivalent excepting the particular field `Sign` which can have `1` and `-1` values. Rows without a pair are kept. For more details see the [Collapsing](#collapsingmergetree-collapsing) section of the document. -Yandex.Metrica has normal logs (such as hit logs) and change logs. Change logs are used for incrementally calculating statistics on data that is constantly changing. Examples are the log of session changes, or logs of changes to user histories. Sessions are constantly changing in Yandex.Metrica. For example, the number of hits per session increases. We refer to changes in any object as a pair (?old values, ?new values). Old values may be missing if the object was created. New values may be missing if the object was deleted. If the object was changed, but existed previously and was not deleted, both values are present. In the change log, one or two entries are made for each change. Each entry contains all the attributes that the object has, plus a special attribute for differentiating between the old and new values. When objects change, only the new entries are added to the change log, and the existing ones are not touched. +The engine may significantly reduce the volume of storage and increase efficiency of `SELECT` query as a consequence. -The change log makes it possible to incrementally calculate almost any statistics. To do this, we need to consider "new" rows with a plus sign, and "old" rows with a minus sign. In other words, incremental calculation is possible for all statistics whose algebraic structure contains an operation for taking the inverse of an element. This is true of most statistics. We can also calculate "idempotent" statistics, such as the number of unique visitors, since the unique visitors are not deleted when making changes to sessions. +## Creating a Table -This is the main concept that allows Yandex.Metrica to work in real time. - -CollapsingMergeTree accepts an additional parameter - the name of an Int8-type column that contains the row's "sign". Example: - -``` sql -CollapsingMergeTree(EventDate, (CounterID, EventDate, intHash32(UniqID), VisitID), 8192, Sign) +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE = CollapsingMergeTree(sign) +[PARTITION BY expr] +[ORDER BY expr] +[SAMPLE BY expr] +[SETTINGS name=value, ...] ``` -Here, `Sign` is a column containing -1 for "old" values and 1 for "new" values. +For a description of request parameters, see [request description](../../query_language/create.md#query_language-queries-create_table). -When merging, each group of consecutive identical primary key values (columns for sorting data) is reduced to no more than one row with the column value 'sign_column = -1' (the "negative row") and no more than one row with the column value 'sign_column = 1' (the "positive row"). In other words, entries from the change log are collapsed. +**CollapsingMergeTree Parameters** -If the number of positive and negative rows matches, the first negative row and the last positive row are written. -If there is one more positive row than negative rows, only the last positive row is written. -If there is one more negative row than positive rows, only the first negative row is written. -Otherwise, there will be a logical error and none of the rows will be written. (A logical error can occur if the same section of the log was accidentally inserted more than once. The error is just recorded in the server log, and the merge continues.) +- `sign` — Name of the column with the type of row: `1` is a "state" row, `-1` is a "cancel" row. + + Column data type — `Int8`. + +**Query clauses** + +When creating a `CollapsingMergeTree` table, the same [clauses](mergetree.md#table_engines-mergetree-configuring) are required, as when creating a `MergeTree` table. + +
    Deprecated Method for Creating a Table + +!!! attention + Do not use this method in new projects and, if possible, switch the old projects to the method described above. + +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE [=] CollapsingMergeTree(date-column [, sampling_expression], (primary, key), index_granularity, sign) +``` + +All of the parameters excepting `sign` have the same meaning as in `MergeTree`. + +- `sign` — Name of the column with the type of row: `1` — "state" row, `-1` — "cancel" row. + + Column Data Type — `Int8`. +
    + + + +## Collapsing + +### Data + +Consider the situation where you need to save continually changing data for some object. It sounds logical to have one row for an object and update it at any change, but update operation is expensive and slow for DBMS because it requires rewriting of the data in the storage. If you need to write data quickly, update not acceptable, but you can write the changes of an object sequentially as follows. + +Use the particular column `Sign` when writing row. If `Sign = 1` it means that the row is a state of an object, let's call it "state" row. If `Sign = -1` it means the cancellation of the state of an object with the same attributes, let's call it "cancel" row. + +For example, we want to calculate how much pages users checked at some site and how long they were there. At some moment of time we write the following row with the state of user activity: + +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 5 │ 146 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +``` + +At some moment later we register the change of user activity and write it with the following two rows. + +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 5 │ 146 │ -1 │ +│ 4324182021466249494 │ 6 │ 185 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +``` + +The first row cancels the previous state of the object (user). It should copy all of the fields of the canceled state excepting `Sign`. + +The second row contains the current state. + +As we need only the last state of user activity, the rows + +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 5 │ 146 │ 1 │ +│ 4324182021466249494 │ 5 │ 146 │ -1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +``` + +can be deleted collapsing the invalid (old) state of an object. `CollapsingMergeTree` does this while merging of the data parts. + +Why we need 2 rows for each change read in the "Algorithm" paragraph. + +**Peculiar properties of such approach** + +1. The program that writes the data should remember the state of an object to be able to cancel it. "Cancel" string should be the copy of "state" string with the opposite `Sign`. It increases the initial size of storage but allows to write the data quickly. +2. Long growing arrays in columns reduce the efficiency of the engine due to load for writing. The more straightforward data, the higher efficiency. +3. `SELECT` results depend strongly on the consistency of object changes history. Be accurate when preparing data for inserting. You can get unpredictable results in inconsistent data, for example, negative values for non-negative metrics such as session depth. + +### Algorithm + +When ClickHouse merges data parts, each group of consecutive rows with the same primary key is reduced to not more than two rows, one with `Sign = 1` ("state" row) and another with `Sign = -1` ("cancel" row). In other words, entries collapse. + +For each resulting data part ClickHouse saves: + + 1. The first "cancel" and the last "state" rows, if the number of "state" and "cancel" rows matches. + 1. The last "state" row, if there is one more "state" row than "cancel" rows. + 1. The first "cancel" row, if there is one more "cancel" row than "state" rows. + 1. None of the rows, in all other cases. + + The merge continues, but ClickHouse treats this situation as a logical error and records it in the server log. This error can occur if the same data were inserted more than once. Thus, collapsing should not change the results of calculating statistics. -Changes are gradually collapsed so that in the end only the last value of almost every object is left. -Compared to MergeTree, the CollapsingMergeTree engine allows a multifold reduction of data volume. +Changes gradually collapsed so that in the end only the last state of almost every object left. -There are several ways to get completely "collapsed" data from a `CollapsingMergeTree` table: +The `Sign` is required because the merging algorithm doesn't guarantee that all of the rows with the same primary key will be in the same resulting data part and even on the same physical server. ClickHouse process `SELECT` queries with multiple threads, and it can not predict the order of rows in the result. The aggregation is required if there is a need to get completely "collapsed" data from `CollapsingMergeTree` table. -1. Write a query with GROUP BY and aggregate functions that accounts for the sign. For example, to calculate quantity, write 'sum(Sign)' instead of 'count()'. To calculate the sum of something, write 'sum(Sign * x)' instead of 'sum(x)', and so on, and also add 'HAVING sum(Sign) `>` 0'. Not all amounts can be calculated this way. For example, the aggregate functions 'min' and 'max' can't be rewritten. -2. If you must extract data without aggregation (for example, to check whether rows are present whose newest values match certain conditions), you can use the FINAL modifier for the FROM clause. This approach is significantly less efficient. +To finalize collapsing write a query with `GROUP BY` clause and aggregate functions that account for the sign. For example, to calculate quantity, use `sum(Sign)` instead of `count()`. To calculate the sum of something, use `sum(Sign * x)` instead of `sum(x)`, and so on, and also add `HAVING sum(Sign) > 0`. +The aggregates `count`, `sum` and `avg` could be calculated this way. The aggregate `uniq` could be calculated if an object has at list one state not collapsed. The aggregates `min` and `max` could not be calculated because `CollapsingMergeTree` does not save values history of the collapsed states. + +If you need to extract data without aggregation (for example, to check whether rows are present whose newest values match certain conditions), you can use the `FINAL` modifier for the `FROM` clause. This approach is significantly less efficient. + +## Example of use + +Example data: + +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 5 │ 146 │ 1 │ +│ 4324182021466249494 │ 5 │ 146 │ -1 │ +│ 4324182021466249494 │ 6 │ 185 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +``` + +Creation of the table: + +```sql +CREATE TABLE UAct +( + UserID UInt64, + PageViews UInt8, + Duration UInt8, + Sign Int8 +) +ENGINE = CollapsingMergeTree(Sign) +ORDER BY UserID +``` + +Insertion of the data: + +```sql +INSERT INTO UAct VALUES (4324182021466249494, 5, 146, 1) +``` +```sql +INSERT INTO UAct VALUES (4324182021466249494, 5, 146, -1),(4324182021466249494, 6, 185, 1) +``` + +We use two `INSERT` queries to create two different data parts. If we insert the data with one query ClickHouse creates one data part and will not perform any merge ever. + +Getting the data: + +``` +SELECT * FROM UAct +``` + +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 5 │ 146 │ -1 │ +│ 4324182021466249494 │ 6 │ 185 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 5 │ 146 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +``` + +What do we see and where is collapsing? +With two `INSERT` queries, we created 2 data parts. The `SELECT` query was performed in 2 threads, and we got a random order of rows. +Collapsing not occurred because there was no merge of the data parts yet. ClickHouse merges data part in an unknown moment of time which we can not predict. + +Thus we need aggregation: + +```sql +SELECT + UserID, + sum(PageViews * Sign) AS PageViews, + sum(Duration * Sign) AS Duration +FROM UAct +GROUP BY UserID +HAVING sum(Sign) > 0 +``` +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┐ +│ 4324182021466249494 │ 6 │ 185 │ +└─────────────────────┴───────────┴──────────┘ +``` + +If we do not need aggregation and want to force collapsing, we can use `FINAL` modifier for `FROM` clause. + +```sql +SELECT * FROM UAct FINAL +``` +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 6 │ 185 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +``` + +This way of selecting the data is very inefficient. Don't use it for big tables. [Original article](https://clickhouse.yandex/docs/en/operations/table_engines/collapsingmergetree/) diff --git a/docs/en/operations/table_engines/graphitemergetree.md b/docs/en/operations/table_engines/graphitemergetree.md index bc557594d76..0aad07a13c3 100644 --- a/docs/en/operations/table_engines/graphitemergetree.md +++ b/docs/en/operations/table_engines/graphitemergetree.md @@ -4,56 +4,118 @@ This engine is designed for rollup (thinning and aggregating/averaging) [Graphite](http://graphite.readthedocs.io/en/latest/index.html) data. It may be helpful to developers who want to use ClickHouse as a data store for Graphite. -Graphite stores full data in ClickHouse, and data can be retrieved in the following ways: +You can use any ClickHouse table engine to store the Graphite data if you don't need rollup, but if you need a rollup use `GraphiteMergeTree`. The engine reduces the volume of storage and increases the efficiency of queries from Graphite. -- Without thinning. +The engine inherits properties from [MergeTree](mergetree.md#table_engines-mergetree). - Uses the [MergeTree](mergetree.md#table_engines-mergetree) engine. +## Creating a Table -- With thinning. +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + Path String, + Time DateTime, + Value , + Version + ... +) ENGINE = GraphiteMergeTree(config_section) +[PARTITION BY expr] +[ORDER BY expr] +[SAMPLE BY expr] +[SETTINGS name=value, ...] +``` - Using the `GraphiteMergeTree` engine. +For a description of request parameters, see [request description](../../query_language/create.md#query_language-queries-create_table). -The engine inherits properties from MergeTree. The settings for thinning data are defined by the [graphite_rollup](../server_settings/settings.md#server_settings-graphite_rollup) parameter in the server configuration. +A table for the Graphite date should have the following columns: -## Using The Engine +- Column with the metric name (Graphite sensor). Data type: `String`. +- Column with the time for measuring the metric. Data type: `DateTime`. +- Column with the value of the metric. Data type: any numeric. +- Column with the version of the metric with the same name and time of measurement. Data type: any numeric. -The Graphite data table must contain the following fields at minimum: + ClickHouse saves the rows with the highest version or the last written if versions are the same. Other rows are deleted during the merge of data parts. -- `Path` – The metric name (Graphite sensor). -- `Time` – The time for measuring the metric. -- `Value` – The value of the metric at the time set in Time. -- `Version` – Determines which value of the metric with the same Path and Time will remain in the database. +The names of these columns should be set in the rollup configuration. -Rollup pattern: +**GraphiteMergeTree parameters** + +- `config_section` — Name of the section in the configuration file, where are the rules of rollup set. + +**Query clauses** + +When creating a `GraphiteMergeTree` table, the same [clauses](mergetree.md#table_engines-mergetree-configuring) are required, as when creating a `MergeTree` table. + +
    Deprecated Method for Creating a Table + +!!! attention + Do not use this method in new projects and, if possible, switch the old projects to the method described above. + +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + EventDate Date, + Path String, + Time DateTime, + Value , + Version + ... +) ENGINE [=] GraphiteMergeTree(date-column [, sampling_expression], (primary, key), index_granularity, config_section) +``` + +All of the parameters excepting `config_section` have the same meaning as in `MergeTree`. + +- `config_section` — Name of the section in the configuration file, where are the rules of rollup set. +
    + +## Rollup configuration + +The settings for rollup are defined by the [graphite_rollup](../server_settings/settings.md#server_settings-graphite_rollup) 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: ``` +required-columns pattern regexp function - age -> precision + age + precision ... pattern ... default function - age -> precision + age + precision ... ``` -When processing a record, ClickHouse will check the rules in the `pattern`clause. If the metric name matches the `regexp`, the rules from `pattern` are applied; otherwise, the rules from `default` are used. +When processing a row, ClickHouse checks the rules in the `pattern` section. If the metric name matches the `regexp`, the rules from the `pattern`section are applied; otherwise, the rules from the `default` section are used. -Fields in the pattern. +The rules are defined with fields `function` and `age + precision`. + +Fields for `pattern` and `default` sections: -- `age` – The minimum age of the data in seconds. -- `function` – The name of the aggregating function to apply to data whose age falls within the range `[age, age + precision]`. -- `precision`– How precisely to define the age of the data in seconds. - `regexp`– A pattern for the metric name. +- `age` – The minimum age of the data in seconds. +- `precision`– How precisely to define the age of the data in seconds. +- `function` – The name of the aggregating function to apply to data whose age falls within the range `[age, age + precision]`. + +The `required-columns`: + +- `path_column_name` — Column with the metric name (Graphite sensor). +- `time_column_name` — Column with the time for measuring the metric. +- `value_column_name` — Column with the value of the metric at the time set in `time_column_name`. +- `version_column_name` — Column with the version timestamp of the metric with the same name and time remains in the database. + Example of settings: ```xml + Path + Time + Value + Version click_cost any @@ -84,5 +146,4 @@ Example of settings: ``` - [Original article](https://clickhouse.yandex/docs/en/operations/table_engines/graphitemergetree/) diff --git a/docs/en/operations/table_engines/mergetree.md b/docs/en/operations/table_engines/mergetree.md index 35cb660f56d..7b4ecd51fe7 100644 --- a/docs/en/operations/table_engines/mergetree.md +++ b/docs/en/operations/table_engines/mergetree.md @@ -4,8 +4,7 @@ The `MergeTree` engine and other engines of this family (`*MergeTree`) are the most robust ClickHousе table engines. -!!! info - The [Merge](merge.md#table_engine-merge) engine does not belong to the `*MergeTree` family. +The basic idea for `MergeTree` engines family is the following. When you have tremendous amount of a data that should be inserted into the table, you should write them quickly part by part and then merge parts by some rules in background. This method is much more efficient than constantly rewriting data in the storage at the insert. Main features: @@ -25,29 +24,47 @@ Main features: If necessary, you can set the data sampling method in the table. -## Engine Configuration When Creating a Table +!!! info + The [Merge](merge.md#table_engine-merge) engine does not belong to the `*MergeTree` family. + + + +## Creating a Table ``` -ENGINE [=] MergeTree() [PARTITION BY expr] [ORDER BY expr] [SAMPLE BY expr] [SETTINGS name=value, ...] +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE = MergeTree() +[PARTITION BY expr] +[ORDER BY expr] +[SAMPLE BY expr] +[SETTINGS name=value, ...] ``` -**ENGINE clauses** +For a description of request parameters, see [request description](../../query_language/create.md#query_language-queries-create_table). + +**Query clauses** + +- `ENGINE` - Name and parameters of the engine. `ENGINE = MergeTree()`. `MergeTree` engine does not have parameters. - `ORDER BY` — Primary key. A tuple of columns or arbitrary expressions. Example: `ORDER BY (CounterID, EventDate)`. -If a sampling key is used, the primary key must contain it. Example: `ORDER BY (CounerID, EventDate, intHash32(UserID))`. +If a sampling expression is used, the primary key must contain it. Example: `ORDER BY (CounerID, EventDate, intHash32(UserID))`. - `PARTITION BY` — The [partitioning key](custom_partitioning_key.md#table_engines-custom_partitioning_key). For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](../../data_types/date.md#data_type-date). The partition names here have the `"YYYYMM"` format. -- `SAMPLE BY` — An expression for sampling (optional). Example: `intHash32(UserID))`. +- `SAMPLE BY` — An expression for sampling. Example: `intHash32(UserID))`. -- `SETTINGS` — Additional parameters that control the behavior of the `MergeTree` (optional): +- `SETTINGS` — Additional parameters that control the behavior of the `MergeTree`: - `index_granularity` — The granularity of an index. The number of data rows between the "marks" of an index. By default, 8192. -**Example** +**Example of sections setting** ``` ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192 @@ -59,13 +76,18 @@ We also set an expression for sampling as a hash by the user ID. This allows you `index_granularity` could be omitted because 8192 is the default value. -### Deprecated Method for Engine Configuration +
    Deprecated Method for Creating a Table !!! attention Do not use this method in new projects and, if possible, switch the old projects to the method described above. ``` -ENGINE [=] MergeTree(date-column [, sampling_expression], (primary, key), index_granularity) +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE [=] MergeTree(date-column [, sampling_expression], (primary, key), index_granularity) ``` **MergeTree() parameters** @@ -82,6 +104,7 @@ MergeTree(EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID) ``` The `MergeTree` engine is configured in the same way as in the example above for the main engine configuration method. +
    ## Data Storage @@ -89,7 +112,7 @@ A table consists of data *parts* sorted by primary key. When data is inserted in a table, separate data parts are created and each of them is lexicographically sorted by primary key. For example, if the primary key is `(CounterID, Date)`, the data in the part is sorted by `CounterID`, and within each `CounterID`, it is ordered by `Date`. -Data belonging to different partitions are separated into different parts. In the background, ClickHouse merges data parts for more efficient storage. Parts belonging to different partitions are not merged. +Data belonging to different partitions are separated into different parts. In the background, ClickHouse merges data parts for more efficient storage. Parts belonging to different partitions are not merged. The merge mechanism does not guarantee that all rows with the same primary key will be in the same data part. For each data part, ClickHouse creates an index file that contains the primary key value for each index row ("mark"). Index row numbers are defined as `n * index_granularity`. The maximum value `n` is equal to the integer part of dividing the total number of rows by the `index_granularity`. For each column, the "marks" are also written for the same index rows as the primary key. These "marks" allow you to find the data directly in the columns. @@ -136,13 +159,13 @@ The number of columns in the primary key is not explicitly limited. Depending on ClickHouse sorts data by primary key, so the higher the consistency, the better the compression. -- To provide additional logic when merging in the [CollapsingMergeTree](collapsingmergetree.md#table_engine-collapsingmergetree) and [SummingMergeTree](summingmergetree.md#table_engine-summingmergetree) engines. +- Provide additional logic when data parts merging in the [CollapsingMergeTree](collapsingmergetree.md#table_engine-collapsingmergetree) and [SummingMergeTree](summingmergetree.md#table_engine-summingmergetree) engines. - You may need to have many fields in the primary key even if they are not necessary for the previous steps. + You may need many fields in the primary key even if they are not necessary for the previous steps. 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. -### Usage of Indexes and Partitions in Queries +### Use of Indexes and Partitions in Queries For`SELECT` queries, ClickHouse analyzes whether an index can be used. An index can be used if the `WHERE/PREWHERE` clause has an expression (as one of the conjunction elements, or entirely) that represents an equality or inequality comparison operation, or if it has `IN` or `LIKE` with a fixed prefix on columns or expressions that are in the primary key or partitioning key, or on certain partially repetitive functions of these columns, or logical relationships of these expressions. diff --git a/docs/en/operations/table_engines/replacingmergetree.md b/docs/en/operations/table_engines/replacingmergetree.md index 482f49982d0..7a4f3ab7443 100644 --- a/docs/en/operations/table_engines/replacingmergetree.md +++ b/docs/en/operations/table_engines/replacingmergetree.md @@ -1,20 +1,58 @@ # ReplacingMergeTree -This engine table differs from `MergeTree` in that it removes duplicate entries with the same primary key value. +The engine differs from [MergeTree](mergetree.md#table_engines-mergetree) in that it removes duplicate entries with the same primary key value. -The last optional parameter for the table engine is the version column. When merging, it reduces all rows with the same primary key value to just one row. If the version column is specified, it leaves the row with the highest version; otherwise, it leaves the last row. - -The version column must have a type from the `UInt` family, `Date`, or `DateTime`. - -``` sql -ReplacingMergeTree(EventDate, (OrderID, EventDate, BannerID, ...), 8192, ver) -``` - -Note that data is only deduplicated during merges. Merging occurs in the background at an unknown time, so you can't plan for it. Some of the data may remain unprocessed. Although you can run an unscheduled merge using the OPTIMIZE query, don't count on using it, because the OPTIMIZE query will read and write a large amount of data. +Data deduplication occurs only during a merge. Merging occurs in the background at an unknown time, so you can't plan for it. Some of the data may remain unprocessed. Although you can run an unscheduled merge using the `OPTIMIZE` query, don't count on using it, because the `OPTIMIZE` query will read and write a large amount of data. Thus, `ReplacingMergeTree` is suitable for clearing out duplicate data in the background in order to save space, but it doesn't guarantee the absence of duplicates. -*This engine is not used in Yandex.Metrica, but it has been applied in other Yandex projects.* +## Creating a Table +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE = ReplacingMergeTree([ver]) +[PARTITION BY expr] +[ORDER BY expr] +[SAMPLE BY expr] +[SETTINGS name=value, ...] +``` + +For a description of request parameters, see [request description](../../query_language/create.md#query_language-queries-create_table). + +**ReplacingMergeTree Parameters** + +- `ver` — column with version. Type `UInt*`, `Date` or `DateTime`. Optional parameter. + + When merging, `ReplacingMergeTree` from all the rows with the same primary key leaves only one: + - Last in the selection, if `ver` not set. + - With the maximum version, if `ver` specified. + +**Query clauses** + +When creating a `ReplacingMergeTree` table the same [clauses](mergetree.md#table_engines-mergetree-configuring) are required, as when creating a `MergeTree` table. + +
    Deprecated Method for Creating a Table + +!!! attention + Do not use this method in new projects and, if possible, switch the old projects to the method described above. + +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE [=] ReplacingMergeTree(date-column [, sampling_expression], (primary, key), index_granularity, [ver]) +``` + +All of the parameters excepting `ver` have the same meaning as in `MergeTree`. + + +- `ver` - column with the version. Optional parameter. For a description, see the text above. +
    [Original article](https://clickhouse.yandex/docs/en/operations/table_engines/replacingmergetree/) diff --git a/docs/en/operations/table_engines/replication.md b/docs/en/operations/table_engines/replication.md index 9574cbaac82..01245edf744 100644 --- a/docs/en/operations/table_engines/replication.md +++ b/docs/en/operations/table_engines/replication.md @@ -74,15 +74,37 @@ The system monitors data synchronicity on replicas and is able to recover after The `Replicated` prefix is added to the table engine name. For example:`ReplicatedMergeTree`. -Two parameters are also added in the beginning of the parameters list – the path to the table in ZooKeeper, and the replica name in ZooKeeper. +**Replicated\*MergeTree parameters** + +- `zoo_path` — The path to the table in ZooKeeper. +- `replica_name` — The replica name in ZooKeeper. Example: -``` -ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/hits', '{replica}', EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID), EventTime), 8192) +```sql +CREATE TABLE table_name +( + EventDate DateTime, + CounterID UInt32, + UserID UInt32 +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/hits', '{replica}') +PARTITION BY toYYYYMM(EventDate) +ORDER BY (CounterID, EventDate, intHash32(UserID)) +SAMPLE BY intHash32(UserID) ``` -As the example shows, these parameters can contain substitutions in curly brackets. The substituted values are taken from the 'macros' section of the config file. Example: +Example in deprecated syntax: + +```sql +CREATE TABLE table_name +( + EventDate DateTime, + CounterID UInt32, + UserID UInt32 +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/hits', '{replica}', EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID), EventTime), 8192) +``` + +As the example shows, these parameters can contain substitutions in curly brackets. The substituted values are taken from the 'macros' section of the configuration file. Example: ```xml diff --git a/docs/en/operations/table_engines/summingmergetree.md b/docs/en/operations/table_engines/summingmergetree.md index 82df821804a..a2d89bfa4c8 100644 --- a/docs/en/operations/table_engines/summingmergetree.md +++ b/docs/en/operations/table_engines/summingmergetree.md @@ -2,33 +2,122 @@ # SummingMergeTree -This engine differs from `MergeTree` in that it totals data while merging. +The engine inherits from [MergeTree](mergetree.md#table_engines-mergetree). The difference is that when merging data parts for `SummingMergeTree` tables ClickHouse replaces all the rows with the same primary key with one row which contains summarized values for the columns with the numeric data type. If the primary key is composed in a way that a single key value corresponds to large number of rows, this significantly reduces storage volume and speeds up data selection. -``` sql -SummingMergeTree(EventDate, (OrderID, EventDate, BannerID, ...), 8192) +We recommend to use the engine together with `MergeTree`. Store complete data in `MergeTree` table, and use `SummingMergeTree` for aggregated data storing, for example, when preparing reports. Such an approach will prevent you from losing valuable data due to an incorrectly composed primary key. + +## Creating a Table + +``` +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE = MergeTree() +[PARTITION BY expr] +[ORDER BY expr] +[SAMPLE BY expr] +[SETTINGS name=value, ...] ``` -The columns to total are implicit. When merging, all rows with the same primary key value (in the example, OrderId, EventDate, BannerID, ...) have their values totaled in numeric columns that are not part of the primary key. +For a description of request parameters, see [request description](../../query_language/create.md#query_language-queries-create_table). -``` sql -SummingMergeTree(EventDate, (OrderID, EventDate, BannerID, ...), 8192, (Shows, Clicks, Cost, ...)) +**Parameters of SummingMergeTree** + +- `columns` - a tuple with the names of columns where values will be summarized. Optional parameter. +The columns must be of a numeric type and must not be in the primary key. + + If `columns` not specified, ClickHouse summarizes the values in all columns with a numeric data type that are not in the primary key. + +**Query clauses** + +When creating a `SummingMergeTree` table the same [clauses](mergetree.md#table_engines-mergetree-configuring) are required, as when creating a `MergeTree` table. + +
    Deprecated Method for Creating a Table + +!!! attention + Do not use this method in new projects and, if possible, switch the old projects to the method described above. + +``` +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE [=] SummingMergeTree(date-column [, sampling_expression], (primary, key), index_granularity, [columns]) ``` -The columns to total are set explicitly (the last parameter – Shows, Clicks, Cost, ...). When merging, all rows with the same primary key value have their values totaled in the specified columns. The specified columns also must be numeric and must not be part of the primary key. +All of the parameters excepting `columns` have the same meaning as in `MergeTree`. -If the values were zero in all of these columns, the row is deleted. +- `columns` — tuple with names of columns values of which will be summarized. Optional parameter. For a description, see the text above. +
    -For the other columns that are not part of the primary key, the first value that occurs is selected when merging. But for the AggregateFunction type of columns, aggregation is performed according to the set function, so this engine actually behaves like `AggregatingMergeTree`. +## Usage Example -Summation is not performed for a read operation. If it is necessary, write the appropriate GROUP BY. +Consider the following table: -In addition, a table can have nested data structures that are processed in a special way. -If the name of a nested table ends in 'Map' and it contains at least two columns that meet the following criteria: +```sql +CREATE TABLE summtt +( + key UInt32, + value UInt32 +) +ENGINE = SummingMergeTree() +ORDER BY key +``` -- The first table is numeric ((U)IntN, Date, DateTime), which we'll refer to as the 'key'. -- The other columns are arithmetic ((U)IntN, Float32/64), which we'll refer to as '(values...)'. +Insert data to it: -Then this nested table is interpreted as a mapping of key `=>` (values...), and when merging its rows, the elements of two data sets are merged by 'key' with a summation of the corresponding (values...). +``` +:) INSERT INTO summtt Values(1,1),(1,2),(2,1) +``` + +ClickHouse may sum all the rows not completely ([see below](#summary-data-processing)), so we use an aggregate function `sum` and `GROUP BY` clause in the query. + +```sql +SELECT key, sum(value) FROM summtt GROUP BY key +``` + +``` +┌─key─┬─sum(value)─┐ +│ 2 │ 1 │ +│ 1 │ 3 │ +└─────┴────────────┘ +``` + + + +## Data Processing + +When data are inserted into a table, they are saved as-is. Clickhouse merges the inserted parts of data periodically and this is when rows with the same primary key are summed and replaced with one for each resulting part of data. + +ClickHouse can merge the data parts so that different resulting parts of data cat consist rows with the same primary key, i.e. the summation will be incomplete. Therefore (`SELECT`) an aggregate function [sum()](../../query_language/agg_functions/reference.md#agg_function-sum) and `GROUP BY` clause should be used in a query as described in the example above. + +### Common rules for summation + +The values in the columns with the numeric data type are summarized. The set of columns is defined by the parameter `columns`. + +If the values were 0 in all of the columns for summation, the row is deleted. + +If column is not in the primary key and is not summarized, an arbitrary value is selected from the existing ones. + +The values are not summarized for columns in the primary key. + +### The Summation in the AggregateFunction Columns + +For columns of [AggregateFunction type](../../data_types/nested_data_structures/aggregatefunction.md#data_type-aggregatefunction) ClickHouse behaves as [AggregatingMergeTree](aggregatingmergetree.md#table_engine-aggregatingmergetree) engine aggregating according to the function. + +### Nested Structures + +Table can have nested data structures that are processed in a special way. + +If the name of a nested table ends with `Map` and it contains at least two columns that meet the following criteria: + +- the first column is numeric `(*Int*, Date, DateTime)`, let's call it `key`, +- the other columns are arithmetic `(*Int*, Float32/64)`, let's call it `(values...)`, + +then this nested table is interpreted as a mapping of `key => (values...)`, and when merging its rows, the elements of two data sets are merged by `key` with a summation of the corresponding `(values...)`. Examples: @@ -39,11 +128,8 @@ Examples: [(1, 100), (2, 150)] + [(1, -100)] -> [(2, 150)] ``` -For aggregation of Map, use the function sumMap(key, value). - -For nested data structures, you don't need to specify the columns as a list of columns for totaling. - -This table engine is not particularly useful. Remember that when saving just pre-aggregated data, you lose some of the system's advantages. +When requesting data, use the [sumMap(key, value)](../../query_language/agg_functions/reference.md#agg_function-summary) function for aggregation of `Map`. +For nested data structure, you do not need to specify its columns in the tuple of columns for summation. [Original article](https://clickhouse.yandex/docs/en/operations/table_engines/summingmergetree/) diff --git a/docs/en/query_language/agg_functions/combinators.md b/docs/en/query_language/agg_functions/combinators.md index cf4f73fc11a..b63beee4a27 100644 --- a/docs/en/query_language/agg_functions/combinators.md +++ b/docs/en/query_language/agg_functions/combinators.md @@ -4,6 +4,8 @@ The name of an aggregate function can have a suffix appended to it. This changes the way the aggregate function works. + + ## -If The suffix -If can be appended to the name of any aggregate function. In this case, the aggregate function accepts an extra argument – a condition (Uint8 type). The aggregate function processes only the rows that trigger the condition. If the condition was not triggered even once, it returns a default value (usually zeros or empty strings). @@ -24,7 +26,7 @@ Example 2: `uniqArray(arr)` – Count the number of unique elements in all 'arr' ## -State -If you apply this combinator, the aggregate function doesn't return the resulting value (such as the number of unique values for the 'uniq' function), but an intermediate state of the aggregation (for `uniq`, this is the hash table for calculating the number of unique values). This is an AggregateFunction(...) that can be used for further processing or stored in a table to finish aggregating later. See the sections "AggregatingMergeTree" and "Functions for working with intermediate aggregation states". +If you apply this combinator, the aggregate function doesn't return the resulting value (such as the number of unique values for the `uniq` function), but an intermediate state of the aggregation (for `uniq`, this is the hash table for calculating the number of unique values). This is an AggregateFunction(...) that can be used for further processing or stored in a table to finish aggregating later. See the sections "AggregatingMergeTree" and "Functions for working with intermediate aggregation states". ## -Merge diff --git a/docs/en/query_language/agg_functions/reference.md b/docs/en/query_language/agg_functions/reference.md index fd0bb213933..c7268393408 100644 --- a/docs/en/query_language/agg_functions/reference.md +++ b/docs/en/query_language/agg_functions/reference.md @@ -9,6 +9,8 @@ The syntax `COUNT(DISTINCT x)` is not supported. The separate `uniq` aggregate f A `SELECT count() FROM table` query is not optimized, because the number of entries in the table is not stored separately. It will select some small column from the table and count the number of values in it. + + ## any(x) Selects the first encountered value. @@ -82,6 +84,8 @@ SELECT argMin(user, salary) FROM salary Calculates the 'arg' value for a maximum 'val' value. If there are several different values of 'arg' for maximum values of 'val', the first of these values encountered is output. + + ## sum(x) Calculates the sum. @@ -93,6 +97,8 @@ Computes the sum of the numbers, using the same data type for the result as for Only works for numbers. + + ## sumMap(key, value) Totals the 'value' array according to the keys specified in the 'key' array. @@ -135,6 +141,8 @@ Calculates the average. Only works for numbers. The result is always Float64. + + ## uniq(x) Calculates the approximate number of different values of the argument. Works for numbers, strings, dates, date-with-time, and for multiple arguments and tuple arguments. @@ -268,7 +276,7 @@ A hash table is used as the algorithm. Because of this, if the passed values ​ Approximates the quantile level using the [t-digest](https://github.com/tdunning/t-digest/blob/master/docs/t-digest-paper/histo.pdf) algorithm. The maximum error is 1%. Memory consumption by State is proportional to the logarithm of the number of passed values. -The performance of the function is lower than for `quantile`, `quantileTiming`. In terms of the ratio of State size to precision, this function is much better than `quantile`. +The performance of the function is lower than for `quantile` or `quantileTiming`. In terms of the ratio of State size to precision, this function is much better than `quantile`. The result depends on the order of running the query, and is nondeterministic. diff --git a/docs/en/query_language/create.md b/docs/en/query_language/create.md index 4956955b30b..6d9e71d3500 100644 --- a/docs/en/query_language/create.md +++ b/docs/en/query_language/create.md @@ -15,8 +15,8 @@ If `IF NOT EXISTS` is included, the query won't return an error if the database The `CREATE TABLE` query can have several forms. -``` sql -CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ( name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], @@ -31,19 +31,21 @@ A column description is `name type` in the simplest case. Example: `RegionID UIn Expressions can also be defined for default values (see below). ``` sql -CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name AS [db2.]name2 [ENGINE = engine] +CREATE TABLE [IF NOT EXISTS] [db.]table_name AS [db2.]name2 [ENGINE = engine] ``` Creates a table with the same structure as another table. You can specify a different engine for the table. If the engine is not specified, the same engine will be used as for the `db2.name2` table. ``` sql -CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name ENGINE = engine AS SELECT ... +CREATE TABLE [IF NOT EXISTS] [db.]table_name ENGINE = engine AS SELECT ... ``` Creates a table with a structure like the result of the `SELECT` query, with the 'engine' engine, and fills it with data from SELECT. In all cases, if `IF NOT EXISTS` is specified, the query won't return an error if the table already exists. In this case, the query won't do anything. +There can be other clauses after the `ENGINE` clause in the query. See detailed documentation on how to create tables in the descriptions of [table engines](../operations/table_engines/index.md#table_engines). + ### Default Values The column description can specify an expression for a default value, in one of the following ways:`DEFAULT expr`, `MATERIALIZED expr`, `ALIAS expr`. @@ -81,18 +83,28 @@ It is not possible to set default values for elements in nested data structures. ### Temporary Tables -In all cases, if `TEMPORARY` is specified, a temporary table will be created. Temporary tables have the following characteristics: +ClickHouse supports temporary tables which have the following characteristics: - Temporary tables disappear when the session ends, including if the connection is lost. -- A temporary table is created with the Memory engine. The other table engines are not supported. +- A temporary table use the Memory engine only. - The DB can't be specified for a temporary table. It is created outside of databases. - If a temporary table has the same name as another one and a query specifies the table name without specifying the DB, the temporary table will be used. - For distributed query processing, temporary tables used in a query are passed to remote servers. +To create a temporary table, use the following syntax: + +```sql +CREATE TEMPORARY TABLE [IF NOT EXISTS] table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) +``` + In most cases, temporary tables are not created manually, but when using external data for a query, or for distributed `(GLOBAL) IN`. For more information, see the appropriate sections -Distributed DDL queries (ON CLUSTER clause) ----------------------------------------------- +## Distributed DDL queries (ON CLUSTER clause) The `CREATE`, `DROP`, `ALTER`, and `RENAME` queries support distributed execution on a cluster. For example, the following query creates the `all_hits` `Distributed` table on each host in `cluster`: @@ -108,7 +120,7 @@ The local version of the query will eventually be implemented on each host in th ## CREATE VIEW ``` sql -CREATE [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]name [TO[db.]name] [ENGINE = engine] [POPULATE] AS SELECT ... +CREATE [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]table_name [TO[db.]name] [ENGINE = engine] [POPULATE] AS SELECT ... ``` Creates a view. There are two types of views: normal and MATERIALIZED. diff --git a/docs/en/query_language/functions/functions_for_nulls.md b/docs/en/query_language/functions/functions_for_nulls.md index 2b088b5321e..d25de3332e5 100644 --- a/docs/en/query_language/functions/functions_for_nulls.md +++ b/docs/en/query_language/functions/functions_for_nulls.md @@ -8,7 +8,7 @@ Checks whether the argument is [NULL](../syntax.md#null-literal). isNull(x) ``` -**Parameters:** +**Parameters** - `x` — A value with a non-compound data type. diff --git a/docs/en/query_language/insert_into.md b/docs/en/query_language/insert_into.md index d34e306777b..8af0c949191 100644 --- a/docs/en/query_language/insert_into.md +++ b/docs/en/query_language/insert_into.md @@ -12,7 +12,7 @@ INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), . The query can specify a list of columns to insert `[(c1, c2, c3)]`. In this case, the rest of the columns are filled with: -- The values calculated from the `DEFAULT` expressions specified in the table definition. +- 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#settings-strict_insert_defaults), columns that do not have `DEFAULT` defined must be listed in the query. @@ -41,6 +41,8 @@ INSERT INTO t FORMAT TabSeparated You can insert data separately from the query by using the command-line client or the HTTP interface. For more information, see the section "[Interfaces](../interfaces/index.md#interfaces)". + + ### Inserting The Results of `SELECT` ``` sql diff --git a/docs/en/query_language/table_functions/file.md b/docs/en/query_language/table_functions/file.md index 8d1c336b049..e7c7e5b8e89 100644 --- a/docs/en/query_language/table_functions/file.md +++ b/docs/en/query_language/table_functions/file.md @@ -2,16 +2,51 @@ # file -`file(path, format, structure)` - returns a table created from a path file with a format type, with columns specified in structure. +Creates a table from a file. -path - a relative path to a file from [user_files_path](../../operations/server_settings/settings.md#user_files_path). +``` +file(path, format, structure) +``` -format - file [format](../../interfaces/formats.md#formats). +**Input parameters** -structure - table structure in 'UserID UInt64, URL String' format. Determines column names and types. +- `path` — The relative path to the file from [user_files_path](../../operations/server_settings/settings.md#user_files_path). +- `format` — The [format](../../interfaces/formats.md#formats) of the file. +- `structure` — Structure of the table. Format `'colunmn1_name column1_ype, column2_name column2_type, ...'`. + +**Returned value** + +A table with the specified structure for reading or writing data in the specified file. **Example** +Setting `user_files_path` and the contents of the file `test.csv`: + +```bash +$ grep user_files_path /etc/clickhouse-server/config.xml + /var/lib/clickhouse/user_files/ + +$ cat /var/lib/clickhouse/user_files/test.csv + 1,2,3 + 3,2,1 + 78,43,45 +``` + +Table from`test.csv` and selection of the first two rows from it: + +```sql +SELECT * +FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32') +LIMIT 2 +``` + +``` +┌─column1─┬─column2─┬─column3─┐ +│ 1 │ 2 │ 3 │ +│ 3 │ 2 │ 1 │ +└─────────┴─────────┴─────────┘ +``` + ``` sql -- getting the first 10 lines of a table that contains 3 columns of UInt32 type from a CSV file SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32') LIMIT 10 diff --git a/docs/ru/data_types/nested_data_structures/aggregatefunction.md b/docs/ru/data_types/nested_data_structures/aggregatefunction.md index 51acc27b299..384a432b494 100644 --- a/docs/ru/data_types/nested_data_structures/aggregatefunction.md +++ b/docs/ru/data_types/nested_data_structures/aggregatefunction.md @@ -1,5 +1,65 @@ + + # AggregateFunction(name, types_of_arguments...) -Промежуточное состояние агрегатной функции. Чтобы его получить, используются агрегатные функции с суффиксом -State. Подробнее смотрите в разделе "AggregatingMergeTree". +Промежуточное состояние агрегатной функции. Чтобы его получить, используются агрегатные функции с суффиксом `-State`. Чтобы в дальнейшем получить агрегированные данные необходимо использовать те же агрегатные функции с суффиксом `-Merge`. + +`AggregateFunction` — параметрический тип данных. + +**Параметры** + +- Имя агрегатной функции. + + Для параметрических агрегатных функций указываются также их параметры. + +- Типы аргументов агрегатной функции. + +**Пример** + +```sql +CREATE TABLE t +( + column1 AggregateFunction(uniq, UInt64), + column2 AggregateFunction(anyIf, String, UInt8), + column3 AggregateFunction(quantiles(0.5, 0.9), UInt64) +) ENGINE = ... +``` + +[uniq](../../query_language/agg_functions/reference.md#agg_function-uniq), anyIf ([any](../../query_language/agg_functions/reference.md#agg_function-any)+[If](../../query_language/agg_functions/combinators.md#agg-functions-combinator-if)) и [quantiles](../../query_language/agg_functions/reference.md#agg_function-quantiles) — агрегатные функции, поддержанные в ClickHouse. + +## Особенности использования + +### Вставка данных + +Для вставки данных используйте `INSERT SELECT` с агрегатными `-State`-функциями. + +**Примеры функций** + +``` +uniqState(UserID) +quantilesState(0.5, 0.9)(SendTiming) +``` + +В отличие от соответствующих функций `uniq` и `quantiles`, `-State`-функциями возвращают не готовое значение, а состояние. То есть, значение типа `AggregateFunction`. + +В запросах `SELECT` значения типа `AggregateFunction` выводятся во всех форматах, которые поддерживает ClickHouse, в виде implementation-specific бинарных данных. Если с помощью `SELECT` выполнить дамп данных, например, в формат `TabSeparated`, то потом этот дамп можно загрузить обратно с помощью запроса `INSERT`. + +### Выборка данных + +При выборке данных из таблицы `AggregatingMergeTree`, используйте `GROUP BY` и те же агрегатные функции, что и при вставке данных, но с суффиксом `-Merge`. + +Агрегатная функция с суффиксом `-Merge` берёт множество состояний, объединяет их, и возвращает результат полной агрегации данных. + +Например, следующие два запроса возвращают один и тот же результат: + +```sql +SELECT uniq(UserID) FROM table + +SELECT uniqMerge(state) FROM (SELECT uniqState(UserID) AS state FROM table GROUP BY RegionID) +``` + +## Пример использования + +Смотрите в описании движка [AggregatingMergeTree](../../operations/table_engines/aggregatingmergetree.md#table_engine-aggregatingmergetree). [Оригинальная статья](https://clickhouse.yandex/docs/ru/data_types/nested_data_structures/aggregatefunction/) diff --git a/docs/ru/operations/configuration_files.md b/docs/ru/operations/configuration_files.md index 563696a78fe..c99351f7308 100644 --- a/docs/ru/operations/configuration_files.md +++ b/docs/ru/operations/configuration_files.md @@ -16,7 +16,7 @@ Также в конфиге могут быть указаны "подстановки". Если у элемента присутствует атрибут `incl`, то в качестве значения будет использована соответствующая подстановка из файла. По умолчанию, путь к файлу с подстановками - `/etc/metrika.xml`. Он может быть изменён в конфигурации сервера в элементе [include_from](server_settings/settings.md#server_settings-include_from). Значения подстановок указываются в элементах `/yandex/имя_подстановки` этого файла. Если подстановка, заданная в `incl` отсутствует, то в лог попадает соответствующая запись. Чтобы ClickHouse не писал в лог об отсутствии подстановки, необходимо указать атрибут `optional="true"` (например, настройка [macros](server_settings/settings.md#server_settings-macros)). -Подстановки могут также выполняться из ZooKeeper. Для этого укажите у элемента атрибут `from_zk="/path/to/node"`. Значение элемента заменится на содержимое узла `/path/to/node` в ZooKeeper. В ZooKeeper-узел также можно положить целое XML-поддерево, оно будет целиком вставлено в исходный элемент. +Подстановки могут также выполняться из ZooKeeper. Для этого укажите у элемента атрибут `from_zk = "/path/to/node"`. Значение элемента заменится на содержимое узла `/path/to/node` в ZooKeeper. В ZooKeeper-узел также можно положить целое XML-поддерево, оно будет целиком вставлено в исходный элемент. В `config.xml` может быть указан отдельный конфиг с настройками пользователей, профилей и квот. Относительный путь к нему указывается в элементе users_config. По умолчанию - `users.xml`. Если `users_config` не указан, то настройки пользователей, профилей и квот, указываются непосредственно в `config.xml`. diff --git a/docs/ru/operations/table_engines/aggregatingmergetree.md b/docs/ru/operations/table_engines/aggregatingmergetree.md index 1cc880e1662..19ea9f9ff1b 100644 --- a/docs/ru/operations/table_engines/aggregatingmergetree.md +++ b/docs/ru/operations/table_engines/aggregatingmergetree.md @@ -1,60 +1,69 @@ + + # AggregatingMergeTree -Отличается от `MergeTree` тем, что при слиянии, выполняет объединение состояний агрегатных функций, хранимых в таблице, для строчек с одинаковым значением первичного ключа. - -Чтобы это работало, используются: тип данных `AggregateFunction`, а также модификаторы `-State` и `-Merge` для агрегатных функций. Рассмотрим подробнее. - -Существует тип данных `AggregateFunction`. Это параметрический тип данных. В качестве параметров передаются: имя агрегатной функции, затем типы её аргументов. - -Примеры: - -``` sql -CREATE TABLE t -( - column1 AggregateFunction(uniq, UInt64), - column2 AggregateFunction(anyIf, String, UInt8), - column3 AggregateFunction(quantiles(0.5, 0.9), UInt64) -) ENGINE = ... -``` - -Столбец такого типа хранит состояние агрегатной функции. - -Чтобы получить значение такого типа, следует использовать агрегатные функции с суффиксом `State`. - -Пример: -`uniqState(UserID), quantilesState(0.5, 0.9)(SendTiming)` - -В отличие от соответствующих функций `uniq`, `quantiles`, такие функции возвращают не готовое значение, а состояние. То есть, значение типа `AggregateFunction`. - -Значение типа `AggregateFunction` нельзя вывести в Pretty-форматах. В других форматах, значения такого типа выводятся в виде implementation-specific бинарных данных. То есть, значения типа `AggregateFunction` не предназначены для вывода, сохранения в дамп. - -Единственную полезную вещь, которую можно сделать со значениями типа `AggregateFunction` — это объединить состояния и получить результат, по сути — доагрегировать до конца. Для этого используются агрегатные функции с суффиксом Merge. -Пример: `uniqMerge(UserIDState)`, где `UserIDState` имеет тип `AggregateFunction`. - -То есть, агрегатная функция с суффиксом Merge берёт множество состояний, объединяет их, и возвращает готовый результат. -Для примера, эти два запроса возвращают один и тот же результат: - -``` sql -SELECT uniq(UserID) FROM table - -SELECT uniqMerge(state) FROM (SELECT uniqState(UserID) AS state FROM table GROUP BY RegionID) -``` - -Существует движок `AggregatingMergeTree`. Он занимается тем, что при слияниях, выполняет объединение состояний агрегатных функций из разных строчек таблицы с одним значением первичного ключа. - -В таблицу, содержащую столбцы типа `AggregateFunction` невозможно вставить строчку обычным запросом INSERT, так как невозможно явно указать значение типа `AggregateFunction`. Вместо этого, для вставки данных, следует использовать `INSERT SELECT` с агрегатными функциями `-State`. - -При SELECT-е из таблицы `AggregatingMergeTree`, используйте GROUP BY и агрегатные функции с модификатором -Merge, чтобы доагрегировать данные. +Движок наследует функциональность [MergeTree](mergetree.md#table_engines-mergetree), изменяя логику слияния кусков данных. Все строки с одинаковым первичным ключом ClickHouse заменяет на одну (в пределах одного куска данных), которая хранит объединение состояний агрегатных функций. Таблицы типа `AggregatingMergeTree` могут использоваться для инкрементальной агрегации данных, в том числе, для агрегирующих материализованных представлений. -Пример: +Движок обрабатывает все столбцы типа [AggregateFunction](../../data_types/nested_data_structures/aggregatefunction.md#data_type-aggregatefunction). + +Использование `AggregatingMergeTree` оправдано только в том случае, когда это уменьшает количество строк на порядки. + +## Создание таблицы + +``` +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE = AggregatingMergeTree() +[PARTITION BY expr] +[ORDER BY expr] +[SAMPLE BY expr] +[SETTINGS name=value, ...] +``` + +Описание параметров запроса смотрите в [описании запроса](../../query_language/create.md#query_language-queries-create_table). + +**Секции запроса** + +При создании таблицы `AggregatingMergeTree` используются те же [секции](mergetree.md#table_engines-mergetree-configuring), что и при создании таблицы `MergeTree`. + + +
    Устаревший способ создания таблицы + +!!! attention + Не используйте этот способ в новых проектах и по возможности переведите старые проекты на способ описанный выше. + +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE [=] AggregatingMergeTree(date-column [, sampling_expression], (primary, key), index_granularity) +``` + +Все параметры имеют то же значение, что в и `MergeTree`. +
    + +## SELECT/INSERT данных + +Для вставки данных используйте `INSERT SELECT` с агрегатными `-State`-функциями. + +При выборке данных из таблицы `AggregatingMergeTree`, используйте `GROUP BY` и те же агрегатные функции, что и при вставке данных, но с суффиксом `-Merge`. + +В запросах `SELECT` значения типа `AggregateFunction` выводятся во всех форматах, которые поддерживает ClickHouse, в виде implementation-specific бинарных данных. Если с помощью `SELECT` выполнить дамп данных, например, в формат `TabSeparated`, то потом этот дамп можно загрузить обратно с помощью запроса `INSERT`. + +## Пример агрегирущего материализованного представления Создаём материализованное представление типа `AggregatingMergeTree`, следящее за таблицей `test.visits`: ``` sql CREATE MATERIALIZED VIEW test.basic -ENGINE = AggregatingMergeTree(StartDate, (CounterID, StartDate), 8192) +ENGINE = AggregatingMergeTree() PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate) AS SELECT CounterID, StartDate, @@ -64,13 +73,15 @@ FROM test.visits GROUP BY CounterID, StartDate; ``` -Вставляем данные в таблицу `test.visits`. Данные будут также вставлены в представление, где они будут агрегированы: +Вставляем данные в таблицу `test.visits`: ``` sql INSERT INTO test.visits ... ``` -Делаем `SELECT` из представления, используя `GROUP BY`, чтобы доагрегировать данные: +Данные окажутся и в таблице и в представлении `test.basic`, которое выполнит агрегацию. + +Чтобы получить агрегированные данные, выполним запрос вида `SELECT ... GROUP BY ...` из представления `test.basic`: ``` sql SELECT @@ -82,8 +93,4 @@ GROUP BY StartDate ORDER BY StartDate; ``` -Вы можете создать такое материализованное представление и навесить на него обычное представление, выполняющее доагрегацию данных. - -Заметим, что в большинстве случаев, использование `AggregatingMergeTree` является неоправданным, так как можно достаточно эффективно выполнять запросы по неагрегированным данным. - [Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/aggregatingmergetree/) diff --git a/docs/ru/operations/table_engines/mergetree.md b/docs/ru/operations/table_engines/mergetree.md index a0a528561df..7d669afb346 100644 --- a/docs/ru/operations/table_engines/mergetree.md +++ b/docs/ru/operations/table_engines/mergetree.md @@ -25,31 +25,45 @@ При необходимости можно задать способ сэмплирования данных в таблице. + -## Конфигурирование движка при создании таблицы +## Создание таблицы -``` -ENGINE [=] MergeTree() [PARTITION BY expr] [ORDER BY expr] [SAMPLE BY expr] [SETTINGS name=value, ...] +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE = MergeTree() +[PARTITION BY expr] +[ORDER BY expr] +[SAMPLE BY expr] +[SETTINGS name=value, ...] ``` -**Секции ENGINE** +Описание параметров запроса смотрите в [описании запроса](../../query_language/create.md#query_language-queries-create_table). + +**Секции запроса** + +- `ENGINE` — Имя и параметры движка. `ENGINE = MergeTree()`. Движок `MergeTree` не имеет параметров. - `ORDER BY` — первичный ключ. Кортеж столбцов или произвольных выражений. Пример: `ORDER BY (CounerID, EventDate)`. - Если используется ключ сэмплирования, то первичный ключ должен содержать его. Пример: `ORDER BY (CounerID, EventDate, intHash32(UserID))`. + Если используется выражение для сэмплирования, то первичный ключ должен содержать его. Пример: `ORDER BY (CounerID, EventDate, intHash32(UserID))`. - `PARTITION BY` — [ключ партиционирования](custom_partitioning_key.md#table_engines-custom_partitioning_key). Для партиционирования по месяцам используйте выражение `toYYYYMM(date_column)`, где `date_column` — столбец с датой типа [Date](../../data_types/date.md#data_type-date). В этом случае имена партиций имеют формат `"YYYYMM"`. -- `SAMPLE BY` — выражение для сэмплирования (не обязательно). Пример: `intHash32(UserID))`. +- `SAMPLE BY` — выражение для сэмплирования. Пример: `intHash32(UserID))`. -- `SETTINGS` — дополнительные параметры, регулирующие поведение `MergeTree` (не обязательно): +- `SETTINGS` — дополнительные параметры, регулирующие поведение `MergeTree`: - `index_granularity` — гранулярность индекса. Число строк данных между «засечками» индекса. По умолчанию — 8192. -**Пример** +**Пример задания секций** ``` ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192 @@ -61,13 +75,18 @@ ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDa `index_granularity` можно было не указывать, поскольку 8192 — это значение по умолчанию. -### Устаревший способ конфигурирования движка +
    Устаревший способ создания таблицы -!!!attention +!!! attention Не используйте этот способ в новых проектах и по возможности переведите старые проекты на способ описанный выше. -``` -ENGINE [=] MergeTree(date-column [, sampling_expression], (primary, key), index_granularity) +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE [=] MergeTree(date-column [, sampling_expression], (primary, key), index_granularity) ``` **Параметры MergeTree()** @@ -84,6 +103,7 @@ MergeTree(EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID) ``` Движок `MergeTree` сконфигурирован таким же образом, как и в примере выше для основного способа конфигурирования движка. +
    ## Хранение данных @@ -91,7 +111,7 @@ MergeTree(EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID) При вставке в таблицу создаются отдельные куски данных, каждый из которых лексикографически отсортирован по первичному ключу. Например, если первичный ключ — `(CounterID, Date)`, то данные в куске будут лежать в порядке `CounterID`, а для каждого `CounterID` в порядке `Date`. -Данные, относящиеся к разным партициям, разбиваются на разные куски. В фоновом режиме ClickHouse выполняет слияния (merge) кусков данных для более эффективного хранения. Куски, относящиеся к разным партициям не объединяются. +Данные, относящиеся к разным партициям, разбиваются на разные куски. В фоновом режиме ClickHouse выполняет слияния (merge) кусков данных для более эффективного хранения. Куски, относящиеся к разным партициям не объединяются. Механизм слияния не гарантирует, что все строки с одинаковым первичным ключом окажутся в одном куске. Для каждого куска данных ClickHouse создаёт индексный файл, который содержит значение первичного ключа для каждой индексной строки («засечка»). Номера индексных строк определяются как `n * index_granularity`, а максимальное значение `n` равно целой части от деления общего количества строк на `index_granularity`. Для каждого столбца также пишутся «засечки» для тех же индексных строк, что и для первичного ключа, эти «засечки» позволяют находить непосредственно данные в столбцах. @@ -139,9 +159,9 @@ ClickHouse не требует уникального первичного кл ClickHouse сортирует данные по первичному ключу, поэтому чем выше однородность, тем лучше сжатие. -- Обеспечить дополнительную логику при слиянии в движках [CollapsingMergeTree](collapsingmergetree.md#table_engine-collapsingmergetree) и [SummingMergeTree](summingmergetree.md#table_engine-summingmergetree). +- Обеспечить дополнительную логику при слиянии кусков данных в движках [CollapsingMergeTree](collapsingmergetree.md#table_engine-collapsingmergetree) и [SummingMergeTree](summingmergetree.md#table_engine-summingmergetree). - Может потребоваться иметь много полей в первичном ключе, даже если они не нужны для выполнения предыдущих пунктов. + Может потребоваться много полей в первичном ключе, даже если они не нужны для выполнения предыдущих пунктов. Длинный первичный ключ будет негативно влиять на производительность вставки и потребление памяти, однако на производительность ClickHouse при запросах `SELECT` лишние столбцы в первичном ключе не влияют. diff --git a/docs/ru/operations/table_engines/replacingmergetree.md b/docs/ru/operations/table_engines/replacingmergetree.md index 3c5658baea4..77f2f4894da 100644 --- a/docs/ru/operations/table_engines/replacingmergetree.md +++ b/docs/ru/operations/table_engines/replacingmergetree.md @@ -1,19 +1,58 @@ # ReplacingMergeTree -Движок таблиц отличается от `MergeTree` тем, что выполняет удаление дублирующихся записей с одинаковым значением первичного ключа. +Движок отличается от [MergeTree](mergetree.md#table_engines-mergetree) тем, что выполняет удаление дублирующихся записей с одинаковым значением первичного ключа. -Последний, необязательный параметр движка таблицы — столбец с версией. При слиянии для всех строк с одинаковым значением первичного ключа оставляет только одну строку: если задан столбец версии — строку с максимальной версией, иначе — последнюю строку. +Дедупликация данных производится лишь во время слияний. Слияние происходят в фоне в неизвестный момент времени, на который вы не можете ориентироваться. Некоторая часть данных может остаться необработанной. Хотя вы можете вызвать внеочередное слияние с помощью запроса `OPTIMIZE`, на это не стоит рассчитывать, так как запрос `OPTIMIZE` приводит к чтению и записи большого объёма данных. -Столбец с версией должен иметь тип из семейства `UInt`, или `Date`, или `DateTime`. +Таким образом, `ReplacingMergeTree` подходит для фоновой чистки дублирующихся данных в целях экономии места, но не даёт гарантии отсутствия дубликатов. -``` sql -ReplacingMergeTree(EventDate, (OrderID, EventDate, BannerID, ...), 8192, ver) +## Создание таблицы + +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE = ReplacingMergeTree([ver]) +[PARTITION BY expr] +[ORDER BY expr] +[SAMPLE BY expr] +[SETTINGS name=value, ...] ``` -Обратите внимание, что дедупликация данных производится лишь во время слияний. Слияние происходят в фоне в неизвестный момент времени, на который вы не можете ориентироваться. Некоторая часть данных может так и остаться необработанной. Хотя вы можете вызвать внеочередное слияние с помощью запроса OPTIMIZE, на это не стоит рассчитывать, так как запрос OPTIMIZE приводит к чтению и записи большого объёма данных. +Описание параметров запроса смотрите в [описании запроса](../../query_language/create.md#query_language-queries-create_table). -Таким образом, `ReplacingMergeTree` подходит для фоновой чистки дублирующихся данных в целях экономии места, но не даёт гарантий отсутствия дубликатов. +**Параметры ReplacingMergeTree** -*Движок не используется в Яндекс.Метрике, но нашёл своё применение в других отделах Яндекса.* +- `ver` — столбец с версией, тип `UInt*`, `Date` или `DateTime`. Необязательный параметр. + + При слиянии, из всех строк с одинаковым значением первичного ключа `ReplacingMergeTree` оставляет только одну: + + - Последнюю в выборке, если `ver` не задан. + - С максимальной версией, если `ver` задан. + +**Секции запроса** + +При создании таблицы `ReplacingMergeTree` используются те же [секции](mergetree.md#table_engines-mergetree-configuring), что и при создании таблицы `MergeTree`. + +
    Устаревший способ создания таблицы + +!!! attention + Не используйте этот способ в новых проектах и по возможности переведите старые проекты на способ описанный выше. + +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE [=] ReplacingMergeTree(date-column [, sampling_expression], (primary, key), index_granularity, [ver]) +``` + +Все параметры, кроме `ver` имеют то же значение, что в и `MergeTree`. + +- `ver` — столбец с версией. Необязательный параметр. Описание смотрите выше по тексту. +
    [Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/replacingmergetree/) diff --git a/docs/ru/operations/table_engines/summingmergetree.md b/docs/ru/operations/table_engines/summingmergetree.md index 6b9d1497337..97025aefe7f 100644 --- a/docs/ru/operations/table_engines/summingmergetree.md +++ b/docs/ru/operations/table_engines/summingmergetree.md @@ -2,33 +2,121 @@ # SummingMergeTree -Отличается от `MergeTree` тем, что суммирует данные при слиянии. +Движок наследует функциональность [MergeTree](mergetree.md#table_engines-mergetree). Отличие заключается в том, что для таблиц `SummingMergeTree` при слиянии кусков данных ClickHouse все строки с одинаковым первичным ключом заменяет на одну, которая хранит только суммы значений из столбцов с цифровым типом данных. Если первичный ключ подобран таким образом, что одному значению ключа соответствует много строк, это значительно уменьшает объем хранения и ускоряет последующую выборку данных. -``` sql -SummingMergeTree(EventDate, (OrderID, EventDate, BannerID, ...), 8192) +Мы рекомендуем использовать движок в паре с `MergeTree`. В `MergeTree` храните полные данные, а `SummingMergeTree` используйте для хранения агрегированных данных, например, при подготовке отчетов. Такой подход позволит не утратить ценные данные из-за неправильно выбранного первичного ключа. + +## Создание таблицы + +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE = MergeTree() +[PARTITION BY expr] +[ORDER BY expr] +[SAMPLE BY expr] +[SETTINGS name=value, ...] ``` -Столбцы для суммирования заданы неявно. При слиянии, для всех строчек с одинаковым значением первичного ключа (в примере - OrderID, EventDate, BannerID, ...), производится суммирование значений в числовых столбцах, не входящих в первичный ключ. +Описание параметров запроса смотрите в [описании запроса](../../query_language/create.md#query_language-queries-create_table). -``` sql -SummingMergeTree(EventDate, (OrderID, EventDate, BannerID, ...), 8192, (Shows, Clicks, Cost, ...)) +**Параметры SummingMergeTree** + +- `columns` — кортеж с именами столбцов, в которых будут суммироваться данные. Необязательный параметр. + Столбцы должны иметь числовой тип и не должны входить в первичный ключ. + + Если `columns` не задан, то ClickHouse суммирует значения во всех столбцах с числовым типом данных, не входящих в первичный ключ. + +**Секции запроса** + +При создании таблицы `SummingMergeTree` использутся те же [секции](mergetree.md#table_engines-mergetree-configuring) запроса, что и при создании таблицы `MergeTree`. + +
    Устаревший способ создания таблицы + +!!! attention + Не используйте этот способ в новых проектах и по возможности переведите старые проекты на способ описанный выше. + +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE [=] SummingMergeTree(date-column [, sampling_expression], (primary, key), index_granularity, [columns]) ``` -Явно заданные столбцы для суммирования (последний параметр - Shows, Clicks, Cost, ...). При слиянии, для всех строчек с одинаковым значением первичного ключа, производится суммирование значений в указанных столбцах. Указанные столбцы также должны быть числовыми и не входить в первичный ключ. +Все параметры, кроме `columns` имеют то же значение, что в и `MergeTree`. -Если значения во всех таких столбцах оказались нулевыми, то строчка удаляется. +- `columns` — кортеж с именами столбцов для суммирования данных. Необязательный параметр. Описание смотрите выше по тексту. +
    -Для остальных столбцов, не входящих в первичный ключ, при слиянии выбирается первое попавшееся значение. Но для столбцов типа AggregateFunction выполняется агрегация согласно заданной функции, так что этот движок фактически ведёт себя как `AggregatingMergeTree`. +## Пример использования -При чтении, суммирование не делается само по себе. Если оно необходимо - напишите соответствующий GROUP BY. +Рассмотрим следующую таблицу: -Дополнительно, таблица может иметь вложенные структуры данных, которые обрабатываются особым образом. -Если название вложенной таблицы заканчивается на Map и она содержит не менее двух столбцов, удовлетворяющих следующим критериям: +```sql +CREATE TABLE summtt +( + key UInt32, + value UInt32 +) +ENGINE = SummingMergeTree() +ORDER BY key +``` -- первый столбец - числовой ((U)IntN, Date, DateTime), назовем его условно key, -- остальные столбцы - арифметические ((U)IntN, Float32/64), условно (values...), +Добавим в неё данные: -то такая вложенная таблица воспринимается как отображение key `=>` (values...) и при слиянии ее строк выполняется слияние элементов двух множеств по key со сложением соответствующих (values...). +``` +:) INSERT INTO summtt Values(1,1),(1,2),(2,1) +``` + +ClickHouse может не полностью просуммировать все строки ([смотрите ниже по тексту](#summingmergetree-data-processing)), поэтому при запросе мы используем агрегатную функцию `sum` и секцию `GROUP BY`. + +```sql +SELECT key, sum(value) FROM summtt GROUP BY key +``` +``` +┌─key─┬─sum(value)─┐ +│ 2 │ 1 │ +│ 1 │ 3 │ +└─────┴────────────┘ +``` + + + +## Обработка данных + +При вставке данных в таблицу они сохраняются как есть. Периодически ClickHouse выполняет слияние вставленных кусков данных и именно в этот момент производится суммирование и замена многих строк с одинаковым первичным ключом на одну для каждого результирующего куска данных. + +ClickHouse может слить куски данных таким образом, что не все строки с одинаковым первичным ключом окажутся в одном финальном куске, т.е. суммирование будет не полным. Поэтому, при выборке данных (`SELECT`) необходимо использовать агрегатную функцию [sum()](../../query_language/agg_functions/reference.md#agg_function-sum) и секцию `GROUP BY` как описано в примере выше. + +### Общие правила суммирования + +Суммируются значения в столбцах с числовым типом данных. Набор столбцов определяется параметром `columns`. + +Если значения во всех столбцах для суммирования оказались нулевыми, то строчка удаляется. + +Для столбцов, не входящих в первичный ключ и не суммирующихся, выбирается произвольное значение из имеющихся. + +Значения для столбцов, входящих в первичный ключ, не суммируются. + +### Суммирование в столбцах AggregateFunction + +Для столбцов типа [AggregateFunction](../../data_types/nested_data_structures/aggregatefunction.md#data_type-aggregatefunction) ClickHouse выполняет агрегацию согласно заданной функции, повторяя поведение движка [AggregatingMergeTree](aggregatingmergetree.md#table_engine-aggregatingmergetree). + +### Вложенные структуры + +Таблица может иметь вложенные структуры данных, которые обрабатываются особым образом. + +Если название вложенной таблицы заканчивается на `Map` и она содержит не менее двух столбцов, удовлетворяющих критериям: + +- первый столбец - числовой `(*Int*, Date, DateTime)`, назовем его условно `key`, +- остальные столбцы - арифметические `(*Int*, Float32/64)`, условно `(values...)`, + +то вложенная таблица воспринимается как отображение `key => (values...)` и при слиянии её строк выполняется слияние элементов двух множеств по `key` со сложением соответствующих `(values...)`. Примеры: @@ -39,10 +127,8 @@ SummingMergeTree(EventDate, (OrderID, EventDate, BannerID, ...), 8192, (Shows, C [(1, 100), (2, 150)] + [(1, -100)] -> [(2, 150)] ``` -Для агрегации Map используйте функцию sumMap(key, value). +При запросе данных используйте функцию [sumMap(key, value)](../../query_language/agg_functions/reference.md#agg_function-summap) для агрегации `Map`. -Для вложенных структур данных не нужно указывать её столбцы в качестве списка столбцов для суммирования. - -Этот движок таблиц разработан по просьбе БК, и является мало полезным. Помните, что при хранении лишь предагрегированных данных, вы теряете часть преимуществ системы. +Для вложенной структуры данных не нужно указывать её столбцы в кортеже столбцов для суммирования. [Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/summingmergetree/) diff --git a/docs/ru/query_language/agg_functions/combinators.md b/docs/ru/query_language/agg_functions/combinators.md index 0e4c1dd8733..b823d6b877c 100644 --- a/docs/ru/query_language/agg_functions/combinators.md +++ b/docs/ru/query_language/agg_functions/combinators.md @@ -4,6 +4,8 @@ К имени агрегатной функции может быть приписан некоторый суффикс. При этом, работа агрегатной функции некоторым образом модифицируется. + + ## -If К имени любой агрегатной функции может быть приписан суффикс -If. В этом случае, агрегатная функция принимает ещё один дополнительный аргумент - условие (типа UInt8). Агрегатная функция будет обрабатывать только те строки, для которых условие сработало. Если условие ни разу не сработало - возвращается некоторое значение по умолчанию (обычно - нули, пустые строки). @@ -25,7 +27,7 @@ ## -State -В случае применения этого комбинатора, агрегатная функция возвращает не готовое значение (например, в случае функции uniq - количество уникальных значений), а промежуточное состояние агрегации (например, в случае функции `uniq` - хэш-таблицу для рассчёта количества уникальных значений), которое имеет тип AggregateFunction(...) и может использоваться для дальнейшей обработки или может быть сохранено в таблицу для последующей доагрегации - смотрите разделы «AggregatingMergeTree» и «функции для работы с промежуточными состояниями агрегации». +В случае применения этого комбинатора, агрегатная функция возвращает не готовое значение (например, в случае функции `uniq` — количество уникальных значений), а промежуточное состояние агрегации (например, в случае функции `uniq` — хэш-таблицу для расчёта количества уникальных значений), которое имеет тип AggregateFunction(...) и может использоваться для дальнейшей обработки или может быть сохранено в таблицу для последующей доагрегации - смотрите разделы «AggregatingMergeTree» и «функции для работы с промежуточными состояниями агрегации». ## -Merge diff --git a/docs/ru/query_language/agg_functions/reference.md b/docs/ru/query_language/agg_functions/reference.md index 1c700851ef0..2cd156591d4 100644 --- a/docs/ru/query_language/agg_functions/reference.md +++ b/docs/ru/query_language/agg_functions/reference.md @@ -11,6 +11,8 @@ Запрос вида `SELECT count() FROM table` не оптимизируется, так как количество записей в таблице нигде не хранится отдельно - из таблицы будет выбран какой-нибудь достаточно маленький столбец, и будет посчитано количество значений в нём. + + ## any(x) Выбирает первое попавшееся значение. @@ -53,7 +55,6 @@ FROM ontime Выбирает последнее попавшееся значение. Результат так же недетерминирован, как и для функции `any`. - ## min(x) Вычисляет минимум. @@ -87,6 +88,7 @@ SELECT argMin(user, salary) FROM salary Вычисляет значение arg при максимальном значении val. Если есть несколько разных значений arg для максимальных значений val, то выдаётся первое попавшееся из таких значений. + ## sum(x) @@ -100,6 +102,7 @@ SELECT argMin(user, salary) FROM salary Работает только для чисел. + ## sumMap(key, value) @@ -144,6 +147,9 @@ GROUP BY timeslot Работает только для чисел. Результат всегда - Float64. + + + ## uniq(x) Приближённо вычисляет количество различных значений аргумента. Работает для чисел, строк, дат, дат-с-временем, для нескольких аргументов и аргументов-кортежей. diff --git a/docs/ru/query_language/create.md b/docs/ru/query_language/create.md index 83a137c7bd7..f1d8f8471de 100644 --- a/docs/ru/query_language/create.md +++ b/docs/ru/query_language/create.md @@ -43,6 +43,8 @@ CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name ENGINE = engine AS SELECT ... Во всех случаях, если указано `IF NOT EXISTS`, то запрос не будет возвращать ошибку, если таблица уже существует. В этом случае, запрос будет ничего не делать. +После секции `ENGINE` в запросе могут использоваться и другие секции в зависимости от движка. Подробную документацию по созданию таблиц смотрите в описаниях [движков](../operations/table_engines/index.md#table_engines). + ### Значения по умолчанию From b298ec843cd96fbc95e13ac44cca8f7f43d4c105 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 19 Oct 2018 18:59:17 +0300 Subject: [PATCH 20/28] Added russian changelog for version 18.14.9 (draft) [#CLICKHOUSE-2] --- CHANGELOG_RU.md | 94 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 94 insertions(+) diff --git a/CHANGELOG_RU.md b/CHANGELOG_RU.md index 6c698595558..9c033094fab 100644 --- a/CHANGELOG_RU.md +++ b/CHANGELOG_RU.md @@ -1,3 +1,97 @@ +## ClickHouse release 18.14.9, 2018-10-16 + +### Новые возможности: + +* Модификатор `WITH CUBE` для `GROUP BY` (также доступен синтаксис: `GROUP BY CUBE(...)`). +* Добавлена функция `formatDateTime`. [Alexandr Krasheninnikov]() +* Добавлен движок таблиц `JDBC` и табличная функция `jdbc` (для работы требуется установка clickhouse-jdbc-bridge). [Alexandr Krasheninnikov]() +* Добавлены функции для работы с ISO номером недели: `toISOWeek`, `toISOYear`, `toStartOfISOYear`. +* Добавлена функция `toDayOfYear`. +* Добавлена возможность использования столбцов типа `Nullable` для таблиц типа `MySQL`, `ODBC`. +* Возможность чтения вложенных структур данных как вложенных объектов в формате `JSONEachRow`. Добавлена настройка `input_format_import_nested_json`. [Veloman Yunkan](). +* Возможность параллельной обработки многих `MATERIALIZED VIEW` при вставке данных. Настройка `parallel_view_processing`. [Marek Vavruša](). +* Добавлен запрос `SYSTEM FLUSH LOGS` (форсированный сброс логов в системные таблицы, такие как например, `query_log`). +* Возможность использования предопределённых макросов `database` и `table` в объявлении `Replicated` таблиц. +* Добавлена возможность чтения значения типа `Decimal` в инженерной нотации (с указанием десятичной экспоненты). + +### Экспериментальные возможности: + +* Оптимизация GROUP BY для типов данных `LowCardinality`. +* Оптимизации вычисления выражений для типов данных `LowCardinality`. + +### Улучшения: + +* Существенно уменьшено потребление памяти для запросов с `ORDER BY` и `LIMIT`. Настройка `max_bytes_before_remerge_sort`. +* При отсутствии указания типа `JOIN` (`LEFT`, `INNER`, ...), подразумевается `INNER JOIN`. +* Корректная работа квалифицированных звёздочек в запросах с `JOIN`. [Winter Zhang]() +* Движок таблиц `ODBC` корректно выбирает способ квотирования идентификаторов в SQL диалекте удалённой СУБД. [Alexandr Krasheninnikov]() +* Настройка `compile_expressions` (JIT компиляция выражений) включена по-умолчанию. +* Исправлено поведение при одновременном DROP DATABASE/TABLE IF EXISTS и CREATE DATABASE/TABLE IF NOT EXISTS. Ранее запрос `CREATE DATABASE ... IF NOT EXISTS` мог выдавать сообщение об ошибке вида "File ... already exists", а запросы `CREATE TABLE ... IF NOT EXISTS` и `DROP TABLE IF EXISTS` могли выдавать сообщение `Table ... is creating or attaching right now`. +* Выражения LIKE и IN с константной правой частью пробрасываются на удалённый сервер при запросах из таблиц типа MySQL и ODBC. +* Сравнения с константными выражениями в секции WHERE пробрасываются на удалённый сервер при запросах из таблиц типа MySQL и ODBC. Ранее пробрасывались только сравнения с константами. +* Корректное вычисление ширины строк в терминале для `Pretty` форматов, в том числе для строк с иероглифами. [Amos Bird](). +* Возможность указания `ON CLUSTER` для запросов `ALTER UPDATE`. +* Увеличена производительность чтения данных в формате `JSONEachRow`. +* Добавлены синонимы функций `LENGTH`, `CHARACTER_LENGTH` для совместимости. Функция `CONCAT` стала регистронезависимой. +* Добавлен синоним `TIMESTAMP` для типа `DateTime`. +* В логах сервера всегда присутствует место для query_id, даже если строчка лога не относится к запросу. Это сделано для более простого парсинга текстовых логов сервера сторонними инструментами. +* Логгирование потребления памяти запросом при превышении очередной отметки целого числа гигабайт. +* Добавлен режим совместимости для случая, когда клиентская библиотека, работающая по Native протоколу, по ошибке отправляет меньшее количество столбцов, чем сервер ожидает для запроса INSERT. Такой сценарий был возможен при использовании библиотеки clickhouse-cpp. Ранее этот сценарий приводил к падению сервера. +* В `clickhouse-copier`, в задаваемом пользователем выражении WHERE, появилась возможность использовать алиас `partition_key` (для дополнительной фильтрации по партициям исходной таблицы). Это полезно, если схема партиционирования изменяется при копировании, но изменяется незначительно. +* Рабочий поток движка `Kafka` перенесён в фоновый пул потоков для того, чтобы автоматически уменьшать скорость чтения данных при большой нагрузке. [Marek Vavruša](). +* Поддержка чтения значений типа `Tuple` и `Nested` структур как `struct` в формате `Cap'n'Proto` [Marek Vavruša](). +* В список доменов верхнего уровня для функции `firstSignificantSubdomain` добавлен домен `biz` [decaseal](). +* В конфигурации внешних словарей, пустое значение `null_value` интерпретируется, как значение типа данных по-умоланию. +* Поддержка функций `intDiv`, `intDivOrZero` для `Decimal`. +* Поддержка типов `Date`, `DateTime`, `UUID`, `Decimal` в качестве ключа для агрегатной функции `sumMap`. +* Поддержка типа данных `Decimal` во внешних словарях. +* Поддержка типа данных `Decimal` в таблицах типа `SummingMergeTree`. +* Добавлена специализация для `UUID` в функции `if`. +* Уменьшено количество системных вызовов `open`, `close` при чтении из таблиц семейства `MergeTree`. +* Возможность выполнения запроса `TRUNCATE TABLE` на любой реплике (запрос пробрасывается на реплику-лидера). [Kirill Shvakov]() + +### Исправление ошибок: + +* Исправлена ошибка в работе таблиц типа `Dictionary` для словарей типа `range_hashed`. Ошибка возникла в версии 18.12.17. +* Исправлена ошибка при загрузке словарей типа `range_hashed` (сообщение `Unsupported type Nullable(...)`). Ошибка возникла в версии 18.12.17. +* Исправлена некорректная работа функции `pointInPolygon` из-за накопления погрешности при вычислениях для полигонов с большим количеством близко расположенных вершин. +* Если после слияния кусков данных, у результирующего куска чексумма отличается от результата того же слияния на другой реплике, то результат слияния удаляется, и вместо этого кусок скачивается с другой реплики (это правильное поведение). Но после скачивания куска, он не мог добавиться в рабочий набор из-за ошибки, что кусок уже существует (так как кусок после слияния удалялся не сразу, а с задержкой). Это приводило к циклическим попыткам скачивания одних и тех же данных. +* Исправлен некорректный учёт общего потребления оперативной памяти запросами (что приводило к неправильной работе настройки `max_memory_usage_for_all_queries` и неправильному значению метрики `MemoryTracking`). Ошибка возникла в версии 18.12.13. [Marek Vavruša]() +* Исправлена работоспособность запросов `CREATE TABLE ... ON CLUSTER ... AS SELECT ...` Ошибка возникла в версии 18.12.13. +* Исправлена лишняя подготовка структуры данных для `JOIN` на сервере-инициаторе запроса, если `JOIN` выполняется только на удалённых серверах. +* Исправлены ошибки в движке `Kafka`: неработоспособность после исключения при начале чтения данных; блокировка при завершении [Marek Vavruša](). +* Для таблиц `Kafka` не передавался опциональный параметр `schema` (схема формата `Cap'n'Proto`). [Vojtech Splichal] +* Если ансамбль серверов ZooKeeper содержит серверы, которые принимают соединение, но сразу же разрывают его вместо ответа на рукопожатие, то ClickHouse выбирает для соединения другой сервер. Ранее в этом случае возникала ошибка `Cannot read all data. Bytes read: 0. Bytes expected: 4.` и сервер не мог стартовать. +* Если ансамбль серверов ZooKeeper содержит серверы, для которых DNS запрос возвращает ошибку, то такие серверы пропускаются. +* Исправлено преобразование типов между `Date` и `DateTime` при вставке данных в формате `VALUES` (в случае, когда `input_format_values_interpret_expressions = 1`). Ранее преобразование производилось между числовым значением количества дней с начала unix эпохи и unix timestamp, что приводило к неожиданным результатам. +* Исправление преобразования типов между `Decimal` и целыми числами. +* Исправлены ошибки в работе настройки `enable_optimize_predicate_expression`. [Winter Zhang]() +* Настройка `enable_optimize_predicate_expression` выключена по-умолчанию. +* Исправлена ошибка парсинга формата CSV с числами с плавающей запятой, если используется разделитель CSV не по-умолчанию, такой как например, `;`. +* Испоавлена функция `arrayCumSumNonNegative` (она не накапливает отрицательные значения, если аккумулятор становится меньше нуля). +* Исправлена работа `Merge` таблицы поверх `Distributed` таблиц при использовании `PREWHERE`. +* Исправления ошибок в запросе ALTER UPDATE. +* Исправления ошибок в табличной функции `odbc`, которые возникли в версии 18.12. +* Исправлена работа агрегатных функций с комбинаторами `StateArray`. +* Исправлено падение при делении значения типа `Decimal` на ноль. +* Исправлен вывод типов для операций с использованием аргументов типа `Decimal` и целых чисел. +* Исправлен segfault при `GROUP BY` по `Decimal128`. +* Настройка `log_query_threads` (логгирование информации о каждом потоке исполнения запроса) теперь имеет эффект только если настройка `log_queries` (логгирование информации о запросах) выставлена в 1. Так как настройка `log_query_threads` включена по-умолчанию, ранее информация о потоках логгировалась даже если логгирование запросов выключено. +* Исправлена ошибка в распределённой работе агрегатной функции quantiles (сообщение об ошибке вида `Not found column quantile...`). +* Исправлена проблема совместимости при одновременной работе на кластере серверов версии 18.12.17 и более старых, приводящая к тому, что при распределённых запросах с GROUP BY по ключам одновременно фиксированной и не фиксированной длины, при условии, что количество данных в процессе агрегации большое, могли возвращаться не до конца агрегированные данные (одни и те же ключи агрегации в двух разных строках). +* Исправлена обработка подстановок в `clickhouse-performance-test`, если запрос содержит только часть из объявленных в тесте подстановок. +* Исправлена ошибка при использовании `FINAL` совместно с `PREWHERE`. +* Исправлена ошибка при использовании `PREWHERE` над столбцами, добавленными при `ALTER`. +* Добавлена проверка отсутствия `arrayJoin` для `DEFAULT`, `MATERIALIZED` выражений. Ранее наличие `arrayJoin` приводило к ошибке при вставке данных. +* Добавлена проверка отсутствия `arrayJoin` в секции `PREWHERE`. Ранее это приводило к сообщениям вида `Size ... doesn't match` или `Unknown compression method` при выполнении запросов. +* Исправлен segfault, который мог возникать в редких случаях после оптимизации - замены цепочек AND из равенства выражения константам на соответствующее выражение IN. [liuyimin](). +* Мелкие исправления `clickhouse-benchmark`: ранее информация о клиенте не передавалась на сервер; более корректный подсчёт числа выполненных запросов при завершении работы и для ограничения числа итераций. + +### Обратно несовместимые изменения: + +* Удалена настройка `allow_experimental_decimal_type`. Тип данных `Decimal` доступен для использования по-умолчанию. + + ## ClickHouse release 18.12.17, 2018-09-16 ### Новые возможности: From 67eb30d19291526329ef77e56d923e1ed6d5bafe Mon Sep 17 00:00:00 2001 From: proller Date: Fri, 19 Oct 2018 19:31:51 +0300 Subject: [PATCH 21/28] CLICKHOUSE-4074 Fix merge_tree_uniform_read_distribution=0 --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- ...tree_uniform_read_distribution_0.reference | 1 + ..._merge_tree_uniform_read_distribution_0.sh | 8 +++++ .../00443_optimize_final_vertical_merge.sh | 34 +++++++++---------- 4 files changed, 27 insertions(+), 18 deletions(-) create mode 120000 dbms/tests/queries/0_stateless/00443_merge_tree_uniform_read_distribution_0.reference create mode 100755 dbms/tests/queries/0_stateless/00443_merge_tree_uniform_read_distribution_0.sh diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 50900021dec..ac7259f4768 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -680,7 +680,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( /// and assign a stream to read from it. while (need_marks > 0 && !parts.empty()) { - RangesInDataPart part = parts.back(); + RangesInDataPart & part = parts.back(); size_t & marks_in_part = sum_marks_in_parts.back(); /// We will not take too few rows from a part. diff --git a/dbms/tests/queries/0_stateless/00443_merge_tree_uniform_read_distribution_0.reference b/dbms/tests/queries/0_stateless/00443_merge_tree_uniform_read_distribution_0.reference new file mode 120000 index 00000000000..4d2577d02a3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00443_merge_tree_uniform_read_distribution_0.reference @@ -0,0 +1 @@ +00443_optimize_final_vertical_merge.reference \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/00443_merge_tree_uniform_read_distribution_0.sh b/dbms/tests/queries/0_stateless/00443_merge_tree_uniform_read_distribution_0.sh new file mode 100755 index 00000000000..93aa9416172 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00443_merge_tree_uniform_read_distribution_0.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +set -e + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +#. $CURDIR/../shell_config.sh + +env CLICKHOUSE_CLIENT_OPT="--merge_tree_uniform_read_distribution=0" bash $CURDIR/00443_optimize_final_vertical_merge.sh diff --git a/dbms/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh b/dbms/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh index c8453119eb9..5cac421fb4f 100755 --- a/dbms/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh +++ b/dbms/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh @@ -11,12 +11,12 @@ name="$db.$table" res_rows=1500000 # >= vertical_merge_algorithm_min_rows_to_activate function get_num_parts { - $CLICKHOUSE_CLIENT -q "SELECT count() FROM system.parts WHERE active AND database='$db' AND table='$table'" + $CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "SELECT count() FROM system.parts WHERE active AND database='$db' AND table='$table'" } -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS $name" +$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "DROP TABLE IF EXISTS $name" -$CLICKHOUSE_CLIENT -q "CREATE TABLE $name ( +$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "CREATE TABLE $name ( date Date, Sign Int8, ki UInt64, @@ -39,19 +39,19 @@ s String ) ENGINE = CollapsingMergeTree(date, (date, ki), 8192, Sign)" -$CLICKHOUSE_CLIENT -q "INSERT INTO $name (date, Sign, ki) SELECT +$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "INSERT INTO $name (date, Sign, ki) SELECT toDate(0) AS date, toInt8(1) AS Sign, toUInt64(0) AS ki FROM system.numbers LIMIT 9000" -$CLICKHOUSE_CLIENT -q "INSERT INTO $name (date, Sign, ki) SELECT +$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "INSERT INTO $name (date, Sign, ki) SELECT toDate(0) AS date, toInt8(1) AS Sign, number AS ki FROM system.numbers LIMIT 9000, 9000" -$CLICKHOUSE_CLIENT -q "INSERT INTO $name SELECT +$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "INSERT INTO $name SELECT toDate(0) AS date, toInt8(1) AS Sign, number AS ki, @@ -70,21 +70,21 @@ number AS di10, [hex(number), hex(number+1)] AS \`n.s\` FROM system.numbers LIMIT $res_rows" -while [[ `get_num_parts` -ne 1 ]] ; do $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE $name PARTITION 197001" --server_logs_file=/dev/null; done +while [[ `get_num_parts` -ne 1 ]] ; do $CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "OPTIMIZE TABLE $name PARTITION 197001" --server_logs_file=/dev/null; done -$CLICKHOUSE_CLIENT -q "ALTER TABLE $name ADD COLUMN n.a Array(String)" -$CLICKHOUSE_CLIENT -q "ALTER TABLE $name ADD COLUMN da Array(String) DEFAULT ['def']" +$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "ALTER TABLE $name ADD COLUMN n.a Array(String)" +$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "ALTER TABLE $name ADD COLUMN da Array(String) DEFAULT ['def']" -$CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE $name PARTITION 197001 FINAL" --server_logs_file=/dev/null +$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "OPTIMIZE TABLE $name PARTITION 197001 FINAL" --server_logs_file=/dev/null -$CLICKHOUSE_CLIENT -q "ALTER TABLE $name MODIFY COLUMN n.a Array(String) DEFAULT ['zzz']" -$CLICKHOUSE_CLIENT -q "ALTER TABLE $name MODIFY COLUMN da Array(String) DEFAULT ['zzz']" +$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "ALTER TABLE $name MODIFY COLUMN n.a Array(String) DEFAULT ['zzz']" +$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "ALTER TABLE $name MODIFY COLUMN da Array(String) DEFAULT ['zzz']" -$CLICKHOUSE_CLIENT -q "SELECT count(), sum(Sign), sum(ki = di05), sum(hex(ki) = ds), sum(ki = n.i[1]), sum([hex(ki), hex(ki+1)] = n.s) FROM $name" -$CLICKHOUSE_CLIENT -q "SELECT groupUniqArray(da), groupUniqArray(n.a) FROM $name" +$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "SELECT count(), sum(Sign), sum(ki = di05), sum(hex(ki) = ds), sum(ki = n.i[1]), sum([hex(ki), hex(ki+1)] = n.s) FROM $name" +$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "SELECT groupUniqArray(da), groupUniqArray(n.a) FROM $name" -hash_src=`$CLICKHOUSE_CLIENT --max_threads=1 -q "SELECT cityHash64(groupArray(ki)) FROM $name"` -hash_ref=`$CLICKHOUSE_CLIENT --max_threads=1 -q "SELECT cityHash64(groupArray(ki)) FROM (SELECT number as ki FROM system.numbers LIMIT $res_rows)"` +hash_src=`$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT --max_threads=1 -q "SELECT cityHash64(groupArray(ki)) FROM $name"` +hash_ref=`$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT --max_threads=1 -q "SELECT cityHash64(groupArray(ki)) FROM (SELECT number as ki FROM system.numbers LIMIT $res_rows)"` echo $(( $hash_src - $hash_ref )) -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS $name" +$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "DROP TABLE IF EXISTS $name" From 2659e945730961589454d41901fd310e77436859 Mon Sep 17 00:00:00 2001 From: proller Date: Fri, 19 Oct 2018 20:27:38 +0300 Subject: [PATCH 22/28] Fix use after pop_back() --- dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index ac7259f4768..cbef565b7f0 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -680,7 +680,9 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( /// and assign a stream to read from it. while (need_marks > 0 && !parts.empty()) { - RangesInDataPart & part = parts.back(); + RangesInDataPart part = parts.back(); + parts.pop_back(); + size_t & marks_in_part = sum_marks_in_parts.back(); /// We will not take too few rows from a part. @@ -704,7 +706,6 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( ranges_to_get_from_part = part.ranges; need_marks -= marks_in_part; - parts.pop_back(); sum_marks_in_parts.pop_back(); } else @@ -727,6 +728,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( if (range.begin == range.end) part.ranges.pop_back(); } + parts.emplace_back(part); } BlockInputStreamPtr source_stream = std::make_shared( From 61568a0e748922a38513bd1f9848a58cf664fa20 Mon Sep 17 00:00:00 2001 From: proller Date: Fri, 19 Oct 2018 20:33:46 +0300 Subject: [PATCH 23/28] Better test --- ..._merge_tree_uniform_read_distribution_0.sh | 1 - .../00443_optimize_final_vertical_merge.sh | 34 +++++++++---------- dbms/tests/queries/shell_config.sh | 2 +- 3 files changed, 18 insertions(+), 19 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00443_merge_tree_uniform_read_distribution_0.sh b/dbms/tests/queries/0_stateless/00443_merge_tree_uniform_read_distribution_0.sh index 93aa9416172..a2d31000dcc 100755 --- a/dbms/tests/queries/0_stateless/00443_merge_tree_uniform_read_distribution_0.sh +++ b/dbms/tests/queries/0_stateless/00443_merge_tree_uniform_read_distribution_0.sh @@ -3,6 +3,5 @@ set -e CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -#. $CURDIR/../shell_config.sh env CLICKHOUSE_CLIENT_OPT="--merge_tree_uniform_read_distribution=0" bash $CURDIR/00443_optimize_final_vertical_merge.sh diff --git a/dbms/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh b/dbms/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh index 5cac421fb4f..c8453119eb9 100755 --- a/dbms/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh +++ b/dbms/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh @@ -11,12 +11,12 @@ name="$db.$table" res_rows=1500000 # >= vertical_merge_algorithm_min_rows_to_activate function get_num_parts { - $CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "SELECT count() FROM system.parts WHERE active AND database='$db' AND table='$table'" + $CLICKHOUSE_CLIENT -q "SELECT count() FROM system.parts WHERE active AND database='$db' AND table='$table'" } -$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "DROP TABLE IF EXISTS $name" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS $name" -$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "CREATE TABLE $name ( +$CLICKHOUSE_CLIENT -q "CREATE TABLE $name ( date Date, Sign Int8, ki UInt64, @@ -39,19 +39,19 @@ s String ) ENGINE = CollapsingMergeTree(date, (date, ki), 8192, Sign)" -$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "INSERT INTO $name (date, Sign, ki) SELECT +$CLICKHOUSE_CLIENT -q "INSERT INTO $name (date, Sign, ki) SELECT toDate(0) AS date, toInt8(1) AS Sign, toUInt64(0) AS ki FROM system.numbers LIMIT 9000" -$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "INSERT INTO $name (date, Sign, ki) SELECT +$CLICKHOUSE_CLIENT -q "INSERT INTO $name (date, Sign, ki) SELECT toDate(0) AS date, toInt8(1) AS Sign, number AS ki FROM system.numbers LIMIT 9000, 9000" -$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "INSERT INTO $name SELECT +$CLICKHOUSE_CLIENT -q "INSERT INTO $name SELECT toDate(0) AS date, toInt8(1) AS Sign, number AS ki, @@ -70,21 +70,21 @@ number AS di10, [hex(number), hex(number+1)] AS \`n.s\` FROM system.numbers LIMIT $res_rows" -while [[ `get_num_parts` -ne 1 ]] ; do $CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "OPTIMIZE TABLE $name PARTITION 197001" --server_logs_file=/dev/null; done +while [[ `get_num_parts` -ne 1 ]] ; do $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE $name PARTITION 197001" --server_logs_file=/dev/null; done -$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "ALTER TABLE $name ADD COLUMN n.a Array(String)" -$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "ALTER TABLE $name ADD COLUMN da Array(String) DEFAULT ['def']" +$CLICKHOUSE_CLIENT -q "ALTER TABLE $name ADD COLUMN n.a Array(String)" +$CLICKHOUSE_CLIENT -q "ALTER TABLE $name ADD COLUMN da Array(String) DEFAULT ['def']" -$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "OPTIMIZE TABLE $name PARTITION 197001 FINAL" --server_logs_file=/dev/null +$CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE $name PARTITION 197001 FINAL" --server_logs_file=/dev/null -$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "ALTER TABLE $name MODIFY COLUMN n.a Array(String) DEFAULT ['zzz']" -$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "ALTER TABLE $name MODIFY COLUMN da Array(String) DEFAULT ['zzz']" +$CLICKHOUSE_CLIENT -q "ALTER TABLE $name MODIFY COLUMN n.a Array(String) DEFAULT ['zzz']" +$CLICKHOUSE_CLIENT -q "ALTER TABLE $name MODIFY COLUMN da Array(String) DEFAULT ['zzz']" -$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "SELECT count(), sum(Sign), sum(ki = di05), sum(hex(ki) = ds), sum(ki = n.i[1]), sum([hex(ki), hex(ki+1)] = n.s) FROM $name" -$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "SELECT groupUniqArray(da), groupUniqArray(n.a) FROM $name" +$CLICKHOUSE_CLIENT -q "SELECT count(), sum(Sign), sum(ki = di05), sum(hex(ki) = ds), sum(ki = n.i[1]), sum([hex(ki), hex(ki+1)] = n.s) FROM $name" +$CLICKHOUSE_CLIENT -q "SELECT groupUniqArray(da), groupUniqArray(n.a) FROM $name" -hash_src=`$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT --max_threads=1 -q "SELECT cityHash64(groupArray(ki)) FROM $name"` -hash_ref=`$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT --max_threads=1 -q "SELECT cityHash64(groupArray(ki)) FROM (SELECT number as ki FROM system.numbers LIMIT $res_rows)"` +hash_src=`$CLICKHOUSE_CLIENT --max_threads=1 -q "SELECT cityHash64(groupArray(ki)) FROM $name"` +hash_ref=`$CLICKHOUSE_CLIENT --max_threads=1 -q "SELECT cityHash64(groupArray(ki)) FROM (SELECT number as ki FROM system.numbers LIMIT $res_rows)"` echo $(( $hash_src - $hash_ref )) -$CLICKHOUSE_CLIENT $CLICKHOUSE_CLIENT_OPT -q "DROP TABLE IF EXISTS $name" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS $name" diff --git a/dbms/tests/queries/shell_config.sh b/dbms/tests/queries/shell_config.sh index 0078c3432ad..0c61af7f4c7 100644 --- a/dbms/tests/queries/shell_config.sh +++ b/dbms/tests/queries/shell_config.sh @@ -2,7 +2,7 @@ export CLICKHOUSE_BINARY=${CLICKHOUSE_BINARY:="clickhouse"} export CLICKHOUSE_CLIENT=${CLICKHOUSE_CLIENT:="${CLICKHOUSE_BINARY}-client"} export CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL:="warning"} -export CLICKHOUSE_CLIENT="${CLICKHOUSE_CLIENT} --send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}" +export CLICKHOUSE_CLIENT="${CLICKHOUSE_CLIENT} --send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL} ${CLICKHOUSE_CLIENT_OPT}" export CLICKHOUSE_LOCAL=${CLICKHOUSE_LOCAL:="${CLICKHOUSE_BINARY}-local"} export CLICKHOUSE_CONFIG=${CLICKHOUSE_CONFIG:="/etc/clickhouse-server/config.xml"} From dcc8ca49d73ac42bc0bed511870683983a5f791a Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 20 Oct 2018 00:03:45 +0300 Subject: [PATCH 24/28] Add global counter to subqueries --- dbms/src/Interpreters/QueryAliasesVisitor.cpp | 7 +++--- .../00745_compile_scalar_subquery.reference | 2 ++ .../00745_compile_scalar_subquery.sql | 23 +++++++++++++++++++ 3 files changed, 28 insertions(+), 4 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00745_compile_scalar_subquery.reference create mode 100644 dbms/tests/queries/0_stateless/00745_compile_scalar_subquery.sql diff --git a/dbms/src/Interpreters/QueryAliasesVisitor.cpp b/dbms/src/Interpreters/QueryAliasesVisitor.cpp index 61543b9ec2a..0202abd3e02 100644 --- a/dbms/src/Interpreters/QueryAliasesVisitor.cpp +++ b/dbms/src/Interpreters/QueryAliasesVisitor.cpp @@ -72,13 +72,12 @@ void QueryAliasesVisitor::getNodeAlias(const ASTPtr & ast, Aliases & aliases, co if (subquery->alias.empty()) { - size_t subquery_index = 1; + static std::atomic_uint64_t subquery_index = 1; while (true) { - alias = "_subquery" + toString(subquery_index); - if (!aliases.count("_subquery" + toString(subquery_index))) + alias = "_subquery" + std::to_string(subquery_index++); + if (!aliases.count(alias)) break; - ++subquery_index; } subquery->setAlias(alias); diff --git a/dbms/tests/queries/0_stateless/00745_compile_scalar_subquery.reference b/dbms/tests/queries/0_stateless/00745_compile_scalar_subquery.reference new file mode 100644 index 00000000000..b261da18d51 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00745_compile_scalar_subquery.reference @@ -0,0 +1,2 @@ +1 +0 diff --git a/dbms/tests/queries/0_stateless/00745_compile_scalar_subquery.sql b/dbms/tests/queries/0_stateless/00745_compile_scalar_subquery.sql new file mode 100644 index 00000000000..0ca0e9b8c7c --- /dev/null +++ b/dbms/tests/queries/0_stateless/00745_compile_scalar_subquery.sql @@ -0,0 +1,23 @@ +SET compile_expressions = 1; +SET min_count_to_compile = 1; +SET optimize_move_to_prewhere = 0; +SET enable_optimize_predicate_expression=0; + +DROP TABLE IF EXISTS test.dt; +DROP TABLE IF EXISTS test.testx; + +CREATE TABLE test.dt(tkey Int32) ENGINE = MergeTree order by tuple(); +INSERT INTO test.dt VALUES (300000); +CREATE TABLE test.testx(t Int32, a UInt8) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO test.testx VALUES (100000, 0); + +SELECT COUNT(*) FROM test.testx WHERE NOT a AND t < (SELECT tkey FROM test.dt); + +DROP TABLE test.dt; +CREATE TABLE test.dt(tkey Int32) ENGINE = MergeTree order by tuple(); +INSERT INTO test.dt VALUES (0); + +SELECT COUNT(*) FROM test.testx WHERE NOT a AND t < (SELECT tkey FROM test.dt); + +DROP TABLE IF EXISTS test.dt; +DROP TABLE IF EXISTS test.testx; From 9127e41415c7ae7d8fcd1674604ac657b360311d Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 20 Oct 2018 13:43:15 +0300 Subject: [PATCH 25/28] Fix hack in test --- dbms/tests/queries/0_stateless/00597_push_down_predicate.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql b/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql index 6098a4ebc3d..7b7f12b8ff3 100644 --- a/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql +++ b/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql @@ -23,7 +23,7 @@ SELECT * FROM (SELECT 1 AS id UNION ALL SELECT 2) WHERE id = 1; SELECT * FROM (SELECT arrayJoin([1, 2, 3]) AS id) WHERE id = 1; SELECT id FROM (SELECT arrayJoin([1, 2, 3]) AS id) WHERE id = 1; -SELECT * FROM (SELECT 1 AS id, (SELECT 1)) WHERE _subquery1 = 1; +SELECT * FROM (SELECT 1 AS id, (SELECT 1) as subquery) WHERE subquery = 1; SELECT * FROM (SELECT toUInt64(b) AS a, sum(id) AS b FROM test.test) WHERE a = 3; SELECT * FROM (SELECT toUInt64(b), sum(id) AS b FROM test.test) WHERE `toUInt64(sum(id))` = 3; SELECT date, id, name, value FROM (SELECT date, name, value, min(id) AS id FROM test.test GROUP BY date, name, value) WHERE id = 1; From d4699bd9d29ababa0c276c5d9fee9caa6445835f Mon Sep 17 00:00:00 2001 From: chertus Date: Mon, 22 Oct 2018 11:25:11 +0300 Subject: [PATCH 26/28] fix FieldToDataType for Decimal128 --- dbms/src/DataTypes/FieldToDataType.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataTypes/FieldToDataType.cpp b/dbms/src/DataTypes/FieldToDataType.cpp index 77980c26418..10f083173e4 100644 --- a/dbms/src/DataTypes/FieldToDataType.cpp +++ b/dbms/src/DataTypes/FieldToDataType.cpp @@ -73,7 +73,7 @@ DataTypePtr FieldToDataType::operator() (const DecimalField & x) cons DataTypePtr FieldToDataType::operator() (const DecimalField & x) const { - using Type = DataTypeDecimal; + using Type = DataTypeDecimal; return std::make_shared(Type::maxPrecision(), x.getScale()); } From c6debd1c33d4b6a03b8e0ec1e755c9c3ee1e29b4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Oct 2018 14:30:24 +0300 Subject: [PATCH 27/28] Better comment --- dbms/src/Interpreters/QueryAliasesVisitor.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/QueryAliasesVisitor.cpp b/dbms/src/Interpreters/QueryAliasesVisitor.cpp index 0202abd3e02..16f5d8d29ba 100644 --- a/dbms/src/Interpreters/QueryAliasesVisitor.cpp +++ b/dbms/src/Interpreters/QueryAliasesVisitor.cpp @@ -67,8 +67,9 @@ void QueryAliasesVisitor::getNodeAlias(const ASTPtr & ast, Aliases & aliases, co } else if (auto subquery = typeid_cast(ast.get())) { - /// Set unique aliases for all subqueries. This is needed, because content of subqueries could change after recursive analysis, - /// and auto-generated column names could become incorrect. + /// Set unique aliases for all subqueries. This is needed, because: + /// 1) content of subqueries could change after recursive analysis, and auto-generated column names could become incorrect + /// 2) result of different scalar subqueries can be cached inside expressions compilation cache and must have different names if (subquery->alias.empty()) { From bd38b4f560e49a72ca108975bd0d976bc23d445c Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Mon, 22 Oct 2018 15:38:04 +0300 Subject: [PATCH 28/28] add clear error message #275 --- dbms/src/Interpreters/Cluster.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/dbms/src/Interpreters/Cluster.cpp b/dbms/src/Interpreters/Cluster.cpp index a27dcdf6265..4bf446b107b 100644 --- a/dbms/src/Interpreters/Cluster.cpp +++ b/dbms/src/Interpreters/Cluster.cpp @@ -156,7 +156,12 @@ void Clusters::updateClusters(Poco::Util::AbstractConfiguration & config, const impl.clear(); for (const auto & key : config_keys) + { + if (key.find('.') != String::npos) + throw Exception("Cluster names with dots are not supported: `" + key + "`", ErrorCodes::SYNTAX_ERROR); + impl.emplace(key, std::make_shared(config, settings, config_name + "." + key)); + } } Clusters::Impl Clusters::getContainer() const