From 5de1975bc4820359dd240f2f5bcdf54ffe3056ce Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 11 Jan 2017 22:05:46 +0300 Subject: [PATCH] parse INTO OUTFILE clause [#CLICKHOUSE-2133] --- dbms/include/DB/Interpreters/executeQuery.h | 1 + dbms/include/DB/Parsers/ASTCheckQuery.h | 8 ++- dbms/include/DB/Parsers/ASTQueryWithOutput.h | 23 +++---- .../DB/Parsers/ASTQueryWithTableAndOutput.h | 15 +---- dbms/include/DB/Parsers/ASTSelectQuery.h | 7 +-- dbms/include/DB/Parsers/ASTShowTablesQuery.h | 17 +----- dbms/include/DB/Parsers/ParserCheckQuery.h | 2 +- .../DB/Parsers/ParserQueryWithOutput.h | 7 ++- dbms/include/DB/Parsers/ParserSelectQuery.h | 2 +- .../DB/Parsers/ParserShowProcesslistQuery.h | 7 +-- .../DB/Parsers/ParserShowTablesQuery.h | 2 +- .../DB/Parsers/ParserTablePropertiesQuery.h | 2 +- dbms/src/Client/Client.cpp | 60 +++++++++++++------ dbms/src/Core/ErrorCodes.cpp | 1 + .../InterpreterShowProcesslistQuery.cpp | 10 +--- .../InterpreterShowTablesQuery.cpp | 8 +-- dbms/src/Interpreters/executeQuery.cpp | 22 ++++++- dbms/src/Interpreters/tests/select_query.cpp | 2 +- dbms/src/Parsers/ASTQueryWithOutput.cpp | 39 ++++++++++++ dbms/src/Parsers/ASTSelectQuery.cpp | 34 ++++------- dbms/src/Parsers/ExpressionElementParsers.cpp | 1 + dbms/src/Parsers/IParserBase.cpp | 2 +- dbms/src/Parsers/ParserCheckQuery.cpp | 5 +- dbms/src/Parsers/ParserQuery.cpp | 23 ++----- dbms/src/Parsers/ParserQueryWithOutput.cpp | 56 ++++++++++++++++- dbms/src/Parsers/ParserSelectQuery.cpp | 14 +---- dbms/src/Parsers/ParserShowTablesQuery.cpp | 7 +-- .../Parsers/ParserTablePropertiesQuery.cpp | 19 ++---- dbms/src/Parsers/parseQuery.cpp | 2 + dbms/src/Parsers/tests/select_parser.cpp | 8 ++- dbms/src/Server/HTTPHandler.cpp | 2 +- dbms/src/Server/LocalServer.cpp | 2 +- .../0_stateless/00415_into_outfile.reference | 13 ++++ .../queries/0_stateless/00415_into_outfile.sh | 36 +++++++++++ 34 files changed, 275 insertions(+), 184 deletions(-) create mode 100644 dbms/src/Parsers/ASTQueryWithOutput.cpp create mode 100644 dbms/tests/queries/0_stateless/00415_into_outfile.reference create mode 100755 dbms/tests/queries/0_stateless/00415_into_outfile.sh diff --git a/dbms/include/DB/Interpreters/executeQuery.h b/dbms/include/DB/Interpreters/executeQuery.h index d5ca892187d..8693e6be6ca 100644 --- a/dbms/include/DB/Interpreters/executeQuery.h +++ b/dbms/include/DB/Interpreters/executeQuery.h @@ -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 set_content_type /// Может быть передан колбэк, с помощью которого может быть сообщён Content-Type формата. diff --git a/dbms/include/DB/Parsers/ASTCheckQuery.h b/dbms/include/DB/Parsers/ASTCheckQuery.h index ad63b8b6456..b52f8c0bef7 100644 --- a/dbms/include/DB/Parsers/ASTCheckQuery.h +++ b/dbms/include/DB/Parsers/ASTCheckQuery.h @@ -14,14 +14,17 @@ struct ASTCheckQuery : public ASTQueryWithOutput ASTPtr clone() const override { - return std::make_shared(*this); + auto res = std::make_shared(*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; } }; diff --git a/dbms/include/DB/Parsers/ASTQueryWithOutput.h b/dbms/include/DB/Parsers/ASTQueryWithOutput.h index c3325e59fad..a6f21508835 100644 --- a/dbms/include/DB/Parsers/ASTQueryWithOutput.h +++ b/dbms/include/DB/Parsers/ASTQueryWithOutput.h @@ -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 res = std::make_shared(*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 : ""); \ } \ diff --git a/dbms/include/DB/Parsers/ASTQueryWithTableAndOutput.h b/dbms/include/DB/Parsers/ASTQueryWithTableAndOutput.h index 91e5ccae9d5..c0d691c6ee4 100644 --- a/dbms/include/DB/Parsers/ASTQueryWithTableAndOutput.h +++ b/dbms/include/DB/Parsers/ASTQueryWithTableAndOutput.h @@ -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 res = std::make_shared(*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); \ } \ diff --git a/dbms/include/DB/Parsers/ASTSelectQuery.h b/dbms/include/DB/Parsers/ASTSelectQuery.h index 05d664d65da..4d529858424 100644 --- a/dbms/include/DB/Parsers/ASTSelectQuery.h +++ b/dbms/include/DB/Parsers/ASTSelectQuery.h @@ -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 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; }; } diff --git a/dbms/include/DB/Parsers/ASTShowTablesQuery.h b/dbms/include/DB/Parsers/ASTShowTablesQuery.h index 625aa82c317..ff3f5b12f95 100644 --- a/dbms/include/DB/Parsers/ASTShowTablesQuery.h +++ b/dbms/include/DB/Parsers/ASTShowTablesQuery.h @@ -29,18 +29,12 @@ public: { auto res = std::make_shared(*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); - } } }; diff --git a/dbms/include/DB/Parsers/ParserCheckQuery.h b/dbms/include/DB/Parsers/ParserCheckQuery.h index cba07d953b8..02150301275 100644 --- a/dbms/include/DB/Parsers/ParserCheckQuery.h +++ b/dbms/include/DB/Parsers/ParserCheckQuery.h @@ -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"; } diff --git a/dbms/include/DB/Parsers/ParserQueryWithOutput.h b/dbms/include/DB/Parsers/ParserQueryWithOutput.h index f1a2bd24b2b..441d01d46a0 100644 --- a/dbms/include/DB/Parsers/ParserQueryWithOutput.h +++ b/dbms/include/DB/Parsers/ParserQueryWithOutput.h @@ -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; diff --git a/dbms/include/DB/Parsers/ParserSelectQuery.h b/dbms/include/DB/Parsers/ParserSelectQuery.h index 482af380b9c..68e7bfd309e 100644 --- a/dbms/include/DB/Parsers/ParserSelectQuery.h +++ b/dbms/include/DB/Parsers/ParserSelectQuery.h @@ -7,7 +7,7 @@ namespace DB { -class ParserSelectQuery : public ParserQueryWithOutput +class ParserSelectQuery : public IParserBase { protected: const char * getName() const override { return "SELECT query"; } diff --git a/dbms/include/DB/Parsers/ParserShowProcesslistQuery.h b/dbms/include/DB/Parsers/ParserShowProcesslistQuery.h index 6ce8c9d854c..d524541f4d3 100644 --- a/dbms/include/DB/Parsers/ParserShowProcesslistQuery.h +++ b/dbms/include/DB/Parsers/ParserShowProcesslistQuery.h @@ -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; diff --git a/dbms/include/DB/Parsers/ParserShowTablesQuery.h b/dbms/include/DB/Parsers/ParserShowTablesQuery.h index e88cfa0009c..6a0c0261aae 100644 --- a/dbms/include/DB/Parsers/ParserShowTablesQuery.h +++ b/dbms/include/DB/Parsers/ParserShowTablesQuery.h @@ -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"; } diff --git a/dbms/include/DB/Parsers/ParserTablePropertiesQuery.h b/dbms/include/DB/Parsers/ParserTablePropertiesQuery.h index 066a46c2283..90b49a55503 100644 --- a/dbms/include/DB/Parsers/ParserTablePropertiesQuery.h +++ b/dbms/include/DB/Parsers/ParserTablePropertiesQuery.h @@ -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"; } diff --git a/dbms/src/Client/Client.cpp b/dbms/src/Client/Client.cpp index f9b8aa046b7..00b9b2d0359 100644 --- a/dbms/src/Client/Client.cpp +++ b/dbms/src/Client/Client.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -25,6 +26,7 @@ #include #include +#include #include #include #include @@ -40,6 +42,7 @@ #include #include #include +#include #include #include #include @@ -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 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(&*parsed_query)) { - if (query_with_output->getFormat() != nullptr) + if (query_with_output->out_file != nullptr) + { + const auto & out_file_node = typeid_cast(*query_with_output->out_file); + const auto & out_file = out_file_node.value.safeGet(); + 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(query_with_output->getFormat())) - current_format = id->name; + const auto & id = typeid_cast(*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(); diff --git a/dbms/src/Core/ErrorCodes.cpp b/dbms/src/Core/ErrorCodes.cpp index 8f11672b426..bd55edfa8ea 100644 --- a/dbms/src/Core/ErrorCodes.cpp +++ b/dbms/src/Core/ErrorCodes.cpp @@ -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; diff --git a/dbms/src/Interpreters/InterpreterShowProcesslistQuery.cpp b/dbms/src/Interpreters/InterpreterShowProcesslistQuery.cpp index 14ce73e7c23..71741d83a9f 100644 --- a/dbms/src/Interpreters/InterpreterShowProcesslistQuery.cpp +++ b/dbms/src/Interpreters/InterpreterShowProcesslistQuery.cpp @@ -19,15 +19,7 @@ BlockIO InterpreterShowProcesslistQuery::execute() String InterpreterShowProcesslistQuery::getRewrittenQuery() { - const ASTQueryWithOutput & query = dynamic_cast(*query_ptr); - - std::stringstream rewritten_query; - rewritten_query << "SELECT * FROM system.processes"; - - if (query.format) - rewritten_query << " FORMAT " << typeid_cast(*query.format).name; - - return rewritten_query.str(); + return "SELECT * FROM system.processes"; } diff --git a/dbms/src/Interpreters/InterpreterShowTablesQuery.cpp b/dbms/src/Interpreters/InterpreterShowTablesQuery.cpp index 6131b9fd598..62032d148b9 100644 --- a/dbms/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/dbms/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -23,13 +23,9 @@ String InterpreterShowTablesQuery::getRewrittenQuery() { const ASTShowTablesQuery & query = typeid_cast(*query_ptr); - String format_or_nothing; - if (query.format) - format_or_nothing = " FORMAT " + typeid_cast(*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(); } diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 0538fbc7441..b2614c093bd 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -12,6 +13,7 @@ #include #include #include +#include #include #include @@ -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 set_content_type) @@ -400,11 +404,23 @@ void executeQuery( { const ASTQueryWithOutput * ast_query_with_output = dynamic_cast(ast.get()); - String format_name = ast_query_with_output && (ast_query_with_output->getFormat() != nullptr) - ? typeid_cast(*ast_query_with_output->getFormat()).name + WriteBuffer * out_buf = &ostr; + std::experimental::optional 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(*ast_query_with_output->out_file).value.safeGet(); + 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(*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(streams.in.get())) { diff --git a/dbms/src/Interpreters/tests/select_query.cpp b/dbms/src/Interpreters/tests/select_query.cpp index db36109f07f..e7bf7fe0442 100644 --- a/dbms/src/Interpreters/tests/select_query.cpp +++ b/dbms/src/Interpreters/tests/select_query.cpp @@ -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) { diff --git a/dbms/src/Parsers/ASTQueryWithOutput.cpp b/dbms/src/Parsers/ASTQueryWithOutput.cpp new file mode 100644 index 00000000000..692940e7207 --- /dev/null +++ b/dbms/src/Parsers/ASTQueryWithOutput.cpp @@ -0,0 +1,39 @@ +#include + +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); + } +} + +} diff --git a/dbms/src/Parsers/ASTSelectQuery.cpp b/dbms/src/Parsers/ASTSelectQuery.cpp index 0ca0b4979dc..f8839464734 100644 --- a/dbms/src/Parsers/ASTSelectQuery.cpp +++ b/dbms/src/Parsers/ASTSelectQuery.cpp @@ -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(&*current)->prev_union_all = nullptr; - ASTPtr next = static_cast(&*current)->next_union_all; + static_cast(current.get())->prev_union_all = nullptr; + ASTPtr next = static_cast(current.get())->next_union_all; while (next != nullptr) { - ASTSelectQuery * next_select_query = static_cast(&*next); + ASTSelectQuery * next_select_query = static_cast(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(&*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::cloneImpl(bool traverse_union_all) const { auto res = std::make_shared(*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(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 : ""); diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 0c5b76283ac..a2d83be475f 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -642,6 +642,7 @@ const char * ParserAliasBase::restricted_keywords[] = "SETTINGS", "FORMAT", "UNION", + "INTO", nullptr }; diff --git a/dbms/src/Parsers/IParserBase.cpp b/dbms/src/Parsers/IParserBase.cpp index a0a7182713e..82c0a3423d1 100644 --- a/dbms/src/Parsers/IParserBase.cpp +++ b/dbms/src/Parsers/IParserBase.cpp @@ -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) diff --git a/dbms/src/Parsers/ParserCheckQuery.cpp b/dbms/src/Parsers/ParserCheckQuery.cpp index ab5f23cea83..a9c9992eeca 100644 --- a/dbms/src/Parsers/ParserCheckQuery.cpp +++ b/dbms/src/Parsers/ParserCheckQuery.cpp @@ -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; } diff --git a/dbms/src/Parsers/ParserQuery.cpp b/dbms/src/Parsers/ParserQuery.cpp index 17f0443f9e9..94bb96cb271 100644 --- a/dbms/src/Parsers/ParserQuery.cpp +++ b/dbms/src/Parsers/ParserQuery.cpp @@ -1,18 +1,13 @@ -#include +#include +#include #include #include #include #include -#include #include -#include #include #include -#include -#include #include -#include -#include //#include @@ -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)) diff --git a/dbms/src/Parsers/ParserQueryWithOutput.cpp b/dbms/src/Parsers/ParserQueryWithOutput.cpp index bde8cdaa149..cfe42796cb5 100644 --- a/dbms/src/Parsers/ParserQueryWithOutput.cpp +++ b/dbms/src/Parsers/ParserQueryWithOutput.cpp @@ -1,4 +1,9 @@ #include +#include +#include +#include +#include +#include #include #include #include @@ -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(*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(*(query.format)).kind = ASTIdentifier::Format; + typeid_cast(*(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; } diff --git a/dbms/src/Parsers/ParserSelectQuery.cpp b/dbms/src/Parsers/ParserSelectQuery.cpp index c2d085b2a95..7d60dce537d 100644 --- a/dbms/src/Parsers/ParserSelectQuery.cpp +++ b/dbms/src/Parsers/ParserSelectQuery.cpp @@ -28,6 +28,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p auto select_query = std::make_shared(); 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); diff --git a/dbms/src/Parsers/ParserShowTablesQuery.cpp b/dbms/src/Parsers/ParserShowTablesQuery.cpp index efd2c2add8e..99219d1eaea 100644 --- a/dbms/src/Parsers/ParserShowTablesQuery.cpp +++ b/dbms/src/Parsers/ParserShowTablesQuery.cpp @@ -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(*database).name; if (like) query->like = safeGet(typeid_cast(*like).value); - if (query->format) - query->children.push_back(query->format); node = query; diff --git a/dbms/src/Parsers/ParserTablePropertiesQuery.cpp b/dbms/src/Parsers/ParserTablePropertiesQuery.cpp index 76e9f607984..516fbc08c53 100644 --- a/dbms/src/Parsers/ParserTablePropertiesQuery.cpp +++ b/dbms/src/Parsers/ParserTablePropertiesQuery.cpp @@ -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 query; ws.ignore(pos, end); if (s_exists.ignore(pos, end, max_parsed_pos, expected)) { - query_ptr = std::make_shared(); + query = std::make_shared(); } 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(); + query = std::make_shared(); } 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(); + query = std::make_shared(); } else { return false; } - ASTQueryWithTableAndOutput * query = dynamic_cast(&*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(*database).name; if (table) query->table = typeid_cast(*table).name; - if (query->format) - query->children.push_back(query->format); - node = query_ptr; + node = query; return true; } diff --git a/dbms/src/Parsers/parseQuery.cpp b/dbms/src/Parsers/parseQuery.cpp index ceb5d2e4b91..c3cef3c4c14 100644 --- a/dbms/src/Parsers/parseQuery.cpp +++ b/dbms/src/Parsers/parseQuery.cpp @@ -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; } diff --git a/dbms/src/Parsers/tests/select_parser.cpp b/dbms/src/Parsers/tests/select_parser.cpp index 51ff9a6211c..e0cf9974055 100644 --- a/dbms/src/Parsers/tests/select_parser.cpp +++ b/dbms/src/Parsers/tests/select_parser.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include #include @@ -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; diff --git a/dbms/src/Server/HTTPHandler.cpp b/dbms/src/Server/HTTPHandler.cpp index d5b68b361ee..ece4373c1ab 100644 --- a/dbms/src/Server/HTTPHandler.cpp +++ b/dbms/src/Server/HTTPHandler.cpp @@ -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 diff --git a/dbms/src/Server/LocalServer.cpp b/dbms/src/Server/LocalServer.cpp index 352ad5572f1..abdbbd485c5 100644 --- a/dbms/src/Server/LocalServer.cpp +++ b/dbms/src/Server/LocalServer.cpp @@ -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); } } diff --git a/dbms/tests/queries/0_stateless/00415_into_outfile.reference b/dbms/tests/queries/0_stateless/00415_into_outfile.reference new file mode 100644 index 00000000000..0aa3993f4c0 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00415_into_outfile.reference @@ -0,0 +1,13 @@ +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 diff --git a/dbms/tests/queries/0_stateless/00415_into_outfile.sh b/dbms/tests/queries/0_stateless/00415_into_outfile.sh new file mode 100755 index 00000000000..6ccd18afb29 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00415_into_outfile.sh @@ -0,0 +1,36 @@ +#!/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"