Simplification of AST #1947

This commit is contained in:
Alexey Milovidov 2018-02-26 06:37:08 +03:00
parent 7093979336
commit 601ff1eee4
78 changed files with 132 additions and 344 deletions

View File

@ -135,7 +135,7 @@ ASTPtr createASTIdentifierForColumnInTable(const String & column, const CollectT
{
ASTPtr database_name_identifier_node;
if (!table.database_name.empty())
database_name_identifier_node = std::make_shared<ASTIdentifier>(StringRange(), table.database_name, ASTIdentifier::Column);
database_name_identifier_node = std::make_shared<ASTIdentifier>(table.database_name, ASTIdentifier::Column);
ASTPtr table_name_identifier_node;
String table_name_or_alias;
@ -146,9 +146,9 @@ ASTPtr createASTIdentifierForColumnInTable(const String & column, const CollectT
table_name_or_alias = table.alias;
if (!table_name_or_alias.empty())
table_name_identifier_node = std::make_shared<ASTIdentifier>(StringRange(), table_name_or_alias, ASTIdentifier::Column);
table_name_identifier_node = std::make_shared<ASTIdentifier>(table_name_or_alias, ASTIdentifier::Column);
ASTPtr column_identifier_node = std::make_shared<ASTIdentifier>(StringRange(), column, ASTIdentifier::Column);
ASTPtr column_identifier_node = std::make_shared<ASTIdentifier>(column, ASTIdentifier::Column);
String compound_name;
if (database_name_identifier_node)
@ -157,8 +157,7 @@ ASTPtr createASTIdentifierForColumnInTable(const String & column, const CollectT
compound_name += table_name_or_alias + ".";
compound_name += column;
auto elem = std::make_shared<ASTIdentifier>(
StringRange(), compound_name, ASTIdentifier::Column);
auto elem = std::make_shared<ASTIdentifier>(compound_name, ASTIdentifier::Column);
if (database_name_identifier_node)
elem->children.emplace_back(std::move(database_name_identifier_node));

View File

@ -123,9 +123,7 @@ std::ostream & operator<<(std::ostream & stream, const SubqueryForSet & what)
std::ostream & operator<<(std::ostream & stream, const IAST & what)
{
stream << "IAST("
<< "query_string = " << what.query_string
<<"){";
stream << "IAST{";
what.dumpTree(stream);
stream << "}";
return stream;

View File

@ -112,7 +112,7 @@ Block Aggregator::getHeader(bool final) const
else
type = std::make_shared<DataTypeAggregateFunction>(params.aggregates[i].function, argument_types, params.aggregates[i].parameters);
res.insert({ type->createColumn(), type, params.aggregates[i].column_name });
res.insert({ type, params.aggregates[i].column_name });
}
}
else if (params.intermediate_header)

View File

@ -359,7 +359,7 @@ void ExpressionAnalyzer::translateQualifiedNamesImpl(ASTPtr & ast, const String
|| (!alias.empty() && ident->name == alias))))
{
/// Replace to plain asterisk.
ast = std::make_shared<ASTAsterisk>(ast->range);
ast = std::make_shared<ASTAsterisk>();
}
}
else
@ -688,8 +688,7 @@ static std::shared_ptr<InterpreterSelectQuery> interpretSubquery(
/// manually substitute column names in place of asterisk
for (const auto & column : columns)
select_expression_list->children.emplace_back(std::make_shared<ASTIdentifier>(
StringRange{}, column.name));
select_expression_list->children.emplace_back(std::make_shared<ASTIdentifier>(column.name));
select_query->replaceDatabaseAndTable(database_table.first, database_table.second);
}
@ -808,7 +807,7 @@ void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name_or_t
* instead of doing a subquery, you just need to read it.
*/
auto database_and_table_name = std::make_shared<ASTIdentifier>(StringRange(), external_table_name, ASTIdentifier::Table);
auto database_and_table_name = std::make_shared<ASTIdentifier>(external_table_name, ASTIdentifier::Table);
if (auto ast_table_expr = typeid_cast<ASTTableExpression *>(subquery_or_table_name_or_table_expression.get()))
{
@ -1032,7 +1031,7 @@ void ExpressionAnalyzer::normalizeTreeImpl(
{
ASTs all_columns;
for (const auto & column_name_type : columns)
all_columns.emplace_back(std::make_shared<ASTIdentifier>(asterisk->range, column_name_type.name));
all_columns.emplace_back(std::make_shared<ASTIdentifier>(column_name_type.name));
asts.erase(asts.begin() + i);
asts.insert(asts.begin() + i, all_columns.begin(), all_columns.end());
@ -1147,17 +1146,17 @@ void ExpressionAnalyzer::executeScalarSubqueries()
static ASTPtr addTypeConversion(std::unique_ptr<ASTLiteral> && ast, const String & type_name)
{
auto func = std::make_shared<ASTFunction>(ast->range);
auto func = std::make_shared<ASTFunction>();
ASTPtr res = func;
func->alias = ast->alias;
func->prefer_alias_to_column_name = ast->prefer_alias_to_column_name;
ast->alias.clear();
func->name = "CAST";
auto exp_list = std::make_shared<ASTExpressionList>(ast->range);
auto exp_list = std::make_shared<ASTExpressionList>();
func->arguments = exp_list;
func->children.push_back(func->arguments);
exp_list->children.emplace_back(ast.release());
exp_list->children.emplace_back(std::make_shared<ASTLiteral>(StringRange(), type_name));
exp_list->children.emplace_back(std::make_shared<ASTLiteral>(type_name));
return res;
}
@ -1200,7 +1199,7 @@ void ExpressionAnalyzer::executeScalarSubqueriesImpl(ASTPtr & ast)
if (!block)
{
/// Interpret subquery with empty result as Null literal
auto ast_new = std::make_unique<ASTLiteral>(ast->range, Null());
auto ast_new = std::make_unique<ASTLiteral>(Null());
ast_new->setAlias(ast->tryGetAlias());
ast = std::move(ast_new);
return;
@ -1220,18 +1219,18 @@ void ExpressionAnalyzer::executeScalarSubqueriesImpl(ASTPtr & ast)
size_t columns = block.columns();
if (columns == 1)
{
auto lit = std::make_unique<ASTLiteral>(ast->range, (*block.safeGetByPosition(0).column)[0]);
auto lit = std::make_unique<ASTLiteral>((*block.safeGetByPosition(0).column)[0]);
lit->alias = subquery->alias;
lit->prefer_alias_to_column_name = subquery->prefer_alias_to_column_name;
ast = addTypeConversion(std::move(lit), block.safeGetByPosition(0).type->getName());
}
else
{
auto tuple = std::make_shared<ASTFunction>(ast->range);
auto tuple = std::make_shared<ASTFunction>();
tuple->alias = subquery->alias;
ast = tuple;
tuple->name = "tuple";
auto exp_list = std::make_shared<ASTExpressionList>(ast->range);
auto exp_list = std::make_shared<ASTExpressionList>();
tuple->arguments = exp_list;
tuple->children.push_back(tuple->arguments);
@ -1239,7 +1238,7 @@ void ExpressionAnalyzer::executeScalarSubqueriesImpl(ASTPtr & ast)
for (size_t i = 0; i < columns; ++i)
{
exp_list->children[i] = addTypeConversion(
std::make_unique<ASTLiteral>(ast->range, (*block.safeGetByPosition(i).column)[0]),
std::make_unique<ASTLiteral>((*block.safeGetByPosition(i).column)[0]),
block.safeGetByPosition(i).type->getName());
}
}
@ -1375,7 +1374,7 @@ void ExpressionAnalyzer::optimizeGroupBy()
}
select_query->group_expression_list = std::make_shared<ASTExpressionList>();
select_query->group_expression_list->children.emplace_back(std::make_shared<ASTLiteral>(StringRange(), UInt64(unused_column)));
select_query->group_expression_list->children.emplace_back(std::make_shared<ASTLiteral>(UInt64(unused_column)));
}
}

View File

@ -102,20 +102,18 @@ StoragePtr tryGetTable(const ASTPtr & database_and_table, const Context & contex
void replaceDatabaseAndTable(ASTPtr & database_and_table, const String & database_name, const String & table_name)
{
ASTPtr table = std::make_shared<ASTIdentifier>(StringRange(), table_name, ASTIdentifier::Table);
ASTPtr table = std::make_shared<ASTIdentifier>(table_name, ASTIdentifier::Table);
if (!database_name.empty())
{
ASTPtr database = std::make_shared<ASTIdentifier>(StringRange(), database_name, ASTIdentifier::Database);
ASTPtr database = std::make_shared<ASTIdentifier>(database_name, ASTIdentifier::Database);
database_and_table = std::make_shared<ASTIdentifier>(
StringRange(), database_name + "." + table_name, ASTIdentifier::Table);
database_and_table = std::make_shared<ASTIdentifier>(database_name + "." + table_name, ASTIdentifier::Table);
database_and_table->children = {database, table};
}
else
{
database_and_table = std::make_shared<ASTIdentifier>(
StringRange(), table_name, ASTIdentifier::Table);
database_and_table = std::make_shared<ASTIdentifier>(table_name, ASTIdentifier::Table);
}
}

View File

@ -106,9 +106,7 @@ void InterpreterAlterQuery::parseAlter(
command.column_name = ast_col_decl.name;
if (ast_col_decl.type)
{
StringRange type_range = ast_col_decl.type->range;
String type_string(type_range.first, type_range.second - type_range.first);
command.data_type = data_type_factory.get(type_string);
command.data_type = data_type_factory.get(ast_col_decl.type);
}
if (ast_col_decl.default_expression)
{
@ -154,9 +152,7 @@ void InterpreterAlterQuery::parseAlter(
command.column_name = ast_col_decl.name;
if (ast_col_decl.type)
{
StringRange type_range = ast_col_decl.type->range;
String type_string(type_range.first, type_range.second - type_range.first);
command.data_type = data_type_factory.get(type_string);
command.data_type = data_type_factory.get(ast_col_decl.type);
}
if (ast_col_decl.default_expression)

View File

@ -199,8 +199,8 @@ static ColumnsAndDefaults parseColumns(const ASTExpressionList & column_list_ast
const auto data_type_ptr = columns.back().type.get();
default_expr_list->children.emplace_back(setAlias(
makeASTFunction("CAST", std::make_shared<ASTIdentifier>(StringRange(), tmp_column_name),
std::make_shared<ASTLiteral>(StringRange(), Field(data_type_ptr->getName()))), final_column_name));
makeASTFunction("CAST", std::make_shared<ASTIdentifier>(tmp_column_name),
std::make_shared<ASTLiteral>(Field(data_type_ptr->getName()))), final_column_name));
default_expr_list->children.emplace_back(setAlias(col_decl.default_expression->clone(), tmp_column_name));
}
else
@ -233,7 +233,7 @@ static ColumnsAndDefaults parseColumns(const ASTExpressionList & column_list_ast
if (!explicit_type->equals(*deduced_type))
{
col_decl_ptr->default_expression = makeASTFunction("CAST", col_decl_ptr->default_expression,
std::make_shared<ASTLiteral>(StringRange(), explicit_type->getName()));
std::make_shared<ASTLiteral>(explicit_type->getName()));
col_decl_ptr->children.clear();
col_decl_ptr->children.push_back(col_decl_ptr->type);
@ -293,7 +293,7 @@ ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns)
ParserIdentifierWithOptionalParameters storage_p;
column_declaration->type = parseQuery(storage_p, pos, end, "data type");
column_declaration->type->query_string = type_name;
column_declaration->type->owned_string = type_name;
columns_list->children.emplace_back(column_declaration);
}
@ -321,7 +321,7 @@ ASTPtr InterpreterCreateQuery::formatColumns(
ParserIdentifierWithOptionalParameters storage_p;
column_declaration->type = parseQuery(storage_p, pos, end, "data type");
column_declaration->type->query_string = type_name;
column_declaration->type->owned_string = type_name;
const auto it = column_defaults.find(column.name);
if (it != std::end(column_defaults))

View File

@ -553,7 +553,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline
if (default_it != std::end(storage->column_defaults) && default_it->second.type == ColumnDefaultType::Alias)
required_columns_expr_list->children.emplace_back(setAlias(default_it->second.expression->clone(), column));
else
required_columns_expr_list->children.emplace_back(std::make_shared<ASTIdentifier>(StringRange(), column));
required_columns_expr_list->children.emplace_back(std::make_shared<ASTIdentifier>(column));
}
alias_actions = ExpressionAnalyzer{required_columns_expr_list, context, storage, source_header.getNamesAndTypesList()}.getActions(true);

View File

@ -55,15 +55,14 @@ ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, const Context &
if (typeid_cast<const ASTLiteral *>(node.get()))
return node;
return std::make_shared<ASTLiteral>(node->range,
evaluateConstantExpression(node, context).first);
return std::make_shared<ASTLiteral>(evaluateConstantExpression(node, context).first);
}
ASTPtr evaluateConstantExpressionOrIdentifierAsLiteral(const ASTPtr & node, const Context & context)
{
if (auto id = typeid_cast<const ASTIdentifier *>(node.get()))
return std::make_shared<ASTLiteral>(node->range, Field(id->name));
return std::make_shared<ASTLiteral>(Field(id->name));
return evaluateConstantExpressionAsLiteral(node, context);
}

View File

@ -33,10 +33,6 @@ void ASTAlterQuery::addParameters(const Parameters & params)
children.push_back(params.primary_key);
}
ASTAlterQuery::ASTAlterQuery(StringRange range_) : ASTQueryWithOutput(range_)
{
}
/** Get the text that identifies this element. */
String ASTAlterQuery::getID() const
{

View File

@ -87,8 +87,6 @@ public:
void addParameters(const Parameters & params);
explicit ASTAlterQuery(StringRange range_ = StringRange());
/** Get the text that identifies this element. */
String getID() const override;

View File

@ -9,8 +9,6 @@ namespace DB
class ASTAsterisk : public IAST
{
public:
ASTAsterisk() = default;
ASTAsterisk(StringRange range_) : IAST(range_) {}
String getID() const override { return "Asterisk"; }
ASTPtr clone() const override { return std::make_shared<ASTAsterisk>(*this); }
String getColumnName() const override { return "*"; }

View File

@ -7,8 +7,6 @@ namespace DB
struct ASTCheckQuery : public ASTQueryWithOutput
{
ASTCheckQuery(StringRange range_ = StringRange()) : ASTQueryWithOutput(range_) {};
/** Get the text that identifies this element. */
String getID() const override { return ("CheckQuery_" + database + "_" + table); };

View File

@ -16,9 +16,6 @@ public:
String default_specifier;
ASTPtr default_expression;
ASTColumnDeclaration() = default;
ASTColumnDeclaration(const StringRange range) : IAST{range} {}
String getID() const override { return "ColumnDeclaration_" + name; }
ASTPtr clone() const override

View File

@ -20,8 +20,6 @@ public:
IAST * sample_by = nullptr;
ASTSetQuery * settings = nullptr;
ASTStorage() = default;
ASTStorage(StringRange range_) : IAST(range_) {}
String getID() const override { return "Storage definition"; }
ASTPtr clone() const override
@ -95,9 +93,6 @@ public:
String as_table;
ASTSelectWithUnionQuery * select = nullptr;
ASTCreateQuery() = default;
ASTCreateQuery(const StringRange range_) : ASTQueryWithOutput(range_) {}
/** Get the text that identifies this element. */
String getID() const override { return (attach ? "AttachQuery_" : "CreateQuery_") + database + "_" + table; };

View File

@ -19,9 +19,6 @@ public:
String database;
String table;
ASTDropQuery() = default;
explicit ASTDropQuery(const StringRange range_) : ASTQueryWithOutput(range_) {}
/** Get the text that identifies this element. */
String getID() const override { return (detach ? "DetachQuery_" : "DropQuery_") + database + "_" + table; };

View File

@ -14,14 +14,14 @@ public:
String name;
Field value;
ASTEnumElement(const StringRange range, const String & name, const Field & value)
: IAST{range}, name{name}, value {value} {}
ASTEnumElement(const String & name, const Field & value)
: name{name}, value {value} {}
String getID() const override { return "EnumElement"; }
ASTPtr clone() const override
{
return std::make_shared<ASTEnumElement>(StringRange(), name, value);
return std::make_shared<ASTEnumElement>(name, value);
}
protected:

View File

@ -11,9 +11,6 @@ namespace DB
class ASTExpressionList : public IAST
{
public:
ASTExpressionList() = default;
ASTExpressionList(const StringRange range_) : IAST(range_) {}
String getID() const override { return "ExpressionList"; }
ASTPtr clone() const override;

View File

@ -18,9 +18,6 @@ public:
ASTPtr parameters;
public:
ASTFunction() = default;
ASTFunction(const StringRange range_) : ASTWithAlias(range_) {}
/** Get text identifying the AST node. */
String getID() const override;
@ -36,7 +33,6 @@ template <typename... Args>
ASTPtr makeASTFunction(const String & name, Args &&... args)
{
const auto function = std::make_shared<ASTFunction>();
ASTPtr result{function};
function->name = name;
function->arguments = std::make_shared<ASTExpressionList>();
@ -44,24 +40,7 @@ ASTPtr makeASTFunction(const String & name, Args &&... args)
function->arguments->children = { std::forward<Args>(args)... };
return result;
}
template <typename... Args>
ASTPtr makeASTFunction(const String & name, const StringRange & function_range,
const StringRange & arguments_range, Args &&... args)
{
const auto function = std::make_shared<ASTFunction>(function_range);
ASTPtr result{function};
function->name = name;
function->arguments = std::make_shared<ASTExpressionList>(arguments_range);
function->children.push_back(function->arguments);
function->arguments->children = { std::forward<Args>(args)... };
return result;
return function;
}
}

View File

@ -25,9 +25,8 @@ public:
/// what this identifier identifies
Kind kind;
ASTIdentifier() = default;
ASTIdentifier(const StringRange range_, const String & name_, const Kind kind_ = Column)
: ASTWithAlias(range_), name(name_), kind(kind_) {}
ASTIdentifier(const String & name_, const Kind kind_ = Column)
: name(name_), kind(kind_) {}
/** Get the text that identifies this element. */
String getID() const override { return "Identifier_" + name; }

View File

@ -26,9 +26,6 @@ public:
const char * data = nullptr;
const char * end = nullptr;
ASTInsertQuery() = default;
explicit ASTInsertQuery(const StringRange range_) : IAST(range_) {}
/** Get the text that identifies this element. */
String getID() const override { return "InsertQuery_" + database + "_" + table; };

View File

@ -11,10 +11,6 @@ public:
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;
ASTKillQueryQuery(const StringRange range_) : ASTQueryWithOutput(range_) {}
ASTPtr clone() const override { return std::make_shared<ASTKillQueryQuery>(*this); }
String getID() const override;

View File

@ -15,8 +15,7 @@ class ASTLiteral : public ASTWithAlias
public:
Field value;
ASTLiteral() = default;
ASTLiteral(const StringRange range_, const Field & value_) : ASTWithAlias(range_), value(value_) {}
ASTLiteral(const Field & value_) : value(value_) {}
/** Get the text that identifies this element. */
String getID() const override { return "Literal_" + applyVisitor(FieldVisitorDump(), value); }

View File

@ -16,9 +16,6 @@ public:
/// type
ASTPtr type;
ASTNameTypePair() = default;
ASTNameTypePair(const StringRange range_) : IAST(range_) {}
/** Get the text that identifies this element. */
String getID() const override { return "NameTypePair_" + name; }

View File

@ -22,9 +22,6 @@ public:
/// Do deduplicate (default: false)
bool deduplicate;
ASTOptimizeQuery() = default;
ASTOptimizeQuery(const StringRange range_) : IAST(range_) {}
/** Get the text that identifies this element. */
String getID() const override { return "OptimizeQuery_" + database + "_" + table + (final ? "_final" : "") + (deduplicate ? "_deduplicate" : ""); };

View File

@ -19,11 +19,12 @@ public:
/** Collation for locale-specific string comparison. If empty, then sorting done by bytes. */
ASTPtr collation;
ASTOrderByElement() = default;
ASTOrderByElement(const StringRange range_,
const int direction_, const int nulls_direction_, const bool nulls_direction_was_explicitly_specified_,
ASTOrderByElement(
const int direction_,
const int nulls_direction_,
const bool nulls_direction_was_explicitly_specified_,
ASTPtr & collation_)
: IAST(range_),
:
direction(direction_), nulls_direction(nulls_direction_),
nulls_direction_was_explicitly_specified(nulls_direction_was_explicitly_specified_),
collation(collation_) {}

View File

@ -17,8 +17,6 @@ public:
String id;
ASTPartition() = default;
ASTPartition(StringRange range_) : IAST(range_) {}
String getID() const override;
ASTPtr clone() const override;

View File

@ -12,8 +12,6 @@ namespace DB
class ASTQualifiedAsterisk : public IAST
{
public:
ASTQualifiedAsterisk() = default;
ASTQualifiedAsterisk(StringRange range_) : IAST(range_) {}
String getID() const override { return "QualifiedAsterisk"; }
ASTPtr clone() const override { return std::make_shared<ASTQualifiedAsterisk>(*this); }
String getColumnName() const override;

View File

@ -14,9 +14,6 @@ public:
ASTPtr out_file;
ASTPtr format;
ASTQueryWithOutput() = default;
explicit ASTQueryWithOutput(const StringRange range_) : IAST(range_) {}
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const final;
/// Remove 'FORMAT <fmt> and INTO OUTFILE <file>' if exists
@ -37,8 +34,6 @@ template <typename ASTIDAndQueryNames>
class ASTQueryWithOutputImpl : public ASTQueryWithOutput
{
public:
explicit ASTQueryWithOutputImpl() = default;
explicit ASTQueryWithOutputImpl(StringRange range_) : ASTQueryWithOutput(range_) {}
String getID() const override { return ASTIDAndQueryNames::ID; };
ASTPtr clone() const override

View File

@ -16,9 +16,6 @@ public:
String database;
String table;
ASTQueryWithTableAndOutput() = default;
explicit ASTQueryWithTableAndOutput(const StringRange range_) : ASTQueryWithOutput(range_) {}
protected:
void formatHelper(const FormatSettings & settings, const char * name) const
{
@ -32,10 +29,6 @@ template <typename AstIDAndQueryNames>
class ASTQueryWithTableAndOutputImpl : public ASTQueryWithTableAndOutput
{
public:
ASTQueryWithTableAndOutputImpl() = default;
explicit ASTQueryWithTableAndOutputImpl(const StringRange range_) : ASTQueryWithTableAndOutput(range_) {}
String getID() const override { return AstIDAndQueryNames::ID + ("_" + database) + "_" + table; };
ASTPtr clone() const override

View File

@ -28,9 +28,6 @@ public:
using Elements = std::vector<Element>;
Elements elements;
ASTRenameQuery() = default;
explicit ASTRenameQuery(const StringRange range_) : ASTQueryWithOutput(range_) {}
/** Get the text that identifies this element. */
String getID() const override { return "Rename"; };

View File

@ -26,9 +26,7 @@ public:
Rational ratio;
ASTSampleRatio() = default;
ASTSampleRatio(const StringRange range_) : IAST(range_) {}
ASTSampleRatio(const StringRange range_, Rational & ratio_) : IAST(range_), ratio(ratio_) {}
ASTSampleRatio(Rational & ratio_) : ratio(ratio_) {}
String getID() const override { return "SampleRatio_" + toString(ratio); }

View File

@ -19,10 +19,6 @@ namespace ErrorCodes
}
ASTSelectQuery::ASTSelectQuery(const StringRange range_) : IAST(range_)
{
}
bool ASTSelectQuery::hasArrayJoin(const ASTPtr & ast)
{
if (const ASTFunction * function = typeid_cast<const ASTFunction *>(&*ast))
@ -101,7 +97,7 @@ void ASTSelectQuery::rewriteSelectExpressionList(const Names & required_column_n
if (!other_required_columns_in_select.count(name) && !columns_with_array_join.count(name))
{
if (asterisk.first)
new_children.push_back({ std::make_shared<ASTIdentifier>(asterisk.first->range, name), asterisk.second });
new_children.push_back({ std::make_shared<ASTIdentifier>(name), asterisk.second });
else
throw Exception("SELECT query doesn't have required column: " + backQuoteIfNeed(name), ErrorCodes::THERE_IS_NO_COLUMN);
}
@ -455,11 +451,11 @@ void ASTSelectQuery::setDatabaseIfNeeded(const String & database_name)
if (table_expression->database_and_table_name->children.empty())
{
ASTPtr database = std::make_shared<ASTIdentifier>(StringRange(), database_name, ASTIdentifier::Database);
ASTPtr database = std::make_shared<ASTIdentifier>(database_name, ASTIdentifier::Database);
ASTPtr table = table_expression->database_and_table_name;
const String & old_name = static_cast<ASTIdentifier &>(*table_expression->database_and_table_name).name;
table_expression->database_and_table_name = std::make_shared<ASTIdentifier>(StringRange(), database_name + "." + old_name, ASTIdentifier::Table);
table_expression->database_and_table_name = std::make_shared<ASTIdentifier>(database_name + "." + old_name, ASTIdentifier::Table);
table_expression->database_and_table_name->children = {database, table};
}
else if (table_expression->database_and_table_name->children.size() != 2)
@ -486,20 +482,18 @@ void ASTSelectQuery::replaceDatabaseAndTable(const String & database_name, const
table_expression = table_expr.get();
}
ASTPtr table = std::make_shared<ASTIdentifier>(StringRange(), table_name, ASTIdentifier::Table);
ASTPtr table = std::make_shared<ASTIdentifier>(table_name, ASTIdentifier::Table);
if (!database_name.empty())
{
ASTPtr database = std::make_shared<ASTIdentifier>(StringRange(), database_name, ASTIdentifier::Database);
ASTPtr database = std::make_shared<ASTIdentifier>(database_name, ASTIdentifier::Database);
table_expression->database_and_table_name = std::make_shared<ASTIdentifier>(
StringRange(), database_name + "." + table_name, ASTIdentifier::Table);
table_expression->database_and_table_name = std::make_shared<ASTIdentifier>(database_name + "." + table_name, ASTIdentifier::Table);
table_expression->database_and_table_name->children = {database, table};
}
else
{
table_expression->database_and_table_name = std::make_shared<ASTIdentifier>(
StringRange(), table_name, ASTIdentifier::Table);
table_expression->database_and_table_name = std::make_shared<ASTIdentifier>(table_name, ASTIdentifier::Table);
}
}

View File

@ -15,9 +15,6 @@ struct ASTTablesInSelectQueryElement;
class ASTSelectQuery : public IAST
{
public:
ASTSelectQuery() = default;
ASTSelectQuery(const StringRange range_);
/** Get the text that identifies this element. */
String getID() const override { return "SelectQuery"; };

View File

@ -25,9 +25,6 @@ public:
using Changes = std::vector<Change>;
Changes changes;
ASTSetQuery() = default;
explicit ASTSetQuery(const StringRange range_) : IAST(range_) {}
/** Get the text that identifies this element. */
String getID() const override { return "Set"; };

View File

@ -20,9 +20,6 @@ public:
String like;
bool not_like{false};
ASTShowTablesQuery() = default;
ASTShowTablesQuery(const StringRange range_) : ASTQueryWithOutput(range_) {}
/** Get the text that identifies this element. */
String getID() const override { return "ShowTables"; };

View File

@ -12,9 +12,6 @@ namespace DB
class ASTSubquery : public ASTWithAlias
{
public:
ASTSubquery() = default;
ASTSubquery(const StringRange range_) : ASTWithAlias(range_) {}
/** Get the text that identifies this element. */
String getID() const override { return "Subquery"; }
@ -44,6 +41,7 @@ protected:
children[0]->formatImpl(settings, state, frame_nested);
settings.ostr << nl_or_nothing << indent_str << ")";
}
String getColumnNameImpl() const override;
};

View File

@ -39,9 +39,6 @@ public:
//String target_replica_database;
//String target_replica_table;
ASTSystemQuery() = default;
explicit ASTSystemQuery(const StringRange range) : IAST(range) {}
String getID() const override { return "SYSTEM query"; };
ASTPtr clone() const override { return std::make_shared<ASTSystemQuery>(*this); }

View File

@ -14,9 +14,6 @@ class ASTUseQuery : public IAST
public:
String database;
ASTUseQuery() = default;
ASTUseQuery(const StringRange range_) : IAST(range_) {}
/** Get the text that identifies this element. */
String getID() const override { return "UseQuery_" + database; };

View File

@ -35,7 +35,6 @@ namespace ErrorCodes
bool ParserArray::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
ASTPtr contents_node;
ParserExpressionList contents(false);
@ -50,7 +49,7 @@ bool ParserArray::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false;
++pos;
auto function_node = std::make_shared<ASTFunction>(StringRange(begin, pos));
auto function_node = std::make_shared<ASTFunction>();
function_node->name = "array";
function_node->arguments = contents_node;
function_node->children.push_back(contents_node);
@ -62,7 +61,6 @@ bool ParserArray::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool ParserParenthesisExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
ASTPtr contents_node;
ParserExpressionList contents(false);
@ -92,7 +90,7 @@ bool ParserParenthesisExpression::parseImpl(Pos & pos, ASTPtr & node, Expected &
}
else
{
auto function_node = std::make_shared<ASTFunction>(StringRange(begin, pos));
auto function_node = std::make_shared<ASTFunction>();
function_node->name = "tuple";
function_node->arguments = contents_node;
function_node->children.push_back(contents_node);
@ -105,7 +103,6 @@ bool ParserParenthesisExpression::parseImpl(Pos & pos, ASTPtr & node, Expected &
bool ParserSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
ASTPtr select_node;
ParserSelectWithUnionQuery select;
@ -120,7 +117,7 @@ bool ParserSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false;
++pos;
node = std::make_shared<ASTSubquery>(StringRange(begin, pos));
node = std::make_shared<ASTSubquery>();
typeid_cast<ASTSubquery &>(*node).children.push_back(select_node);
return true;
}
@ -128,8 +125,6 @@ bool ParserSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool ParserIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected &)
{
Pos begin = pos;
/// Identifier in backquotes or in double quotes
if (pos->type == TokenType::QuotedIdentifier)
{
@ -144,14 +139,14 @@ bool ParserIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected &)
if (s.empty()) /// Identifiers "empty string" are not allowed.
return false;
node = std::make_shared<ASTIdentifier>(s);
++pos;
node = std::make_shared<ASTIdentifier>(StringRange(begin), s);
return true;
}
else if (pos->type == TokenType::BareWord)
{
node = std::make_shared<ASTIdentifier>(String(pos->begin, pos->end));
++pos;
node = std::make_shared<ASTIdentifier>(StringRange(begin), String(begin->begin, begin->end));
return true;
}
@ -161,8 +156,6 @@ bool ParserIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected &)
bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
ASTPtr id_list;
if (!ParserList(std::make_unique<ParserIdentifier>(), std::make_unique<ParserToken>(TokenType::Dot), false)
.parse(pos, id_list, expected))
@ -177,7 +170,7 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex
name += static_cast<const ASTIdentifier &>(*child.get()).name;
}
node = std::make_shared<ASTIdentifier>(StringRange(begin, pos), name);
node = std::make_shared<ASTIdentifier>(name);
/// In `children`, remember the identifiers-components, if there are more than one.
if (list.children.size() > 1)
@ -189,8 +182,6 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex
bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
ParserIdentifier id_parser;
ParserKeyword distinct("DISTINCT");
ParserExpressionList contents(false);
@ -266,7 +257,7 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
++pos;
}
auto function_node = std::make_shared<ASTFunction>(StringRange(begin, pos));
auto function_node = std::make_shared<ASTFunction>();
function_node->name = typeid_cast<ASTIdentifier &>(*identifier).name;
/// func(DISTINCT ...) is equivalent to funcDistinct(...)
@ -334,10 +325,10 @@ bool ParserCastExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
return false;
}
expr_list_args = std::make_shared<ASTExpressionList>(StringRange{contents_begin, pos});
expr_list_args = std::make_shared<ASTExpressionList>();
first_argument->setAlias({});
expr_list_args->children.push_back(first_argument);
expr_list_args->children.emplace_back(std::make_shared<ASTLiteral>(StringRange(), type));
expr_list_args->children.emplace_back(std::make_shared<ASTLiteral>(type));
}
else
{
@ -362,7 +353,7 @@ bool ParserCastExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
return false;
}
expr_list_args = std::make_shared<ASTExpressionList>(StringRange{contents_begin, pos});
expr_list_args = std::make_shared<ASTExpressionList>();
expr_list_args->children.push_back(first_argument);
expr_list_args->children.push_back(type_as_literal);
}
@ -371,7 +362,7 @@ bool ParserCastExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
return false;
++pos;
const auto function_node = std::make_shared<ASTFunction>(StringRange(begin, pos));
const auto function_node = std::make_shared<ASTFunction>();
ASTPtr node_holder{function_node};
function_node->name = name;
@ -385,11 +376,10 @@ bool ParserCastExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
bool ParserNull::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
ParserKeyword nested_parser("NULL");
if (nested_parser.parse(pos, node, expected))
{
node = std::make_shared<ASTLiteral>(StringRange(StringRange(begin, pos)), Null());
node = std::make_shared<ASTLiteral>(Null());
return true;
}
else
@ -411,7 +401,6 @@ bool ParserNumber::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
Field res;
Pos begin = pos;
if (!pos.isValid())
return false;
@ -463,7 +452,7 @@ bool ParserNumber::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
++pos;
node = std::make_shared<ASTLiteral>(StringRange(begin, pos), res);
node = std::make_shared<ASTLiteral>(res);
return true;
}
@ -472,7 +461,6 @@ bool ParserUnsignedInteger::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
{
Field res;
Pos begin = pos;
if (!pos.isValid())
return false;
@ -486,7 +474,7 @@ bool ParserUnsignedInteger::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
res = x;
++pos;
node = std::make_shared<ASTLiteral>(StringRange(begin, pos), res);
node = std::make_shared<ASTLiteral>(res);
return true;
}
@ -496,8 +484,6 @@ bool ParserStringLiteral::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte
if (pos->type != TokenType::StringLiteral)
return false;
Pos begin = pos;
String s;
ReadBufferFromMemory in(pos->begin, pos->size());
@ -518,7 +504,7 @@ bool ParserStringLiteral::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte
}
++pos;
node = std::make_shared<ASTLiteral>(StringRange(begin, pos), s);
node = std::make_shared<ASTLiteral>(s);
return true;
}
@ -528,7 +514,6 @@ bool ParserArrayOfLiterals::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
if (pos->type != TokenType::OpeningSquareBracket)
return false;
Pos begin = pos;
Array arr;
ParserLiteral literal_p;
@ -542,7 +527,7 @@ bool ParserArrayOfLiterals::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
if (pos->type == TokenType::ClosingSquareBracket)
{
++pos;
node = std::make_shared<ASTLiteral>(StringRange(begin, pos), arr);
node = std::make_shared<ASTLiteral>(arr);
return true;
}
else if (pos->type == TokenType::Comma)
@ -654,11 +639,10 @@ template class ParserAliasImpl<ParserTypeInCastExpression>;
bool ParserAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected &)
{
Pos begin = pos;
if (pos->type == TokenType::Asterisk)
{
++pos;
node = std::make_shared<ASTAsterisk>(StringRange(begin, pos));
node = std::make_shared<ASTAsterisk>();
return true;
}
return false;
@ -667,8 +651,6 @@ bool ParserAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected &)
bool ParserQualifiedAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
if (!ParserCompoundIdentifier().parse(pos, node, expected))
return false;
@ -680,7 +662,7 @@ bool ParserQualifiedAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
return false;
++pos;
auto res = std::make_shared<ASTQualifiedAsterisk>(StringRange(begin, pos));
auto res = std::make_shared<ASTQualifiedAsterisk>();
res->children.push_back(node);
node = std::move(res);
return true;
@ -787,8 +769,6 @@ template class ParserWithOptionalAliasImpl<ParserCastExpressionAlias>;
bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
ParserExpressionWithOptionalAlias elem_p(false);
ParserKeyword ascending("ASCENDING");
ParserKeyword descending("DESCENDING");
@ -833,7 +813,7 @@ bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
return false;
}
node = std::make_shared<ASTOrderByElement>(StringRange(begin, pos), direction, nulls_direction, nulls_direction_was_explicitly_specified, locale_node);
node = std::make_shared<ASTOrderByElement>(direction, nulls_direction, nulls_direction_was_explicitly_specified, locale_node);
node->children.push_back(expr_elem);
if (locale_node)
node->children.push_back(locale_node);

