Merge branch 'master' of https://github.com/yandex/ClickHouse into merging_with_ties_and_with_fill

This commit is contained in:
CurtizJ 2019-08-14 20:24:47 +03:00
commit a785d4a435
29 changed files with 418 additions and 287 deletions

View File

@ -2,7 +2,7 @@
#include <Poco/Logger.h> #include <Poco/Logger.h>
#include <DataStreams/IBlockInputStream.h> #include <DataStreams/IBlockInputStream.h>
#include <Interpreters/ExpressionAnalyzer.h> /// SubqueriesForSets #include <Interpreters/SubqueryForSet.h>
namespace Poco { class Logger; } namespace Poco { class Logger; }

View File

@ -1,5 +1,6 @@
#include "LibraryDictionarySource.h" #include "LibraryDictionarySource.h"
#include <DataStreams/OneBlockInputStream.h> #include <DataStreams/OneBlockInputStream.h>
#include <Core/Defines.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Poco/File.h> #include <Poco/File.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
@ -134,7 +135,7 @@ LibraryDictionarySource::LibraryDictionarySource(
ErrorCodes::FILE_DOESNT_EXIST); ErrorCodes::FILE_DOESNT_EXIST);
description.init(sample_block); description.init(sample_block);
library = std::make_shared<SharedLibrary>(path, RTLD_LAZY library = std::make_shared<SharedLibrary>(path, RTLD_LAZY
#if defined(RTLD_DEEPBIND) // Does not exists in freebsd #if defined(RTLD_DEEPBIND) && !defined(ADDRESS_SANITIZER) // Does not exists in FreeBSD. Cannot work with Address Sanitizer.
| RTLD_DEEPBIND | RTLD_DEEPBIND
#endif #endif
); );

View File

@ -0,0 +1,54 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <Interpreters/Context.h>
#include <DataTypes/DataTypeString.h>
namespace DB
{
class FunctionCurrentUser : public IFunction
{
const String user_name;
public:
static constexpr auto name = "currentUser";
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionCurrentUser>(context.getClientInfo().initial_user);
}
explicit FunctionCurrentUser(const String & user_name_) : user_name{user_name_}
{
}
String getName() const override
{
return name;
}
size_t getNumberOfArguments() const override
{
return 0;
}
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override
{
return std::make_shared<DataTypeString>();
}
bool isDeterministic() const override { return false; }
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, user_name);
}
};
void registerFunctionCurrentUser(FunctionFactory & factory)
{
factory.registerFunction<FunctionCurrentUser>();
factory.registerAlias("user", FunctionCurrentUser::name, FunctionFactory::CaseInsensitive);
}
}

View File

@ -6,6 +6,7 @@ namespace DB
class FunctionFactory; class FunctionFactory;
void registerFunctionCurrentDatabase(FunctionFactory &); void registerFunctionCurrentDatabase(FunctionFactory &);
void registerFunctionCurrentUser(FunctionFactory &);
void registerFunctionHostName(FunctionFactory &); void registerFunctionHostName(FunctionFactory &);
void registerFunctionVisibleWidth(FunctionFactory &); void registerFunctionVisibleWidth(FunctionFactory &);
void registerFunctionToTypeName(FunctionFactory &); void registerFunctionToTypeName(FunctionFactory &);
@ -56,6 +57,7 @@ void registerFunctionConvertCharset(FunctionFactory &);
void registerFunctionsMiscellaneous(FunctionFactory & factory) void registerFunctionsMiscellaneous(FunctionFactory & factory)
{ {
registerFunctionCurrentDatabase(factory); registerFunctionCurrentDatabase(factory);
registerFunctionCurrentUser(factory);
registerFunctionHostName(factory); registerFunctionHostName(factory);
registerFunctionVisibleWidth(factory); registerFunctionVisibleWidth(factory);
registerFunctionToTypeName(factory); registerFunctionToTypeName(factory);

View File

@ -54,7 +54,6 @@ struct ScopeStack
/// Collect ExpressionAction from AST. Returns PreparedSets and SubqueriesForSets too. /// Collect ExpressionAction from AST. Returns PreparedSets and SubqueriesForSets too.
/// After AST is visited source ExpressionActions should be updated with popActionsLevel() method.
class ActionsVisitor class ActionsVisitor
{ {
public: public:
@ -63,9 +62,11 @@ public:
PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_,
bool no_subqueries_, bool only_consts_, bool no_storage_or_local_, std::ostream * ostr_ = nullptr); bool no_subqueries_, bool only_consts_, bool no_storage_or_local_, std::ostream * ostr_ = nullptr);
void visit(const ASTPtr & ast); void visit(const ASTPtr & ast, ExpressionActionsPtr & actions)
{
ExpressionActionsPtr popActionsLevel() { return actions_stack.popLevel(); } visit(ast);
actions = actions_stack.popLevel();
}
private: private:
const Context & context; const Context & context;
@ -81,6 +82,7 @@ private:
std::ostream * ostr; std::ostream * ostr;
ScopeStack actions_stack; ScopeStack actions_stack;
void visit(const ASTPtr & ast);
SetPtr makeSet(const ASTFunction * node, const Block & sample_block); SetPtr makeSet(const ASTFunction * node, const Block & sample_block);
}; };

View File

