Merge pull request #72723 from vitlibar/formatimpl-refactoring-reworked

Remove ostr from structure IAST::FormatSettings.
This commit is contained in:
Vitaly Baranov 2024-12-04 12:43:26 +00:00 committed by GitHub
commit 9ce4ac3997
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
220 changed files with 1867 additions and 1886 deletions

View File

@ -277,10 +277,10 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
{
WriteBufferFromOwnString str_buf;
bool oneline_current_query = oneline || approx_query_length < max_line_length;
IAST::FormatSettings settings(str_buf, oneline_current_query, hilite);
IAST::FormatSettings settings(oneline_current_query, hilite);
settings.show_secrets = true;
settings.print_pretty_type_names = !oneline_current_query;
res->format(settings);
res->format(str_buf, settings);
if (insert_query_payload)
{
@ -324,10 +324,10 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
{
WriteBufferFromOwnString str_buf;
bool oneline_current_query = oneline || approx_query_length < max_line_length;
IAST::FormatSettings settings(str_buf, oneline_current_query, hilite);
IAST::FormatSettings settings(oneline_current_query, hilite);
settings.show_secrets = true;
settings.print_pretty_type_names = !oneline_current_query;
res->format(settings);
res->format(str_buf, settings);
auto res_string = str_buf.str();
WriteBufferFromOStream res_cout(std::cout, 4096);

View File

@ -348,11 +348,11 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, const Setting
{
output_stream << std::endl;
WriteBufferFromOStream res_buf(output_stream, 4096);
IAST::FormatSettings format_settings(res_buf, /* one_line */ false);
IAST::FormatSettings format_settings(/* one_line */ false);
format_settings.hilite = true;
format_settings.show_secrets = true;
format_settings.print_pretty_type_names = true;
res->format(format_settings);
res->format(res_buf, format_settings);
res_buf.finalize();
output_stream << std::endl << std::endl;
}

View File

@ -143,10 +143,10 @@ private:
throw;
}
IAST::FormatSettings settings(buf, output_formatting == OutputFormatting::SingleLine, /*hilite*/ false);
IAST::FormatSettings settings(output_formatting == OutputFormatting::SingleLine, /*hilite*/ false);
settings.show_secrets = true;
settings.print_pretty_type_names = print_pretty_type_names;
ast->format(settings);
ast->format(buf, settings);
auto formatted = buf.stringView();

View File

@ -392,7 +392,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl()
ExplainAnalyzedSyntaxVisitor::Data data(getContext());
ExplainAnalyzedSyntaxVisitor(data).visit(query);
ast.getExplainedQuery()->format(IAST::FormatSettings(buf, settings.oneline));
ast.getExplainedQuery()->format(buf, IAST::FormatSettings(settings.oneline));
break;
}
case ASTExplainQuery::QueryTree:
@ -441,7 +441,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl()
if (need_newline)
buf << "\n\n";
query_tree->toAST()->format(IAST::FormatSettings(buf, false));
query_tree->toAST()->format(buf, IAST::FormatSettings(false));
}
break;

View File

@ -88,7 +88,7 @@ namespace
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method clone is not supported");
}
void formatImpl(const FormatSettings &, FormatState &, FormatStateStacked) const override
void formatImpl(WriteBuffer &, const FormatSettings &, FormatState &, FormatStateStacked) const override
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method formatImpl is not supported");
}

View File

@ -1041,9 +1041,9 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
if (settings[Setting::enforce_strict_identifier_format])
{
WriteBufferFromOwnString buf;
IAST::FormatSettings enforce_strict_identifier_format_settings(buf, true);
IAST::FormatSettings enforce_strict_identifier_format_settings(true);
enforce_strict_identifier_format_settings.enforce_strict_identifier_format = true;
ast->format(enforce_strict_identifier_format_settings);
ast->format(buf, enforce_strict_identifier_format_settings);
}
if (auto * insert_query = ast->as<ASTInsertQuery>())

View File

@ -12,46 +12,46 @@ ASTPtr ASTAlterNamedCollectionQuery::clone() const
return std::make_shared<ASTAlterNamedCollectionQuery>(*this);
}
void ASTAlterNamedCollectionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
void ASTAlterNamedCollectionQuery::formatImpl(WriteBuffer & ostr, const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ALTER NAMED COLLECTION ";
ostr << (settings.hilite ? hilite_keyword : "") << "ALTER NAMED COLLECTION ";
if (if_exists)
settings.ostr << "IF EXISTS ";
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(collection_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(settings);
ostr << "IF EXISTS ";
ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(collection_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(ostr, settings);
if (!changes.empty())
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " SET " << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << " SET " << (settings.hilite ? hilite_none : "");
bool first = true;
for (const auto & change : changes)
{
if (!first)
settings.ostr << ", ";
ostr << ", ";
else
first = false;
formatSettingName(change.name, settings.ostr);
formatSettingName(change.name, ostr);
if (settings.show_secrets)
settings.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value);
ostr << " = " << applyVisitor(FieldVisitorToString(), change.value);
else
settings.ostr << " = '[HIDDEN]'";
ostr << " = '[HIDDEN]'";
auto override_value = overridability.find(change.name);
if (override_value != overridability.end())
settings.ostr << " " << (override_value->second ? "" : "NOT ") << "OVERRIDABLE";
ostr << " " << (override_value->second ? "" : "NOT ") << "OVERRIDABLE";
}
}
if (!delete_keys.empty())
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " DELETE " << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << " DELETE " << (settings.hilite ? hilite_none : "");
bool first = true;
for (const auto & key : delete_keys)
{
if (!first)
settings.ostr << ", ";
ostr << ", ";
else
first = false;
formatSettingName(key, settings.ostr);
formatSettingName(key, ostr);
}
}
}

View File

@ -21,7 +21,7 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster<ASTAlterNamedCollectionQuery>(clone()); }

View File

@ -69,270 +69,270 @@ ASTPtr ASTAlterCommand::clone() const
return res;
}
void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTAlterCommand::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
scope_guard closing_bracket_guard;
if (format_alter_commands_with_parentheses)
{
settings.ostr << "(";
closing_bracket_guard = make_scope_guard(std::function<void(void)>([&settings]() { settings.ostr << ")"; }));
ostr << "(";
closing_bracket_guard = make_scope_guard(std::function<void(void)>([&ostr]() { ostr << ")"; }));
}
if (type == ASTAlterCommand::ADD_COLUMN)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ADD COLUMN " << (if_not_exists ? "IF NOT EXISTS " : "")
ostr << (settings.hilite ? hilite_keyword : "") << "ADD COLUMN " << (if_not_exists ? "IF NOT EXISTS " : "")
<< (settings.hilite ? hilite_none : "");
col_decl->formatImpl(settings, state, frame);
col_decl->formatImpl(ostr, settings, state, frame);
if (first)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FIRST " << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << " FIRST " << (settings.hilite ? hilite_none : "");
else if (column) /// AFTER
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AFTER " << (settings.hilite ? hilite_none : "");
column->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " AFTER " << (settings.hilite ? hilite_none : "");
column->formatImpl(ostr, settings, state, frame);
}
}
else if (type == ASTAlterCommand::DROP_COLUMN)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << (clear_column ? "CLEAR " : "DROP ") << "COLUMN "
ostr << (settings.hilite ? hilite_keyword : "") << (clear_column ? "CLEAR " : "DROP ") << "COLUMN "
<< (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : "");
column->formatImpl(settings, state, frame);
column->formatImpl(ostr, settings, state, frame);
if (partition)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(ostr, settings, state, frame);
}
}
else if (type == ASTAlterCommand::MODIFY_COLUMN)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY COLUMN " << (if_exists ? "IF EXISTS " : "")
ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY COLUMN " << (if_exists ? "IF EXISTS " : "")
<< (settings.hilite ? hilite_none : "");
col_decl->formatImpl(settings, state, frame);
col_decl->formatImpl(ostr, settings, state, frame);
if (!remove_property.empty())
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " REMOVE " << remove_property;
ostr << (settings.hilite ? hilite_keyword : "") << " REMOVE " << remove_property;
}
else if (settings_changes)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " MODIFY SETTING " << (settings.hilite ? hilite_none : "");
settings_changes->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " MODIFY SETTING " << (settings.hilite ? hilite_none : "");
settings_changes->formatImpl(ostr, settings, state, frame);
}
else if (settings_resets)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " RESET SETTING " << (settings.hilite ? hilite_none : "");
settings_resets->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " RESET SETTING " << (settings.hilite ? hilite_none : "");
settings_resets->formatImpl(ostr, settings, state, frame);
}
else
{
if (first)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FIRST " << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << " FIRST " << (settings.hilite ? hilite_none : "");
else if (column) /// AFTER
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AFTER " << (settings.hilite ? hilite_none : "");
column->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " AFTER " << (settings.hilite ? hilite_none : "");
column->formatImpl(ostr, settings, state, frame);
}
}
}
else if (type == ASTAlterCommand::MATERIALIZE_COLUMN)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "MATERIALIZE COLUMN " << (settings.hilite ? hilite_none : "");
column->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << "MATERIALIZE COLUMN " << (settings.hilite ? hilite_none : "");
column->formatImpl(ostr, settings, state, frame);
if (partition)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(ostr, settings, state, frame);
}
}
else if (type == ASTAlterCommand::COMMENT_COLUMN)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "COMMENT COLUMN " << (if_exists ? "IF EXISTS " : "")
ostr << (settings.hilite ? hilite_keyword : "") << "COMMENT COLUMN " << (if_exists ? "IF EXISTS " : "")
<< (settings.hilite ? hilite_none : "");
column->formatImpl(settings, state, frame);
settings.ostr << " " << (settings.hilite ? hilite_none : "");
comment->formatImpl(settings, state, frame);
column->formatImpl(ostr, settings, state, frame);
ostr << " " << (settings.hilite ? hilite_none : "");
comment->formatImpl(ostr, settings, state, frame);
}
else if (type == ASTAlterCommand::MODIFY_COMMENT)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY COMMENT" << (settings.hilite ? hilite_none : "");
settings.ostr << " " << (settings.hilite ? hilite_none : "");
comment->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY COMMENT" << (settings.hilite ? hilite_none : "");
ostr << " " << (settings.hilite ? hilite_none : "");
comment->formatImpl(ostr, settings, state, frame);
}
else if (type == ASTAlterCommand::MODIFY_ORDER_BY)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY ORDER BY " << (settings.hilite ? hilite_none : "");
order_by->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY ORDER BY " << (settings.hilite ? hilite_none : "");
order_by->formatImpl(ostr, settings, state, frame);
}
else if (type == ASTAlterCommand::MODIFY_SAMPLE_BY)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY SAMPLE BY " << (settings.hilite ? hilite_none : "");
sample_by->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY SAMPLE BY " << (settings.hilite ? hilite_none : "");
sample_by->formatImpl(ostr, settings, state, frame);
}
else if (type == ASTAlterCommand::REMOVE_SAMPLE_BY)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "REMOVE SAMPLE BY" << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << "REMOVE SAMPLE BY" << (settings.hilite ? hilite_none : "");
}
else if (type == ASTAlterCommand::ADD_INDEX)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ADD INDEX " << (if_not_exists ? "IF NOT EXISTS " : "")
ostr << (settings.hilite ? hilite_keyword : "") << "ADD INDEX " << (if_not_exists ? "IF NOT EXISTS " : "")
<< (settings.hilite ? hilite_none : "");
index_decl->formatImpl(settings, state, frame);
index_decl->formatImpl(ostr, settings, state, frame);
if (first)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FIRST " << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << " FIRST " << (settings.hilite ? hilite_none : "");
else if (index) /// AFTER
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AFTER " << (settings.hilite ? hilite_none : "");
index->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " AFTER " << (settings.hilite ? hilite_none : "");
index->formatImpl(ostr, settings, state, frame);
}
}
else if (type == ASTAlterCommand::DROP_INDEX)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << (clear_index ? "CLEAR " : "DROP ") << "INDEX "
ostr << (settings.hilite ? hilite_keyword : "") << (clear_index ? "CLEAR " : "DROP ") << "INDEX "
<< (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : "");
index->formatImpl(settings, state, frame);
index->formatImpl(ostr, settings, state, frame);
if (partition)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(ostr, settings, state, frame);
}
}
else if (type == ASTAlterCommand::MATERIALIZE_INDEX)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "MATERIALIZE INDEX " << (settings.hilite ? hilite_none : "");
index->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << "MATERIALIZE INDEX " << (settings.hilite ? hilite_none : "");
index->formatImpl(ostr, settings, state, frame);
if (partition)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(ostr, settings, state, frame);
}
}
else if (type == ASTAlterCommand::ADD_STATISTICS)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ADD STATISTICS " << (if_not_exists ? "IF NOT EXISTS " : "")
ostr << (settings.hilite ? hilite_keyword : "") << "ADD STATISTICS " << (if_not_exists ? "IF NOT EXISTS " : "")
<< (settings.hilite ? hilite_none : "");
statistics_decl->formatImpl(settings, state, frame);
statistics_decl->formatImpl(ostr, settings, state, frame);
}
else if (type == ASTAlterCommand::MODIFY_STATISTICS)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY STATISTICS "
ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY STATISTICS "
<< (settings.hilite ? hilite_none : "");
statistics_decl->formatImpl(settings, state, frame);
statistics_decl->formatImpl(ostr, settings, state, frame);
}
else if (type == ASTAlterCommand::DROP_STATISTICS)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << (clear_statistics ? "CLEAR " : "DROP ") << "STATISTICS "
ostr << (settings.hilite ? hilite_keyword : "") << (clear_statistics ? "CLEAR " : "DROP ") << "STATISTICS "
<< (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : "");
statistics_decl->formatImpl(settings, state, frame);
statistics_decl->formatImpl(ostr, settings, state, frame);
if (partition)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(ostr, settings, state, frame);
}
}
else if (type == ASTAlterCommand::MATERIALIZE_STATISTICS)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "MATERIALIZE STATISTICS " << (settings.hilite ? hilite_none : "");
statistics_decl->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << "MATERIALIZE STATISTICS " << (settings.hilite ? hilite_none : "");
statistics_decl->formatImpl(ostr, settings, state, frame);
if (partition)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(ostr, settings, state, frame);
}
}
else if (type == ASTAlterCommand::ADD_CONSTRAINT)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ADD CONSTRAINT " << (if_not_exists ? "IF NOT EXISTS " : "")
ostr << (settings.hilite ? hilite_keyword : "") << "ADD CONSTRAINT " << (if_not_exists ? "IF NOT EXISTS " : "")
<< (settings.hilite ? hilite_none : "");
constraint_decl->formatImpl(settings, state, frame);
constraint_decl->formatImpl(ostr, settings, state, frame);
}
else if (type == ASTAlterCommand::DROP_CONSTRAINT)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP CONSTRAINT " << (if_exists ? "IF EXISTS " : "")
ostr << (settings.hilite ? hilite_keyword : "") << "DROP CONSTRAINT " << (if_exists ? "IF EXISTS " : "")
<< (settings.hilite ? hilite_none : "");
constraint->formatImpl(settings, state, frame);
constraint->formatImpl(ostr, settings, state, frame);
}
else if (type == ASTAlterCommand::ADD_PROJECTION)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ADD PROJECTION " << (if_not_exists ? "IF NOT EXISTS " : "")
ostr << (settings.hilite ? hilite_keyword : "") << "ADD PROJECTION " << (if_not_exists ? "IF NOT EXISTS " : "")
<< (settings.hilite ? hilite_none : "");
projection_decl->formatImpl(settings, state, frame);
projection_decl->formatImpl(ostr, settings, state, frame);
if (first)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FIRST " << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << " FIRST " << (settings.hilite ? hilite_none : "");
else if (projection)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AFTER " << (settings.hilite ? hilite_none : "");
projection->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " AFTER " << (settings.hilite ? hilite_none : "");
projection->formatImpl(ostr, settings, state, frame);
}
}
else if (type == ASTAlterCommand::DROP_PROJECTION)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << (clear_projection ? "CLEAR " : "DROP ") << "PROJECTION "
ostr << (settings.hilite ? hilite_keyword : "") << (clear_projection ? "CLEAR " : "DROP ") << "PROJECTION "
<< (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : "");
projection->formatImpl(settings, state, frame);
projection->formatImpl(ostr, settings, state, frame);
if (partition)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(ostr, settings, state, frame);
}
}
else if (type == ASTAlterCommand::MATERIALIZE_PROJECTION)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "MATERIALIZE PROJECTION " << (settings.hilite ? hilite_none : "");
projection->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << "MATERIALIZE PROJECTION " << (settings.hilite ? hilite_none : "");
projection->formatImpl(ostr, settings, state, frame);
if (partition)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(ostr, settings, state, frame);
}
}
else if (type == ASTAlterCommand::DROP_PARTITION)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << (detach ? "DETACH" : "DROP") << (part ? " PART " : " PARTITION ")
ostr << (settings.hilite ? hilite_keyword : "") << (detach ? "DETACH" : "DROP") << (part ? " PART " : " PARTITION ")
<< (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
partition->formatImpl(ostr, settings, state, frame);
}
else if (type == ASTAlterCommand::DROP_DETACHED_PARTITION)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP DETACHED" << (part ? " PART " : " PARTITION ")
ostr << (settings.hilite ? hilite_keyword : "") << "DROP DETACHED" << (part ? " PART " : " PARTITION ")
<< (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
partition->formatImpl(ostr, settings, state, frame);
}
else if (type == ASTAlterCommand::FORGET_PARTITION)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "FORGET PARTITION "
ostr << (settings.hilite ? hilite_keyword : "") << "FORGET PARTITION "
<< (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
partition->formatImpl(ostr, settings, state, frame);
}
else if (type == ASTAlterCommand::ATTACH_PARTITION)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ATTACH " << (part ? "PART " : "PARTITION ")
ostr << (settings.hilite ? hilite_keyword : "") << "ATTACH " << (part ? "PART " : "PARTITION ")
<< (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
partition->formatImpl(ostr, settings, state, frame);
}
else if (type == ASTAlterCommand::MOVE_PARTITION)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "MOVE " << (part ? "PART " : "PARTITION ")
ostr << (settings.hilite ? hilite_keyword : "") << "MOVE " << (part ? "PART " : "PARTITION ")
<< (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
settings.ostr << " TO ";
partition->formatImpl(ostr, settings, state, frame);
ostr << " TO ";
switch (move_destination_type)
{
case DataDestinationType::DISK:
settings.ostr << "DISK ";
ostr << "DISK ";
break;
case DataDestinationType::VOLUME:
settings.ostr << "VOLUME ";
ostr << "VOLUME ";
break;
case DataDestinationType::TABLE:
settings.ostr << "TABLE ";
ostr << "TABLE ";
if (!to_database.empty())
{
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(to_database)
ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(to_database)
<< (settings.hilite ? hilite_none : "") << ".";
}
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(to_table)
ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(to_table)
<< (settings.hilite ? hilite_none : "");
return;
default:
@ -340,165 +340,165 @@ void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState &
}
if (move_destination_type != DataDestinationType::TABLE)
{
settings.ostr << quoteString(move_destination_name);
ostr << quoteString(move_destination_name);
}
}
else if (type == ASTAlterCommand::REPLACE_PARTITION)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << (replace ? "REPLACE" : "ATTACH") << " PARTITION "
ostr << (settings.hilite ? hilite_keyword : "") << (replace ? "REPLACE" : "ATTACH") << " PARTITION "
<< (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "");
partition->formatImpl(ostr, settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "");
if (!from_database.empty())
{
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(from_database)
ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(from_database)
<< (settings.hilite ? hilite_none : "") << ".";
}
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(from_table) << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(from_table) << (settings.hilite ? hilite_none : "");
}
else if (type == ASTAlterCommand::FETCH_PARTITION)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "FETCH " << (part ? "PART " : "PARTITION ")
ostr << (settings.hilite ? hilite_keyword : "") << "FETCH " << (part ? "PART " : "PARTITION ")
<< (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "") << DB::quote << from;
partition->formatImpl(ostr, settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "") << DB::quote << from;
}
else if (type == ASTAlterCommand::FREEZE_PARTITION)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "FREEZE PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << "FREEZE PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(ostr, settings, state, frame);
if (!with_name.empty())
{
settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << "WITH NAME" << (settings.hilite ? hilite_none : "") << " "
ostr << " " << (settings.hilite ? hilite_keyword : "") << "WITH NAME" << (settings.hilite ? hilite_none : "") << " "
<< DB::quote << with_name;
}
}
else if (type == ASTAlterCommand::FREEZE_ALL)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "FREEZE" << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << "FREEZE" << (settings.hilite ? hilite_none : "");
if (!with_name.empty())
{
settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << "WITH NAME" << (settings.hilite ? hilite_none : "") << " "
ostr << " " << (settings.hilite ? hilite_keyword : "") << "WITH NAME" << (settings.hilite ? hilite_none : "") << " "
<< DB::quote << with_name;
}
}
else if (type == ASTAlterCommand::UNFREEZE_PARTITION)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "UNFREEZE PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << "UNFREEZE PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(ostr, settings, state, frame);
if (!with_name.empty())
{
settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << "WITH NAME" << (settings.hilite ? hilite_none : "") << " "
ostr << " " << (settings.hilite ? hilite_keyword : "") << "WITH NAME" << (settings.hilite ? hilite_none : "") << " "
<< DB::quote << with_name;
}
}
else if (type == ASTAlterCommand::UNFREEZE_ALL)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "UNFREEZE" << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << "UNFREEZE" << (settings.hilite ? hilite_none : "");
if (!with_name.empty())
{
settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << "WITH NAME" << (settings.hilite ? hilite_none : "") << " "
ostr << " " << (settings.hilite ? hilite_keyword : "") << "WITH NAME" << (settings.hilite ? hilite_none : "") << " "
<< DB::quote << with_name;
}
}
else if (type == ASTAlterCommand::DELETE)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "DELETE" << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << "DELETE" << (settings.hilite ? hilite_none : "");
if (partition)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(ostr, settings, state, frame);
}
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : "");
predicate->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : "");
predicate->formatImpl(ostr, settings, state, frame);
}
else if (type == ASTAlterCommand::UPDATE)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "UPDATE " << (settings.hilite ? hilite_none : "");
update_assignments->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << "UPDATE " << (settings.hilite ? hilite_none : "");
update_assignments->formatImpl(ostr, settings, state, frame);
if (partition)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(ostr, settings, state, frame);
}
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : "");
predicate->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : "");
predicate->formatImpl(ostr, settings, state, frame);
}
else if (type == ASTAlterCommand::MODIFY_TTL)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY TTL " << (settings.hilite ? hilite_none : "");
ttl->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY TTL " << (settings.hilite ? hilite_none : "");
ttl->formatImpl(ostr, settings, state, frame);
}
else if (type == ASTAlterCommand::REMOVE_TTL)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "REMOVE TTL" << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << "REMOVE TTL" << (settings.hilite ? hilite_none : "");
}
else if (type == ASTAlterCommand::MATERIALIZE_TTL)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "MATERIALIZE TTL" << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << "MATERIALIZE TTL" << (settings.hilite ? hilite_none : "");
if (partition)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(ostr, settings, state, frame);
}
}
else if (type == ASTAlterCommand::MODIFY_SETTING)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY SETTING " << (settings.hilite ? hilite_none : "");
settings_changes->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY SETTING " << (settings.hilite ? hilite_none : "");
settings_changes->formatImpl(ostr, settings, state, frame);
}
else if (type == ASTAlterCommand::RESET_SETTING)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "RESET SETTING " << (settings.hilite ? hilite_none : "");
settings_resets->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << "RESET SETTING " << (settings.hilite ? hilite_none : "");
settings_resets->formatImpl(ostr, settings, state, frame);
}
else if (type == ASTAlterCommand::MODIFY_DATABASE_SETTING)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY SETTING " << (settings.hilite ? hilite_none : "");
settings_changes->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY SETTING " << (settings.hilite ? hilite_none : "");
settings_changes->formatImpl(ostr, settings, state, frame);
}
else if (type == ASTAlterCommand::MODIFY_QUERY)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY QUERY" << settings.nl_or_ws
ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY QUERY" << settings.nl_or_ws
<< (settings.hilite ? hilite_none : "");
select->formatImpl(settings, state, frame);
select->formatImpl(ostr, settings, state, frame);
}
else if (type == ASTAlterCommand::MODIFY_REFRESH)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY" << settings.nl_or_ws
ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY" << settings.nl_or_ws
<< (settings.hilite ? hilite_none : "");
refresh->formatImpl(settings, state, frame);
refresh->formatImpl(ostr, settings, state, frame);
}
else if (type == ASTAlterCommand::RENAME_COLUMN)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "RENAME COLUMN " << (if_exists ? "IF EXISTS " : "")
ostr << (settings.hilite ? hilite_keyword : "") << "RENAME COLUMN " << (if_exists ? "IF EXISTS " : "")
<< (settings.hilite ? hilite_none : "");
column->formatImpl(settings, state, frame);
column->formatImpl(ostr, settings, state, frame);
settings.ostr << (settings.hilite ? hilite_keyword : "") << " TO ";
rename_to->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " TO ";
rename_to->formatImpl(ostr, settings, state, frame);
}
else if (type == ASTAlterCommand::MODIFY_SQL_SECURITY)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY " << (settings.hilite ? hilite_none : "");
sql_security->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY " << (settings.hilite ? hilite_none : "");
sql_security->formatImpl(ostr, settings, state, frame);
}
else if (type == ASTAlterCommand::APPLY_DELETED_MASK)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "APPLY DELETED MASK" << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << "APPLY DELETED MASK" << (settings.hilite ? hilite_none : "");
if (partition)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(ostr, settings, state, frame);
}
}
else
@ -615,58 +615,58 @@ ASTPtr ASTAlterQuery::clone() const
return res;
}
void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTAlterQuery::formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
frame.need_parens = false;
std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' ');
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str;
ostr << (settings.hilite ? hilite_keyword : "") << indent_str;
switch (alter_object)
{
case AlterObjectType::TABLE:
settings.ostr << "ALTER TABLE ";
ostr << "ALTER TABLE ";
break;
case AlterObjectType::DATABASE:
settings.ostr << "ALTER DATABASE ";
ostr << "ALTER DATABASE ";
break;
default:
break;
}
settings.ostr << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_none : "");
if (table)
{
settings.ostr << indent_str;
ostr << indent_str;
if (database)
{
database->formatImpl(settings, state, frame);
settings.ostr << '.';
database->formatImpl(ostr, settings, state, frame);
ostr << '.';
}
chassert(table);
table->formatImpl(settings, state, frame);
table->formatImpl(ostr, settings, state, frame);
}
else if (alter_object == AlterObjectType::DATABASE && database)
{
settings.ostr << indent_str;
database->formatImpl(settings, state, frame);
ostr << indent_str;
database->formatImpl(ostr, settings, state, frame);
}
formatOnCluster(settings);
formatOnCluster(ostr, settings);
FormatStateStacked frame_nested = frame;
frame_nested.need_parens = false;
if (settings.one_line)
{
frame_nested.expression_list_prepend_whitespace = true;
command_list->formatImpl(settings, state, frame_nested);
command_list->formatImpl(ostr, settings, state, frame_nested);
}
else
{
frame_nested.expression_list_always_start_on_new_line = true;
command_list->as<ASTExpressionList &>().formatImplMultiline(settings, state, frame_nested);
command_list->as<ASTExpressionList &>().formatImplMultiline(ostr, settings, state, frame_nested);
}
}

