提交 5de1975b 编写于 作者: A Alexey Zatelepin 提交者: alexey-milovidov

parse INTO OUTFILE clause [#CLICKHOUSE-2133]

上级 22797da2
......@@ -13,6 +13,7 @@ namespace DB
void executeQuery(
ReadBuffer & istr, /// Откуда читать запрос (а также данные для INSERT-а, если есть)
WriteBuffer & ostr, /// Куда писать результат
bool allow_into_outfile, /// If true and the query contains INTO OUTFILE section, output will be redirected to that file.
Context & context, /// БД, таблицы, типы данных, движки таблиц, функции, агрегатные функции...
BlockInputStreamPtr & query_plan, /// Сюда может быть записано описание, как выполнялся запрос
std::function<void(const String &)> set_content_type /// Может быть передан колбэк, с помощью которого может быть сообщён Content-Type формата.
......
......@@ -14,14 +14,17 @@ struct ASTCheckQuery : public ASTQueryWithOutput
ASTPtr clone() const override
{
return std::make_shared<ASTCheckQuery>(*this);
auto res = std::make_shared<ASTCheckQuery>(*this);
res->children.clear();
cloneOutputOptions(*res);
return res;
}
std::string database;
std::string table;
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
std::string nl_or_nothing = settings.one_line ? "" : "\n";
......@@ -39,7 +42,6 @@ protected:
}
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << backQuoteIfNeed(table) << (settings.hilite ? hilite_none : "");
}
settings.ostr << nl_or_ws;
}
};
......
......@@ -6,20 +6,25 @@
namespace DB
{
/** Запрос с секцией FORMAT.
/** Query with output options (supporting [INTO OUTFILE 'file_name'] [FORMAT format_name] suffix).
*/
class ASTQueryWithOutput : public IAST
{
public:
ASTPtr out_file;
ASTPtr format;
ASTQueryWithOutput() = default;
ASTQueryWithOutput(const StringRange range_) : IAST(range_) {}
/** Возвращает указатель на формат. Если типом объекта является ASTSelectQuery,
* то эта функция возвращает указатель на формат из последнего SELECT'а цепочки UNION ALL.
*/
virtual const IAST * getFormat() const { return format.get(); }
protected:
/// NOTE: call this helper at the end of the clone() method of descendant class.
void cloneOutputOptions(ASTQueryWithOutput& cloned) const;
/// Format only the query part of the AST (without output options).
virtual void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const = 0;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override final;
};
......@@ -36,16 +41,12 @@ public: \
{ \
std::shared_ptr<Name> res = std::make_shared<Name>(*this); \
res->children.clear(); \
if (format) \
{ \
res->format = format->clone(); \
res->children.push_back(res->format); \
} \
cloneOutputOptions(*res); \
return res; \
} \
\
protected: \
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override \
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override \
{ \
settings.ostr << (settings.hilite ? hilite_keyword : "") << Query << (settings.hilite ? hilite_none : ""); \
} \
......
......@@ -24,13 +24,6 @@ protected:
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << name << " " << (settings.hilite ? hilite_none : "")
<< (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table);
if (format)
{
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << indent_str << "FORMAT " << (settings.hilite ? hilite_none : "");
format->formatImpl(settings, state, frame);
}
}
};
......@@ -48,16 +41,12 @@ protected:
{ \
std::shared_ptr<Name> res = std::make_shared<Name>(*this); \
res->children.clear(); \
if (format) \
{ \
res->format = format->clone(); \
res->children.push_back(res->format); \
} \
cloneOutputOptions(*res); \
return res; \
} \
\
protected: \
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override \
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override \
{ \
formatHelper(settings, state, frame, Query); \
} \
......
......@@ -41,11 +41,8 @@ public:
/// Получить глубокую копию дерева первого запроса SELECT.
ASTPtr cloneFirstSelect() const;
/// Возвращает указатель на формат из последнего SELECT'а цепочки UNION ALL.
const IAST * getFormat() const override;
private:
ASTPtr cloneImpl(bool traverse_union_all) const;
std::shared_ptr<ASTSelectQuery> cloneImpl(bool traverse_union_all) const;
public:
bool distinct = false;
......@@ -85,7 +82,7 @@ public:
IAST * prev_union_all = nullptr;
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
}
......@@ -29,18 +29,12 @@ public:
{
auto res = std::make_shared<ASTShowTablesQuery>(*this);
res->children.clear();
if (format)
{
res->format = format->clone();
res->children.push_back(res->format);
}
cloneOutputOptions(*res);
return res;
}
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
if (databases)
{
......@@ -58,13 +52,6 @@ protected:
settings.ostr << (settings.hilite ? hilite_keyword : "") << " LIKE " << (settings.hilite ? hilite_none : "")
<< mysqlxx::quote << like;
}
if (format)
{
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << indent_str << "FORMAT " << (settings.hilite ? hilite_none : "");
format->formatImpl(settings, state, frame);
}
}
};
......
......@@ -7,7 +7,7 @@ namespace DB
/** Запрос вида
* CHECK [TABLE] [database.]table
*/
class ParserCheckQuery : public ParserQueryWithOutput
class ParserCheckQuery : public IParserBase
{
protected:
const char * getName() const { return "ALTER query"; }
......
......@@ -7,12 +7,13 @@
namespace DB
{
/** Парсер для запросов поддерживающих секцию FORMAT.
*/
/// Parse queries supporting [INTO OUTFILE 'file_name'] [FORMAT format_name] suffix.
class ParserQueryWithOutput : public IParserBase
{
protected:
bool parseFormat(ASTQueryWithOutput & query, Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
const char * getName() const override { return "Query with output"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected) override;
protected:
ParserWhiteSpaceOrComments ws;
......
......@@ -7,7 +7,7 @@ namespace DB
{
class ParserSelectQuery : public ParserQueryWithOutput
class ParserSelectQuery : public IParserBase
{
protected:
const char * getName() const override { return "SELECT query"; }
......
......@@ -12,7 +12,7 @@ namespace DB
/** Запрос SHOW PROCESSLIST
*/
class ParserShowProcesslistQuery : public ParserQueryWithOutput
class ParserShowProcesslistQuery : public IParserBase
{
protected:
const char * getName() const { return "SHOW PROCESSLIST query"; }
......@@ -21,6 +21,7 @@ protected:
{
Pos begin = pos;
ParserWhiteSpaceOrComments ws;
ParserString s_show("SHOW", true, true);
ParserString s_processlist("PROCESSLIST", true, true);
......@@ -38,10 +39,6 @@ protected:
ws.ignore(pos, end);
/// FORMAT format_name
if (!parseFormat(*query, pos, end, node, max_parsed_pos, expected))
return false;
query->range = StringRange(begin, pos);
node = query;
......
......@@ -11,7 +11,7 @@ namespace DB
* или
* SHOW DATABASES.
*/
class ParserShowTablesQuery : public ParserQueryWithOutput
class ParserShowTablesQuery : public IParserBase
{
protected:
const char * getName() const { return "SHOW TABLES|DATABASES query"; }
......
......@@ -10,7 +10,7 @@ namespace DB
/** Запрос (EXISTS | SHOW CREATE | (DESCRIBE | DESC) ) [TABLE] [db.]name [FORMAT format]
*/
class ParserTablePropertiesQuery : public ParserQueryWithOutput
class ParserTablePropertiesQuery : public IParserBase
{
protected:
const char * getName() const { return "EXISTS, SHOW CREATE or DESCRIBE query"; }
......
......@@ -6,6 +6,7 @@
#include <iostream>
#include <fstream>
#include <iomanip>
#include <experimental/optional>
#include <unordered_set>
#include <algorithm>
......@@ -25,6 +26,7 @@
#include <DB/IO/ReadBufferFromFileDescriptor.h>
#include <DB/IO/WriteBufferFromFileDescriptor.h>
#include <DB/IO/WriteBufferFromFile.h>
#include <DB/IO/WriteBufferFromString.h>
#include <DB/IO/ReadBufferFromMemory.h>
#include <DB/IO/ReadHelpers.h>
......@@ -40,6 +42,7 @@
#include <DB/Parsers/ASTInsertQuery.h>
#include <DB/Parsers/ASTSelectQuery.h>
#include <DB/Parsers/ASTQueryWithOutput.h>
#include <DB/Parsers/ASTLiteral.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/formatAST.h>
#include <DB/Parsers/parseQuery.h>
......@@ -140,6 +143,7 @@ private:
String query; /// Текущий запрос.
String format; /// Формат вывода результата в консоль.
bool is_default_format = true; /// false, если взяли формат из конфига или командной строки.
size_t format_max_block_size = 0; /// Максимальный размер блока при выводе в консоль.
String insert_format; /// Формат данных для INSERT-а при чтении их из stdin в batch режиме
size_t insert_format_max_block_size = 0; /// Максимальный размер блока при чтении данных INSERT-а.
......@@ -153,7 +157,9 @@ private:
/// Вывод в консоль
WriteBufferFromFileDescriptor std_out {STDOUT_FILENO};
BlockOutputStreamPtr block_std_out;
/// Клиент может попросить вывести результат в файл.
std::experimental::optional<WriteBufferFromFile> out_file_buf;
BlockOutputStreamPtr block_out_stream;
String home_path;
......@@ -307,6 +313,7 @@ private:
if (is_interactive)
showClientVersion();
is_default_format = !config().has("vertical") && !config().has("format");
if (config().has("vertical"))
format = config().getString("format", "Vertical");
else
......@@ -842,7 +849,12 @@ private:
/** Сбросить все данные, что ещё остались в буферах. */
void resetOutput()
{
block_std_out = nullptr;
block_out_stream = nullptr;
if (out_file_buf)
{
out_file_buf->next();
out_file_buf = std::experimental::nullopt;
}
std_out.next();
}
......@@ -957,27 +969,39 @@ private:
void initBlockOutputStream(const Block & block)
{
if (!block_std_out)
if (!block_out_stream)
{
WriteBuffer * out_buf = &std_out;
String current_format = format;
/// Формат может быть указан в запросе.
if (ASTQueryWithOutput * query_with_output = dynamic_cast<ASTQueryWithOutput *>(&*parsed_query))
{
if (query_with_output->getFormat() != nullptr)
if (query_with_output->out_file != nullptr)
{
const auto & out_file_node = typeid_cast<const ASTLiteral &>(*query_with_output->out_file);
const auto & out_file = out_file_node.value.safeGet<std::string>();
out_file_buf.emplace(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT);
out_buf = &out_file_buf.value();
// We are writing to file, so default format is the same as in non-interactive mode.
if (is_interactive && is_default_format)
current_format = "TabSeparated";
}
if (query_with_output->format != nullptr)
{
if (has_vertical_output_suffix)
throw Exception("Output format already specified", ErrorCodes::CLIENT_OUTPUT_FORMAT_SPECIFIED);
if (const ASTIdentifier * id = typeid_cast<const ASTIdentifier *>(query_with_output->getFormat()))
current_format = id->name;
const auto & id = typeid_cast<const ASTIdentifier &>(*query_with_output->format);
current_format = id.name;
}
}
if (has_vertical_output_suffix)
current_format = "Vertical";
block_std_out = context.getOutputFormat(current_format, std_out, block);
block_std_out->writePrefix();
block_out_stream = context.getOutputFormat(current_format, *out_buf, block);
block_out_stream->writePrefix();
}
}
......@@ -993,36 +1017,36 @@ private:
processed_rows += block.rows();
initBlockOutputStream(block);
/// Заголовочный блок с нулем строк использовался для инициализации block_std_out,
/// Заголовочный блок с нулем строк использовался для инициализации block_out_stream,
/// выводить его не нужно
if (block.rows() != 0)
{
block_std_out->write(block);
block_out_stream->write(block);
written_first_block = true;
}
/// Полученный блок данных сразу выводится клиенту.
block_std_out->flush();
block_out_stream->flush();
}
void onTotals(Block & block)
{
initBlockOutputStream(block);
block_std_out->setTotals(block);
block_out_stream->setTotals(block);
}
void onExtremes(Block & block)
{
initBlockOutputStream(block);
block_std_out->setExtremes(block);
block_out_stream->setExtremes(block);
}
void onProgress(const Progress & value)
{
progress.incrementPiecewiseAtomically(value);
block_std_out->onProgress(value);
block_out_stream->onProgress(value);
writeProgress();
}
......@@ -1139,15 +1163,15 @@ private:
void onProfileInfo(const BlockStreamProfileInfo & profile_info)
{
if (profile_info.hasAppliedLimit() && block_std_out)
block_std_out->setRowsBeforeLimit(profile_info.getRowsBeforeLimit());
if (profile_info.hasAppliedLimit() && block_out_stream)
block_out_stream->setRowsBeforeLimit(profile_info.getRowsBeforeLimit());
}
void onEndOfStream()
{
if (block_std_out)
block_std_out->writeSuffix();
if (block_out_stream)
block_out_stream->writeSuffix();
resetOutput();
......
......@@ -360,6 +360,7 @@ namespace ErrorCodes
extern const int ZLIB_DEFLATE_FAILED = 355;
extern const int BAD_LAMBDA = 356;
extern const int RESERVED_IDENTIFIER_NAME = 357;
extern const int INTO_OUTFILE_NOT_ALLOWED = 358;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;
......
......@@ -19,15 +19,7 @@ BlockIO InterpreterShowProcesslistQuery::execute()
String InterpreterShowProcesslistQuery::getRewrittenQuery()
{
const ASTQueryWithOutput & query = dynamic_cast<const ASTQueryWithOutput &>(*query_ptr);
std::stringstream rewritten_query;
rewritten_query << "SELECT * FROM system.processes";
if (query.format)
rewritten_query << " FORMAT " << typeid_cast<const ASTIdentifier &>(*query.format).name;
return rewritten_query.str();
return "SELECT * FROM system.processes";
}
......
......@@ -23,13 +23,9 @@ String InterpreterShowTablesQuery::getRewrittenQuery()
{
const ASTShowTablesQuery & query = typeid_cast<const ASTShowTablesQuery &>(*query_ptr);
String format_or_nothing;
if (query.format)
format_or_nothing = " FORMAT " + typeid_cast<const ASTIdentifier &>(*query.format).name;
/// SHOW DATABASES
if (query.databases)
return "SELECT name FROM system.databases" + format_or_nothing;
return "SELECT name FROM system.databases";
String database = query.from.empty() ? context.getCurrentDatabase() : query.from;
......@@ -45,8 +41,6 @@ String InterpreterShowTablesQuery::getRewrittenQuery()
if (!query.like.empty())
rewritten_query << " AND name " << (query.not_like ? "NOT " : "") << "LIKE " << mysqlxx::quote << query.like;
rewritten_query << format_or_nothing;
return rewritten_query.str();
}
......
......@@ -2,6 +2,7 @@
#include <DB/Common/formatReadable.h>
#include <DB/IO/ConcatReadBuffer.h>
#include <DB/IO/WriteBufferFromFile.h>
#include <DB/DataStreams/BlockIO.h>
#include <DB/DataStreams/copyData.h>
......@@ -12,6 +13,7 @@
#include <DB/Parsers/ASTInsertQuery.h>
#include <DB/Parsers/ASTShowProcesslistQuery.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/ASTLiteral.h>
#include <DB/Parsers/ParserQuery.h>
#include <DB/Parsers/parseQuery.h>
......@@ -34,6 +36,7 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int QUERY_IS_TOO_LARGE;
extern const int INTO_OUTFILE_NOT_ALLOWED;
}
......@@ -353,6 +356,7 @@ BlockIO executeQuery(
void executeQuery(
ReadBuffer & istr,
WriteBuffer & ostr,
bool allow_into_outfile,
Context & context,
BlockInputStreamPtr & query_plan,
std::function<void(const String &)> set_content_type)
......@@ -400,11 +404,23 @@ void executeQuery(
{
const ASTQueryWithOutput * ast_query_with_output = dynamic_cast<const ASTQueryWithOutput *>(ast.get());
String format_name = ast_query_with_output && (ast_query_with_output->getFormat() != nullptr)
? typeid_cast<const ASTIdentifier &>(*ast_query_with_output->getFormat()).name
WriteBuffer * out_buf = &ostr;
std::experimental::optional<WriteBufferFromFile> out_file_buf;
if (ast_query_with_output && ast_query_with_output->out_file)
{
if (!allow_into_outfile)
throw Exception("INTO OUTFILE is not allowed", ErrorCodes::INTO_OUTFILE_NOT_ALLOWED);
const auto & out_file = typeid_cast<const ASTLiteral &>(*ast_query_with_output->out_file).value.safeGet<std::string>();
out_file_buf.emplace(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT);
out_buf = &out_file_buf.value();
}
String format_name = ast_query_with_output && (ast_query_with_output->format != nullptr)
? typeid_cast<const ASTIdentifier &>(*ast_query_with_output->format).name
: context.getDefaultFormat();
BlockOutputStreamPtr out = context.getOutputFormat(format_name, ostr, streams.in_sample);
BlockOutputStreamPtr out = context.getOutputFormat(format_name, *out_buf, streams.in_sample);
if (auto stream = dynamic_cast<IProfilingBlockInputStream *>(streams.in.get()))
{
......
......@@ -45,7 +45,7 @@ try
WriteBufferFromOStream out(std::cout);
BlockInputStreamPtr query_plan;
executeQuery(in, out, context, query_plan, {});
executeQuery(in, out, /* allow_into_outfile = */ false, context, query_plan, {});
if (query_plan)
{
......
#include <DB/Parsers/ASTQueryWithOutput.h>
namespace DB
{
void ASTQueryWithOutput::cloneOutputOptions(ASTQueryWithOutput& cloned) const
{
if (out_file)
{
cloned.out_file = out_file->clone();
cloned.children.push_back(cloned.out_file);
}
if (format)
{
cloned.format = format->clone();
cloned.children.push_back(cloned.format);
}
}
void ASTQueryWithOutput::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const
{
formatQueryImpl(s, state, frame);
std::string indent_str = s.one_line ? "" : std::string(4 * frame.indent, ' ');
if (out_file)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "INTO OUTFILE " << (s.hilite ? hilite_none : "");
out_file->formatImpl(s, state, frame);
}
if (format)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "FORMAT " << (s.hilite ? hilite_none : "");
format->formatImpl(s, state, frame);
}
}
}
......@@ -154,31 +154,33 @@ void ASTSelectQuery::rewriteSelectExpressionList(const Names & required_column_n
ASTPtr ASTSelectQuery::clone() const
{
ASTPtr ptr = cloneImpl(true);
auto ptr = cloneImpl(true);
/// Установить указатели на предыдущие запросы SELECT.
ASTPtr current = ptr;
static_cast<ASTSelectQuery *>(&*current)->prev_union_all = nullptr;
ASTPtr next = static_cast<ASTSelectQuery *>(&*current)->next_union_all;
static_cast<ASTSelectQuery *>(current.get())->prev_union_all = nullptr;
ASTPtr next = static_cast<ASTSelectQuery *>(current.get())->next_union_all;
while (next != nullptr)
{
ASTSelectQuery * next_select_query = static_cast<ASTSelectQuery *>(&*next);
ASTSelectQuery * next_select_query = static_cast<ASTSelectQuery *>(next.get());
next_select_query->prev_union_all = current.get();
current = next;
next = next_select_query->next_union_all;
}
cloneOutputOptions(*ptr);
return ptr;
}
ASTPtr ASTSelectQuery::cloneFirstSelect() const
{
ASTPtr res = cloneImpl(false);
static_cast<ASTSelectQuery *>(&*res)->prev_union_all = nullptr;
auto res = cloneImpl(false);
res->prev_union_all = nullptr;
return res;
}
ASTPtr ASTSelectQuery::cloneImpl(bool traverse_union_all) const
std::shared_ptr<ASTSelectQuery> ASTSelectQuery::cloneImpl(bool traverse_union_all) const
{
auto res = std::make_shared<ASTSelectQuery>(*this);
res->children.clear();
......@@ -207,7 +209,6 @@ ASTPtr ASTSelectQuery::cloneImpl(bool traverse_union_all) const
CLONE(limit_offset)
CLONE(limit_length)
CLONE(settings)
CLONE(format)
#undef CLONE
......@@ -225,16 +226,7 @@ ASTPtr ASTSelectQuery::cloneImpl(bool traverse_union_all) const
return res;
}
const IAST * ASTSelectQuery::getFormat() const
{
const ASTSelectQuery * query = this;
while (query->next_union_all != nullptr)
query = static_cast<const ASTSelectQuery *>(query->next_union_all.get());
return query->format.get();
}
void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const
void ASTSelectQuery::formatQueryImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const
{
frame.current_select = this;
frame.need_parens = false;
......@@ -324,12 +316,6 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F
}
}
if (format)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "FORMAT " << (s.hilite ? hilite_none : "");
format->formatImpl(s, state, frame);
}
if (next_union_all)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "UNION ALL " << s.nl_or_ws << (s.hilite ? hilite_none : "");
......
......@@ -642,6 +642,7 @@ const char * ParserAliasBase::restricted_keywords[] =
"SETTINGS",
"FORMAT",
"UNION",
"INTO",
nullptr
};
......
......@@ -24,7 +24,7 @@ bool IParserBase::parse(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos,
if (new_max_parsed_pos > max_parsed_pos)
max_parsed_pos = new_max_parsed_pos;
if (new_max_parsed_pos >= max_parsed_pos)
if (!res && new_max_parsed_pos >= max_parsed_pos)
expected = new_expected;
if (pos > end)
......
......@@ -12,6 +12,7 @@ namespace DB
bool ParserCheckQuery::parseImpl(IParser::Pos & pos, IParser::Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
ParserWhiteSpaceOrComments ws;
ParserString s_check("CHECK", true, true);
ParserString s_table("TABLE", true, true);
ParserString s_dot(".");
......@@ -51,10 +52,6 @@ bool ParserCheckQuery::parseImpl(IParser::Pos & pos, IParser::Pos end, ASTPtr &
ws.ignore(pos, end);
/// FORMAT format_name
if (!parseFormat(*query, pos, end, node, max_parsed_pos, expected))
return false;
node = query;
return true;
}
......
#include <DB/Parsers/ParserSelectQuery.h>
#include <DB/Parsers/ParserQuery.h>
#include <DB/Parsers/ParserQueryWithOutput.h>
#include <DB/Parsers/ParserCreateQuery.h>
#include <DB/Parsers/ParserInsertQuery.h>
#include <DB/Parsers/ParserDropQuery.h>
#include <DB/Parsers/ParserRenameQuery.h>
#include <DB/Parsers/ParserShowTablesQuery.h>
#include <DB/Parsers/ParserOptimizeQuery.h>
#include <DB/Parsers/TablePropertiesQueriesASTs.h>
#include <DB/Parsers/ParserUseQuery.h>
#include <DB/Parsers/ParserSetQuery.h>
#include <DB/Parsers/ParserQuery.h>
#include <DB/Parsers/ParserTablePropertiesQuery.h>
#include <DB/Parsers/ParserAlterQuery.h>
#include <DB/Parsers/ParserShowProcesslistQuery.h>
#include <DB/Parsers/ParserCheckQuery.h>
//#include <DB/Parsers/ParserMultiQuery.h>
......@@ -22,8 +17,7 @@ namespace DB
bool ParserQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
ParserShowTablesQuery show_tables_p;
ParserSelectQuery select_p;
ParserQueryWithOutput query_with_output_p;
ParserInsertQuery insert_p;
ParserCreateQuery create_p;
ParserRenameQuery rename_p;
......@@ -32,13 +26,9 @@ bool ParserQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_
ParserUseQuery use_p;
ParserSetQuery set_p;
ParserOptimizeQuery optimize_p;
ParserTablePropertiesQuery table_p;
ParserShowProcesslistQuery show_processlist_p;
ParserCheckQuery check_p;
// ParserMultiQuery multi_p;
bool res = show_tables_p.parse(pos, end, node, max_parsed_pos, expected)
|| select_p.parse(pos, end, node, max_parsed_pos, expected)
bool res = query_with_output_p.parse(pos, end, node, max_parsed_pos, expected)
|| insert_p.parse(pos, end, node, max_parsed_pos, expected)
|| create_p.parse(pos, end, node, max_parsed_pos, expected)
|| rename_p.parse(pos, end, node, max_parsed_pos, expected)
......@@ -46,10 +36,7 @@ bool ParserQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_
|| alter_p.parse(pos, end, node, max_parsed_pos, expected)
|| use_p.parse(pos, end, node, max_parsed_pos, expected)
|| set_p.parse(pos, end, node, max_parsed_pos, expected)
|| optimize_p.parse(pos, end, node, max_parsed_pos, expected)
|| table_p.parse(pos, end, node, max_parsed_pos, expected)
|| show_processlist_p.parse(pos, end, node, max_parsed_pos, expected)
|| check_p.parse(pos, end, node, max_parsed_pos, expected);
|| optimize_p.parse(pos, end, node, max_parsed_pos, expected);
/* || multi_p.parse(pos, end, node, max_parsed_pos, expected)*/;
if (!res && (!expected || !*expected))
......
#include <DB/Parsers/ParserQueryWithOutput.h>
#include <DB/Parsers/ParserShowTablesQuery.h>
#include <DB/Parsers/ParserSelectQuery.h>
#include <DB/Parsers/ParserTablePropertiesQuery.h>
#include <DB/Parsers/ParserShowProcesslistQuery.h>
#include <DB/Parsers/ParserCheckQuery.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/ExpressionElementParsers.h>
#include <DB/Common/typeid_cast.h>
......@@ -7,8 +12,50 @@
namespace DB
{
bool ParserQueryWithOutput::parseFormat(ASTQueryWithOutput & query, Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
bool ParserQueryWithOutput::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
ParserShowTablesQuery show_tables_p;
ParserSelectQuery select_p;
ParserTablePropertiesQuery table_p;
ParserShowProcesslistQuery show_processlist_p;
ParserCheckQuery check_p;
ASTPtr query;
bool parsed = select_p.parse(pos, end, query, max_parsed_pos, expected)
|| show_tables_p.parse(pos, end, query, max_parsed_pos, expected)
|| table_p.parse(pos, end, query, max_parsed_pos, expected)
|| show_processlist_p.parse(pos, end, query, max_parsed_pos, expected)
|| check_p.parse(pos, end, query, max_parsed_pos, expected);
if (!parsed)
return false;
auto & query_with_output = dynamic_cast<ASTQueryWithOutput &>(*query);
ParserString s_into("INTO", /* word_boundary_ = */ true, /* case_insensitive_ = */ true);
if (s_into.ignore(pos, end, max_parsed_pos, expected))
{
ws.ignore(pos, end);
ParserString s_into("OUTFILE", true, true);
if (!s_into.ignore(pos, end, max_parsed_pos, expected))
{
expected = "OUTFILE";
return false;
}
ws.ignore(pos, end);
ParserStringLiteral out_file_p;
if (!out_file_p.parse(pos, end, query_with_output.out_file, max_parsed_pos, expected))
return false;
query_with_output.children.push_back(query_with_output.out_file);
ws.ignore(pos, end);
}
ParserString s_format("FORMAT", true, true);
if (s_format.ignore(pos, end, max_parsed_pos, expected))
......@@ -17,13 +64,16 @@ bool ParserQueryWithOutput::parseFormat(ASTQueryWithOutput & query, Pos & pos, P
ParserIdentifier format_p;
if (!format_p.parse(pos, end, query.format, max_parsed_pos, expected))
if (!format_p.parse(pos, end, query_with_output.format, max_parsed_pos, expected))
return false;
typeid_cast<ASTIdentifier &>(*(query.format)).kind = ASTIdentifier::Format;
typeid_cast<ASTIdentifier &>(*(query_with_output.format)).kind = ASTIdentifier::Format;
query_with_output.children.push_back(query_with_output.format);
ws.ignore(pos, end);
}
node = query;
return true;
}
......
......@@ -28,6 +28,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
auto select_query = std::make_shared<ASTSelectQuery>();
node = select_query;
ParserWhiteSpaceOrComments ws;
ParserString s_select("SELECT", true, true);
ParserString s_distinct("DISTINCT", true, true);
ParserString s_from("FROM", true, true);
......@@ -226,10 +227,6 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
ws.ignore(pos, end);
}
/// FORMAT format_name
if (!parseFormat(*select_query, pos, end, node, max_parsed_pos, expected))
return false;
// UNION ALL select query
if (s_union.ignore(pos, end, max_parsed_pos, expected))
{
......@@ -237,13 +234,6 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
if (s_all.ignore(pos, end, max_parsed_pos, expected))
{
if (select_query->format)
{
/// FORMAT может быть задан только в последнем запросе цепочки UNION ALL.
expected = "FORMAT only in the last SELECT of the UNION ALL chain";
return false;
}
ParserSelectQuery select_p;
if (!select_p.parse(pos, end, select_query->next_union_all, max_parsed_pos, expected))
return false;
......@@ -281,8 +271,6 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
select_query->children.push_back(select_query->limit_length);
if (select_query->settings)
select_query->children.push_back(select_query->settings);
if (select_query->format)
select_query->children.push_back(select_query->format);
if (select_query->next_union_all)
select_query->children.push_back(select_query->next_union_all);
......
......@@ -17,6 +17,7 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & m
{
Pos begin = pos;
ParserWhiteSpaceOrComments ws;
ParserString s_show("SHOW", true, true);
ParserString s_tables("TABLES", true, true);
ParserString s_databases("DATABASES", true, true);
......@@ -78,18 +79,12 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & m
ws.ignore(pos, end);
/// FORMAT format_name
if (!parseFormat(*query, pos, end, node, max_parsed_pos, expected))
return false;
query->range = StringRange(begin, pos);
if (database)
query->from = typeid_cast<ASTIdentifier &>(*database).name;
if (like)
query->like = safeGet<const String &>(typeid_cast<ASTLiteral &>(*like).value);
if (query->format)
query->children.push_back(query->format);
node = query;
......
......@@ -15,6 +15,7 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Po
{
Pos begin = pos;
ParserWhiteSpaceOrComments ws;
ParserString s_exists("EXISTS", true, true);
ParserString s_describe("DESCRIBE", true, true);
ParserString s_desc("DESC", true, true);
......@@ -26,17 +27,17 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Po
ASTPtr database;
ASTPtr table;
ASTPtr query_ptr;
std::shared_ptr<ASTQueryWithTableAndOutput> query;
ws.ignore(pos, end);
if (s_exists.ignore(pos, end, max_parsed_pos, expected))
{
query_ptr = std::make_shared<ASTExistsQuery>();
query = std::make_shared<ASTExistsQuery>();
}
else if (s_describe.ignore(pos, end, max_parsed_pos, expected) || s_desc.ignore(pos, end, max_parsed_pos, expected))
{
query_ptr = std::make_shared<ASTDescribeQuery>();
query = std::make_shared<ASTDescribeQuery>();
}
else if (s_show.ignore(pos, end, max_parsed_pos, expected))
{
......@@ -45,15 +46,13 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Po
if (!s_create.ignore(pos, end, max_parsed_pos, expected))
return false;
query_ptr = std::make_shared<ASTShowCreateQuery>();
query = std::make_shared<ASTShowCreateQuery>();
}
else
{
return false;
}
ASTQueryWithTableAndOutput * query = dynamic_cast<ASTQueryWithTableAndOutput *>(&*query_ptr);
ws.ignore(pos, end);
s_table.ignore(pos, end, max_parsed_pos, expected);
......@@ -76,20 +75,14 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Po
ws.ignore(pos, end);
/// FORMAT format_name
if (!parseFormat(*query, pos, end, node, max_parsed_pos, expected))
return false;
query->range = StringRange(begin, pos);
if (database)
query->database = typeid_cast<ASTIdentifier &>(*database).name;
if (table)
query->table = typeid_cast<ASTIdentifier &>(*table).name;
if (query->format)
query->children.push_back(query->format);
node = query_ptr;
node = query;
return true;
}
......
......@@ -133,6 +133,8 @@ ASTPtr tryParseQuery(
/// Parsed query must end with end of data or semicolon.
if (!parse_res || (pos != end && *pos != ';'))
{
if (!expected || !*expected)
expected = "end of query";
out_error_message = getSyntaxErrorMessage(begin, end, max_parsed_pos, expected, hilite, description);
return nullptr;
}
......
#include <iostream>
#include <DB/Parsers/ParserSelectQuery.h>
#include <DB/Parsers/ParserQueryWithOutput.h>
#include <DB/Parsers/parseQuery.h>
#include <DB/Parsers/formatAST.h>
#include <DB/Parsers/parseQuery.h>
......@@ -17,9 +17,11 @@ int main(int argc, char ** argv)
" GROUP BY UniqID"
" HAVING SUM(Refresh) > 100"
" ORDER BY Visits, PageViews"
" LIMIT 1000, 10";
" LIMIT 1000, 10"
" INTO OUTFILE 'test.out'"
" FORMAT TabSeparated";
ParserSelectQuery parser;
ParserQueryWithOutput parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "");
std::cout << "Success." << std::endl;
......
......@@ -257,7 +257,7 @@ void HTTPHandler::processQuery(
client_info.http_method = http_method;
client_info.http_user_agent = request.get("User-Agent", "");
executeQuery(*in, *used_output.out_maybe_compressed, context, query_plan,
executeQuery(*in, *used_output.out_maybe_compressed, /* allow_into_outfile = */ false, context, query_plan,
[&response] (const String & content_type) { response.setContentType(content_type); });
/// Send HTTP headers with code 200 if no exception happened and the data is still not sent to
......
......@@ -439,7 +439,7 @@ void LocalServer::processQueries()
if (verbose)
LOG_INFO(log, "Executing query: " << query);
executeQuery(read_buf, write_buf, *context, plan, nullptr);
executeQuery(read_buf, write_buf, /* allow_into_outfile = */ true, *context, plan, nullptr);
}
}
......
performing test: select
1 2 3
performing test: union_all
1 2
3 4
performing test: bad_union_all
query failed
performing test: describe_table
dummy UInt8
performing test: clickhouse-local
2 3
performing test: http
query failed
#!/usr/bin/env bash
function perform()
{
local test_id=$1
local query=$2
echo "performing test: $test_id"
clickhouse-client --query "$query" 2>/dev/null
if [ "$?" -eq 0 ]; then
cat "./test_into_outfile_$test_id.out"
else
echo "query failed"
fi
rm -f "./test_into_outfile_$test_id.out"
}
perform "select" "SELECT 1, 2, 3 INTO OUTFILE './test_into_outfile_select.out'"
perform "union_all" "SELECT 1, 2 UNION ALL SELECT 3, 4 INTO OUTFILE './test_into_outfile_union_all.out' FORMAT TSV"
perform "bad_union_all" "SELECT 1, 2 INTO OUTFILE './test_into_outfile_bad_union_all.out' UNION ALL SELECT 3, 4"
perform "describe_table" "DESCRIBE TABLE system.one INTO OUTFILE './test_into_outfile_describe_table.out'"
echo "performing test: clickhouse-local"
echo -e '1\t2' | clickhouse-local --structure 'col1 UInt32, col2 UInt32' --query "SELECT col1 + 1, col2 + 1 FROM table INTO OUTFILE './test_into_outfile_clickhouse-local.out'" 2>/dev/null
if [ "$?" -eq 0 ]; then
cat "./test_into_outfile_clickhouse-local.out"
else
echo "query failed"
fi
rm -f "./test_into_outfile_clickhouse-local.out"
echo "performing test: http"
echo "SELECT 1, 2 INTO OUTFILE './test_into_outfile_http.out'" | curl -s 'http://localhost:8123' -d @- --fail || echo "query failed"
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册