提交 74dfc96c 编写于 作者: A Alexey Milovidov

dbms: development [#CONV-2944].

上级 48178920
......@@ -293,7 +293,7 @@ struct MatchImpl
size_t size = offsets.size();
for (size_t i = 0; i < size; ++i)
res[i] = revert ^ regexp.match(reinterpret_cast<const char *>(&data[i != 0 ? offsets[i - 1] : 0]), i != 0 ? offsets[i] - offsets[i - 1] : offsets[0]);
res[i] = revert ^ regexp.match(reinterpret_cast<const char *>(&data[i != 0 ? offsets[i - 1] : 0]), (i != 0 ? offsets[i] - offsets[i - 1] : offsets[0]) - 1);
}
}
......
......@@ -72,7 +72,7 @@ struct Context
/// Проверка существования таблицы. database может быть пустой - в этом случае используется текущая БД.
bool exists(const String & database_name, const String & table_name)
bool isTableExist(const String & database_name, const String & table_name)
{
Poco::ScopedLock<Poco::Mutex> lock(*mutex);
......@@ -83,7 +83,7 @@ struct Context
}
void assertExists(const String & database_name, const String & table_name)
void assertTableExists(const String & database_name, const String & table_name)
{
Poco::ScopedLock<Poco::Mutex> lock(*mutex);
......@@ -97,7 +97,7 @@ struct Context
}
void assertDoesntExist(const String & database_name, const String & table_name)
void assertTableDoesntExist(const String & database_name, const String & table_name)
{
Poco::ScopedLock<Poco::Mutex> lock(*mutex);
......@@ -107,6 +107,17 @@ struct Context
&& (*databases)[db].end() != (*databases)[db].find(table_name))
throw Exception("Table " + db + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
}
void assertDatabaseExists(const String & database_name)
{
Poco::ScopedLock<Poco::Mutex> lock(*mutex);
String db = database_name.empty() ? current_database : database_name;
if (databases->end() == databases->find(db))
throw Exception("Database " + db + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE);
}
};
......
#pragma once
#include <DB/DataStreams/BlockIO.h>
#include <DB/Interpreters/Context.h>
namespace DB
{
/** Вывести список имён таблиц/баз данных по некоторым условиям.
* Интерпретирует запрос путём замены его на запрос SELECT из таблицы system.tables или system.databases.
*/
class InterpreterShowTablesQuery
{
public:
InterpreterShowTablesQuery(ASTPtr query_ptr_, Context & context_);
BlockIO execute();
BlockInputStreamPtr executeAndFormat(WriteBuffer & buf);
private:
ASTPtr query_ptr;
Context context;
String getRewrittenQuery();
};
}
#pragma once
#include <DB/Parsers/IAST.h>
namespace DB
{
/** Запрос SHOW TABLES или SHOW DATABASES
*/
class ASTShowTablesQuery : public IAST
{
public:
bool databases;
String from;
String like;
ASTShowTablesQuery() : databases(false) {}
ASTShowTablesQuery(StringRange range_) : IAST(range_), databases(false) {}
/** Получить текст, который идентифицирует этот элемент. */
String getID() { return "ShowTables"; };
ASTPtr clone() const { return new ASTShowTablesQuery(*this); }
};
}
#pragma once
#include <DB/Parsers/IParserBase.h>
namespace DB
{
/** Запрос типа такого:
* SHOW TABLES [FROM db] [LIKE 'str']
* или
* SHOW DATABASES.
*/
class ParserShowTablesQuery : public IParserBase
{
protected:
String getName() { return "SHOW TABLES|DATABASES query"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected);
};
}
......@@ -8,6 +8,7 @@
#include <DB/Parsers/ASTDropQuery.h>
#include <DB/Parsers/ASTInsertQuery.h>
#include <DB/Parsers/ASTRenameQuery.h>
#include <DB/Parsers/ASTShowTablesQuery.h>
#include <DB/Parsers/ASTExpressionList.h>
#include <DB/Parsers/ASTFunction.h>
#include <DB/Parsers/ASTIdentifier.h>
......@@ -30,6 +31,7 @@ void formatAST(const ASTCreateQuery & ast, std::ostream & s, size_t indent = 0,
void formatAST(const ASTDropQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false);
void formatAST(const ASTInsertQuery & 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 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 ASTIdentifier & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false);
......
#pragma once
#include <Poco/SharedPtr.h>
#include <DB/Storages/IStorage.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/Interpreters/Context.h>
namespace DB
{
using Poco::SharedPtr;
/** Реализует системную таблицу databases, которая позволяет получить информацию о всех БД.
*/
class StorageSystemDatabases : public IStorage
{
public:
StorageSystemDatabases(const std::string & name_, const Context & context_);
std::string getName() const { return "SystemDatabases"; }
std::string getTableName() const { return name; }
const NamesAndTypesList & getColumnsList() const { return columns; }
BlockInputStreams read(
const Names & column_names,
ASTPtr query,
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size = DEFAULT_BLOCK_SIZE,
unsigned threads = 1);
private:
const std::string name;
const Context & context;
NamesAndTypesList columns;
};
}
......@@ -3,12 +3,14 @@
#include <DB/Parsers/ASTCreateQuery.h>
#include <DB/Parsers/ASTDropQuery.h>
#include <DB/Parsers/ASTRenameQuery.h>
#include <DB/Parsers/ASTShowTablesQuery.h>
#include <DB/Interpreters/InterpreterSelectQuery.h>
#include <DB/Interpreters/InterpreterInsertQuery.h>
#include <DB/Interpreters/InterpreterCreateQuery.h>
#include <DB/Interpreters/InterpreterDropQuery.h>
#include <DB/Interpreters/InterpreterRenameQuery.h>
#include <DB/Interpreters/InterpreterShowTablesQuery.h>
#include <DB/Interpreters/InterpreterQuery.h>
......@@ -49,6 +51,11 @@ void InterpreterQuery::execute(WriteBuffer & ostr, ReadBuffer * remaining_data_i
InterpreterRenameQuery interpreter(query_ptr, context);
interpreter.execute();
}
else if (dynamic_cast<ASTShowTablesQuery *>(&*query_ptr))
{
InterpreterShowTablesQuery interpreter(query_ptr, context);
query_plan = interpreter.executeAndFormat(ostr);
}
else
throw Exception("Unknown type of query: " + query_ptr->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY);
}
......@@ -85,6 +92,11 @@ BlockIO InterpreterQuery::execute()
InterpreterRenameQuery interpreter(query_ptr, context);
interpreter.execute();
}
else if (dynamic_cast<ASTShowTablesQuery *>(&*query_ptr))
{
InterpreterShowTablesQuery interpreter(query_ptr, context);
res = interpreter.execute();
}
else
throw Exception("Unknown type of query: " + query_ptr->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY);
......
......@@ -50,8 +50,8 @@ void InterpreterRenameQuery::execute()
String to_table_name_escaped = escapeForFileName(to_table_name);
String to_metadata_path = context.path + "metadata/" + to_database_name_escaped + "/" + (!to_table_name.empty() ? to_table_name_escaped + ".sql" : "");
context.assertExists(from_database_name, from_table_name);
context.assertDoesntExist(to_database_name, to_table_name);
context.assertTableExists(from_database_name, from_table_name);
context.assertTableDoesntExist(to_database_name, to_table_name);
/// Уведомляем таблицу о том, что она переименовается.
(*context.databases)[from_database_name][from_table_name]->rename(context.path + "data/" + to_database_name_escaped + "/", to_table_name);
......
#include <DB/IO/ReadBufferFromString.h>
#include <DB/Parsers/ASTShowTablesQuery.h>
#include <DB/Interpreters/executeQuery.h>
#include <DB/Interpreters/InterpreterShowTablesQuery.h>
#include <mysqlxx/Manip.h>
namespace DB
{
InterpreterShowTablesQuery::InterpreterShowTablesQuery(ASTPtr query_ptr_, Context & context_)
: query_ptr(query_ptr_), context(context_)
{
}
String InterpreterShowTablesQuery::getRewrittenQuery()
{
ASTShowTablesQuery query = dynamic_cast<const ASTShowTablesQuery &>(*query_ptr);
/// SHOW DATABASES
if (query.databases)
return "SELECT name FROM system.databases";
String database = query.from.empty() ? context.current_database : query.from;
context.assertDatabaseExists(database);
std::stringstream rewritten_query;
rewritten_query << "SELECT name FROM system.tables WHERE database = " << mysqlxx::quote << database;
if (!query.like.empty())
rewritten_query << " AND name LIKE " << mysqlxx::quote << query.like;
return rewritten_query.str();
}
BlockIO InterpreterShowTablesQuery::execute()
{
return executeQuery(getRewrittenQuery(), context);
}
BlockInputStreamPtr InterpreterShowTablesQuery::executeAndFormat(WriteBuffer & buf)
{
String query = getRewrittenQuery();
ReadBufferFromString in(query);
BlockInputStreamPtr query_plan;
executeQuery(in, buf, context, query_plan);
return query_plan;
}
}
......@@ -3,6 +3,7 @@
#include <DB/Parsers/ParserInsertQuery.h>
#include <DB/Parsers/ParserDropQuery.h>
#include <DB/Parsers/ParserRenameQuery.h>
#include <DB/Parsers/ParserShowTablesQuery.h>
#include <DB/Parsers/ParserQuery.h>
......@@ -12,13 +13,15 @@ namespace DB
bool ParserQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected)
{
ParserShowTablesQuery show_tables_p;
ParserSelectQuery select_p;
ParserInsertQuery insert_p;
ParserCreateQuery create_p;
ParserRenameQuery rename_p;
ParserDropQuery drop_p;
return select_p.parse(pos, end, node, expected)
return show_tables_p.parse(pos, end, node, expected)
|| select_p.parse(pos, end, node, expected)
|| insert_p.parse(pos, end, node, expected)
|| create_p.parse(pos, end, node, expected)
|| rename_p.parse(pos, end, node, expected)
......
#include <DB/Parsers/ASTLiteral.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/ASTShowTablesQuery.h>
#include <DB/Parsers/CommonParsers.h>
#include <DB/Parsers/ParserShowTablesQuery.h>
#include <DB/Parsers/ExpressionElementParsers.h>
namespace DB
{
bool ParserShowTablesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected)
{
Pos begin = pos;
ParserWhiteSpaceOrComments ws;
ParserString s_show("SHOW", true, true);
ParserString s_tables("TABLES", true, true);
ParserString s_databases("DATABASES", true, true);
ParserString s_from("FROM", true, true);
ParserString s_like("LIKE");
ParserStringLiteral like_p;
ParserIdentifier name_p;
ASTPtr like_ast;
ASTPtr database_ast;
ws.ignore(pos, end);
if (!s_show.ignore(pos, end, expected))
return false;
ws.ignore(pos, end);
if (s_databases.ignore(pos, end))
{
ASTShowTablesQuery * query = new ASTShowTablesQuery(StringRange(begin, pos));
query->databases = true;
node = query;
return true;
}
if (!s_tables.ignore(pos, end, expected))
return false;
ws.ignore(pos, end);
if (s_from.ignore(pos, end, expected))
{
ws.ignore(pos, end);
if (!name_p.parse(pos, end, database_ast, expected))
return false;
}
ws.ignore(pos, end);
if (s_like.ignore(pos, end, expected))
{
ws.ignore(pos, end);
if (!like_p.parse(pos, end, like_ast, expected))
return false;
}
ws.ignore(pos, end);
ASTShowTablesQuery * query = new ASTShowTablesQuery(StringRange(begin, pos));
node = query;
if (database_ast)
query->from = dynamic_cast<ASTIdentifier &>(*database_ast).name;
if (like_ast)
query->like = boost::get<const String &>(dynamic_cast<ASTLiteral &>(*like_ast).value);
return true;
}
}
......@@ -26,6 +26,9 @@ static const char * hilite_alias = "\033[0;32m";
static const char * hilite_none = "\033[0m";
// TODO: Правильно квотировать идентификаторы (в обратных кавычках, если идентификатор необычный).
void formatAST(const IAST & ast, std::ostream & s, size_t indent, bool hilite, bool one_line)
{
const ASTSelectQuery * select = dynamic_cast<const ASTSelectQuery *>(&ast);
......@@ -62,6 +65,13 @@ void formatAST(const IAST & ast, std::ostream & s, size_t indent, bool hilite, b
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 ASTExpressionList * exp_list = dynamic_cast<const ASTExpressionList *>(&ast);
if (exp_list)
......@@ -263,6 +273,25 @@ 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)
{
if (ast.databases)
{
s << (hilite ? hilite_keyword : "") << "SHOW DATABASES" << (hilite ? hilite_none : "");
return;
}
s << (hilite ? hilite_keyword : "") << "SHOW TABLES" << (hilite ? hilite_none : "");
if (!ast.from.empty())
s << (hilite ? hilite_keyword : "") << " FROM " << (hilite ? hilite_none : "")
<< ast.from;
if (!ast.like.empty())
s << (hilite ? hilite_keyword : "") << " LIKE " << (hilite ? hilite_none : "")
<< mysqlxx::quote << ast.like;
}
void formatAST(const ASTInsertQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line)
{
s << (hilite ? hilite_keyword : "") << "INSERT INTO " << (hilite ? hilite_none : "") << (!ast.database.empty() ? ast.database + "." : "") << ast.table;
......@@ -306,7 +335,7 @@ static void writeAlias(const String & name, std::ostream & s, bool hilite, bool
{
s << (hilite ? hilite_keyword : "") << " AS " << (hilite ? hilite_alias : "");
{
WriteBufferFromOStream wb(s);
WriteBufferFromOStream wb(s, 32);
writeProbablyBackQuotedString(name, wb);
}
s << (hilite ? hilite_none : "");
......@@ -331,7 +360,7 @@ void formatAST(const ASTIdentifier & ast, std::ostream & s, size_t indent, bo
{
s << (hilite ? hilite_identifier : "");
{
WriteBufferFromOStream wb(s);
WriteBufferFromOStream wb(s, 32);
writeProbablyBackQuotedString(ast.name, wb);
}
s << (hilite ? hilite_none : "");
......
......@@ -6,6 +6,7 @@
#include <DB/Interpreters/loadMetadata.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/StorageSystemTables.h>
#include <DB/Storages/StorageSystemDatabases.h>
#include <DB/Storages/StorageSystemOne.h>
#include "Server.h"
......@@ -74,9 +75,11 @@ int Server::main(const std::vector<std::string> & args)
loadMetadata(global_context);
/// Создаём системные таблицы.
(*global_context.databases)["system"]["one"] = new StorageSystemOne("one");
(*global_context.databases)["system"]["numbers"] = new StorageSystemNumbers("numbers");
(*global_context.databases)["system"]["tables"] = new StorageSystemTables("tables", global_context);
(*global_context.databases)["system"]["databases"] = new StorageSystemDatabases("databases", global_context);
global_context.current_database = config.getString("default_database", "default");
......
#include <DB/Columns/ColumnString.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Storages/StorageSystemDatabases.h>
namespace DB
{
StorageSystemDatabases::StorageSystemDatabases(const std::string & name_, const Context & context_)
: name(name_), context(context_)
{
columns.push_back(NameAndTypePair("name", new DataTypeString));
}
BlockInputStreams StorageSystemDatabases::read(
const Names & column_names, ASTPtr query, QueryProcessingStage::Enum & processed_stage, size_t max_block_size, unsigned threads)
{
check(column_names);
processed_stage = QueryProcessingStage::FetchColumns;
Block block;
ColumnWithNameAndType col_name;
col_name.name = "name";
col_name.type = new DataTypeString;
col_name.column = new ColumnString;
block.insert(col_name);
Poco::ScopedLock<Poco::Mutex> lock(*context.mutex);
for (Databases::const_iterator it = context.databases->begin(); it != context.databases->end(); ++it)
col_name.column->insert(it->first);
return BlockInputStreams(1, new OneBlockInputStream(block));
}
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册