View File

@ -226,7 +226,7 @@ public:
static void setFormatAlterCommandsWithParentheses(bool value) { format_alter_commands_with_parentheses = value; }
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void forEachPointerToChild(std::function<void(void**)> f) override;
@ -273,7 +273,7 @@ public:
QueryKind getQueryKind() const override { return QueryKind::Alter; }
protected:
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
bool isOneCommandTypeOnly(const ASTAlterCommand::Type & type) const;

View File

@ -26,16 +26,16 @@ public:
}
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
settings.ostr << (settings.hilite ? hilite_identifier : "");
settings.writeIdentifier(column_name, /*ambiguous=*/false);
settings.ostr << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_identifier : "");
settings.writeIdentifier(ostr, column_name, /*ambiguous=*/false);
ostr << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_operator : "") << " = " << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_operator : "") << " = " << (settings.hilite ? hilite_none : "");
expression()->formatImpl(settings, state, frame);
expression()->formatImpl(ostr, settings, state, frame);
}
};

View File

@ -27,19 +27,19 @@ void ASTAsterisk::appendColumnName(WriteBuffer & ostr) const
ostr.write('*');
}
void ASTAsterisk::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTAsterisk::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
if (expression)
{
expression->formatImpl(settings, state, frame);
settings.ostr << ".";
expression->formatImpl(ostr, settings, state, frame);
ostr << ".";
}
settings.ostr << "*";
ostr << "*";
if (transformers)
{
transformers->formatImpl(settings, state, frame);
transformers->formatImpl(ostr, settings, state, frame);
}
}

View File

@ -19,7 +19,7 @@ public:
ASTPtr expression;
ASTPtr transformers;
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};
}

View File

@ -15,162 +15,162 @@ namespace
using Element = ASTBackupQuery::Element;
using ElementType = ASTBackupQuery::ElementType;
void formatPartitions(const ASTs & partitions, const IAST::FormatSettings & format)
void formatPartitions(const ASTs & partitions, WriteBuffer & ostr, const IAST::FormatSettings & format)
{
format.ostr << " " << (format.hilite ? IAST::hilite_keyword : "") << ((partitions.size() == 1) ? "PARTITION" : "PARTITIONS") << " "
ostr << " " << (format.hilite ? IAST::hilite_keyword : "") << ((partitions.size() == 1) ? "PARTITION" : "PARTITIONS") << " "
<< (format.hilite ? IAST::hilite_none : "");
bool need_comma = false;
for (const auto & partition : partitions)
{
if (std::exchange(need_comma, true))
format.ostr << ",";
format.ostr << " ";
partition->format(format);
ostr << ",";
ostr << " ";
partition->format(ostr, format);
}
}
void formatExceptDatabases(const std::set<String> & except_databases, const IAST::FormatSettings & format)
void formatExceptDatabases(const std::set<String> & except_databases, WriteBuffer & ostr, const IAST::FormatSettings & format)
{
if (except_databases.empty())
return;
format.ostr << (format.hilite ? IAST::hilite_keyword : "") << " EXCEPT "
ostr << (format.hilite ? IAST::hilite_keyword : "") << " EXCEPT "
<< (except_databases.size() == 1 ? "DATABASE" : "DATABASES") << " " << (format.hilite ? IAST::hilite_none : "");
bool need_comma = false;
for (const auto & database_name : except_databases)
{
if (std::exchange(need_comma, true))
format.ostr << ",";
format.ostr << backQuoteIfNeed(database_name);
ostr << ",";
ostr << backQuoteIfNeed(database_name);
}
}
void formatExceptTables(const std::set<DatabaseAndTableName> & except_tables, const IAST::FormatSettings & format, bool only_table_names=false)
void formatExceptTables(const std::set<DatabaseAndTableName> & except_tables, WriteBuffer & ostr, const IAST::FormatSettings & format, bool only_table_names=false)
{
if (except_tables.empty())
return;
format.ostr << (format.hilite ? IAST::hilite_keyword : "") << " EXCEPT " << (except_tables.size() == 1 ? "TABLE" : "TABLES") << " "
ostr << (format.hilite ? IAST::hilite_keyword : "") << " EXCEPT " << (except_tables.size() == 1 ? "TABLE" : "TABLES") << " "
<< (format.hilite ? IAST::hilite_none : "");
bool need_comma = false;
for (const auto & table_name : except_tables)
{
if (std::exchange(need_comma, true))
format.ostr << ", ";
ostr << ", ";
if (!table_name.first.empty() && !only_table_names)
format.ostr << backQuoteIfNeed(table_name.first) << ".";
format.ostr << backQuoteIfNeed(table_name.second);
ostr << backQuoteIfNeed(table_name.first) << ".";
ostr << backQuoteIfNeed(table_name.second);
}
}
void formatElement(const Element & element, const IAST::FormatSettings & format)
void formatElement(const Element & element, WriteBuffer & ostr, const IAST::FormatSettings & format)
{
switch (element.type)
{
case ElementType::TABLE:
{
format.ostr << (format.hilite ? IAST::hilite_keyword : "") << "TABLE " << (format.hilite ? IAST::hilite_none : "");
ostr << (format.hilite ? IAST::hilite_keyword : "") << "TABLE " << (format.hilite ? IAST::hilite_none : "");
if (!element.database_name.empty())
format.ostr << backQuoteIfNeed(element.database_name) << ".";
format.ostr << backQuoteIfNeed(element.table_name);
ostr << backQuoteIfNeed(element.database_name) << ".";
ostr << backQuoteIfNeed(element.table_name);
if ((element.new_table_name != element.table_name) || (element.new_database_name != element.database_name))
{
format.ostr << (format.hilite ? IAST::hilite_keyword : "") << " AS " << (format.hilite ? IAST::hilite_none : "");
ostr << (format.hilite ? IAST::hilite_keyword : "") << " AS " << (format.hilite ? IAST::hilite_none : "");
if (!element.new_database_name.empty())
format.ostr << backQuoteIfNeed(element.new_database_name) << ".";
format.ostr << backQuoteIfNeed(element.new_table_name);
ostr << backQuoteIfNeed(element.new_database_name) << ".";
ostr << backQuoteIfNeed(element.new_table_name);
}
if (element.partitions)
formatPartitions(*element.partitions, format);
formatPartitions(*element.partitions, ostr, format);
break;
}
case ElementType::TEMPORARY_TABLE:
{
format.ostr << (format.hilite ? IAST::hilite_keyword : "") << "TEMPORARY TABLE " << (format.hilite ? IAST::hilite_none : "");
format.ostr << backQuoteIfNeed(element.table_name);
ostr << (format.hilite ? IAST::hilite_keyword : "") << "TEMPORARY TABLE " << (format.hilite ? IAST::hilite_none : "");
ostr << backQuoteIfNeed(element.table_name);
if (element.new_table_name != element.table_name)
{
format.ostr << (format.hilite ? IAST::hilite_keyword : "") << " AS " << (format.hilite ? IAST::hilite_none : "");
format.ostr << backQuoteIfNeed(element.new_table_name);
ostr << (format.hilite ? IAST::hilite_keyword : "") << " AS " << (format.hilite ? IAST::hilite_none : "");
ostr << backQuoteIfNeed(element.new_table_name);
}
break;
}
case ElementType::DATABASE:
{
format.ostr << (format.hilite ? IAST::hilite_keyword : "");
format.ostr << "DATABASE ";
format.ostr << (format.hilite ? IAST::hilite_none : "");
format.ostr << backQuoteIfNeed(element.database_name);
ostr << (format.hilite ? IAST::hilite_keyword : "");
ostr << "DATABASE ";
ostr << (format.hilite ? IAST::hilite_none : "");
ostr << backQuoteIfNeed(element.database_name);
if (element.new_database_name != element.database_name)
{
format.ostr << (format.hilite ? IAST::hilite_keyword : "") << " AS " << (format.hilite ? IAST::hilite_none : "");
format.ostr << backQuoteIfNeed(element.new_database_name);
ostr << (format.hilite ? IAST::hilite_keyword : "") << " AS " << (format.hilite ? IAST::hilite_none : "");
ostr << backQuoteIfNeed(element.new_database_name);
}
formatExceptTables(element.except_tables, format, /*only_table_names*/true);
formatExceptTables(element.except_tables, ostr, format, /*only_table_names*/true);
break;
}
case ElementType::ALL:
{
format.ostr << (format.hilite ? IAST::hilite_keyword : "") << "ALL" << (format.hilite ? IAST::hilite_none : "");
formatExceptDatabases(element.except_databases, format);
formatExceptTables(element.except_tables, format);
ostr << (format.hilite ? IAST::hilite_keyword : "") << "ALL" << (format.hilite ? IAST::hilite_none : "");
formatExceptDatabases(element.except_databases, ostr, format);
formatExceptTables(element.except_tables, ostr, format);
break;
}
}
}
void formatElements(const std::vector<Element> & elements, const IAST::FormatSettings & format)
void formatElements(const std::vector<Element> & elements, WriteBuffer & ostr, const IAST::FormatSettings & format)
{
bool need_comma = false;
for (const auto & element : elements)
{
if (std::exchange(need_comma, true))
format.ostr << ", ";
formatElement(element, format);
ostr << ", ";
formatElement(element, ostr, format);
}
}
void formatSettings(const ASTPtr & settings, const ASTFunction * base_backup_name, const ASTPtr & cluster_host_ids, const IAST::FormatSettings & format)
void formatSettings(const ASTPtr & settings, const ASTFunction * base_backup_name, const ASTPtr & cluster_host_ids, WriteBuffer & ostr, const IAST::FormatSettings & format)
{
if (!settings && !base_backup_name && !cluster_host_ids)
return;
format.ostr << (format.hilite ? IAST::hilite_keyword : "") << " SETTINGS " << (format.hilite ? IAST::hilite_none : "");
ostr << (format.hilite ? IAST::hilite_keyword : "") << " SETTINGS " << (format.hilite ? IAST::hilite_none : "");
bool empty = true;
if (base_backup_name)
{
format.ostr << "base_backup = ";
base_backup_name->format(format);
ostr << "base_backup = ";
base_backup_name->format(ostr, format);
empty = false;
}
if (settings)
{
if (!empty)
format.ostr << ", ";
settings->format(format);
ostr << ", ";
settings->format(ostr, format);
empty = false;
}
if (cluster_host_ids)
{
if (!empty)
format.ostr << ", ";
format.ostr << "cluster_host_ids = ";
cluster_host_ids->format(format);
ostr << ", ";
ostr << "cluster_host_ids = ";
cluster_host_ids->format(ostr, format);
}
}
@ -267,18 +267,18 @@ ASTPtr ASTBackupQuery::clone() const
}
void ASTBackupQuery::formatQueryImpl(const FormatSettings & fs, FormatState &, FormatStateStacked) const
void ASTBackupQuery::formatQueryImpl(WriteBuffer & ostr, const FormatSettings & fs, FormatState &, FormatStateStacked) const
{
fs.ostr << (fs.hilite ? hilite_keyword : "") << ((kind == Kind::BACKUP) ? "BACKUP " : "RESTORE ") << (fs.hilite ? hilite_none : "");
ostr << (fs.hilite ? hilite_keyword : "") << ((kind == Kind::BACKUP) ? "BACKUP " : "RESTORE ") << (fs.hilite ? hilite_none : "");
formatElements(elements, fs);
formatOnCluster(fs);
formatElements(elements, ostr, fs);
formatOnCluster(ostr, fs);
fs.ostr << (fs.hilite ? hilite_keyword : "") << ((kind == Kind::BACKUP) ? " TO " : " FROM ") << (fs.hilite ? hilite_none : "");
backup_name->format(fs);
ostr << (fs.hilite ? hilite_keyword : "") << ((kind == Kind::BACKUP) ? " TO " : " FROM ") << (fs.hilite ? hilite_none : "");
backup_name->format(ostr, fs);
if (settings || base_backup_name)
formatSettings(settings, base_backup_name, cluster_host_ids, fs);
formatSettings(settings, base_backup_name, cluster_host_ids, ostr, fs);
}
ASTPtr ASTBackupQuery::getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams & params) const