View File

@ -206,7 +206,6 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node
bool ParserVariableArityOperatorList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
ASTPtr arguments;
if (!elem_parser->parse(pos, node, expected))
@ -230,9 +229,6 @@ bool ParserVariableArityOperatorList::parseImpl(Pos & pos, ASTPtr & node, Expect
arguments->children.push_back(elem);
}
if (arguments)
arguments->range = node->range = StringRange(begin, pos);
return true;
}
@ -575,8 +571,6 @@ bool ParserNullityChecking::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
if (!ParserComparisonExpression{}.parse(pos, node_comp, expected))
return false;
Pos begin = pos;
ParserKeyword s_is{"IS"};
ParserKeyword s_not{"NOT"};
ParserKeyword s_null{"NULL"};
@ -593,7 +587,7 @@ bool ParserNullityChecking::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
auto args = std::make_shared<ASTExpressionList>();
args->children.push_back(node_comp);
auto function = std::make_shared<ASTFunction>(StringRange{begin, pos});
auto function = std::make_shared<ASTFunction>();
function->name = is_not ? "isNotNull" : "isNull";
function->arguments = args;
function->children.push_back(function->arguments);

View File

@ -40,13 +40,9 @@ public:
ASTs children;
StringRange range;
/** A string with a full query.
* This pointer does not allow it to be deleted while the range refers to it.
*/
StringPtr query_string;
/// This pointer does not allow it to be deleted while the range refers to it.
StringPtr owned_string;
IAST() = default;
IAST(const StringRange range_) : range(range_) {}
virtual ~IAST() = default;
/** Get the canonical name of the column if the element is a column */

