mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
parse INTO OUTFILE clause [#CLICKHOUSE-2133]
This commit is contained in:
parent
22797da2a6
commit
5de1975bc4
@ -13,6 +13,7 @@ namespace DB
|
|||||||
void executeQuery(
|
void executeQuery(
|
||||||
ReadBuffer & istr, /// Откуда читать запрос (а также данные для INSERT-а, если есть)
|
ReadBuffer & istr, /// Откуда читать запрос (а также данные для INSERT-а, если есть)
|
||||||
WriteBuffer & ostr, /// Куда писать результат
|
WriteBuffer & ostr, /// Куда писать результат
|
||||||
|
bool allow_into_outfile, /// If true and the query contains INTO OUTFILE section, output will be redirected to that file.
|
||||||
Context & context, /// БД, таблицы, типы данных, движки таблиц, функции, агрегатные функции...
|
Context & context, /// БД, таблицы, типы данных, движки таблиц, функции, агрегатные функции...
|
||||||
BlockInputStreamPtr & query_plan, /// Сюда может быть записано описание, как выполнялся запрос
|
BlockInputStreamPtr & query_plan, /// Сюда может быть записано описание, как выполнялся запрос
|
||||||
std::function<void(const String &)> set_content_type /// Может быть передан колбэк, с помощью которого может быть сообщён Content-Type формата.
|
std::function<void(const String &)> set_content_type /// Может быть передан колбэк, с помощью которого может быть сообщён Content-Type формата.
|
||||||
|
@ -14,14 +14,17 @@ struct ASTCheckQuery : public ASTQueryWithOutput
|
|||||||
|
|
||||||
ASTPtr clone() const override
|
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 database;
|
||||||
std::string table;
|
std::string table;
|
||||||
|
|
||||||
protected:
|
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";
|
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 << (settings.hilite ? hilite_keyword : "") << indent_str << backQuoteIfNeed(table) << (settings.hilite ? hilite_none : "");
|
||||||
}
|
}
|
||||||
settings.ostr << nl_or_ws;
|
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -6,20 +6,25 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
/** Запрос с секцией FORMAT.
|
/** Query with output options (supporting [INTO OUTFILE 'file_name'] [FORMAT format_name] suffix).
|
||||||
*/
|
*/
|
||||||
class ASTQueryWithOutput : public IAST
|
class ASTQueryWithOutput : public IAST
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
ASTPtr out_file;
|
||||||
ASTPtr format;
|
ASTPtr format;
|
||||||
|
|
||||||
ASTQueryWithOutput() = default;
|
ASTQueryWithOutput() = default;
|
||||||
ASTQueryWithOutput(const StringRange range_) : IAST(range_) {}
|
ASTQueryWithOutput(const StringRange range_) : IAST(range_) {}
|
||||||
|
|
||||||
/** Возвращает указатель на формат. Если типом объекта является ASTSelectQuery,
|
protected:
|
||||||
* то эта функция возвращает указатель на формат из последнего SELECT'а цепочки UNION ALL.
|
/// NOTE: call this helper at the end of the clone() method of descendant class.
|
||||||
*/
|
void cloneOutputOptions(ASTQueryWithOutput& cloned) const;
|
||||||
virtual const IAST * getFormat() const { return format.get(); }
|
|
||||||
|
/// 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); \
|
std::shared_ptr<Name> res = std::make_shared<Name>(*this); \
|
||||||
res->children.clear(); \
|
res->children.clear(); \
|
||||||
if (format) \
|
cloneOutputOptions(*res); \
|
||||||
{ \
|
|
||||||
res->format = format->clone(); \
|
|
||||||
res->children.push_back(res->format); \
|
|
||||||
} \
|
|
||||||
return res; \
|
return res; \
|
||||||
} \
|
} \
|
||||||
\
|
\
|
||||||
protected: \
|
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 : ""); \
|
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 : "")
|
settings.ostr << (settings.hilite ? hilite_keyword : "") << name << " " << (settings.hilite ? hilite_none : "")
|
||||||
<< (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table);
|
<< (!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); \
|
std::shared_ptr<Name> res = std::make_shared<Name>(*this); \
|
||||||
res->children.clear(); \
|
res->children.clear(); \
|
||||||
if (format) \
|
cloneOutputOptions(*res); \
|
||||||
{ \
|
|
||||||
res->format = format->clone(); \
|
|
||||||
res->children.push_back(res->format); \
|
|
||||||
} \
|
|
||||||
return res; \
|
return res; \
|
||||||
} \
|
} \
|
||||||
\
|
\
|
||||||
protected: \
|
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); \
|
formatHelper(settings, state, frame, Query); \
|
||||||
} \
|
} \
|
||||||
|
@ -41,11 +41,8 @@ public:
|
|||||||
/// Получить глубокую копию дерева первого запроса SELECT.
|
/// Получить глубокую копию дерева первого запроса SELECT.
|
||||||
ASTPtr cloneFirstSelect() const;
|
ASTPtr cloneFirstSelect() const;
|
||||||
|
|
||||||
/// Возвращает указатель на формат из последнего SELECT'а цепочки UNION ALL.
|
|
||||||
const IAST * getFormat() const override;
|
|
||||||
|
|
||||||
private:
|
private:
|
||||||
ASTPtr cloneImpl(bool traverse_union_all) const;
|
std::shared_ptr<ASTSelectQuery> cloneImpl(bool traverse_union_all) const;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
bool distinct = false;
|
bool distinct = false;
|
||||||
@ -85,7 +82,7 @@ public:
|
|||||||
IAST * prev_union_all = nullptr;
|
IAST * prev_union_all = nullptr;
|
||||||
|
|
||||||
protected:
|
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);
|
auto res = std::make_shared<ASTShowTablesQuery>(*this);
|
||||||
res->children.clear();
|
res->children.clear();
|
||||||
|
cloneOutputOptions(*res);
|
||||||
if (format)
|
|
||||||
{
|
|
||||||
res->format = format->clone();
|
|
||||||
res->children.push_back(res->format);
|
|
||||||
}
|
|
||||||
|
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
protected:
|
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)
|
if (databases)
|
||||||
{
|
{
|
||||||
@ -58,13 +52,6 @@ protected:
|
|||||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " LIKE " << (settings.hilite ? hilite_none : "")
|
settings.ostr << (settings.hilite ? hilite_keyword : "") << " LIKE " << (settings.hilite ? hilite_none : "")
|
||||||
<< mysqlxx::quote << like;
|
<< 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
|
* CHECK [TABLE] [database.]table
|
||||||
*/
|
*/
|
||||||
class ParserCheckQuery : public ParserQueryWithOutput
|
class ParserCheckQuery : public IParserBase
|
||||||
{
|
{
|
||||||
protected:
|
protected:
|
||||||
const char * getName() const { return "ALTER query"; }
|
const char * getName() const { return "ALTER query"; }
|
||||||
|
@ -7,12 +7,13 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
/** Парсер для запросов поддерживающих секцию FORMAT.
|
/// Parse queries supporting [INTO OUTFILE 'file_name'] [FORMAT format_name] suffix.
|
||||||
*/
|
|
||||||
class ParserQueryWithOutput : public IParserBase
|
class ParserQueryWithOutput : public IParserBase
|
||||||
{
|
{
|
||||||
protected:
|
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:
|
protected:
|
||||||
ParserWhiteSpaceOrComments ws;
|
ParserWhiteSpaceOrComments ws;
|
||||||
|
@ -7,7 +7,7 @@ namespace DB
|
|||||||
{
|
{
|
||||||
|
|
||||||
|
|
||||||
class ParserSelectQuery : public ParserQueryWithOutput
|
class ParserSelectQuery : public IParserBase
|
||||||
{
|
{
|
||||||
protected:
|
protected:
|
||||||
const char * getName() const override { return "SELECT query"; }
|
const char * getName() const override { return "SELECT query"; }
|
||||||
|
@ -12,7 +12,7 @@ namespace DB
|
|||||||
|
|
||||||
/** Запрос SHOW PROCESSLIST
|
/** Запрос SHOW PROCESSLIST
|
||||||
*/
|
*/
|
||||||
class ParserShowProcesslistQuery : public ParserQueryWithOutput
|
class ParserShowProcesslistQuery : public IParserBase
|
||||||
{
|
{
|
||||||
protected:
|
protected:
|
||||||
const char * getName() const { return "SHOW PROCESSLIST query"; }
|
const char * getName() const { return "SHOW PROCESSLIST query"; }
|
||||||
@ -21,6 +21,7 @@ protected:
|
|||||||
{
|
{
|
||||||
Pos begin = pos;
|
Pos begin = pos;
|
||||||
|
|
||||||
|
ParserWhiteSpaceOrComments ws;
|
||||||
ParserString s_show("SHOW", true, true);
|
ParserString s_show("SHOW", true, true);
|
||||||
ParserString s_processlist("PROCESSLIST", true, true);
|
ParserString s_processlist("PROCESSLIST", true, true);
|
||||||
|
|
||||||
@ -38,10 +39,6 @@ protected:
|
|||||||
|
|
||||||
ws.ignore(pos, end);
|
ws.ignore(pos, end);
|
||||||
|
|
||||||
/// FORMAT format_name
|
|
||||||
if (!parseFormat(*query, pos, end, node, max_parsed_pos, expected))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
query->range = StringRange(begin, pos);
|
query->range = StringRange(begin, pos);
|
||||||
node = query;
|
node = query;
|
||||||
|
|
||||||
|
@ -11,7 +11,7 @@ namespace DB
|
|||||||
* или
|
* или
|
||||||
* SHOW DATABASES.
|
* SHOW DATABASES.
|
||||||
*/
|
*/
|
||||||
class ParserShowTablesQuery : public ParserQueryWithOutput
|
class ParserShowTablesQuery : public IParserBase
|
||||||
{
|
{
|
||||||
protected:
|
protected:
|
||||||
const char * getName() const { return "SHOW TABLES|DATABASES query"; }
|
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]
|
/** Запрос (EXISTS | SHOW CREATE | (DESCRIBE | DESC) ) [TABLE] [db.]name [FORMAT format]
|
||||||
*/
|
*/
|
||||||
class ParserTablePropertiesQuery : public ParserQueryWithOutput
|
class ParserTablePropertiesQuery : public IParserBase
|
||||||
{
|
{
|
||||||
protected:
|
protected:
|
||||||
const char * getName() const { return "EXISTS, SHOW CREATE or DESCRIBE query"; }
|
const char * getName() const { return "EXISTS, SHOW CREATE or DESCRIBE query"; }
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
#include <iostream>
|
#include <iostream>
|
||||||
#include <fstream>
|
#include <fstream>
|
||||||
#include <iomanip>
|
#include <iomanip>
|
||||||
|
#include <experimental/optional>
|
||||||
|
|
||||||
#include <unordered_set>
|
#include <unordered_set>
|
||||||
#include <algorithm>
|
#include <algorithm>
|
||||||
@ -25,6 +26,7 @@
|
|||||||
|
|
||||||
#include <DB/IO/ReadBufferFromFileDescriptor.h>
|
#include <DB/IO/ReadBufferFromFileDescriptor.h>
|
||||||
#include <DB/IO/WriteBufferFromFileDescriptor.h>
|
#include <DB/IO/WriteBufferFromFileDescriptor.h>
|
||||||
|
#include <DB/IO/WriteBufferFromFile.h>
|
||||||
#include <DB/IO/WriteBufferFromString.h>
|
#include <DB/IO/WriteBufferFromString.h>
|
||||||
#include <DB/IO/ReadBufferFromMemory.h>
|
#include <DB/IO/ReadBufferFromMemory.h>
|
||||||
#include <DB/IO/ReadHelpers.h>
|
#include <DB/IO/ReadHelpers.h>
|
||||||
@ -40,6 +42,7 @@
|
|||||||
#include <DB/Parsers/ASTInsertQuery.h>
|
#include <DB/Parsers/ASTInsertQuery.h>
|
||||||
#include <DB/Parsers/ASTSelectQuery.h>
|
#include <DB/Parsers/ASTSelectQuery.h>
|
||||||
#include <DB/Parsers/ASTQueryWithOutput.h>
|
#include <DB/Parsers/ASTQueryWithOutput.h>
|
||||||
|
#include <DB/Parsers/ASTLiteral.h>
|
||||||
#include <DB/Parsers/ASTIdentifier.h>
|
#include <DB/Parsers/ASTIdentifier.h>
|
||||||
#include <DB/Parsers/formatAST.h>
|
#include <DB/Parsers/formatAST.h>
|
||||||
#include <DB/Parsers/parseQuery.h>
|
#include <DB/Parsers/parseQuery.h>
|
||||||
@ -140,6 +143,7 @@ private:
|
|||||||
String query; /// Текущий запрос.
|
String query; /// Текущий запрос.
|
||||||
|
|
||||||
String format; /// Формат вывода результата в консоль.
|
String format; /// Формат вывода результата в консоль.
|
||||||
|
bool is_default_format = true; /// false, если взяли формат из конфига или командной строки.
|
||||||
size_t format_max_block_size = 0; /// Максимальный размер блока при выводе в консоль.
|
size_t format_max_block_size = 0; /// Максимальный размер блока при выводе в консоль.
|
||||||
String insert_format; /// Формат данных для INSERT-а при чтении их из stdin в batch режиме
|
String insert_format; /// Формат данных для INSERT-а при чтении их из stdin в batch режиме
|
||||||
size_t insert_format_max_block_size = 0; /// Максимальный размер блока при чтении данных INSERT-а.
|
size_t insert_format_max_block_size = 0; /// Максимальный размер блока при чтении данных INSERT-а.
|
||||||
@ -153,7 +157,9 @@ private:
|
|||||||
|
|
||||||
/// Вывод в консоль
|
/// Вывод в консоль
|
||||||
WriteBufferFromFileDescriptor std_out {STDOUT_FILENO};
|
WriteBufferFromFileDescriptor std_out {STDOUT_FILENO};
|
||||||
BlockOutputStreamPtr block_std_out;
|
/// Клиент может попросить вывести результат в файл.
|
||||||
|
std::experimental::optional<WriteBufferFromFile> out_file_buf;
|
||||||
|
BlockOutputStreamPtr block_out_stream;
|
||||||
|
|
||||||
String home_path;
|
String home_path;
|
||||||
|
|
||||||
@ -307,6 +313,7 @@ private:
|
|||||||
if (is_interactive)
|
if (is_interactive)
|
||||||
showClientVersion();
|
showClientVersion();
|
||||||
|
|
||||||
|
is_default_format = !config().has("vertical") && !config().has("format");
|
||||||
if (config().has("vertical"))
|
if (config().has("vertical"))
|
||||||
format = config().getString("format", "Vertical");
|
format = config().getString("format", "Vertical");
|
||||||
else
|
else
|
||||||
@ -842,7 +849,12 @@ private:
|
|||||||
/** Сбросить все данные, что ещё остались в буферах. */
|
/** Сбросить все данные, что ещё остались в буферах. */
|
||||||
void resetOutput()
|
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();
|
std_out.next();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -957,27 +969,39 @@ private:
|
|||||||
|
|
||||||
void initBlockOutputStream(const Block & block)
|
void initBlockOutputStream(const Block & block)
|
||||||
{
|
{
|
||||||
if (!block_std_out)
|
if (!block_out_stream)
|
||||||
{
|
{
|
||||||
|
WriteBuffer * out_buf = &std_out;
|
||||||
String current_format = format;
|
String current_format = format;
|
||||||
|
|
||||||
/// Формат может быть указан в запросе.
|
/// Формат может быть указан в запросе.
|
||||||
if (ASTQueryWithOutput * query_with_output = dynamic_cast<ASTQueryWithOutput *>(&*parsed_query))
|
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)
|
if (has_vertical_output_suffix)
|
||||||
throw Exception("Output format already specified", ErrorCodes::CLIENT_OUTPUT_FORMAT_SPECIFIED);
|
throw Exception("Output format already specified", ErrorCodes::CLIENT_OUTPUT_FORMAT_SPECIFIED);
|
||||||
if (const ASTIdentifier * id = typeid_cast<const ASTIdentifier *>(query_with_output->getFormat()))
|
const auto & id = typeid_cast<const ASTIdentifier &>(*query_with_output->format);
|
||||||
current_format = id->name;
|
current_format = id.name;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (has_vertical_output_suffix)
|
if (has_vertical_output_suffix)
|
||||||
current_format = "Vertical";
|
current_format = "Vertical";
|
||||||
|
|
||||||
block_std_out = context.getOutputFormat(current_format, std_out, block);
|
block_out_stream = context.getOutputFormat(current_format, *out_buf, block);
|
||||||
block_std_out->writePrefix();
|
block_out_stream->writePrefix();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -993,36 +1017,36 @@ private:
|
|||||||
processed_rows += block.rows();
|
processed_rows += block.rows();
|
||||||
initBlockOutputStream(block);
|
initBlockOutputStream(block);
|
||||||
|
|
||||||
/// Заголовочный блок с нулем строк использовался для инициализации block_std_out,
|
/// Заголовочный блок с нулем строк использовался для инициализации block_out_stream,
|
||||||
/// выводить его не нужно
|
/// выводить его не нужно
|
||||||
if (block.rows() != 0)
|
if (block.rows() != 0)
|
||||||
{
|
{
|
||||||
block_std_out->write(block);
|
block_out_stream->write(block);
|
||||||
written_first_block = true;
|
written_first_block = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Полученный блок данных сразу выводится клиенту.
|
/// Полученный блок данных сразу выводится клиенту.
|
||||||
block_std_out->flush();
|
block_out_stream->flush();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void onTotals(Block & block)
|
void onTotals(Block & block)
|
||||||
{
|
{
|
||||||
initBlockOutputStream(block);
|
initBlockOutputStream(block);
|
||||||
block_std_out->setTotals(block);
|
block_out_stream->setTotals(block);
|
||||||
}
|
}
|
||||||
|
|
||||||
void onExtremes(Block & block)
|
void onExtremes(Block & block)
|
||||||
{
|
{
|
||||||
initBlockOutputStream(block);
|
initBlockOutputStream(block);
|
||||||
block_std_out->setExtremes(block);
|
block_out_stream->setExtremes(block);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void onProgress(const Progress & value)
|
void onProgress(const Progress & value)
|
||||||
{
|
{
|
||||||
progress.incrementPiecewiseAtomically(value);
|
progress.incrementPiecewiseAtomically(value);
|
||||||
block_std_out->onProgress(value);
|
block_out_stream->onProgress(value);
|
||||||
writeProgress();
|
writeProgress();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1139,15 +1163,15 @@ private:
|
|||||||
|
|
||||||
void onProfileInfo(const BlockStreamProfileInfo & profile_info)
|
void onProfileInfo(const BlockStreamProfileInfo & profile_info)
|
||||||
{
|
{
|
||||||
if (profile_info.hasAppliedLimit() && block_std_out)
|
if (profile_info.hasAppliedLimit() && block_out_stream)
|
||||||
block_std_out->setRowsBeforeLimit(profile_info.getRowsBeforeLimit());
|
block_out_stream->setRowsBeforeLimit(profile_info.getRowsBeforeLimit());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void onEndOfStream()
|
void onEndOfStream()
|
||||||
{
|
{
|
||||||
if (block_std_out)
|
if (block_out_stream)
|
||||||
block_std_out->writeSuffix();
|
block_out_stream->writeSuffix();
|
||||||
|
|
||||||
resetOutput();
|
resetOutput();
|
||||||
|
|
||||||
|
@ -360,6 +360,7 @@ namespace ErrorCodes
|
|||||||
extern const int ZLIB_DEFLATE_FAILED = 355;
|
extern const int ZLIB_DEFLATE_FAILED = 355;
|
||||||
extern const int BAD_LAMBDA = 356;
|
extern const int BAD_LAMBDA = 356;
|
||||||
extern const int RESERVED_IDENTIFIER_NAME = 357;
|
extern const int RESERVED_IDENTIFIER_NAME = 357;
|
||||||
|
extern const int INTO_OUTFILE_NOT_ALLOWED = 358;
|
||||||
|
|
||||||
extern const int KEEPER_EXCEPTION = 999;
|
extern const int KEEPER_EXCEPTION = 999;
|
||||||
extern const int POCO_EXCEPTION = 1000;
|
extern const int POCO_EXCEPTION = 1000;
|
||||||
|
@ -19,15 +19,7 @@ BlockIO InterpreterShowProcesslistQuery::execute()
|
|||||||
|
|
||||||
String InterpreterShowProcesslistQuery::getRewrittenQuery()
|
String InterpreterShowProcesslistQuery::getRewrittenQuery()
|
||||||
{
|
{
|
||||||
const ASTQueryWithOutput & query = dynamic_cast<const ASTQueryWithOutput &>(*query_ptr);
|
return "SELECT * FROM system.processes";
|
||||||
|
|
||||||
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();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -23,13 +23,9 @@ String InterpreterShowTablesQuery::getRewrittenQuery()
|
|||||||
{
|
{
|
||||||
const ASTShowTablesQuery & query = typeid_cast<const ASTShowTablesQuery &>(*query_ptr);
|
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
|
/// SHOW DATABASES
|
||||||
if (query.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;
|
String database = query.from.empty() ? context.getCurrentDatabase() : query.from;
|
||||||
|
|
||||||
@ -45,8 +41,6 @@ String InterpreterShowTablesQuery::getRewrittenQuery()
|
|||||||
if (!query.like.empty())
|
if (!query.like.empty())
|
||||||
rewritten_query << " AND name " << (query.not_like ? "NOT " : "") << "LIKE " << mysqlxx::quote << query.like;
|
rewritten_query << " AND name " << (query.not_like ? "NOT " : "") << "LIKE " << mysqlxx::quote << query.like;
|
||||||
|
|
||||||
rewritten_query << format_or_nothing;
|
|
||||||
|
|
||||||
return rewritten_query.str();
|
return rewritten_query.str();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -2,6 +2,7 @@
|
|||||||
#include <DB/Common/formatReadable.h>
|
#include <DB/Common/formatReadable.h>
|
||||||
|
|
||||||
#include <DB/IO/ConcatReadBuffer.h>
|
#include <DB/IO/ConcatReadBuffer.h>
|
||||||
|
#include <DB/IO/WriteBufferFromFile.h>
|
||||||
|
|
||||||
#include <DB/DataStreams/BlockIO.h>
|
#include <DB/DataStreams/BlockIO.h>
|
||||||
#include <DB/DataStreams/copyData.h>
|
#include <DB/DataStreams/copyData.h>
|
||||||
@ -12,6 +13,7 @@
|
|||||||
#include <DB/Parsers/ASTInsertQuery.h>
|
#include <DB/Parsers/ASTInsertQuery.h>
|
||||||
#include <DB/Parsers/ASTShowProcesslistQuery.h>
|
#include <DB/Parsers/ASTShowProcesslistQuery.h>
|
||||||
#include <DB/Parsers/ASTIdentifier.h>
|
#include <DB/Parsers/ASTIdentifier.h>
|
||||||
|
#include <DB/Parsers/ASTLiteral.h>
|
||||||
#include <DB/Parsers/ParserQuery.h>
|
#include <DB/Parsers/ParserQuery.h>
|
||||||
#include <DB/Parsers/parseQuery.h>
|
#include <DB/Parsers/parseQuery.h>
|
||||||
|
|
||||||
@ -34,6 +36,7 @@ namespace ErrorCodes
|
|||||||
{
|
{
|
||||||
extern const int LOGICAL_ERROR;
|
extern const int LOGICAL_ERROR;
|
||||||
extern const int QUERY_IS_TOO_LARGE;
|
extern const int QUERY_IS_TOO_LARGE;
|
||||||
|
extern const int INTO_OUTFILE_NOT_ALLOWED;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -353,6 +356,7 @@ BlockIO executeQuery(
|
|||||||
void executeQuery(
|
void executeQuery(
|
||||||
ReadBuffer & istr,
|
ReadBuffer & istr,
|
||||||
WriteBuffer & ostr,
|
WriteBuffer & ostr,
|
||||||
|
bool allow_into_outfile,
|
||||||
Context & context,
|
Context & context,
|
||||||
BlockInputStreamPtr & query_plan,
|
BlockInputStreamPtr & query_plan,
|
||||||
std::function<void(const String &)> set_content_type)
|
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());
|
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)
|
WriteBuffer * out_buf = &ostr;
|
||||||
? typeid_cast<const ASTIdentifier &>(*ast_query_with_output->getFormat()).name
|
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();
|
: 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()))
|
if (auto stream = dynamic_cast<IProfilingBlockInputStream *>(streams.in.get()))
|
||||||
{
|
{
|
||||||
|
@ -45,7 +45,7 @@ try
|
|||||||
WriteBufferFromOStream out(std::cout);
|
WriteBufferFromOStream out(std::cout);
|
||||||
BlockInputStreamPtr query_plan;
|
BlockInputStreamPtr query_plan;
|
||||||
|
|
||||||
executeQuery(in, out, context, query_plan, {});
|
executeQuery(in, out, /* allow_into_outfile = */ false, context, query_plan, {});
|
||||||
|
|
||||||
if (query_plan)
|
if (query_plan)
|
||||||
{
|
{
|
||||||
|
39
dbms/src/Parsers/ASTQueryWithOutput.cpp
Normal file
39
dbms/src/Parsers/ASTQueryWithOutput.cpp
Normal file
@ -0,0 +1,39 @@
|
|||||||
|
#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 ASTSelectQuery::clone() const
|
||||||
{
|
{
|
||||||
ASTPtr ptr = cloneImpl(true);
|
auto ptr = cloneImpl(true);
|
||||||
|
|
||||||
/// Установить указатели на предыдущие запросы SELECT.
|
/// Установить указатели на предыдущие запросы SELECT.
|
||||||
ASTPtr current = ptr;
|
ASTPtr current = ptr;
|
||||||
static_cast<ASTSelectQuery *>(&*current)->prev_union_all = nullptr;
|
static_cast<ASTSelectQuery *>(current.get())->prev_union_all = nullptr;
|
||||||
ASTPtr next = static_cast<ASTSelectQuery *>(&*current)->next_union_all;
|
ASTPtr next = static_cast<ASTSelectQuery *>(current.get())->next_union_all;
|
||||||
while (next != nullptr)
|
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();
|
next_select_query->prev_union_all = current.get();
|
||||||
current = next;
|
current = next;
|
||||||
next = next_select_query->next_union_all;
|
next = next_select_query->next_union_all;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
cloneOutputOptions(*ptr);
|
||||||
|
|
||||||
return ptr;
|
return ptr;
|
||||||
}
|
}
|
||||||
|
|
||||||
ASTPtr ASTSelectQuery::cloneFirstSelect() const
|
ASTPtr ASTSelectQuery::cloneFirstSelect() const
|
||||||
{
|
{
|
||||||
ASTPtr res = cloneImpl(false);
|
auto res = cloneImpl(false);
|
||||||
static_cast<ASTSelectQuery *>(&*res)->prev_union_all = nullptr;
|
res->prev_union_all = nullptr;
|
||||||
return res;
|
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);
|
auto res = std::make_shared<ASTSelectQuery>(*this);
|
||||||
res->children.clear();
|
res->children.clear();
|
||||||
@ -207,7 +209,6 @@ ASTPtr ASTSelectQuery::cloneImpl(bool traverse_union_all) const
|
|||||||
CLONE(limit_offset)
|
CLONE(limit_offset)
|
||||||
CLONE(limit_length)
|
CLONE(limit_length)
|
||||||
CLONE(settings)
|
CLONE(settings)
|
||||||
CLONE(format)
|
|
||||||
|
|
||||||
#undef CLONE
|
#undef CLONE
|
||||||
|
|
||||||
@ -225,16 +226,7 @@ ASTPtr ASTSelectQuery::cloneImpl(bool traverse_union_all) const
|
|||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
const IAST * ASTSelectQuery::getFormat() const
|
void ASTSelectQuery::formatQueryImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) 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
|
|
||||||
{
|
{
|
||||||
frame.current_select = this;
|
frame.current_select = this;
|
||||||
frame.need_parens = false;
|
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)
|
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 : "");
|
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",
|
"SETTINGS",
|
||||||
"FORMAT",
|
"FORMAT",
|
||||||
"UNION",
|
"UNION",
|
||||||
|
"INTO",
|
||||||
nullptr
|
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)
|
if (new_max_parsed_pos > max_parsed_pos)
|
||||||
max_parsed_pos = new_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;
|
expected = new_expected;
|
||||||
|
|
||||||
if (pos > end)
|
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)
|
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_check("CHECK", true, true);
|
||||||
ParserString s_table("TABLE", true, true);
|
ParserString s_table("TABLE", true, true);
|
||||||
ParserString s_dot(".");
|
ParserString s_dot(".");
|
||||||
@ -51,10 +52,6 @@ bool ParserCheckQuery::parseImpl(IParser::Pos & pos, IParser::Pos end, ASTPtr &
|
|||||||
|
|
||||||
ws.ignore(pos, end);
|
ws.ignore(pos, end);
|
||||||
|
|
||||||
/// FORMAT format_name
|
|
||||||
if (!parseFormat(*query, pos, end, node, max_parsed_pos, expected))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
node = query;
|
node = query;
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
@ -1,18 +1,13 @@
|
|||||||
#include <DB/Parsers/ParserSelectQuery.h>
|
#include <DB/Parsers/ParserQuery.h>
|
||||||
|
#include <DB/Parsers/ParserQueryWithOutput.h>
|
||||||
#include <DB/Parsers/ParserCreateQuery.h>
|
#include <DB/Parsers/ParserCreateQuery.h>
|
||||||
#include <DB/Parsers/ParserInsertQuery.h>
|
#include <DB/Parsers/ParserInsertQuery.h>
|
||||||
#include <DB/Parsers/ParserDropQuery.h>
|
#include <DB/Parsers/ParserDropQuery.h>
|
||||||
#include <DB/Parsers/ParserRenameQuery.h>
|
#include <DB/Parsers/ParserRenameQuery.h>
|
||||||
#include <DB/Parsers/ParserShowTablesQuery.h>
|
|
||||||
#include <DB/Parsers/ParserOptimizeQuery.h>
|
#include <DB/Parsers/ParserOptimizeQuery.h>
|
||||||
#include <DB/Parsers/TablePropertiesQueriesASTs.h>
|
|
||||||
#include <DB/Parsers/ParserUseQuery.h>
|
#include <DB/Parsers/ParserUseQuery.h>
|
||||||
#include <DB/Parsers/ParserSetQuery.h>
|
#include <DB/Parsers/ParserSetQuery.h>
|
||||||
#include <DB/Parsers/ParserQuery.h>
|
|
||||||
#include <DB/Parsers/ParserTablePropertiesQuery.h>
|
|
||||||
#include <DB/Parsers/ParserAlterQuery.h>
|
#include <DB/Parsers/ParserAlterQuery.h>
|
||||||
#include <DB/Parsers/ParserShowProcesslistQuery.h>
|
|
||||||
#include <DB/Parsers/ParserCheckQuery.h>
|
|
||||||
//#include <DB/Parsers/ParserMultiQuery.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)
|
bool ParserQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
|
||||||
{
|
{
|
||||||
ParserShowTablesQuery show_tables_p;
|
ParserQueryWithOutput query_with_output_p;
|
||||||
ParserSelectQuery select_p;
|
|
||||||
ParserInsertQuery insert_p;
|
ParserInsertQuery insert_p;
|
||||||
ParserCreateQuery create_p;
|
ParserCreateQuery create_p;
|
||||||
ParserRenameQuery rename_p;
|
ParserRenameQuery rename_p;
|
||||||
@ -32,13 +26,9 @@ bool ParserQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_
|
|||||||
ParserUseQuery use_p;
|
ParserUseQuery use_p;
|
||||||
ParserSetQuery set_p;
|
ParserSetQuery set_p;
|
||||||
ParserOptimizeQuery optimize_p;
|
ParserOptimizeQuery optimize_p;
|
||||||
ParserTablePropertiesQuery table_p;
|
|
||||||
ParserShowProcesslistQuery show_processlist_p;
|
|
||||||
ParserCheckQuery check_p;
|
|
||||||
// ParserMultiQuery multi_p;
|
// ParserMultiQuery multi_p;
|
||||||
|
|
||||||
bool res = show_tables_p.parse(pos, end, node, max_parsed_pos, expected)
|
bool res = query_with_output_p.parse(pos, end, node, max_parsed_pos, expected)
|
||||||
|| select_p.parse(pos, end, node, max_parsed_pos, expected)
|
|
||||||
|| insert_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)
|
|| create_p.parse(pos, end, node, max_parsed_pos, expected)
|
||||||
|| rename_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)
|
|| alter_p.parse(pos, end, node, max_parsed_pos, expected)
|
||||||
|| use_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)
|
|| set_p.parse(pos, end, node, max_parsed_pos, expected)
|
||||||
|| optimize_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);
|
|
||||||
/* || multi_p.parse(pos, end, node, max_parsed_pos, expected)*/;
|
/* || multi_p.parse(pos, end, node, max_parsed_pos, expected)*/;
|
||||||
|
|
||||||
if (!res && (!expected || !*expected))
|
if (!res && (!expected || !*expected))
|
||||||
|
@ -1,4 +1,9 @@
|
|||||||
#include <DB/Parsers/ParserQueryWithOutput.h>
|
#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/ASTIdentifier.h>
|
||||||
#include <DB/Parsers/ExpressionElementParsers.h>
|
#include <DB/Parsers/ExpressionElementParsers.h>
|
||||||
#include <DB/Common/typeid_cast.h>
|
#include <DB/Common/typeid_cast.h>
|
||||||
@ -7,8 +12,50 @@
|
|||||||
namespace DB
|
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);
|
ParserString s_format("FORMAT", true, true);
|
||||||
|
|
||||||
if (s_format.ignore(pos, end, max_parsed_pos, expected))
|
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;
|
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;
|
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);
|
ws.ignore(pos, end);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
node = query;
|
||||||
return true;
|
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>();
|
auto select_query = std::make_shared<ASTSelectQuery>();
|
||||||
node = select_query;
|
node = select_query;
|
||||||
|
|
||||||
|
ParserWhiteSpaceOrComments ws;
|
||||||
ParserString s_select("SELECT", true, true);
|
ParserString s_select("SELECT", true, true);
|
||||||
ParserString s_distinct("DISTINCT", true, true);
|
ParserString s_distinct("DISTINCT", true, true);
|
||||||
ParserString s_from("FROM", 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);
|
ws.ignore(pos, end);
|
||||||
}
|
}
|
||||||
|
|
||||||
/// FORMAT format_name
|
|
||||||
if (!parseFormat(*select_query, pos, end, node, max_parsed_pos, expected))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
// UNION ALL select query
|
// UNION ALL select query
|
||||||
if (s_union.ignore(pos, end, max_parsed_pos, expected))
|
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 (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;
|
ParserSelectQuery select_p;
|
||||||
if (!select_p.parse(pos, end, select_query->next_union_all, max_parsed_pos, expected))
|
if (!select_p.parse(pos, end, select_query->next_union_all, max_parsed_pos, expected))
|
||||||
return false;
|
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);
|
select_query->children.push_back(select_query->limit_length);
|
||||||
if (select_query->settings)
|
if (select_query->settings)
|
||||||
select_query->children.push_back(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)
|
if (select_query->next_union_all)
|
||||||
select_query->children.push_back(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;
|
Pos begin = pos;
|
||||||
|
|
||||||
|
ParserWhiteSpaceOrComments ws;
|
||||||
ParserString s_show("SHOW", true, true);
|
ParserString s_show("SHOW", true, true);
|
||||||
ParserString s_tables("TABLES", true, true);
|
ParserString s_tables("TABLES", true, true);
|
||||||
ParserString s_databases("DATABASES", 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);
|
ws.ignore(pos, end);
|
||||||
|
|
||||||
/// FORMAT format_name
|
|
||||||
if (!parseFormat(*query, pos, end, node, max_parsed_pos, expected))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
query->range = StringRange(begin, pos);
|
query->range = StringRange(begin, pos);
|
||||||
|
|
||||||
if (database)
|
if (database)
|
||||||
query->from = typeid_cast<ASTIdentifier &>(*database).name;
|
query->from = typeid_cast<ASTIdentifier &>(*database).name;
|
||||||
if (like)
|
if (like)
|
||||||
query->like = safeGet<const String &>(typeid_cast<ASTLiteral &>(*like).value);
|
query->like = safeGet<const String &>(typeid_cast<ASTLiteral &>(*like).value);
|
||||||
if (query->format)
|
|
||||||
query->children.push_back(query->format);
|
|
||||||
|
|
||||||
node = query;
|
node = query;
|
||||||
|
|
||||||
|
@ -15,6 +15,7 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Po
|
|||||||
{
|
{
|
||||||
Pos begin = pos;
|
Pos begin = pos;
|
||||||
|
|
||||||
|
ParserWhiteSpaceOrComments ws;
|
||||||
ParserString s_exists("EXISTS", true, true);
|
ParserString s_exists("EXISTS", true, true);
|
||||||
ParserString s_describe("DESCRIBE", true, true);
|
ParserString s_describe("DESCRIBE", true, true);
|
||||||
ParserString s_desc("DESC", 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 database;
|
||||||
ASTPtr table;
|
ASTPtr table;
|
||||||
ASTPtr query_ptr;
|
std::shared_ptr<ASTQueryWithTableAndOutput> query;
|
||||||
|
|
||||||
ws.ignore(pos, end);
|
ws.ignore(pos, end);
|
||||||
|
|
||||||
if (s_exists.ignore(pos, end, max_parsed_pos, expected))
|
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))
|
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))
|
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))
|
if (!s_create.ignore(pos, end, max_parsed_pos, expected))
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
query_ptr = std::make_shared<ASTShowCreateQuery>();
|
query = std::make_shared<ASTShowCreateQuery>();
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
ASTQueryWithTableAndOutput * query = dynamic_cast<ASTQueryWithTableAndOutput *>(&*query_ptr);
|
|
||||||
|
|
||||||
ws.ignore(pos, end);
|
ws.ignore(pos, end);
|
||||||
|
|
||||||
s_table.ignore(pos, end, max_parsed_pos, expected);
|
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);
|
ws.ignore(pos, end);
|
||||||
|
|
||||||
/// FORMAT format_name
|
|
||||||
if (!parseFormat(*query, pos, end, node, max_parsed_pos, expected))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
query->range = StringRange(begin, pos);
|
query->range = StringRange(begin, pos);
|
||||||
|
|
||||||
if (database)
|
if (database)
|
||||||
query->database = typeid_cast<ASTIdentifier &>(*database).name;
|
query->database = typeid_cast<ASTIdentifier &>(*database).name;
|
||||||
if (table)
|
if (table)
|
||||||
query->table = typeid_cast<ASTIdentifier &>(*table).name;
|
query->table = typeid_cast<ASTIdentifier &>(*table).name;
|
||||||
if (query->format)
|
|
||||||
query->children.push_back(query->format);
|
|
||||||
|
|
||||||
node = query_ptr;
|
node = query;
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
@ -133,6 +133,8 @@ ASTPtr tryParseQuery(
|
|||||||
/// Parsed query must end with end of data or semicolon.
|
/// Parsed query must end with end of data or semicolon.
|
||||||
if (!parse_res || (pos != end && *pos != ';'))
|
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);
|
out_error_message = getSyntaxErrorMessage(begin, end, max_parsed_pos, expected, hilite, description);
|
||||||
return nullptr;
|
return nullptr;
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,6 @@
|
|||||||
#include <iostream>
|
#include <iostream>
|
||||||
|
|
||||||
#include <DB/Parsers/ParserSelectQuery.h>
|
#include <DB/Parsers/ParserQueryWithOutput.h>
|
||||||
#include <DB/Parsers/parseQuery.h>
|
#include <DB/Parsers/parseQuery.h>
|
||||||
#include <DB/Parsers/formatAST.h>
|
#include <DB/Parsers/formatAST.h>
|
||||||
#include <DB/Parsers/parseQuery.h>
|
#include <DB/Parsers/parseQuery.h>
|
||||||
@ -17,9 +17,11 @@ int main(int argc, char ** argv)
|
|||||||
" GROUP BY UniqID"
|
" GROUP BY UniqID"
|
||||||
" HAVING SUM(Refresh) > 100"
|
" HAVING SUM(Refresh) > 100"
|
||||||
" ORDER BY Visits, PageViews"
|
" 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(), "");
|
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "");
|
||||||
|
|
||||||
std::cout << "Success." << std::endl;
|
std::cout << "Success." << std::endl;
|
||||||
|
@ -257,7 +257,7 @@ void HTTPHandler::processQuery(
|
|||||||
client_info.http_method = http_method;
|
client_info.http_method = http_method;
|
||||||
client_info.http_user_agent = request.get("User-Agent", "");
|
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); });
|
[&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
|
/// 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)
|
if (verbose)
|
||||||
LOG_INFO(log, "Executing query: " << query);
|
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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
13
dbms/tests/queries/0_stateless/00415_into_outfile.reference
Normal file
13
dbms/tests/queries/0_stateless/00415_into_outfile.reference
Normal file
@ -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
|
36
dbms/tests/queries/0_stateless/00415_into_outfile.sh
Executable file
36
dbms/tests/queries/0_stateless/00415_into_outfile.sh
Executable file
@ -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"
|
Loading…
Reference in New Issue
Block a user