View File

@ -91,7 +91,7 @@ public:
String getID(char) const override;
ASTPtr clone() const override;
void formatQueryImpl(const FormatSettings & fs, FormatState &, FormatStateStacked) const override;
void formatQueryImpl(WriteBuffer & ostr, const FormatSettings & fs, FormatState &, FormatStateStacked) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override;
QueryKind getQueryKind() const override;

View File

@ -36,32 +36,32 @@ struct ASTCheckTableQuery : public ASTQueryWithTableAndOutput
}
protected:
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
void formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "CHECK TABLE " << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "CHECK TABLE " << (settings.hilite ? hilite_none : "");
if (table)
{
if (database)
{
database->formatImpl(settings, state, frame);
settings.ostr << '.';
database->formatImpl(ostr, settings, state, frame);
ostr << '.';
}
chassert(table);
table->formatImpl(settings, state, frame);
table->formatImpl(ostr, settings, state, frame);
}
if (partition)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(ostr, settings, state, frame);
}
if (!part_name.empty())
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " PART " << (settings.hilite ? hilite_none : "")
ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " PART " << (settings.hilite ? hilite_none : "")
<< quoteString(part_name);
}
}
@ -84,10 +84,10 @@ struct ASTCheckAllTablesQuery : public ASTQueryWithOutput
QueryKind getQueryKind() const override { return QueryKind::Check; }
protected:
void formatQueryImpl(const FormatSettings & settings, FormatState & /* state */, FormatStateStacked frame) const override
void formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & /* state */, FormatStateStacked frame) const override
{
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "CHECK ALL TABLES" << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "CHECK ALL TABLES" << (settings.hilite ? hilite_none : "");
}
};

View File

@ -9,13 +9,10 @@ namespace DB
return res;
}
void ASTCollation::formatImpl(const FormatSettings &s, FormatState &state, FormatStateStacked frame) const
void ASTCollation::formatImpl(WriteBuffer & ostr, const FormatSettings &s, FormatState &state, FormatStateStacked frame) const
{
if (collation)
{
collation->formatImpl(s, state, frame);
}
collation->formatImpl(ostr, s, state, frame);
}
}

View File

@ -14,7 +14,7 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
};
}

View File

@ -62,69 +62,69 @@ ASTPtr ASTColumnDeclaration::clone() const
return res;
}
void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, FormatState & state, FormatStateStacked frame) const
void ASTColumnDeclaration::formatImpl(WriteBuffer & ostr, const FormatSettings & format_settings, FormatState & state, FormatStateStacked frame) const
{
frame.need_parens = false;
format_settings.writeIdentifier(name, /*ambiguous=*/true);
format_settings.writeIdentifier(ostr, name, /*ambiguous=*/true);
if (type)
{
format_settings.ostr << ' ';
type->formatImpl(format_settings, state, frame);
ostr << ' ';
type->formatImpl(ostr, format_settings, state, frame);
}
if (null_modifier)
{
format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "")
ostr << ' ' << (format_settings.hilite ? hilite_keyword : "")
<< (*null_modifier ? "" : "NOT ") << "NULL" << (format_settings.hilite ? hilite_none : "");
}
if (default_expression)
{
format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << default_specifier << (format_settings.hilite ? hilite_none : "");
ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << default_specifier << (format_settings.hilite ? hilite_none : "");
if (!ephemeral_default)
{
format_settings.ostr << ' ';
default_expression->formatImpl(format_settings, state, frame);
ostr << ' ';
default_expression->formatImpl(ostr, format_settings, state, frame);
}
}
if (comment)
{
format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "COMMENT" << (format_settings.hilite ? hilite_none : "") << ' ';
comment->formatImpl(format_settings, state, frame);
ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "COMMENT" << (format_settings.hilite ? hilite_none : "") << ' ';
comment->formatImpl(ostr, format_settings, state, frame);
}
if (codec)
{
format_settings.ostr << ' ';
codec->formatImpl(format_settings, state, frame);
ostr << ' ';
codec->formatImpl(ostr, format_settings, state, frame);
}
if (statistics_desc)
{
format_settings.ostr << ' ';
statistics_desc->formatImpl(format_settings, state, frame);
ostr << ' ';
statistics_desc->formatImpl(ostr, format_settings, state, frame);
}
if (ttl)
{
format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "TTL" << (format_settings.hilite ? hilite_none : "") << ' ';
ttl->formatImpl(format_settings, state, frame);
ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "TTL" << (format_settings.hilite ? hilite_none : "") << ' ';
ttl->formatImpl(ostr, format_settings, state, frame);
}
if (collation)
{
format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "COLLATE" << (format_settings.hilite ? hilite_none : "") << ' ';
collation->formatImpl(format_settings, state, frame);
ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "COLLATE" << (format_settings.hilite ? hilite_none : "") << ' ';
collation->formatImpl(ostr, format_settings, state, frame);
}
if (settings)
{
format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "SETTINGS" << (format_settings.hilite ? hilite_none : "") << ' ' << '(';
settings->formatImpl(format_settings, state, frame);
format_settings.ostr << ')';
ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "SETTINGS" << (format_settings.hilite ? hilite_none : "") << ' ' << '(';
settings->formatImpl(ostr, format_settings, state, frame);
ostr << ')';
}
}

View File

@ -28,7 +28,7 @@ public:
String getID(char delim) const override { return "ColumnDeclaration" + (delim + name); }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & format_settings, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & format_settings, FormatState & state, FormatStateStacked frame) const override;
protected:
void forEachPointerToChild(std::function<void(void **)> f) override;

View File

@ -39,23 +39,23 @@ void ASTColumnsRegexpMatcher::updateTreeHashImpl(SipHash & hash_state, bool igno
IAST::updateTreeHashImpl(hash_state, ignore_aliases);
}
void ASTColumnsRegexpMatcher::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTColumnsRegexpMatcher::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "");
ostr << (settings.hilite ? hilite_keyword : "");
if (expression)
{
expression->formatImpl(settings, state, frame);
settings.ostr << ".";
expression->formatImpl(ostr, settings, state, frame);
ostr << ".";
}
settings.ostr << "COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
settings.ostr << quoteString(pattern);
settings.ostr << ")";
ostr << "COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
ostr << quoteString(pattern);
ostr << ")";
if (transformers)
{
transformers->formatImpl(settings, state, frame);
transformers->formatImpl(ostr, settings, state, frame);
}
}
@ -101,31 +101,31 @@ void ASTColumnsListMatcher::appendColumnName(WriteBuffer & ostr) const
writeChar(')', ostr);
}
void ASTColumnsListMatcher::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTColumnsListMatcher::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "");
ostr << (settings.hilite ? hilite_keyword : "");
if (expression)
{
expression->formatImpl(settings, state, frame);
settings.ostr << ".";
expression->formatImpl(ostr, settings, state, frame);
ostr << ".";
}
settings.ostr << "COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
ostr << "COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
for (ASTs::const_iterator it = column_list->children.begin(); it != column_list->children.end(); ++it)
{
if (it != column_list->children.begin())
{
settings.ostr << ", ";
ostr << ", ";
}
(*it)->formatImpl(settings, state, frame);
(*it)->formatImpl(ostr, settings, state, frame);
}
settings.ostr << ")";
ostr << ")";
if (transformers)
{
transformers->formatImpl(settings, state, frame);
transformers->formatImpl(ostr, settings, state, frame);
}
}
@ -167,19 +167,19 @@ void ASTQualifiedColumnsRegexpMatcher::updateTreeHashImpl(SipHash & hash_state,
IAST::updateTreeHashImpl(hash_state, ignore_aliases);
}
void ASTQualifiedColumnsRegexpMatcher::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTQualifiedColumnsRegexpMatcher::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "");
ostr << (settings.hilite ? hilite_keyword : "");
qualifier->formatImpl(settings, state, frame);
qualifier->formatImpl(ostr, settings, state, frame);
settings.ostr << ".COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
settings.ostr << quoteString(pattern);
settings.ostr << ")";
ostr << ".COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
ostr << quoteString(pattern);
ostr << ")";
if (transformers)
{
transformers->formatImpl(settings, state, frame);
transformers->formatImpl(ostr, settings, state, frame);
}
}
@ -214,24 +214,24 @@ void ASTQualifiedColumnsListMatcher::appendColumnName(WriteBuffer & ostr) const
writeChar(')', ostr);
}
void ASTQualifiedColumnsListMatcher::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTQualifiedColumnsListMatcher::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "");
qualifier->formatImpl(settings, state, frame);
settings.ostr << ".COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
ostr << (settings.hilite ? hilite_keyword : "");
qualifier->formatImpl(ostr, settings, state, frame);
ostr << ".COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
for (ASTs::const_iterator it = column_list->children.begin(); it != column_list->children.end(); ++it)
{
if (it != column_list->children.begin())
settings.ostr << ", ";
ostr << ", ";
(*it)->formatImpl(settings, state, frame);
(*it)->formatImpl(ostr, settings, state, frame);
}
settings.ostr << ")";
ostr << ")";
if (transformers)
{
transformers->formatImpl(settings, state, frame);
transformers->formatImpl(ostr, settings, state, frame);
}
}

View File

@ -25,7 +25,7 @@ public:
ASTPtr expression;
ASTPtr transformers;
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
private:
String pattern;
@ -43,7 +43,7 @@ public:
ASTPtr column_list;
ASTPtr transformers;
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};
/// Same as ASTColumnsRegexpMatcher. Qualified identifier is first child.
@ -61,7 +61,7 @@ public:
ASTPtr qualifier;
ASTPtr transformers;
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
private:
String pattern;
@ -79,7 +79,7 @@ public:
ASTPtr column_list;
ASTPtr transformers;
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};
}

View File

@ -20,12 +20,12 @@ namespace ErrorCodes
extern const int CANNOT_COMPILE_REGEXP;
}
void ASTColumnsTransformerList::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTColumnsTransformerList::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
for (const auto & child : children)
{
settings.ostr << ' ';
child->formatImpl(settings, state, frame);
ostr << ' ';
child->formatImpl(ostr, settings, state, frame);
}
}
@ -45,33 +45,33 @@ void IASTColumnsTransformer::transform(const ASTPtr & transformer, ASTs & nodes)
}
}
void ASTColumnsApplyTransformer::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTColumnsApplyTransformer::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "APPLY" << (settings.hilite ? hilite_none : "") << " ";
ostr << (settings.hilite ? hilite_keyword : "") << "APPLY" << (settings.hilite ? hilite_none : "") << " ";
if (!column_name_prefix.empty())
settings.ostr << "(";
ostr << "(";
if (lambda)
{
lambda->formatImpl(settings, state, frame);
lambda->formatImpl(ostr, settings, state, frame);
}
else
{
settings.ostr << func_name;
ostr << func_name;
if (parameters)
{
auto nested_frame = frame;
nested_frame.expression_list_prepend_whitespace = false;
settings.ostr << "(";
parameters->formatImpl(settings, state, nested_frame);
settings.ostr << ")";
ostr << "(";
parameters->formatImpl(ostr, settings, state, nested_frame);
ostr << ")";
}
}
if (!column_name_prefix.empty())
settings.ostr << ", '" << column_name_prefix << "')";
ostr << ", '" << column_name_prefix << "')";
}
void ASTColumnsApplyTransformer::transform(ASTs & nodes) const
@ -164,27 +164,27 @@ void ASTColumnsApplyTransformer::updateTreeHashImpl(SipHash & hash_state, bool i
IAST::updateTreeHashImpl(hash_state, ignore_aliases);
}
void ASTColumnsExceptTransformer::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTColumnsExceptTransformer::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "EXCEPT" << (is_strict ? " STRICT " : " ") << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << "EXCEPT" << (is_strict ? " STRICT " : " ") << (settings.hilite ? hilite_none : "");
if (children.size() > 1)
settings.ostr << "(";
ostr << "(";
for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it)
{
if (it != children.begin())
{
settings.ostr << ", ";
ostr << ", ";
}
(*it)->formatImpl(settings, state, frame);
(*it)->formatImpl(ostr, settings, state, frame);
}
if (pattern)
settings.ostr << quoteString(*pattern);
ostr << quoteString(*pattern);
if (children.size() > 1)
settings.ostr << ")";
ostr << ")";
}
void ASTColumnsExceptTransformer::appendColumnName(WriteBuffer & ostr) const
@ -292,12 +292,12 @@ std::shared_ptr<re2::RE2> ASTColumnsExceptTransformer::getMatcher() const
}
void ASTColumnsReplaceTransformer::Replacement::formatImpl(
const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
assert(children.size() == 1);
children[0]->formatImpl(settings, state, frame);
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(name);
children[0]->formatImpl(ostr, settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(name);
}
void ASTColumnsReplaceTransformer::Replacement::appendColumnName(WriteBuffer & ostr) const
@ -319,19 +319,19 @@ void ASTColumnsReplaceTransformer::Replacement::updateTreeHashImpl(SipHash & has
IAST::updateTreeHashImpl(hash_state, ignore_aliases);
}
void ASTColumnsReplaceTransformer::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTColumnsReplaceTransformer::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "REPLACE" << (is_strict ? " STRICT " : " ") << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << "REPLACE" << (is_strict ? " STRICT " : " ") << (settings.hilite ? hilite_none : "");
settings.ostr << "(";
ostr << "(";
for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it)
{
if (it != children.begin())
settings.ostr << ", ";
ostr << ", ";
(*it)->formatImpl(settings, state, frame);
(*it)->formatImpl(ostr, settings, state, frame);
}
settings.ostr << ")";
ostr << ")";
}
void ASTColumnsReplaceTransformer::appendColumnName(WriteBuffer & ostr) const

View File

@ -24,7 +24,7 @@ public:
}
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
class IASTColumnsTransformer : public IAST
@ -62,7 +62,7 @@ public:
String column_name_prefix;
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};
class ASTColumnsExceptTransformer : public IASTColumnsTransformer
@ -83,7 +83,7 @@ public:
void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
std::optional<String> pattern;
};
@ -107,7 +107,7 @@ public:
String name;
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};
bool is_strict = false;
@ -123,7 +123,7 @@ public:
void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
private:
static void replaceChildren(ASTPtr & node, const ASTPtr & replacement, const String & name);

View File

@ -19,11 +19,11 @@ ASTPtr ASTConstraintDeclaration::clone() const
return res;
}
void ASTConstraintDeclaration::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const
void ASTConstraintDeclaration::formatImpl(WriteBuffer & ostr, const FormatSettings & s, FormatState & state, FormatStateStacked frame) const
{
s.ostr << backQuoteIfNeed(name);
s.ostr << (s.hilite ? hilite_keyword : "") << (type == Type::CHECK ? " CHECK " : " ASSUME ") << (s.hilite ? hilite_none : "");
expr->formatImpl(s, state, frame);
ostr << backQuoteIfNeed(name);
ostr << (s.hilite ? hilite_keyword : "") << (type == Type::CHECK ? " CHECK " : " ASSUME ") << (s.hilite ? hilite_none : "");
expr->formatImpl(ostr, s, state, frame);
}
}

View File

@ -24,7 +24,7 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
void forEachPointerToChild(std::function<void(void**)> f) override
{

View File

@ -21,26 +21,26 @@ ASTPtr ASTCreateFunctionQuery::clone() const
return res;
}
void ASTCreateFunctionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState & state, IAST::FormatStateStacked frame) const
void ASTCreateFunctionQuery::formatImpl(WriteBuffer & ostr, const IAST::FormatSettings & settings, IAST::FormatState & state, IAST::FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE ";
ostr << (settings.hilite ? hilite_keyword : "") << "CREATE ";
if (or_replace)
settings.ostr << "OR REPLACE ";
ostr << "OR REPLACE ";
settings.ostr << "FUNCTION ";
ostr << "FUNCTION ";
if (if_not_exists)
settings.ostr << "IF NOT EXISTS ";
ostr << "IF NOT EXISTS ";
settings.ostr << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getFunctionName()) << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getFunctionName()) << (settings.hilite ? hilite_none : "");
formatOnCluster(settings);
formatOnCluster(ostr, settings);
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "");
function_core->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "");
function_core->formatImpl(ostr, settings, state, frame);
}
String ASTCreateFunctionQuery::getFunctionName() const