View File

@ -17,13 +17,13 @@ bool IParserBase::parse(Pos & pos, ASTPtr & node, Expected & expected)
bool res = parseImpl(pos, node, expected);
/// TODO expected
if (!res)
{
node = nullptr;
pos = begin;
}
else
node->range = StringRange(begin, pos);
return res;
}

View File

@ -12,8 +12,6 @@ namespace DB
bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
ParserKeyword s_alter_table("ALTER TABLE");
ParserKeyword s_add_column("ADD COLUMN");
ParserKeyword s_drop_column("DROP COLUMN");
@ -218,7 +216,6 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
while (!parsing_finished);
query->range = StringRange(begin, pos);
query->cluster = cluster_str;
node = query;

View File

@ -10,8 +10,6 @@ namespace DB
bool ParserCase::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
ParserKeyword s_case{"CASE"};
ParserKeyword s_when{"WHEN"};
ParserKeyword s_then{"THEN"};
@ -22,7 +20,6 @@ bool ParserCase::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!s_case.parse(pos, node, expected))
{
/// Parse as a simple ASTFunction.
pos = begin;
return ParserFunction{}.parse(pos, node, expected);
}
@ -80,10 +77,10 @@ bool ParserCase::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!parse_branches())
return false;
auto function_args = std::make_shared<ASTExpressionList>(StringRange{begin, pos});
auto function_args = std::make_shared<ASTExpressionList>();
function_args->children = std::move(args);
auto function = std::make_shared<ASTFunction>(StringRange{begin, pos});
auto function = std::make_shared<ASTFunction>();
function->name = "caseWithExpression";
function->arguments = function_args;
function->children.push_back(function->arguments);
@ -95,10 +92,10 @@ bool ParserCase::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!parse_branches())
return false;
auto function_args = std::make_shared<ASTExpressionList>(StringRange{begin, pos});
auto function_args = std::make_shared<ASTExpressionList>();
function_args->children = std::move(args);
auto function = std::make_shared<ASTFunction>(StringRange{begin, pos});
auto function = std::make_shared<ASTFunction>();
function->name = "multiIf";
function->arguments = function_args;
function->children.push_back(function->arguments);

