提交 a34ab820 编写于 作者: V Vitaliy Lyudvichenko 提交者: alexey-milovidov

Add FORMAT section for DDL queries CREATE, ALTER, RENAME, DROP. [#CLICKHOUSE-3128]

上级 4ef5d147
......@@ -10,7 +10,7 @@ namespace ErrorCodes
extern const int UNEXPECTED_AST_STRUCTURE;
}
ASTAlterQuery::Parameters::Parameters() : type(NO_TYPE) {}
ASTAlterQuery::Parameters::Parameters() {}
void ASTAlterQuery::Parameters::clone(Parameters & p) const
{
......@@ -42,7 +42,7 @@ void ASTAlterQuery::addParameters(const Parameters & params)
children.push_back(params.primary_key);
}
ASTAlterQuery::ASTAlterQuery(StringRange range_) : IAST(range_)
ASTAlterQuery::ASTAlterQuery(StringRange range_) : ASTQueryWithOutput(range_)
{
}
......@@ -57,13 +57,14 @@ ASTPtr ASTAlterQuery::clone() const
auto res = std::make_shared<ASTAlterQuery>(*this);
for (ParameterContainer::size_type i = 0; i < parameters.size(); ++i)
parameters[i].clone(res->parameters[i]);
cloneOutputOptions(*res);
return res;
}
ASTPtr ASTAlterQuery::getRewrittenASTWithoutOnCluster(const std::string & new_database) const
{
auto query_ptr = clone();
ASTAlterQuery & query = static_cast<ASTAlterQuery &>(*query_ptr);
auto & query = static_cast<ASTAlterQuery &>(*query_ptr);
query.cluster.clear();
if (query.database.empty())
......@@ -72,11 +73,11 @@ ASTPtr ASTAlterQuery::getRewrittenASTWithoutOnCluster(const std::string & new_da
return query_ptr;
}
void ASTAlterQuery::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
frame.need_parens = false;
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' ');
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER TABLE " << (settings.hilite ? hilite_none : "");
......
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTQueryWithOutput.h>
#include <Parsers/ASTQueryWithOnCluster.h>
......@@ -19,7 +20,7 @@ namespace DB
* [COORDINATE WITH 'coordinator_id']
*/
class ASTAlterQuery : public IAST, public ASTQueryWithOnCluster
class ASTAlterQuery : public ASTQueryWithOutput, public ASTQueryWithOnCluster
{
public:
enum ParameterType
......@@ -96,17 +97,17 @@ public:
void addParameters(const Parameters & params);
ASTAlterQuery(StringRange range_ = StringRange());
explicit ASTAlterQuery(StringRange range_ = StringRange());
/** Get the text that identifies this element. */
String getID() const override;
ASTPtr clone() const override;
ASTPtr getRewrittenASTWithoutOnCluster(const std::string & new_database = {}) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const std::string & new_database) const override;
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
}
......@@ -2,6 +2,7 @@
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTQueryWithOutput.h>
#include <Parsers/ASTQueryWithOnCluster.h>
......@@ -10,7 +11,7 @@ namespace DB
/// CREATE TABLE or ATTACH TABLE query
class ASTCreateQuery : public IAST, public ASTQueryWithOnCluster
class ASTCreateQuery : public ASTQueryWithOutput, public ASTQueryWithOnCluster
{
public:
bool attach{false}; /// Query ATTACH TABLE, not CREATE TABLE.
......@@ -29,7 +30,7 @@ public:
ASTPtr select;
ASTCreateQuery() = default;
ASTCreateQuery(const StringRange range_) : IAST(range_) {}
ASTCreateQuery(const StringRange range_) : ASTQueryWithOutput(range_) {}
/** Get the text that identifies this element. */
String getID() const override { return (attach ? "AttachQuery_" : "CreateQuery_") + database + "_" + table; };
......@@ -44,6 +45,8 @@ public:
if (select) { res->select = select->clone(); res->children.push_back(res->select); }
if (inner_storage) { res->inner_storage = inner_storage->clone(); res->children.push_back(res->inner_storage); }
cloneOutputOptions(*res);
return res;
}
......@@ -60,7 +63,7 @@ public:
}
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
frame.need_parens = false;
......
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTQueryWithOutput.h>
#include <Parsers/ASTQueryWithOnCluster.h>
namespace DB
......@@ -9,7 +10,7 @@ namespace DB
/** DROP query
*/
class ASTDropQuery : public IAST, public ASTQueryWithOnCluster
class ASTDropQuery : public ASTQueryWithOutput, public ASTQueryWithOnCluster
{
public:
bool detach{false}; /// DETACH query, not DROP.
......@@ -18,17 +19,22 @@ public:
String table;
ASTDropQuery() = default;
ASTDropQuery(const StringRange range_) : IAST(range_) {}
explicit ASTDropQuery(const StringRange range_) : ASTQueryWithOutput(range_) {}
/** Get the text that identifies this element. */
String getID() const override { return (detach ? "DetachQuery_" : "DropQuery_") + database + "_" + table; };
ASTPtr clone() const override { return std::make_shared<ASTDropQuery>(*this); }
ASTPtr clone() const override
{
auto res = std::make_shared<ASTDropQuery>(*this);
cloneOutputOptions(*res);
return res;
}
ASTPtr getRewrittenASTWithoutOnCluster(const std::string & new_database) const override
{
auto query_ptr = clone();
ASTDropQuery & query = static_cast<ASTDropQuery &>(*query_ptr);
auto & query = static_cast<ASTDropQuery &>(*query_ptr);
query.cluster.clear();
if (query.database.empty())
......@@ -38,7 +44,7 @@ public:
}
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
if (table.empty() && !database.empty())
{
......
......@@ -8,8 +8,8 @@ class ASTKillQueryQuery : public ASTQueryWithOutput
{
public:
ASTPtr where_expression; // expression to filter processes from system.processes table
bool sync = false; // SYNC or ASYNC mode
bool test = false; // does it TEST mode? (doesn't cancel queries just checks and shows them)
bool sync = false; // SYNC or ASYNC mode
bool test = false; // does it TEST mode? (doesn't cancel queries just checks and shows them)
ASTKillQueryQuery() = default;
......
......@@ -21,7 +21,7 @@ void ASTQueryWithOutput::formatImpl(const FormatSettings & s, FormatState & stat
{
formatQueryImpl(s, state, frame);
std::string indent_str = s.one_line ? "" : std::string(4 * frame.indent, ' ');
std::string indent_str = s.one_line ? "" : std::string(4u * frame.indent, ' ');
if (out_file)
{
......
......@@ -15,7 +15,7 @@ public:
ASTPtr format;
ASTQueryWithOutput() = default;
ASTQueryWithOutput(const StringRange range_) : IAST(range_) {}
explicit ASTQueryWithOutput(const StringRange range_) : IAST(range_) {}
protected:
/// NOTE: call this helper at the end of the clone() method of descendant class.
......
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTQueryWithOutput.h>
#include <Parsers/ASTQueryWithOnCluster.h>
namespace DB
......@@ -9,7 +10,7 @@ namespace DB
/** RENAME query
*/
class ASTRenameQuery : public IAST, public ASTQueryWithOnCluster
class ASTRenameQuery : public ASTQueryWithOutput, public ASTQueryWithOnCluster
{
public:
struct Table
......@@ -28,17 +29,22 @@ public:
Elements elements;
ASTRenameQuery() = default;
ASTRenameQuery(const StringRange range_) : IAST(range_) {}
explicit ASTRenameQuery(const StringRange range_) : ASTQueryWithOutput(range_) {}
/** Get the text that identifies this element. */
String getID() const override { return "Rename"; };
ASTPtr clone() const override { return std::make_shared<ASTRenameQuery>(*this); }
ASTPtr clone() const override
{
auto res = std::make_shared<ASTRenameQuery>(*this);
cloneOutputOptions(*res);
return res;
}
ASTPtr getRewrittenASTWithoutOnCluster(const std::string & new_database = {}) const override
ASTPtr getRewrittenASTWithoutOnCluster(const std::string & new_database) const override
{
auto query_ptr = clone();
ASTRenameQuery & query = static_cast<ASTRenameQuery &>(*query_ptr);
auto & query = static_cast<ASTRenameQuery &>(*query_ptr);
query.cluster.clear();
for (Element & elem : query.elements)
......@@ -53,7 +59,7 @@ public:
}
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "RENAME TABLE " << (settings.hilite ? hilite_none : "");
......
......@@ -18,20 +18,13 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserQueryWithOutput query_with_output_p;
ParserInsertQuery insert_p(end);
ParserCreateQuery create_p;
ParserRenameQuery rename_p;
ParserDropQuery drop_p;
ParserAlterQuery alter_p;
ParserUseQuery use_p;
ParserSetQuery set_p;
ParserOptimizeQuery optimize_p;
bool res = query_with_output_p.parse(pos, node, expected)
|| insert_p.parse(pos, node, expected)
|| create_p.parse(pos, node, expected)
|| rename_p.parse(pos, node, expected)
|| drop_p.parse(pos, node, expected)
|| alter_p.parse(pos, node, expected)
|| use_p.parse(pos, node, expected)
|| set_p.parse(pos, node, expected)
|| optimize_p.parse(pos, node, expected);
......
......@@ -4,10 +4,11 @@
#include <Parsers/ParserTablePropertiesQuery.h>
#include <Parsers/ParserShowProcesslistQuery.h>
#include <Parsers/ParserCheckQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/ParserRenameQuery.h>
#include <Parsers/ParserAlterQuery.h>
#include <Parsers/ParserDropQuery.h>
#include <Parsers/ParserKillQueryQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Common/typeid_cast.h>
namespace DB
......@@ -19,6 +20,10 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
ParserSelectQuery select_p;
ParserTablePropertiesQuery table_p;
ParserShowProcesslistQuery show_processlist_p;
ParserCreateQuery create_p;
ParserAlterQuery alter_p;
ParserRenameQuery rename_p;
ParserDropQuery drop_p;
ParserCheckQuery check_p;
ParserKillQueryQuery kill_query_p;
......@@ -28,6 +33,10 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|| show_tables_p.parse(pos, query, expected)
|| table_p.parse(pos, query, expected)
|| show_processlist_p.parse(pos, query, expected)
|| create_p.parse(pos, query, expected)
|| alter_p.parse(pos, query, expected)
|| rename_p.parse(pos, query, expected)
|| drop_p.parse(pos, query, expected)
|| check_p.parse(pos, query, expected)
|| kill_query_p.parse(pos, query, expected);
......
......@@ -125,8 +125,8 @@ def started_cluster():
def test_default_database(started_cluster):
instance = cluster.instances['ch3']
ddl_check_query(instance, "CREATE DATABASE IF NOT EXISTS test2 ON CLUSTER 'cluster'")
ddl_check_query(instance, "DROP TABLE IF EXISTS null ON CLUSTER 'cluster'")
ddl_check_query(instance, "CREATE DATABASE IF NOT EXISTS test2 ON CLUSTER 'cluster' FORMAT TSV")
ddl_check_query(instance, "DROP TABLE IF EXISTS null ON CLUSTER 'cluster' FORMAT TSV")
ddl_check_query(instance, "CREATE TABLE null ON CLUSTER 'cluster2' (s String DEFAULT 'escape\t\nme') ENGINE = Null")
contents = instance.query("SELECT hostName() AS h, database FROM all_tables WHERE name = 'null' ORDER BY h")
......@@ -138,10 +138,14 @@ def test_default_database(started_cluster):
def test_create_view(started_cluster):
instance = cluster.instances['ch3']
ddl_check_query(instance, "CREATE VIEW test.super_simple_view ON CLUSTER 'cluster' AS SELECT * FROM system.numbers")
ddl_check_query(instance, "CREATE MATERIALIZED VIEW test.simple_mat_view ON CLUSTER 'cluster' ENGINE = Memory AS SELECT * FROM system.numbers")
ddl_check_query(instance, "DROP TABLE test.simple_mat_view ON CLUSTER 'cluster'")
ddl_check_query(instance, "DROP TABLE test.super_simple_view ON CLUSTER 'cluster'")
ddl_check_query(instance, "CREATE VIEW test.super_simple_view ON CLUSTER 'cluster' AS SELECT * FROM system.numbers FORMAT TSV")
ddl_check_query(instance, "CREATE MATERIALIZED VIEW test.simple_mat_view ON CLUSTER 'cluster' ENGINE = Memory AS SELECT * FROM system.numbers FORMAT TSV")
ddl_check_query(instance, "DROP TABLE test.simple_mat_view ON CLUSTER 'cluster' FORMAT TSV")
ddl_check_query(instance, "DROP TABLE test.super_simple_view2 ON CLUSTER 'cluster' FORMAT TSV")
ddl_check_query(instance, "CREATE TABLE test.super_simple (i Int8) ON CLUSTER 'cluster'")
ddl_check_query(instance, "RENAME TABLE test.super_simple TO test.super_simple2 ON CLUSTER 'cluster' FORMAT TSV")
ddl_check_query(instance, "DROP TABLE test.super_simple2 ON CLUSTER 'cluster'")
def test_on_server_fail(started_cluster):
......@@ -274,7 +278,7 @@ ENGINE = Distributed(cluster_without_replication, default, merge, i)
ddl_check_query(instance, "ALTER TABLE merge ON CLUSTER cluster_without_replication MODIFY COLUMN i Int64")
ddl_check_query(instance, "ALTER TABLE merge ON CLUSTER cluster_without_replication ADD COLUMN s DEFAULT toString(i)")
ddl_check_query(instance, "ALTER TABLE merge ON CLUSTER cluster_without_replication ADD COLUMN s DEFAULT toString(i) FORMAT TSV")
assert TSV(instance.query("SELECT i, s FROM all_merge_64 ORDER BY i")) == TSV(''.join(['{}\t{}\n'.format(x,x) for x in xrange(4)]))
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册