View File

@ -20,7 +20,7 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster<ASTCreateFunctionQuery>(clone()); }

View File

@ -30,37 +30,37 @@ ASTPtr ASTCreateIndexQuery::clone() const
return res;
}
void ASTCreateIndexQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTCreateIndexQuery::formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
frame.need_parens = false;
std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' ');
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str;
ostr << (settings.hilite ? hilite_keyword : "") << indent_str;
settings.ostr << "CREATE " << (unique ? "UNIQUE " : "") << "INDEX " << (if_not_exists ? "IF NOT EXISTS " : "");
index_name->formatImpl(settings, state, frame);
settings.ostr << " ON ";
ostr << "CREATE " << (unique ? "UNIQUE " : "") << "INDEX " << (if_not_exists ? "IF NOT EXISTS " : "");
index_name->formatImpl(ostr, settings, state, frame);
ostr << " ON ";
settings.ostr << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_none : "");
if (table)
{
if (database)
{
database->formatImpl(settings, state, frame);
settings.ostr << '.';
database->formatImpl(ostr, settings, state, frame);
ostr << '.';
}
chassert(table);
table->formatImpl(settings, state, frame);
table->formatImpl(ostr, settings, state, frame);
}
formatOnCluster(settings);
formatOnCluster(ostr, settings);
settings.ostr << " ";
ostr << " ";
index_decl->formatImpl(settings, state, frame);
index_decl->formatImpl(ostr, settings, state, frame);
}
ASTPtr ASTCreateIndexQuery::convertToASTAlterCommand() const

View File

@ -37,7 +37,7 @@ public:
ASTPtr convertToASTAlterCommand() const;
protected:
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
}

View File

@ -15,33 +15,33 @@ ASTPtr ASTCreateNamedCollectionQuery::clone() const
return std::make_shared<ASTCreateNamedCollectionQuery>(*this);
}
void ASTCreateNamedCollectionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
void ASTCreateNamedCollectionQuery::formatImpl(WriteBuffer & ostr, const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE NAMED COLLECTION ";
ostr << (settings.hilite ? hilite_keyword : "") << "CREATE NAMED COLLECTION ";
if (if_not_exists)
settings.ostr << "IF NOT EXISTS ";
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(collection_name) << (settings.hilite ? hilite_none : "");
ostr << "IF NOT EXISTS ";
ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(collection_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(settings);
formatOnCluster(ostr, settings);
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "");
bool first = true;
for (const auto & change : changes)
{
if (!first)
settings.ostr << ", ";
ostr << ", ";
else
first = false;
formatSettingName(change.name, settings.ostr);
formatSettingName(change.name, ostr);
if (settings.show_secrets)
settings.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value);
ostr << " = " << applyVisitor(FieldVisitorToString(), change.value);
else
settings.ostr << " = '[HIDDEN]'";
ostr << " = '[HIDDEN]'";
auto override_value = overridability.find(change.name);
if (override_value != overridability.end())
settings.ostr << " " << (override_value->second ? "" : "NOT ") << "OVERRIDABLE";
ostr << " " << (override_value->second ? "" : "NOT ") << "OVERRIDABLE";
}
}

View File

@ -20,7 +20,7 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster<ASTCreateNamedCollectionQuery>(clone()); }

View File

@ -14,33 +14,33 @@
namespace DB
{
void ASTSQLSecurity::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTSQLSecurity::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
if (!type)
return;
if (definer || is_definer_current_user)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "DEFINER" << (settings.hilite ? hilite_none : "");
settings.ostr << " = ";
ostr << (settings.hilite ? hilite_keyword : "") << "DEFINER" << (settings.hilite ? hilite_none : "");
ostr << " = ";
if (definer)
definer->formatImpl(settings, state, frame);
definer->formatImpl(ostr, settings, state, frame);
else
settings.ostr << "CURRENT_USER";
settings.ostr << " ";
ostr << "CURRENT_USER";
ostr << " ";
}
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SQL SECURITY" << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << "SQL SECURITY" << (settings.hilite ? hilite_none : "");
switch (*type)
{
case SQLSecurityType::INVOKER:
settings.ostr << " INVOKER";
ostr << " INVOKER";
break;
case SQLSecurityType::DEFINER:
settings.ostr << " DEFINER";
ostr << " DEFINER";
break;
case SQLSecurityType::NONE:
settings.ostr << " NONE";
ostr << " NONE";
break;
}
}
@ -69,42 +69,42 @@ ASTPtr ASTStorage::clone() const
return res;
}
void ASTStorage::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const
void ASTStorage::formatImpl(WriteBuffer & ostr, const FormatSettings & s, FormatState & state, FormatStateStacked frame) const
{
if (engine)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "ENGINE" << (s.hilite ? hilite_none : "") << " = ";
engine->formatImpl(s, state, frame);
ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "ENGINE" << (s.hilite ? hilite_none : "") << " = ";
engine->formatImpl(ostr, s, state, frame);
}
if (partition_by)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "PARTITION BY " << (s.hilite ? hilite_none : "");
partition_by->formatImpl(s, state, frame);
ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "PARTITION BY " << (s.hilite ? hilite_none : "");
partition_by->formatImpl(ostr, s, state, frame);
}
if (primary_key)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "PRIMARY KEY " << (s.hilite ? hilite_none : "");
primary_key->formatImpl(s, state, frame);
ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "PRIMARY KEY " << (s.hilite ? hilite_none : "");
primary_key->formatImpl(ostr, s, state, frame);
}
if (order_by)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "ORDER BY " << (s.hilite ? hilite_none : "");
order_by->formatImpl(s, state, frame);
ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "ORDER BY " << (s.hilite ? hilite_none : "");
order_by->formatImpl(ostr, s, state, frame);
}
if (sample_by)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "SAMPLE BY " << (s.hilite ? hilite_none : "");
sample_by->formatImpl(s, state, frame);
ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "SAMPLE BY " << (s.hilite ? hilite_none : "");
sample_by->formatImpl(ostr, s, state, frame);
}
if (ttl_table)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "TTL " << (s.hilite ? hilite_none : "");
ttl_table->formatImpl(s, state, frame);
ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "TTL " << (s.hilite ? hilite_none : "");
ttl_table->formatImpl(ostr, s, state, frame);
}
if (settings)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "SETTINGS " << (s.hilite ? hilite_none : "");
settings->formatImpl(s, state, frame);
ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "SETTINGS " << (s.hilite ? hilite_none : "");
settings->formatImpl(ostr, s, state, frame);
}
}
@ -124,7 +124,7 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
void forEachPointerToChild(std::function<void(void**)> f) override
{
@ -141,20 +141,20 @@ ASTPtr ASTColumnsElement::clone() const
return res;
}
void ASTColumnsElement::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const
void ASTColumnsElement::formatImpl(WriteBuffer & ostr, const FormatSettings & s, FormatState & state, FormatStateStacked frame) const
{
if (!elem)
return;
if (prefix.empty())
{
elem->formatImpl(s, state, frame);
elem->formatImpl(ostr, s, state, frame);
return;
}
s.ostr << (s.hilite ? hilite_keyword : "") << prefix << (s.hilite ? hilite_none : "");
s.ostr << ' ';
elem->formatImpl(s, state, frame);
ostr << (s.hilite ? hilite_keyword : "") << prefix << (s.hilite ? hilite_none : "");
ostr << ' ';
elem->formatImpl(ostr, s, state, frame);
}
@ -178,7 +178,7 @@ ASTPtr ASTColumns::clone() const
return res;
}
void ASTColumns::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const
void ASTColumns::formatImpl(WriteBuffer & ostr, const FormatSettings & s, FormatState & state, FormatStateStacked frame) const
{
ASTExpressionList list;
@ -226,9 +226,9 @@ void ASTColumns::formatImpl(const FormatSettings & s, FormatState & state, Forma
if (!list.children.empty())
{
if (s.one_line)
list.formatImpl(s, state, frame);
list.formatImpl(ostr, s, state, frame);
else
list.formatImplMultiline(s, state, frame);
list.formatImplMultiline(ostr, s, state, frame);
}
}
@ -279,40 +279,40 @@ String ASTCreateQuery::getID(char delim) const
return res;
}
void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTCreateQuery::formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
frame.need_parens = false;
if (database && !table)
{
settings.ostr << (settings.hilite ? hilite_keyword : "")
ostr << (settings.hilite ? hilite_keyword : "")
<< (attach ? "ATTACH DATABASE " : "CREATE DATABASE ")
<< (if_not_exists ? "IF NOT EXISTS " : "")
<< (settings.hilite ? hilite_none : "");
database->formatImpl(settings, state, frame);
database->formatImpl(ostr, settings, state, frame);
if (uuid != UUIDHelpers::Nil)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " UUID " << (settings.hilite ? hilite_none : "")
ostr << (settings.hilite ? hilite_keyword : "") << " UUID " << (settings.hilite ? hilite_none : "")
<< quoteString(toString(uuid));
}
formatOnCluster(settings);
formatOnCluster(ostr, settings);
if (storage)
storage->formatImpl(settings, state, frame);
storage->formatImpl(ostr, settings, state, frame);
if (table_overrides)
{
settings.ostr << settings.nl_or_ws;
table_overrides->formatImpl(settings, state, frame);
ostr << settings.nl_or_ws;
table_overrides->formatImpl(ostr, settings, state, frame);
}
if (comment)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "COMMENT " << (settings.hilite ? hilite_none : "");
comment->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "COMMENT " << (settings.hilite ? hilite_none : "");
comment->formatImpl(ostr, settings, state, frame);
}
return;
@ -340,40 +340,40 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
else if (is_window_view)
what = "WINDOW VIEW";
settings.ostr << (settings.hilite ? hilite_keyword : "") << action << (settings.hilite ? hilite_none : "");
settings.ostr << " ";
settings.ostr << (settings.hilite ? hilite_keyword : "") << (temporary ? "TEMPORARY " : "")
ostr << (settings.hilite ? hilite_keyword : "") << action << (settings.hilite ? hilite_none : "");
ostr << " ";
ostr << (settings.hilite ? hilite_keyword : "") << (temporary ? "TEMPORARY " : "")
<< what << " "
<< (if_not_exists ? "IF NOT EXISTS " : "")
<< (settings.hilite ? hilite_none : "");
if (database)
{
database->formatImpl(settings, state, frame);
settings.ostr << '.';
database->formatImpl(ostr, settings, state, frame);
ostr << '.';
}
chassert(table);
table->formatImpl(settings, state, frame);
table->formatImpl(ostr, settings, state, frame);
if (uuid != UUIDHelpers::Nil)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " UUID " << (settings.hilite ? hilite_none : "")
ostr << (settings.hilite ? hilite_keyword : "") << " UUID " << (settings.hilite ? hilite_none : "")
<< quoteString(toString(uuid));
assert(attach || !attach_from_path);
if (attach_from_path)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "")
ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "")
<< quoteString(*attach_from_path);
if (attach_as_replicated.has_value())
{
if (attach_as_replicated.value())
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS REPLICATED" << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << " AS REPLICATED" << (settings.hilite ? hilite_none : "");
else
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS NOT REPLICATED" << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << " AS NOT REPLICATED" << (settings.hilite ? hilite_none : "");
}
formatOnCluster(settings);
formatOnCluster(ostr, settings);
}
else
{
@ -386,33 +386,33 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
action = "REPLACE";
/// Always DICTIONARY
settings.ostr << (settings.hilite ? hilite_keyword : "") << action << " DICTIONARY "
ostr << (settings.hilite ? hilite_keyword : "") << action << " DICTIONARY "
<< (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : "");
if (database)
{
database->formatImpl(settings, state, frame);
settings.ostr << '.';
database->formatImpl(ostr, settings, state, frame);
ostr << '.';
}
chassert(table);
table->formatImpl(settings, state, frame);
table->formatImpl(ostr, settings, state, frame);
if (uuid != UUIDHelpers::Nil)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " UUID " << (settings.hilite ? hilite_none : "")
ostr << (settings.hilite ? hilite_keyword : "") << " UUID " << (settings.hilite ? hilite_none : "")
<< quoteString(toString(uuid));
formatOnCluster(settings);
formatOnCluster(ostr, settings);
}
if (refresh_strategy)
{
settings.ostr << settings.nl_or_ws;
refresh_strategy->formatImpl(settings, state, frame);
ostr << settings.nl_or_ws;
refresh_strategy->formatImpl(ostr, settings, state, frame);
}
if (auto to_table_id = getTargetTableID(ViewTarget::To))
{
settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << toStringView(Keyword::TO)
ostr << " " << (settings.hilite ? hilite_keyword : "") << toStringView(Keyword::TO)
<< (settings.hilite ? hilite_none : "") << " "
<< (!to_table_id.database_name.empty() ? backQuoteIfNeed(to_table_id.database_name) + "." : "")
<< backQuoteIfNeed(to_table_id.table_name);
@ -420,7 +420,7 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
if (auto to_inner_uuid = getTargetInnerUUID(ViewTarget::To); to_inner_uuid != UUIDHelpers::Nil)
{
settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << toStringView(Keyword::TO_INNER_UUID)
ostr << " " << (settings.hilite ? hilite_keyword : "") << toStringView(Keyword::TO_INNER_UUID)
<< (settings.hilite ? hilite_none : "") << " " << quoteString(toString(to_inner_uuid));
}
@ -430,7 +430,7 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
if (!should_add_empty)
return;
should_add_empty = false;
settings.ostr << (settings.hilite ? hilite_keyword : "") << " EMPTY" << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << " EMPTY" << (settings.hilite ? hilite_none : "");
};
bool should_add_clone = is_clone_as;
@ -439,14 +439,14 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
if (!should_add_clone)
return;
should_add_clone = false;
settings.ostr << (settings.hilite ? hilite_keyword : "") << " CLONE" << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << " CLONE" << (settings.hilite ? hilite_none : "");
};
if (!as_table.empty())
{
add_empty_if_needed();
add_clone_if_needed();
settings.ostr
ostr
<< (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "")
<< (!as_database.empty() ? backQuoteIfNeed(as_database) + "." : "") << backQuoteIfNeed(as_table);
}
@ -456,108 +456,108 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
if (columns_list && !columns_list->empty())
{
frame.expression_list_always_start_on_new_line = true;
settings.ostr << (settings.one_line ? " (" : "\n(");
ostr << (settings.one_line ? " (" : "\n(");
FormatStateStacked frame_nested = frame;
columns_list->formatImpl(settings, state, frame_nested);
settings.ostr << (settings.one_line ? ")" : "\n)");
columns_list->formatImpl(ostr, settings, state, frame_nested);
ostr << (settings.one_line ? ")" : "\n)");
frame.expression_list_always_start_on_new_line = false;
}
add_empty_if_needed();
add_clone_if_needed();
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "");
as_table_function->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "");
as_table_function->formatImpl(ostr, settings, state, frame);
}
frame.expression_list_always_start_on_new_line = true;
if (columns_list && !columns_list->empty() && !as_table_function)
{
settings.ostr << (settings.one_line ? " (" : "\n(");
ostr << (settings.one_line ? " (" : "\n(");
FormatStateStacked frame_nested = frame;
columns_list->formatImpl(settings, state, frame_nested);
settings.ostr << (settings.one_line ? ")" : "\n)");
columns_list->formatImpl(ostr, settings, state, frame_nested);
ostr << (settings.one_line ? ")" : "\n)");
}
if (dictionary_attributes_list)
{
settings.ostr << (settings.one_line ? " (" : "\n(");
ostr << (settings.one_line ? " (" : "\n(");
FormatStateStacked frame_nested = frame;
if (settings.one_line)
dictionary_attributes_list->formatImpl(settings, state, frame_nested);
dictionary_attributes_list->formatImpl(ostr, settings, state, frame_nested);
else
dictionary_attributes_list->formatImplMultiline(settings, state, frame_nested);
settings.ostr << (settings.one_line ? ")" : "\n)");
dictionary_attributes_list->formatImplMultiline(ostr, settings, state, frame_nested);
ostr << (settings.one_line ? ")" : "\n)");
}
frame.expression_list_always_start_on_new_line = false;
if (storage)
storage->formatImpl(settings, state, frame);
storage->formatImpl(ostr, settings, state, frame);
if (auto inner_storage = getTargetInnerEngine(ViewTarget::Inner))
{
settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << toStringView(Keyword::INNER) << (settings.hilite ? hilite_none : "");
inner_storage->formatImpl(settings, state, frame);
ostr << " " << (settings.hilite ? hilite_keyword : "") << toStringView(Keyword::INNER) << (settings.hilite ? hilite_none : "");
inner_storage->formatImpl(ostr, settings, state, frame);
}
if (auto to_storage = getTargetInnerEngine(ViewTarget::To))
to_storage->formatImpl(settings, state, frame);
to_storage->formatImpl(ostr, settings, state, frame);
if (targets)
{
targets->formatTarget(ViewTarget::Data, settings, state, frame);
targets->formatTarget(ViewTarget::Tags, settings, state, frame);
targets->formatTarget(ViewTarget::Metrics, settings, state, frame);
targets->formatTarget(ViewTarget::Data, ostr, settings, state, frame);
targets->formatTarget(ViewTarget::Tags, ostr, settings, state, frame);
targets->formatTarget(ViewTarget::Metrics, ostr, settings, state, frame);
}
if (dictionary)
dictionary->formatImpl(settings, state, frame);
dictionary->formatImpl(ostr, settings, state, frame);
if (is_watermark_strictly_ascending)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WATERMARK STRICTLY_ASCENDING" << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << " WATERMARK STRICTLY_ASCENDING" << (settings.hilite ? hilite_none : "");
}
else if (is_watermark_ascending)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WATERMARK ASCENDING" << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << " WATERMARK ASCENDING" << (settings.hilite ? hilite_none : "");
}
else if (is_watermark_bounded)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WATERMARK " << (settings.hilite ? hilite_none : "");
watermark_function->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " WATERMARK " << (settings.hilite ? hilite_none : "");
watermark_function->formatImpl(ostr, settings, state, frame);
}
if (allowed_lateness)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " ALLOWED_LATENESS " << (settings.hilite ? hilite_none : "");
lateness_function->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " ALLOWED_LATENESS " << (settings.hilite ? hilite_none : "");
lateness_function->formatImpl(ostr, settings, state, frame);
}
if (is_populate)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " POPULATE" << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << " POPULATE" << (settings.hilite ? hilite_none : "");
add_empty_if_needed();
if (sql_security && supportSQLSecurity() && sql_security->as<ASTSQLSecurity &>().type.has_value())
{
settings.ostr << settings.nl_or_ws;
sql_security->formatImpl(settings, state, frame);
ostr << settings.nl_or_ws;
sql_security->formatImpl(ostr, settings, state, frame);
}
if (select)
{
settings.ostr << settings.nl_or_ws;
settings.ostr << (settings.hilite ? hilite_keyword : "") << "AS "
ostr << settings.nl_or_ws;
ostr << (settings.hilite ? hilite_keyword : "") << "AS "
<< (comment ? "(" : "") << (settings.hilite ? hilite_none : "");
select->formatImpl(settings, state, frame);
settings.ostr << (settings.hilite ? hilite_keyword : "") << (comment ? ")" : "") << (settings.hilite ? hilite_none : "");
select->formatImpl(ostr, settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << (comment ? ")" : "") << (settings.hilite ? hilite_none : "");
}
if (comment)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "COMMENT " << (settings.hilite ? hilite_none : "");
comment->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "COMMENT " << (settings.hilite ? hilite_none : "");
comment->formatImpl(ostr, settings, state, frame);
}
}