View File

@ -12,8 +12,6 @@ namespace DB
bool ParserCheckQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
ParserKeyword s_check_table("CHECK TABLE");
ParserToken s_dot(TokenType::Dot);
@ -32,7 +30,7 @@ bool ParserCheckQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!table_parser.parse(pos, table, expected))
return false;
auto query = std::make_shared<ASTCheckQuery>(StringRange(begin, pos));
auto query = std::make_shared<ASTCheckQuery>();
query->database = typeid_cast<const ASTIdentifier &>(*database).name;
query->table = typeid_cast<const ASTIdentifier &>(*table).name;
node = query;
@ -40,7 +38,7 @@ bool ParserCheckQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
else
{
table = database;
auto query = std::make_shared<ASTCheckQuery>(StringRange(begin, pos));
auto query = std::make_shared<ASTCheckQuery>();
query->table = typeid_cast<const ASTIdentifier &>(*table).name;
node = query;
}

View File

@ -21,8 +21,6 @@ bool ParserNestedTable::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ASTPtr name;
ASTPtr columns;
Pos begin = pos;
/// For now `name == 'Nested'`, probably alternative nested data structures will appear
if (!name_p.parse(pos, name, expected))
return false;
@ -36,7 +34,7 @@ bool ParserNestedTable::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!close.ignore(pos))
return false;
auto func = std::make_shared<ASTFunction>(StringRange(begin, pos));
auto func = std::make_shared<ASTFunction>();
func->name = typeid_cast<ASTIdentifier &>(*name).name;
func->arguments = columns;
func->children.push_back(columns);
@ -65,15 +63,13 @@ bool ParserIdentifierWithOptionalParameters::parseImpl(Pos & pos, ASTPtr & node,
ParserIdentifier non_parametric;
ParserIdentifierWithParameters parametric;
Pos begin = pos;
if (parametric.parse(pos, node, expected))
return true;
ASTPtr ident;
if (non_parametric.parse(pos, ident, expected))
{
auto func = std::make_shared<ASTFunction>(StringRange(begin));
auto func = std::make_shared<ASTFunction>();
func->name = typeid_cast<ASTIdentifier &>(*ident).name;
node = func;
return true;
@ -87,7 +83,7 @@ bool ParserTypeInCastExpression::parseImpl(Pos & pos, ASTPtr & node, Expected &
if (ParserIdentifierWithOptionalParameters::parseImpl(pos, node, expected))
{
const auto & id_with_params = typeid_cast<const ASTFunction &>(*node);
node = std::make_shared<ASTIdentifier>(id_with_params.range, String{ id_with_params.range.first, id_with_params.range.second });
node = std::make_shared<ASTIdentifier>(String{ id_with_params.range.first, id_with_params.range.second });
return true;
}
@ -120,8 +116,6 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserExpression expression_p;
ParserSetQuery settings_p(/* parse_only_internals_ = */ true);
Pos begin = pos;
ASTPtr engine;
ASTPtr partition_by;
ASTPtr order_by;
@ -171,7 +165,7 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
break;
}
auto storage = std::make_shared<ASTStorage>(StringRange(begin, pos));
auto storage = std::make_shared<ASTStorage>();
storage->set(storage->engine, engine);
storage->set(storage->partition_by, partition_by);
storage->set(storage->order_by, order_by);
@ -185,8 +179,6 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
ParserKeyword s_create("CREATE");
ParserKeyword s_temporary("TEMPORARY");
ParserKeyword s_attach("ATTACH");
@ -259,7 +251,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
// Shortcut for ATTACH a previously detached table
if (attach && (!pos.isValid() || pos.get().type == TokenType::Semicolon))
{
auto query = std::make_shared<ASTCreateQuery>(StringRange(begin, pos));
auto query = std::make_shared<ASTCreateQuery>();
node = query;
query->attach = attach;
@ -402,7 +394,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false;
}
auto query = std::make_shared<ASTCreateQuery>(StringRange(begin, pos));
auto query = std::make_shared<ASTCreateQuery>();
node = query;
query->attach = attach;

View File

@ -75,13 +75,11 @@ bool IParserNameTypePair<NameParser>::parseImpl(Pos & pos, ASTPtr & node, Expect
NameParser name_parser;
ParserIdentifierWithOptionalParameters type_parser;
Pos begin = pos;
ASTPtr name, type;
if (name_parser.parse(pos, name, expected)
&& type_parser.parse(pos, type, expected))
{
auto name_type_pair = std::make_shared<ASTNameTypePair>(StringRange(begin, pos));
auto name_type_pair = std::make_shared<ASTNameTypePair>();
name_type_pair->name = typeid_cast<const ASTIdentifier &>(*name).name;
name_type_pair->type = type;
name_type_pair->children.push_back(type);
@ -122,8 +120,6 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
ParserKeyword s_alias{"ALIAS"};
ParserTernaryOperatorExpression expr_parser;
const auto begin = pos;
/// mandatory column name
ASTPtr name;
if (!name_parser.parse(pos, name, expected))
@ -160,7 +156,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
else if (!type)
return false; /// reject sole column name without type
const auto column_declaration = std::make_shared<ASTColumnDeclaration>(StringRange{begin, pos});
const auto column_declaration = std::make_shared<ASTColumnDeclaration>();
node = column_declaration;
column_declaration->name = typeid_cast<ASTIdentifier &>(*name).name;
if (type)

View File

@ -14,8 +14,6 @@ namespace DB
bool ParserDescribeTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
ParserKeyword s_describe("DESCRIBE");
ParserKeyword s_desc("DESC");
ParserKeyword s_table("TABLE");
@ -36,7 +34,6 @@ bool ParserDescribeTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & ex
if (!ParserTableExpression().parse(pos, table_expression, expected))
return false;
query->range = StringRange(begin, pos);
query->table_expression = table_expression;
node = query;

View File

@ -13,8 +13,6 @@ namespace DB
bool ParserDropQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
ParserKeyword s_drop("DROP");
ParserKeyword s_detach("DETACH");
ParserKeyword s_temporary("TEMPORARY");
@ -81,7 +79,7 @@ bool ParserDropQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
}
auto query = std::make_shared<ASTDropQuery>(StringRange(begin, pos));
auto query = std::make_shared<ASTDropQuery>();
node = query;
query->detach = detach;

View File

@ -24,8 +24,6 @@ namespace ErrorCodes
bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
ParserKeyword s_insert_into("INSERT INTO");
ParserKeyword s_table("TABLE");
ParserKeyword s_function("FUNCTION");
@ -130,7 +128,7 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false;
}
auto query = std::make_shared<ASTInsertQuery>(StringRange(begin, pos));
auto query = std::make_shared<ASTInsertQuery>();
node = query;
if (table_function)

View File

@ -11,7 +11,6 @@ namespace DB
bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
auto query = std::make_shared<ASTKillQueryQuery>();
if (!ParserKeyword{"KILL QUERY"}.ignore(pos, expected))
@ -31,8 +30,6 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
else if (ParserKeyword{"TEST"}.ignore(pos))
query->test = true;
query->range = StringRange(begin, pos);
node = std::move(query);
return true;

View File

@ -15,8 +15,6 @@ namespace DB
bool ParserOptimizeQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
ParserKeyword s_optimize_table("OPTIMIZE TABLE");
ParserKeyword s_partition("PARTITION");
ParserKeyword s_final("FINAL");
@ -56,7 +54,7 @@ bool ParserOptimizeQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte
if (s_deduplicate.ignore(pos, expected))
deduplicate = true;
auto query = std::make_shared<ASTOptimizeQuery>(StringRange(begin, pos));
auto query = std::make_shared<ASTOptimizeQuery>();
node = query;
if (database)

View File

@ -73,7 +73,6 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
partition->fields_count = fields_count;
}
partition->range = StringRange(begin, pos);
node = partition;
return true;
}

