dbms: added EXISTS TABLE query [#CONV-2944].

This commit is contained in:
Alexey Milovidov 2012-12-15 03:09:04 +00:00
parent 4a06e09f79
commit 22de09547e
8 changed files with 218 additions and 2 deletions

View File

@ -0,0 +1,81 @@
#pragma once
#include <DB/Storages/IStorage.h>
#include <DB/Parsers/ASTExistsQuery.h>
#include <DB/Interpreters/Context.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/DataStreams/BlockIO.h>
#include <DB/DataStreams/copyData.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
namespace DB
{
/** Проверить, существует ли таблица. Вернуть одну строку с одним столбцом result типа UInt8 со значением 0 или 1.
*/
class InterpreterExistsQuery
{
public:
InterpreterExistsQuery(ASTPtr query_ptr_, Context & context_)
: query_ptr(query_ptr_), context(context_) {}
BlockIO execute()
{
BlockIO res;
res.in = executeImpl();
res.in_sample = getSampleBlock();
return res;
}
BlockInputStreamPtr executeAndFormat(WriteBuffer & buf)
{
Block sample = getSampleBlock();
String format_name = "TabSeparated";
BlockInputStreamPtr in = executeImpl();
BlockOutputStreamPtr out = context.getFormatFactory().getOutput(format_name, buf, sample);
copyData(*in, *out);
return in;
}
private:
ASTPtr query_ptr;
Context context;
Block getSampleBlock()
{
ColumnWithNameAndType col;
col.name = "result";
col.type = new DataTypeUInt8;
Block block;
block.insert(col);
return block;
}
BlockInputStreamPtr executeImpl()
{
const ASTExistsQuery & ast = dynamic_cast<const ASTExistsQuery &>(*query_ptr);
bool res = context.isTableExist(ast.database, ast.table);
ColumnWithNameAndType col;
col.name = "result";
col.type = new DataTypeUInt8;
col.column = new ColumnConstUInt8(1, res);
Block block;
block.insert(col);
return new OneBlockInputStream(block);
}
};
}

View File

@ -0,0 +1,27 @@
#pragma once
#include <DB/Parsers/IAST.h>
namespace DB
{
/** EXISTS запрос
*/
class ASTExistsQuery : public IAST
{
public:
String database;
String table;
ASTExistsQuery() {}
ASTExistsQuery(StringRange range_) : IAST(range_) {}
/** Получить текст, который идентифицирует этот элемент. */
String getID() { return "ExistsQuery_" + database + "_" + table; };
ASTPtr clone() const { return new ASTExistsQuery(*this); }
};
}

View File

@ -0,0 +1,19 @@
#pragma once
#include <DB/Parsers/IParserBase.h>
#include <DB/Parsers/ExpressionElementParsers.h>
namespace DB
{
/** Запрос EXISTS TABLE [db.]name
*/
class ParserExistsQuery : public IParserBase
{
protected:
String getName() { return "EXISTS query"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected);
};
}

View File

@ -12,6 +12,7 @@
#include <DB/Parsers/ASTUseQuery.h>
#include <DB/Parsers/ASTSetQuery.h>
#include <DB/Parsers/ASTOptimizeQuery.h>
#include <DB/Parsers/ASTExistsQuery.h>
#include <DB/Parsers/ASTExpressionList.h>
#include <DB/Parsers/ASTFunction.h>
#include <DB/Parsers/ASTIdentifier.h>
@ -39,6 +40,7 @@ void formatAST(const ASTShowTablesQuery & ast, std::ostream & s, size_t indent =
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 ASTExistsQuery & 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);

View File

@ -7,6 +7,7 @@
#include <DB/Parsers/ASTUseQuery.h>
#include <DB/Parsers/ASTSetQuery.h>
#include <DB/Parsers/ASTOptimizeQuery.h>
#include <DB/Parsers/ASTExistsQuery.h>
#include <DB/Interpreters/InterpreterSelectQuery.h>
#include <DB/Interpreters/InterpreterInsertQuery.h>
@ -17,6 +18,7 @@
#include <DB/Interpreters/InterpreterUseQuery.h>
#include <DB/Interpreters/InterpreterSetQuery.h>
#include <DB/Interpreters/InterpreterOptimizeQuery.h>
#include <DB/Interpreters/InterpreterExistsQuery.h>
#include <DB/Interpreters/InterpreterQuery.h>
@ -77,6 +79,11 @@ void InterpreterQuery::execute(WriteBuffer & ostr, ReadBuffer * remaining_data_i
InterpreterOptimizeQuery interpreter(query_ptr, context);
interpreter.execute();
}
if (dynamic_cast<ASTExistsQuery *>(&*query_ptr))
{
InterpreterExistsQuery interpreter(query_ptr, context);
query_plan = interpreter.executeAndFormat(ostr);
}
else
throw Exception("Unknown type of query: " + query_ptr->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY);
}
@ -133,6 +140,11 @@ BlockIO InterpreterQuery::execute()
InterpreterOptimizeQuery interpreter(query_ptr, context);
interpreter.execute();
}
else if (dynamic_cast<ASTExistsQuery *>(&*query_ptr))
{
InterpreterExistsQuery interpreter(query_ptr, context);
res = interpreter.execute();
}
else
throw Exception("Unknown type of query: " + query_ptr->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY);