View File

@ -34,7 +34,7 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
bool isExtendedStorageDefinition() const;
@ -67,7 +67,7 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
bool empty() const
{
@ -179,7 +179,7 @@ public:
bool is_materialized_view_with_inner_table() const { return is_materialized_view && !hasTargetTableID(ViewTarget::To); }
protected:
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void forEachPointerToChild(std::function<void(void**)> f) override
{

View File

@ -20,31 +20,31 @@ ASTPtr ASTCreateResourceQuery::clone() const
return res;
}
void ASTCreateResourceQuery::formatImpl(const IAST::FormatSettings & format, IAST::FormatState &, IAST::FormatStateStacked) const
void ASTCreateResourceQuery::formatImpl(WriteBuffer & ostr, const IAST::FormatSettings & format, IAST::FormatState &, IAST::FormatStateStacked) const
{
format.ostr << (format.hilite ? hilite_keyword : "") << "CREATE ";
ostr << (format.hilite ? hilite_keyword : "") << "CREATE ";
if (or_replace)
format.ostr << "OR REPLACE ";
ostr << "OR REPLACE ";
format.ostr << "RESOURCE ";
ostr << "RESOURCE ";
if (if_not_exists)
format.ostr << "IF NOT EXISTS ";
ostr << "IF NOT EXISTS ";
format.ostr << (format.hilite ? hilite_none : "");
ostr << (format.hilite ? hilite_none : "");
format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getResourceName()) << (format.hilite ? hilite_none : "");
ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getResourceName()) << (format.hilite ? hilite_none : "");
formatOnCluster(format);
formatOnCluster(ostr, format);
format.ostr << " (";
ostr << " (";
bool first = true;
for (const auto & operation : operations)
{
if (!first)
format.ostr << ", ";
ostr << ", ";
else
first = false;
@ -52,25 +52,25 @@ void ASTCreateResourceQuery::formatImpl(const IAST::FormatSettings & format, IAS
{
case AccessMode::Read:
{
format.ostr << (format.hilite ? hilite_keyword : "") << "READ ";
ostr << (format.hilite ? hilite_keyword : "") << "READ ";
break;
}
case AccessMode::Write:
{
format.ostr << (format.hilite ? hilite_keyword : "") << "WRITE ";
ostr << (format.hilite ? hilite_keyword : "") << "WRITE ";
break;
}
}
if (operation.disk)
{
format.ostr << "DISK " << (format.hilite ? hilite_none : "");
format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(*operation.disk) << (format.hilite ? hilite_none : "");
ostr << "DISK " << (format.hilite ? hilite_none : "");
ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(*operation.disk) << (format.hilite ? hilite_none : "");
}
else
format.ostr << "ANY DISK" << (format.hilite ? hilite_none : "");
ostr << "ANY DISK" << (format.hilite ? hilite_none : "");
}
format.ostr << ")";
ostr << ")";
}
String ASTCreateResourceQuery::getResourceName() const

View File

@ -36,7 +36,7 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & format, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & format, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster<ASTCreateResourceQuery>(clone()); }

View File

@ -27,47 +27,47 @@ ASTPtr ASTCreateWorkloadQuery::clone() const
return res;
}
void ASTCreateWorkloadQuery::formatImpl(const IAST::FormatSettings & format, IAST::FormatState &, IAST::FormatStateStacked) const
void ASTCreateWorkloadQuery::formatImpl(WriteBuffer & ostr, const IAST::FormatSettings & format, IAST::FormatState &, IAST::FormatStateStacked) const
{
format.ostr << (format.hilite ? hilite_keyword : "") << "CREATE ";
ostr << (format.hilite ? hilite_keyword : "") << "CREATE ";
if (or_replace)
format.ostr << "OR REPLACE ";
ostr << "OR REPLACE ";
format.ostr << "WORKLOAD ";
ostr << "WORKLOAD ";
if (if_not_exists)
format.ostr << "IF NOT EXISTS ";
ostr << "IF NOT EXISTS ";
format.ostr << (format.hilite ? hilite_none : "");
ostr << (format.hilite ? hilite_none : "");
format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadName()) << (format.hilite ? hilite_none : "");
ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadName()) << (format.hilite ? hilite_none : "");
formatOnCluster(format);
formatOnCluster(ostr, format);
if (hasParent())
{
format.ostr << (format.hilite ? hilite_keyword : "") << " IN " << (format.hilite ? hilite_none : "");
format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadParent()) << (format.hilite ? hilite_none : "");
ostr << (format.hilite ? hilite_keyword : "") << " IN " << (format.hilite ? hilite_none : "");
ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadParent()) << (format.hilite ? hilite_none : "");
}
if (!changes.empty())
{
format.ostr << ' ' << (format.hilite ? hilite_keyword : "") << "SETTINGS" << (format.hilite ? hilite_none : "") << ' ';
ostr << ' ' << (format.hilite ? hilite_keyword : "") << "SETTINGS" << (format.hilite ? hilite_none : "") << ' ';
bool first = true;
for (const auto & change : changes)
{
if (!first)
format.ostr << ", ";
ostr << ", ";
else
first = false;
format.ostr << change.name << " = " << applyVisitor(FieldVisitorToString(), change.value);
ostr << change.name << " = " << applyVisitor(FieldVisitorToString(), change.value);
if (!change.resource.empty())
{
format.ostr << ' ' << (format.hilite ? hilite_keyword : "") << "FOR" << (format.hilite ? hilite_none : "") << ' ';
format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(change.resource) << (format.hilite ? hilite_none : "");
ostr << ' ' << (format.hilite ? hilite_keyword : "") << "FOR" << (format.hilite ? hilite_none : "") << ' ';
ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(change.resource) << (format.hilite ? hilite_none : "");
}
}
}

View File

@ -39,7 +39,7 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & format, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & format, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster<ASTCreateWorkloadQuery>(clone()); }

View File

@ -32,13 +32,13 @@ void ASTDataType::updateTreeHashImpl(SipHash & hash_state, bool) const
/// Children are hashed automatically.
}
void ASTDataType::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTDataType::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_function : "") << name;
ostr << (settings.hilite ? hilite_function : "") << name;
if (arguments && !arguments->children.empty())
{
settings.ostr << '(' << (settings.hilite ? hilite_none : "");
ostr << '(' << (settings.hilite ? hilite_none : "");
if (!settings.one_line && settings.print_pretty_type_names && name == "Tuple")
{
@ -47,21 +47,21 @@ void ASTDataType::formatImpl(const FormatSettings & settings, FormatState & stat
for (size_t i = 0, size = arguments->children.size(); i < size; ++i)
{
if (i != 0)
settings.ostr << ',';
settings.ostr << indent_str;
arguments->children[i]->formatImpl(settings, state, frame);
ostr << ',';
ostr << indent_str;
arguments->children[i]->formatImpl(ostr, settings, state, frame);
}
}
else
{
frame.expression_list_prepend_whitespace = false;
arguments->formatImpl(settings, state, frame);
arguments->formatImpl(ostr, settings, state, frame);
}
settings.ostr << (settings.hilite ? hilite_function : "") << ')';
ostr << (settings.hilite ? hilite_function : "") << ')';
}
settings.ostr << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_none : "");
}
}

View File

@ -16,7 +16,7 @@ public:
String getID(char delim) const override;
ASTPtr clone() const override;
void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
template <typename... Args>

View File

@ -4,14 +4,14 @@
namespace DB
{
void ASTDatabaseOrNone::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
void ASTDatabaseOrNone::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
if (none)
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "NONE" << (settings.hilite ? IAST::hilite_none : "");
ostr << (settings.hilite ? IAST::hilite_keyword : "") << "NONE" << (settings.hilite ? IAST::hilite_none : "");
return;
}
settings.ostr << backQuoteIfNeed(database_name);
ostr << backQuoteIfNeed(database_name);
}
}

View File

@ -14,7 +14,7 @@ public:
bool isNone() const { return none; }
String getID(char) const override { return "DatabaseOrNone"; }
ASTPtr clone() const override { return std::make_shared<ASTDatabaseOrNone>(*this); }
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};
}

View File

@ -30,29 +30,29 @@ ASTPtr ASTDeleteQuery::clone() const
return res;
}
void ASTDeleteQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTDeleteQuery::formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "DELETE FROM " << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << "DELETE FROM " << (settings.hilite ? hilite_none : "");
if (database)
{
database->formatImpl(settings, state, frame);
settings.ostr << '.';
database->formatImpl(ostr, settings, state, frame);
ostr << '.';
}
chassert(table);
table->formatImpl(settings, state, frame);
table->formatImpl(ostr, settings, state, frame);
formatOnCluster(settings);
formatOnCluster(ostr, settings);
if (partition)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(ostr, settings, state, frame);
}
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : "");
predicate->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : "");
predicate->formatImpl(ostr, settings, state, frame);
}
}

View File

@ -27,7 +27,7 @@ public:
ASTPtr predicate;
protected:
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
}

View File

@ -14,9 +14,9 @@ ASTPtr ASTDescribeCacheQuery::clone() const
return res;
}
void ASTDescribeCacheQuery::formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
void ASTDescribeCacheQuery::formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "DESCRIBE FILESYSTEM CACHE" << (settings.hilite ? hilite_none : "")
ostr << (settings.hilite ? hilite_keyword : "") << "DESCRIBE FILESYSTEM CACHE" << (settings.hilite ? hilite_none : "")
<< " " << quoteString(cache_name);
}

View File

@ -15,7 +15,7 @@ public:
ASTPtr clone() const override;
protected:
void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
void formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};
}

View File

@ -16,11 +16,12 @@ ASTPtr ASTDictionaryRange::clone() const
}
void ASTDictionaryRange::formatImpl(const FormatSettings & settings,
void ASTDictionaryRange::formatImpl(WriteBuffer & ostr,
const FormatSettings & settings,
FormatState &,
FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "")
ostr << (settings.hilite ? hilite_keyword : "")
<< "RANGE"
<< (settings.hilite ? hilite_none : "")
<< "("
@ -44,11 +45,12 @@ ASTPtr ASTDictionaryLifetime::clone() const
}
void ASTDictionaryLifetime::formatImpl(const FormatSettings & settings,
void ASTDictionaryLifetime::formatImpl(WriteBuffer & ostr,
const FormatSettings & settings,
FormatState &,
FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "")
ostr << (settings.hilite ? hilite_keyword : "")
<< "LIFETIME"
<< (settings.hilite ? hilite_none : "")
<< "("
@ -73,11 +75,12 @@ ASTPtr ASTDictionaryLayout::clone() const
}
void ASTDictionaryLayout::formatImpl(const FormatSettings & settings,
void ASTDictionaryLayout::formatImpl(WriteBuffer & ostr,
const FormatSettings & settings,
FormatState & state,
FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "")
ostr << (settings.hilite ? hilite_keyword : "")
<< "LAYOUT"
<< (settings.hilite ? hilite_none : "")
<< "("
@ -86,14 +89,14 @@ void ASTDictionaryLayout::formatImpl(const FormatSettings & settings,
<< (settings.hilite ? hilite_none : "");
if (has_brackets)
settings.ostr << "(";
ostr << "(";
if (parameters) parameters->formatImpl(settings, state, frame);
if (parameters) parameters->formatImpl(ostr, settings, state, frame);
if (has_brackets)
settings.ostr << ")";
ostr << ")";
settings.ostr << ")";
ostr << ")";
}
ASTPtr ASTDictionarySettings::clone() const
@ -104,23 +107,24 @@ ASTPtr ASTDictionarySettings::clone() const
return res;
}
void ASTDictionarySettings::formatImpl(const FormatSettings & settings,
FormatState &,
FormatStateStacked) const
void ASTDictionarySettings::formatImpl(WriteBuffer & ostr,
const FormatSettings & settings,
FormatState &,
FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "")
ostr << (settings.hilite ? hilite_keyword : "")
<< "SETTINGS"
<< (settings.hilite ? hilite_none : "")
<< "(";
for (auto it = changes.begin(); it != changes.end(); ++it)
{
if (it != changes.begin())
settings.ostr << ", ";
ostr << ", ";
settings.ostr << it->name << " = " << applyVisitor(FieldVisitorToString(), it->value);
ostr << it->name << " = " << applyVisitor(FieldVisitorToString(), it->value);
}
settings.ostr << (settings.hilite ? hilite_none : "") << ")";
ostr << (settings.hilite ? hilite_none : "") << ")";
}
@ -150,46 +154,46 @@ ASTPtr ASTDictionary::clone() const
}
void ASTDictionary::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTDictionary::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
if (primary_key)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "PRIMARY KEY "
ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "PRIMARY KEY "
<< (settings.hilite ? hilite_none : "");
primary_key->formatImpl(settings, state, frame);
primary_key->formatImpl(ostr, settings, state, frame);
}
if (source)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "SOURCE"
ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "SOURCE"
<< (settings.hilite ? hilite_none : "");
settings.ostr << "(";
source->formatImpl(settings, state, frame);
settings.ostr << ")";
ostr << "(";
source->formatImpl(ostr, settings, state, frame);
ostr << ")";
}
if (lifetime)
{
settings.ostr << settings.nl_or_ws;
lifetime->formatImpl(settings, state, frame);
ostr << settings.nl_or_ws;
lifetime->formatImpl(ostr, settings, state, frame);
}
if (layout)
{
settings.ostr << settings.nl_or_ws;
layout->formatImpl(settings, state, frame);
ostr << settings.nl_or_ws;
layout->formatImpl(ostr, settings, state, frame);
}
if (range)
{
settings.ostr << settings.nl_or_ws;
range->formatImpl(settings, state, frame);
ostr << settings.nl_or_ws;
range->formatImpl(ostr, settings, state, frame);
}
if (dict_settings)
{
settings.ostr << settings.nl_or_ws;
dict_settings->formatImpl(settings, state, frame);
ostr << settings.nl_or_ws;
dict_settings->formatImpl(ostr, settings, state, frame);
}
}

View File

@ -25,7 +25,7 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
/// AST for external dictionary layout. Has name and contain single parameter
@ -46,7 +46,7 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void forEachPointerToChild(std::function<void(void**)> f) override
{
@ -68,7 +68,7 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
class ASTDictionarySettings : public IAST
@ -80,7 +80,7 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
@ -107,7 +107,7 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
}

View File

@ -31,41 +31,41 @@ ASTPtr ASTDictionaryAttributeDeclaration::clone() const
return res;
}
void ASTDictionaryAttributeDeclaration::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTDictionaryAttributeDeclaration::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
frame.need_parens = false;
settings.writeIdentifier(name, /*ambiguous=*/true);
settings.writeIdentifier(ostr, name, /*ambiguous=*/true);
if (type)
{
settings.ostr << ' ';
type->formatImpl(settings, state, frame);
ostr << ' ';
type->formatImpl(ostr, settings, state, frame);
}
if (default_value)
{
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "DEFAULT" << (settings.hilite ? hilite_none : "") << ' ';
default_value->formatImpl(settings, state, frame);
ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "DEFAULT" << (settings.hilite ? hilite_none : "") << ' ';
default_value->formatImpl(ostr, settings, state, frame);
}
if (expression)
{
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "EXPRESSION" << (settings.hilite ? hilite_none : "") << ' ';
expression->formatImpl(settings, state, frame);
ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "EXPRESSION" << (settings.hilite ? hilite_none : "") << ' ';
expression->formatImpl(ostr, settings, state, frame);
}
if (hierarchical)
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "HIERARCHICAL" << (settings.hilite ? hilite_none : "");
ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "HIERARCHICAL" << (settings.hilite ? hilite_none : "");
if (bidirectional)
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "BIDIRECTIONAL" << (settings.hilite ? hilite_none : "");
ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "BIDIRECTIONAL" << (settings.hilite ? hilite_none : "");
if (injective)
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "INJECTIVE" << (settings.hilite ? hilite_none : "");
ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "INJECTIVE" << (settings.hilite ? hilite_none : "");
if (is_object_id)
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "IS_OBJECT_ID" << (settings.hilite ? hilite_none : "");
ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "IS_OBJECT_ID" << (settings.hilite ? hilite_none : "");
}
}

View File

@ -30,7 +30,7 @@ public:
String getID(char delim) const override { return "DictionaryAttributeDeclaration" + (delim + name); }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
}

View File

@ -10,16 +10,16 @@ ASTPtr ASTDropFunctionQuery::clone() const
return std::make_shared<ASTDropFunctionQuery>(*this);
}
void ASTDropFunctionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
void ASTDropFunctionQuery::formatImpl(WriteBuffer & ostr, const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP FUNCTION ";
ostr << (settings.hilite ? hilite_keyword : "") << "DROP FUNCTION ";
if (if_exists)
settings.ostr << "IF EXISTS ";
ostr << "IF EXISTS ";
settings.ostr << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(function_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(settings);
ostr << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(function_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(ostr, settings);
}
}

View File

@ -18,7 +18,7 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster<ASTDropFunctionQuery>(clone()); }

View File