View File

@ -40,8 +40,6 @@ static bool parseDatabaseAndTable(
bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
ParserKeyword s_rename_table("RENAME TABLE");
ParserKeyword s_to("TO");
ParserToken s_comma(TokenType::Comma);
@ -71,7 +69,7 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false;
}
auto query = std::make_shared<ASTRenameQuery>(StringRange(begin, pos));
auto query = std::make_shared<ASTRenameQuery>();
query->cluster = cluster_str;
node = query;

View File

@ -83,8 +83,6 @@ static bool parseDecimal(const char * pos, const char * end, ASTSampleRatio::Rat
*/
bool ParserSampleRatio::parseImpl(Pos & pos, ASTPtr & node, Expected &)
{
auto begin = pos;
ASTSampleRatio::Rational numerator;
ASTSampleRatio::Rational denominator;
ASTSampleRatio::Rational res;
@ -111,7 +109,7 @@ bool ParserSampleRatio::parseImpl(Pos & pos, ASTPtr & node, Expected &)
res = numerator;
}
node = std::make_shared<ASTSampleRatio>(StringRange(begin, pos), res);
node = std::make_shared<ASTSampleRatio>(res);
return true;
}

View File

@ -22,8 +22,6 @@ namespace ErrorCodes
bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
auto select_query = std::make_shared<ASTSelectQuery>();
node = select_query;
@ -173,8 +171,6 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false;
}
select_query->range = StringRange(begin, pos);
if (select_query->with_expression_list)
select_query->children.push_back(select_query->with_expression_list);
select_query->children.push_back(select_query->select_expression_list);