@ -24,6 +24,7 @@
#include <Interpreters/ExpressionAnalyzer.h> #include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h> #include <Interpreters/ExpressionActions.h>
#include <Interpreters/ActionsVisitor.h>
#include <Interpreters/InJoinSubqueriesPreprocessor.h> #include <Interpreters/InJoinSubqueriesPreprocessor.h>
#include <Interpreters/LogicalExpressionsOptimizer.h> #include <Interpreters/LogicalExpressionsOptimizer.h>
#include <Interpreters/PredicateExpressionsOptimizer.h> #include <Interpreters/PredicateExpressionsOptimizer.h>
@ -58,6 +59,7 @@
#include <Interpreters/ActionsVisitor.h> #include <Interpreters/ActionsVisitor.h>
#include <Interpreters/ExternalTablesVisitor.h> #include <Interpreters/ExternalTablesVisitor.h>
#include <Interpreters/GlobalSubqueriesVisitor.h> #include <Interpreters/GlobalSubqueriesVisitor.h>
#include <Interpreters/GetAggregatesVisitor.h>
namespace DB namespace DB
{ {
@ -68,7 +70,6 @@ using LogAST = DebugASTLog<false>; /// set to true to enable logs
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int UNKNOWN_IDENTIFIER; extern const int UNKNOWN_IDENTIFIER;
extern const int ILLEGAL_AGGREGATION;
extern const int EXPECTED_ALL_OR_ANY; extern const int EXPECTED_ALL_OR_ANY;
} }
@ -78,16 +79,15 @@ ExpressionAnalyzer::ExpressionAnalyzer(
const Context & context_, const Context & context_,
const NameSet & required_result_columns_, const NameSet & required_result_columns_,
size_t subquery_depth_, size_t subquery_depth_,
bool do_global_, bool do_global)
const SubqueriesForSets & subqueries_for_sets_) : ExpressionAnalyzerData(required_result_columns_)
: ExpressionAnalyzerData(required_result_columns_, subqueries_for_sets_)
, query(query_), context(context_), settings(context.getSettings()) , query(query_), context(context_), settings(context.getSettings())
, subquery_depth(subquery_depth_), do_global(do_global_) , subquery_depth(subquery_depth_)
, syntax(syntax_analyzer_result_) , syntax(syntax_analyzer_result_)
{ {
/// external_tables, subqueries_for_sets for global subqueries. /// external_tables, subqueries_for_sets for global subqueries.
/// Replaces global subqueries with the generated names of temporary tables that will be sent to remote servers. /// Replaces global subqueries with the generated names of temporary tables that will be sent to remote servers.
initGlobalSubqueriesAndExternalTables(); initGlobalSubqueriesAndExternalTables(do_global);
/// has_aggregation, aggregation_keys, aggregate_descriptions, aggregated_columns. /// has_aggregation, aggregation_keys, aggregate_descriptions, aggregated_columns.
/// This analysis should be performed after processing global subqueries, because otherwise, /// This analysis should be performed after processing global subqueries, because otherwise,
@ -116,9 +116,6 @@ void ExpressionAnalyzer::analyzeAggregation()
auto * select_query = query->as<ASTSelectQuery>(); auto * select_query = query->as<ASTSelectQuery>();
if (select_query && (select_query->groupBy() || select_query->having()))
has_aggregation = true;
ExpressionActionsPtr temp_actions = std::make_shared<ExpressionActions>(sourceColumns(), context); ExpressionActionsPtr temp_actions = std::make_shared<ExpressionActions>(sourceColumns(), context);
if (select_query) if (select_query)
@ -150,11 +147,13 @@ void ExpressionAnalyzer::analyzeAggregation()
} }
} }
getAggregates(query, temp_actions); has_aggregation = makeAggregateDescriptions(temp_actions);
if (select_query && (select_query->groupBy() || select_query->having()))
has_aggregation = true;
if (has_aggregation) if (has_aggregation)
{ {
assertSelect(); getSelectQuery(); /// assertSelect()
/// Find out aggregation keys. /// Find out aggregation keys.
if (select_query->groupBy()) if (select_query->groupBy())
@ -223,7 +222,7 @@ void ExpressionAnalyzer::analyzeAggregation()
} }
void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables() void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables(bool do_global)
{ {
/// Adds existing external tables (not subqueries) to the external_tables dictionary. /// Adds existing external tables (not subqueries) to the external_tables dictionary.
ExternalTablesVisitor::Data tables_data{context, external_tables}; ExternalTablesVisitor::Data tables_data{context, external_tables};
@ -238,20 +237,6 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables()
} }
void ExpressionAnalyzer::makeSetsForIndex()
{
const auto * select_query = query->as<ASTSelectQuery>();
if (storage() && select_query && storage()->supportsIndexForIn())
{
if (select_query->where())
makeSetsForIndexImpl(select_query->where());
if (select_query->prewhere())
makeSetsForIndexImpl(select_query->prewhere());
}
}
void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name) void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name)
{ {
auto set_key = PreparedSetKey::forSubquery(*subquery_or_table_name); auto set_key = PreparedSetKey::forSubquery(*subquery_or_table_name);
@ -277,8 +262,12 @@ void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_
} }
void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node) /// Perfomance optimisation for IN() if storage supports it.
void ExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node)
{ {
if (!node || !storage() || !storage()->supportsIndexForIn())
return;
for (auto & child : node->children) for (auto & child : node->children)
{ {
/// Don't descend into subqueries. /// Don't descend into subqueries.
@ -290,15 +279,16 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node)
if (func && func->name == "lambda") if (func && func->name == "lambda")
continue; continue;
makeSetsForIndexImpl(child); makeSetsForIndex(child);
} }
const auto * func = node->as<ASTFunction>(); const auto * func = node->as<ASTFunction>();
if (func && functionIsInOperator(func->name)) if (func && functionIsInOperator(func->name))
{ {
const IAST & args = *func->arguments; const IAST & args = *func->arguments;
const ASTPtr & left_in_operand = args.children.at(0);
if (storage() && storage()->mayBenefitFromIndexForIn(args.children.at(0), context)) if (storage()->mayBenefitFromIndexForIn(left_in_operand, context))
{ {
const ASTPtr & arg = args.children.at(1); const ASTPtr & arg = args.children.at(1);
if (arg->as<ASTSubquery>() || arg->as<ASTIdentifier>()) if (arg->as<ASTSubquery>() || arg->as<ASTIdentifier>())
@ -310,13 +300,13 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node)
{ {
NamesAndTypesList temp_columns = sourceColumns(); NamesAndTypesList temp_columns = sourceColumns();
temp_columns.insert(temp_columns.end(), array_join_columns.begin(), array_join_columns.end()); temp_columns.insert(temp_columns.end(), array_join_columns.begin(), array_join_columns.end());
for (const auto & joined_column : columnsAddedByJoin()) temp_columns.insert(temp_columns.end(), columnsAddedByJoin().begin(), columnsAddedByJoin().end());
temp_columns.push_back(joined_column);
ExpressionActionsPtr temp_actions = std::make_shared<ExpressionActions>(temp_columns, context); ExpressionActionsPtr temp_actions = std::make_shared<ExpressionActions>(temp_columns, context);
getRootActions(func->arguments->children.at(0), true, temp_actions); getRootActions(left_in_operand, true, temp_actions);
Block sample_block_with_calculated_columns = temp_actions->getSampleBlock(); Block sample_block_with_calculated_columns = temp_actions->getSampleBlock();
if (sample_block_with_calculated_columns.has(args.children.at(0)->getColumnName())) if (sample_block_with_calculated_columns.has(left_in_operand->getColumnName()))
makeExplicitSet(func, sample_block_with_calculated_columns, true, context, makeExplicitSet(func, sample_block_with_calculated_columns, true, context,
settings.size_limits_for_set, prepared_sets); settings.size_limits_for_set, prepared_sets);
} }
@ -331,71 +321,23 @@ void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries,
ActionsVisitor actions_visitor(context, settings.size_limits_for_set, subquery_depth, ActionsVisitor actions_visitor(context, settings.size_limits_for_set, subquery_depth,
sourceColumns(), actions, prepared_sets, subqueries_for_sets, sourceColumns(), actions, prepared_sets, subqueries_for_sets,
no_subqueries, only_consts, !isRemoteStorage(), log.stream()); no_subqueries, only_consts, !isRemoteStorage(), log.stream());
actions_visitor.visit(ast); actions_visitor.visit(ast, actions);
actions = actions_visitor.popActionsLevel();
} }
void ExpressionAnalyzer::getActionsFromJoinKeys(const ASTTableJoin & table_join, bool no_subqueries, ExpressionActionsPtr & actions) bool ExpressionAnalyzer::makeAggregateDescriptions(ExpressionActionsPtr & actions)
{ {
bool only_consts = false; for (const ASTFunction * node : aggregates())
LogAST log;
ActionsVisitor actions_visitor(context, settings.size_limits_for_set, subquery_depth,
sourceColumns(), actions, prepared_sets, subqueries_for_sets,
no_subqueries, only_consts, !isRemoteStorage(), log.stream());
if (table_join.using_expression_list)
actions_visitor.visit(table_join.using_expression_list);
else if (table_join.on_expression)
{ {
for (const auto & ast : analyzedJoin().key_asts_left)
actions_visitor.visit(ast);
}
actions = actions_visitor.popActionsLevel();
}
void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr & actions)
{
const auto * select_query = query->as<ASTSelectQuery>();
/// There can not be aggregate functions inside the WHERE and PREWHERE.
if (select_query && (ast.get() == select_query->where().get() || ast.get() == select_query->prewhere().get()))
{
assertNoAggregates(ast, "in WHERE or PREWHERE");
return;
}
/// If we are not analyzing a SELECT query, but a separate expression, then there can not be aggregate functions in it.
if (!select_query)
{
assertNoAggregates(ast, "in wrong place");
return;
}
const auto * node = ast->as<ASTFunction>();
if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
{
has_aggregation = true;
AggregateDescription aggregate; AggregateDescription aggregate;
aggregate.column_name = node->getColumnName(); aggregate.column_name = node->getColumnName();
/// Make unique aggregate functions.
for (size_t i = 0; i < aggregate_descriptions.size(); ++i)
if (aggregate_descriptions[i].column_name == aggregate.column_name)
return;
const ASTs & arguments = node->arguments->children; const ASTs & arguments = node->arguments->children;
aggregate.argument_names.resize(arguments.size()); aggregate.argument_names.resize(arguments.size());
DataTypes types(arguments.size()); DataTypes types(arguments.size());
for (size_t i = 0; i < arguments.size(); ++i) for (size_t i = 0; i < arguments.size(); ++i)
{ {
/// There can not be other aggregate functions within the aggregate functions.
assertNoAggregates(arguments[i], "inside another aggregate function");
getRootActions(arguments[i], true, actions); getRootActions(arguments[i], true, actions);
const std::string & name = arguments[i]->getColumnName(); const std::string & name = arguments[i]->getColumnName();
types[i] = actions->getSampleBlock().getByName(name).type; types[i] = actions->getSampleBlock().getByName(name).type;
@ -407,41 +349,24 @@ void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr
aggregate_descriptions.push_back(aggregate); aggregate_descriptions.push_back(aggregate);
} }
else
{ return !aggregates().empty();
for (const auto & child : ast->children)
if (!child->as<ASTSubquery>() && !child->as<ASTSelectQuery>())
getAggregates(child, actions);
}
} }
void ExpressionAnalyzer::assertNoAggregates(const ASTPtr & ast, const char * description) const ASTSelectQuery * ExpressionAnalyzer::getSelectQuery() const
{
const auto * node = ast->as<ASTFunction>();
if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
throw Exception("Aggregate function " + node->getColumnName()
+ " is found " + String(description) + " in query", ErrorCodes::ILLEGAL_AGGREGATION);
for (const auto & child : ast->children)
if (!child->as<ASTSubquery>() && !child->as<ASTSelectQuery>())
assertNoAggregates(child, description);
}
void ExpressionAnalyzer::assertSelect() const
{ {
const auto * select_query = query->as<ASTSelectQuery>(); const auto * select_query = query->as<ASTSelectQuery>();
if (!select_query) if (!select_query)
throw Exception("Not a select query", ErrorCodes::LOGICAL_ERROR); throw Exception("Not a select query", ErrorCodes::LOGICAL_ERROR);
return select_query;
} }
void ExpressionAnalyzer::assertAggregation() const const ASTSelectQuery * ExpressionAnalyzer::getAggregatingQuery() const
{ {
if (!has_aggregation) if (!has_aggregation)
throw Exception("No aggregation", ErrorCodes::LOGICAL_ERROR); throw Exception("No aggregation", ErrorCodes::LOGICAL_ERROR);
return getSelectQuery();
} }
void ExpressionAnalyzer::initChain(ExpressionActionsChain & chain, const NamesAndTypesList & columns) const void ExpressionAnalyzer::initChain(ExpressionActionsChain & chain, const NamesAndTypesList & columns) const
@ -471,9 +396,7 @@ void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActionsPtr & actio
bool ExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, bool only_types) bool ExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, bool only_types)
{ {
const auto * select_query = query->as<ASTSelectQuery>(); const auto * select_query = getSelectQuery();
assertSelect();
bool is_array_join_left; bool is_array_join_left;
ASTPtr array_join_expression_list = select_query->array_join_expression_list(is_array_join_left); ASTPtr array_join_expression_list = select_query->array_join_expression_list(is_array_join_left);
@ -515,9 +438,7 @@ static void appendRequiredColumns(
bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_types) bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_types)
{ {
const auto * select_query = query->as<ASTSelectQuery>(); const auto * select_query = getSelectQuery();
assertSelect();
if (!select_query->join()) if (!select_query->join())
return false; return false;
@ -538,9 +459,18 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
throw Exception("Expected ANY or ALL in JOIN section, because setting (join_default_strictness) is empty", DB::ErrorCodes::EXPECTED_ALL_OR_ANY); throw Exception("Expected ANY or ALL in JOIN section, because setting (join_default_strictness) is empty", DB::ErrorCodes::EXPECTED_ALL_OR_ANY);
} }
const auto & table_to_join = join_element.table_expression->as<ASTTableExpression &>(); if (join_params.using_expression_list)
{
getRootActions(join_params.using_expression_list, only_types, step.actions);
}
else if (join_params.on_expression)
{
auto list = std::make_shared<ASTExpressionList>();
list->children = analyzedJoin().key_asts_left;
getRootActions(list, only_types, step.actions);
}
getActionsFromJoinKeys(join_params, only_types, step.actions); const auto & table_to_join = join_element.table_expression->as<ASTTableExpression &>();
/// Two JOINs are not supported with the same subquery, but different USINGs. /// Two JOINs are not supported with the same subquery, but different USINGs.
auto join_hash = join_element.getTreeHash(); auto join_hash = join_element.getTreeHash();
@ -626,9 +556,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
bool ExpressionAnalyzer::appendPrewhere( bool ExpressionAnalyzer::appendPrewhere(
ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns) ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns)
{ {
const auto * select_query = query->as<ASTSelectQuery>(); const auto * select_query = getSelectQuery();
assertSelect();
if (!select_query->prewhere()) if (!select_query->prewhere())
return false; return false;
@ -701,9 +629,7 @@ bool ExpressionAnalyzer::appendPrewhere(
bool ExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, bool only_types) bool ExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, bool only_types)
{ {
const auto * select_query = query->as<ASTSelectQuery>(); const auto * select_query = getSelectQuery();
assertSelect();
if (!select_query->where()) if (!select_query->where())
return false; return false;
@ -721,9 +647,7 @@ bool ExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, bool only_t
bool ExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain, bool only_types) bool ExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain, bool only_types)
{ {
const auto * select_query = query->as<ASTSelectQuery>(); const auto * select_query = getAggregatingQuery();
assertAggregation();
if (!select_query->groupBy()) if (!select_query->groupBy())
return false; return false;
@ -743,9 +667,7 @@ bool ExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain, bool only
void ExpressionAnalyzer::appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types) void ExpressionAnalyzer::appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types)
{ {
const auto * select_query = query->as<ASTSelectQuery>(); const auto * select_query = getAggregatingQuery();
assertAggregation();
initChain(chain, sourceColumns()); initChain(chain, sourceColumns());
ExpressionActionsChain::Step & step = chain.steps.back(); ExpressionActionsChain::Step & step = chain.steps.back();
@ -758,20 +680,27 @@ void ExpressionAnalyzer::appendAggregateFunctionsArguments(ExpressionActionsChai
} }
} }
getActionsBeforeAggregation(select_query->select(), step.actions, only_types); /// Collect aggregates removing duplicates by node.getColumnName()
/// It's not clear why we recollect aggregates (for query parts) while we're able to use previously collected ones (for entire query)
/// @note The original recollection logic didn't remove duplicates.
GetAggregatesVisitor::Data data;
GetAggregatesVisitor(data).visit(select_query->select());
if (select_query->having()) if (select_query->having())
getActionsBeforeAggregation(select_query->having(), step.actions, only_types); GetAggregatesVisitor(data).visit(select_query->having());
if (select_query->orderBy()) if (select_query->orderBy())
getActionsBeforeAggregation(select_query->orderBy(), step.actions, only_types); GetAggregatesVisitor(data).visit(select_query->orderBy());
/// TODO: data.aggregates -> aggregates()
for (const ASTFunction * node : data.aggregates)
for (auto & argument : node->arguments->children)
getRootActions(argument, only_types, step.actions);
} }
bool ExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_types) bool ExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_types)
{ {
const auto * select_query = query->as<ASTSelectQuery>(); const auto * select_query = getAggregatingQuery();
assertAggregation();
if (!select_query->having()) if (!select_query->having())
return false; return false;
@ -787,9 +716,7 @@ bool ExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_
void ExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, bool only_types) void ExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, bool only_types)
{ {
const auto * select_query = query->as<ASTSelectQuery>(); const auto * select_query = getSelectQuery();
assertSelect();
initChain(chain, aggregated_columns); initChain(chain, aggregated_columns);
ExpressionActionsChain::Step & step = chain.steps.back(); ExpressionActionsChain::Step & step = chain.steps.back();
@ -802,9 +729,7 @@ void ExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, bool only_
bool ExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only_types) bool ExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only_types)
{ {
const auto * select_query = query->as<ASTSelectQuery>(); const auto * select_query = getSelectQuery();
assertSelect();
if (!select_query->orderBy()) if (!select_query->orderBy())
return false; return false;
@ -828,9 +753,7 @@ bool ExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only
bool ExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain, bool only_types) bool ExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain, bool only_types)
{ {
const auto * select_query = query->as<ASTSelectQuery>(); const auto * select_query = getSelectQuery();
assertSelect();
if (!select_query->limitBy()) if (!select_query->limitBy())
return false; return false;
@ -859,9 +782,7 @@ bool ExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain, bool only
void ExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) const void ExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) const
{ {
const auto * select_query = query->as<ASTSelectQuery>(); const auto * select_query = getSelectQuery();
assertSelect();
initChain(chain, aggregated_columns); initChain(chain, aggregated_columns);
ExpressionActionsChain::Step & step = chain.steps.back(); ExpressionActionsChain::Step & step = chain.steps.back();
@ -892,19 +813,6 @@ void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const
} }
void ExpressionAnalyzer::getActionsBeforeAggregation(const ASTPtr & ast, ExpressionActionsPtr & actions, bool no_subqueries)
{
const auto * node = ast->as<ASTFunction>();
if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
for (auto & argument : node->arguments->children)
getRootActions(argument, no_subqueries, actions);
else
for (auto & child : ast->children)
getActionsBeforeAggregation(child, actions, no_subqueries);
}
ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool project_result) ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool project_result)
{ {
ExpressionActionsPtr actions = std::make_shared<ExpressionActions>(sourceColumns(), context); ExpressionActionsPtr actions = std::make_shared<ExpressionActions>(sourceColumns(), context);

View File

@ -2,9 +2,9 @@
#include <Core/Settings.h> #include <Core/Settings.h>
#include <DataStreams/IBlockStream_fwd.h> #include <DataStreams/IBlockStream_fwd.h>
#include <Interpreters/ActionsVisitor.h>
#include <Interpreters/AggregateDescription.h> #include <Interpreters/AggregateDescription.h>
#include <Interpreters/SyntaxAnalyzer.h> #include <Interpreters/SyntaxAnalyzer.h>
#include <Interpreters/SubqueryForSet.h>
#include <Parsers/IAST_fwd.h> #include <Parsers/IAST_fwd.h>
#include <Storages/IStorage_fwd.h> #include <Storages/IStorage_fwd.h>
@ -51,10 +51,8 @@ struct ExpressionAnalyzerData
Tables external_tables; Tables external_tables;
protected: protected:
ExpressionAnalyzerData(const NameSet & required_result_columns_, ExpressionAnalyzerData(const NameSet & required_result_columns_)
const SubqueriesForSets & subqueries_for_sets_) : required_result_columns(required_result_columns_)
: required_result_columns(required_result_columns_),
subqueries_for_sets(subqueries_for_sets_)
{} {}
}; };
@ -91,8 +89,7 @@ public:
const Context & context_, const Context & context_,
const NameSet & required_result_columns_ = {}, const NameSet & required_result_columns_ = {},
size_t subquery_depth_ = 0, size_t subquery_depth_ = 0,
bool do_global_ = false, bool do_global_ = false);
const SubqueriesForSets & subqueries_for_set_ = {});
/// Does the expression have aggregate functions or a GROUP BY or HAVING section. /// Does the expression have aggregate functions or a GROUP BY or HAVING section.
bool hasAggregation() const { return has_aggregation; } bool hasAggregation() const { return has_aggregation; }
@ -161,7 +158,7 @@ public:
const ExpressionAnalyzerData & getAnalyzedData() const { return *this; } const ExpressionAnalyzerData & getAnalyzedData() const { return *this; }
/// Create Set-s that we can from IN section to use the index on them. /// Create Set-s that we can from IN section to use the index on them.
void makeSetsForIndex(); void makeSetsForIndex(const ASTPtr & node);
bool hasGlobalSubqueries() { return has_global_subqueries; } bool hasGlobalSubqueries() { return has_global_subqueries; }
@ -170,7 +167,6 @@ private:
const Context & context; const Context & context;
const ExtractedSettings settings; const ExtractedSettings settings;
size_t subquery_depth; size_t subquery_depth;
bool do_global; /// Do I need to prepare for execution global subqueries when analyzing the query.
SyntaxAnalyzerResultPtr syntax; SyntaxAnalyzerResultPtr syntax;
@ -178,35 +174,30 @@ private:
const AnalyzedJoin & analyzedJoin() const { return syntax->analyzed_join; } const AnalyzedJoin & analyzedJoin() const { return syntax->analyzed_join; }
const NamesAndTypesList & sourceColumns() const { return syntax->required_source_columns; } const NamesAndTypesList & sourceColumns() const { return syntax->required_source_columns; }
const NamesAndTypesList & columnsAddedByJoin() const { return syntax->columns_added_by_join; } const NamesAndTypesList & columnsAddedByJoin() const { return syntax->columns_added_by_join; }
const std::vector<const ASTFunction *> & aggregates() const { return syntax->aggregates; }
/// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables. /// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables.
void initGlobalSubqueriesAndExternalTables(); void initGlobalSubqueriesAndExternalTables(bool do_global);
void addMultipleArrayJoinAction(ExpressionActionsPtr & actions, bool is_left) const; void addMultipleArrayJoinAction(ExpressionActionsPtr & actions, bool is_left) const;
void addJoinAction(ExpressionActionsPtr & actions, bool only_types) const; void addJoinAction(ExpressionActionsPtr & actions, bool only_types) const;
/// If ast is ASTSelectQuery with JOIN, add actions for JOIN key columns.
void getActionsFromJoinKeys(const ASTTableJoin & table_join, bool no_subqueries, ExpressionActionsPtr & actions);
void getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts = false); void getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts = false);
void getActionsBeforeAggregation(const ASTPtr & ast, ExpressionActionsPtr & actions, bool no_subqueries);
/** Add aggregation keys to aggregation_keys, aggregate functions to aggregate_descriptions, /** Add aggregation keys to aggregation_keys, aggregate functions to aggregate_descriptions,
* Create a set of columns aggregated_columns resulting after the aggregation, if any, * Create a set of columns aggregated_columns resulting after the aggregation, if any,
* or after all the actions that are normally performed before aggregation. * or after all the actions that are normally performed before aggregation.
* Set has_aggregation = true if there is GROUP BY or at least one aggregate function. * Set has_aggregation = true if there is GROUP BY or at least one aggregate function.
*/ */
void analyzeAggregation(); void analyzeAggregation();
void getAggregates(const ASTPtr & ast, ExpressionActionsPtr & actions); bool makeAggregateDescriptions(ExpressionActionsPtr & actions);
void assertNoAggregates(const ASTPtr & ast, const char * description);
/// columns - the columns that are present before the transformations begin. /// columns - the columns that are present before the transformations begin.
void initChain(ExpressionActionsChain & chain, const NamesAndTypesList & columns) const; void initChain(ExpressionActionsChain & chain, const NamesAndTypesList & columns) const;
void assertSelect() const; const ASTSelectQuery * getSelectQuery() const;
void assertAggregation() const; const ASTSelectQuery * getAggregatingQuery() const;
/** /**
* Create Set from a subquery or a table expression in the query. The created set is suitable for using the index. * Create Set from a subquery or a table expression in the query. The created set is suitable for using the index.
@ -214,8 +205,6 @@ private:
*/ */
void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name); void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name);
void makeSetsForIndexImpl(const ASTPtr & node);
bool isRemoteStorage() const; bool isRemoteStorage() const;
}; };