@ -25,33 +25,33 @@ ASTPtr ASTDropIndexQuery::clone() const
return res;
}
void ASTDropIndexQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTDropIndexQuery::formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
frame.need_parens = false;
std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' ');
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str;
ostr << (settings.hilite ? hilite_keyword : "") << indent_str;
settings.ostr << "DROP INDEX " << (if_exists ? "IF EXISTS " : "");
index_name->formatImpl(settings, state, frame);
settings.ostr << " ON ";
ostr << "DROP INDEX " << (if_exists ? "IF EXISTS " : "");
index_name->formatImpl(ostr, settings, state, frame);
ostr << " ON ";
settings.ostr << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_none : "");
if (table)
{
if (database)
{
database->formatImpl(settings, state, frame);
settings.ostr << '.';
database->formatImpl(ostr, settings, state, frame);
ostr << '.';
}
chassert(table);
table->formatImpl(settings, state, frame);
table->formatImpl(ostr, settings, state, frame);
}
formatOnCluster(settings);
formatOnCluster(ostr, settings);
}
ASTPtr ASTDropIndexQuery::convertToASTAlterCommand() const

View File

@ -37,7 +37,7 @@ public:
ASTPtr convertToASTAlterCommand() const;
protected:
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
}

View File

@ -10,13 +10,13 @@ ASTPtr ASTDropNamedCollectionQuery::clone() const
return std::make_shared<ASTDropNamedCollectionQuery>(*this);
}
void ASTDropNamedCollectionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
void ASTDropNamedCollectionQuery::formatImpl(WriteBuffer & ostr, const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP NAMED COLLECTION ";
ostr << (settings.hilite ? hilite_keyword : "") << "DROP NAMED COLLECTION ";
if (if_exists)
settings.ostr << "IF EXISTS ";
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(collection_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(settings);
ostr << "IF EXISTS ";
ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(collection_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(ostr, settings);
}
}

View File

@ -17,7 +17,7 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster<ASTDropNamedCollectionQuery>(clone()); }

View File

@ -33,43 +33,43 @@ ASTPtr ASTDropQuery::clone() const
return res;
}
void ASTDropQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTDropQuery::formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "");
ostr << (settings.hilite ? hilite_keyword : "");
if (kind == ASTDropQuery::Kind::Drop)
settings.ostr << "DROP ";
ostr << "DROP ";
else if (kind == ASTDropQuery::Kind::Detach)
settings.ostr << "DETACH ";
ostr << "DETACH ";
else if (kind == ASTDropQuery::Kind::Truncate)
settings.ostr << "TRUNCATE ";
ostr << "TRUNCATE ";
else
throw Exception(ErrorCodes::SYNTAX_ERROR, "Not supported kind of drop query.");
if (temporary)
settings.ostr << "TEMPORARY ";
ostr << "TEMPORARY ";
if (has_all_tables)
settings.ostr << "ALL TABLES FROM ";
ostr << "ALL TABLES FROM ";
else if (!table && !database_and_tables && database)
settings.ostr << "DATABASE ";
ostr << "DATABASE ";
else if (is_dictionary)
settings.ostr << "DICTIONARY ";
ostr << "DICTIONARY ";
else if (is_view)
settings.ostr << "VIEW ";
ostr << "VIEW ";
else
settings.ostr << "TABLE ";
ostr << "TABLE ";
if (if_exists)
settings.ostr << "IF EXISTS ";
ostr << "IF EXISTS ";
if (if_empty)
settings.ostr << "IF EMPTY ";
ostr << "IF EMPTY ";
settings.ostr << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_none : "");
if (!table && !database_and_tables && database)
{
database->formatImpl(settings, state, frame);
database->formatImpl(ostr, settings, state, frame);
}
else if (database_and_tables)
{
@ -77,7 +77,7 @@ void ASTDropQuery::formatQueryImpl(const FormatSettings & settings, FormatState
for (auto * it = list.children.begin(); it != list.children.end(); ++it)
{
if (it != list.children.begin())
settings.ostr << ", ";
ostr << ", ";
auto identifier = dynamic_pointer_cast<ASTTableIdentifier>(*it);
if (!identifier)
@ -85,34 +85,34 @@ void ASTDropQuery::formatQueryImpl(const FormatSettings & settings, FormatState
if (auto db = identifier->getDatabase())
{
db->formatImpl(settings, state, frame);
settings.ostr << '.';
db->formatImpl(ostr, settings, state, frame);
ostr << '.';
}
auto tb = identifier->getTable();
chassert(tb);
tb->formatImpl(settings, state, frame);
tb->formatImpl(ostr, settings, state, frame);
}
}
else
{
if (database)
{
database->formatImpl(settings, state, frame);
settings.ostr << '.';
database->formatImpl(ostr, settings, state, frame);
ostr << '.';
}
chassert(table);
table->formatImpl(settings, state, frame);
table->formatImpl(ostr, settings, state, frame);
}
formatOnCluster(settings);
formatOnCluster(ostr, settings);
if (permanently)
settings.ostr << " PERMANENTLY";
ostr << " PERMANENTLY";
if (sync)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " SYNC" << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << " SYNC" << (settings.hilite ? hilite_none : "");
}
ASTs ASTDropQuery::getRewrittenASTsOfSingleTable()

View File

@ -58,7 +58,7 @@ public:
QueryKind getQueryKind() const override { return QueryKind::Drop; }
protected:
void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
void formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};
}

View File

@ -10,16 +10,16 @@ ASTPtr ASTDropResourceQuery::clone() const
return std::make_shared<ASTDropResourceQuery>(*this);
}
void ASTDropResourceQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
void ASTDropResourceQuery::formatImpl(WriteBuffer & ostr, const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP RESOURCE ";
ostr << (settings.hilite ? hilite_keyword : "") << "DROP RESOURCE ";
if (if_exists)
settings.ostr << "IF EXISTS ";
ostr << "IF EXISTS ";
settings.ostr << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(resource_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(settings);
ostr << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(resource_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(ostr, settings);
}
}

View File

@ -18,7 +18,7 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster<ASTDropResourceQuery>(clone()); }

View File

@ -10,16 +10,16 @@ ASTPtr ASTDropWorkloadQuery::clone() const
return std::make_shared<ASTDropWorkloadQuery>(*this);
}
void ASTDropWorkloadQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
void ASTDropWorkloadQuery::formatImpl(WriteBuffer & ostr, const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP WORKLOAD ";
ostr << (settings.hilite ? hilite_keyword : "") << "DROP WORKLOAD ";
if (if_exists)
settings.ostr << "IF EXISTS ";
ostr << "IF EXISTS ";
settings.ostr << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(workload_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(settings);
ostr << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(workload_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(ostr, settings);
}
}

View File

@ -18,7 +18,7 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster<ASTDropWorkloadQuery>(clone()); }

View File

@ -112,30 +112,30 @@ public:
QueryKind getQueryKind() const override { return QueryKind::Explain; }
protected:
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
void formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << toString(kind) << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << toString(kind) << (settings.hilite ? hilite_none : "");
if (ast_settings)
{
settings.ostr << ' ';
ast_settings->formatImpl(settings, state, frame);
ostr << ' ';
ast_settings->formatImpl(ostr, settings, state, frame);
}
if (query)
{
settings.ostr << settings.nl_or_ws;
query->formatImpl(settings, state, frame);
ostr << settings.nl_or_ws;
query->formatImpl(ostr, settings, state, frame);
}
if (table_function)
{
settings.ostr << settings.nl_or_ws;
table_function->formatImpl(settings, state, frame);
ostr << settings.nl_or_ws;
table_function->formatImpl(ostr, settings, state, frame);
}
if (table_override)
{
settings.ostr << settings.nl_or_ws;
table_override->formatImpl(settings, state, frame);
ostr << settings.nl_or_ws;
table_override->formatImpl(ostr, settings, state, frame);
}
}

View File