View File

@ -15,7 +15,7 @@ bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
if (!parser.parse(pos, list_node, expected))
return false;
auto res = std::make_shared<ASTSelectWithUnionQuery>(list_node->range);
auto res = std::make_shared<ASTSelectWithUnionQuery>();
res->list_of_selects = std::move(list_node);
res->children.push_back(res->list_of_selects);

View File

@ -40,8 +40,6 @@ static bool parseNameValuePair(ASTSetQuery::Change & change, IParser::Pos & pos,
bool ParserSetQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
ParserToken s_comma(TokenType::Comma);
if (!parse_only_internals)
@ -65,7 +63,7 @@ bool ParserSetQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false;
}
auto query = std::make_shared<ASTSetQuery>(StringRange(begin, pos));
auto query = std::make_shared<ASTSetQuery>();
node = query;
query->is_standalone = !parse_only_internals;

View File

@ -19,14 +19,11 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
auto query = std::make_shared<ASTShowProcesslistQuery>();
if (!ParserKeyword("SHOW PROCESSLIST").ignore(pos, expected))
return false;
query->range = StringRange(begin, pos);
node = query;
return true;

View File

@ -15,8 +15,6 @@ namespace DB
bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
ParserKeyword s_show("SHOW");
ParserKeyword s_temporary("TEMPORARY");
ParserKeyword s_tables("TABLES");
@ -67,8 +65,6 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
return false;
}
query->range = StringRange(begin, pos);
if (database)
query->from = typeid_cast<ASTIdentifier &>(*database).name;
if (like)

