mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
use WriteBuffer in formatAST(...)
This commit is contained in:
parent
a06be511df
commit
62ff00ee8b
@ -54,6 +54,7 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/UseSSL.h>
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
#include <DataStreams/AsynchronousBlockInputStream.h>
|
||||
#include <DataStreams/AddingDefaultsBlockInputStream.h>
|
||||
#include <DataStreams/InternalTextLogsRowOutputStream.h>
|
||||
@ -1529,7 +1530,8 @@ private:
|
||||
if (is_interactive)
|
||||
{
|
||||
std::cout << std::endl;
|
||||
formatAST(*res, std::cout);
|
||||
WriteBufferFromOStream res_buf(std::cout, 4096);
|
||||
formatAST(*res, res_buf);
|
||||
std::cout << std::endl << std::endl;
|
||||
}
|
||||
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Core/Types.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/UseSSL.h>
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
@ -419,7 +420,8 @@ void QueryFuzzer::fuzzMain(ASTPtr & ast)
|
||||
fuzz(ast);
|
||||
|
||||
std::cout << std::endl;
|
||||
formatAST(*ast, std::cout, false /*highlight*/);
|
||||
WriteBufferFromOStream ast_buf(std::cout, 4096);
|
||||
formatAST(*ast, ast_buf, false /*highlight*/);
|
||||
std::cout << std::endl << std::endl;
|
||||
}
|
||||
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <IO/ReadBufferFromFileDescriptor.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromFileDescriptor.h>
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
#include <Parsers/ParserQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
@ -129,7 +130,8 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
|
||||
ASTPtr res = parseQueryAndMovePosition(parser, pos, end, "query", multiple, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||
if (!quiet)
|
||||
{
|
||||
formatAST(*res, std::cout, hilite, oneline);
|
||||
WriteBufferFromOStream res_buf(std::cout, 4096);
|
||||
formatAST(*res, res_buf, hilite, oneline);
|
||||
if (multiple)
|
||||
std::cout << "\n;\n";
|
||||
std::cout << std::endl;
|
||||
|
@ -113,16 +113,16 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques
|
||||
|
||||
/// TODO Why not do SQLColumns instead?
|
||||
std::string name = schema_name.empty() ? backQuoteIfNeed(table_name) : backQuoteIfNeed(schema_name) + "." + backQuoteIfNeed(table_name);
|
||||
std::stringstream ss;
|
||||
WriteBufferFromOwnString buf;
|
||||
std::string input = "SELECT * FROM " + name + " WHERE 1 = 0";
|
||||
ParserQueryWithOutput parser;
|
||||
ASTPtr select = parseQuery(parser, input.data(), input.data() + input.size(), "", context_settings.max_query_size, context_settings.max_parser_depth);
|
||||
|
||||
IAST::FormatSettings settings(ss, true);
|
||||
IAST::FormatSettings settings(buf, true);
|
||||
settings.always_quote_identifiers = true;
|
||||
settings.identifier_quoting_style = getQuotingStyle(hdbc);
|
||||
select->format(settings);
|
||||
std::string query = ss.str();
|
||||
std::string query = buf.str();
|
||||
|
||||
LOG_TRACE(log, "Inferring structure with query '{}'", query);
|
||||
|
||||
|
@ -32,12 +32,12 @@ namespace
|
||||
for (const auto & column : columns)
|
||||
query.columns->children.emplace_back(std::make_shared<ASTIdentifier>(column.name));
|
||||
|
||||
std::stringstream ss;
|
||||
IAST::FormatSettings settings(ss, true);
|
||||
WriteBufferFromOwnString buf;
|
||||
IAST::FormatSettings settings(buf, true);
|
||||
settings.always_quote_identifiers = true;
|
||||
settings.identifier_quoting_style = quoting;
|
||||
query.IAST::format(settings);
|
||||
return ss.str();
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
std::string getQuestionMarks(size_t n)
|
||||
|
@ -197,11 +197,13 @@ namespace
|
||||
boost::range::push_back(queries, InterpreterShowGrantsQuery::getAttachGrantQueries(entity));
|
||||
|
||||
/// Serialize the list of ATTACH queries to a string.
|
||||
std::stringstream ss;
|
||||
ss.exceptions(std::ios::failbit);
|
||||
WriteBufferFromOwnString buf;
|
||||
for (const ASTPtr & query : queries)
|
||||
ss << *query << ";\n";
|
||||
String file_contents = std::move(ss).str();
|
||||
{
|
||||
formatAST(*query, buf, false, true);
|
||||
buf.write(";\n", 2);
|
||||
}
|
||||
String file_contents = buf.str();
|
||||
|
||||
/// First we save *.tmp file and then we rename if everything's ok.
|
||||
auto tmp_file_path = std::filesystem::path{file_path}.replace_extension(".tmp");
|
||||
|
@ -94,10 +94,9 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query)
|
||||
|
||||
if (!create)
|
||||
{
|
||||
std::ostringstream query_stream;
|
||||
query_stream.exceptions(std::ios::failbit);
|
||||
formatAST(*query, query_stream, true);
|
||||
throw Exception("Query '" + query_stream.str() + "' is not CREATE query", ErrorCodes::LOGICAL_ERROR);
|
||||
WriteBufferFromOwnString query_buf;
|
||||
formatAST(*query, query_buf, true);
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Query '{}' is not CREATE query", query_buf.str());
|
||||
}
|
||||
|
||||
if (!create->is_dictionary)
|
||||
@ -121,11 +120,10 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query)
|
||||
if (create->uuid != UUIDHelpers::Nil)
|
||||
create->table = TABLE_WITH_UUID_NAME_PLACEHOLDER;
|
||||
|
||||
std::ostringstream statement_stream;
|
||||
statement_stream.exceptions(std::ios::failbit);
|
||||
formatAST(*create, statement_stream, false);
|
||||
statement_stream << '\n';
|
||||
return statement_stream.str();
|
||||
WriteBufferFromOwnString statement_buf;
|
||||
formatAST(*create, statement_buf, false);
|
||||
writeChar('\n', statement_buf);
|
||||
return statement_buf.str();
|
||||
}
|
||||
|
||||
DatabaseOnDisk::DatabaseOnDisk(
|
||||
|
@ -45,6 +45,7 @@ struct BinaryManipReadBuffer : std::reference_wrapper<ReadBuffer> { usin
|
||||
template <typename T> WriteBuffer & operator<< (WriteBuffer & buf, const T & x) { writeText(x, buf); return buf; }
|
||||
/// If you do not use the manipulators, the string is displayed without an escape, as is.
|
||||
template <> inline WriteBuffer & operator<< (WriteBuffer & buf, const String & x) { writeString(x, buf); return buf; }
|
||||
template <> inline WriteBuffer & operator<< (WriteBuffer & buf, const std::string_view & x) { writeString(StringRef(x), buf); return buf; }
|
||||
template <> inline WriteBuffer & operator<< (WriteBuffer & buf, const char & x) { writeChar(x, buf); return buf; }
|
||||
template <> inline WriteBuffer & operator<< (WriteBuffer & buf, const pcg32_fast & x) { PcgSerializer::serializePcg32(x, buf); return buf; }
|
||||
|
||||
|
@ -271,6 +271,11 @@ inline void writeString(const StringRef & ref, WriteBuffer & buf)
|
||||
writeString(ref.data, ref.size, buf);
|
||||
}
|
||||
|
||||
//inline void writeString(const std::string_view & view, WriteBuffer & buf)
|
||||
//{
|
||||
// writeString(view.data(), view.size(), buf);
|
||||
//}
|
||||
|
||||
|
||||
/** Writes a C-string without creating a temporary object. If the string is a literal, then `strlen` is executed at the compilation stage.
|
||||
* Use when the string is a literal.
|
||||
|
@ -25,7 +25,7 @@ class AddDefaultDatabaseVisitor
|
||||
{
|
||||
public:
|
||||
explicit AddDefaultDatabaseVisitor(
|
||||
const String & database_name_, bool only_replace_current_database_function_ = false, std::ostream * ostr_ = nullptr)
|
||||
const String & database_name_, bool only_replace_current_database_function_ = false, WriteBuffer * ostr_ = nullptr)
|
||||
: database_name(database_name_)
|
||||
, only_replace_current_database_function(only_replace_current_database_function_)
|
||||
, visit_depth(0)
|
||||
@ -66,7 +66,7 @@ private:
|
||||
const String database_name;
|
||||
bool only_replace_current_database_function = false;
|
||||
mutable size_t visit_depth;
|
||||
std::ostream * ostr;
|
||||
WriteBuffer * ostr;
|
||||
|
||||
void visit(ASTSelectWithUnionQuery & select, ASTPtr &) const
|
||||
{
|
||||
|
@ -106,10 +106,9 @@ String formattedAST(const ASTPtr & ast)
|
||||
{
|
||||
if (!ast)
|
||||
return {};
|
||||
std::stringstream ss;
|
||||
ss.exceptions(std::ios::failbit);
|
||||
formatAST(*ast, ss, false, true);
|
||||
return ss.str();
|
||||
WriteBufferFromOwnString buf;
|
||||
formatAST(*ast, buf, false, true);
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -16,7 +16,7 @@ class InDepthNodeVisitor
|
||||
public:
|
||||
using Data = typename Matcher::Data;
|
||||
|
||||
InDepthNodeVisitor(Data & data_, std::ostream * ostr_ = nullptr)
|
||||
InDepthNodeVisitor(Data & data_, WriteBuffer * ostr_ = nullptr)
|
||||
: data(data_),
|
||||
visit_depth(0),
|
||||
ostr(ostr_)
|
||||
@ -46,7 +46,7 @@ public:
|
||||
private:
|
||||
Data & data;
|
||||
size_t visit_depth;
|
||||
std::ostream * ostr;
|
||||
WriteBuffer * ostr;
|
||||
|
||||
void visitChildren(T & ast)
|
||||
{
|
||||
|
@ -135,10 +135,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
||||
else if ((create.columns_list && create.columns_list->indices && !create.columns_list->indices->children.empty()))
|
||||
{
|
||||
/// Currently, there are no database engines, that support any arguments.
|
||||
std::stringstream ostr;
|
||||
ostr.exceptions(std::ios::failbit);
|
||||
formatAST(*create.storage, ostr, false, false);
|
||||
throw Exception("Unknown database engine: " + ostr.str(), ErrorCodes::UNKNOWN_DATABASE_ENGINE);
|
||||
throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, "Unknown database engine: {}", serializeAST(*create.storage));
|
||||
}
|
||||
|
||||
if (create.storage->engine->name == "Atomic")
|
||||
@ -182,11 +179,10 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
||||
create.attach = true;
|
||||
create.if_not_exists = false;
|
||||
|
||||
std::ostringstream statement_stream;
|
||||
statement_stream.exceptions(std::ios::failbit);
|
||||
formatAST(create, statement_stream, false);
|
||||
statement_stream << '\n';
|
||||
String statement = statement_stream.str();
|
||||
WriteBufferFromOwnString statement_buf;
|
||||
formatAST(create, statement_buf, false);
|
||||
writeChar('\n', statement_buf);
|
||||
String statement = statement_buf.str();
|
||||
|
||||
/// Exclusive flag guarantees, that database is not created right now in another thread.
|
||||
WriteBufferFromFile out(metadata_file_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL);
|
||||
|
@ -222,15 +222,14 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl()
|
||||
Block sample_block = getSampleBlock();
|
||||
MutableColumns res_columns = sample_block.cloneEmptyColumns();
|
||||
|
||||
std::stringstream ss;
|
||||
ss.exceptions(std::ios::failbit);
|
||||
WriteBufferFromOwnString buf;
|
||||
|
||||
if (ast.getKind() == ASTExplainQuery::ParsedAST)
|
||||
{
|
||||
if (ast.getSettings())
|
||||
throw Exception("Settings are not supported for EXPLAIN AST query.", ErrorCodes::UNKNOWN_SETTING);
|
||||
|
||||
dumpAST(*ast.getExplainedQuery(), ss);
|
||||
dumpAST(*ast.getExplainedQuery(), buf);
|
||||
}
|
||||
else if (ast.getKind() == ASTExplainQuery::AnalyzedSyntax)
|
||||
{
|
||||
@ -240,7 +239,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl()
|
||||
ExplainAnalyzedSyntaxVisitor::Data data{.context = context};
|
||||
ExplainAnalyzedSyntaxVisitor(data).visit(query);
|
||||
|
||||
ast.getExplainedQuery()->format(IAST::FormatSettings(ss, false));
|
||||
ast.getExplainedQuery()->format(IAST::FormatSettings(buf, false));
|
||||
}
|
||||
else if (ast.getKind() == ASTExplainQuery::QueryPlan)
|
||||
{
|
||||
@ -256,8 +255,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl()
|
||||
if (settings.optimize)
|
||||
plan.optimize();
|
||||
|
||||
WriteBufferFromOStream buffer(ss);
|
||||
plan.explainPlan(buffer, settings.query_plan_options);
|
||||
plan.explainPlan(buf, settings.query_plan_options);
|
||||
}
|
||||
else if (ast.getKind() == ASTExplainQuery::QueryPipeline)
|
||||
{
|
||||
@ -271,8 +269,6 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl()
|
||||
interpreter.buildQueryPlan(plan);
|
||||
auto pipeline = plan.buildQueryPipeline();
|
||||
|
||||
WriteBufferFromOStream buffer(ss);
|
||||
|
||||
if (settings.graph)
|
||||
{
|
||||
/// Pipe holds QueryPlan, should not go out-of-scope
|
||||
@ -280,17 +276,17 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl()
|
||||
const auto & processors = pipe.getProcessors();
|
||||
|
||||
if (settings.compact)
|
||||
printPipelineCompact(processors, buffer, settings.query_pipeline_options.header);
|
||||
printPipelineCompact(processors, buf, settings.query_pipeline_options.header);
|
||||
else
|
||||
printPipeline(processors, buffer);
|
||||
printPipeline(processors, buf);
|
||||
}
|
||||
else
|
||||
{
|
||||
plan.explainPipeline(buffer, settings.query_pipeline_options);
|
||||
plan.explainPipeline(buf, settings.query_pipeline_options);
|
||||
}
|
||||
}
|
||||
|
||||
fillColumn(*res_columns[0], ss.str());
|
||||
fillColumn(*res_columns[0], buf.str());
|
||||
|
||||
return std::make_shared<OneBlockInputStream>(sample_block.cloneWithColumns(std::move(res_columns)));
|
||||
}
|
||||
|
@ -34,13 +34,12 @@ BlockInputStreamPtr InterpreterShowAccessQuery::executeImpl() const
|
||||
|
||||
/// Build the result column.
|
||||
MutableColumnPtr column = ColumnString::create();
|
||||
std::stringstream ss;
|
||||
ss.exceptions(std::ios::failbit);
|
||||
WriteBufferFromOwnString buf;
|
||||
for (const auto & query : queries)
|
||||
{
|
||||
ss.str("");
|
||||
formatAST(*query, ss, false, true);
|
||||
column->insert(ss.str());
|
||||
buf.restart();
|
||||
formatAST(*query, buf, false, true);
|
||||
column->insert(buf.str());
|
||||
}
|
||||
|
||||
String desc = "ACCESS";
|
||||
|
@ -238,21 +238,19 @@ BlockInputStreamPtr InterpreterShowCreateAccessEntityQuery::executeImpl()
|
||||
|
||||
/// Build the result column.
|
||||
MutableColumnPtr column = ColumnString::create();
|
||||
std::stringstream create_query_ss;
|
||||
create_query_ss.exceptions(std::ios::failbit);
|
||||
WriteBufferFromOwnString create_query_buf;
|
||||
for (const auto & create_query : create_queries)
|
||||
{
|
||||
formatAST(*create_query, create_query_ss, false, true);
|
||||
column->insert(create_query_ss.str());
|
||||
create_query_ss.str("");
|
||||
formatAST(*create_query, create_query_buf, false, true);
|
||||
column->insert(create_query_buf.str());
|
||||
create_query_buf.restart();
|
||||
}
|
||||
|
||||
/// Prepare description of the result column.
|
||||
std::stringstream desc_ss;
|
||||
desc_ss.exceptions(std::ios::failbit);
|
||||
WriteBufferFromOwnString desc_buf;
|
||||
const auto & show_query = query_ptr->as<const ASTShowCreateAccessEntityQuery &>();
|
||||
formatAST(show_query, desc_ss, false, true);
|
||||
String desc = desc_ss.str();
|
||||
formatAST(show_query, desc_buf, false, true);
|
||||
String desc = desc_buf.str();
|
||||
String prefix = "SHOW ";
|
||||
if (startsWith(desc, prefix))
|
||||
desc = desc.substr(prefix.length()); /// `desc` always starts with "SHOW ", so we can trim this prefix.
|
||||
|
@ -78,10 +78,9 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl()
|
||||
create.uuid = UUIDHelpers::Nil;
|
||||
}
|
||||
|
||||
std::stringstream stream;
|
||||
stream.exceptions(std::ios::failbit);
|
||||
formatAST(*create_query, stream, false, false);
|
||||
String res = stream.str();
|
||||
WriteBufferFromOwnString buf;
|
||||
formatAST(*create_query, buf, false, false);
|
||||
String res = buf.str();
|
||||
|
||||
MutableColumnPtr column = ColumnString::create();
|
||||
column->insert(res);
|
||||
|
@ -118,21 +118,19 @@ BlockInputStreamPtr InterpreterShowGrantsQuery::executeImpl()
|
||||
|
||||
/// Build the result column.
|
||||
MutableColumnPtr column = ColumnString::create();
|
||||
std::stringstream grant_ss;
|
||||
grant_ss.exceptions(std::ios::failbit);
|
||||
WriteBufferFromOwnString grant_buf;
|
||||
for (const auto & grant_query : grant_queries)
|
||||
{
|
||||
grant_ss.str("");
|
||||
formatAST(*grant_query, grant_ss, false, true);
|
||||
column->insert(grant_ss.str());
|
||||
grant_buf.restart();
|
||||
formatAST(*grant_query, grant_buf, false, true);
|
||||
column->insert(grant_buf.str());
|
||||
}
|
||||
|
||||
/// Prepare description of the result column.
|
||||
std::stringstream desc_ss;
|
||||
desc_ss.exceptions(std::ios::failbit);
|
||||
WriteBufferFromOwnString desc_buf;
|
||||
const auto & show_query = query_ptr->as<const ASTShowGrantsQuery &>();
|
||||
formatAST(show_query, desc_ss, false, true);
|
||||
String desc = desc_ss.str();
|
||||
formatAST(show_query, desc_buf, false, true);
|
||||
String desc = desc_buf.str();
|
||||
String prefix = "SHOW ";
|
||||
if (desc.starts_with(prefix))
|
||||
desc = desc.substr(prefix.length()); /// `desc` always starts with "SHOW ", so we can trim this prefix.
|
||||
|
@ -20,13 +20,12 @@ namespace ErrorCodes
|
||||
|
||||
static String wrongAliasMessage(const ASTPtr & ast, const ASTPtr & prev_ast, const String & alias)
|
||||
{
|
||||
std::stringstream message;
|
||||
message.exceptions(std::ios::failbit);
|
||||
message << "Different expressions with the same alias " << backQuoteIfNeed(alias) << ":" << std::endl;
|
||||
WriteBufferFromOwnString message;
|
||||
message << "Different expressions with the same alias " << backQuoteIfNeed(alias) << ":\n";
|
||||
formatAST(*ast, message, false, true);
|
||||
message << std::endl << "and" << std::endl;
|
||||
message << "\nand\n";
|
||||
formatAST(*prev_ast, message, false, true);
|
||||
message << std::endl;
|
||||
message << '\n';
|
||||
return message.str();
|
||||
}
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Parsers/ASTAlterQuery.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <iomanip>
|
||||
#include <Common/quoteString.h>
|
||||
|
||||
@ -246,7 +247,7 @@ void ASTAlterCommand::formatImpl(
|
||||
<< "PARTITION " << (settings.hilite ? hilite_none : "");
|
||||
partition->formatImpl(settings, state, frame);
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "")
|
||||
<< " FROM " << (settings.hilite ? hilite_none : "") << std::quoted(from, '\'');
|
||||
<< " FROM " << (settings.hilite ? hilite_none : "") << DB::quote << from;
|
||||
}
|
||||
else if (type == ASTAlterCommand::FREEZE_PARTITION)
|
||||
{
|
||||
@ -256,7 +257,7 @@ void ASTAlterCommand::formatImpl(
|
||||
if (!with_name.empty())
|
||||
{
|
||||
settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << "WITH NAME" << (settings.hilite ? hilite_none : "")
|
||||
<< " " << std::quoted(with_name, '\'');
|
||||
<< " " << DB::quote << with_name;
|
||||
}
|
||||
}
|
||||
else if (type == ASTAlterCommand::FREEZE_ALL)
|
||||
@ -266,7 +267,7 @@ void ASTAlterCommand::formatImpl(
|
||||
if (!with_name.empty())
|
||||
{
|
||||
settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << "WITH NAME" << (settings.hilite ? hilite_none : "")
|
||||
<< " " << std::quoted(with_name, '\'');
|
||||
<< " " << DB::quote << with_name;
|
||||
}
|
||||
}
|
||||
else if (type == ASTAlterCommand::DELETE)
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Parsers/ASTAsterisk.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Parsers/ASTColumnDeclaration.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Common/quoteString.h>
|
||||
#include <re2/re2.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Parsers/ASTConstraintDeclaration.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Interpreters/StorageID.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/IntervalKind.h>
|
||||
#include <ext/range.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Parsers/ASTCreateRoleQuery.h>
|
||||
#include <Parsers/ASTSettingsProfileElement.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <ext/range.h>
|
||||
#include <boost/range/algorithm/transform.hpp>
|
||||
#include <sstream>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -62,14 +63,13 @@ namespace
|
||||
void formatForClauses(const std::vector<std::pair<ConditionType, ASTPtr>> & conditions, bool alter, const IAST::FormatSettings & settings)
|
||||
{
|
||||
std::vector<std::pair<ConditionType, String>> conditions_as_strings;
|
||||
std::stringstream temp_sstream;
|
||||
temp_sstream.exceptions(std::ios::failbit);
|
||||
IAST::FormatSettings temp_settings(temp_sstream, settings);
|
||||
WriteBufferFromOwnString temp_buf;
|
||||
IAST::FormatSettings temp_settings(temp_buf, settings);
|
||||
for (const auto & [condition_type, condition] : conditions)
|
||||
{
|
||||
formatConditionalExpression(condition, temp_settings);
|
||||
conditions_as_strings.emplace_back(condition_type, temp_sstream.str());
|
||||
temp_sstream.str("");
|
||||
conditions_as_strings.emplace_back(condition_type, temp_buf.str());
|
||||
temp_buf.restart();
|
||||
}
|
||||
|
||||
boost::container::flat_set<std::string_view> commands;
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Parsers/ASTSettingsProfileElement.h>
|
||||
#include <Parsers/ASTRolesOrUsersSet.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Parsers/ASTRolesOrUsersSet.h>
|
||||
#include <Parsers/ASTSettingsProfileElement.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Parsers/ASTDictionary.h>
|
||||
#include <Poco/String.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Parsers/ASTDictionaryAttributeDeclaration.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Parsers/ASTDropAccessEntityQuery.h>
|
||||
#include <Parsers/ASTRowPolicyName.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Parsers/ASTDropQuery.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Poco/String.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Parsers/ASTGrantQuery.h>
|
||||
#include <Parsers/ASTRolesOrUsersSet.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Interpreters/StorageID.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Parsers/ASTIndexDeclaration.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Parsers/ASTKillQueryQuery.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -73,4 +73,9 @@ void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr) const
|
||||
}
|
||||
}
|
||||
|
||||
void ASTLiteral::formatImplWithoutAlias(const FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
|
||||
{
|
||||
settings.ostr << applyVisitor(FieldVisitorToString(), value);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -43,10 +43,7 @@ public:
|
||||
void updateTreeHashImpl(SipHash & hash_state) const override;
|
||||
|
||||
protected:
|
||||
void formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const override
|
||||
{
|
||||
settings.ostr << applyVisitor(FieldVisitorToString(), value);
|
||||
}
|
||||
void formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
|
||||
|
||||
void appendColumnNameImpl(WriteBuffer & ostr) const override;
|
||||
};
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Parsers/ASTNameTypePair.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Parsers/ASTOptimizeQuery.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Columns/Collator.h>
|
||||
#include <Parsers/ASTOrderByElement.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Parsers/ASTPartition.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Parsers/ASTQualifiedAsterisk.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Parsers/ASTQueryParameter.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Parsers/parseIdentifierOrStringLiteral.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Parsers/ASTQueryWithTableAndOutput.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Parsers/ASTQueryWithOutput.h>
|
||||
#include <Parsers/ASTQueryWithOnCluster.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Parsers/ASTRolesOrUsersSet.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Parsers/ASTRowPolicyName.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Parsers/ASTSampleRatio.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -34,5 +35,9 @@ String ASTSampleRatio::toString(Rational ratio)
|
||||
return toString(ratio.numerator) + " / " + toString(ratio.denominator);
|
||||
}
|
||||
|
||||
void ASTSampleRatio::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
|
||||
{
|
||||
settings.ostr << toString(ratio);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -35,10 +35,7 @@ public:
|
||||
static String toString(BigNum num);
|
||||
static String toString(Rational ratio);
|
||||
|
||||
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override
|
||||
{
|
||||
settings.ostr << toString(ratio);
|
||||
}
|
||||
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Parsers/ASTOrderByElement.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Interpreters/StorageID.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Parsers/formatSettingName.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Parsers/ASTSetRoleQuery.h>
|
||||
#include <Parsers/ASTRolesOrUsersSet.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Parsers/formatSettingName.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Parsers/ASTShowAccessEntitiesQuery.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Parsers/ASTShowCreateAccessEntityQuery.h>
|
||||
#include <Parsers/ASTRowPolicyName.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Parsers/ASTShowGrantsQuery.h>
|
||||
#include <Parsers/ASTRolesOrUsersSet.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <iomanip>
|
||||
#include <Parsers/ASTShowTablesQuery.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -21,7 +22,7 @@ void ASTShowTablesQuery::formatLike(const FormatSettings & settings) const
|
||||
<< (not_like ? " NOT" : "")
|
||||
<< (case_insensitive_like ? " ILIKE " : " LIKE ")
|
||||
<< (settings.hilite ? hilite_none : "")
|
||||
<< std::quoted(like, '\'');
|
||||
<< DB::quote << like;
|
||||
}
|
||||
|
||||
void ASTShowTablesQuery::formatLimit(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTSystemQuery.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Columns/Collator.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Parsers/ASTTTLElement.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Parsers/ASTUserNameWithHost.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Parsers/ASTWithAlias.h>
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Parsers/ASTWithElement.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <common/find_symbols.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
#include <string.h> /// strncmp, strncasecmp
|
||||
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <sstream>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Poco/Util/Application.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
|
||||
@ -14,7 +15,7 @@ namespace DB
|
||||
class DumpASTNode
|
||||
{
|
||||
public:
|
||||
DumpASTNode(const IAST & ast_, std::ostream * ostr_, size_t & depth, const char * label_ = nullptr)
|
||||
DumpASTNode(const IAST & ast_, WriteBuffer * ostr_, size_t & depth, const char * label_ = nullptr)
|
||||
: ast(ast_),
|
||||
ostr(ostr_),
|
||||
indent(depth),
|
||||
@ -24,12 +25,12 @@ public:
|
||||
if (!ostr)
|
||||
return;
|
||||
if (label && visit_depth == 0)
|
||||
(*ostr) << "-- " << label << std::endl;
|
||||
(*ostr) << "-- " << label << '\n';
|
||||
++visit_depth;
|
||||
|
||||
(*ostr) << String(indent, ' ');
|
||||
printNode();
|
||||
(*ostr) << std::endl;
|
||||
(*ostr) << '\n';
|
||||
}
|
||||
|
||||
~DumpASTNode()
|
||||
@ -38,7 +39,7 @@ public:
|
||||
return;
|
||||
--visit_depth;
|
||||
if (label && visit_depth == 0)
|
||||
(*ostr) << "--" << std::endl;
|
||||
(*ostr) << "--\n";
|
||||
}
|
||||
|
||||
template <typename T, typename U>
|
||||
@ -50,14 +51,14 @@ public:
|
||||
(*ostr) << (str_indent ? String(str_indent) : String(indent, ' '));
|
||||
(*ostr) << '(' << name << ' ' << value << ')';
|
||||
if (!str_indent)
|
||||
(*ostr) << std::endl;
|
||||
(*ostr) << '\n';
|
||||
}
|
||||
|
||||
size_t & getDepth() { return visit_depth; }
|
||||
|
||||
private:
|
||||
const IAST & ast;
|
||||
std::ostream * ostr;
|
||||
WriteBuffer * ostr;
|
||||
size_t indent;
|
||||
size_t & visit_depth; /// shared with children
|
||||
const char * label;
|
||||
@ -77,7 +78,7 @@ private:
|
||||
}
|
||||
};
|
||||
|
||||
inline void dumpAST(const IAST & ast, std::ostream & ostr, DumpASTNode * parent = nullptr)
|
||||
inline void dumpAST(const IAST & ast, WriteBuffer & ostr, DumpASTNode * parent = nullptr)
|
||||
{
|
||||
size_t depth = 0;
|
||||
DumpASTNode dump(ast, &ostr, (parent ? parent->getDepth() : depth));
|
||||
@ -95,7 +96,6 @@ public:
|
||||
DebugASTLog()
|
||||
: log(nullptr)
|
||||
{
|
||||
ss.exceptions(std::ios::failbit);
|
||||
if constexpr (_enable)
|
||||
log = &Poco::Logger::get("AST");
|
||||
}
|
||||
@ -103,14 +103,14 @@ public:
|
||||
~DebugASTLog()
|
||||
{
|
||||
if constexpr (_enable)
|
||||
LOG_DEBUG(log, ss.str());
|
||||
LOG_DEBUG(log, buf.str());
|
||||
}
|
||||
|
||||
std::ostream * stream() { return (_enable ? &ss : nullptr); }
|
||||
WriteBuffer * stream() { return (_enable ? &buf : nullptr); }
|
||||
|
||||
private:
|
||||
Poco::Logger * log;
|
||||
std::stringstream ss;
|
||||
WriteBufferFromOwnString buf;
|
||||
};
|
||||
|
||||
|
||||
|
@ -89,10 +89,9 @@ size_t IAST::checkDepthImpl(size_t max_depth, size_t level) const
|
||||
|
||||
std::string IAST::formatForErrorMessage() const
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss.exceptions(std::ios::failbit);
|
||||
format(FormatSettings(ss, true /* one line */));
|
||||
return ss.str();
|
||||
WriteBufferFromOwnString buf;
|
||||
format(FormatSettings(buf, true /* one line */));
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
void IAST::cloneChildren()
|
||||
@ -112,8 +111,6 @@ String IAST::getColumnName() const
|
||||
|
||||
void IAST::FormatSettings::writeIdentifier(const String & name) const
|
||||
{
|
||||
WriteBufferFromOStream out(ostr, 32);
|
||||
|
||||
switch (identifier_quoting_style)
|
||||
{
|
||||
case IdentifierQuotingStyle::None:
|
||||
@ -121,36 +118,34 @@ void IAST::FormatSettings::writeIdentifier(const String & name) const
|
||||
if (always_quote_identifiers)
|
||||
throw Exception("Incompatible arguments: always_quote_identifiers = true && identifier_quoting_style == IdentifierQuotingStyle::None",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
writeString(name, out);
|
||||
writeString(name, ostr);
|
||||
break;
|
||||
}
|
||||
case IdentifierQuotingStyle::Backticks:
|
||||
{
|
||||
if (always_quote_identifiers)
|
||||
writeBackQuotedString(name, out);
|
||||
writeBackQuotedString(name, ostr);
|
||||
else
|
||||
writeProbablyBackQuotedString(name, out);
|
||||
writeProbablyBackQuotedString(name, ostr);
|
||||
break;
|
||||
}
|
||||
case IdentifierQuotingStyle::DoubleQuotes:
|
||||
{
|
||||
if (always_quote_identifiers)
|
||||
writeDoubleQuotedString(name, out);
|
||||
writeDoubleQuotedString(name, ostr);
|
||||
else
|
||||
writeProbablyDoubleQuotedString(name, out);
|
||||
writeProbablyDoubleQuotedString(name, ostr);
|
||||
break;
|
||||
}
|
||||
case IdentifierQuotingStyle::BackticksMySQL:
|
||||
{
|
||||
if (always_quote_identifiers)
|
||||
writeBackQuotedStringMySQL(name, out);
|
||||
writeBackQuotedStringMySQL(name, ostr);
|
||||
else
|
||||
writeProbablyBackQuotedStringMySQL(name, out);
|
||||
writeProbablyBackQuotedStringMySQL(name, ostr);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
out.next();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Parsers/IdentifierQuotingStyle.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/TypePromotion.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <ostream>
|
||||
@ -161,7 +162,7 @@ public:
|
||||
/// Format settings.
|
||||
struct FormatSettings
|
||||
{
|
||||
std::ostream & ostr;
|
||||
WriteBuffer & ostr;
|
||||
bool hilite = false;
|
||||
bool one_line;
|
||||
bool always_quote_identifiers = false;
|
||||
@ -169,13 +170,13 @@ public:
|
||||
|
||||
char nl_or_ws;
|
||||
|
||||
FormatSettings(std::ostream & ostr_, bool one_line_)
|
||||
FormatSettings(WriteBuffer & ostr_, bool one_line_)
|
||||
: ostr(ostr_), one_line(one_line_)
|
||||
{
|
||||
nl_or_ws = one_line ? ' ' : '\n';
|
||||
}
|
||||
|
||||
FormatSettings(std::ostream & ostr_, const FormatSettings & other)
|
||||
FormatSettings(WriteBuffer & ostr_, const FormatSettings & other)
|
||||
: ostr(ostr_), hilite(other.hilite), one_line(other.one_line),
|
||||
always_quote_identifiers(other.always_quote_identifiers), identifier_quoting_style(other.identifier_quoting_style)
|
||||
{
|
||||
@ -242,17 +243,17 @@ private:
|
||||
template <typename AstArray>
|
||||
std::string IAST::formatForErrorMessage(const AstArray & array)
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss.exceptions(std::ios::failbit);
|
||||
WriteBufferFromOwnString buf;
|
||||
for (size_t i = 0; i < array.size(); ++i)
|
||||
{
|
||||
if (i > 0)
|
||||
{
|
||||
ss << ", ";
|
||||
const char * delim = ", ";
|
||||
buf.write(delim, strlen(delim));
|
||||
}
|
||||
array[i]->format(IAST::FormatSettings(ss, true /* one line */));
|
||||
array[i]->format(IAST::FormatSettings(buf, true /* one line */));
|
||||
}
|
||||
return ss.str();
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -5,9 +5,9 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void formatAST(const IAST & ast, std::ostream & s, bool hilite, bool one_line)
|
||||
void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite, bool one_line)
|
||||
{
|
||||
IAST::FormatSettings settings(s, one_line);
|
||||
IAST::FormatSettings settings(buf, one_line);
|
||||
settings.hilite = hilite;
|
||||
|
||||
ast.format(settings);
|
||||
@ -15,10 +15,9 @@ void formatAST(const IAST & ast, std::ostream & s, bool hilite, bool one_line)
|
||||
|
||||
String serializeAST(const IAST & ast, bool one_line)
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss.exceptions(std::ios::failbit);
|
||||
formatAST(ast, ss, false, one_line);
|
||||
return ss.str();
|
||||
WriteBufferFromOwnString buf;
|
||||
formatAST(ast, buf, false, one_line);
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -7,23 +7,25 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class WriteBuffer;
|
||||
|
||||
/** Takes a syntax tree and turns it back into text.
|
||||
* In case of INSERT query, the data will be missing.
|
||||
*/
|
||||
void formatAST(const IAST & ast, std::ostream & s, bool hilite = true, bool one_line = false);
|
||||
void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite = true, bool one_line = false);
|
||||
|
||||
String serializeAST(const IAST & ast, bool one_line = true);
|
||||
|
||||
inline std::ostream & operator<<(std::ostream & os, const IAST & ast)
|
||||
{
|
||||
formatAST(ast, os, false, true);
|
||||
return os;
|
||||
}
|
||||
|
||||
inline std::ostream & operator<<(std::ostream & os, const ASTPtr & ast)
|
||||
{
|
||||
formatAST(*ast, os, false, true);
|
||||
return os;
|
||||
}
|
||||
//inline std::ostream & operator<<(std::ostream & os, const IAST & ast)
|
||||
//{
|
||||
// formatAST(ast, os, false, true);
|
||||
// return os;
|
||||
//}
|
||||
//
|
||||
//inline std::ostream & operator<<(std::ostream & os, const ASTPtr & ast)
|
||||
//{
|
||||
// formatAST(*ast, os, false, true);
|
||||
// return os;
|
||||
//}
|
||||
|
||||
}
|
||||
|
@ -2,13 +2,13 @@
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <common/find_symbols.h>
|
||||
#include <ostream>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void formatSettingName(const String & setting_name, std::ostream & out)
|
||||
void formatSettingName(const String & setting_name, WriteBuffer & out)
|
||||
{
|
||||
if (isValidIdentifier(setting_name))
|
||||
{
|
||||
|
@ -7,9 +7,11 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class WriteBuffer;
|
||||
|
||||
/// Outputs built-in or custom setting's name.
|
||||
/// The function is like backQuoteIfNeed() but didn't quote with backticks
|
||||
/// if the name consists of identifiers joined with dots.
|
||||
void formatSettingName(const String & setting_name, std::ostream & out);
|
||||
void formatSettingName(const String & setting_name, WriteBuffer & out);
|
||||
|
||||
}
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <sstream>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -11,9 +10,6 @@ namespace DB
|
||||
|
||||
String queryToString(const IAST & query)
|
||||
{
|
||||
std::ostringstream out;
|
||||
out.exceptions(std::ios::failbit);
|
||||
formatAST(query, out, false, true);
|
||||
return out.str();
|
||||
return serializeAST(query);
|
||||
}
|
||||
}
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
|
||||
|
||||
int main(int, char **)
|
||||
@ -14,7 +15,8 @@ int main(int, char **)
|
||||
ParserCreateQuery parser;
|
||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0);
|
||||
|
||||
formatAST(*ast, std::cerr);
|
||||
WriteBufferFromOStream out(std::cerr, 4096);
|
||||
formatAST(*ast, out);
|
||||
std::cerr << std::endl;
|
||||
|
||||
return 0;
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Parsers/DumpASTNode.h>
|
||||
#include <Parsers/TablePropertiesQueriesASTs.h>
|
||||
#include <Parsers/ParserTablePropertiesQuery.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
@ -17,10 +18,9 @@ using namespace DB;
|
||||
|
||||
static String astToString(IAST * ast)
|
||||
{
|
||||
std::ostringstream oss;
|
||||
oss.exceptions(std::ios::failbit);
|
||||
dumpAST(*ast, oss);
|
||||
return oss.str();
|
||||
WriteBufferFromOwnString buf;
|
||||
dumpAST(*ast, buf);
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
/// Tests for external dictionaries DDL parser
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Parsers/ParserQueryWithOutput.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
|
||||
|
||||
int main(int, char **)
|
||||
@ -25,7 +26,8 @@ try
|
||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0);
|
||||
|
||||
std::cout << "Success." << std::endl;
|
||||
formatAST(*ast, std::cerr);
|
||||
WriteBufferFromOStream out(std::cerr, 4096);
|
||||
formatAST(*ast, out);
|
||||
std::cout << std::endl;
|
||||
|
||||
return 0;
|
||||
|
@ -1693,13 +1693,12 @@ std::vector<MergeTreeMutationStatus> ReplicatedMergeTreeQueue::getMutationsStatu
|
||||
|
||||
for (const MutationCommand & command : entry.commands)
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss.exceptions(std::ios::failbit);
|
||||
formatAST(*command.ast, ss, false, true);
|
||||
WriteBufferFromOwnString buf;
|
||||
formatAST(*command.ast, buf, false, true);
|
||||
result.push_back(MergeTreeMutationStatus
|
||||
{
|
||||
entry.znode_name,
|
||||
ss.str(),
|
||||
buf.str(),
|
||||
entry.create_time,
|
||||
entry.block_numbers,
|
||||
parts_to_mutate,
|
||||
|
@ -18,10 +18,9 @@ static String formattedAST(const ASTPtr & ast)
|
||||
{
|
||||
if (!ast)
|
||||
return "";
|
||||
std::stringstream ss;
|
||||
ss.exceptions(std::ios::failbit);
|
||||
formatAST(*ast, ss, false, true);
|
||||
return ss.str();
|
||||
WriteBufferFromOwnString buf;
|
||||
formatAST(*ast, buf, false, true);
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTreeData & data, const StorageMetadataPtr & metadata_snapshot)
|
||||
|
@ -126,10 +126,9 @@ std::shared_ptr<ASTAlterCommandList> MutationCommands::ast() const
|
||||
|
||||
void MutationCommands::writeText(WriteBuffer & out) const
|
||||
{
|
||||
std::stringstream commands_ss;
|
||||
commands_ss.exceptions(std::ios::failbit);
|
||||
formatAST(*ast(), commands_ss, /* hilite = */ false, /* one_line = */ true);
|
||||
out << escape << commands_ss.str();
|
||||
WriteBufferFromOwnString commands_buf;
|
||||
formatAST(*ast(), commands_buf, /* hilite = */ false, /* one_line = */ true);
|
||||
out << escape << commands_buf.str();
|
||||
}
|
||||
|
||||
void MutationCommands::readText(ReadBuffer & in)
|
||||
|
@ -538,13 +538,12 @@ std::vector<MergeTreeMutationStatus> StorageMergeTree::getMutationsStatus() cons
|
||||
|
||||
for (const MutationCommand & command : entry.commands)
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss.exceptions(std::ios::failbit);
|
||||
formatAST(*command.ast, ss, false, true);
|
||||
WriteBufferFromOwnString buf;
|
||||
formatAST(*command.ast, buf, false, true);
|
||||
result.push_back(MergeTreeMutationStatus
|
||||
{
|
||||
entry.file_name,
|
||||
ss.str(),
|
||||
buf.str(),
|
||||
entry.create_time,
|
||||
block_numbers_map,
|
||||
parts_to_do_names,
|
||||
|
@ -220,8 +220,7 @@ String transformQueryForExternalDatabase(
|
||||
ASTPtr select_ptr = select;
|
||||
dropAliases(select_ptr);
|
||||
|
||||
std::stringstream out;
|
||||
out.exceptions(std::ios::failbit);
|
||||
WriteBufferFromOwnString out;
|
||||
IAST::FormatSettings settings(out, true);
|
||||
settings.identifier_quoting_style = identifier_quoting_style;
|
||||
settings.always_quote_identifiers = identifier_quoting_style != IdentifierQuotingStyle::None;
|
||||
|
@ -16,6 +16,7 @@
|
||||
#include <Parsers/ParserQueryWithOutput.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
||||
#include <boost/algorithm/string.hpp>
|
||||
#include <boost/program_options.hpp>
|
||||
@ -828,9 +829,9 @@ FuncRet arithmeticFunc(DB::ASTPtr ch, std::map<std::string, Column> & columns)
|
||||
FuncRet r(ret_type, "");
|
||||
if (no_indent)
|
||||
{
|
||||
std::ostringstream ss;
|
||||
formatAST(*ch, ss);
|
||||
r.value = ss.str();
|
||||
DB::WriteBufferFromOwnString buf;
|
||||
formatAST(*ch, buf);
|
||||
r.value = buf.str();
|
||||
}
|
||||
return r;
|
||||
}
|
||||
@ -990,10 +991,10 @@ FuncRet simpleFunc(DB::ASTPtr ch, std::map<std::string, Column> & columns)
|
||||
{
|
||||
if (no_indent)
|
||||
{
|
||||
std::ostringstream ss;
|
||||
formatAST(*ch, ss);
|
||||
DB::WriteBufferFromOwnString buf;
|
||||
formatAST(*ch, buf);
|
||||
auto r = func_to_return_type[boost::algorithm::to_lower_copy(x->name)];
|
||||
r.value = ss.str();
|
||||
r.value = buf.str();
|
||||
return r;
|
||||
}
|
||||
return func_to_return_type[boost::algorithm::to_lower_copy(x->name)];
|
||||
@ -1003,11 +1004,11 @@ FuncRet simpleFunc(DB::ASTPtr ch, std::map<std::string, Column> & columns)
|
||||
{
|
||||
if (no_indent)
|
||||
{
|
||||
std::ostringstream ss;
|
||||
formatAST(*ch, ss);
|
||||
DB::WriteBufferFromOwnString buf;
|
||||
formatAST(*ch, buf);
|
||||
return FuncRet(
|
||||
func_to_param_type[boost::algorithm::to_lower_copy(x->name)],
|
||||
ss.str());
|
||||
buf.str());
|
||||
}
|
||||
return FuncRet(
|
||||
func_to_param_type[boost::algorithm::to_lower_copy(x->name)],
|
||||
|
Loading…
Reference in New Issue
Block a user