@ -12,18 +12,18 @@ ASTPtr ASTExpressionList::clone() const
return clone;
}
void ASTExpressionList::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTExpressionList::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
if (frame.expression_list_prepend_whitespace)
settings.ostr << ' ';
ostr << ' ';
for (size_t i = 0, size = children.size(); i < size; ++i)
{
if (i)
{
if (separator)
settings.ostr << separator;
settings.ostr << ' ';
ostr << separator;
ostr << ' ';
}
FormatStateStacked frame_nested = frame;
@ -31,16 +31,16 @@ void ASTExpressionList::formatImpl(const FormatSettings & settings, FormatState
frame_nested.list_element_index = i;
if (frame.surround_each_list_element_with_parens)
settings.ostr << "(";
ostr << "(";
children[i]->formatImpl(settings, state, frame_nested);
children[i]->formatImpl(ostr, settings, state, frame_nested);
if (frame.surround_each_list_element_with_parens)
settings.ostr << ")";
ostr << ")";
}
}
void ASTExpressionList::formatImplMultiline(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTExpressionList::formatImplMultiline(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
++frame.indent;
std::string indent_str = "\n" + std::string(4 * frame.indent, ' ');
@ -48,16 +48,16 @@ void ASTExpressionList::formatImplMultiline(const FormatSettings & settings, For
if (frame.expression_list_prepend_whitespace)
{
if (!(children.size() > 1 || frame.expression_list_always_start_on_new_line))
settings.ostr << ' ';
ostr << ' ';
}
for (size_t i = 0, size = children.size(); i < size; ++i)
{
if (i && separator)
settings.ostr << separator;
ostr << separator;
if (size > 1 || frame.expression_list_always_start_on_new_line)
settings.ostr << indent_str;
ostr << indent_str;
FormatStateStacked frame_nested = frame;
frame_nested.expression_list_always_start_on_new_line = false;
@ -65,12 +65,12 @@ void ASTExpressionList::formatImplMultiline(const FormatSettings & settings, For
frame_nested.list_element_index = i;
if (frame.surround_each_list_element_with_parens)
settings.ostr << "(";
ostr << "(";
children[i]->formatImpl(settings, state, frame_nested);
children[i]->formatImpl(ostr, settings, state, frame_nested);
if (frame.surround_each_list_element_with_parens)
settings.ostr << ")";
ostr << ")";
}
}

View File

@ -16,8 +16,8 @@ public:
String getID(char) const override { return "ExpressionList"; }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImplMultiline(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImplMultiline(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const;
char separator;
};

View File

@ -33,11 +33,11 @@ public:
String getID(char) const override { return "external ddl query"; }
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked stacked) const override
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked stacked) const override
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "EXTERNAL DDL FROM " << (settings.hilite ? hilite_none : "");
from->formatImpl(settings, state, stacked);
external_ddl->formatImpl(settings, state, stacked);
ostr << (settings.hilite ? hilite_keyword : "") << "EXTERNAL DDL FROM " << (settings.hilite ? hilite_none : "");
from->formatImpl(ostr, settings, state, stacked);
external_ddl->formatImpl(ostr, settings, state, stacked);
}
QueryKind getQueryKind() const override { return QueryKind::ExternalDDL; }

View File

@ -91,36 +91,36 @@ void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const
}
else
{
FormatSettings format_settings{ostr, true /* one_line */};
FormatSettings format_settings{true /* one_line */};
FormatState state;
FormatStateStacked frame;
writeCString("(", ostr);
window_definition->formatImpl(format_settings, state, frame);
window_definition->formatImpl(ostr, format_settings, state, frame);
writeCString(")", ostr);
}
}
}
void ASTFunction::finishFormatWithWindow(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTFunction::finishFormatWithWindow(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
if (nulls_action == NullsAction::RESPECT_NULLS)
settings.ostr << " RESPECT NULLS";
ostr << " RESPECT NULLS";
else if (nulls_action == NullsAction::IGNORE_NULLS)
settings.ostr << " IGNORE NULLS";
ostr << " IGNORE NULLS";
if (!is_window_function)
return;
settings.ostr << " OVER ";
ostr << " OVER ";
if (!window_name.empty())
{
settings.ostr << backQuoteIfNeed(window_name);
ostr << backQuoteIfNeed(window_name);
}
else
{
settings.ostr << "(";
window_definition->formatImpl(settings, state, frame);
settings.ostr << ")";
ostr << "(";
window_definition->formatImpl(ostr, settings, state, frame);
ostr << ")";
}
}
@ -212,7 +212,7 @@ ASTPtr ASTFunction::toLiteral() const
*
* Another case is regexp match. Suppose the user types match(URL, 'www.clickhouse.com'). It often means that the user is unaware that . is a metacharacter.
*/
static bool highlightStringLiteralWithMetacharacters(const ASTPtr & node, const IAST::FormatSettings & settings, const char * metacharacters)
static bool highlightStringLiteralWithMetacharacters(const ASTPtr & node, WriteBuffer & ostr, const char * metacharacters)
{
if (const auto * literal = node->as<ASTLiteral>())
{
@ -225,7 +225,7 @@ static bool highlightStringLiteralWithMetacharacters(const ASTPtr & node, const
{
if (c == '\\')
{
settings.ostr << c;
ostr << c;
if (escaping == 2)
escaping = 0;
++escaping;
@ -233,14 +233,14 @@ static bool highlightStringLiteralWithMetacharacters(const ASTPtr & node, const
else if (nullptr != strchr(metacharacters, c))
{
if (escaping == 2) /// Properly escaped metacharacter
settings.ostr << c;
ostr << c;
else /// Unescaped metacharacter
settings.ostr << "\033[1;35m" << c << "\033[0m";
ostr << "\033[1;35m" << c << "\033[0m";
escaping = 0;
}
else
{
settings.ostr << c;
ostr << c;
escaping = 0;
}
}
@ -263,7 +263,7 @@ ASTSelectWithUnionQuery * ASTFunction::tryGetQueryArgument() const
}
static bool formatNamedArgWithHiddenValue(IAST * arg, const IAST::FormatSettings & settings, IAST::FormatState & state, IAST::FormatStateStacked frame)
static bool formatNamedArgWithHiddenValue(IAST * arg, WriteBuffer & ostr, const IAST::FormatSettings & settings, IAST::FormatState & state, IAST::FormatStateStacked frame)
{
const auto * equals_func = arg->as<ASTFunction>();
if (!equals_func || (equals_func->name != "equals"))
@ -275,14 +275,14 @@ static bool formatNamedArgWithHiddenValue(IAST * arg, const IAST::FormatSettings
if (equal_args.size() != 2)
return false;
equal_args[0]->formatImpl(settings, state, frame);
settings.ostr << (settings.hilite ? IAST::hilite_operator : "") << " = " << (settings.hilite ? IAST::hilite_none : "");
settings.ostr << "'[HIDDEN]'";
equal_args[0]->formatImpl(ostr, settings, state, frame);
ostr << (settings.hilite ? IAST::hilite_operator : "") << " = " << (settings.hilite ? IAST::hilite_none : "");
ostr << "'[HIDDEN]'";
return true;
}
void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTFunction::formatImplWithoutAlias(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
frame.expression_list_prepend_whitespace = false;
if (kind == Kind::CODEC || kind == Kind::STATISTICS || kind == Kind::BACKUP_NAME)
@ -296,15 +296,15 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
{
std::string nl_or_nothing = settings.one_line ? "" : "\n";
std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' ');
settings.ostr << (settings.hilite ? hilite_function : "") << name << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_function : "") << "(" << (settings.hilite ? hilite_none : "");
settings.ostr << nl_or_nothing;
ostr << (settings.hilite ? hilite_function : "") << name << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_function : "") << "(" << (settings.hilite ? hilite_none : "");
ostr << nl_or_nothing;
FormatStateStacked frame_nested = frame;
frame_nested.need_parens = false;
++frame_nested.indent;
query->formatImpl(settings, state, frame_nested);
settings.ostr << nl_or_nothing << indent_str;
settings.ostr << (settings.hilite ? hilite_function : "") << ")" << (settings.hilite ? hilite_none : "");
query->formatImpl(ostr, settings, state, frame_nested);
ostr << nl_or_nothing << indent_str;
ostr << (settings.hilite ? hilite_function : "") << ")" << (settings.hilite ? hilite_none : "");
return;
}
@ -357,21 +357,21 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
nested_need_parens.need_parens = false;
if (outside_parens)
settings.ostr << '(';
ostr << '(';
settings.ostr << (settings.hilite ? hilite_operator : "") << func[1] << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_operator : "") << func[1] << (settings.hilite ? hilite_none : "");
if (inside_parens)
settings.ostr << '(';
ostr << '(';
arguments->formatImpl(settings, state, nested_need_parens);
arguments->formatImpl(ostr, settings, state, nested_need_parens);
written = true;
if (inside_parens)
settings.ostr << ')';
ostr << ')';
if (outside_parens)
settings.ostr << ')';
ostr << ')';
break;
}
@ -395,11 +395,11 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
}
if (frame.need_parens)
settings.ostr << '(';
arguments->formatImpl(settings, state, nested_need_parens);
settings.ostr << (settings.hilite ? hilite_operator : "") << func[1] << (settings.hilite ? hilite_none : "");
ostr << '(';
arguments->formatImpl(ostr, settings, state, nested_need_parens);
ostr << (settings.hilite ? hilite_operator : "") << func[1] << (settings.hilite ? hilite_none : "");
if (frame.need_parens)
settings.ostr << ')';
ostr << ')';
written = true;
@ -443,13 +443,13 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
if (name == std::string_view(func[0]))
{
if (frame.need_parens)
settings.ostr << '(';
arguments->children[0]->formatImpl(settings, state, nested_need_parens);
settings.ostr << (settings.hilite ? hilite_operator : "") << func[1] << (settings.hilite ? hilite_none : "");
ostr << '(';
arguments->children[0]->formatImpl(ostr, settings, state, nested_need_parens);
ostr << (settings.hilite ? hilite_operator : "") << func[1] << (settings.hilite ? hilite_none : "");
bool special_hilite = settings.hilite
&& (name == "like" || name == "notLike" || name == "ilike" || name == "notILike")
&& highlightStringLiteralWithMetacharacters(arguments->children[1], settings, "%_");
&& highlightStringLiteralWithMetacharacters(arguments->children[1], ostr, "%_");
/// Format x IN 1 as x IN (1): put parens around rhs even if there is a single element in set.
const auto * second_arg_func = arguments->children[1]->as<ASTFunction>();
@ -463,16 +463,16 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
if (extra_parents_around_in_rhs)
{
settings.ostr << '(';
arguments->children[1]->formatImpl(settings, state, nested_dont_need_parens);
settings.ostr << ')';
ostr << '(';
arguments->children[1]->formatImpl(ostr, settings, state, nested_dont_need_parens);
ostr << ')';
}
if (!special_hilite && !extra_parents_around_in_rhs)
arguments->children[1]->formatImpl(settings, state, nested_need_parens);
arguments->children[1]->formatImpl(ostr, settings, state, nested_need_parens);
if (frame.need_parens)
settings.ostr << ')';
ostr << ')';
written = true;
}
}
@ -480,16 +480,16 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
if (!written && name == "arrayElement"sv)
{
if (frame.need_parens)
settings.ostr << '(';
ostr << '(';
arguments->children[0]->formatImpl(settings, state, nested_need_parens);
settings.ostr << (settings.hilite ? hilite_operator : "") << '[' << (settings.hilite ? hilite_none : "");
arguments->children[1]->formatImpl(settings, state, nested_dont_need_parens);
settings.ostr << (settings.hilite ? hilite_operator : "") << ']' << (settings.hilite ? hilite_none : "");
arguments->children[0]->formatImpl(ostr, settings, state, nested_need_parens);
ostr << (settings.hilite ? hilite_operator : "") << '[' << (settings.hilite ? hilite_none : "");
arguments->children[1]->formatImpl(ostr, settings, state, nested_dont_need_parens);
ostr << (settings.hilite ? hilite_operator : "") << ']' << (settings.hilite ? hilite_none : "");
written = true;
if (frame.need_parens)
settings.ostr << ')';
ostr << ')';
}
if (!written && name == "tupleElement"sv)
@ -530,15 +530,15 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
&& lit_right->value.safeGet<Int64>() >= 0)
{
if (frame.need_parens)
settings.ostr << '(';
ostr << '(';
arguments->children[0]->formatImpl(settings, state, nested_need_parens);
settings.ostr << (settings.hilite ? hilite_operator : "") << "." << (settings.hilite ? hilite_none : "");
arguments->children[1]->formatImpl(settings, state, nested_dont_need_parens);
arguments->children[0]->formatImpl(ostr, settings, state, nested_need_parens);
ostr << (settings.hilite ? hilite_operator : "") << "." << (settings.hilite ? hilite_none : "");
arguments->children[1]->formatImpl(ostr, settings, state, nested_dont_need_parens);
written = true;
if (frame.need_parens)
settings.ostr << ')';
ostr << ')';
}
}
}
@ -559,24 +559,24 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
/// Example: f(x, (y -> z)) should not be printed as f((x, y) -> z).
if (frame.need_parens || frame.list_element_index > 0)
settings.ostr << '(';
ostr << '(';
if (first_argument_is_tuple
&& first_argument_function->arguments
&& (first_argument_function->arguments->children.size() == 1 || first_argument_function->arguments->children.empty()))
{
if (first_argument_function->arguments->children.size() == 1)
first_argument_function->arguments->children[0]->formatImpl(settings, state, nested_need_parens);
first_argument_function->arguments->children[0]->formatImpl(ostr, settings, state, nested_need_parens);
else
settings.ostr << "()";
ostr << "()";
}
else
first_argument->formatImpl(settings, state, nested_need_parens);
first_argument->formatImpl(ostr, settings, state, nested_need_parens);
settings.ostr << (settings.hilite ? hilite_operator : "") << " -> " << (settings.hilite ? hilite_none : "");
arguments->children[1]->formatImpl(settings, state, nested_need_parens);
ostr << (settings.hilite ? hilite_operator : "") << " -> " << (settings.hilite ? hilite_none : "");
arguments->children[1]->formatImpl(ostr, settings, state, nested_need_parens);
if (frame.need_parens || frame.list_element_index > 0)
settings.ostr << ')';
ostr << ')';
written = true;
}
@ -587,15 +587,15 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
auto indent0 = settings.one_line ? "" : String(4u * frame.indent, ' ');
auto indent1 = settings.one_line ? "" : String(4u * (frame.indent + 1), ' ');
auto indent2 = settings.one_line ? "" : String(4u * (frame.indent + 2), ' ');
settings.ostr << (settings.hilite ? hilite_function : "") << name << "(" << (settings.hilite ? hilite_none : "") << nl_or_nothing;
ostr << (settings.hilite ? hilite_function : "") << name << "(" << (settings.hilite ? hilite_none : "") << nl_or_nothing;
FormatStateStacked frame_nested = frame;
frame_nested.need_parens = false;
frame_nested.indent += 2;
arguments->children[0]->formatImpl(settings, state, frame_nested);
settings.ostr << nl_or_nothing << indent1 << (settings.hilite ? hilite_keyword : "") << (settings.one_line ? " " : "")
arguments->children[0]->formatImpl(ostr, settings, state, frame_nested);
ostr << nl_or_nothing << indent1 << (settings.hilite ? hilite_keyword : "") << (settings.one_line ? " " : "")
<< "ELSE " << (settings.hilite ? hilite_none : "") << nl_or_nothing << indent2;
arguments->children[1]->formatImpl(settings, state, frame_nested);
settings.ostr << nl_or_nothing << indent0 << ")";
arguments->children[1]->formatImpl(ostr, settings, state, frame_nested);
ostr << nl_or_nothing << indent0 << ")";
return;
}
}
@ -614,17 +614,17 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
if (name == std::string_view(func[0]))
{
if (frame.need_parens)
settings.ostr << '(';
ostr << '(';
for (size_t i = 0; i < arguments->children.size(); ++i)
{
if (i != 0)
settings.ostr << (settings.hilite ? hilite_operator : "") << func[1] << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_operator : "") << func[1] << (settings.hilite ? hilite_none : "");
if (arguments->children[i]->as<ASTSetQuery>())
settings.ostr << "SETTINGS ";
arguments->children[i]->formatImpl(settings, state, nested_need_parens);
ostr << "SETTINGS ";
arguments->children[i]->formatImpl(ostr, settings, state, nested_need_parens);
}
if (frame.need_parens)
settings.ostr << ')';
ostr << ')';
written = true;
}
}
@ -632,72 +632,72 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
if (!written && name == "array"sv)
{
settings.ostr << (settings.hilite ? hilite_operator : "") << '[' << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_operator : "") << '[' << (settings.hilite ? hilite_none : "");
for (size_t i = 0; i < arguments->children.size(); ++i)
{
if (i != 0)
settings.ostr << ", ";
ostr << ", ";
if (arguments->children[i]->as<ASTSetQuery>())
settings.ostr << "SETTINGS ";
ostr << "SETTINGS ";
nested_dont_need_parens.list_element_index = i;
arguments->children[i]->formatImpl(settings, state, nested_dont_need_parens);
arguments->children[i]->formatImpl(ostr, settings, state, nested_dont_need_parens);
}
settings.ostr << (settings.hilite ? hilite_operator : "") << ']' << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_operator : "") << ']' << (settings.hilite ? hilite_none : "");
written = true;
}
if (!written && arguments->children.size() >= 2 && name == "tuple"sv)
{
settings.ostr << (settings.hilite ? hilite_operator : "") << ((frame.need_parens && !alias.empty()) ? "tuple" : "") << '('
ostr << (settings.hilite ? hilite_operator : "") << ((frame.need_parens && !alias.empty()) ? "tuple" : "") << '('
<< (settings.hilite ? hilite_none : "");
for (size_t i = 0; i < arguments->children.size(); ++i)
{
if (i != 0)
settings.ostr << ", ";
ostr << ", ";
if (arguments->children[i]->as<ASTSetQuery>())
settings.ostr << "SETTINGS ";
ostr << "SETTINGS ";
nested_dont_need_parens.list_element_index = i;
arguments->children[i]->formatImpl(settings, state, nested_dont_need_parens);
arguments->children[i]->formatImpl(ostr, settings, state, nested_dont_need_parens);
}
settings.ostr << (settings.hilite ? hilite_operator : "") << ')' << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_operator : "") << ')' << (settings.hilite ? hilite_none : "");
written = true;
}
if (!written && name == "map"sv)
{
settings.ostr << (settings.hilite ? hilite_operator : "") << "map(" << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_operator : "") << "map(" << (settings.hilite ? hilite_none : "");
for (size_t i = 0; i < arguments->children.size(); ++i)
{
if (i != 0)
settings.ostr << ", ";
ostr << ", ";
if (arguments->children[i]->as<ASTSetQuery>())
settings.ostr << "SETTINGS ";
ostr << "SETTINGS ";
nested_dont_need_parens.list_element_index = i;
arguments->children[i]->formatImpl(settings, state, nested_dont_need_parens);
arguments->children[i]->formatImpl(ostr, settings, state, nested_dont_need_parens);
}
settings.ostr << (settings.hilite ? hilite_operator : "") << ')' << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_operator : "") << ')' << (settings.hilite ? hilite_none : "");
written = true;
}
}
if (written)
{
finishFormatWithWindow(settings, state, frame);
finishFormatWithWindow(ostr, settings, state, frame);
return;
}
settings.ostr << (settings.hilite ? hilite_function : "") << name;
ostr << (settings.hilite ? hilite_function : "") << name;
if (parameters)
{
settings.ostr << '(' << (settings.hilite ? hilite_none : "");
parameters->formatImpl(settings, state, nested_dont_need_parens);
settings.ostr << (settings.hilite ? hilite_function : "") << ')';
ostr << '(' << (settings.hilite ? hilite_none : "");
parameters->formatImpl(ostr, settings, state, nested_dont_need_parens);
ostr << (settings.hilite ? hilite_function : "") << ')';
}
if ((arguments && !arguments->children.empty()) || !no_empty_args)
settings.ostr << '(' << (settings.hilite ? hilite_none : "");
ostr << '(' << (settings.hilite ? hilite_none : "");
if (arguments)
{
@ -712,11 +712,11 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
for (size_t i = 0, size = arguments->children.size(); i < size; ++i)
{
if (i != 0)
settings.ostr << ", ";
ostr << ", ";
const auto & argument = arguments->children[i];
if (argument->as<ASTSetQuery>())
settings.ostr << "SETTINGS ";
ostr << "SETTINGS ";
if (!settings.show_secrets)
{
@ -725,18 +725,18 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
if (secret_arguments.are_named)
{
if (const auto * func_ast = typeid_cast<const ASTFunction *>(argument.get()))
func_ast->arguments->children[0]->formatImpl(settings, state, nested_dont_need_parens);
func_ast->arguments->children[0]->formatImpl(ostr, settings, state, nested_dont_need_parens);
else
argument->formatImpl(settings, state, nested_dont_need_parens);
settings.ostr << (settings.hilite ? hilite_operator : "") << " = " << (settings.hilite ? hilite_none : "");
argument->formatImpl(ostr, settings, state, nested_dont_need_parens);
ostr << (settings.hilite ? hilite_operator : "") << " = " << (settings.hilite ? hilite_none : "");
}
if (!secret_arguments.replacement.empty())
{
settings.ostr << "'" << secret_arguments.replacement << "'";
ostr << "'" << secret_arguments.replacement << "'";
}
else
{
settings.ostr << "'[HIDDEN]'";
ostr << "'[HIDDEN]'";
}
if (size <= secret_arguments.start + secret_arguments.count && !secret_arguments.are_named)
break; /// All other arguments should also be hidden.
@ -747,36 +747,36 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
if (function && function->arguments && std::count(secret_arguments.nested_maps.begin(), secret_arguments.nested_maps.end(), function->name) != 0)
{
/// headers('foo' = '[HIDDEN]', 'bar' = '[HIDDEN]')
settings.ostr << (settings.hilite ? hilite_function : "") << function->name << (settings.hilite ? hilite_none : "") << "(";
ostr << (settings.hilite ? hilite_function : "") << function->name << (settings.hilite ? hilite_none : "") << "(";
for (size_t j = 0; j < function->arguments->children.size(); ++j)
{
if (j != 0)
settings.ostr << ", ";
ostr << ", ";
auto inner_arg = function->arguments->children[j];
if (!formatNamedArgWithHiddenValue(inner_arg.get(), settings, state, nested_dont_need_parens))
inner_arg->formatImpl(settings, state, nested_dont_need_parens);
if (!formatNamedArgWithHiddenValue(inner_arg.get(), ostr, settings, state, nested_dont_need_parens))
inner_arg->formatImpl(ostr, settings, state, nested_dont_need_parens);
}
settings.ostr << ")";
ostr << ")";
continue;
}
}
if ((i == 1) && special_hilite_regexp
&& highlightStringLiteralWithMetacharacters(argument, settings, "|()^$.[]?*+{:-"))
&& highlightStringLiteralWithMetacharacters(argument, ostr, "|()^$.[]?*+{:-"))
{
continue;
}
nested_dont_need_parens.list_element_index = i;
argument->formatImpl(settings, state, nested_dont_need_parens);
argument->formatImpl(ostr, settings, state, nested_dont_need_parens);
}
}
if ((arguments && !arguments->children.empty()) || !no_empty_args)
settings.ostr << (settings.hilite ? hilite_function : "") << ')';
ostr << (settings.hilite ? hilite_function : "") << ')';
settings.ostr << (settings.hilite ? hilite_none : "");
finishFormatWithWindow(settings, state, frame);
ostr << (settings.hilite ? hilite_none : "");
finishFormatWithWindow(ostr, settings, state, frame);
}
bool ASTFunction::hasSecretParts() const

View File

@ -82,10 +82,10 @@ public:
bool hasSecretParts() const override;
protected:
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImplWithoutAlias(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void appendColumnNameImpl(WriteBuffer & ostr) const override;
private:
void finishFormatWithWindow(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const;
void finishFormatWithWindow(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const;
};

View File

@ -23,39 +23,39 @@ ASTPtr ASTPair::clone() const
}
void ASTPair::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTPair::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << Poco::toUpper(first) << " " << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << Poco::toUpper(first) << " " << (settings.hilite ? hilite_none : "");
if (second_with_brackets)
settings.ostr << (settings.hilite ? hilite_keyword : "") << "(";
ostr << (settings.hilite ? hilite_keyword : "") << "(";
if (!settings.show_secrets && (first == "password"))
{
/// Hide password in the definition of a dictionary:
/// SOURCE(CLICKHOUSE(host 'example01-01-1' port 9000 user 'default' password '[HIDDEN]' db 'default' table 'ids'))
settings.ostr << "'[HIDDEN]'";
ostr << "'[HIDDEN]'";
}
else if (!settings.show_secrets && (first == "uri"))
{
// Hide password from URI in the defention of a dictionary
WriteBufferFromOwnString temp_buf;
FormatSettings tmp_settings(temp_buf, settings.one_line);
FormatSettings tmp_settings(settings.one_line);
FormatState tmp_state;
second->formatImpl(tmp_settings, tmp_state, frame);
second->formatImpl(temp_buf, tmp_settings, tmp_state, frame);
maskURIPassword(&temp_buf.str());
settings.ostr << temp_buf.str();
ostr << temp_buf.str();
}
else
{
second->formatImpl(settings, state, frame);
second->formatImpl(ostr, settings, state, frame);
}
if (second_with_brackets)
settings.ostr << (settings.hilite ? hilite_keyword : "") << ")";
ostr << (settings.hilite ? hilite_keyword : "") << ")";
settings.ostr << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_none : "");
}
@ -95,12 +95,12 @@ ASTPtr ASTFunctionWithKeyValueArguments::clone() const
}
void ASTFunctionWithKeyValueArguments::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTFunctionWithKeyValueArguments::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << Poco::toUpper(name) << (settings.hilite ? hilite_none : "") << (has_brackets ? "(" : "");
elements->formatImpl(settings, state, frame);
settings.ostr << (has_brackets ? ")" : "");
settings.ostr << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << Poco::toUpper(name) << (settings.hilite ? hilite_none : "") << (has_brackets ? "(" : "");
elements->formatImpl(ostr, settings, state, frame);
ostr << (has_brackets ? ")" : "");
ostr << (settings.hilite ? hilite_none : "");
}

View File

@ -28,7 +28,7 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
bool hasSecretParts() const override;
@ -64,7 +64,7 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
};

View File