View File

@ -18,8 +18,6 @@ namespace DB
bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
auto begin = pos;
if (!ParserKeyword{"SYSTEM"}.ignore(pos))
return false;
@ -51,7 +49,6 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
throw Exception("SYNC REPLICA is not supported yet", ErrorCodes::NOT_IMPLEMENTED);
}
res->range = {begin, pos};
node = std::move(res);
return true;
}

View File

@ -13,8 +13,6 @@ namespace DB
bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
ParserKeyword s_exists("EXISTS");
ParserKeyword s_describe("DESCRIBE");
ParserKeyword s_desc("DESC");
@ -56,8 +54,6 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
return false;
}
query->range = StringRange(begin, pos);
if (database)
query->database = typeid_cast<ASTIdentifier &>(*database).name;
if (table)

View File

@ -9,10 +9,9 @@
namespace DB
{
bool ParserUseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Pos begin = pos;
ParserKeyword s_use("USE");
ParserIdentifier name_p;
@ -24,10 +23,11 @@ bool ParserUseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!name_p.parse(pos, database, expected))
return false;
auto query = std::make_shared<ASTUseQuery>(StringRange(begin, pos));
auto query = std::make_shared<ASTUseQuery>();
query->database = typeid_cast<ASTIdentifier &>(*database).name;
node = query;
return true;
}
}

View File

@ -11,8 +11,8 @@ namespace DB
struct StringRange
{
const char * first;
const char * second;
const char * first = nullptr;
const char * second = nullptr;
StringRange() {}
StringRange(const char * begin, const char * end) : first(begin), second(end) {}
@ -41,7 +41,7 @@ using StringPtr = std::shared_ptr<String>;
inline String toString(const StringRange & range)
{
return String(range.first, range.second);
return range.first ? String(range.first, range.second) : String();
}
}

View File

@ -32,8 +32,6 @@ class ASTDescribeQuery : public ASTQueryWithOutput
public:
ASTPtr table_expression;
ASTDescribeQuery() = default;
explicit ASTDescribeQuery(StringRange range_) : ASTQueryWithOutput(range_) {}
String getID() const override { return "DescribeQuery"; };
ASTPtr clone() const override

View File