View File

@ -0,0 +1,75 @@
#pragma once
#include <Interpreters/InDepthNodeVisitor.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_AGGREGATION;
}
class GetAggregatesMatcher
{
public:
using Visitor = ConstInDepthNodeVisitor<GetAggregatesMatcher, true>;
struct Data
{
const char * assert_no_aggregates = nullptr;
std::unordered_set<String> uniq_names;
std::vector<const ASTFunction *> aggregates;
};
static bool needChildVisit(const ASTPtr & node, const ASTPtr & child)
{
if (child->as<ASTSubquery>() || child->as<ASTSelectQuery>())
return false;
if (auto * func = node->as<ASTFunction>())
if (isAggregateFunction(func->name))
return false;
return true;
}
static void visit(const ASTPtr & ast, Data & data)
{
if (auto * func = ast->as<ASTFunction>())
visit(*func, ast, data);
}
private:
static void visit(const ASTFunction & node, const ASTPtr &, Data & data)
{
if (!isAggregateFunction(node.name))
return;
if (data.assert_no_aggregates)
throw Exception("Aggregate function " + node.getColumnName() + " is found " + String(data.assert_no_aggregates) + " in query",
ErrorCodes::ILLEGAL_AGGREGATION);
String column_name = node.getColumnName();
if (data.uniq_names.count(column_name))
return;
data.uniq_names.insert(column_name);
data.aggregates.push_back(&node);
}
static bool isAggregateFunction(const String & name)
{
return AggregateFunctionFactory::instance().isAggregateFunctionName(name);
}
};
using GetAggregatesVisitor = GetAggregatesMatcher::Visitor;
inline void assertNoAggregates(const ASTPtr & ast, const char * description)
{
GetAggregatesVisitor::Data data{description, {}, {}};
GetAggregatesVisitor(data).visit(ast);
}
}