@ -103,13 +103,13 @@ const String & ASTIdentifier::name() const
return full_name;
}
void ASTIdentifier::formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTIdentifier::formatImplWithoutAlias(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
auto format_element = [&](const String & elem_name)
{
settings.ostr << (settings.hilite ? hilite_identifier : "");
settings.writeIdentifier(elem_name, /*ambiguous=*/false);
settings.ostr << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_identifier : "");
settings.writeIdentifier(ostr, elem_name, /*ambiguous=*/false);
ostr << (settings.hilite ? hilite_none : "");
};
if (compound())
@ -117,14 +117,14 @@ void ASTIdentifier::formatImplWithoutAlias(const FormatSettings & settings, Form
for (size_t i = 0, j = 0, size = name_parts.size(); i < size; ++i)
{
if (i != 0)
settings.ostr << '.';
ostr << '.';
/// Some AST rewriting code, like IdentifierSemantic::setColumnLongName,
/// does not respect children of identifier.
/// Here we also ignore children if they are empty.
if (name_parts[i].empty() && j < children.size())
{
children[j]->formatImpl(settings, state, frame);
children[j]->formatImpl(ostr, settings, state, frame);
++j;
}
else
@ -135,7 +135,7 @@ void ASTIdentifier::formatImplWithoutAlias(const FormatSettings & settings, Form
{
const auto & name = shortName();
if (name.empty() && !children.empty())
children.front()->formatImpl(settings, state, frame);
children.front()->formatImpl(ostr, settings, state, frame);
else
format_element(name);
}

View File

@ -58,7 +58,7 @@ public:
protected:
std::shared_ptr<IdentifierSemanticImpl> semantic; /// pimpl
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImplWithoutAlias(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void appendColumnNameImpl(WriteBuffer & ostr) const override;
private:

View File

@ -62,7 +62,7 @@ std::shared_ptr<ASTFunction> ASTIndexDeclaration::getType() const
return func_ast;
}
void ASTIndexDeclaration::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const
void ASTIndexDeclaration::formatImpl(WriteBuffer & ostr, const FormatSettings & s, FormatState & state, FormatStateStacked frame) const
{
if (auto expr = getExpression())
{
@ -70,31 +70,31 @@ void ASTIndexDeclaration::formatImpl(const FormatSettings & s, FormatState & sta
{
if (expr->as<ASTExpressionList>())
{
s.ostr << "(";
expr->formatImpl(s, state, frame);
s.ostr << ")";
ostr << "(";
expr->formatImpl(ostr, s, state, frame);
ostr << ")";
}
else
expr->formatImpl(s, state, frame);
expr->formatImpl(ostr, s, state, frame);
}
else
{
s.writeIdentifier(name, /*ambiguous=*/false);
s.ostr << " ";
expr->formatImpl(s, state, frame);
s.writeIdentifier(ostr, name, /*ambiguous=*/false);
ostr << " ";
expr->formatImpl(ostr, s, state, frame);
}
}
if (auto type = getType())
{
s.ostr << (s.hilite ? hilite_keyword : "") << " TYPE " << (s.hilite ? hilite_none : "");
type->formatImpl(s, state, frame);
ostr << (s.hilite ? hilite_keyword : "") << " TYPE " << (s.hilite ? hilite_none : "");
type->formatImpl(ostr, s, state, frame);
}
if (granularity)
{
s.ostr << (s.hilite ? hilite_keyword : "") << " GRANULARITY " << (s.hilite ? hilite_none : "");
s.ostr << granularity;
ostr << (s.hilite ? hilite_keyword : "") << " GRANULARITY " << (s.hilite ? hilite_none : "");
ostr << granularity;
}
}

View File

@ -25,7 +25,7 @@ public:
String getID(char) const override { return "Index"; }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getExpression() const;
std::shared_ptr<ASTFunction> getType() const;

View File

@ -46,54 +46,54 @@ void ASTInsertQuery::setTable(const String & name)
table = std::make_shared<ASTIdentifier>(name);
}
void ASTInsertQuery::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTInsertQuery::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
frame.need_parens = false;
settings.ostr << (settings.hilite ? hilite_keyword : "") << "INSERT INTO ";
ostr << (settings.hilite ? hilite_keyword : "") << "INSERT INTO ";
if (table_function)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "FUNCTION ";
table_function->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << "FUNCTION ";
table_function->formatImpl(ostr, settings, state, frame);
if (partition_by)
{
settings.ostr << " PARTITION BY ";
partition_by->formatImpl(settings, state, frame);
ostr << " PARTITION BY ";
partition_by->formatImpl(ostr, settings, state, frame);
}
}
else if (table_id)
{
settings.ostr << (settings.hilite ? hilite_none : "")
ostr << (settings.hilite ? hilite_none : "")
<< (!table_id.database_name.empty() ? backQuoteIfNeed(table_id.database_name) + "." : "") << backQuoteIfNeed(table_id.table_name);
}
else
{
if (database)
{
database->formatImpl(settings, state, frame);
settings.ostr << '.';
database->formatImpl(ostr, settings, state, frame);
ostr << '.';
}
chassert(table);
table->formatImpl(settings, state, frame);
table->formatImpl(ostr, settings, state, frame);
}
if (columns)
{
settings.ostr << " (";
columns->formatImpl(settings, state, frame);
settings.ostr << ")";
ostr << " (";
columns->formatImpl(ostr, settings, state, frame);
ostr << ")";
}
if (infile)
{
settings.ostr
ostr
<< (settings.hilite ? hilite_keyword : "")
<< " FROM INFILE "
<< (settings.hilite ? hilite_none : "")
<< quoteString(infile->as<ASTLiteral &>().value.safeGet<std::string>());
if (compression)
settings.ostr
ostr
<< (settings.hilite ? hilite_keyword : "")
<< " COMPRESSION "
<< (settings.hilite ? hilite_none : "")
@ -102,8 +102,8 @@ void ASTInsertQuery::formatImpl(const FormatSettings & settings, FormatState & s
if (settings_ast)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "SETTINGS " << (settings.hilite ? hilite_none : "");
settings_ast->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "SETTINGS " << (settings.hilite ? hilite_none : "");
settings_ast->formatImpl(ostr, settings, state, frame);
}
/// Compatibility for INSERT without SETTINGS to format in oneline, i.e.:
@ -120,20 +120,20 @@ void ASTInsertQuery::formatImpl(const FormatSettings & settings, FormatState & s
if (select)
{
settings.ostr << delim;
select->formatImpl(settings, state, frame);
ostr << delim;
select->formatImpl(ostr, settings, state, frame);
}
if (!select)
{
if (!format.empty())
{
settings.ostr << delim
ostr << delim
<< (settings.hilite ? hilite_keyword : "") << "FORMAT " << (settings.hilite ? hilite_none : "") << format;
}
else if (!infile)
{
settings.ostr << delim
ostr << delim
<< (settings.hilite ? hilite_keyword : "") << "VALUES" << (settings.hilite ? hilite_none : "");
}
}

View File

@ -71,7 +71,7 @@ public:
QueryKind getQueryKind() const override { return async_insert_flush ? QueryKind::AsyncInsertFlush : QueryKind::Insert; }
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
};

View File

@ -7,10 +7,10 @@
namespace DB
{
void ASTInterpolateElement::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTInterpolateElement::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << column << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "");
expr->formatImpl(settings, state, frame);
ostr << column << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "");
expr->formatImpl(ostr, settings, state, frame);
}
}

View File

@ -25,7 +25,7 @@ public:
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
}

View File

@ -9,37 +9,37 @@ String ASTKillQueryQuery::getID(char delim) const
return String("KillQueryQuery") + delim + (where_expression ? where_expression->getID() : "") + delim + String(sync ? "SYNC" : "ASYNC");
}
void ASTKillQueryQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTKillQueryQuery::formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "KILL ";
ostr << (settings.hilite ? hilite_keyword : "") << "KILL ";
switch (type)
{
case Type::Query:
settings.ostr << "QUERY";
ostr << "QUERY";
break;
case Type::Mutation:
settings.ostr << "MUTATION";
ostr << "MUTATION";
break;
case Type::PartMoveToShard:
settings.ostr << "PART_MOVE_TO_SHARD";
ostr << "PART_MOVE_TO_SHARD";
break;
case Type::Transaction:
settings.ostr << "TRANSACTION";
ostr << "TRANSACTION";
break;
}
settings.ostr << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_none : "");
formatOnCluster(settings);
formatOnCluster(ostr, settings);
if (where_expression)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : "");
where_expression->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : "");
where_expression->formatImpl(ostr, settings, state, frame);
}
settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << (test ? "TEST" : (sync ? "SYNC" : "ASYNC")) << (settings.hilite ? hilite_none : "");
ostr << " " << (settings.hilite ? hilite_keyword : "") << (test ? "TEST" : (sync ? "SYNC" : "ASYNC")) << (settings.hilite ? hilite_none : "");
}
}

View File

@ -36,7 +36,7 @@ public:
String getID(char) const override;
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override
{

View File

@ -148,12 +148,12 @@ String FieldVisitorToStringPostgreSQL::operator() (const String & x) const
return wb.str();
}
void ASTLiteral::formatImplWithoutAlias(const FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
void ASTLiteral::formatImplWithoutAlias(WriteBuffer & ostr, const FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
{
if (settings.literal_escaping_style == LiteralEscapingStyle::Regular)
settings.ostr << applyVisitor(FieldVisitorToString(), value);
ostr << applyVisitor(FieldVisitorToString(), value);
else
settings.ostr << applyVisitor(FieldVisitorToStringPostgreSQL(), value);
ostr << applyVisitor(FieldVisitorToStringPostgreSQL(), value);
}
}

View File

@ -52,7 +52,7 @@ public:
void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
protected:
void formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
void formatImplWithoutAlias(WriteBuffer & ostr, const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
void appendColumnNameImpl(WriteBuffer & ostr) const override;

View File

@ -21,10 +21,10 @@ ASTPtr ASTNameTypePair::clone() const
}
void ASTNameTypePair::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTNameTypePair::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << backQuoteIfNeed(name) << ' ';
type->formatImpl(settings, state, frame);
ostr << backQuoteIfNeed(name) << ' ';
type->formatImpl(ostr, settings, state, frame);
}
}

View File

@ -21,7 +21,7 @@ public:
ASTPtr clone() const override;
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};

View File

@ -35,27 +35,27 @@ ASTPtr ASTObjectTypeArgument::clone() const
return res;
}
void ASTObjectTypeArgument::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTObjectTypeArgument::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
if (path_with_type)
{
path_with_type->formatImpl(settings, state, frame);
path_with_type->formatImpl(ostr, settings, state, frame);
}
else if (parameter)
{
parameter->formatImpl(settings, state, frame);
parameter->formatImpl(ostr, settings, state, frame);
}
else if (skip_path)
{
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
settings.ostr << indent_str << "SKIP" << ' ';
skip_path->formatImpl(settings, state, frame);
ostr << indent_str << "SKIP" << ' ';
skip_path->formatImpl(ostr, settings, state, frame);
}
else if (skip_path_regexp)
{
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
settings.ostr << indent_str << "SKIP REGEXP" << ' ';
skip_path_regexp->formatImpl(settings, state, frame);
ostr << indent_str << "SKIP REGEXP" << ' ';
skip_path_regexp->formatImpl(ostr, settings, state, frame);
}
}

View File

@ -25,7 +25,7 @@ public:
ASTPtr clone() const override;
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};

View File

@ -5,40 +5,40 @@
namespace DB
{
void ASTOptimizeQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTOptimizeQuery::formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "OPTIMIZE TABLE " << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << "OPTIMIZE TABLE " << (settings.hilite ? hilite_none : "");
if (database)
{
database->formatImpl(settings, state, frame);
settings.ostr << '.';
database->formatImpl(ostr, settings, state, frame);
ostr << '.';
}
chassert(table);
table->formatImpl(settings, state, frame);
table->formatImpl(ostr, settings, state, frame);
formatOnCluster(settings);
formatOnCluster(ostr, settings);
if (partition)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(ostr, settings, state, frame);
}
if (final)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FINAL" << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << " FINAL" << (settings.hilite ? hilite_none : "");
if (deduplicate)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " DEDUPLICATE" << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << " DEDUPLICATE" << (settings.hilite ? hilite_none : "");
if (cleanup)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " CLEANUP" << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << " CLEANUP" << (settings.hilite ? hilite_none : "");
if (deduplicate_by_columns)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " BY " << (settings.hilite ? hilite_none : "");
deduplicate_by_columns->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " BY " << (settings.hilite ? hilite_none : "");
deduplicate_by_columns->formatImpl(ostr, settings, state, frame);
}
}

View File

@ -49,7 +49,7 @@ public:
return res;
}
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams & params) const override
{

View File

@ -15,16 +15,16 @@ void ASTOrderByElement::updateTreeHashImpl(SipHash & hash_state, bool ignore_ali
IAST::updateTreeHashImpl(hash_state, ignore_aliases);
}
void ASTOrderByElement::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTOrderByElement::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
children.front()->formatImpl(settings, state, frame);
settings.ostr << (settings.hilite ? hilite_keyword : "")
children.front()->formatImpl(ostr, settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "")
<< (direction == -1 ? " DESC" : " ASC")
<< (settings.hilite ? hilite_none : "");
if (nulls_direction_was_explicitly_specified)
{
settings.ostr << (settings.hilite ? hilite_keyword : "")
ostr << (settings.hilite ? hilite_keyword : "")
<< " NULLS "
<< (nulls_direction == direction ? "LAST" : "FIRST")
<< (settings.hilite ? hilite_none : "");
@ -32,32 +32,32 @@ void ASTOrderByElement::formatImpl(const FormatSettings & settings, FormatState
if (auto collation = getCollation())
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " COLLATE " << (settings.hilite ? hilite_none : "");
collation->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " COLLATE " << (settings.hilite ? hilite_none : "");
collation->formatImpl(ostr, settings, state, frame);
}
if (with_fill)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH FILL" << (settings.hilite ? hilite_none : "");
ostr << (settings.hilite ? hilite_keyword : "") << " WITH FILL" << (settings.hilite ? hilite_none : "");
if (auto fill_from = getFillFrom())
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "");
fill_from->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "");
fill_from->formatImpl(ostr, settings, state, frame);
}
if (auto fill_to = getFillTo())
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " TO " << (settings.hilite ? hilite_none : "");
fill_to->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " TO " << (settings.hilite ? hilite_none : "");
fill_to->formatImpl(ostr, settings, state, frame);
}
if (auto fill_step = getFillStep())
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " STEP " << (settings.hilite ? hilite_none : "");
fill_step->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " STEP " << (settings.hilite ? hilite_none : "");
fill_step->formatImpl(ostr, settings, state, frame);
}
if (auto fill_staleness = getFillStaleness())
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " STALENESS " << (settings.hilite ? hilite_none : "");
fill_staleness->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << " STALENESS " << (settings.hilite ? hilite_none : "");
fill_staleness->formatImpl(ostr, settings, state, frame);
}
}
}

View File

@ -54,7 +54,7 @@ public:
void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
private:
ASTPtr getChild(Child child) const

View File

@ -61,20 +61,20 @@ ASTPtr ASTPartition::clone() const
return res;
}
void ASTPartition::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTPartition::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
if (value)
{
value->formatImpl(settings, state, frame);
value->formatImpl(ostr, settings, state, frame);
}
else if (all)
{
settings.ostr << "ALL";
ostr << "ALL";
}
else
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ID " << (settings.hilite ? hilite_none : "");
id->formatImpl(settings, state, frame);
ostr << (settings.hilite ? hilite_keyword : "") << "ID " << (settings.hilite ? hilite_none : "");
id->formatImpl(ostr, settings, state, frame);
}
}
}

View File

@ -29,7 +29,7 @@ public:
}
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
}

View File

@ -15,17 +15,17 @@ ASTPtr ASTProjectionDeclaration::clone() const
}
void ASTProjectionDeclaration::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTProjectionDeclaration::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.writeIdentifier(name, /*ambiguous=*/false);
settings.writeIdentifier(ostr, name, /*ambiguous=*/false);
std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' ');
std::string nl_or_nothing = settings.one_line ? "" : "\n";
settings.ostr << settings.nl_or_ws << indent_str << "(" << nl_or_nothing;
ostr << settings.nl_or_ws << indent_str << "(" << nl_or_nothing;
FormatStateStacked frame_nested = frame;
frame_nested.need_parens = false;
++frame_nested.indent;
query->formatImpl(settings, state, frame_nested);
settings.ostr << nl_or_nothing << indent_str << ")";
query->formatImpl(ostr, settings, state, frame_nested);
ostr << nl_or_nothing << indent_str << ")";
}
}

View File

@ -17,7 +17,7 @@ public:
String getID(char) const override { return "Projection"; }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
void forEachPointerToChild(std::function<void(void**)> f) override
{

View File

@ -48,7 +48,7 @@ ASTPtr ASTProjectionSelectQuery::clone() const
}
void ASTProjectionSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const
void ASTProjectionSelectQuery::formatImpl(WriteBuffer & ostr, const FormatSettings & s, FormatState & state, FormatStateStacked frame) const
{
frame.current_select = this;
frame.need_parens = false;
@ -56,26 +56,26 @@ void ASTProjectionSelectQuery::formatImpl(const FormatSettings & s, FormatState
if (with())
{
s.ostr << (s.hilite ? hilite_keyword : "") << indent_str << "WITH " << (s.hilite ? hilite_none : "");
s.one_line ? with()->formatImpl(s, state, frame) : with()->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
s.ostr << s.nl_or_ws;
ostr << (s.hilite ? hilite_keyword : "") << indent_str << "WITH " << (s.hilite ? hilite_none : "");
s.one_line ? with()->formatImpl(ostr, s, state, frame) : with()->as<ASTExpressionList &>().formatImplMultiline(ostr, s, state, frame);
ostr << s.nl_or_ws;
}
s.ostr << (s.hilite ? hilite_keyword : "") << indent_str << "SELECT " << (s.hilite ? hilite_none : "");
ostr << (s.hilite ? hilite_keyword : "") << indent_str << "SELECT " << (s.hilite ? hilite_none : "");
s.one_line ? select()->formatImpl(s, state, frame) : select()->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
s.one_line ? select()->formatImpl(ostr, s, state, frame) : select()->as<ASTExpressionList &>().formatImplMultiline(ostr, s, state, frame);
if (groupBy())
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "GROUP BY " << (s.hilite ? hilite_none : "");
s.one_line ? groupBy()->formatImpl(s, state, frame) : groupBy()->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "GROUP BY " << (s.hilite ? hilite_none : "");
s.one_line ? groupBy()->formatImpl(ostr, s, state, frame) : groupBy()->as<ASTExpressionList &>().formatImplMultiline(ostr, s, state, frame);
}
if (orderBy())
{
/// Let's convert tuple ASTFunction into ASTExpressionList, which generates consistent format
/// between GROUP BY and ORDER BY projection definition.
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "ORDER BY " << (s.hilite ? hilite_none : "");
ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "ORDER BY " << (s.hilite ? hilite_none : "");
ASTPtr order_by;
if (auto * func = orderBy()->as<ASTFunction>(); func && func->name == "tuple")
order_by = func->arguments;
@ -84,7 +84,7 @@ void ASTProjectionSelectQuery::formatImpl(const FormatSettings & s, FormatState
order_by = std::make_shared<ASTExpressionList>();
order_by->children.push_back(orderBy());
}
s.one_line ? order_by->formatImpl(s, state, frame) : order_by->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
s.one_line ? order_by->formatImpl(ostr, s, state, frame) : order_by->as<ASTExpressionList &>().formatImplMultiline(ostr, s, state, frame);
}
}

View File

@ -45,7 +45,7 @@ public:
ASTPtr cloneToASTSelect() const;
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
private:
std::unordered_map<Expression, size_t> positions;

View File

@ -11,14 +11,14 @@ void ASTQualifiedAsterisk::appendColumnName(WriteBuffer & ostr) const
writeCString(".*", ostr);
}
void ASTQualifiedAsterisk::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTQualifiedAsterisk::formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
qualifier->formatImpl(settings, state, frame);
settings.ostr << ".*";
qualifier->formatImpl(ostr, settings, state, frame);
ostr << ".*";
if (transformers)
{
transformers->formatImpl(settings, state, frame);
transformers->formatImpl(ostr, settings, state, frame);
}
}

View File

@ -35,7 +35,7 @@ public:
ASTPtr qualifier;
ASTPtr transformers;
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
}

View File

@ -7,9 +7,9 @@
namespace DB
{
void ASTQueryParameter::formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const
void ASTQueryParameter::formatImplWithoutAlias(WriteBuffer & ostr, const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
settings.ostr
ostr
<< (settings.hilite ? hilite_substitution : "") << '{'
<< (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(name)
<< (settings.hilite ? hilite_substitution : "") << ':'

View File

@ -24,7 +24,7 @@ public:
void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
protected:
void formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
void formatImplWithoutAlias(WriteBuffer & ostr, const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
void appendColumnNameImpl(WriteBuffer & ostr) const override;
};

Some files were not shown because too many files have changed in this diff Show More