mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
dbms: development [#CONV-2944].
This commit is contained in:
parent
57854230c1
commit
9a14f5538a
@ -119,6 +119,7 @@ namespace ErrorCodes
|
|||||||
CANNOT_WAIT_FOR_SIGNAL,
|
CANNOT_WAIT_FOR_SIGNAL,
|
||||||
THERE_IS_NO_SESSION,
|
THERE_IS_NO_SESSION,
|
||||||
CANNOT_CLOCK_GETTIME,
|
CANNOT_CLOCK_GETTIME,
|
||||||
|
UNKNOWN_SETTING,
|
||||||
|
|
||||||
POCO_EXCEPTION = 1000,
|
POCO_EXCEPTION = 1000,
|
||||||
STD_EXCEPTION,
|
STD_EXCEPTION,
|
||||||
|
@ -93,6 +93,9 @@ public:
|
|||||||
|
|
||||||
Settings getSettings() const;
|
Settings getSettings() const;
|
||||||
void setSettings(const Settings & settings_);
|
void setSettings(const Settings & settings_);
|
||||||
|
|
||||||
|
/// Установить настройку по имени.
|
||||||
|
void setSetting(const String & name, const Field & value);
|
||||||
|
|
||||||
const Functions & getFunctions() const { return shared->functions; }
|
const Functions & getFunctions() const { return shared->functions; }
|
||||||
const AggregateFunctionFactory & getAggregateFunctionsFactory() const { return shared->aggregate_function_factory; }
|
const AggregateFunctionFactory & getAggregateFunctionsFactory() const { return shared->aggregate_function_factory; }
|
||||||
|
35
dbms/include/DB/Interpreters/InterpreterSetQuery.h
Normal file
35
dbms/include/DB/Interpreters/InterpreterSetQuery.h
Normal file
@ -0,0 +1,35 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <DB/Parsers/ASTSetQuery.h>
|
||||||
|
#include <DB/Interpreters/Context.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
|
||||||
|
/** Установить один или несколько параметров, для сессии или глобально.
|
||||||
|
*/
|
||||||
|
class InterpreterSetQuery
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
InterpreterSetQuery(ASTPtr query_ptr_, Context & context_)
|
||||||
|
: query_ptr(query_ptr_), context(context_) {}
|
||||||
|
|
||||||
|
void execute()
|
||||||
|
{
|
||||||
|
ASTSetQuery & ast = dynamic_cast<ASTSetQuery &>(*query_ptr);
|
||||||
|
|
||||||
|
Context & target = ast.global ? context.getGlobalContext() : context.getSessionContext();
|
||||||
|
|
||||||
|
for (ASTSetQuery::Changes::const_iterator it = ast.changes.begin(); it != ast.changes.end(); ++it)
|
||||||
|
target.setSetting(it->name, it->value);
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
ASTPtr query_ptr;
|
||||||
|
Context & context;
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
}
|
@ -2,6 +2,7 @@
|
|||||||
|
|
||||||
#include <Poco/Timespan.h>
|
#include <Poco/Timespan.h>
|
||||||
#include <DB/Core/Defines.h>
|
#include <DB/Core/Defines.h>
|
||||||
|
#include <DB/Core/Field.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -31,6 +32,21 @@ struct Settings
|
|||||||
send_timeout(DBMS_DEFAULT_SEND_TIMEOUT_SEC, 0)
|
send_timeout(DBMS_DEFAULT_SEND_TIMEOUT_SEC, 0)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Установить настройку по имени.
|
||||||
|
void set(const String & name, const Field & value)
|
||||||
|
{
|
||||||
|
if (name == "max_block_size") max_block_size = boost::get<UInt64>(value);
|
||||||
|
else if (name == "max_threads") max_threads = boost::get<UInt64>(value);
|
||||||
|
else if (name == "max_query_size") max_query_size = boost::get<UInt64>(value);
|
||||||
|
else if (name == "asynchronous") asynchronous = boost::get<UInt64>(value);
|
||||||
|
else if (name == "interactive_delay") interactive_delay = boost::get<UInt64>(value);
|
||||||
|
else if (name == "connect_timeout") connect_timeout = Poco::Timespan(boost::get<UInt64>(value), 0);
|
||||||
|
else if (name == "receive_timeout") receive_timeout = Poco::Timespan(boost::get<UInt64>(value), 0);
|
||||||
|
else if (name == "send_timeout") send_timeout = Poco::Timespan(boost::get<UInt64>(value), 0);
|
||||||
|
else
|
||||||
|
throw Exception("Unknown setting " + name, ErrorCodes::UNKNOWN_SETTING);
|
||||||
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
@ -10,6 +10,7 @@
|
|||||||
#include <DB/Parsers/ASTRenameQuery.h>
|
#include <DB/Parsers/ASTRenameQuery.h>
|
||||||
#include <DB/Parsers/ASTShowTablesQuery.h>
|
#include <DB/Parsers/ASTShowTablesQuery.h>
|
||||||
#include <DB/Parsers/ASTUseQuery.h>
|
#include <DB/Parsers/ASTUseQuery.h>
|
||||||
|
#include <DB/Parsers/ASTSetQuery.h>
|
||||||
#include <DB/Parsers/ASTOptimizeQuery.h>
|
#include <DB/Parsers/ASTOptimizeQuery.h>
|
||||||
#include <DB/Parsers/ASTExpressionList.h>
|
#include <DB/Parsers/ASTExpressionList.h>
|
||||||
#include <DB/Parsers/ASTFunction.h>
|
#include <DB/Parsers/ASTFunction.h>
|
||||||
@ -35,6 +36,7 @@ void formatAST(const ASTInsertQuery & ast, std::ostream & s, size_t indent = 0,
|
|||||||
void formatAST(const ASTRenameQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false);
|
void formatAST(const ASTRenameQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false);
|
||||||
void formatAST(const ASTShowTablesQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false);
|
void formatAST(const ASTShowTablesQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false);
|
||||||
void formatAST(const ASTUseQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false);
|
void formatAST(const ASTUseQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false);
|
||||||
|
void formatAST(const ASTSetQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false);
|
||||||
void formatAST(const ASTOptimizeQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false);
|
void formatAST(const ASTOptimizeQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false);
|
||||||
void formatAST(const ASTExpressionList & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false);
|
void formatAST(const ASTExpressionList & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false);
|
||||||
void formatAST(const ASTFunction & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false);
|
void formatAST(const ASTFunction & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false);
|
||||||
|
@ -165,6 +165,13 @@ void Context::setSettings(const Settings & settings_)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void Context::setSetting(const String & name, const Field & value)
|
||||||
|
{
|
||||||
|
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
|
||||||
|
settings.set(name, value);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
String Context::getCurrentDatabase() const
|
String Context::getCurrentDatabase() const
|
||||||
{
|
{
|
||||||
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
|
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
|
||||||
|
@ -5,6 +5,7 @@
|
|||||||
#include <DB/Parsers/ASTRenameQuery.h>
|
#include <DB/Parsers/ASTRenameQuery.h>
|
||||||
#include <DB/Parsers/ASTShowTablesQuery.h>
|
#include <DB/Parsers/ASTShowTablesQuery.h>
|
||||||
#include <DB/Parsers/ASTUseQuery.h>
|
#include <DB/Parsers/ASTUseQuery.h>
|
||||||
|
#include <DB/Parsers/ASTSetQuery.h>
|
||||||
#include <DB/Parsers/ASTOptimizeQuery.h>
|
#include <DB/Parsers/ASTOptimizeQuery.h>
|
||||||
|
|
||||||
#include <DB/Interpreters/InterpreterSelectQuery.h>
|
#include <DB/Interpreters/InterpreterSelectQuery.h>
|
||||||
@ -14,6 +15,7 @@
|
|||||||
#include <DB/Interpreters/InterpreterRenameQuery.h>
|
#include <DB/Interpreters/InterpreterRenameQuery.h>
|
||||||
#include <DB/Interpreters/InterpreterShowTablesQuery.h>
|
#include <DB/Interpreters/InterpreterShowTablesQuery.h>
|
||||||
#include <DB/Interpreters/InterpreterUseQuery.h>
|
#include <DB/Interpreters/InterpreterUseQuery.h>
|
||||||
|
#include <DB/Interpreters/InterpreterSetQuery.h>
|
||||||
#include <DB/Interpreters/InterpreterOptimizeQuery.h>
|
#include <DB/Interpreters/InterpreterOptimizeQuery.h>
|
||||||
#include <DB/Interpreters/InterpreterQuery.h>
|
#include <DB/Interpreters/InterpreterQuery.h>
|
||||||
|
|
||||||
@ -65,6 +67,11 @@ void InterpreterQuery::execute(WriteBuffer & ostr, ReadBuffer * remaining_data_i
|
|||||||
InterpreterUseQuery interpreter(query_ptr, context);
|
InterpreterUseQuery interpreter(query_ptr, context);
|
||||||
interpreter.execute();
|
interpreter.execute();
|
||||||
}
|
}
|
||||||
|
else if (dynamic_cast<ASTSetQuery *>(&*query_ptr))
|
||||||
|
{
|
||||||
|
InterpreterSetQuery interpreter(query_ptr, context);
|
||||||
|
interpreter.execute();
|
||||||
|
}
|
||||||
else if (dynamic_cast<ASTOptimizeQuery *>(&*query_ptr))
|
else if (dynamic_cast<ASTOptimizeQuery *>(&*query_ptr))
|
||||||
{
|
{
|
||||||
InterpreterOptimizeQuery interpreter(query_ptr, context);
|
InterpreterOptimizeQuery interpreter(query_ptr, context);
|
||||||
@ -116,6 +123,11 @@ BlockIO InterpreterQuery::execute()
|
|||||||
InterpreterUseQuery interpreter(query_ptr, context);
|
InterpreterUseQuery interpreter(query_ptr, context);
|
||||||
interpreter.execute();
|
interpreter.execute();
|
||||||
}
|
}
|
||||||
|
else if (dynamic_cast<ASTSetQuery *>(&*query_ptr))
|
||||||
|
{
|
||||||
|
InterpreterSetQuery interpreter(query_ptr, context);
|
||||||
|
interpreter.execute();
|
||||||
|
}
|
||||||
else if (dynamic_cast<ASTOptimizeQuery *>(&*query_ptr))
|
else if (dynamic_cast<ASTOptimizeQuery *>(&*query_ptr))
|
||||||
{
|
{
|
||||||
InterpreterOptimizeQuery interpreter(query_ptr, context);
|
InterpreterOptimizeQuery interpreter(query_ptr, context);
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
#include <DB/Parsers/ParserShowTablesQuery.h>
|
#include <DB/Parsers/ParserShowTablesQuery.h>
|
||||||
#include <DB/Parsers/ParserOptimizeQuery.h>
|
#include <DB/Parsers/ParserOptimizeQuery.h>
|
||||||
#include <DB/Parsers/ParserUseQuery.h>
|
#include <DB/Parsers/ParserUseQuery.h>
|
||||||
|
#include <DB/Parsers/ParserSetQuery.h>
|
||||||
#include <DB/Parsers/ParserQuery.h>
|
#include <DB/Parsers/ParserQuery.h>
|
||||||
|
|
||||||
|
|
||||||
@ -22,8 +23,9 @@ bool ParserQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected
|
|||||||
ParserRenameQuery rename_p;
|
ParserRenameQuery rename_p;
|
||||||
ParserDropQuery drop_p;
|
ParserDropQuery drop_p;
|
||||||
ParserUseQuery use_p;
|
ParserUseQuery use_p;
|
||||||
|
ParserSetQuery set_p;
|
||||||
ParserOptimizeQuery optimize_p;
|
ParserOptimizeQuery optimize_p;
|
||||||
|
|
||||||
bool res = show_tables_p.parse(pos, end, node, expected)
|
bool res = show_tables_p.parse(pos, end, node, expected)
|
||||||
|| select_p.parse(pos, end, node, expected)
|
|| select_p.parse(pos, end, node, expected)
|
||||||
|| insert_p.parse(pos, end, node, expected)
|
|| insert_p.parse(pos, end, node, expected)
|
||||||
@ -31,10 +33,11 @@ bool ParserQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected
|
|||||||
|| rename_p.parse(pos, end, node, expected)
|
|| rename_p.parse(pos, end, node, expected)
|
||||||
|| drop_p.parse(pos, end, node, expected)
|
|| drop_p.parse(pos, end, node, expected)
|
||||||
|| use_p.parse(pos, end, node, expected)
|
|| use_p.parse(pos, end, node, expected)
|
||||||
|
|| set_p.parse(pos, end, node, expected)
|
||||||
|| optimize_p.parse(pos, end, node, expected);
|
|| optimize_p.parse(pos, end, node, expected);
|
||||||
|
|
||||||
if (!res)
|
if (!res)
|
||||||
expected = "One of: SHOW TABLES, SHOW DATABASES, SELECT, INSERT, CREATE, ATTACH, RENAME, DROP, DETACH, USE, OPTIMIZE";
|
expected = "One of: SHOW TABLES, SHOW DATABASES, SELECT, INSERT, CREATE, ATTACH, RENAME, DROP, DETACH, USE, SET, OPTIMIZE";
|
||||||
|
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
93
dbms/src/Parsers/ParserSetQuery.cpp
Normal file
93
dbms/src/Parsers/ParserSetQuery.cpp
Normal file
@ -0,0 +1,93 @@
|
|||||||
|
#include <DB/Parsers/ASTIdentifier.h>
|
||||||
|
#include <DB/Parsers/ASTLiteral.h>
|
||||||
|
#include <DB/Parsers/ASTSetQuery.h>
|
||||||
|
|
||||||
|
#include <DB/Parsers/CommonParsers.h>
|
||||||
|
#include <DB/Parsers/ParserSetQuery.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
|
||||||
|
/// Парсит name = value.
|
||||||
|
static bool parseNameValuePair(ASTSetQuery::Change & change, IParser::Pos & pos, IParser::Pos end, String & expected)
|
||||||
|
{
|
||||||
|
ParserIdentifier name_p;
|
||||||
|
ParserLiteral value_p;
|
||||||
|
ParserWhiteSpaceOrComments ws;
|
||||||
|
ParserString s_eq("=");
|
||||||
|
|
||||||
|
ASTPtr name;
|
||||||
|
ASTPtr value;
|
||||||
|
|
||||||
|
ws.ignore(pos, end);
|
||||||
|
|
||||||
|
if (!name_p.parse(pos, end, name, expected))
|
||||||
|
return false;
|
||||||
|
|
||||||
|
ws.ignore(pos, end);
|
||||||
|
|
||||||
|
if (!s_eq.ignore(pos, end, expected))
|
||||||
|
return false;
|
||||||
|
|
||||||
|
ws.ignore(pos, end);
|
||||||
|
|
||||||
|
if (!value_p.parse(pos, end, value, expected))
|
||||||
|
return false;
|
||||||
|
|
||||||
|
ws.ignore(pos, end);
|
||||||
|
|
||||||
|
change.name = dynamic_cast<const ASTIdentifier &>(*name).name;
|
||||||
|
change.value = dynamic_cast<const ASTLiteral &>(*value).value;
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
bool ParserSetQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected)
|
||||||
|
{
|
||||||
|
Pos begin = pos;
|
||||||
|
|
||||||
|
ParserWhiteSpaceOrComments ws;
|
||||||
|
ParserString s_set("SET", true, true);
|
||||||
|
ParserString s_global("GLOBAL", true, true);
|
||||||
|
ParserString s_comma(",");
|
||||||
|
|
||||||
|
ws.ignore(pos, end);
|
||||||
|
|
||||||
|
if (!s_set.ignore(pos, end, expected))
|
||||||
|
return false;
|
||||||
|
|
||||||
|
ws.ignore(pos, end);
|
||||||
|
|
||||||
|
bool global = s_global.ignore(pos, end, expected);
|
||||||
|
|
||||||
|
ASTSetQuery::Changes changes;
|
||||||
|
|
||||||
|
while (true)
|
||||||
|
{
|
||||||
|
ws.ignore(pos, end);
|
||||||
|
|
||||||
|
if (!changes.empty() && !s_comma.ignore(pos, end))
|
||||||
|
break;
|
||||||
|
|
||||||
|
ws.ignore(pos, end);
|
||||||
|
|
||||||
|
changes.push_back(ASTSetQuery::Change());
|
||||||
|
|
||||||
|
if (!parseNameValuePair(changes.back(), pos, end, expected))
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
ASTSetQuery * query = new ASTSetQuery(StringRange(begin, pos));
|
||||||
|
node = query;
|
||||||
|
|
||||||
|
query->changes = changes;
|
||||||
|
query->global = global;
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
}
|
@ -31,114 +31,36 @@ static const char * hilite_none = "\033[0m";
|
|||||||
|
|
||||||
void formatAST(const IAST & ast, std::ostream & s, size_t indent, bool hilite, bool one_line)
|
void formatAST(const IAST & ast, std::ostream & s, size_t indent, bool hilite, bool one_line)
|
||||||
{
|
{
|
||||||
const ASTSelectQuery * select = dynamic_cast<const ASTSelectQuery *>(&ast);
|
|
||||||
if (select)
|
|
||||||
{
|
|
||||||
formatAST(*select, s, indent, hilite, one_line);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
const ASTInsertQuery * insert = dynamic_cast<const ASTInsertQuery *>(&ast);
|
#define DISPATCH(NAME) \
|
||||||
if (insert)
|
else if (const AST ## NAME * concrete = dynamic_cast<const AST ## NAME *>(&ast)) \
|
||||||
{
|
formatAST(*concrete, s, indent, hilite, one_line);
|
||||||
formatAST(*insert, s, indent, hilite, one_line);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
const ASTCreateQuery * create = dynamic_cast<const ASTCreateQuery *>(&ast);
|
if (false) {}
|
||||||
if (create)
|
DISPATCH(SelectQuery)
|
||||||
{
|
DISPATCH(InsertQuery)
|
||||||
formatAST(*create, s, indent, hilite, one_line);
|
DISPATCH(CreateQuery)
|
||||||
return;
|
DISPATCH(DropQuery)
|
||||||
}
|
DISPATCH(RenameQuery)
|
||||||
|
DISPATCH(ShowTablesQuery)
|
||||||
const ASTDropQuery * drop = dynamic_cast<const ASTDropQuery *>(&ast);
|
DISPATCH(UseQuery)
|
||||||
if (drop)
|
DISPATCH(SetQuery)
|
||||||
{
|
DISPATCH(OptimizeQuery)
|
||||||
formatAST(*drop, s, indent, hilite, one_line);
|
DISPATCH(ExpressionList)
|
||||||
return;
|
DISPATCH(Function)
|
||||||
}
|
DISPATCH(Identifier)
|
||||||
|
DISPATCH(Literal)
|
||||||
const ASTRenameQuery * rename = dynamic_cast<const ASTRenameQuery *>(&ast);
|
DISPATCH(NameTypePair)
|
||||||
if (rename)
|
DISPATCH(Asterisk)
|
||||||
{
|
DISPATCH(OrderByElement)
|
||||||
formatAST(*rename, s, indent, hilite, one_line);
|
else
|
||||||
return;
|
throw DB::Exception("Unknown element in AST: " + std::string(ast.range.first, ast.range.second - ast.range.first),
|
||||||
}
|
ErrorCodes::UNKNOWN_ELEMENT_IN_AST);
|
||||||
|
|
||||||
const ASTShowTablesQuery * show_tables = dynamic_cast<const ASTShowTablesQuery *>(&ast);
|
|
||||||
if (show_tables)
|
|
||||||
{
|
|
||||||
formatAST(*show_tables, s, indent, hilite, one_line);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
const ASTUseQuery * use = dynamic_cast<const ASTUseQuery *>(&ast);
|
|
||||||
if (use)
|
|
||||||
{
|
|
||||||
formatAST(*use, s, indent, hilite, one_line);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
const ASTOptimizeQuery * optimize = dynamic_cast<const ASTOptimizeQuery *>(&ast);
|
|
||||||
if (optimize)
|
|
||||||
{
|
|
||||||
formatAST(*optimize, s, indent, hilite, one_line);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
const ASTExpressionList * exp_list = dynamic_cast<const ASTExpressionList *>(&ast);
|
#undef DISPATCH
|
||||||
if (exp_list)
|
|
||||||
{
|
|
||||||
formatAST(*exp_list, s, indent, hilite, one_line);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
const ASTFunction * func = dynamic_cast<const ASTFunction *>(&ast);
|
|
||||||
if (func)
|
|
||||||
{
|
|
||||||
formatAST(*func, s, indent, hilite, one_line);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
const ASTIdentifier * id = dynamic_cast<const ASTIdentifier *>(&ast);
|
|
||||||
if (id)
|
|
||||||
{
|
|
||||||
formatAST(*id, s, indent, hilite, one_line);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
const ASTLiteral * lit = dynamic_cast<const ASTLiteral *>(&ast);
|
|
||||||
if (lit)
|
|
||||||
{
|
|
||||||
formatAST(*lit, s, indent, hilite, one_line);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
const ASTNameTypePair * ntp = dynamic_cast<const ASTNameTypePair *>(&ast);
|
|
||||||
if (ntp)
|
|
||||||
{
|
|
||||||
formatAST(*ntp, s, indent, hilite, one_line);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
const ASTAsterisk * asterisk = dynamic_cast<const ASTAsterisk *>(&ast);
|
|
||||||
if (asterisk)
|
|
||||||
{
|
|
||||||
formatAST(*asterisk, s, indent, hilite, one_line);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
const ASTOrderByElement * order_by_elem = dynamic_cast<const ASTOrderByElement *>(&ast);
|
|
||||||
if (order_by_elem)
|
|
||||||
{
|
|
||||||
formatAST(*order_by_elem, s, indent, hilite, one_line);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
throw DB::Exception("Unknown element in AST: " + std::string(ast.range.first, ast.range.second - ast.range.first), ErrorCodes::UNKNOWN_ELEMENT_IN_AST);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void formatAST(const ASTSelectQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line)
|
void formatAST(const ASTSelectQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line)
|
||||||
{
|
{
|
||||||
std::string indent_str = one_line ? "" : std::string(4 * indent, ' ');
|
std::string indent_str = one_line ? "" : std::string(4 * indent, ' ');
|
||||||
@ -293,7 +215,20 @@ void formatAST(const ASTRenameQuery & ast, std::ostream & s, size_t indent, bo
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void formatAST(const ASTShowTablesQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line)
|
void formatAST(const ASTSetQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line)
|
||||||
|
{
|
||||||
|
s << (hilite ? hilite_keyword : "") << "SET " << (ast.global ? "GLOBAL " : "") << (hilite ? hilite_none : "");
|
||||||
|
|
||||||
|
for (ASTSetQuery::Changes::const_iterator it = ast.changes.begin(); it != ast.changes.end(); ++it)
|
||||||
|
{
|
||||||
|
if (it != ast.changes.begin())
|
||||||
|
s << ", ";
|
||||||
|
|
||||||
|
s << it->name << " = " << boost::apply_visitor(FieldVisitorToString(), it->value);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void formatAST(const ASTShowTablesQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line)
|
||||||
{
|
{
|
||||||
if (ast.databases)
|
if (ast.databases)
|
||||||
{
|
{
|
||||||
|
Loading…
Reference in New Issue
Block a user