View File

@ -0,0 +1,65 @@
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/ASTExistsQuery.h>
#include <DB/Parsers/CommonParsers.h>
#include <DB/Parsers/ParserExistsQuery.h>
namespace DB
{
bool ParserExistsQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected)
{
Pos begin = pos;
ParserWhiteSpaceOrComments ws;
ParserString s_exists("EXISTS", true, true);
ParserString s_table("TABLE", true, true);
ParserString s_dot(".");
ParserIdentifier name_p;
ASTPtr database;
ASTPtr table;
ws.ignore(pos, end);
if (!s_exists.ignore(pos, end, expected))
return false;
ws.ignore(pos, end);
if (!s_table.ignore(pos, end, expected))
return false;
ws.ignore(pos, end);
if (!name_p.parse(pos, end, table, expected))
return false;
ws.ignore(pos, end);
if (s_dot.ignore(pos, end, expected))
{
database = table;
if (!name_p.parse(pos, end, table, expected))
return false;
ws.ignore(pos, end);
}
ws.ignore(pos, end);
ASTExistsQuery * query = new ASTExistsQuery(StringRange(begin, pos));
node = query;
if (database)
query->database = dynamic_cast<ASTIdentifier &>(*database).name;
if (table)
query->table = dynamic_cast<ASTIdentifier &>(*table).name;
return true;
}
}

View File

@ -5,6 +5,7 @@
#include <DB/Parsers/ParserRenameQuery.h>
#include <DB/Parsers/ParserShowTablesQuery.h>
#include <DB/Parsers/ParserOptimizeQuery.h>
#include <DB/Parsers/ParserExistsQuery.h>
#include <DB/Parsers/ParserUseQuery.h>
#include <DB/Parsers/ParserSetQuery.h>
#include <DB/Parsers/ParserQuery.h>
@ -25,6 +26,7 @@ bool ParserQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected
ParserUseQuery use_p;
ParserSetQuery set_p;
ParserOptimizeQuery optimize_p;
ParserExistsQuery exists_p;
bool res = show_tables_p.parse(pos, end, node, expected)
|| select_p.parse(pos, end, node, expected)
@ -34,10 +36,11 @@ bool ParserQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & 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);
|| optimize_p.parse(pos, end, node, expected)
|| exists_p.parse(pos, end, node, expected);
if (!res)
expected = "One of: SHOW TABLES, SHOW DATABASES, SELECT, INSERT, CREATE, ATTACH, RENAME, DROP, DETACH, USE, SET, OPTIMIZE";
expected = "One of: SHOW TABLES, SHOW DATABASES, SELECT, INSERT, CREATE, ATTACH, RENAME, DROP, DETACH, USE, SET, OPTIMIZE, EXISTS";
return res;
}

View File

@ -56,6 +56,7 @@ void formatAST(const IAST & ast, std::ostream & s, size_t indent, bool hilite, b
DISPATCH(UseQuery)
DISPATCH(SetQuery)
DISPATCH(OptimizeQuery)
DISPATCH(ExistsQuery)
DISPATCH(ExpressionList)
DISPATCH(Function)
DISPATCH(Identifier)
@ -224,6 +225,12 @@ void formatAST(const ASTOptimizeQuery & ast, std::ostream & s, size_t indent, b
<< (!ast.database.empty() ? backQuoteIfNeed(ast.database) + "." : "") << backQuoteIfNeed(ast.table);
}
void formatAST(const ASTExistsQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line)
{
s << (hilite ? hilite_keyword : "") << "EXISTS TABLE " << (hilite ? hilite_none : "")
<< (!ast.database.empty() ? backQuoteIfNeed(ast.database) + "." : "") << backQuoteIfNeed(ast.table);
}
void formatAST(const ASTRenameQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line)
{
s << (hilite ? hilite_keyword : "") << "RENAME TABLE " << (hilite ? hilite_none : "");