View File

@ -7,7 +7,6 @@
#include <Parsers/ASTSelectQuery.h> #include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/ActionsVisitor.h>
#include <Interpreters/interpretSubquery.h> #include <Interpreters/interpretSubquery.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <Core/Block.h> #include <Core/Block.h>

View File

@ -840,7 +840,8 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS
if (!dry_run) if (!dry_run)
from_stage = storage->getQueryProcessingStage(context); from_stage = storage->getQueryProcessingStage(context);
query_analyzer->makeSetsForIndex(); query_analyzer->makeSetsForIndex(query.where());
query_analyzer->makeSetsForIndex(query.prewhere());
auto optimize_prewhere = [&](auto & merge_tree) auto optimize_prewhere = [&](auto & merge_tree)
{ {

View File

@ -1,7 +1,6 @@
#pragma once #pragma once
#include "DatabaseAndTableWithAlias.h" #include "DatabaseAndTableWithAlias.h"
#include "ExpressionAnalyzer.h"
#include <Parsers/ASTSelectQuery.h> #include <Parsers/ASTSelectQuery.h>
#include <map> #include <map>

View File

@ -12,7 +12,6 @@ class Join;
using JoinPtr = std::shared_ptr<Join>; using JoinPtr = std::shared_ptr<Join>;
class InterpreterSelectWithUnionQuery; class InterpreterSelectWithUnionQuery;
struct AnalyzedJoin;
/// Information on what to do when executing a subquery in the [GLOBAL] IN/JOIN section. /// Information on what to do when executing a subquery in the [GLOBAL] IN/JOIN section.

View File

@ -14,6 +14,8 @@
#include <Interpreters/ExternalDictionaries.h> #include <Interpreters/ExternalDictionaries.h>
#include <Interpreters/OptimizeIfWithConstantConditionVisitor.h> #include <Interpreters/OptimizeIfWithConstantConditionVisitor.h>
#include <Interpreters/RequiredSourceColumnsVisitor.h> #include <Interpreters/RequiredSourceColumnsVisitor.h>
#include <Interpreters/GetAggregatesVisitor.h>
#include <Interpreters/ExpressionActions.h> /// getSmallestColumn()
#include <Parsers/ASTExpressionList.h> #include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>
@ -558,6 +560,30 @@ void checkJoin(const ASTTablesInSelectQueryElement * join)
ErrorCodes::NOT_IMPLEMENTED); ErrorCodes::NOT_IMPLEMENTED);
} }
std::vector<const ASTFunction *> getAggregates(const ASTPtr & query)
{
if (const auto * select_query = query->as<ASTSelectQuery>())
{
/// There can not be aggregate functions inside the WHERE and PREWHERE.
if (select_query->where())
assertNoAggregates(select_query->where(), "in WHERE");
if (select_query->prewhere())
assertNoAggregates(select_query->prewhere(), "in PREWHERE");
GetAggregatesVisitor::Data data;
GetAggregatesVisitor(data).visit(query);
/// There can not be other aggregate functions within the aggregate functions.
for (const ASTFunction * node : data.aggregates)
for (auto & arg : node->arguments->children)
assertNoAggregates(arg, "inside another aggregate function");
return data.aggregates;
}
else
assertNoAggregates(query, "in wrong place");
return {};
}
} }
/// Calculate which columns are required to execute the expression. /// Calculate which columns are required to execute the expression.
@ -840,6 +866,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
collectJoinedColumns(result.analyzed_join, *select_query, source_columns_set, result.aliases, settings.join_use_nulls); collectJoinedColumns(result.analyzed_join, *select_query, source_columns_set, result.aliases, settings.join_use_nulls);
} }
result.aggregates = getAggregates(query);
result.collectUsedColumns(query, additional_source_columns); result.collectUsedColumns(query, additional_source_columns);
return std::make_shared<const SyntaxAnalyzerResult>(result); return std::make_shared<const SyntaxAnalyzerResult>(result);
} }

