mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #16401 from abyss7/ast-table-identifier-2
ASTTableIdentifier part #2: Introduce ASTTableIdentifier
This commit is contained in:
commit
a52bba91b7
@ -463,9 +463,7 @@ void QueryFuzzer::fuzz(ASTPtr & ast)
|
||||
}
|
||||
else if (auto * table_expr = typeid_cast<ASTTableExpression *>(ast.get()))
|
||||
{
|
||||
fuzz(table_expr->database_and_table_name);
|
||||
fuzz(table_expr->subquery);
|
||||
fuzz(table_expr->table_function);
|
||||
fuzz(table_expr->children);
|
||||
}
|
||||
else if (auto * expr_list = typeid_cast<ASTExpressionList *>(ast.get()))
|
||||
{
|
||||
|
@ -603,6 +603,7 @@ bool ActionsMatcher::needChildVisit(const ASTPtr & node, const ASTPtr & child)
|
||||
{
|
||||
/// Visit children themself
|
||||
if (node->as<ASTIdentifier>() ||
|
||||
node->as<ASTTableIdentifier>() ||
|
||||
node->as<ASTFunction>() ||
|
||||
node->as<ASTLiteral>() ||
|
||||
node->as<ASTExpressionList>())
|
||||
@ -620,6 +621,8 @@ void ActionsMatcher::visit(const ASTPtr & ast, Data & data)
|
||||
{
|
||||
if (const auto * identifier = ast->as<ASTIdentifier>())
|
||||
visit(*identifier, ast, data);
|
||||
else if (const auto * table = ast->as<ASTTableIdentifier>())
|
||||
visit(*table, ast, data);
|
||||
else if (const auto * node = ast->as<ASTFunction>())
|
||||
visit(*node, ast, data);
|
||||
else if (const auto * literal = ast->as<ASTLiteral>())
|
||||
@ -735,9 +738,9 @@ void ActionsMatcher::visit(ASTExpressionList & expression_list, const ASTPtr &,
|
||||
}
|
||||
}
|
||||
|
||||
void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr & ast, Data & data)
|
||||
void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr &, Data & data)
|
||||
{
|
||||
auto column_name = ast->getColumnName();
|
||||
auto column_name = identifier.getColumnName();
|
||||
if (data.hasColumn(column_name))
|
||||
return;
|
||||
|
||||
@ -880,7 +883,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
||||
auto & child = node.arguments->children[arg];
|
||||
|
||||
const auto * function = child->as<ASTFunction>();
|
||||
const auto * identifier = child->as<ASTIdentifier>();
|
||||
const auto * identifier = child->as<ASTTableIdentifier>();
|
||||
if (function && function->name == "lambda")
|
||||
{
|
||||
/// If the argument is a lambda expression, just remember its approximate type.
|
||||
@ -949,7 +952,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
||||
}
|
||||
else if (identifier && (functionIsJoinGet(node.name) || functionIsDictGet(node.name)) && arg == 0)
|
||||
{
|
||||
auto table_id = IdentifierSemantic::extractDatabaseAndTable(*identifier);
|
||||
auto table_id = identifier->getTableId();
|
||||
table_id = data.getContext()->resolveStorageID(table_id, Context::ResolveOrdinary);
|
||||
auto column_string = ColumnString::create();
|
||||
column_string->insert(table_id.getDatabaseName() + "." + table_id.getTableName());
|
||||
@ -1120,7 +1123,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su
|
||||
const ASTPtr & right_in_operand = args.children.at(1);
|
||||
|
||||
/// If the subquery or table name for SELECT.
|
||||
const auto * identifier = right_in_operand->as<ASTIdentifier>();
|
||||
const auto * identifier = right_in_operand->as<ASTTableIdentifier>();
|
||||
if (right_in_operand->as<ASTSubquery>() || identifier)
|
||||
{
|
||||
if (no_subqueries)
|
||||
|
@ -97,16 +97,15 @@ private:
|
||||
void visit(ASTTableExpression & table_expression, ASTPtr &) const
|
||||
{
|
||||
if (table_expression.database_and_table_name)
|
||||
tryVisit<ASTIdentifier>(table_expression.database_and_table_name);
|
||||
tryVisit<ASTTableIdentifier>(table_expression.database_and_table_name);
|
||||
else if (table_expression.subquery)
|
||||
tryVisit<ASTSubquery>(table_expression.subquery);
|
||||
}
|
||||
|
||||
/// @note It expects that only table (not column) identifiers are visited.
|
||||
void visit(const ASTIdentifier & identifier, ASTPtr & ast) const
|
||||
void visit(const ASTTableIdentifier & identifier, ASTPtr & ast) const
|
||||
{
|
||||
if (!identifier.compound())
|
||||
ast = createTableIdentifier(database_name, identifier.name());
|
||||
ast = std::make_shared<ASTTableIdentifier>(database_name, identifier.name());
|
||||
}
|
||||
|
||||
void visit(ASTSubquery & subquery, ASTPtr &) const
|
||||
@ -134,9 +133,14 @@ private:
|
||||
{
|
||||
if (is_operator_in && i == 1)
|
||||
{
|
||||
/// XXX: for some unknown reason this place assumes that argument can't be an alias,
|
||||
/// like in the similar code in `MarkTableIdentifierVisitor`.
|
||||
if (auto * identifier = child->children[i]->as<ASTIdentifier>())
|
||||
child->children[i] = identifier->createTable();
|
||||
|
||||
/// Second argument of the "in" function (or similar) may be a table name or a subselect.
|
||||
/// Rewrite the table name or descend into subselect.
|
||||
if (!tryVisit<ASTIdentifier>(child->children[i]))
|
||||
if (!tryVisit<ASTTableIdentifier>(child->children[i]))
|
||||
visit(child->children[i]);
|
||||
}
|
||||
else
|
||||
|
@ -59,7 +59,7 @@ void ApplyWithSubqueryVisitor::visit(ASTTableExpression & table, const Data & da
|
||||
{
|
||||
if (table.database_and_table_name)
|
||||
{
|
||||
auto table_id = IdentifierSemantic::extractDatabaseAndTable(table.database_and_table_name->as<ASTIdentifier &>());
|
||||
auto table_id = table.database_and_table_name->as<ASTTableIdentifier>()->getTableId();
|
||||
if (table_id.database_name.empty())
|
||||
{
|
||||
auto subquery_it = data.subqueries.find(table_id.table_name);
|
||||
@ -80,20 +80,23 @@ void ApplyWithSubqueryVisitor::visit(ASTTableExpression & table, const Data & da
|
||||
|
||||
void ApplyWithSubqueryVisitor::visit(ASTFunction & func, const Data & data)
|
||||
{
|
||||
/// Special CTE case, where the right argument of IN is alias (ASTIdentifier) from WITH clause.
|
||||
|
||||
if (checkFunctionIsInOrGlobalInOperator(func))
|
||||
{
|
||||
auto & ast = func.arguments->children.at(1);
|
||||
if (const auto * ident = ast->as<ASTIdentifier>())
|
||||
if (const auto * identifier = ast->as<ASTIdentifier>())
|
||||
{
|
||||
auto table_id = IdentifierSemantic::extractDatabaseAndTable(*ident);
|
||||
if (table_id.database_name.empty())
|
||||
if (identifier->isShort())
|
||||
{
|
||||
auto subquery_it = data.subqueries.find(table_id.table_name);
|
||||
/// Clang-tidy is wrong on this line, because `func.arguments->children.at(1)` gets replaced before last use of `name`.
|
||||
auto name = identifier->shortName(); // NOLINT
|
||||
auto subquery_it = data.subqueries.find(name);
|
||||
if (subquery_it != data.subqueries.end())
|
||||
{
|
||||
auto old_alias = func.arguments->children[1]->tryGetAlias();
|
||||
func.arguments->children[1] = subquery_it->second->clone();
|
||||
func.arguments->children[1]->as<ASTSubquery &>().cte_name = table_id.table_name;
|
||||
func.arguments->children[1]->as<ASTSubquery &>().cte_name = name;
|
||||
if (!old_alias.empty())
|
||||
func.arguments->children[1]->setAlias(old_alias);
|
||||
}
|
||||
|
@ -18,11 +18,11 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database)
|
||||
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTTableIdentifier & identifier, const String & current_database)
|
||||
{
|
||||
alias = identifier.tryGetAlias();
|
||||
|
||||
auto table_id = IdentifierSemantic::extractDatabaseAndTable(identifier);
|
||||
auto table_id = identifier.getTableId();
|
||||
std::tie(database, table, uuid) = std::tie(table_id.database_name, table_id.table_name, table_id.uuid);
|
||||
if (database.empty())
|
||||
database = current_database;
|
||||
@ -30,9 +30,9 @@ DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTIdentifier & ident
|
||||
|
||||
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTPtr & node, const String & current_database)
|
||||
{
|
||||
const auto * identifier = node->as<ASTIdentifier>();
|
||||
const auto * identifier = node->as<ASTTableIdentifier>();
|
||||
if (!identifier)
|
||||
throw Exception("Logical error: identifier expected", ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception("Logical error: table identifier expected", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
*this = DatabaseAndTableWithAlias(*identifier, current_database);
|
||||
}
|
||||
@ -100,7 +100,7 @@ std::optional<DatabaseAndTableWithAlias> getDatabaseAndTable(const ASTSelectQuer
|
||||
return {};
|
||||
|
||||
ASTPtr database_and_table_name = table_expression->database_and_table_name;
|
||||
if (!database_and_table_name || !database_and_table_name->as<ASTIdentifier>())
|
||||
if (!database_and_table_name || !database_and_table_name->as<ASTTableIdentifier>())
|
||||
return {};
|
||||
|
||||
return DatabaseAndTableWithAlias(database_and_table_name);
|
||||
|
@ -14,12 +14,13 @@ namespace DB
|
||||
{
|
||||
|
||||
class ASTSelectQuery;
|
||||
class ASTIdentifier;
|
||||
class ASTTableIdentifier;
|
||||
struct ASTTableExpression;
|
||||
|
||||
/// Extracts database name (and/or alias) from table expression or identifier
|
||||
struct DatabaseAndTableWithAlias
|
||||
{
|
||||
// TODO(ilezhankin): replace with ASTTableIdentifier
|
||||
String database;
|
||||
String table;
|
||||
String alias;
|
||||
@ -27,9 +28,10 @@ struct DatabaseAndTableWithAlias
|
||||
|
||||
DatabaseAndTableWithAlias() = default;
|
||||
explicit DatabaseAndTableWithAlias(const ASTPtr & identifier_node, const String & current_database = "");
|
||||
explicit DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database = "");
|
||||
explicit DatabaseAndTableWithAlias(const ASTTableIdentifier & identifier, const String & current_database = "");
|
||||
explicit DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, const String & current_database = "");
|
||||
|
||||
|
||||
/// "alias." or "table." if alias is empty
|
||||
String getQualifiedNamePrefix(bool with_dot = true) const;
|
||||
|
||||
|
@ -347,7 +347,7 @@ void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_
|
||||
|
||||
SetPtr ExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & subquery_or_table_name)
|
||||
{
|
||||
const auto * table = subquery_or_table_name->as<ASTIdentifier>();
|
||||
const auto * table = subquery_or_table_name->as<ASTTableIdentifier>();
|
||||
if (!table)
|
||||
return nullptr;
|
||||
auto table_id = getContext()->resolveStorageID(subquery_or_table_name);
|
||||
@ -388,7 +388,7 @@ void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node)
|
||||
if (storage()->mayBenefitFromIndexForIn(left_in_operand, getContext(), metadata_snapshot))
|
||||
{
|
||||
const ASTPtr & arg = args.children.at(1);
|
||||
if (arg->as<ASTSubquery>() || arg->as<ASTIdentifier>())
|
||||
if (arg->as<ASTSubquery>() || arg->as<ASTTableIdentifier>())
|
||||
{
|
||||
if (settings.use_index_for_in_with_subqueries)
|
||||
tryMakeSetForIndexFromSubquery(arg, query_options);
|
||||
|
@ -62,7 +62,7 @@ public:
|
||||
return;
|
||||
|
||||
bool is_table = false;
|
||||
ASTPtr subquery_or_table_name = ast; /// ASTIdentifier | ASTSubquery | ASTTableExpression
|
||||
ASTPtr subquery_or_table_name = ast; /// ASTTableIdentifier | ASTSubquery | ASTTableExpression
|
||||
|
||||
if (const auto * ast_table_expr = ast->as<ASTTableExpression>())
|
||||
{
|
||||
@ -74,7 +74,7 @@ public:
|
||||
is_table = true;
|
||||
}
|
||||
}
|
||||
else if (ast->as<ASTIdentifier>())
|
||||
else if (ast->as<ASTTableIdentifier>())
|
||||
is_table = true;
|
||||
|
||||
if (!subquery_or_table_name)
|
||||
@ -122,11 +122,11 @@ public:
|
||||
* instead of doing a subquery, you just need to read it.
|
||||
*/
|
||||
|
||||
auto database_and_table_name = createTableIdentifier("", external_table_name);
|
||||
auto database_and_table_name = std::make_shared<ASTTableIdentifier>(external_table_name);
|
||||
if (set_alias)
|
||||
{
|
||||
String alias = subquery_or_table_name->tryGetAlias();
|
||||
if (auto * table_name = subquery_or_table_name->as<ASTIdentifier>())
|
||||
if (auto * table_name = subquery_or_table_name->as<ASTTableIdentifier>())
|
||||
if (alias.empty())
|
||||
alias = table_name->shortName();
|
||||
database_and_table_name->setAlias(alias);
|
||||
|
@ -10,7 +10,6 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int SYNTAX_ERROR;
|
||||
extern const int AMBIGUOUS_COLUMN_NAME;
|
||||
}
|
||||
|
||||
@ -81,22 +80,6 @@ std::optional<String> IdentifierSemantic::getColumnName(const ASTPtr & ast)
|
||||
return {};
|
||||
}
|
||||
|
||||
std::optional<String> IdentifierSemantic::getTableName(const ASTIdentifier & node)
|
||||
{
|
||||
if (node.semantic->special)
|
||||
return node.name();
|
||||
return {};
|
||||
}
|
||||
|
||||
std::optional<String> IdentifierSemantic::getTableName(const ASTPtr & ast)
|
||||
{
|
||||
if (ast)
|
||||
if (const auto * id = ast->as<ASTIdentifier>())
|
||||
if (id->semantic->special)
|
||||
return id->name();
|
||||
return {};
|
||||
}
|
||||
|
||||
std::optional<ASTIdentifier> IdentifierSemantic::uncover(const ASTIdentifier & identifier)
|
||||
{
|
||||
if (identifier.semantic->covered)
|
||||
@ -146,16 +129,6 @@ std::optional<size_t> IdentifierSemantic::chooseTableColumnMatch(const ASTIdenti
|
||||
return tryChooseTable<TableWithColumnNamesAndTypes>(identifier, tables, ambiguous, true);
|
||||
}
|
||||
|
||||
StorageID IdentifierSemantic::extractDatabaseAndTable(const ASTIdentifier & identifier)
|
||||
{
|
||||
if (identifier.name_parts.size() > 2)
|
||||
throw Exception("Syntax 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 };
|
||||
}
|
||||
|
||||
std::optional<String> IdentifierSemantic::extractNestedName(const ASTIdentifier & identifier, const String & table_name)
|
||||
{
|
||||
if (identifier.name_parts.size() == 3 && table_name == identifier.name_parts[0])
|
||||
|
@ -1,14 +1,11 @@
|
||||
#pragma once
|
||||
|
||||
#include <optional>
|
||||
|
||||
#include <Interpreters/Aliases.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Interpreters/QueryAliasesVisitor.h>
|
||||
#include <Interpreters/getHeaderForProcessingStage.h>
|
||||
#include <Interpreters/getTableExpressions.h>
|
||||
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
|
||||
@ -44,9 +41,6 @@ struct IdentifierSemantic
|
||||
static std::optional<String> getColumnName(const ASTPtr & ast);
|
||||
|
||||
/// @returns name for 'not a column' identifiers
|
||||
static std::optional<String> getTableName(const ASTIdentifier & node);
|
||||
static std::optional<String> getTableName(const ASTPtr & ast);
|
||||
static StorageID extractDatabaseAndTable(const ASTIdentifier & identifier);
|
||||
static std::optional<String> extractNestedName(const ASTIdentifier & identifier, const String & table_name);
|
||||
|
||||
static ColumnMatch canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
|
||||
@ -83,8 +77,8 @@ private:
|
||||
class IdentifiersCollector
|
||||
{
|
||||
public:
|
||||
using ASTIdentPtr = const ASTIdentifier *;
|
||||
using ASTIdentifiers = std::vector<ASTIdentPtr>;
|
||||
using ASTIdentifierPtr = const ASTIdentifier *;
|
||||
using ASTIdentifiers = std::vector<ASTIdentifierPtr>;
|
||||
struct Data
|
||||
{
|
||||
ASTIdentifiers idents;
|
||||
|
@ -111,7 +111,7 @@ private:
|
||||
throw Exception("Distributed table should have an alias when distributed_product_mode set to local",
|
||||
ErrorCodes::DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED);
|
||||
|
||||
auto & identifier = database_and_table->as<ASTIdentifier &>();
|
||||
auto & identifier = database_and_table->as<ASTTableIdentifier &>();
|
||||
renamed_tables.emplace_back(identifier.clone());
|
||||
identifier.resetTable(database, table);
|
||||
}
|
||||
|
@ -65,8 +65,7 @@ BlockIO InterpreterAlterQuery::execute()
|
||||
auto alter_lock = table->lockForAlter(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout);
|
||||
auto metadata_snapshot = table->getInMemoryMetadataPtr();
|
||||
|
||||
/// Add default database to table identifiers that we can encounter in e.g. default expressions,
|
||||
/// mutation expression, etc.
|
||||
/// Add default database to table identifiers that we can encounter in e.g. default expressions, mutation expression, etc.
|
||||
AddDefaultDatabaseVisitor visitor(table_id.getDatabaseName());
|
||||
ASTPtr command_list_ptr = alter.command_list->ptr();
|
||||
visitor.visit(command_list_ptr);
|
||||
|
@ -134,7 +134,7 @@ String InterpreterSelectQuery::generateFilterActions(ActionsDAGPtr & actions, co
|
||||
tables->children.push_back(tables_elem);
|
||||
tables_elem->table_expression = table_expr;
|
||||
tables_elem->children.push_back(table_expr);
|
||||
table_expr->database_and_table_name = createTableIdentifier(db_name, table_name);
|
||||
table_expr->database_and_table_name = std::make_shared<ASTTableIdentifier>(db_name, table_name);
|
||||
table_expr->children.push_back(table_expr->database_and_table_name);
|
||||
|
||||
/// Using separate expression analyzer to prevent any possible alias injection
|
||||
|
@ -48,7 +48,7 @@ ASTPtr makeSubqueryTemplate()
|
||||
ASTPtr makeSubqueryQualifiedAsterisk()
|
||||
{
|
||||
auto asterisk = std::make_shared<ASTQualifiedAsterisk>();
|
||||
asterisk->children.emplace_back(std::make_shared<ASTIdentifier>("--.s"));
|
||||
asterisk->children.emplace_back(std::make_shared<ASTTableIdentifier>("--.s"));
|
||||
return asterisk;
|
||||
}
|
||||
|
||||
@ -142,7 +142,7 @@ private:
|
||||
|
||||
if (child->children.size() != 1)
|
||||
throw Exception("Logical error: qualified asterisk must have exactly one child", ErrorCodes::LOGICAL_ERROR);
|
||||
ASTIdentifier & identifier = child->children[0]->as<ASTIdentifier &>();
|
||||
auto & identifier = child->children[0]->as<ASTTableIdentifier &>();
|
||||
|
||||
data.addTableColumns(identifier.name());
|
||||
}
|
||||
@ -501,7 +501,6 @@ std::vector<TableNeededColumns> normalizeColumnNamesExtractNeeded(
|
||||
|
||||
for (ASTIdentifier * ident : identifiers)
|
||||
{
|
||||
|
||||
bool got_alias = aliases.count(ident->name());
|
||||
bool allow_ambiguous = got_alias; /// allow ambiguous column overridden by an alias
|
||||
|
||||
@ -512,14 +511,9 @@ std::vector<TableNeededColumns> normalizeColumnNamesExtractNeeded(
|
||||
if (got_alias)
|
||||
{
|
||||
auto alias = aliases.find(ident->name())->second;
|
||||
auto alias_table = IdentifierSemantic::getTableName(alias->ptr());
|
||||
bool alias_equals_column_name = false;
|
||||
if ((!ident->isShort() && alias->ptr()->getColumnNameWithoutAlias() == ident->getColumnNameWithoutAlias())
|
||||
|| (alias_table == IdentifierSemantic::getTableName(ident->ptr())
|
||||
&& ident->shortName() == alias->as<ASTIdentifier>()->shortName()))
|
||||
{
|
||||
alias_equals_column_name = true;
|
||||
}
|
||||
auto alias_ident = alias->clone();
|
||||
alias_ident->as<ASTIdentifier>()->restoreTable();
|
||||
bool alias_equals_column_name = alias_ident->getColumnNameWithoutAlias() == ident->getColumnNameWithoutAlias();
|
||||
if (!alias_equals_column_name)
|
||||
throw Exception("Alias clashes with qualified column '" + ident->name() + "'", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
|
||||
}
|
||||
|
@ -47,7 +47,7 @@ void replaceJoinedTable(const ASTSelectQuery & select_query)
|
||||
auto & table_expr = join->table_expression->as<ASTTableExpression &>();
|
||||
if (table_expr.database_and_table_name)
|
||||
{
|
||||
const auto & table_id = table_expr.database_and_table_name->as<ASTIdentifier &>();
|
||||
const auto & table_id = table_expr.database_and_table_name->as<ASTTableIdentifier &>();
|
||||
String expr = "(select * from " + table_id.name() + ") as " + table_id.shortName();
|
||||
|
||||
// FIXME: since the expression "a as b" exposes both "a" and "b" names, which is not equivalent to "(select * from a) as b",
|
||||
@ -109,7 +109,7 @@ private:
|
||||
|
||||
static void visit(const ASTQualifiedAsterisk & node, const ASTPtr &, Data & data)
|
||||
{
|
||||
ASTIdentifier & identifier = *node.children[0]->as<ASTIdentifier>();
|
||||
auto & identifier = node.children[0]->as<ASTTableIdentifier &>();
|
||||
bool rewritten = false;
|
||||
for (const auto & table : data)
|
||||
{
|
||||
@ -240,7 +240,7 @@ void JoinedTables::rewriteDistributedInAndJoins(ASTPtr & query)
|
||||
std::vector<DatabaseAndTableWithAlias> renamed;
|
||||
renamed.reserve(ast_tables.size());
|
||||
for (auto & ast : ast_tables)
|
||||
renamed.emplace_back(DatabaseAndTableWithAlias(*ast->as<ASTIdentifier>(), database));
|
||||
renamed.emplace_back(DatabaseAndTableWithAlias(ast->as<ASTTableIdentifier &>(), database));
|
||||
|
||||
/// Change qualified column names in distributed subqueries using table aliases.
|
||||
RenameQualifiedIdentifiersVisitor::Data data(renamed);
|
||||
|
@ -1,11 +1,13 @@
|
||||
#include <Poco/String.h>
|
||||
#include <Interpreters/misc.h>
|
||||
#include <Interpreters/MarkTableIdentifiersVisitor.h>
|
||||
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Interpreters/misc.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -22,39 +24,36 @@ void MarkTableIdentifiersMatcher::visit(ASTPtr & ast, Data & data)
|
||||
{
|
||||
if (auto * node_func = ast->as<ASTFunction>())
|
||||
visit(*node_func, ast, data);
|
||||
else if (auto * node_table = ast->as<ASTTableExpression>())
|
||||
visit(*node_table, ast, data);
|
||||
}
|
||||
|
||||
void MarkTableIdentifiersMatcher::visit(ASTTableExpression & table, ASTPtr &, Data &)
|
||||
{
|
||||
if (table.database_and_table_name)
|
||||
setIdentifierSpecial(table.database_and_table_name);
|
||||
}
|
||||
|
||||
void MarkTableIdentifiersMatcher::visit(const ASTFunction & func, ASTPtr &, Data & data)
|
||||
void MarkTableIdentifiersMatcher::visit(const ASTFunction & func, ASTPtr & ptr, Data & data)
|
||||
{
|
||||
/// `IN t` can be specified, where t is a table, which is equivalent to `IN (SELECT * FROM t)`.
|
||||
if (checkFunctionIsInOrGlobalInOperator(func))
|
||||
{
|
||||
auto & ast = func.arguments->children.at(1);
|
||||
auto ast = func.arguments->children.at(1);
|
||||
auto opt_name = tryGetIdentifierName(ast);
|
||||
if (opt_name && !data.aliases.count(*opt_name))
|
||||
setIdentifierSpecial(ast);
|
||||
if (opt_name && !data.aliases.count(*opt_name) && ast->as<ASTIdentifier>())
|
||||
{
|
||||
ptr->as<ASTFunction>()->arguments->children[1] = ast->as<ASTIdentifier>()->createTable();
|
||||
assert(ptr->as<ASTFunction>()->arguments->children[1]);
|
||||
}
|
||||
}
|
||||
|
||||
// First argument of joinGet can be a table name, perhaps with a database.
|
||||
// First argument of dictGet can be a dictionary name, perhaps with a database.
|
||||
if (functionIsJoinGet(func.name) || functionIsDictGet(func.name))
|
||||
else if (functionIsJoinGet(func.name) || functionIsDictGet(func.name))
|
||||
{
|
||||
if (!func.arguments || func.arguments->children.empty())
|
||||
{
|
||||
return;
|
||||
}
|
||||
auto & ast = func.arguments->children.at(0);
|
||||
|
||||
auto ast = func.arguments->children.at(0);
|
||||
auto opt_name = tryGetIdentifierName(ast);
|
||||
if (opt_name && !data.aliases.count(*opt_name))
|
||||
setIdentifierSpecial(ast);
|
||||
if (opt_name && !data.aliases.count(*opt_name) && ast->as<ASTIdentifier>())
|
||||
{
|
||||
ptr->as<ASTFunction>()->arguments->children[0] = ast->as<ASTIdentifier>()->createTable();
|
||||
assert(ptr->as<ASTFunction>()->arguments->children[0]);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -24,7 +24,6 @@ public:
|
||||
static void visit(ASTPtr & ast, Data & data);
|
||||
|
||||
private:
|
||||
static void visit(ASTTableExpression & table, ASTPtr &, Data &);
|
||||
static void visit(const ASTFunction & func, ASTPtr &, Data &);
|
||||
};
|
||||
|
||||
|
@ -272,7 +272,7 @@ MutationsInterpreter::MutationsInterpreter(
|
||||
: storage(std::move(storage_))
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, commands(std::move(commands_))
|
||||
, context(context_)
|
||||
, context(Context::createCopy(context_))
|
||||
, can_execute(can_execute_)
|
||||
, select_limits(SelectQueryOptions().analyze(!can_execute).ignoreLimits())
|
||||
{
|
||||
|
@ -27,7 +27,7 @@ void ReplaceQueryParameterVisitor::visit(ASTPtr & ast)
|
||||
{
|
||||
if (ast->as<ASTQueryParameter>())
|
||||
visitQueryParameter(ast);
|
||||
else if (ast->as<ASTIdentifier>())
|
||||
else if (ast->as<ASTIdentifier>() || ast->as<ASTTableIdentifier>())
|
||||
visitIdentifier(ast);
|
||||
else
|
||||
visitChildren(ast);
|
||||
@ -77,25 +77,26 @@ void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast)
|
||||
|
||||
void ReplaceQueryParameterVisitor::visitIdentifier(ASTPtr & ast)
|
||||
{
|
||||
auto & ast_identifier = ast->as<ASTIdentifier &>();
|
||||
if (ast_identifier.children.empty())
|
||||
auto ast_identifier = dynamic_pointer_cast<ASTIdentifier>(ast);
|
||||
if (ast_identifier->children.empty())
|
||||
return;
|
||||
|
||||
auto & name_parts = ast_identifier.name_parts;
|
||||
auto & name_parts = ast_identifier->name_parts;
|
||||
for (size_t i = 0, j = 0, size = name_parts.size(); i < size; ++i)
|
||||
{
|
||||
if (name_parts[i].empty())
|
||||
{
|
||||
const auto & ast_param = ast_identifier.children[j++]->as<ASTQueryParameter &>();
|
||||
const auto & ast_param = ast_identifier->children[j++]->as<ASTQueryParameter &>();
|
||||
name_parts[i] = getParamValue(ast_param.name);
|
||||
}
|
||||
}
|
||||
|
||||
if (!ast_identifier.semantic->special && name_parts.size() >= 2)
|
||||
ast_identifier.semantic->table = ast_identifier.name_parts.end()[-2];
|
||||
/// FIXME: what should this mean?
|
||||
if (!ast_identifier->semantic->special && name_parts.size() >= 2)
|
||||
ast_identifier->semantic->table = ast_identifier->name_parts.end()[-2];
|
||||
|
||||
ast_identifier.resetFullName();
|
||||
ast_identifier.children.clear();
|
||||
ast_identifier->resetFullName();
|
||||
ast_identifier->children.clear();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -24,7 +24,7 @@ StorageID::StorageID(const ASTQueryWithTableAndOutput & query)
|
||||
assertNotEmpty();
|
||||
}
|
||||
|
||||
StorageID::StorageID(const ASTIdentifier & table_identifier_node)
|
||||
StorageID::StorageID(const ASTTableIdentifier & table_identifier_node)
|
||||
{
|
||||
DatabaseAndTableWithAlias database_table(table_identifier_node);
|
||||
database_name = database_table.database;
|
||||
@ -35,7 +35,7 @@ StorageID::StorageID(const ASTIdentifier & table_identifier_node)
|
||||
|
||||
StorageID::StorageID(const ASTPtr & node)
|
||||
{
|
||||
if (const auto * identifier = dynamic_cast<const ASTIdentifier *>(node.get()))
|
||||
if (const auto * identifier = node->as<ASTTableIdentifier>())
|
||||
*this = StorageID(*identifier);
|
||||
else if (const auto * simple_query = dynamic_cast<const ASTQueryWithTableAndOutput *>(node.get()))
|
||||
*this = StorageID(*simple_query);
|
||||
|
@ -25,7 +25,7 @@ namespace ErrorCodes
|
||||
static constexpr char const * TABLE_WITH_UUID_NAME_PLACEHOLDER = "_";
|
||||
|
||||
class ASTQueryWithTableAndOutput;
|
||||
class ASTIdentifier;
|
||||
class ASTTableIdentifier;
|
||||
class Context;
|
||||
|
||||
// TODO(ilezhankin): refactor and merge |ASTTableIdentifier|
|
||||
@ -42,7 +42,7 @@ struct StorageID
|
||||
}
|
||||
|
||||
StorageID(const ASTQueryWithTableAndOutput & query);
|
||||
StorageID(const ASTIdentifier & table_identifier_node);
|
||||
StorageID(const ASTTableIdentifier & table_identifier_node);
|
||||
StorageID(const ASTPtr & node);
|
||||
|
||||
String getDatabaseName() const;
|
||||
|
@ -47,7 +47,7 @@ std::shared_ptr<InterpreterSelectWithUnionQuery> interpretSubquery(
|
||||
/// Subquery or table name. The name of the table is similar to the subquery `SELECT * FROM t`.
|
||||
const auto * subquery = table_expression->as<ASTSubquery>();
|
||||
const auto * function = table_expression->as<ASTFunction>();
|
||||
const auto * table = table_expression->as<ASTIdentifier>();
|
||||
const auto * table = table_expression->as<ASTTableIdentifier>();
|
||||
|
||||
if (!subquery && !table && !function)
|
||||
throw Exception("Table expression is undefined, Method: ExpressionAnalyzer::interpretSubquery." , ErrorCodes::LOGICAL_ERROR);
|
||||
|
@ -25,7 +25,7 @@ ASTPtr processColumnTransformers(
|
||||
TablesWithColumns tables_with_columns;
|
||||
{
|
||||
auto table_expr = std::make_shared<ASTTableExpression>();
|
||||
table_expr->database_and_table_name = createTableIdentifier(table->getStorageID());
|
||||
table_expr->database_and_table_name = std::make_shared<ASTTableIdentifier>(table->getStorageID());
|
||||
table_expr->children.push_back(table_expr->database_and_table_name);
|
||||
tables_with_columns.emplace_back(DatabaseAndTableWithAlias(*table_expr, current_database), names_and_types);
|
||||
}
|
||||
|
@ -9,7 +9,6 @@ namespace DB
|
||||
|
||||
struct ASTCheckQuery : public ASTQueryWithTableAndOutput
|
||||
{
|
||||
|
||||
ASTPtr partition;
|
||||
|
||||
/** Get the text that identifies this element. */
|
||||
|
@ -13,13 +13,12 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNEXPECTED_AST_STRUCTURE;
|
||||
extern const int SYNTAX_ERROR;
|
||||
}
|
||||
|
||||
ASTIdentifier::ASTIdentifier(const String & short_name, ASTPtr && name_param)
|
||||
: full_name(short_name), name_parts{short_name}, semantic(std::make_shared<IdentifierSemanticImpl>())
|
||||
{
|
||||
if (name_param == nullptr)
|
||||
if (!name_param)
|
||||
assert(!full_name.empty());
|
||||
else
|
||||
children.push_back(std::move(name_param));
|
||||
@ -80,7 +79,6 @@ void ASTIdentifier::setShortName(const String & new_name)
|
||||
name_parts = {new_name};
|
||||
|
||||
bool special = semantic->special;
|
||||
/// How about keep the semantic info here, such as table
|
||||
auto table = semantic->table;
|
||||
|
||||
*semantic = IdentifierSemanticImpl();
|
||||
@ -108,8 +106,7 @@ void ASTIdentifier::formatImplWithoutAlias(const FormatSettings & settings, Form
|
||||
settings.ostr << (settings.hilite ? hilite_none : "");
|
||||
};
|
||||
|
||||
/// It could be compound but short
|
||||
if (!isShort())
|
||||
if (compound())
|
||||
{
|
||||
for (size_t i = 0, j = 0, size = name_parts.size(); i < size; ++i)
|
||||
{
|
||||
@ -152,19 +149,11 @@ void ASTIdentifier::restoreTable()
|
||||
}
|
||||
}
|
||||
|
||||
void ASTIdentifier::resetTable(const String & database_name, const String & table_name)
|
||||
std::shared_ptr<ASTTableIdentifier> ASTIdentifier::createTable() const
|
||||
{
|
||||
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;
|
||||
}
|
||||
|
||||
void ASTIdentifier::updateTreeHashImpl(SipHash & hash_state) const
|
||||
{
|
||||
hash_state.update(uuid);
|
||||
IAST::updateTreeHashImpl(hash_state);
|
||||
if (name_parts.size() == 1) return std::make_shared<ASTTableIdentifier>(name_parts[0]);
|
||||
if (name_parts.size() == 2) return std::make_shared<ASTTableIdentifier>(name_parts[0], name_parts[1]);
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
void ASTIdentifier::resetFullName()
|
||||
@ -174,21 +163,56 @@ void ASTIdentifier::resetFullName()
|
||||
full_name += '.' + name_parts[i];
|
||||
}
|
||||
|
||||
ASTPtr createTableIdentifier(const String & database_name, const String & table_name)
|
||||
ASTTableIdentifier::ASTTableIdentifier(const String & table_name, std::vector<ASTPtr> && name_params)
|
||||
: ASTIdentifier({table_name}, true, std::move(name_params))
|
||||
{
|
||||
assert(database_name != "_temporary_and_external_tables");
|
||||
return createTableIdentifier(StorageID(database_name, table_name));
|
||||
}
|
||||
|
||||
ASTPtr createTableIdentifier(const StorageID & table_id)
|
||||
ASTTableIdentifier::ASTTableIdentifier(const StorageID & table_id, std::vector<ASTPtr> && name_params)
|
||||
: ASTIdentifier(
|
||||
table_id.database_name.empty() ? std::vector<String>{table_id.table_name}
|
||||
: std::vector<String>{table_id.database_name, table_id.table_name},
|
||||
true, std::move(name_params))
|
||||
{
|
||||
std::shared_ptr<ASTIdentifier> res;
|
||||
if (table_id.database_name.empty())
|
||||
res = std::make_shared<ASTIdentifier>(std::vector<String>{table_id.table_name}, true);
|
||||
else
|
||||
res = std::make_shared<ASTIdentifier>(std::vector<String>{table_id.database_name, table_id.table_name}, true);
|
||||
res->uuid = table_id.uuid;
|
||||
return res;
|
||||
uuid = table_id.uuid;
|
||||
}
|
||||
|
||||
ASTTableIdentifier::ASTTableIdentifier(const String & database_name, const String & table_name, std::vector<ASTPtr> && name_params)
|
||||
: ASTIdentifier({database_name, table_name}, true, std::move(name_params))
|
||||
{
|
||||
}
|
||||
|
||||
ASTPtr ASTTableIdentifier::clone() const
|
||||
{
|
||||
auto ret = std::make_shared<ASTTableIdentifier>(*this);
|
||||
ret->semantic = std::make_shared<IdentifierSemanticImpl>(*ret->semantic);
|
||||
return ret;
|
||||
}
|
||||
|
||||
StorageID ASTTableIdentifier::getTableId() const
|
||||
{
|
||||
if (name_parts.size() == 2) return {name_parts[0], name_parts[1], uuid};
|
||||
else return {{}, name_parts[0], uuid};
|
||||
}
|
||||
|
||||
String ASTTableIdentifier::getDatabaseName() const
|
||||
{
|
||||
if (name_parts.size() == 2) return name_parts[0];
|
||||
else return {};
|
||||
}
|
||||
|
||||
void ASTTableIdentifier::resetTable(const String & database_name, const String & table_name)
|
||||
{
|
||||
auto identifier = std::make_shared<ASTTableIdentifier>(database_name, table_name);
|
||||
full_name.swap(identifier->full_name);
|
||||
name_parts.swap(identifier->name_parts);
|
||||
uuid = identifier->uuid;
|
||||
}
|
||||
|
||||
void ASTTableIdentifier::updateTreeHashImpl(SipHash & hash_state) const
|
||||
{
|
||||
hash_state.update(uuid);
|
||||
IAST::updateTreeHashImpl(hash_state);
|
||||
}
|
||||
|
||||
String getIdentifierName(const IAST * ast)
|
||||
@ -211,7 +235,7 @@ bool tryGetIdentifierNameInto(const IAST * ast, String & name)
|
||||
{
|
||||
if (ast)
|
||||
{
|
||||
if (const auto * node = ast->as<ASTIdentifier>())
|
||||
if (const auto * node = dynamic_cast<const ASTIdentifier *>(ast))
|
||||
{
|
||||
name = node->name();
|
||||
return true;
|
||||
@ -227,17 +251,4 @@ void setIdentifierSpecial(ASTPtr & ast)
|
||||
id->semantic->special = true;
|
||||
}
|
||||
|
||||
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 };
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,10 +1,10 @@
|
||||
#pragma once
|
||||
|
||||
#include <optional>
|
||||
|
||||
#include <Core/UUID.h>
|
||||
#include <Parsers/ASTQueryParameter.h>
|
||||
#include <Parsers/ASTWithAlias.h>
|
||||
#include <Core/UUID.h>
|
||||
|
||||
#include <optional>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -14,14 +14,15 @@ struct IdentifierSemantic;
|
||||
struct IdentifierSemanticImpl;
|
||||
struct StorageID;
|
||||
|
||||
class ASTTableIdentifier;
|
||||
|
||||
/// Identifier (column, table or alias)
|
||||
/// FIXME: rewrite code about params - they should be substituted at the parsing stage,
|
||||
/// or parsed as a separate AST entity.
|
||||
|
||||
/// Generic identifier. ASTTableIdentifier - for table identifier.
|
||||
class ASTIdentifier : public ASTWithAlias
|
||||
{
|
||||
friend class ReplaceQueryParameterVisitor;
|
||||
public:
|
||||
UUID uuid = UUIDHelpers::Nil;
|
||||
|
||||
explicit ASTIdentifier(const String & short_name, ASTPtr && name_param = {});
|
||||
explicit ASTIdentifier(std::vector<String> && name_parts, bool special = false, std::vector<ASTPtr> && name_params = {});
|
||||
|
||||
@ -47,15 +48,12 @@ public:
|
||||
const String & name() const;
|
||||
|
||||
void restoreTable(); // TODO(ilezhankin): get rid of this
|
||||
|
||||
// FIXME: used only when it's needed to rewrite distributed table name to real remote table name.
|
||||
void resetTable(const String & database_name, const String & table_name); // TODO(ilezhankin): get rid of this
|
||||
|
||||
void updateTreeHashImpl(SipHash & hash_state) const override;
|
||||
std::shared_ptr<ASTTableIdentifier> createTable() const; // returns |nullptr| if identifier is not table.
|
||||
|
||||
protected:
|
||||
String full_name;
|
||||
std::vector<String> name_parts;
|
||||
std::shared_ptr<IdentifierSemanticImpl> semantic; /// pimpl
|
||||
|
||||
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||
void appendColumnNameImpl(WriteBuffer & ostr) const override;
|
||||
@ -63,26 +61,42 @@ protected:
|
||||
private:
|
||||
using ASTWithAlias::children; /// ASTIdentifier is child free
|
||||
|
||||
std::shared_ptr<IdentifierSemanticImpl> semantic; /// pimpl
|
||||
|
||||
friend class ReplaceQueryParameterVisitor;
|
||||
friend struct IdentifierSemantic;
|
||||
friend ASTPtr createTableIdentifier(const StorageID & table_id);
|
||||
friend void setIdentifierSpecial(ASTPtr & ast);
|
||||
friend StorageID getTableIdentifier(const ASTPtr & ast);
|
||||
|
||||
void resetFullName();
|
||||
};
|
||||
|
||||
class ASTTableIdentifier : public ASTIdentifier
|
||||
{
|
||||
public:
|
||||
explicit ASTTableIdentifier(const String & table_name, std::vector<ASTPtr> && name_params = {});
|
||||
explicit ASTTableIdentifier(const StorageID & table_id, std::vector<ASTPtr> && name_params = {});
|
||||
ASTTableIdentifier(const String & database_name, const String & table_name, std::vector<ASTPtr> && name_params = {});
|
||||
|
||||
String getID(char delim) const override { return "TableIdentifier" + (delim + name()); }
|
||||
ASTPtr clone() const override;
|
||||
|
||||
UUID uuid = UUIDHelpers::Nil; // FIXME(ilezhankin): make private
|
||||
|
||||
StorageID getTableId() const;
|
||||
String getDatabaseName() const;
|
||||
|
||||
// FIXME: used only when it's needed to rewrite distributed table name to real remote table name.
|
||||
void resetTable(const String & database_name, const String & table_name); // TODO(ilezhankin): get rid of this
|
||||
|
||||
void updateTreeHashImpl(SipHash & hash_state) const override;
|
||||
};
|
||||
|
||||
|
||||
/// ASTIdentifier Helpers: hide casts and semantic.
|
||||
|
||||
ASTPtr createTableIdentifier(const String & database_name, const String & table_name);
|
||||
ASTPtr createTableIdentifier(const StorageID & table_id);
|
||||
void setIdentifierSpecial(ASTPtr & ast);
|
||||
|
||||
String getIdentifierName(const IAST * ast);
|
||||
std::optional<String> tryGetIdentifierName(const IAST * ast);
|
||||
bool tryGetIdentifierNameInto(const IAST * ast, String & name);
|
||||
StorageID getTableIdentifier(const ASTPtr & ast);
|
||||
|
||||
inline String getIdentifierName(const ASTPtr & ast) { return getIdentifierName(ast.get()); }
|
||||
inline std::optional<String> tryGetIdentifierName(const ASTPtr & ast) { return tryGetIdentifierName(ast.get()); }
|
||||
|
@ -376,7 +376,7 @@ void ASTSelectQuery::replaceDatabaseAndTable(const StorageID & table_id)
|
||||
}
|
||||
|
||||
String table_alias = getTableExpressionAlias(table_expression);
|
||||
table_expression->database_and_table_name = createTableIdentifier(table_id);
|
||||
table_expression->database_and_table_name = std::make_shared<ASTTableIdentifier>(table_id);
|
||||
|
||||
if (!table_alias.empty())
|
||||
table_expression->database_and_table_name->setAlias(table_alias);
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
@ -244,17 +244,26 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex
|
||||
ParserKeyword s_uuid("UUID");
|
||||
UUID uuid = UUIDHelpers::Nil;
|
||||
|
||||
if (table_name_with_optional_uuid && parts.size() <= 2 && s_uuid.ignore(pos, expected))
|
||||
if (table_name_with_optional_uuid)
|
||||
{
|
||||
ParserStringLiteral uuid_p;
|
||||
ASTPtr ast_uuid;
|
||||
if (!uuid_p.parse(pos, ast_uuid, expected))
|
||||
if (parts.size() > 2)
|
||||
return false;
|
||||
uuid = parseFromString<UUID>(ast_uuid->as<ASTLiteral>()->value.get<String>());
|
||||
}
|
||||
|
||||
node = std::make_shared<ASTIdentifier>(std::move(parts), false, std::move(params));
|
||||
node->as<ASTIdentifier>()->uuid = uuid;
|
||||
if (s_uuid.ignore(pos, expected))
|
||||
{
|
||||
ParserStringLiteral uuid_p;
|
||||
ASTPtr ast_uuid;
|
||||
if (!uuid_p.parse(pos, ast_uuid, expected))
|
||||
return false;
|
||||
uuid = parseFromString<UUID>(ast_uuid->as<ASTLiteral>()->value.get<String>());
|
||||
}
|
||||
|
||||
if (parts.size() == 1) node = std::make_shared<ASTTableIdentifier>(parts[0], std::move(params));
|
||||
else node = std::make_shared<ASTTableIdentifier>(parts[0], parts[1], std::move(params));
|
||||
node->as<ASTTableIdentifier>()->uuid = uuid;
|
||||
}
|
||||
else
|
||||
node = std::make_shared<ASTIdentifier>(std::move(parts), false, std::move(params));
|
||||
|
||||
return true;
|
||||
}
|
||||
@ -1729,7 +1738,7 @@ bool ParserAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
bool ParserColumnsMatcher::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ParserKeyword columns("COLUMNS");
|
||||
ParserList columns_p(std::make_unique<ParserCompoundIdentifier>(true), std::make_unique<ParserToken>(TokenType::Comma), false);
|
||||
ParserList columns_p(std::make_unique<ParserCompoundIdentifier>(false, true), std::make_unique<ParserToken>(TokenType::Comma), false);
|
||||
ParserStringLiteral regex;
|
||||
|
||||
if (!columns.ignore(pos, expected))
|
||||
@ -1955,7 +1964,7 @@ bool ParserAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
|
||||
bool ParserQualifiedAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
if (!ParserCompoundIdentifier(false, true).parse(pos, node, expected))
|
||||
if (!ParserCompoundIdentifier(true, true).parse(pos, node, expected))
|
||||
return false;
|
||||
|
||||
if (pos->type != TokenType::Dot)
|
||||
|
@ -46,6 +46,7 @@ class ParserIdentifier : public IParserBase
|
||||
{
|
||||
public:
|
||||
explicit ParserIdentifier(bool allow_query_parameter_ = false) : allow_query_parameter(allow_query_parameter_) {}
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "identifier"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
@ -378,8 +379,8 @@ protected:
|
||||
class ParserAlias : public IParserBase
|
||||
{
|
||||
public:
|
||||
ParserAlias(bool allow_alias_without_as_keyword_)
|
||||
: allow_alias_without_as_keyword(allow_alias_without_as_keyword_) {}
|
||||
explicit ParserAlias(bool allow_alias_without_as_keyword_) : allow_alias_without_as_keyword(allow_alias_without_as_keyword_) { }
|
||||
|
||||
private:
|
||||
static const char * restricted_keywords[];
|
||||
|
||||
@ -466,8 +467,10 @@ protected:
|
||||
class ParserFunctionWithKeyValueArguments : public IParserBase
|
||||
{
|
||||
public:
|
||||
ParserFunctionWithKeyValueArguments(bool brackets_can_be_omitted_ = false)
|
||||
: brackets_can_be_omitted(brackets_can_be_omitted_) {}
|
||||
explicit ParserFunctionWithKeyValueArguments(bool brackets_can_be_omitted_ = false) : brackets_can_be_omitted(brackets_can_be_omitted_)
|
||||
{
|
||||
}
|
||||
|
||||
protected:
|
||||
|
||||
const char * getName() const override { return "function with key-value arguments"; }
|
||||
|
@ -242,10 +242,10 @@ static inline bool parseRenameCommand(IParser::Pos & pos, ASTPtr & node, Expecte
|
||||
}
|
||||
else if (ParserKeyword("TO").ignore(pos, expected) || ParserKeyword("AS").ignore(pos, expected))
|
||||
{
|
||||
if (!ParserCompoundIdentifier(false).parse(pos, new_name, expected))
|
||||
if (!ParserCompoundIdentifier(true).parse(pos, new_name, expected))
|
||||
return false;
|
||||
|
||||
StorageID new_table_id = getTableIdentifier(new_name);
|
||||
auto new_table_id = new_name->as<ASTTableIdentifier>()->getTableId();
|
||||
alter_command->type = ASTAlterCommand::RENAME_TABLE;
|
||||
alter_command->new_table_name = new_table_id.table_name;
|
||||
alter_command->new_database_name = new_table_id.database_name;
|
||||
|
@ -36,7 +36,7 @@ bool ParserAlterQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & e
|
||||
if (!ParserKeyword("ALTER TABLE").ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (!ParserCompoundIdentifier(false).parse(pos, table, expected))
|
||||
if (!ParserCompoundIdentifier(true).parse(pos, table, expected))
|
||||
return false;
|
||||
|
||||
if (!ParserList(std::make_unique<ParserAlterCommand>(), std::make_unique<ParserToken>(TokenType::Comma)).parse(pos, command_list, expected))
|
||||
@ -46,7 +46,7 @@ bool ParserAlterQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & e
|
||||
|
||||
node = alter_query;
|
||||
alter_query->command_list = command_list;
|
||||
StorageID table_id = getTableIdentifier(table);
|
||||
auto table_id = table->as<ASTTableIdentifier>()->getTableId();
|
||||
alter_query->table = table_id.table_name;
|
||||
alter_query->database = table_id.database_name;
|
||||
|
||||
|
@ -63,19 +63,19 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
if (ParserKeyword("IF NOT EXISTS").ignore(pos, expected))
|
||||
if_not_exists = true;
|
||||
|
||||
if (!ParserCompoundIdentifier(false).parse(pos, table, expected))
|
||||
if (!ParserCompoundIdentifier(true).parse(pos, table, expected))
|
||||
return false;
|
||||
|
||||
if (ParserKeyword("LIKE").ignore(pos, expected))
|
||||
{
|
||||
if (!ParserCompoundIdentifier(false).parse(pos, like_table, expected))
|
||||
if (!ParserCompoundIdentifier(true).parse(pos, like_table, expected))
|
||||
return false;
|
||||
}
|
||||
else if (ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected))
|
||||
{
|
||||
if (ParserKeyword("LIKE").ignore(pos, expected))
|
||||
{
|
||||
if (!ParserCompoundIdentifier(false).parse(pos, like_table, expected))
|
||||
if (!ParserCompoundIdentifier(true).parse(pos, like_table, expected))
|
||||
return false;
|
||||
|
||||
if (!ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected))
|
||||
@ -101,7 +101,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
create_query->temporary = is_temporary;
|
||||
create_query->if_not_exists = if_not_exists;
|
||||
|
||||
StorageID table_id = getTableIdentifier(table);
|
||||
auto table_id = table->as<ASTTableIdentifier>()->getTableId();
|
||||
create_query->table = table_id.table_name;
|
||||
create_query->database = table_id.database_name;
|
||||
create_query->like_table = like_table;
|
||||
|
@ -316,10 +316,9 @@ ASTPtr AlterTableClause::convertToOld() const
|
||||
|
||||
if (has(FROM))
|
||||
{
|
||||
auto table_id = getTableIdentifier(get(FROM)->convertToOld());
|
||||
|
||||
command->from_database = table_id.database_name;
|
||||
command->from_table = table_id.table_name;
|
||||
auto table = get(FROM)->convertToOld();
|
||||
command->from_database = table->as<ASTTableIdentifier>()->getDatabaseName();
|
||||
command->from_table = table->as<ASTTableIdentifier>()->shortName();
|
||||
command->replace = false;
|
||||
command->type = ASTAlterCommand::REPLACE_PARTITION;
|
||||
}
|
||||
@ -451,9 +450,9 @@ ASTPtr AlterTableClause::convertToOld() const
|
||||
command->partition = get(PARTITION)->convertToOld();
|
||||
command->move_destination_type = DataDestinationType::TABLE;
|
||||
{
|
||||
auto table_id = getTableIdentifier(get(TO)->convertToOld());
|
||||
command->to_database = table_id.database_name;
|
||||
command->to_table = table_id.table_name;
|
||||
auto table = get(TO)->convertToOld();
|
||||
command->to_database = table->as<ASTTableIdentifier>()->getDatabaseName();
|
||||
command->to_table = table->as<ASTTableIdentifier>()->shortName();
|
||||
}
|
||||
break;
|
||||
|
||||
@ -515,9 +514,9 @@ ASTPtr AlterTableClause::convertToOld() const
|
||||
command->replace = true;
|
||||
command->partition = get(PARTITION)->convertToOld();
|
||||
{
|
||||
auto table_id = getTableIdentifier(get(FROM)->convertToOld());
|
||||
command->from_database = table_id.database_name;
|
||||
command->from_table = table_id.table_name;
|
||||
auto table = get(FROM)->convertToOld();
|
||||
command->from_database = table->as<ASTTableIdentifier>()->getDatabaseName();
|
||||
command->from_table = table->as<ASTTableIdentifier>()->shortName();
|
||||
}
|
||||
break;
|
||||
|
||||
@ -573,9 +572,9 @@ ASTPtr AlterTableQuery::convertToOld() const
|
||||
auto query = std::make_shared<ASTAlterQuery>();
|
||||
|
||||
{
|
||||
auto table_id = getTableIdentifier(get(TABLE)->convertToOld());
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
auto table = get(TABLE)->convertToOld();
|
||||
query->database = table->as<ASTTableIdentifier>()->getDatabaseName();
|
||||
query->table = table->as<ASTTableIdentifier>()->shortName();
|
||||
}
|
||||
|
||||
query->cluster = cluster_name;
|
||||
|
@ -29,9 +29,9 @@ ASTPtr AttachQuery::convertToOld() const
|
||||
case QueryType::DICTIONARY:
|
||||
query->is_dictionary = true;
|
||||
{
|
||||
auto table_id = getTableIdentifier(get(NAME)->convertToOld());
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
auto table = get(NAME)->convertToOld();
|
||||
query->database = table->as<ASTTableIdentifier>()->getDatabaseName();
|
||||
query->table = table->as<ASTTableIdentifier>()->shortName();
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
@ -19,9 +19,9 @@ ASTPtr CheckQuery::convertToOld() const
|
||||
{
|
||||
auto query = std::make_shared<ASTCheckQuery>();
|
||||
|
||||
auto table_id = getTableIdentifier(get(NAME)->convertToOld());
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
auto table = std::static_pointer_cast<ASTTableIdentifier>(get(NAME)->convertToOld());
|
||||
query->database = table->getDatabaseName();
|
||||
query->table = table->shortName();
|
||||
|
||||
if (has(PARTITION)) query->partition = get(PARTITION)->convertToOld();
|
||||
|
||||
|
@ -239,11 +239,11 @@ ASTPtr CreateDictionaryQuery::convertToOld() const
|
||||
auto query = std::make_shared<ASTCreateQuery>();
|
||||
|
||||
{
|
||||
auto table_id = getTableIdentifier(get(NAME)->convertToOld());
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
query->uuid
|
||||
= has(UUID) ? parseFromString<DB::UUID>(get(UUID)->convertToOld()->as<ASTLiteral>()->value.get<String>()) : table_id.uuid;
|
||||
auto table = get(NAME)->convertToOld();
|
||||
query->database = table->as<ASTTableIdentifier>()->getDatabaseName();
|
||||
query->table = table->as<ASTTableIdentifier>()->shortName();
|
||||
query->uuid = has(UUID) ? parseFromString<DB::UUID>(get(UUID)->convertToOld()->as<ASTLiteral>()->value.get<String>())
|
||||
: table->as<ASTTableIdentifier>()->uuid;
|
||||
}
|
||||
|
||||
query->cluster = cluster_name;
|
||||
|
@ -29,18 +29,17 @@ ASTPtr CreateLiveViewQuery::convertToOld() const
|
||||
auto query = std::make_shared<ASTCreateQuery>();
|
||||
|
||||
{
|
||||
auto table_id = getTableIdentifier(get(NAME)->convertToOld());
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
query->uuid
|
||||
= has(UUID) ? parseFromString<DB::UUID>(get(UUID)->convertToOld()->as<ASTLiteral>()->value.get<String>()) : table_id.uuid;
|
||||
auto table = std::static_pointer_cast<ASTTableIdentifier>(get(NAME)->convertToOld());
|
||||
query->database = table->getDatabaseName();
|
||||
query->table = table->shortName();
|
||||
query->uuid = has(UUID) ? parseFromString<DB::UUID>(get(UUID)->convertToOld()->as<ASTLiteral>()->value.get<String>()) : table->uuid;
|
||||
}
|
||||
|
||||
if (has(TIMEOUT))
|
||||
query->live_view_timeout.emplace(get(TIMEOUT)->convertToOld()->as<ASTLiteral>()->value.get<UInt64>());
|
||||
|
||||
if (has(DESTINATION))
|
||||
query->to_table_id = getTableIdentifier(get(DESTINATION)->convertToOld());
|
||||
query->to_table_id = get(DESTINATION)->convertToOld()->as<ASTTableIdentifier>()->getTableId();
|
||||
|
||||
if (has(SCHEMA))
|
||||
{
|
||||
|
@ -35,15 +35,14 @@ ASTPtr CreateMaterializedViewQuery::convertToOld() const
|
||||
auto query = std::make_shared<ASTCreateQuery>();
|
||||
|
||||
{
|
||||
auto table_id = getTableIdentifier(get(NAME)->convertToOld());
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
query->uuid
|
||||
= has(UUID) ? parseFromString<DB::UUID>(get(UUID)->convertToOld()->as<ASTLiteral>()->value.get<String>()) : table_id.uuid;
|
||||
auto table = std::static_pointer_cast<ASTTableIdentifier>(get(NAME)->convertToOld());
|
||||
query->database = table->getDatabaseName();
|
||||
query->table = table->shortName();
|
||||
query->uuid = has(UUID) ? parseFromString<DB::UUID>(get(UUID)->convertToOld()->as<ASTLiteral>()->value.get<String>()) : table->uuid;
|
||||
}
|
||||
|
||||
if (has(DESTINATION))
|
||||
query->to_table_id = getTableIdentifier(get(DESTINATION)->convertToOld());
|
||||
query->to_table_id = get(DESTINATION)->convertToOld()->as<ASTTableIdentifier>()->getTableId();
|
||||
else if (has(ENGINE))
|
||||
{
|
||||
query->set(query->storage, get(ENGINE)->convertToOld());
|
||||
|
@ -113,11 +113,11 @@ ASTPtr CreateTableQuery::convertToOld() const
|
||||
auto query = std::make_shared<ASTCreateQuery>();
|
||||
|
||||
{
|
||||
auto table_id = getTableIdentifier(get(NAME)->convertToOld());
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
query->uuid
|
||||
= has(UUID) ? parseFromString<DB::UUID>(get(UUID)->convertToOld()->as<ASTLiteral>()->value.get<String>()) : table_id.uuid;
|
||||
auto table = get(NAME)->convertToOld();
|
||||
query->database = table->as<ASTTableIdentifier>()->getDatabaseName();
|
||||
query->table = table->as<ASTTableIdentifier>()->shortName();
|
||||
query->uuid = has(UUID) ? parseFromString<DB::UUID>(get(UUID)->convertToOld()->as<ASTLiteral>()->value.get<String>())
|
||||
: table->as<ASTTableIdentifier>()->uuid;
|
||||
}
|
||||
|
||||
query->cluster = cluster_name;
|
||||
@ -137,9 +137,9 @@ ASTPtr CreateTableQuery::convertToOld() const
|
||||
}
|
||||
case TableSchemaClause::ClauseType::TABLE:
|
||||
{
|
||||
auto table_id = getTableIdentifier(get(SCHEMA)->convertToOld());
|
||||
query->as_database = table_id.database_name;
|
||||
query->as_table = table_id.table_name;
|
||||
auto table = std::static_pointer_cast<ASTTableIdentifier>(get(SCHEMA)->convertToOld());
|
||||
query->as_database = table->getDatabaseName();
|
||||
query->as_table = table->shortName();
|
||||
break;
|
||||
}
|
||||
case TableSchemaClause::ClauseType::FUNCTION:
|
||||
|
@ -26,10 +26,10 @@ ASTPtr CreateViewQuery::convertToOld() const
|
||||
auto query = std::make_shared<ASTCreateQuery>();
|
||||
|
||||
{
|
||||
auto table_id = getTableIdentifier(get(NAME)->convertToOld());
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
query->uuid = table_id.uuid;
|
||||
auto table = std::static_pointer_cast<ASTTableIdentifier>(get(NAME)->convertToOld());
|
||||
query->database = table->getDatabaseName();
|
||||
query->table = table->shortName();
|
||||
query->uuid = table->uuid;
|
||||
}
|
||||
|
||||
query->attach = attach;
|
||||
|
@ -50,10 +50,10 @@ ASTPtr ExistsQuery::convertToOld() const
|
||||
}
|
||||
|
||||
// FIXME: this won't work if table doesn't exist
|
||||
auto table_id = getTableIdentifier(get<TableIdentifier>(IDENTIFIER)->convertToOld());
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
query->uuid = table_id.uuid;
|
||||
auto table_id = std::static_pointer_cast<ASTTableIdentifier>(get<TableIdentifier>(IDENTIFIER)->convertToOld());
|
||||
query->database = table_id->getDatabaseName();
|
||||
query->table = table_id->shortName();
|
||||
query->uuid = table_id->uuid;
|
||||
query->temporary = temporary;
|
||||
|
||||
return query;
|
||||
|
@ -7,15 +7,37 @@
|
||||
namespace DB::AST
|
||||
{
|
||||
|
||||
ExplainQuery::ExplainQuery(PtrTo<Query> query) : Query{query}
|
||||
// static
|
||||
PtrTo<ExplainQuery> ExplainQuery::createExplainAST(PtrTo<Query> query)
|
||||
{
|
||||
return PtrTo<ExplainQuery>(new ExplainQuery(QueryType::AST, {query}));
|
||||
}
|
||||
|
||||
// static
|
||||
PtrTo<ExplainQuery> ExplainQuery::createExplainSyntax(PtrTo<Query> query)
|
||||
{
|
||||
return PtrTo<ExplainQuery>(new ExplainQuery(QueryType::SYNTAX, {query}));
|
||||
}
|
||||
|
||||
ExplainQuery::ExplainQuery(QueryType type, PtrList exprs) : Query{exprs}, query_type(type)
|
||||
{
|
||||
}
|
||||
|
||||
ASTPtr ExplainQuery::convertToOld() const
|
||||
{
|
||||
auto query = std::make_shared<ASTExplainQuery>(ASTExplainQuery::AnalyzedSyntax);
|
||||
ASTPtr query;
|
||||
|
||||
query->setExplainedQuery(get(QUERY)->convertToOld());
|
||||
switch (query_type)
|
||||
{
|
||||
case QueryType::AST:
|
||||
query = std::make_shared<ASTExplainQuery>(ASTExplainQuery::ParsedAST);
|
||||
break;
|
||||
case QueryType::SYNTAX:
|
||||
query = std::make_shared<ASTExplainQuery>(ASTExplainQuery::AnalyzedSyntax);
|
||||
break;
|
||||
}
|
||||
|
||||
query->as<ASTExplainQuery>()->setExplainedQuery(get(QUERY)->convertToOld());
|
||||
|
||||
return query;
|
||||
}
|
||||
@ -27,9 +49,14 @@ namespace DB
|
||||
|
||||
using namespace DB::AST;
|
||||
|
||||
antlrcpp::Any ParseTreeVisitor::visitExplainStmt(ClickHouseParser::ExplainStmtContext *ctx)
|
||||
antlrcpp::Any ParseTreeVisitor::visitExplainASTStmt(ClickHouseParser::ExplainASTStmtContext *ctx)
|
||||
{
|
||||
return std::make_shared<ExplainQuery>(visit(ctx->query()).as<PtrTo<Query>>());
|
||||
return ExplainQuery::createExplainAST(visit(ctx->query()).as<PtrTo<Query>>());
|
||||
}
|
||||
|
||||
antlrcpp::Any ParseTreeVisitor::visitExplainSyntaxStmt(ClickHouseParser::ExplainSyntaxStmtContext *ctx)
|
||||
{
|
||||
return ExplainQuery::createExplainSyntax(visit(ctx->query()).as<PtrTo<Query>>());
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -9,7 +9,8 @@ namespace DB::AST
|
||||
class ExplainQuery : public Query
|
||||
{
|
||||
public:
|
||||
explicit ExplainQuery(PtrTo<Query> query);
|
||||
static PtrTo<ExplainQuery> createExplainAST(PtrTo<Query> query);
|
||||
static PtrTo<ExplainQuery> createExplainSyntax(PtrTo<Query> query);
|
||||
|
||||
ASTPtr convertToOld() const override;
|
||||
|
||||
@ -18,6 +19,16 @@ class ExplainQuery : public Query
|
||||
{
|
||||
QUERY = 0, // Query
|
||||
};
|
||||
|
||||
enum class QueryType
|
||||
{
|
||||
AST,
|
||||
SYNTAX,
|
||||
};
|
||||
|
||||
const QueryType query_type;
|
||||
|
||||
ExplainQuery(QueryType type, PtrList exprs);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -58,12 +58,8 @@ void TableIdentifier::makeCompound() const
|
||||
|
||||
ASTPtr TableIdentifier::convertToOld() const
|
||||
{
|
||||
std::vector<String> parts;
|
||||
|
||||
if (db && !db->getName().empty()) parts.push_back(db->getName());
|
||||
parts.push_back(getName());
|
||||
|
||||
return std::make_shared<ASTIdentifier>(std::move(parts));
|
||||
if (db) return std::make_shared<ASTTableIdentifier>(db->getName(), getName());
|
||||
else return std::make_shared<ASTTableIdentifier>(getName());
|
||||
}
|
||||
|
||||
ColumnIdentifier::ColumnIdentifier(PtrTo<TableIdentifier> table_, PtrTo<Identifier> name) : Identifier(name->getName()), table(table_)
|
||||
|
@ -70,7 +70,7 @@ ASTPtr InsertQuery::convertToOld() const
|
||||
query->table_function = get(FUNCTION)->convertToOld();
|
||||
break;
|
||||
case QueryType::TABLE:
|
||||
query->table_id = getTableIdentifier(get(IDENTIFIER)->convertToOld());
|
||||
query->table_id = get(IDENTIFIER)->convertToOld()->as<ASTTableIdentifier>()->getTableId();
|
||||
break;
|
||||
}
|
||||
|
||||
|
@ -23,10 +23,10 @@ ASTPtr OptimizeQuery::convertToOld() const
|
||||
auto query = std::make_shared<ASTOptimizeQuery>();
|
||||
|
||||
{
|
||||
auto table_id = getTableIdentifier(get(TABLE)->convertToOld());
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
query->uuid = table_id.uuid;
|
||||
auto table = std::static_pointer_cast<ASTTableIdentifier>(get(TABLE)->convertToOld());
|
||||
query->database = table->getDatabaseName();
|
||||
query->table = table->shortName();
|
||||
query->uuid = table->uuid;
|
||||
}
|
||||
|
||||
if (has(PARTITION))
|
||||
|
@ -47,22 +47,22 @@ ASTPtr ShowCreateQuery::convertToOld() const
|
||||
case QueryType::DICTIONARY:
|
||||
{
|
||||
auto query = std::make_shared<ASTShowCreateDictionaryQuery>();
|
||||
auto table_id = getTableIdentifier(get(IDENTIFIER)->convertToOld());
|
||||
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
query->uuid = table_id.uuid;
|
||||
auto table = std::static_pointer_cast<ASTTableIdentifier>(get(IDENTIFIER)->convertToOld());
|
||||
query->database = table->getDatabaseName();
|
||||
query->table = table->shortName();
|
||||
query->uuid = table->uuid;
|
||||
|
||||
return query;
|
||||
}
|
||||
case QueryType::TABLE:
|
||||
{
|
||||
auto query = std::make_shared<ASTShowCreateTableQuery>();
|
||||
auto table_id = getTableIdentifier(get(IDENTIFIER)->convertToOld());
|
||||
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
query->uuid = table_id.uuid;
|
||||
auto table = std::static_pointer_cast<ASTTableIdentifier>(get(IDENTIFIER)->convertToOld());
|
||||
query->database = table->getDatabaseName();
|
||||
query->table = table->shortName();
|
||||
query->uuid = table->uuid;
|
||||
query->temporary = temporary;
|
||||
|
||||
return query;
|
||||
|
@ -93,25 +93,25 @@ ASTPtr SystemQuery::convertToOld() const
|
||||
case QueryType::DISTRIBUTED_SENDS:
|
||||
query->type = stop ? ASTSystemQuery::Type::STOP_DISTRIBUTED_SENDS : ASTSystemQuery::Type::START_DISTRIBUTED_SENDS;
|
||||
{
|
||||
auto table_id = getTableIdentifier(get(TABLE)->convertToOld());
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
auto table = std::static_pointer_cast<ASTTableIdentifier>(get(TABLE)->convertToOld());
|
||||
query->database = table->getDatabaseName();
|
||||
query->table = table->shortName();
|
||||
}
|
||||
break;
|
||||
case QueryType::FETCHES:
|
||||
query->type = stop ? ASTSystemQuery::Type::STOP_FETCHES : ASTSystemQuery::Type::START_FETCHES;
|
||||
{
|
||||
auto table_id = getTableIdentifier(get(TABLE)->convertToOld());
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
auto table = std::static_pointer_cast<ASTTableIdentifier>(get(TABLE)->convertToOld());
|
||||
query->database = table->getDatabaseName();
|
||||
query->table = table->shortName();
|
||||
}
|
||||
break;
|
||||
case QueryType::FLUSH_DISTRIBUTED:
|
||||
query->type = ASTSystemQuery::Type::FLUSH_DISTRIBUTED;
|
||||
{
|
||||
auto table_id = getTableIdentifier(get(TABLE)->convertToOld());
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
auto table = std::static_pointer_cast<ASTTableIdentifier>(get(TABLE)->convertToOld());
|
||||
query->database = table->getDatabaseName();
|
||||
query->table = table->shortName();
|
||||
}
|
||||
break;
|
||||
case QueryType::FLUSH_LOGS:
|
||||
@ -120,9 +120,9 @@ ASTPtr SystemQuery::convertToOld() const
|
||||
case QueryType::MERGES:
|
||||
query->type = stop ? ASTSystemQuery::Type::STOP_MERGES : ASTSystemQuery::Type::START_MERGES;
|
||||
{
|
||||
auto table_id = getTableIdentifier(get(TABLE)->convertToOld());
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
auto table = std::static_pointer_cast<ASTTableIdentifier>(get(TABLE)->convertToOld());
|
||||
query->database = table->getDatabaseName();
|
||||
query->table = table->shortName();
|
||||
}
|
||||
break;
|
||||
case QueryType::RELOAD_DICTIONARIES:
|
||||
@ -131,9 +131,9 @@ ASTPtr SystemQuery::convertToOld() const
|
||||
case QueryType::RELOAD_DICTIONARY:
|
||||
query->type = ASTSystemQuery::Type::RELOAD_DICTIONARY;
|
||||
{
|
||||
auto table_id = getTableIdentifier(get(TABLE)->convertToOld());
|
||||
query->database = table_id.database_name;
|
||||
query->target_dictionary = table_id.table_name;
|
||||
auto table = std::static_pointer_cast<ASTTableIdentifier>(get(TABLE)->convertToOld());
|
||||
query->database = table->getDatabaseName();
|
||||
query->target_dictionary = table->shortName();
|
||||
}
|
||||
break;
|
||||
case QueryType::REPLICATED_SENDS:
|
||||
@ -142,17 +142,17 @@ ASTPtr SystemQuery::convertToOld() const
|
||||
case QueryType::SYNC_REPLICA:
|
||||
query->type = ASTSystemQuery::Type::SYNC_REPLICA;
|
||||
{
|
||||
auto table_id = getTableIdentifier(get(TABLE)->convertToOld());
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
auto table = std::static_pointer_cast<ASTTableIdentifier>(get(TABLE)->convertToOld());
|
||||
query->database = table->getDatabaseName();
|
||||
query->table = table->shortName();
|
||||
}
|
||||
break;
|
||||
case QueryType::TTL_MERGES:
|
||||
query->type = stop ? ASTSystemQuery::Type::STOP_TTL_MERGES : ASTSystemQuery::Type::START_TTL_MERGES;
|
||||
{
|
||||
auto table_id = getTableIdentifier(get(TABLE)->convertToOld());
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
auto table = std::static_pointer_cast<ASTTableIdentifier>(get(TABLE)->convertToOld());
|
||||
query->database = table->getDatabaseName();
|
||||
query->table = table->shortName();
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
@ -21,7 +21,7 @@ TableArgExpr::TableArgExpr(PtrTo<TableFunctionExpr> function) : INode{function}
|
||||
{
|
||||
}
|
||||
|
||||
TableArgExpr::TableArgExpr(PtrTo<TableIdentifier> identifier) : INode{identifier}
|
||||
TableArgExpr::TableArgExpr(PtrTo<Identifier> identifier) : INode{identifier}
|
||||
{
|
||||
}
|
||||
|
||||
@ -149,7 +149,7 @@ antlrcpp::Any ParseTreeVisitor::visitTableArgExpr(ClickHouseParser::TableArgExpr
|
||||
{
|
||||
if (ctx->literal()) return std::make_shared<TableArgExpr>(visit(ctx->literal()).as<PtrTo<Literal>>());
|
||||
if (ctx->tableFunctionExpr()) return std::make_shared<TableArgExpr>(visit(ctx->tableFunctionExpr()).as<PtrTo<TableFunctionExpr>>());
|
||||
if (ctx->tableIdentifier()) return std::make_shared<TableArgExpr>(visit(ctx->tableIdentifier()).as<PtrTo<TableIdentifier>>());
|
||||
if (ctx->nestedIdentifier()) return std::make_shared<TableArgExpr>(visit(ctx->nestedIdentifier()).as<PtrTo<Identifier>>());
|
||||
__builtin_unreachable();
|
||||
}
|
||||
|
||||
|
@ -11,14 +11,14 @@ class TableArgExpr : public INode
|
||||
public:
|
||||
explicit TableArgExpr(PtrTo<Literal> literal);
|
||||
explicit TableArgExpr(PtrTo<TableFunctionExpr> function);
|
||||
explicit TableArgExpr(PtrTo<TableIdentifier> identifier);
|
||||
explicit TableArgExpr(PtrTo<Identifier> identifier);
|
||||
|
||||
ASTPtr convertToOld() const override;
|
||||
|
||||
private:
|
||||
enum ChildIndex : UInt8
|
||||
{
|
||||
EXPR = 0, // Literal or TableFunctionExpr or TableIdentifier
|
||||
EXPR = 0, // Literal or TableFunctionExpr or Identifier
|
||||
};
|
||||
};
|
||||
|
||||
|
@ -20,10 +20,10 @@ ASTPtr WatchQuery::convertToOld() const
|
||||
{
|
||||
auto query = std::make_shared<ASTWatchQuery>();
|
||||
|
||||
auto table_id = getTableIdentifier(get(TABLE)->convertToOld());
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
query->uuid = table_id.uuid;
|
||||
auto table = std::static_pointer_cast<ASTTableIdentifier>(get(TABLE)->convertToOld());
|
||||
query->database = table->getDatabaseName();
|
||||
query->table = table->shortName();
|
||||
query->uuid = table->uuid;
|
||||
|
||||
query->is_watch_events = events;
|
||||
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -16,6 +16,7 @@ ARRAY: A R R A Y;
|
||||
AS: A S;
|
||||
ASCENDING: A S C | A S C E N D I N G;
|
||||
ASOF: A S O F;
|
||||
AST: A S T;
|
||||
ASYNC: A S Y N C;
|
||||
ATTACH: A T T A C H;
|
||||
BETWEEN: B E T W E E N;
|
||||
|
@ -14,46 +14,46 @@ class ClickHouseLexer : public antlr4::Lexer {
|
||||
public:
|
||||
enum {
|
||||
ADD = 1, AFTER = 2, ALIAS = 3, ALL = 4, ALTER = 5, AND = 6, ANTI = 7,
|
||||
ANY = 8, ARRAY = 9, AS = 10, ASCENDING = 11, ASOF = 12, ASYNC = 13,
|
||||
ATTACH = 14, BETWEEN = 15, BOTH = 16, BY = 17, CASE = 18, CAST = 19,
|
||||
CHECK = 20, CLEAR = 21, CLUSTER = 22, CODEC = 23, COLLATE = 24, COLUMN = 25,
|
||||
COMMENT = 26, CONSTRAINT = 27, CREATE = 28, CROSS = 29, CUBE = 30, DATABASE = 31,
|
||||
DATABASES = 32, DATE = 33, DAY = 34, DEDUPLICATE = 35, DEFAULT = 36,
|
||||
DELAY = 37, DELETE = 38, DESC = 39, DESCENDING = 40, DESCRIBE = 41,
|
||||
DETACH = 42, DICTIONARIES = 43, DICTIONARY = 44, DISK = 45, DISTINCT = 46,
|
||||
DISTRIBUTED = 47, DROP = 48, ELSE = 49, END = 50, ENGINE = 51, EVENTS = 52,
|
||||
EXISTS = 53, EXPLAIN = 54, EXPRESSION = 55, EXTRACT = 56, FETCHES = 57,
|
||||
FINAL = 58, FIRST = 59, FLUSH = 60, FOR = 61, FORMAT = 62, FREEZE = 63,
|
||||
FROM = 64, FULL = 65, FUNCTION = 66, GLOBAL = 67, GRANULARITY = 68,
|
||||
GROUP = 69, HAVING = 70, HIERARCHICAL = 71, HOUR = 72, ID = 73, IF = 74,
|
||||
ILIKE = 75, IN = 76, INDEX = 77, INF = 78, INJECTIVE = 79, INNER = 80,
|
||||
INSERT = 81, INTERVAL = 82, INTO = 83, IS = 84, IS_OBJECT_ID = 85, JOIN = 86,
|
||||
KEY = 87, KILL = 88, LAST = 89, LAYOUT = 90, LEADING = 91, LEFT = 92,
|
||||
LIFETIME = 93, LIKE = 94, LIMIT = 95, LIVE = 96, LOCAL = 97, LOGS = 98,
|
||||
MATERIALIZED = 99, MATERIALIZE = 100, MAX = 101, MERGES = 102, MIN = 103,
|
||||
MINUTE = 104, MODIFY = 105, MONTH = 106, MOVE = 107, MUTATION = 108,
|
||||
NAN_SQL = 109, NO = 110, NOT = 111, NULL_SQL = 112, NULLS = 113, OFFSET = 114,
|
||||
ON = 115, OPTIMIZE = 116, OR = 117, ORDER = 118, OUTER = 119, OUTFILE = 120,
|
||||
PARTITION = 121, POPULATE = 122, PREWHERE = 123, PRIMARY = 124, PROJECTION = 125,
|
||||
QUARTER = 126, RANGE = 127, RELOAD = 128, REMOVE = 129, RENAME = 130,
|
||||
REPLACE = 131, REPLICA = 132, REPLICATED = 133, RIGHT = 134, ROLLUP = 135,
|
||||
SAMPLE = 136, SECOND = 137, SELECT = 138, SEMI = 139, SENDS = 140, SET = 141,
|
||||
SETTINGS = 142, SHOW = 143, SOURCE = 144, START = 145, STOP = 146, SUBSTRING = 147,
|
||||
SYNC = 148, SYNTAX = 149, SYSTEM = 150, TABLE = 151, TABLES = 152, TEMPORARY = 153,
|
||||
TEST = 154, THEN = 155, TIES = 156, TIMEOUT = 157, TIMESTAMP = 158,
|
||||
TO = 159, TOP = 160, TOTALS = 161, TRAILING = 162, TRIM = 163, TRUNCATE = 164,
|
||||
TTL = 165, TYPE = 166, UNION = 167, UPDATE = 168, USE = 169, USING = 170,
|
||||
UUID = 171, VALUES = 172, VIEW = 173, VOLUME = 174, WATCH = 175, WEEK = 176,
|
||||
WHEN = 177, WHERE = 178, WITH = 179, YEAR = 180, JSON_FALSE = 181, JSON_TRUE = 182,
|
||||
IDENTIFIER = 183, FLOATING_LITERAL = 184, OCTAL_LITERAL = 185, DECIMAL_LITERAL = 186,
|
||||
HEXADECIMAL_LITERAL = 187, STRING_LITERAL = 188, ARROW = 189, ASTERISK = 190,
|
||||
BACKQUOTE = 191, BACKSLASH = 192, COLON = 193, COMMA = 194, CONCAT = 195,
|
||||
DASH = 196, DOT = 197, EQ_DOUBLE = 198, EQ_SINGLE = 199, GE = 200, GT = 201,
|
||||
LBRACE = 202, LBRACKET = 203, LE = 204, LPAREN = 205, LT = 206, NOT_EQ = 207,
|
||||
PERCENT = 208, PLUS = 209, QUERY = 210, QUOTE_DOUBLE = 211, QUOTE_SINGLE = 212,
|
||||
RBRACE = 213, RBRACKET = 214, RPAREN = 215, SEMICOLON = 216, SLASH = 217,
|
||||
UNDERSCORE = 218, MULTI_LINE_COMMENT = 219, SINGLE_LINE_COMMENT = 220,
|
||||
WHITESPACE = 221
|
||||
ANY = 8, ARRAY = 9, AS = 10, ASCENDING = 11, ASOF = 12, AST = 13, ASYNC = 14,
|
||||
ATTACH = 15, BETWEEN = 16, BOTH = 17, BY = 18, CASE = 19, CAST = 20,
|
||||
CHECK = 21, CLEAR = 22, CLUSTER = 23, CODEC = 24, COLLATE = 25, COLUMN = 26,
|
||||
COMMENT = 27, CONSTRAINT = 28, CREATE = 29, CROSS = 30, CUBE = 31, DATABASE = 32,
|
||||
DATABASES = 33, DATE = 34, DAY = 35, DEDUPLICATE = 36, DEFAULT = 37,
|
||||
DELAY = 38, DELETE = 39, DESC = 40, DESCENDING = 41, DESCRIBE = 42,
|
||||
DETACH = 43, DICTIONARIES = 44, DICTIONARY = 45, DISK = 46, DISTINCT = 47,
|
||||
DISTRIBUTED = 48, DROP = 49, ELSE = 50, END = 51, ENGINE = 52, EVENTS = 53,
|
||||
EXISTS = 54, EXPLAIN = 55, EXPRESSION = 56, EXTRACT = 57, FETCHES = 58,
|
||||
FINAL = 59, FIRST = 60, FLUSH = 61, FOR = 62, FORMAT = 63, FREEZE = 64,
|
||||
FROM = 65, FULL = 66, FUNCTION = 67, GLOBAL = 68, GRANULARITY = 69,
|
||||
GROUP = 70, HAVING = 71, HIERARCHICAL = 72, HOUR = 73, ID = 74, IF = 75,
|
||||
ILIKE = 76, IN = 77, INDEX = 78, INF = 79, INJECTIVE = 80, INNER = 81,
|
||||
INSERT = 82, INTERVAL = 83, INTO = 84, IS = 85, IS_OBJECT_ID = 86, JOIN = 87,
|
||||
KEY = 88, KILL = 89, LAST = 90, LAYOUT = 91, LEADING = 92, LEFT = 93,
|
||||
LIFETIME = 94, LIKE = 95, LIMIT = 96, LIVE = 97, LOCAL = 98, LOGS = 99,
|
||||
MATERIALIZE = 100, MATERIALIZED = 101, MAX = 102, MERGES = 103, MIN = 104,
|
||||
MINUTE = 105, MODIFY = 106, MONTH = 107, MOVE = 108, MUTATION = 109,
|
||||
NAN_SQL = 110, NO = 111, NOT = 112, NULL_SQL = 113, NULLS = 114, OFFSET = 115,
|
||||
ON = 116, OPTIMIZE = 117, OR = 118, ORDER = 119, OUTER = 120, OUTFILE = 121,
|
||||
PARTITION = 122, POPULATE = 123, PREWHERE = 124, PRIMARY = 125, PROJECTION = 126,
|
||||
QUARTER = 127, RANGE = 128, RELOAD = 129, REMOVE = 130, RENAME = 131,
|
||||
REPLACE = 132, REPLICA = 133, REPLICATED = 134, RIGHT = 135, ROLLUP = 136,
|
||||
SAMPLE = 137, SECOND = 138, SELECT = 139, SEMI = 140, SENDS = 141, SET = 142,
|
||||
SETTINGS = 143, SHOW = 144, SOURCE = 145, START = 146, STOP = 147, SUBSTRING = 148,
|
||||
SYNC = 149, SYNTAX = 150, SYSTEM = 151, TABLE = 152, TABLES = 153, TEMPORARY = 154,
|
||||
TEST = 155, THEN = 156, TIES = 157, TIMEOUT = 158, TIMESTAMP = 159,
|
||||
TO = 160, TOP = 161, TOTALS = 162, TRAILING = 163, TRIM = 164, TRUNCATE = 165,
|
||||
TTL = 166, TYPE = 167, UNION = 168, UPDATE = 169, USE = 170, USING = 171,
|
||||
UUID = 172, VALUES = 173, VIEW = 174, VOLUME = 175, WATCH = 176, WEEK = 177,
|
||||
WHEN = 178, WHERE = 179, WITH = 180, YEAR = 181, JSON_FALSE = 182, JSON_TRUE = 183,
|
||||
IDENTIFIER = 184, FLOATING_LITERAL = 185, OCTAL_LITERAL = 186, DECIMAL_LITERAL = 187,
|
||||
HEXADECIMAL_LITERAL = 188, STRING_LITERAL = 189, ARROW = 190, ASTERISK = 191,
|
||||
BACKQUOTE = 192, BACKSLASH = 193, COLON = 194, COMMA = 195, CONCAT = 196,
|
||||
DASH = 197, DOT = 198, EQ_DOUBLE = 199, EQ_SINGLE = 200, GE = 201, GT = 202,
|
||||
LBRACE = 203, LBRACKET = 204, LE = 205, LPAREN = 206, LT = 207, NOT_EQ = 208,
|
||||
PERCENT = 209, PLUS = 210, QUERY = 211, QUOTE_DOUBLE = 212, QUOTE_SINGLE = 213,
|
||||
RBRACE = 214, RBRACKET = 215, RPAREN = 216, SEMICOLON = 217, SLASH = 218,
|
||||
UNDERSCORE = 219, MULTI_LINE_COMMENT = 220, SINGLE_LINE_COMMENT = 221,
|
||||
WHITESPACE = 222
|
||||
};
|
||||
|
||||
ClickHouseLexer(antlr4::CharStream *input);
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -193,7 +193,10 @@ existsStmt
|
||||
|
||||
// EXPLAIN statement
|
||||
|
||||
explainStmt: EXPLAIN SYNTAX query;
|
||||
explainStmt
|
||||
: EXPLAIN AST query # ExplainASTStmt
|
||||
| EXPLAIN SYNTAX query # ExplainSyntaxStmt
|
||||
;
|
||||
|
||||
// INSERT statement
|
||||
|
||||
@ -425,7 +428,7 @@ tableFunctionExpr: identifier LPAREN tableArgList? RPAREN;
|
||||
tableIdentifier: (databaseIdentifier DOT)? identifier;
|
||||
tableArgList: tableArgExpr (COMMA tableArgExpr)*;
|
||||
tableArgExpr
|
||||
: tableIdentifier
|
||||
: nestedIdentifier
|
||||
| tableFunctionExpr
|
||||
| literal
|
||||
;
|
||||
@ -450,18 +453,18 @@ literal
|
||||
interval: SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR;
|
||||
keyword
|
||||
// except NULL_SQL, INF, NAN_SQL
|
||||
: AFTER | ALIAS | ALL | ALTER | AND | ANTI | ANY | ARRAY | AS | ASCENDING | ASOF | ASYNC | ATTACH | BETWEEN | BOTH | BY | CASE | CAST
|
||||
| CHECK | CLEAR | CLUSTER | CODEC | COLLATE | COLUMN | COMMENT | CONSTRAINT | CREATE | CROSS | CUBE | DATABASE | DATABASES | DATE
|
||||
| DEDUPLICATE | DEFAULT | DELAY | DELETE | DESCRIBE | DESC | DESCENDING | DETACH | DICTIONARIES | DICTIONARY | DISK | DISTINCT
|
||||
: AFTER | ALIAS | ALL | ALTER | AND | ANTI | ANY | ARRAY | AS | ASCENDING | ASOF | AST | ASYNC | ATTACH | BETWEEN | BOTH | BY | CASE
|
||||
| CAST | CHECK | CLEAR | CLUSTER | CODEC | COLLATE | COLUMN | COMMENT | CONSTRAINT | CREATE | CROSS | CUBE | DATABASE | DATABASES
|
||||
| DATE | DEDUPLICATE | DEFAULT | DELAY | DELETE | DESCRIBE | DESC | DESCENDING | DETACH | DICTIONARIES | DICTIONARY | DISK | DISTINCT
|
||||
| DISTRIBUTED | DROP | ELSE | END | ENGINE | EVENTS | EXISTS | EXPLAIN | EXPRESSION | EXTRACT | FETCHES | FINAL | FIRST | FLUSH | FOR
|
||||
| FORMAT | FREEZE | FROM | FULL | FUNCTION | GLOBAL | GRANULARITY | GROUP | HAVING | HIERARCHICAL | ID | IF | ILIKE | IN | INDEX
|
||||
| INJECTIVE | INNER | INSERT | INTERVAL | INTO | IS | IS_OBJECT_ID | JOIN | JSON_FALSE | JSON_TRUE | KEY | KILL | LAST | LAYOUT
|
||||
| LEADING | LEFT | LIFETIME | LIKE | LIMIT | LIVE | LOCAL | LOGS | MATERIALIZE | MATERIALIZED | MAX | MERGES | MIN | MODIFY | MOVE | MUTATION | NO
|
||||
| NOT | NULLS | OFFSET | ON | OPTIMIZE | OR | ORDER | OUTER | OUTFILE | PARTITION | POPULATE | PREWHERE | PRIMARY | RANGE | RELOAD
|
||||
| REMOVE | RENAME | REPLACE | REPLICA | REPLICATED | RIGHT | ROLLUP | SAMPLE | SELECT | SEMI | SENDS | SET | SETTINGS | SHOW | SOURCE
|
||||
| START | STOP | SUBSTRING | SYNC | SYNTAX | SYSTEM | TABLE | TABLES | TEMPORARY | TEST | THEN | TIES | TIMEOUT | TIMESTAMP | TOTALS
|
||||
| TRAILING | TRIM | TRUNCATE | TO | TOP | TTL | TYPE | UNION | UPDATE | USE | USING | UUID | VALUES | VIEW | VOLUME | WATCH | WHEN
|
||||
| WHERE | WITH
|
||||
| LEADING | LEFT | LIFETIME | LIKE | LIMIT | LIVE | LOCAL | LOGS | MATERIALIZE | MATERIALIZED | MAX | MERGES | MIN | MODIFY | MOVE
|
||||
| MUTATION | NO | NOT | NULLS | OFFSET | ON | OPTIMIZE | OR | ORDER | OUTER | OUTFILE | PARTITION | POPULATE | PREWHERE | PRIMARY
|
||||
| RANGE | RELOAD | REMOVE | RENAME | REPLACE | REPLICA | REPLICATED | RIGHT | ROLLUP | SAMPLE | SELECT | SEMI | SENDS | SET | SETTINGS
|
||||
| SHOW | SOURCE | START | STOP | SUBSTRING | SYNC | SYNTAX | SYSTEM | TABLE | TABLES | TEMPORARY | TEST | THEN | TIES | TIMEOUT
|
||||
| TIMESTAMP | TOTALS | TRAILING | TRIM | TRUNCATE | TO | TOP | TTL | TYPE | UNION | UPDATE | USE | USING | UUID | VALUES | VIEW
|
||||
| VOLUME | WATCH | WHEN | WHERE | WITH
|
||||
;
|
||||
keywordForAlias
|
||||
: DATE | FIRST | ID | KEY
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -184,7 +184,9 @@ public:
|
||||
|
||||
virtual antlrcpp::Any visitExistsTableStmt(ClickHouseParser::ExistsTableStmtContext *context) = 0;
|
||||
|
||||
virtual antlrcpp::Any visitExplainStmt(ClickHouseParser::ExplainStmtContext *context) = 0;
|
||||
virtual antlrcpp::Any visitExplainASTStmt(ClickHouseParser::ExplainASTStmtContext *context) = 0;
|
||||
|
||||
virtual antlrcpp::Any visitExplainSyntaxStmt(ClickHouseParser::ExplainSyntaxStmtContext *context) = 0;
|
||||
|
||||
virtual antlrcpp::Any visitInsertStmt(ClickHouseParser::InsertStmtContext *context) = 0;
|
||||
|
||||
|
@ -146,7 +146,8 @@ public:
|
||||
antlrcpp::Any visitExistsDatabaseStmt(ClickHouseParser::ExistsDatabaseStmtContext * ctx) override;
|
||||
|
||||
// ExplainQuery
|
||||
antlrcpp::Any visitExplainStmt(ClickHouseParser::ExplainStmtContext * ctx) override;
|
||||
antlrcpp::Any visitExplainASTStmt(ClickHouseParser::ExplainASTStmtContext * ctx) override;
|
||||
antlrcpp::Any visitExplainSyntaxStmt(ClickHouseParser::ExplainSyntaxStmtContext * ctx) override;
|
||||
|
||||
// Identifier
|
||||
antlrcpp::Any visitTableIdentifier(ClickHouseParser::TableIdentifierContext * ctx) override;
|
||||
|
@ -492,7 +492,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
||||
return false;
|
||||
}
|
||||
|
||||
StorageID table_id = getTableIdentifier(table);
|
||||
auto table_id = table->as<ASTTableIdentifier>()->getTableId();
|
||||
|
||||
// Shortcut for ATTACH a previously detached table
|
||||
bool short_attach = attach && !from_path;
|
||||
@ -729,14 +729,14 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
|
||||
query->if_not_exists = if_not_exists;
|
||||
query->is_live_view = true;
|
||||
|
||||
StorageID table_id = getTableIdentifier(table);
|
||||
auto table_id = table->as<ASTTableIdentifier>()->getTableId();
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
query->uuid = table_id.uuid;
|
||||
query->cluster = cluster_str;
|
||||
|
||||
if (to_table)
|
||||
query->to_table_id = getTableIdentifier(to_table);
|
||||
query->to_table_id = to_table->as<ASTTableIdentifier>()->getTableId();
|
||||
|
||||
query->set(query->columns_list, columns_list);
|
||||
|
||||
@ -945,14 +945,14 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
query->is_populate = is_populate;
|
||||
query->replace_view = replace_view;
|
||||
|
||||
StorageID table_id = getTableIdentifier(table);
|
||||
auto table_id = table->as<ASTTableIdentifier>()->getTableId();
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
query->uuid = table_id.uuid;
|
||||
query->cluster = cluster_str;
|
||||
|
||||
if (to_table)
|
||||
query->to_table_id = getTableIdentifier(to_table);
|
||||
query->to_table_id = to_table->as<ASTTableIdentifier>()->getTableId();
|
||||
if (to_inner_uuid)
|
||||
query->to_inner_uuid = parseFromString<UUID>(to_inner_uuid->as<ASTLiteral>()->value.get<String>());
|
||||
|
||||
@ -1032,7 +1032,7 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E
|
||||
query->is_dictionary = true;
|
||||
query->attach = attach;
|
||||
|
||||
StorageID dict_id = getTableIdentifier(name);
|
||||
auto dict_id = name->as<ASTTableIdentifier>()->getTableId();
|
||||
query->database = dict_id.database_name;
|
||||
query->table = dict_id.table_name;
|
||||
query->uuid = dict_id.uuid;
|
||||
|
@ -23,7 +23,8 @@ bool ParserTableExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
|
||||
if (!ParserWithOptionalAlias(std::make_unique<ParserSubquery>(), true).parse(pos, res->subquery, expected)
|
||||
&& !ParserWithOptionalAlias(std::make_unique<ParserFunction>(true, true), true).parse(pos, res->table_function, expected)
|
||||
&& !ParserWithOptionalAlias(std::make_unique<ParserCompoundIdentifier>(false, true), true).parse(pos, res->database_and_table_name, expected))
|
||||
&& !ParserWithOptionalAlias(std::make_unique<ParserCompoundIdentifier>(true, true), true)
|
||||
.parse(pos, res->database_and_table_name, expected))
|
||||
return false;
|
||||
|
||||
/// FINAL
|
||||
@ -57,6 +58,8 @@ bool ParserTableExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
if (res->sample_offset)
|
||||
res->children.emplace_back(res->sample_offset);
|
||||
|
||||
assert(res->database_and_table_name || res->table_function || res->subquery);
|
||||
|
||||
node = res;
|
||||
return true;
|
||||
}
|
||||
|
@ -883,7 +883,7 @@ bool KeyCondition::tryPrepareSetIndex(
|
||||
const ASTPtr & right_arg = args[1];
|
||||
|
||||
SetPtr prepared_set;
|
||||
if (right_arg->as<ASTSubquery>() || right_arg->as<ASTIdentifier>())
|
||||
if (right_arg->as<ASTSubquery>() || right_arg->as<ASTTableIdentifier>())
|
||||
{
|
||||
auto set_it = prepared_sets.find(PreparedSetKey::forSubquery(*right_arg));
|
||||
if (set_it == prepared_sets.end())
|
||||
|
@ -129,7 +129,7 @@ void StorageView::replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_
|
||||
{
|
||||
// If it's a view table function, add a fake db.table name.
|
||||
if (table_expression->table_function && table_expression->table_function->as<ASTFunction>()->name == "view")
|
||||
table_expression->database_and_table_name = std::make_shared<ASTIdentifier>("__view");
|
||||
table_expression->database_and_table_name = std::make_shared<ASTTableIdentifier>("__view");
|
||||
else
|
||||
throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
@ -86,7 +86,7 @@ void buildSets(const ASTPtr & expression, ExpressionAnalyzer & analyzer)
|
||||
{
|
||||
const IAST & args = *func->arguments;
|
||||
const ASTPtr & arg = args.children.at(1);
|
||||
if (arg->as<ASTSubquery>() || arg->as<ASTIdentifier>())
|
||||
if (arg->as<ASTSubquery>() || arg->as<ASTTableIdentifier>())
|
||||
{
|
||||
analyzer.tryMakeSetForIndexFromSubquery(arg);
|
||||
}
|
||||
|
@ -176,7 +176,7 @@ TEST(TransformQueryForExternalDatabase, MultipleAndSubqueries)
|
||||
R"(SELECT "column" FROM "test"."table" WHERE 1 AND ("column" = 42) AND ("column" IN (1, 42)) AND ("column" != 4))");
|
||||
check(state, 1,
|
||||
"SELECT column FROM test.table WHERE toString(column) = '42' AND left(column, 10) = RIGHT(column, 10) AND column = 42",
|
||||
R"(SELECT "column" FROM "test"."table" WHERE ("column" = 42))");
|
||||
R"(SELECT "column" FROM "test"."table" WHERE "column" = 42)");
|
||||
}
|
||||
|
||||
TEST(TransformQueryForExternalDatabase, Issue7245)
|
||||
|
@ -138,10 +138,9 @@ bool isCompatible(const IAST & node)
|
||||
if (name == "tuple" && function->arguments->children.size() <= 1)
|
||||
return false;
|
||||
|
||||
/// If the right hand side of IN is an identifier (example: x IN table), then it's not compatible.
|
||||
/// If the right hand side of IN is a table identifier (example: x IN table), then it's not compatible.
|
||||
if ((name == "in" || name == "notIn")
|
||||
&& (function->arguments->children.size() != 2
|
||||
|| function->arguments->children[1]->as<ASTIdentifier>()))
|
||||
&& (function->arguments->children.size() != 2 || function->arguments->children[1]->as<ASTTableIdentifier>()))
|
||||
return false;
|
||||
|
||||
for (const auto & expr : function->arguments->children)
|
||||
@ -274,20 +273,15 @@ String transformQueryForExternalDatabase(
|
||||
{
|
||||
if (function->name == "and")
|
||||
{
|
||||
bool compatible_found = false;
|
||||
auto new_function_and = makeASTFunction("and");
|
||||
for (const auto & elem : function->arguments->children)
|
||||
{
|
||||
if (isCompatible(*elem))
|
||||
{
|
||||
new_function_and->arguments->children.push_back(elem);
|
||||
compatible_found = true;
|
||||
}
|
||||
}
|
||||
if (new_function_and->arguments->children.size() == 1)
|
||||
new_function_and->name = "";
|
||||
|
||||
if (compatible_found)
|
||||
select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(new_function_and->arguments->children[0]));
|
||||
else if (new_function_and->arguments->children.size() > 1)
|
||||
select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(new_function_and));
|
||||
}
|
||||
}
|
||||
|
@ -31,7 +31,7 @@ echo "Binary representation:"
|
||||
hexdump -C $BINARY_FILE_PATH
|
||||
|
||||
echo
|
||||
(cd $SCHEMADIR && protoc --decode Message 00825_protobuf_format_no_length_delimiter.proto) < $BINARY_FILE_PATH
|
||||
(cd $SCHEMADIR && $PROTOC_BINARY --decode Message 00825_protobuf_format_no_length_delimiter.proto) < $BINARY_FILE_PATH
|
||||
|
||||
# Check the input in the ProtobufSingle format.
|
||||
echo
|
||||
|
@ -8,7 +8,8 @@ INSERT INTO xp SELECT '2020-01-01', number, '' FROM numbers(100000);
|
||||
|
||||
CREATE TABLE xp_d AS xp ENGINE = Distributed(test_shard_localhost, currentDatabase(), xp);
|
||||
|
||||
SELECT count(7 = (SELECT number FROM numbers(0) ORDER BY number ASC NULLS FIRST LIMIT 7)) FROM xp_d PREWHERE toYYYYMM(A) GLOBAL IN (SELECT NULL = (SELECT number FROM numbers(1) ORDER BY number DESC NULLS LAST LIMIT 1), toYYYYMM(min(A)) FROM xp_d) WHERE B > NULL; -- { serverError 8 }
|
||||
-- FIXME: this query spontaneously returns either 8 or 20 error code. Looks like it's potentially flaky.
|
||||
-- SELECT count(7 = (SELECT number FROM numbers(0) ORDER BY number ASC NULLS FIRST LIMIT 7)) FROM xp_d PREWHERE toYYYYMM(A) GLOBAL IN (SELECT NULL = (SELECT number FROM numbers(1) ORDER BY number DESC NULLS LAST LIMIT 1), toYYYYMM(min(A)) FROM xp_d) WHERE B > NULL; -- { serverError 8 }
|
||||
|
||||
SELECT count() FROM xp_d WHERE A GLOBAL IN (SELECT NULL); -- { serverError 53 }
|
||||
|
||||
|
@ -65,5 +65,4 @@ SELECT tuple(inf, inf) as key, dictGet('01760_db.dict_array', 'name', key); --{s
|
||||
DROP DICTIONARY 01760_db.dict_array;
|
||||
DROP TABLE 01760_db.points;
|
||||
DROP TABLE 01760_db.polygons;
|
||||
|
||||
DROP DATABASE 01760_db;
|
||||
|
@ -1,7 +1,7 @@
|
||||
#!/usr/bin/env python3
|
||||
|
||||
# The protobuf compiler protoc doesn't support encoding or decoding length-delimited protobuf message.
|
||||
# To do that this script has been written.
|
||||
# To do that this script has been written.
|
||||
|
||||
import argparse
|
||||
import os.path
|
||||
@ -69,7 +69,8 @@ def decode(input, output, format_schema):
|
||||
msg = input.read(sz)
|
||||
if len(msg) < sz:
|
||||
raise EOFError('Unexpected end of file')
|
||||
with subprocess.Popen(["protoc",
|
||||
protoc = os.getenv('PROTOC_BINARY', 'protoc')
|
||||
with subprocess.Popen([protoc,
|
||||
"--decode", format_schema.message_type, format_schema.schemaname],
|
||||
cwd=format_schema.schemadir,
|
||||
stdin=subprocess.PIPE,
|
||||
@ -98,7 +99,8 @@ def encode(input, output, format_schema):
|
||||
if line.startswith(b"MESSAGE #") or len(line) == 0:
|
||||
break
|
||||
msg += line
|
||||
with subprocess.Popen(["protoc",
|
||||
protoc = os.getenv('PROTOC_BINARY', 'protoc')
|
||||
with subprocess.Popen([protoc,
|
||||
"--encode", format_schema.message_type, format_schema.schemaname],
|
||||
cwd=format_schema.schemadir,
|
||||
stdin=subprocess.PIPE,
|
||||
@ -155,7 +157,7 @@ if __name__ == "__main__":
|
||||
group.add_argument('--decode_and_check', action='store_true', help='The same as --decode, and the utility will then encode '
|
||||
' the decoded data back to the binary form to check that the result of that encoding is the same as the input was.')
|
||||
args = parser.parse_args()
|
||||
|
||||
|
||||
custom_input_file = None
|
||||
custom_output_file = None
|
||||
try:
|
||||
|
@ -1,6 +1,6 @@
|
||||
import os
|
||||
import sys
|
||||
import re
|
||||
import sys
|
||||
|
||||
import pytest
|
||||
|
||||
@ -10,9 +10,8 @@ from .server import ServerThread
|
||||
# Command-line arguments
|
||||
|
||||
def pytest_addoption(parser):
|
||||
parser.addoption(
|
||||
"--builddir", action="store", default=None, help="Path to build directory to use binaries from",
|
||||
)
|
||||
parser.addoption("--builddir", action="store", default=None, help="Path to build directory to use binaries from")
|
||||
parser.addoption("--antlr", action="store_true", default=False, help="Use ANTLR parser")
|
||||
|
||||
|
||||
# HTML report hooks
|
||||
@ -34,6 +33,7 @@ def pytest_itemcollected(item):
|
||||
def cmdopts(request):
|
||||
return {
|
||||
'builddir': request.config.getoption("--builddir"),
|
||||
'antlr': request.config.getoption("--antlr"),
|
||||
}
|
||||
|
||||
|
||||
@ -48,6 +48,11 @@ def bin_prefix(cmdopts):
|
||||
return prefix
|
||||
|
||||
|
||||
@pytest.fixture(scope='module')
|
||||
def use_antlr(cmdopts):
|
||||
return cmdopts['antlr']
|
||||
|
||||
|
||||
# TODO: also support stateful queries.
|
||||
QUERIES_PATH = os.path.join(os.path.dirname(os.path.abspath(__file__)), '0_stateless')
|
||||
|
||||
|
@ -74,6 +74,8 @@ SKIP_LIST = [
|
||||
"01300_client_save_history_when_terminated", # expect-test
|
||||
"01304_direct_io",
|
||||
"01306_benchmark_json",
|
||||
"01035_lc_empty_part_bug", # FLAKY
|
||||
"01175_distributed_ddl_output_mode_long", # tcp port in reference
|
||||
"01320_create_sync_race_condition_zookeeper",
|
||||
"01355_CSV_input_format_allow_errors",
|
||||
"01370_client_autocomplete_word_break_characters", # expect-test
|
||||
@ -114,7 +116,7 @@ SKIP_LIST = [
|
||||
"01685_ssd_cache_dictionary_complex_key",
|
||||
"01737_clickhouse_server_wait_server_pool_long",
|
||||
"01746_executable_pool_dictionary",
|
||||
"01747_executable_pool_dictionary_implicit_key",
|
||||
"01747_executable_pool_dictionary_implicit_key.sql",
|
||||
"01747_join_view_filter_dictionary",
|
||||
"01748_dictionary_table_dot",
|
||||
"01754_cluster_all_replicas_shard_num",
|
||||
@ -153,17 +155,19 @@ def check_result(result, error, return_code, reference, replace_map):
|
||||
pytrace=False)
|
||||
|
||||
|
||||
def run_client(bin_prefix, port, database, query, reference, replace_map=None):
|
||||
def run_client(use_antlr, bin_prefix, port, database, query, reference, replace_map=None):
|
||||
# We can't use `text=True` since some tests may return binary data
|
||||
client = subprocess.Popen([bin_prefix + '-client', '--port', str(port), '-d', database, '-m', '-n', '--testmode'],
|
||||
stdin=subprocess.PIPE, stdout=subprocess.PIPE, stderr=subprocess.PIPE)
|
||||
cmd = [bin_prefix + '-client', '--port', str(port), '-d', database, '-m', '-n', '--testmode']
|
||||
if use_antlr:
|
||||
cmd.append('--use_antlr_parser=1')
|
||||
client = subprocess.Popen(cmd, stdin=subprocess.PIPE, stdout=subprocess.PIPE, stderr=subprocess.PIPE)
|
||||
result, error = client.communicate(query.encode('utf-8'))
|
||||
assert client.returncode is not None, "Client should exit after processing all queries"
|
||||
|
||||
check_result(result, error, client.returncode, reference, replace_map)
|
||||
|
||||
|
||||
def run_shell(bin_prefix, server, database, path, reference, replace_map=None):
|
||||
def run_shell(use_antlr, bin_prefix, server, database, path, reference, replace_map=None):
|
||||
env = {
|
||||
'CLICKHOUSE_BINARY': bin_prefix,
|
||||
'CLICKHOUSE_DATABASE': database,
|
||||
@ -174,8 +178,11 @@ def run_shell(bin_prefix, server, database, path, reference, replace_map=None):
|
||||
'CLICKHOUSE_PORT_INTERSERVER': str(server.inter_port),
|
||||
'CLICKHOUSE_PORT_POSTGRESQL': str(server.postgresql_port),
|
||||
'CLICKHOUSE_TMP': server.tmp_dir,
|
||||
'CLICKHOUSE_CONFIG_CLIENT': server.client_config
|
||||
'CLICKHOUSE_CONFIG_CLIENT': server.client_config,
|
||||
'PROTOC_BINARY': os.path.abspath(os.path.join(os.path.dirname(bin_prefix), '..', 'contrib', 'protobuf', 'protoc')), # FIXME: adhoc solution
|
||||
}
|
||||
if use_antlr:
|
||||
env['CLICKHOUSE_CLIENT_OPT'] = '--use_antlr_parser=1'
|
||||
shell = subprocess.Popen([path], env=env, shell=True, stdout=subprocess.PIPE, stderr=subprocess.PIPE)
|
||||
result, error = shell.communicate()
|
||||
assert shell.returncode is not None, "Script should exit after executing all commands"
|
||||
@ -189,7 +196,7 @@ def random_str(length=10):
|
||||
return ''.join(random.choice(alphabet) for _ in range(length))
|
||||
|
||||
|
||||
def test_sql_query(bin_prefix, sql_query, standalone_server):
|
||||
def test_sql_query(use_antlr, bin_prefix, sql_query, standalone_server):
|
||||
for test in SKIP_LIST:
|
||||
if test in sql_query:
|
||||
pytest.skip("Test matches skip-list: " + test)
|
||||
@ -209,21 +216,21 @@ def test_sql_query(bin_prefix, sql_query, standalone_server):
|
||||
reference = file.read()
|
||||
|
||||
random_name = 'test_{random}'.format(random=random_str())
|
||||
run_client(bin_prefix, tcp_port, 'default', 'CREATE DATABASE {random};'.format(random=random_name), b'')
|
||||
run_client(use_antlr, bin_prefix, tcp_port, 'default', 'CREATE DATABASE {random};'.format(random=random_name), b'')
|
||||
|
||||
run_client(bin_prefix, tcp_port, random_name, query, reference, {random_name: 'default'})
|
||||
run_client(use_antlr, bin_prefix, tcp_port, random_name, query, reference, {random_name: 'default'})
|
||||
|
||||
query = "SELECT 'SHOW ORPHANED TABLES'; SELECT name FROM system.tables WHERE database != 'system' ORDER BY (database, name);"
|
||||
run_client(bin_prefix, tcp_port, 'default', query, b'SHOW ORPHANED TABLES\n')
|
||||
run_client(use_antlr, bin_prefix, tcp_port, 'default', query, b'SHOW ORPHANED TABLES\n')
|
||||
|
||||
query = 'DROP DATABASE {random};'.format(random=random_name)
|
||||
run_client(bin_prefix, tcp_port, 'default', query, b'')
|
||||
run_client(use_antlr, bin_prefix, tcp_port, 'default', query, b'')
|
||||
|
||||
query = "SELECT 'SHOW ORPHANED DATABASES'; SHOW DATABASES;"
|
||||
run_client(bin_prefix, tcp_port, 'default', query, b'SHOW ORPHANED DATABASES\ndefault\nsystem\n')
|
||||
run_client(use_antlr, bin_prefix, tcp_port, 'default', query, b'SHOW ORPHANED DATABASES\ndefault\nsystem\n')
|
||||
|
||||
|
||||
def test_shell_query(bin_prefix, shell_query, standalone_server):
|
||||
def test_shell_query(use_antlr, bin_prefix, shell_query, standalone_server):
|
||||
for test in SKIP_LIST:
|
||||
if test in shell_query:
|
||||
pytest.skip("Test matches skip-list: " + test)
|
||||
@ -242,15 +249,15 @@ def test_shell_query(bin_prefix, shell_query, standalone_server):
|
||||
|
||||
random_name = 'test_{random}'.format(random=random_str())
|
||||
query = 'CREATE DATABASE {random};'.format(random=random_name)
|
||||
run_client(bin_prefix, tcp_port, 'default', query, b'')
|
||||
run_client(use_antlr, bin_prefix, tcp_port, 'default', query, b'')
|
||||
|
||||
run_shell(bin_prefix, standalone_server, random_name, shell_path, reference, {random_name: 'default'})
|
||||
run_shell(use_antlr, bin_prefix, standalone_server, random_name, shell_path, reference, {random_name: 'default'})
|
||||
|
||||
query = "SELECT 'SHOW ORPHANED TABLES'; SELECT name FROM system.tables WHERE database != 'system' ORDER BY (database, name);"
|
||||
run_client(bin_prefix, tcp_port, 'default', query, b'SHOW ORPHANED TABLES\n')
|
||||
run_client(use_antlr, bin_prefix, tcp_port, 'default', query, b'SHOW ORPHANED TABLES\n')
|
||||
|
||||
query = 'DROP DATABASE {random};'.format(random=random_name)
|
||||
run_client(bin_prefix, tcp_port, 'default', query, b'')
|
||||
run_client(use_antlr, bin_prefix, tcp_port, 'default', query, b'')
|
||||
|
||||
query = "SELECT 'SHOW ORPHANED DATABASES'; SHOW DATABASES;"
|
||||
run_client(bin_prefix, tcp_port, 'default', query, b'SHOW ORPHANED DATABASES\ndefault\nsystem\n')
|
||||
run_client(use_antlr, bin_prefix, tcp_port, 'default', query, b'SHOW ORPHANED DATABASES\ndefault\nsystem\n')
|
||||
|
@ -292,20 +292,33 @@ ServerThread.DEFAULT_SERVER_CONFIG = \
|
||||
</shard>
|
||||
</test_cluster_with_incorrect_pw>
|
||||
|
||||
<test_cluster_two_replicas_different_databases>
|
||||
<shard>
|
||||
<replica>
|
||||
<default_database>shard_0</default_database>
|
||||
<host>localhost</host>
|
||||
<port>{tcp_port}</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<default_database>shard_1</default_database>
|
||||
<host>localhost</host>
|
||||
<port>{tcp_port}</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_cluster_two_replicas_different_databases>
|
||||
<test_cluster_one_shard_two_replicas>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>127.0.0.1</host>
|
||||
<port>{tcp_port}</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>127.0.0.2</host>
|
||||
<port>{tcp_port}</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_cluster_one_shard_two_replicas>
|
||||
|
||||
<test_cluster_two_replicas_different_databases>
|
||||
<shard>
|
||||
<replica>
|
||||
<default_database>shard_0</default_database>
|
||||
<host>localhost</host>
|
||||
<port>{tcp_port}</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<default_database>shard_1</default_database>
|
||||
<host>localhost</host>
|
||||
<port>{tcp_port}</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_cluster_two_replicas_different_databases>
|
||||
</remote_servers>
|
||||
|
||||
<storage_configuration>
|
||||
|
@ -116,6 +116,8 @@ MYSQL_CLIENT_OPT0+=" --user ${MYSQL_CLIENT_CLICKHOUSE_USER} "
|
||||
export MYSQL_CLIENT_OPT="${MYSQL_CLIENT_OPT0:-} ${MYSQL_CLIENT_OPT:-}"
|
||||
export MYSQL_CLIENT=${MYSQL_CLIENT:="$MYSQL_CLIENT_BINARY ${MYSQL_CLIENT_OPT:-}"}
|
||||
|
||||
export PROTOC_BINARY=${PROTOC_BINARY:="protoc"}
|
||||
|
||||
function clickhouse_client_removed_host_parameter()
|
||||
{
|
||||
# removing only `--host=value` and `--host value` (removing '-hvalue' feels to dangerous) with python regex.
|
||||
|
Loading…
Reference in New Issue
Block a user