ClickHouse/src/Parsers/ASTIdentifier.cpp

241 lines
6.5 KiB
C++
Raw Normal View History

#include <Parsers/ASTIdentifier.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/IdentifierSemantic.h>
2020-03-13 10:30:55 +00:00
#include <Interpreters/StorageID.h>
#include <Parsers/queryToString.h>
2020-11-09 16:05:40 +00:00
#include <IO/Operators.h>
namespace DB
{
2019-08-08 20:02:30 +00:00
namespace ErrorCodes
{
extern const int UNEXPECTED_AST_STRUCTURE;
2020-04-07 14:05:51 +00:00
extern const int SYNTAX_ERROR;
2019-08-08 20:02:30 +00:00
}
2020-11-02 10:03:52 +00:00
ASTIdentifier::ASTIdentifier(const String & short_name, ASTPtr && name_param)
: full_name(short_name), name_parts{short_name}, semantic(std::make_shared<IdentifierSemanticImpl>())
{
2020-11-02 10:03:52 +00:00
if (name_param == nullptr)
assert(!full_name.empty());
else
children.push_back(std::move(name_param));
}
2019-08-08 20:02:30 +00:00
2020-11-02 10:03:52 +00:00
ASTIdentifier::ASTIdentifier(std::vector<String> && name_parts_, bool special, std::vector<ASTPtr> && name_params)
: name_parts(name_parts_), semantic(std::make_shared<IdentifierSemanticImpl>())
{
assert(!name_parts.empty());
semantic->special = special;
semantic->legacy_compound = true;
2020-11-02 10:03:52 +00:00
if (!name_params.empty())
{
size_t params = 0;
for (const auto & part [[maybe_unused]] : name_parts)
{
if (part.empty())
++params;
}
assert(params == name_params.size());
children = std::move(name_params);
}
else
{
for (const auto & part [[maybe_unused]] : name_parts)
assert(!part.empty());
2020-11-02 10:03:52 +00:00
if (!special && name_parts.size() >= 2)
semantic->table = name_parts.end()[-2];
2020-11-02 10:03:52 +00:00
resetFullName();
}
}
ASTPtr ASTIdentifier::getParam() const
{
assert(full_name.empty() && children.size() == 1);
return children.front()->clone();
}
ASTPtr ASTIdentifier::clone() const
{
auto ret = std::make_shared<ASTIdentifier>(*this);
ret->semantic = std::make_shared<IdentifierSemanticImpl>(*ret->semantic);
return ret;
}
bool ASTIdentifier::supposedToBeCompound() const
{
return semantic->legacy_compound;
}
2019-02-12 15:08:21 +00:00
void ASTIdentifier::setShortName(const String & new_name)
{
assert(!new_name.empty());
full_name = new_name;
name_parts = {new_name};
2019-02-12 15:08:21 +00:00
bool special = semantic->special;
//how about keep the semantic info here, such as table
auto table = semantic->table;
*semantic = IdentifierSemanticImpl();
semantic->special = special;
semantic->table = table;
2019-02-12 15:08:21 +00:00
}
const String & ASTIdentifier::name() const
{
2020-11-02 10:03:52 +00:00
if (children.empty())
{
assert(!name_parts.empty());
assert(!full_name.empty());
}
return full_name;
}
2020-11-02 10:03:52 +00:00
void ASTIdentifier::formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
2018-08-26 02:19:18 +00:00
auto format_element = [&](const String & elem_name)
{
settings.ostr << (settings.hilite ? hilite_identifier : "");
settings.writeIdentifier(elem_name);
settings.ostr << (settings.hilite ? hilite_none : "");
};
/// It could be compound but short
if (!isShort())
{
2020-11-02 10:03:52 +00:00
for (size_t i = 0, j = 0, size = name_parts.size(); i < size; ++i)
{
if (i != 0)
settings.ostr << '.';
/// Some AST revriting code, like IdentifierSemantic::setColumnLongName,
/// does not respect children of identifier.
/// Here we also ingore children if they are empty.
if (name_parts[i].empty() && j < children.size())
2020-11-02 10:03:52 +00:00
children[j++]->formatImpl(settings, state, frame);
else
format_element(name_parts[i]);
}
}
else
{
2020-11-02 10:03:52 +00:00
const auto & name = shortName();
if (name.empty() && !children.empty())
2020-11-02 10:03:52 +00:00
children.front()->formatImpl(settings, state, frame);
else
format_element(name);
}
}
void ASTIdentifier::appendColumnNameImpl(WriteBuffer & ostr) const
{
writeString(name(), ostr);
}
void ASTIdentifier::restoreTable()
{
if (!compound())
{
name_parts.insert(name_parts.begin(), semantic->table);
resetFullName();
}
}
void ASTIdentifier::resetTable(const String & database_name, const String & table_name)
{
auto ast = createTableIdentifier(database_name, table_name);
auto & ident = ast->as<ASTIdentifier &>();
full_name.swap(ident.full_name);
name_parts.swap(ident.name_parts);
uuid = ident.uuid;
}
2020-08-20 02:01:40 +00:00
void ASTIdentifier::updateTreeHashImpl(SipHash & hash_state) const
{
hash_state.update(uuid);
IAST::updateTreeHashImpl(hash_state);
}
void ASTIdentifier::resetFullName()
{
full_name = name_parts[0];
for (size_t i = 1; i < name_parts.size(); ++i)
full_name += '.' + name_parts[i];
}
2019-01-15 12:28:17 +00:00
ASTPtr createTableIdentifier(const String & database_name, const String & table_name)
{
2020-03-12 18:04:29 +00:00
assert(database_name != "_temporary_and_external_tables");
return createTableIdentifier(StorageID(database_name, table_name));
}
2019-01-15 12:28:17 +00:00
2020-03-12 18:04:29 +00:00
ASTPtr createTableIdentifier(const StorageID & table_id)
{
std::shared_ptr<ASTIdentifier> res;
if (table_id.database_name.empty())
res = std::make_shared<ASTIdentifier>(std::vector<String>{table_id.table_name}, true);
2020-03-12 18:04:29 +00:00
else
res = std::make_shared<ASTIdentifier>(std::vector<String>{table_id.database_name, table_id.table_name}, true);
2020-03-12 18:04:29 +00:00
res->uuid = table_id.uuid;
return res;
2019-01-15 12:28:17 +00:00
}
2019-08-08 20:02:30 +00:00
String getIdentifierName(const IAST * ast)
{
2019-08-08 20:02:30 +00:00
String res;
if (tryGetIdentifierNameInto(ast, res))
return res;
2019-08-08 20:26:42 +00:00
throw Exception(ast ? queryToString(*ast) + " is not an identifier" : "AST node is nullptr", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
2019-08-08 20:02:30 +00:00
}
std::optional<String> tryGetIdentifierName(const IAST * ast)
{
String res;
if (tryGetIdentifierNameInto(ast, res))
return res;
return {};
}
2019-08-08 20:26:42 +00:00
bool tryGetIdentifierNameInto(const IAST * ast, String & name)
{
if (ast)
2019-08-08 20:02:30 +00:00
{
2019-03-11 13:22:51 +00:00
if (const auto * node = ast->as<ASTIdentifier>())
{
name = node->name();
return true;
}
2019-08-08 20:02:30 +00:00
}
return false;
}
void setIdentifierSpecial(ASTPtr & ast)
{
if (ast)
2019-03-11 13:22:51 +00:00
if (auto * id = ast->as<ASTIdentifier>())
id->semantic->special = true;
2019-01-16 17:26:14 +00:00
}
2020-04-07 14:05:51 +00:00
StorageID getTableIdentifier(const ASTPtr & ast)
{
if (!ast)
throw Exception("AST node is nullptr", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
const auto & identifier = dynamic_cast<const ASTIdentifier &>(*ast);
if (identifier.name_parts.size() > 2)
throw Exception("Logical error: more than two components in table expression", ErrorCodes::SYNTAX_ERROR);
if (identifier.name_parts.size() == 2)
return { identifier.name_parts[0], identifier.name_parts[1], identifier.uuid };
return { "", identifier.name_parts[0], identifier.uuid };
2020-04-07 14:05:51 +00:00
}
}