View File

@ -10,6 +10,8 @@ namespace DB
NameSet removeDuplicateColumns(NamesAndTypesList & columns); NameSet removeDuplicateColumns(NamesAndTypesList & columns);
class ASTFunction;
struct SyntaxAnalyzerResult struct SyntaxAnalyzerResult
{ {
StoragePtr storage; StoragePtr storage;
@ -22,6 +24,7 @@ struct SyntaxAnalyzerResult
NamesAndTypesList columns_added_by_join; NamesAndTypesList columns_added_by_join;
Aliases aliases; Aliases aliases;
std::vector<const ASTFunction *> aggregates;
/// Which column is needed to be ARRAY-JOIN'ed to get the specified. /// Which column is needed to be ARRAY-JOIN'ed to get the specified.
/// For example, for `SELECT s.v ... ARRAY JOIN a AS s` will get "s.v" -> "a.v". /// For example, for `SELECT s.v ... ARRAY JOIN a AS s` will get "s.v" -> "a.v".

View File

@ -16,6 +16,7 @@
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>
#include <Parsers/ASTSampleRatio.h> #include <Parsers/ASTSampleRatio.h>
#include <Interpreters/ExpressionAnalyzer.h>
/// Allow to use __uint128_t as a template parameter for boost::rational. /// Allow to use __uint128_t as a template parameter for boost::rational.
// https://stackoverflow.com/questions/41198673/uint128-t-not-working-with-clang-and-libstdc // https://stackoverflow.com/questions/41198673/uint128-t-not-working-with-clang-and-libstdc

View File

@ -4,7 +4,6 @@
#include <Core/Block.h> #include <Core/Block.h>
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Parsers/ASTExpressionList.h> #include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTSelectQuery.h> #include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>

View File

@ -0,0 +1,4 @@
1
1
1 1
1

View File

@ -0,0 +1,5 @@
-- since actual user name is unknown, have to perform just smoke tests
select currentUser() IS NOT NULL;
select length(currentUser()) > 0;
select currentUser() = user(), currentUser() = USER();
select currentUser() = initial_user from system.processes where query like '%$!@#%';

View File

@ -1,9 +1,11 @@
# Database Engines # Database Engines
Database engines provide working with tables. Database engines allow you to work with tables.
By default, ClickHouse uses its native database engine which provides configurable [table engines](../operations/table_engines/index.md) and [SQL dialect](../query_language/syntax.md). By default, ClickHouse uses its native database engine, which provides configurable [table engines](../operations/table_engines/index.md) and an [SQL dialect](../query_language/syntax.md).
Also you can use the following database engines: You can also use the following database engines:
- [MySQL](mysql.md) - [MySQL](mysql.md)
[Original article](https://clickhouse.yandex/docs/en/database_engines/) <!--hide-->

View File

@ -1,10 +1,10 @@
# MySQL # MySQL
Allows to connect to some database on remote MySQL server and perform `INSERT` and `SELECT` queries with tables to exchange data between ClickHouse and MySQL. Allows to connect to databases on a remote MySQL server and perform `INSERT` and `SELECT` queries with tables to exchange data between ClickHouse and MySQL.
The `MySQL` database engine translate queries to the MySQL server, so you can perform operations such as `SHOW TABLES` or `SHOW CREATE TABLE`. The `MySQL` database engine translate queries to the MySQL server so you can perform operations such as `SHOW TABLES` or `SHOW CREATE TABLE`.
You cannot perform with tables the following queries: You cannot perform the following queries:
- `ATTACH`/`DETACH` - `ATTACH`/`DETACH`
- `DROP` - `DROP`
@ -48,7 +48,7 @@ BINARY | [FixedString](../data_types/fixedstring.md)
All other MySQL data types are converted into [String](../data_types/string.md). All other MySQL data types are converted into [String](../data_types/string.md).
[Nullable](../data_types/nullable.md) data type is supported. [Nullable](../data_types/nullable.md) is supported.
## Examples of Use ## Examples of Use
@ -120,3 +120,5 @@ SELECT * FROM mysql_db.mysql_table
│ 3 │ 4 │ │ 3 │ 4 │
└────────┴───────┘ └────────┴───────┘
``` ```
[Original article](https://clickhouse.yandex/docs/en/database_engines/mysql/) <!--hide-->

View File

@ -28,7 +28,7 @@ Main features:
## Creating a Table {#table_engine-mergetree-creating-a-table} ## Creating a Table {#table_engine-mergetree-creating-a-table}
``` ```sql
CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
( (
name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1], name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1],
@ -47,7 +47,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
For descriptions of request parameters, see the [request description](../../query_language/create.md). For descriptions of request parameters, see the [request description](../../query_language/create.md).
**Query clauses** ### Query Clauses
- `ENGINE` — Name and parameters of the engine. `ENGINE = MergeTree()`. The `MergeTree` engine does not have parameters. - `ENGINE` — Name and parameters of the engine. `ENGINE = MergeTree()`. The `MergeTree` engine does not have parameters.
@ -81,7 +81,7 @@ For descriptions of request parameters, see the [request description](../../quer
<a name="mergetree_setting-merge_with_ttl_timeout"></a> <a name="mergetree_setting-merge_with_ttl_timeout"></a>
- `merge_with_ttl_timeout` — Minimum delay in seconds before repeating a merge with TTL. Default value: 86400 (1 day). - `merge_with_ttl_timeout` — Minimum delay in seconds before repeating a merge with TTL. Default value: 86400 (1 day).
**Example of setting the sections** **Example of Sections Setting**
```sql ```sql
ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192 ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192
@ -107,7 +107,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
) ENGINE [=] MergeTree(date-column [, sampling_expression], (primary, key), index_granularity) ) ENGINE [=] MergeTree(date-column [, sampling_expression], (primary, key), index_granularity)
``` ```
**MergeTree() parameters** **MergeTree() Parameters**
- `date-column` — The name of a column of the [Date](../../data_types/date.md) type. ClickHouse automatically creates partitions by month based on this column. The partition names are in the `"YYYYMM"` format. - `date-column` — The name of a column of the [Date](../../data_types/date.md) type. ClickHouse automatically creates partitions by month based on this column. The partition names are in the `"YYYYMM"` format.
- `sampling_expression` — An expression for sampling. - `sampling_expression` — An expression for sampling.
@ -312,16 +312,16 @@ Reading from a table is automatically parallelized.
Determines the lifetime of values. Determines the lifetime of values.
The `TTL` clause can be set for the whole table and for each individual column. If `TTL` is set for the whole table, individual `TTL` for columns are ignored. The `TTL` clause can be set for the whole table and for each individual column. If both `TTL` are set, ClickHouse uses that `TTL` which expires earlier.
The table must have the column in the [Date](../../data_types/date.md) or [DateTime](../../data_types/datetime.md) data type. To define the lifetime of data, use operations on this time column, for example:
The table must have the column of the [Date](../../data_types/date.md) or [DateTime](../../data_types/datetime.md) data type. This date column should be used in the `TTL` clause. You can only set lifetime of the data as an interval from the date column value.
``` ```
TTL date_time + interval TTL time_column
TTL time_column + interval
``` ```
You can set the `interval` by any expression, returning the value of the `DateTime` data type. For example, you can use [time interval](../../query_language/operators.md#operators-datetime) operators. To define `interval`, use [time interval](../../query_language/operators.md#operators-datetime) operators.
``` ```
TTL date_time + INTERVAL 1 MONTH TTL date_time + INTERVAL 1 MONTH
@ -330,20 +330,20 @@ TTL date_time + INTERVAL 15 HOUR
**Column TTL** **Column TTL**
When the values in the column expire, ClickHouse replace them with the default values for the column data type. If all the column values in the data part become expired, ClickHouse deletes this column from the data part in a filesystem. When the values in the column expire, ClickHouse replaces them with the default values for the column data type. If all the column values in the data part expire, ClickHouse deletes this column from the data part in a filesystem.
The `TTL` clause cannot be used for key columns. The `TTL` clause can't be used for key columns.
**Table TTL** **Table TTL**
When some data in table expires, ClickHouse deletes all the corresponding rows. When data in a table expires, ClickHouse deletes all corresponding rows.
**Cleaning up of Data** **Removing Data**
Data with expired TTL is removed, when ClickHouse merges data parts. Data with an expired TTL is removed when ClickHouse merges data parts.
When ClickHouse see that some data is expired, it performs off-schedule merge. To control frequency of such merges, you can set [merge_with_ttl_timeout](#mergetree_setting-merge_with_ttl_timeout). If it is too low, many off-schedule merges consume much resources. When ClickHouse see that data is expired, it performs an off-schedule merge. To control the frequency of such merges, you can set [merge_with_ttl_timeout](#mergetree_setting-merge_with_ttl_timeout). If the value is too low, it will perform many off-schedule merges that may consume a lot of resources.
If you perform the `SELECT` query between merges you can get the expired data. To avoid it, use the [OPTIMIZE](../../query_language/misc.md#misc_operations-optimize) query before `SELECT`. If you perform the `SELECT` query between merges, you may get expired data. To avoid it, use the [OPTIMIZE](../../query_language/misc.md#misc_operations-optimize) query before `SELECT`.
[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/mergetree/) <!--hide--> [Original article](https://clickhouse.yandex/docs/en/operations/table_engines/mergetree/) <!--hide-->

View File

@ -10,10 +10,10 @@ CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] [ENGINE = engine(..
- `IF NOT EXISTS` - `IF NOT EXISTS`
If the `db_name` database already exists then: If the `db_name` database already exists, then ClickHouse doesn't create a new database and:
- If clause is specified, ClickHouse doesn't create a new database and doesn't throw an exception. - Doesn't throw an exception if clause is specified.
- If clause is not specified, then ClickHouse doesn't create a new database and throw and exception. - Throws an exception if clause isn't specified.
- `ON CLUSTER` - `ON CLUSTER`
@ -23,7 +23,7 @@ CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] [ENGINE = engine(..
- [MySQL](../database_engines/mysql.md) - [MySQL](../database_engines/mysql.md)
Allows to retrieve data from the remote MySQL server. Allows you to retrieve data from the remote MySQL server.
By default, ClickHouse uses its own [database engine](../database_engines/index.md). By default, ClickHouse uses its own [database engine](../database_engines/index.md).

View File

@ -102,6 +102,9 @@ Sleeps 'seconds' seconds on each row. You can specify an integer or a floating-p
Returns the name of the current database. Returns the name of the current database.
You can use this function in table engine parameters in a CREATE TABLE query where you need to specify the database. You can use this function in table engine parameters in a CREATE TABLE query where you need to specify the database.
## currentUser()
Returns the login of authorized user (initiator of query execution).
## isFinite(x) ## isFinite(x)
Accepts Float32 and Float64 and returns UInt8 equal to 1 if the argument is not infinite and not a NaN, otherwise 0. Accepts Float32 and Float64 and returns UInt8 equal to 1 if the argument is not infinite and not a NaN, otherwise 0.

View File

@ -1,9 +1,10 @@
# Database Engines # Движки баз данных
Database engines provide working with tables. Движки баз данных обеспечивают работу с таблицами.
By default, ClickHouse uses its native database engine which provides configurable [table engines](../operations/table_engines/index.md) and [SQL dialect](../query_language/syntax.md). По умолчанию ClickHouse использует собственный движок баз данных, который поддерживает конфигурируемые [движки таблиц](../operations/table_engines/index.md) и [диалект SQL](../query_language/syntax.md).
Also you can use the following database engines: Также можно использовать следующие движки баз данных:
- [MySQL](mysql.md) - [MySQL](mysql.md)

View File

@ -1,10 +1,10 @@
# MySQL # MySQL
Allows to connect to some database on remote MySQL server and perform `INSERT` and `SELECT` queries with tables to exchange data between ClickHouse and MySQL. Позволяет подключаться к базам данных на удалённом MySQL сервере и выполнять запросы `INSERT` и `SELECT` для обмена данными между ClickHouse и MySQL.
The `MySQL` database engine translate queries to the MySQL server, so you can perform operations such as `SHOW TABLES` or `SHOW CREATE TABLE`. Движок баз данных `MySQL` транслирует запросы при передаче на сервер MySQL, что позволяет выполнять и другие виды запросов, например `SHOW TABLES` или `SHOW CREATE TABLE`.
You cannot perform with tables the following queries: Не поддерживаемые виды запросов:
- `ATTACH`/`DETACH` - `ATTACH`/`DETACH`
- `DROP` - `DROP`
@ -12,48 +12,45 @@ You cannot perform with tables the following queries:
- `CREATE TABLE` - `CREATE TABLE`
- `ALTER` - `ALTER`
## Создание базы данных
## Creating a Database ```sql
``` sql
CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster]
ENGINE = MySQL('host:port', 'database', 'user', 'password') ENGINE = MySQL('host:port', 'database', 'user', 'password')
``` ```
**Engine Parameters** **Параметры движка**
- `host:port` — MySQL server address. - `host:port`адрес сервера MySQL.
- `database`Remote database name. - `database`имя базы данных на удалённом сервере.
- `user` — MySQL user. - `user`пользователь MySQL.
- `password`User password. - `password`пароль пользователя.
## Поддержка типов данных
## Data Types Support | MySQL | ClickHouse |
| ------ | ------------ |
| UNSIGNED TINYINT | [UInt8](../data_types/int_uint.md) |
| TINYINT | [Int8](../data_types/int_uint.md) |
| UNSIGNED SMALLINT | [UInt16](../data_types/int_uint.md) |
| SMALLINT | [Int16](../data_types/int_uint.md) |
| UNSIGNED INT, UNSIGNED MEDIUMINT | [UInt32](../data_types/int_uint.md) |
| INT, MEDIUMINT | [Int32](../data_types/int_uint.md) |
| UNSIGNED BIGINT | [UInt64](../data_types/int_uint.md) |
| BIGINT | [Int64](../data_types/int_uint.md) |
| FLOAT | [Float32](../data_types/float.md) |
| DOUBLE | [Float64](../data_types/float.md) |
| DATE | [Date](../data_types/date.md) |
| DATETIME, TIMESTAMP | [DateTime](../data_types/datetime.md) |
| BINARY | [FixedString](../data_types/fixedstring.md) |
MySQL | ClickHouse Все прочие типы данных преобразуются в [String](../data_types/string.md).
------|------------
UNSIGNED TINYINT | [UInt8](../data_types/int_uint.md)
TINYINT | [Int8](../data_types/int_uint.md)
UNSIGNED SMALLINT | [UInt16](../data_types/int_uint.md)
SMALLINT | [Int16](../data_types/int_uint.md)
UNSIGNED INT, UNSIGNED MEDIUMINT | [UInt32](../data_types/int_uint.md)
INT, MEDIUMINT | [Int32](../data_types/int_uint.md)
UNSIGNED BIGINT | [UInt64](../data_types/int_uint.md)
BIGINT | [Int64](../data_types/int_uint.md)
FLOAT | [Float32](../data_types/float.md)
DOUBLE | [Float64](../data_types/float.md)
DATE | [Date](../data_types/date.md)
DATETIME, TIMESTAMP | [DateTime](../data_types/datetime.md)
BINARY | [FixedString](../data_types/fixedstring.md)
All other MySQL data types are converted into [String](../data_types/string.md). [Nullable](../data_types/nullable.md) поддержан.
[Nullable](../data_types/nullable.md) data type is supported. ## Примеры использования
Таблица в MySQL:
## Examples of Use
Table in MySQL:
``` ```
mysql> USE test; mysql> USE test;
@ -77,14 +74,16 @@ mysql> select * from mysql_table;
1 row in set (0,00 sec) 1 row in set (0,00 sec)
``` ```
Database in ClickHouse, exchanging data with the MySQL server: База данных в ClickHouse, позволяющая обмениваться данными с сервером MySQL:
```sql ```sql
CREATE DATABASE mysql_db ENGINE = MySQL('localhost:3306', 'test', 'my_user', 'user_password') CREATE DATABASE mysql_db ENGINE = MySQL('localhost:3306', 'test', 'my_user', 'user_password')
``` ```
```sql ```sql
SHOW DATABASES SHOW DATABASES
``` ```
```text ```text
┌─name─────┐ ┌─name─────┐
│ default │ │ default │
@ -92,31 +91,39 @@ SHOW DATABASES
│ system │ │ system │
└──────────┘ └──────────┘
``` ```
```sql ```sql
SHOW TABLES FROM mysql_db SHOW TABLES FROM mysql_db
``` ```
```text ```text
┌─name─────────┐ ┌─name─────────┐
│ mysql_table │ │ mysql_table │
└──────────────┘ └──────────────┘
``` ```
```sql ```sql
SELECT * FROM mysql_db.mysql_table SELECT * FROM mysql_db.mysql_table
``` ```
```text ```text
┌─int_id─┬─value─┐ ┌─int_id─┬─value─┐
│ 1 │ 2 │ │ 1 │ 2 │
└────────┴───────┘ └────────┴───────┘
``` ```
```sql ```sql
INSERT INTO mysql_db.mysql_table VALUES (3,4) INSERT INTO mysql_db.mysql_table VALUES (3,4)
``` ```
```sql ```sql
SELECT * FROM mysql_db.mysql_table SELECT * FROM mysql_db.mysql_table
``` ```
```text ```text
┌─int_id─┬─value─┐ ┌─int_id─┬─value─┐
│ 1 │ 2 │ │ 1 │ 2 │
│ 3 │ 4 │ │ 3 │ 4 │
└────────┴───────┘ └────────┴───────┘
``` ```

View File

@ -40,14 +40,15 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
[ORDER BY expr] [ORDER BY expr]
[PRIMARY KEY expr] [PRIMARY KEY expr]
[SAMPLE BY expr] [SAMPLE BY expr]
[TTL expr]
[SETTINGS name=value, ...] [SETTINGS name=value, ...]
``` ```
Описание параметров запроса смотрите в [описании запроса](../../query_language/create.md). Описание параметров запроса смотрите в [описании запроса](../../query_language/create.md).
**Секции запроса** ### Секции запроса
- `ENGINE`Имя и параметры движка. `ENGINE = MergeTree()`. `MergeTree` не имеет параметров. - `ENGINE`имя и параметры движка. `ENGINE = MergeTree()`. `MergeTree` не имеет параметров.
- `PARTITION BY` — [ключ партиционирования](custom_partitioning_key.md). - `PARTITION BY` — [ключ партиционирования](custom_partitioning_key.md).
@ -66,22 +67,22 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
Если используется выражение для сэмплирования, то первичный ключ должен содержать его. Пример: Если используется выражение для сэмплирования, то первичный ключ должен содержать его. Пример:
`SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`.
- `TTL` - выражение для задания времени хранения строк. - `TTL` — выражение, определяющее длительность хранения строк.
Оно должно зависеть от стобца типа `Date` или `DateTime` и в качестве результата вычислять столбец типа `Date` или `DateTime`. Пример: Должно зависеть от столбца `Date` или `DateTime` и возвращать столбец `Date` или `DateTime`. Пример:`TTL date + INTERVAL 1 DAY`
`TTL date + INTERVAL 1 DAY`
Подробнее смотрите в [TTL для стоблцов и таблиц](mergetree.md) Дополнительные сведения смотрите в разделе [TTL для столбцов и таблиц](mergetree.md)
- `SETTINGS` — дополнительные параметры, регулирующие поведение `MergeTree`: - `SETTINGS` — дополнительные параметры, регулирующие поведение `MergeTree`:
- `index_granularity` — гранулярность индекса. Число строк данных между «засечками» индекса. По умолчанию — 8192. Список всех доступных параметров можно посмотреть в [MergeTreeSettings.h](https://github.com/yandex/ClickHouse/blob/master/dbms/src/Storages/MergeTree/MergeTreeSettings.h). - `index_granularity` — гранулярность индекса. Число строк данных между «засечками» индекса. По умолчанию — 8192. Список всех доступных параметров можно посмотреть в [MergeTreeSettings.h](https://github.com/yandex/ClickHouse/blob/master/dbms/src/Storages/MergeTree/MergeTreeSettings.h).
- `min_merge_bytes_to_use_direct_io` — минимальный объем данных, необходимый для прямого (небуферизованного) чтения/записи (direct I/O) на диск. При слиянии частей данных ClickHouse вычисляет общий объем хранения всех данных, подлежащих слиянию. Если общий объем хранения всех данных для чтения превышает `min_bytes_to_use_direct_io` байт, тогда ClickHouse использует флаг `O_DIRECT` при чтении данных с диска. Если `min_merge_bytes_to_use_direct_io = 0`, тогда прямой ввод-вывод отключен. Значение по умолчанию: `10 * 1024 * 1024 * 1024` байт. - `min_merge_bytes_to_use_direct_io` — минимальный объем данных, необходимый для прямого (небуферизованного) чтения/записи (direct I/O) на диск. При слиянии частей данных ClickHouse вычисляет общий объем хранения всех данных, подлежащих слиянию. Если общий объем хранения всех данных для чтения превышает `min_bytes_to_use_direct_io` байт, тогда ClickHouse использует флаг `O_DIRECT` при чтении данных с диска. Если `min_merge_bytes_to_use_direct_io = 0`, тогда прямой ввод-вывод отключен. Значение по умолчанию: `10 * 1024 * 1024 * 1024` байт.
<a name="mergetree_setting-merge_with_ttl_timeout"></a>
- `merge_with_ttl_timeout` - Минимальное время в секундах для повторного выполнения слияний с TTL. По умолчанию - 86400 (1 день). - `merge_with_ttl_timeout` - Минимальное время в секундах для повторного выполнения слияний с TTL. По умолчанию - 86400 (1 день).
**Пример задания секций** **Пример задания секций**
``` ```sql
ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192 ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192
``` ```
@ -109,7 +110,8 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
- `date-column` — имя столбца с типом [Date](../../data_types/date.md). На основе этого столбца ClickHouse автоматически создаёт партиции по месяцам. Имена партиций имеют формат `"YYYYMM"`. - `date-column` — имя столбца с типом [Date](../../data_types/date.md). На основе этого столбца ClickHouse автоматически создаёт партиции по месяцам. Имена партиций имеют формат `"YYYYMM"`.
- `sampling_expression` — выражение для сэмплирования. - `sampling_expression` — выражение для сэмплирования.
- `(primary, key)` — первичный ключ. Тип — [Tuple()](../../data_types/tuple.md- `index_granularity` — гранулярность индекса. Число строк данных между «засечками» индекса. Для большинства задач подходит значение 8192. - `(primary, key)` — первичный ключ. Тип — [Tuple()](../../data_types/tuple.md)
- `index_granularity` — гранулярность индекса. Число строк данных между «засечками» индекса. Для большинства задач подходит значение 8192.
**Пример** **Пример**
@ -118,6 +120,7 @@ MergeTree(EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID)
``` ```
Движок `MergeTree` сконфигурирован таким же образом, как и в примере выше для основного способа конфигурирования движка. Движок `MergeTree` сконфигурирован таким же образом, как и в примере выше для основного способа конфигурирования движка.
</details> </details>
## Хранение данных ## Хранение данных
@ -298,13 +301,42 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT
Чтения из таблицы автоматически распараллеливаются. Чтения из таблицы автоматически распараллеливаются.
## TTL для столбцов и таблиц {#table_engine-mergetree-ttl}
## TTL для столбцов и таблиц Определяет время жизни значений.
Данные с истекшим TTL удаляются во время слияний. Секция `TTL` может быть установлена как для всей таблицы, так и для каждого отдельного столбца. Если установлены оба `TTL`, то ClickHouse использует тот, что истекает раньше.
Если TTL указан для столбца, то когда он истекает, значение заменяется на значение по умолчанию. Если все значения столбца обнулены в куске, то данные этого столбца удаляются с диска в куске. Если TTL указан для таблицы, то когда он истекает, удаляется строка. Таблица должна иметь столбец типа [Date](../../data_types/date.md) или [DateTime](../../data_types/datetime.md). Для установки времени жизни данных, следует использовать операцию со столбцом с временем, например:
Когда истекает TTL для какого-нибудь значения или строки в куске, назначается внеочередное слияние. Чтобы контролировать частоту слияний с TTL, вы можете задать настройку `merge_with_ttl_timeout`. Если ее значение слишком мало, то будет происходить слишком много внеочередных слияний и мало обычных, вследствие чего может ухудшиться производительность. ```
TTL time_column
TTL time_column + interval
```
Чтобы задать `interval`, используйте операторы [интервала времени](../../query_language/operators.md#operators-datetime).
```
TTL date_time + INTERVAL 1 MONTH
TTL date_time + INTERVAL 15 HOUR
```
**TTL столбца**
Когда срок действия значений в столбце истечет, ClickHouse заменит их значениями по умолчанию для типа данных столбца. Если срок действия всех значений столбцов в части данных истек, ClickHouse удаляет столбец из куска данных в файловой системе.
Секцию `TTL` нельзя использовать для ключевых столбцов.
**TTL таблицы**
Когда некоторые данные в таблице устаревают, ClickHouse удаляет все соответствующие строки.
**Удаление данных**
Данные с истекшим TTL удаляются, когда ClickHouse мёржит куски данных.
Когда ClickHouse видит, что некоторые данные устарели, он выполняет внеплановые мёржи. Для управление частотой подобных мёржей, можно задать настройку [merge_with_ttl_timeout](#mergetree_setting-merge_with_ttl_timeout). Если её значение слишком низкое, придется выполнять много внеплановых мёржей, которые могут начать потреблять значительную долю ресурсов сервера.
Если вы выполните запрос `SELECT` между слияниями вы можете получить устаревшие данные. Чтобы избежать этого используйте запрос [OPTIMIZE](../../query_language/misc.md#misc_operations-optimize) перед `SELECT`.
[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/mergetree/) <!--hide--> [Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/mergetree/) <!--hide-->

View File

@ -1,13 +1,30 @@
## CREATE DATABASE ## CREATE DATABASE {#query_language-create-database}
Создание базы данных db\_name. Создает базу данных.
```sql ```sql
CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] [ENGINE = engine(...)]
``` ```
`База данных` - это просто директория для таблиц. ### Секции
Если написано `IF NOT EXISTS`, то запрос не будет возвращать ошибку, если база данных уже существует.
- `IF NOT EXISTS`
Если база данных с именем `db_name` уже существует, то ClickHouse не создаёт базу данных и:
- Не генерирует исключение, если секция указана.
- Генерирует исключение, если секция не указана.
- `ON CLUSTER`
ClickHouse создаёт базу данных `db_name` на всех серверах указанного кластера.
- `ENGINE`
- [MySQL](../database_engines/mysql.md)
Позволяет получать данные с удаленного сервера MySQL.
По умолчанию ClickHouse использует собственный [движок баз данных](../database_engines/index.md).
## CREATE TABLE {#create-table-query} ## CREATE TABLE {#create-table-query}
@ -48,7 +65,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name ENGINE = engine AS SELECT ...
Во всех случаях, если указано `IF NOT EXISTS`, то запрос не будет возвращать ошибку, если таблица уже существует. В этом случае, запрос будет ничего не делать. Во всех случаях, если указано `IF NOT EXISTS`, то запрос не будет возвращать ошибку, если таблица уже существует. В этом случае, запрос будет ничего не делать.
После секции `ENGINE` в запросе могут использоваться и другие секции в зависимости от движка. Подробную документацию по созданию таблиц смотрите в описаниях [движков](../operations/table_engines/index.md#table_engines). После секции `ENGINE` в запросе могут использоваться и другие секции в зависимости от движка. Подробную документацию по созданию таблиц смотрите в описаниях [движков таблиц](../operations/table_engines/index.md#table_engines).
### Значения по умолчанию {#create-default-values} ### Значения по умолчанию {#create-default-values}
@ -88,11 +105,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name ENGINE = engine AS SELECT ...
### Выражение для TTL ### Выражение для TTL
Может быть указано только для таблиц семейства MergeTree. Выражение для указания времени хранения значений. Оно должно зависеть от стобца типа `Date` или `DateTime` и в качестве результата вычислять столбец типа `Date` или `DateTime`. Пример: Определяет время хранения значений. Может быть указано только для таблиц семейства MergeTree. Подробнее смотрите в [TTL для столбцов и таблиц](../operations/table_engines/mergetree.md#table_engine-mergetree-ttl).
`TTL date + INTERVAL 1 DAY`
Нельзя указывать TTL для ключевых столбцов. Подробнее смотрите в [TTL для стоблцов и таблиц](../operations/table_engines/mergetree.md)
## Форматы сжатия для колонок ## Форматы сжатия для колонок

View File

@ -97,6 +97,9 @@ SELECT visibleWidth(NULL)
Возвращает имя текущей базы данных. Возвращает имя текущей базы данных.
Эта функция может использоваться в параметрах движка таблицы в запросе CREATE TABLE там, где нужно указать базу данных. Эта функция может использоваться в параметрах движка таблицы в запросе CREATE TABLE там, где нужно указать базу данных.
## currentUser()
Возвращает логин пользователя, от имени которого исполняется запрос. В случае распределённого запроса, возвращается имя пользователя - инициатора запроса.
## isFinite(x) ## isFinite(x)
Принимает Float32 или Float64 и возвращает UInt8, равный 1, если аргумент не бесконечный и не NaN, иначе 0. Принимает Float32 или Float64 и возвращает UInt8, равный 1, если аргумент не бесконечный и не NaN, иначе 0.

View File

@ -65,13 +65,13 @@
`a GLOBAL NOT IN ...` - функция `globalNotIn(a, b)` `a GLOBAL NOT IN ...` - функция `globalNotIn(a, b)`
## Оператор для работы с датами и временем ## Оператор для работы с датами и временем {#operators-datetime}
``` sql ``` sql
EXTRACT(part FROM date); EXTRACT(part FROM date);
``` ```
Позволяет извлечь отдельные части из переданной даты. Например, можно получить месяц из даты, или минуты из времени. Позволяет извлечь отдельные части из переданной даты. Например, можно получить месяц из даты, или минуты из времени.
В параметре `part` указывается, какой фрагмент даты нужно получить. Доступные значения: В параметре `part` указывается, какой фрагмент даты нужно получить. Доступные значения:
@ -99,8 +99,8 @@ SELECT EXTRACT(YEAR FROM toDate('2017-06-15'));
``` sql ``` sql
CREATE TABLE test.Orders CREATE TABLE test.Orders
( (
OrderId UInt64, OrderId UInt64,
OrderName String, OrderName String,
OrderDate DateTime OrderDate DateTime
) )
ENGINE = Log; ENGINE = Log;
@ -110,11 +110,11 @@ ENGINE = Log;
INSERT INTO test.Orders VALUES (1, 'Jarlsberg Cheese', toDateTime('2008-10-11 13:23:44')); INSERT INTO test.Orders VALUES (1, 'Jarlsberg Cheese', toDateTime('2008-10-11 13:23:44'));
``` ```
``` sql ``` sql
SELECT SELECT
toYear(OrderDate) AS OrderYear, toYear(OrderDate) AS OrderYear,
toMonth(OrderDate) AS OrderMonth, toMonth(OrderDate) AS OrderMonth,
toDayOfMonth(OrderDate) AS OrderDay, toDayOfMonth(OrderDate) AS OrderDay,
toHour(OrderDate) AS OrderHour, toHour(OrderDate) AS OrderHour,
toMinute(OrderDate) AS OrderMinute, toMinute(OrderDate) AS OrderMinute,
toSecond(OrderDate) AS OrderSecond toSecond(OrderDate) AS OrderSecond
FROM test.Orders; FROM test.Orders;