dbms: development [#CONV-2944].

This commit is contained in:
Alexey Milovidov 2012-08-02 19:03:32 +00:00
parent 57854230c1
commit 9a14f5538a
10 changed files with 213 additions and 106 deletions

View File

@ -119,6 +119,7 @@ namespace ErrorCodes
CANNOT_WAIT_FOR_SIGNAL,
THERE_IS_NO_SESSION,
CANNOT_CLOCK_GETTIME,
UNKNOWN_SETTING,
POCO_EXCEPTION = 1000,
STD_EXCEPTION,

View File

@ -93,6 +93,9 @@ public:
Settings getSettings() const;
void setSettings(const Settings & settings_);
/// Установить настройку по имени.
void setSetting(const String & name, const Field & value);
const Functions & getFunctions() const { return shared->functions; }
const AggregateFunctionFactory & getAggregateFunctionsFactory() const { return shared->aggregate_function_factory; }

View 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;
};
}

View File

@ -2,6 +2,7 @@
#include <Poco/Timespan.h>
#include <DB/Core/Defines.h>
#include <DB/Core/Field.h>
namespace DB
@ -31,6 +32,21 @@ struct Settings
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);
}
};

View File

@ -10,6 +10,7 @@
#include <DB/Parsers/ASTRenameQuery.h>
#include <DB/Parsers/ASTShowTablesQuery.h>
#include <DB/Parsers/ASTUseQuery.h>
#include <DB/Parsers/ASTSetQuery.h>
#include <DB/Parsers/ASTOptimizeQuery.h>
#include <DB/Parsers/ASTExpressionList.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 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 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 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);

View File

@ -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
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);

View File

@ -5,6 +5,7 @@
#include <DB/Parsers/ASTRenameQuery.h>
#include <DB/Parsers/ASTShowTablesQuery.h>
#include <DB/Parsers/ASTUseQuery.h>
#include <DB/Parsers/ASTSetQuery.h>
#include <DB/Parsers/ASTOptimizeQuery.h>
#include <DB/Interpreters/InterpreterSelectQuery.h>
@ -14,6 +15,7 @@
#include <DB/Interpreters/InterpreterRenameQuery.h>
#include <DB/Interpreters/InterpreterShowTablesQuery.h>
#include <DB/Interpreters/InterpreterUseQuery.h>
#include <DB/Interpreters/InterpreterSetQuery.h>
#include <DB/Interpreters/InterpreterOptimizeQuery.h>
#include <DB/Interpreters/InterpreterQuery.h>
@ -65,6 +67,11 @@ void InterpreterQuery::execute(WriteBuffer & ostr, ReadBuffer * remaining_data_i
InterpreterUseQuery interpreter(query_ptr, context);
interpreter.execute();
}
else if (dynamic_cast<ASTSetQuery *>(&*query_ptr))
{
InterpreterSetQuery interpreter(query_ptr, context);
interpreter.execute();
}
else if (dynamic_cast<ASTOptimizeQuery *>(&*query_ptr))
{
InterpreterOptimizeQuery interpreter(query_ptr, context);
@ -116,6 +123,11 @@ BlockIO InterpreterQuery::execute()
InterpreterUseQuery interpreter(query_ptr, context);
interpreter.execute();
}
else if (dynamic_cast<ASTSetQuery *>(&*query_ptr))
{
InterpreterSetQuery interpreter(query_ptr, context);
interpreter.execute();
}
else if (dynamic_cast<ASTOptimizeQuery *>(&*query_ptr))
{
InterpreterOptimizeQuery interpreter(query_ptr, context);

View File

@ -6,6 +6,7 @@
#include <DB/Parsers/ParserShowTablesQuery.h>
#include <DB/Parsers/ParserOptimizeQuery.h>
#include <DB/Parsers/ParserUseQuery.h>
#include <DB/Parsers/ParserSetQuery.h>
#include <DB/Parsers/ParserQuery.h>
@ -22,8 +23,9 @@ bool ParserQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected
ParserRenameQuery rename_p;
ParserDropQuery drop_p;
ParserUseQuery use_p;
ParserSetQuery set_p;
ParserOptimizeQuery optimize_p;
bool res = show_tables_p.parse(pos, end, node, expected)
|| select_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)
|| drop_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);
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;
}

View 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;
}
}

View File

@ -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)
{
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);
if (insert)
{
formatAST(*insert, s, indent, hilite, one_line);
return;
}
#define DISPATCH(NAME) \
else if (const AST ## NAME * concrete = dynamic_cast<const AST ## NAME *>(&ast)) \
formatAST(*concrete, s, indent, hilite, one_line);
const ASTCreateQuery * create = dynamic_cast<const ASTCreateQuery *>(&ast);
if (create)
{
formatAST(*create, s, indent, hilite, one_line);
return;
}
const ASTDropQuery * drop = dynamic_cast<const ASTDropQuery *>(&ast);
if (drop)
{
formatAST(*drop, s, indent, hilite, one_line);
return;
}
const ASTRenameQuery * rename = dynamic_cast<const ASTRenameQuery *>(&ast);
if (rename)
{
formatAST(*rename, s, indent, hilite, one_line);
return;
}
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;
}
if (false) {}
DISPATCH(SelectQuery)
DISPATCH(InsertQuery)
DISPATCH(CreateQuery)
DISPATCH(DropQuery)
DISPATCH(RenameQuery)
DISPATCH(ShowTablesQuery)
DISPATCH(UseQuery)
DISPATCH(SetQuery)
DISPATCH(OptimizeQuery)
DISPATCH(ExpressionList)
DISPATCH(Function)
DISPATCH(Identifier)
DISPATCH(Literal)
DISPATCH(NameTypePair)
DISPATCH(Asterisk)
DISPATCH(OrderByElement)
else
throw DB::Exception("Unknown element in AST: " + std::string(ast.range.first, ast.range.second - ast.range.first),
ErrorCodes::UNKNOWN_ELEMENT_IN_AST);
const ASTExpressionList * exp_list = dynamic_cast<const ASTExpressionList *>(&ast);
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);
#undef DISPATCH
}
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, ' ');
@ -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)
{