@ -10,12 +10,10 @@ namespace DB
bool parseIdentifierOrStringLiteral(IParser::Pos & pos, Expected & expected, String & result)
{
IParser::Pos begin = pos;
ASTPtr res;
if (!ParserIdentifier().parse(pos, res, expected))
{
pos = begin;
if (!ParserStringLiteral().parse(pos, res, expected))
return false;

View File

@ -13,7 +13,6 @@ namespace DB
{
std::ostringstream out;
formatAST(query, out, false, true);
return out.str();
}
}

View File

@ -373,9 +373,9 @@ std::shared_ptr<ASTStorage> createASTStorageDistributed(
const String & cluster_name, const String & database, const String & table, const ASTPtr & sharding_key_ast = nullptr)
{
auto args = std::make_shared<ASTExpressionList>();
args->children.emplace_back(std::make_shared<ASTLiteral>(StringRange(nullptr, nullptr), cluster_name));
args->children.emplace_back(std::make_shared<ASTIdentifier>(StringRange(nullptr, nullptr), database));
args->children.emplace_back(std::make_shared<ASTIdentifier>(StringRange(nullptr, nullptr), table));
args->children.emplace_back(std::make_shared<ASTLiteral>(cluster_name));
args->children.emplace_back(std::make_shared<ASTIdentifier>(database));
args->children.emplace_back(std::make_shared<ASTIdentifier>(table));
if (sharding_key_ast)
args->children.emplace_back(sharding_key_ast);
@ -487,7 +487,7 @@ static ASTPtr extractPartitionKey(const ASTPtr & storage_ast)
return storage.partition_by->clone();
static const char * all = "all";
return std::make_shared<ASTLiteral>(StringRange(all, all + strlen(all)), Field(all, strlen(all)));
return std::make_shared<ASTLiteral>(Field(all, strlen(all)));
}
else
{

View File

@ -245,8 +245,8 @@ void AlterCommands::validate(IStorage * table, const Context & context)
const auto column_type_raw_ptr = command.data_type.get();
default_expr_list->children.emplace_back(setAlias(
makeASTFunction("CAST", std::make_shared<ASTIdentifier>(StringRange(), tmp_column_name),
std::make_shared<ASTLiteral>(StringRange(), Field(column_type_raw_ptr->getName()))),
makeASTFunction("CAST", std::make_shared<ASTIdentifier>(tmp_column_name),
std::make_shared<ASTLiteral>(Field(column_type_raw_ptr->getName()))),
final_column_name));
default_expr_list->children.emplace_back(setAlias(command.default_expression->clone(), tmp_column_name));
@ -299,8 +299,8 @@ void AlterCommands::validate(IStorage * table, const Context & context)
const auto & column_type_ptr = column_it->type;
default_expr_list->children.emplace_back(setAlias(
makeASTFunction("CAST", std::make_shared<ASTIdentifier>(StringRange(), tmp_column_name),
std::make_shared<ASTLiteral>(StringRange(), Field(column_type_ptr->getName()))),
makeASTFunction("CAST", std::make_shared<ASTIdentifier>(tmp_column_name),
std::make_shared<ASTLiteral>(Field(column_type_ptr->getName()))),
column_name));
default_expr_list->children.emplace_back(setAlias(col_def.second.expression->clone(), tmp_column_name));
@ -345,7 +345,7 @@ void AlterCommands::validate(IStorage * table, const Context & context)
}
command_ptr->default_expression = makeASTFunction("CAST", command_ptr->default_expression->clone(),
std::make_shared<ASTLiteral>(StringRange(), Field(explicit_type->getName())));
std::make_shared<ASTLiteral>(Field(explicit_type->getName())));
}
}
else

View File

@ -434,7 +434,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
ASTPtr args = std::make_shared<ASTExpressionList>();
args->children.push_back(data.sampling_expression);
args->children.push_back(std::make_shared<ASTLiteral>(StringRange(), lower));
args->children.push_back(std::make_shared<ASTLiteral>(lower));
lower_function = std::make_shared<ASTFunction>();
lower_function->name = "greaterOrEquals";
@ -451,7 +451,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
ASTPtr args = std::make_shared<ASTExpressionList>();
args->children.push_back(data.sampling_expression);
args->children.push_back(std::make_shared<ASTLiteral>(StringRange(), upper));
args->children.push_back(std::make_shared<ASTLiteral>(upper));
upper_function = std::make_shared<ASTFunction>();
upper_function->name = "less";
@ -846,8 +846,8 @@ void MergeTreeDataSelectExecutor::createPositiveSignCondition(
{
auto function = std::make_shared<ASTFunction>();
auto arguments = std::make_shared<ASTExpressionList>();
auto sign = std::make_shared<ASTIdentifier>();
auto one = std::make_shared<ASTLiteral>();
auto sign = std::make_shared<ASTIdentifier>(data.merging_params.sign_column);
auto one = std::make_shared<ASTLiteral>(Field(static_cast<Int64>(1)));
function->name = "equals";
function->arguments = arguments;
@ -856,11 +856,6 @@ void MergeTreeDataSelectExecutor::createPositiveSignCondition(
arguments->children.push_back(sign);
arguments->children.push_back(one);
sign->name = data.merging_params.sign_column;
sign->kind = ASTIdentifier::Column;
one->value = Field(static_cast<Int64>(1));
out_expression = ExpressionAnalyzer(function, context, {}, data.getColumnsList()).getActions(false);
out_column = function->getColumnName();
}

View File

@ -13,6 +13,7 @@
#include <Common/typeid_cast.h>
#include <Interpreters/convertFieldToType.h>
#include <Interpreters/Set.h>
#include <Parsers/queryToString.h>
namespace DB
@ -593,7 +594,7 @@ static void castValueToType(const DataTypePtr & desired_type, Field & src_value,
{
throw Exception("Primary key expression contains comparison between inconvertible types: " +
desired_type->getName() + " and " + src_type->getName() +
" inside " + DB::toString(node->range),
" inside " + queryToString(node),
ErrorCodes::BAD_TYPE_OF_FIELD);
}
}

View File

@ -558,7 +558,7 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl
insert->columns = list_of_columns;
list_of_columns->children.reserve(columns_intersection.size());
for (const String & column : columns_intersection)
list_of_columns->children.push_back(std::make_shared<ASTIdentifier>(StringRange(), column, ASTIdentifier::Column));
list_of_columns->children.push_back(std::make_shared<ASTIdentifier>(column, ASTIdentifier::Column));
InterpreterInsertQuery interpreter{insert, context, allow_materialized};

View File

@ -243,13 +243,12 @@ void StorageCatBoostPool::createSampleBlockAndColumns()
if (!desc.alias.empty())
{
auto alias = std::make_shared<ASTIdentifier>();
alias->name = desc.column_name;
auto alias = std::make_shared<ASTIdentifier>(desc.column_name);
column_defaults[desc.alias] = {ColumnDefaultType::Alias, alias};
alias_columns.emplace_back(desc.alias, type);
}
sample_block.insert(ColumnWithTypeAndName(type->createColumn(), type, desc.column_name));
sample_block.insert(ColumnWithTypeAndName(type, desc.column_name));
}
columns.insert(columns.end(), num_columns.begin(), num_columns.end());
columns.insert(columns.end(), cat_columns.begin(), cat_columns.end());

View File

@ -267,13 +267,10 @@ BlockInputStreams StorageDistributed::describe(const Context & context, const Se
std::string name = remote_database + '.' + remote_table;
auto id = std::make_shared<ASTIdentifier>();
id->name = name;
auto id = std::make_shared<ASTIdentifier>(name);
auto desc_database = std::make_shared<ASTIdentifier>();
auto desc_table = std::make_shared<ASTIdentifier>();
desc_database->name = remote_database;
desc_table->name = remote_table;
auto desc_database = std::make_shared<ASTIdentifier>(remote_database);
auto desc_table = std::make_shared<ASTIdentifier>(remote_table);
id->children.push_back(desc_database);
id->children.push_back(desc_table);

View File

@ -201,9 +201,10 @@ void StorageMergeTree::alter(
IDatabase::ASTModifier storage_modifier;
if (primary_key_is_modified)
{
storage_modifier = [&new_primary_key_ast] (IAST & ast)
{
auto tuple = std::make_shared<ASTFunction>(new_primary_key_ast->range);
auto tuple = std::make_shared<ASTFunction>();
tuple->name = "tuple";
tuple->arguments = new_primary_key_ast;
tuple->children.push_back(tuple->arguments);
@ -213,6 +214,7 @@ void StorageMergeTree::alter(
auto & storage_ast = typeid_cast<ASTStorage &>(ast);
typeid_cast<ASTExpressionList &>(*storage_ast.engine->arguments).children.at(1) = tuple;
};
}
context.getDatabase(database_name)->alterTable(
context, table_name,

View File

@ -81,7 +81,7 @@ void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & va
}
ASTExpressionList & with = typeid_cast<ASTExpressionList &>(*select.with_expression_list);
auto literal = std::make_shared<ASTLiteral>(StringRange(), value);
auto literal = std::make_shared<ASTLiteral>(value);
literal->alias = column_name;
literal->prefer_alias_to_column_name = true;
with.children.push_back(literal);

View File

@ -70,7 +70,7 @@ String transformQueryForExternalDatabase(
auto select_expr_list = std::make_shared<ASTExpressionList>();
for (const auto & name : used_columns)
select_expr_list->children.push_back(std::make_shared<ASTIdentifier>(StringRange(), name));
select_expr_list->children.push_back(std::make_shared<ASTIdentifier>(name));
select->select_expression_list = std::move(select_expr_list);