mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Rewrite CROSS/COMMA to INNER JOIN using table's columns knowledge (#9512)
use column names in CrossToInnerJoinVisitor
This commit is contained in:
parent
3b770d8b1b
commit
3ed270dcb7
@ -4,8 +4,10 @@
|
||||
#include <Interpreters/CrossToInnerJoinVisitor.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Interpreters/QueryAliasesVisitor.h>
|
||||
#include <Interpreters/misc.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
@ -27,41 +29,26 @@ namespace ErrorCodes
|
||||
namespace
|
||||
{
|
||||
|
||||
struct JoinedTable
|
||||
struct JoinedElement
|
||||
{
|
||||
DatabaseAndTableWithAlias table;
|
||||
ASTTablesInSelectQueryElement * element = nullptr;
|
||||
ASTTableJoin * join = nullptr;
|
||||
ASTPtr array_join = nullptr;
|
||||
bool has_using = false;
|
||||
|
||||
JoinedTable(ASTPtr table_element)
|
||||
JoinedElement(const ASTTablesInSelectQueryElement & table_element)
|
||||
: element(table_element)
|
||||
{
|
||||
element = table_element->as<ASTTablesInSelectQueryElement>();
|
||||
if (!element)
|
||||
throw Exception("Logical error: TablesInSelectQueryElement expected", ErrorCodes::LOGICAL_ERROR);
|
||||
if (element.table_join)
|
||||
join = element.table_join->as<ASTTableJoin>();
|
||||
}
|
||||
|
||||
if (element->table_join)
|
||||
{
|
||||
join = element->table_join->as<ASTTableJoin>();
|
||||
if (join->kind == ASTTableJoin::Kind::Cross ||
|
||||
join->kind == ASTTableJoin::Kind::Comma)
|
||||
{
|
||||
if (!join->children.empty())
|
||||
throw Exception("Logical error: CROSS JOIN has expressions", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
void checkTableName(const DatabaseAndTableWithAlias & table, const String & current_database) const
|
||||
{
|
||||
if (!element.table_expression)
|
||||
throw Exception("Not a table expression in JOIN (ARRAY JOIN?)", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (join->using_expression_list)
|
||||
has_using = true;
|
||||
}
|
||||
ASTTableExpression * table_expression = element.table_expression->as<ASTTableExpression>();
|
||||
if (!table_expression)
|
||||
throw Exception("Wrong table expression in JOIN", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (element->table_expression)
|
||||
{
|
||||
const auto & expr = element->table_expression->as<ASTTableExpression &>();
|
||||
table = DatabaseAndTableWithAlias(expr);
|
||||
}
|
||||
|
||||
array_join = element->array_join;
|
||||
if (!table.same(DatabaseAndTableWithAlias(*table_expression, current_database)))
|
||||
throw Exception("Inconsistent table names", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
void rewriteCommaToCross()
|
||||
@ -70,7 +57,24 @@ struct JoinedTable
|
||||
join->kind = ASTTableJoin::Kind::Cross;
|
||||
}
|
||||
|
||||
void rewriteCrossToInner(ASTPtr on_expression)
|
||||
{
|
||||
join->kind = ASTTableJoin::Kind::Inner;
|
||||
join->strictness = ASTTableJoin::Strictness::All;
|
||||
|
||||
join->on_expression = on_expression;
|
||||
join->children.push_back(join->on_expression);
|
||||
}
|
||||
|
||||
ASTPtr arrayJoin() const { return element.array_join; }
|
||||
const ASTTableJoin * tableJoin() const { return join; }
|
||||
|
||||
bool canAttachOnExpression() const { return join && !join->on_expression; }
|
||||
bool hasUsing() const { return join && join->using_expression_list; }
|
||||
|
||||
private:
|
||||
const ASTTablesInSelectQueryElement & element;
|
||||
ASTTableJoin * join = nullptr;
|
||||
};
|
||||
|
||||
bool isComparison(const String & name)
|
||||
@ -89,13 +93,14 @@ class CheckExpressionVisitorData
|
||||
public:
|
||||
using TypeToVisit = const ASTFunction;
|
||||
|
||||
CheckExpressionVisitorData(const std::vector<JoinedTable> & tables_)
|
||||
CheckExpressionVisitorData(const std::vector<JoinedElement> & tables_,
|
||||
const std::vector<TableWithColumnNamesAndTypes> & tables_with_columns,
|
||||
Aliases && aliases_)
|
||||
: joined_tables(tables_)
|
||||
, tables(tables_with_columns)
|
||||
, aliases(aliases_)
|
||||
, ands_only(true)
|
||||
{
|
||||
for (auto & joined : joined_tables)
|
||||
tables.push_back(joined.table);
|
||||
}
|
||||
{}
|
||||
|
||||
void visit(const ASTFunction & node, const ASTPtr & ast)
|
||||
{
|
||||
@ -160,9 +165,10 @@ public:
|
||||
}
|
||||
|
||||
private:
|
||||
const std::vector<JoinedTable> & joined_tables;
|
||||
std::vector<DatabaseAndTableWithAlias> tables;
|
||||
const std::vector<JoinedElement> & joined_tables;
|
||||
const std::vector<TableWithColumnNamesAndTypes> & tables;
|
||||
std::map<size_t, std::vector<ASTPtr>> asts_to_join_on;
|
||||
Aliases aliases;
|
||||
bool ands_only;
|
||||
|
||||
size_t canMoveEqualsToJoinOn(const ASTFunction & node)
|
||||
@ -177,6 +183,12 @@ private:
|
||||
if (!left || !right)
|
||||
return false;
|
||||
|
||||
/// Moving expressions that use column aliases is not supported.
|
||||
if (left->isShort() && aliases.count(left->shortName()))
|
||||
return false;
|
||||
if (right->isShort() && aliases.count(right->shortName()))
|
||||
return false;
|
||||
|
||||
return checkIdentifiers(*left, *right);
|
||||
}
|
||||
|
||||
@ -185,15 +197,17 @@ private:
|
||||
/// @return table position to attach expression to or 0.
|
||||
size_t checkIdentifiers(const ASTIdentifier & left, const ASTIdentifier & right)
|
||||
{
|
||||
size_t left_table_pos = 0;
|
||||
bool left_match = IdentifierSemantic::chooseTable(left, tables, left_table_pos);
|
||||
std::optional<size_t> left_table_pos = IdentifierSemantic::getMembership(left);
|
||||
if (!left_table_pos)
|
||||
left_table_pos = IdentifierSemantic::chooseTable(left, tables);
|
||||
|
||||
size_t right_table_pos = 0;
|
||||
bool right_match = IdentifierSemantic::chooseTable(right, tables, right_table_pos);
|
||||
std::optional<size_t> right_table_pos = IdentifierSemantic::getMembership(right);
|
||||
if (!right_table_pos)
|
||||
right_table_pos = IdentifierSemantic::chooseTable(right, tables);
|
||||
|
||||
if (left_match && right_match && (left_table_pos != right_table_pos))
|
||||
if (left_table_pos && right_table_pos && (*left_table_pos != *right_table_pos))
|
||||
{
|
||||
size_t table_pos = std::max(left_table_pos, right_table_pos);
|
||||
size_t table_pos = std::max(*left_table_pos, *right_table_pos);
|
||||
if (joined_tables[table_pos].canAttachOnExpression())
|
||||
return table_pos;
|
||||
}
|
||||
@ -205,7 +219,7 @@ using CheckExpressionMatcher = ConstOneTypeMatcher<CheckExpressionVisitorData, f
|
||||
using CheckExpressionVisitor = ConstInDepthNodeVisitor<CheckExpressionMatcher, true>;
|
||||
|
||||
|
||||
bool getTables(ASTSelectQuery & select, std::vector<JoinedTable> & joined_tables, size_t & num_comma)
|
||||
bool getTables(ASTSelectQuery & select, std::vector<JoinedElement> & joined_tables, size_t & num_comma)
|
||||
{
|
||||
if (!select.tables())
|
||||
return false;
|
||||
@ -224,23 +238,37 @@ bool getTables(ASTSelectQuery & select, std::vector<JoinedTable> & joined_tables
|
||||
|
||||
for (auto & child : tables->children)
|
||||
{
|
||||
joined_tables.emplace_back(JoinedTable(child));
|
||||
JoinedTable & t = joined_tables.back();
|
||||
if (t.array_join)
|
||||
auto table_element = child->as<ASTTablesInSelectQueryElement>();
|
||||
if (!table_element)
|
||||
throw Exception("Logical error: TablesInSelectQueryElement expected", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
joined_tables.emplace_back(JoinedElement(*table_element));
|
||||
JoinedElement & t = joined_tables.back();
|
||||
|
||||
if (t.arrayJoin())
|
||||
{
|
||||
++num_array_join;
|
||||
continue;
|
||||
}
|
||||
|
||||
if (t.has_using)
|
||||
if (t.hasUsing())
|
||||
{
|
||||
++num_using;
|
||||
continue;
|
||||
}
|
||||
|
||||
if (auto * join = t.join)
|
||||
if (auto * join = t.tableJoin())
|
||||
{
|
||||
if (join->kind == ASTTableJoin::Kind::Cross ||
|
||||
join->kind == ASTTableJoin::Kind::Comma)
|
||||
{
|
||||
if (!join->children.empty())
|
||||
throw Exception("Logical error: CROSS JOIN has expressions", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
if (join->kind == ASTTableJoin::Kind::Comma)
|
||||
++num_comma;
|
||||
}
|
||||
}
|
||||
|
||||
if (num_using && (num_tables - num_array_join) > 2)
|
||||
@ -251,12 +279,20 @@ bool getTables(ASTSelectQuery & select, std::vector<JoinedTable> & joined_tables
|
||||
|
||||
if (num_array_join || num_using)
|
||||
return false;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
bool CrossToInnerJoinMatcher::needChildVisit(ASTPtr & node, const ASTPtr &)
|
||||
{
|
||||
if (node->as<ASTSubquery>())
|
||||
return false;
|
||||
return true;
|
||||
}
|
||||
|
||||
void CrossToInnerJoinMatcher::visit(ASTPtr & ast, Data & data)
|
||||
{
|
||||
if (auto * t = ast->as<ASTSelectQuery>())
|
||||
@ -266,10 +302,19 @@ void CrossToInnerJoinMatcher::visit(ASTPtr & ast, Data & data)
|
||||
void CrossToInnerJoinMatcher::visit(ASTSelectQuery & select, ASTPtr &, Data & data)
|
||||
{
|
||||
size_t num_comma = 0;
|
||||
std::vector<JoinedTable> joined_tables;
|
||||
std::vector<JoinedElement> joined_tables;
|
||||
if (!getTables(select, joined_tables, num_comma))
|
||||
return;
|
||||
|
||||
/// Check if joined_tables are consistent with known tables_with_columns
|
||||
{
|
||||
if (joined_tables.size() != data.tables_with_columns.size())
|
||||
throw Exception("Logical error: inconsistent number of tables", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
for (size_t i = 0; i < joined_tables.size(); ++i)
|
||||
joined_tables[i].checkTableName(data.tables_with_columns[i].table, data.current_database);
|
||||
}
|
||||
|
||||
/// COMMA to CROSS
|
||||
|
||||
if (num_comma)
|
||||
@ -283,7 +328,13 @@ void CrossToInnerJoinMatcher::visit(ASTSelectQuery & select, ASTPtr &, Data & da
|
||||
if (!select.where())
|
||||
return;
|
||||
|
||||
CheckExpressionVisitor::Data visitor_data{joined_tables};
|
||||
Aliases aliases;
|
||||
QueryAliasesVisitor::Data query_aliases_data{aliases};
|
||||
if (ASTPtr with = select.with())
|
||||
QueryAliasesVisitor(query_aliases_data).visit(with);
|
||||
QueryAliasesVisitor(query_aliases_data).visit(select.select());
|
||||
|
||||
CheckExpressionVisitor::Data visitor_data{joined_tables, data.tables_with_columns, std::move(aliases)};
|
||||
CheckExpressionVisitor(visitor_data).visit(select.where());
|
||||
|
||||
if (visitor_data.complex())
|
||||
@ -293,12 +344,7 @@ void CrossToInnerJoinMatcher::visit(ASTSelectQuery & select, ASTPtr &, Data & da
|
||||
{
|
||||
if (visitor_data.matchAny(i))
|
||||
{
|
||||
ASTTableJoin & join = *joined_tables[i].join;
|
||||
join.kind = ASTTableJoin::Kind::Inner;
|
||||
join.strictness = ASTTableJoin::Strictness::All;
|
||||
|
||||
join.on_expression = visitor_data.makeOnExpression(i);
|
||||
join.children.push_back(join.on_expression);
|
||||
joined_tables[i].rewriteCrossToInner(visitor_data.makeOnExpression(i));
|
||||
data.done = true;
|
||||
}
|
||||
}
|
||||
|
@ -6,6 +6,7 @@ namespace DB
|
||||
{
|
||||
|
||||
class ASTSelectQuery;
|
||||
struct TableWithColumnNamesAndTypes;
|
||||
|
||||
/// AST transformer. It replaces cross joins with equivalented inner join if possible.
|
||||
class CrossToInnerJoinMatcher
|
||||
@ -13,10 +14,12 @@ class CrossToInnerJoinMatcher
|
||||
public:
|
||||
struct Data
|
||||
{
|
||||
const std::vector<TableWithColumnNamesAndTypes> & tables_with_columns;
|
||||
const String current_database;
|
||||
bool done = false;
|
||||
};
|
||||
|
||||
static bool needChildVisit(ASTPtr &, const ASTPtr &) { return true; }
|
||||
static bool needChildVisit(ASTPtr &, const ASTPtr &);
|
||||
static void visit(ASTPtr & ast, Data & data);
|
||||
|
||||
private:
|
||||
|
@ -35,6 +35,12 @@ struct DatabaseAndTableWithAlias
|
||||
|
||||
/// Check if it satisfies another db_table name. @note opterion is not symmetric.
|
||||
bool satisfies(const DatabaseAndTableWithAlias & table, bool table_may_be_an_alias);
|
||||
|
||||
/// Exactly the same table name
|
||||
bool same(const DatabaseAndTableWithAlias & db_table) const
|
||||
{
|
||||
return database == db_table.database && table == db_table.table && alias == db_table.alias;
|
||||
}
|
||||
};
|
||||
|
||||
struct TableWithColumnNames
|
||||
@ -80,6 +86,19 @@ struct TableWithColumnNamesAndTypes
|
||||
, columns(columns_)
|
||||
{}
|
||||
|
||||
bool hasColumn(const String & name) const
|
||||
{
|
||||
if (names.empty())
|
||||
{
|
||||
for (auto & col : columns)
|
||||
names.insert(col.name);
|
||||
for (auto & col : hidden_columns)
|
||||
names.insert(col.name);
|
||||
}
|
||||
|
||||
return names.count(name);
|
||||
}
|
||||
|
||||
void addHiddenColumns(const NamesAndTypesList & addition)
|
||||
{
|
||||
hidden_columns.insert(hidden_columns.end(), addition.begin(), addition.end());
|
||||
@ -99,6 +118,9 @@ struct TableWithColumnNamesAndTypes
|
||||
|
||||
return TableWithColumnNames(table, std::move(out_columns), std::move(out_hidden_columns));
|
||||
}
|
||||
|
||||
private:
|
||||
mutable NameSet names;
|
||||
};
|
||||
|
||||
std::vector<DatabaseAndTableWithAlias> getDatabaseAndTables(const ASTSelectQuery & select_query, const String & current_database);
|
||||
|
@ -50,9 +50,8 @@ void ExpressionInfoMatcher::visit(const ASTIdentifier & identifier, const ASTPtr
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t best_table_pos = 0;
|
||||
if (IdentifierSemantic::chooseTable(identifier, data.tables, best_table_pos))
|
||||
data.unique_reference_tables_pos.emplace(best_table_pos);
|
||||
if (auto best_table_pos = IdentifierSemantic::chooseTable(identifier, data.tables))
|
||||
data.unique_reference_tables_pos.emplace(*best_table_pos);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -14,29 +14,18 @@ namespace ErrorCodes
|
||||
namespace
|
||||
{
|
||||
|
||||
const DatabaseAndTableWithAlias & extractTable(const DatabaseAndTableWithAlias & table)
|
||||
{
|
||||
return table;
|
||||
}
|
||||
|
||||
const DatabaseAndTableWithAlias & extractTable(const TableWithColumnNames & table)
|
||||
{
|
||||
return table.table;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
IdentifierSemantic::ColumnMatch tryChooseTable(const ASTIdentifier & identifier, const std::vector<T> & tables,
|
||||
size_t & best_table_pos, bool allow_ambiguous)
|
||||
std::optional<size_t> tryChooseTable(const ASTIdentifier & identifier, const std::vector<T> & tables, bool allow_ambiguous)
|
||||
{
|
||||
using ColumnMatch = IdentifierSemantic::ColumnMatch;
|
||||
|
||||
best_table_pos = 0;
|
||||
size_t best_table_pos = 0;
|
||||
auto best_match = ColumnMatch::NoMatch;
|
||||
size_t same_match = 0;
|
||||
|
||||
for (size_t i = 0; i < tables.size(); ++i)
|
||||
{
|
||||
auto match = IdentifierSemantic::canReferColumnToTable(identifier, extractTable(tables[i]));
|
||||
auto match = IdentifierSemantic::canReferColumnToTable(identifier, tables[i]);
|
||||
if (match != ColumnMatch::NoMatch)
|
||||
{
|
||||
if (match > best_match)
|
||||
@ -54,9 +43,13 @@ IdentifierSemantic::ColumnMatch tryChooseTable(const ASTIdentifier & identifier,
|
||||
{
|
||||
if (!allow_ambiguous)
|
||||
throw Exception("Ambiguous column '" + identifier.name + "'", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
|
||||
return ColumnMatch::Ambiguous;
|
||||
best_match = ColumnMatch::Ambiguous;
|
||||
return {};
|
||||
}
|
||||
return best_match;
|
||||
|
||||
if (best_match != ColumnMatch::NoMatch)
|
||||
return best_table_pos;
|
||||
return {};
|
||||
}
|
||||
|
||||
}
|
||||
@ -125,18 +118,22 @@ std::optional<size_t> IdentifierSemantic::getMembership(const ASTIdentifier & id
|
||||
return identifier.semantic->membership;
|
||||
}
|
||||
|
||||
bool IdentifierSemantic::chooseTable(const ASTIdentifier & identifier, const std::vector<DatabaseAndTableWithAlias> & tables,
|
||||
size_t & best_table_pos, bool ambiguous)
|
||||
std::optional<size_t> IdentifierSemantic::chooseTable(const ASTIdentifier & identifier, const std::vector<DatabaseAndTableWithAlias> & tables,
|
||||
bool ambiguous)
|
||||
{
|
||||
static constexpr auto no_match = IdentifierSemantic::ColumnMatch::NoMatch;
|
||||
return tryChooseTable<DatabaseAndTableWithAlias>(identifier, tables, best_table_pos, ambiguous) != no_match;
|
||||
return tryChooseTable<DatabaseAndTableWithAlias>(identifier, tables, ambiguous);
|
||||
}
|
||||
|
||||
bool IdentifierSemantic::chooseTable(const ASTIdentifier & identifier, const std::vector<TableWithColumnNames> & tables,
|
||||
size_t & best_table_pos, bool ambiguous)
|
||||
std::optional<size_t> IdentifierSemantic::chooseTable(const ASTIdentifier & identifier, const std::vector<TableWithColumnNames> & tables,
|
||||
bool ambiguous)
|
||||
{
|
||||
static constexpr auto no_match = IdentifierSemantic::ColumnMatch::NoMatch;
|
||||
return tryChooseTable<TableWithColumnNames>(identifier, tables, best_table_pos, ambiguous) != no_match;
|
||||
return tryChooseTable<TableWithColumnNames>(identifier, tables, ambiguous);
|
||||
}
|
||||
|
||||
std::optional<size_t> IdentifierSemantic::chooseTable(const ASTIdentifier & identifier, const std::vector<TableWithColumnNamesAndTypes> & tables,
|
||||
bool ambiguous)
|
||||
{
|
||||
return tryChooseTable<TableWithColumnNamesAndTypes>(identifier, tables, ambiguous);
|
||||
}
|
||||
|
||||
std::pair<String, String> IdentifierSemantic::extractDatabaseAndTable(const ASTIdentifier & identifier)
|
||||
@ -198,6 +195,22 @@ IdentifierSemantic::ColumnMatch IdentifierSemantic::canReferColumnToTable(const
|
||||
return ColumnMatch::NoMatch;
|
||||
}
|
||||
|
||||
IdentifierSemantic::ColumnMatch IdentifierSemantic::canReferColumnToTable(const ASTIdentifier & identifier,
|
||||
const TableWithColumnNames & db_and_table)
|
||||
{
|
||||
/// TODO: ColumnName match logic is disabled cause caller's code is not ready for it
|
||||
return canReferColumnToTable(identifier, db_and_table.table);
|
||||
}
|
||||
|
||||
IdentifierSemantic::ColumnMatch IdentifierSemantic::canReferColumnToTable(const ASTIdentifier & identifier,
|
||||
const TableWithColumnNamesAndTypes & db_and_table)
|
||||
{
|
||||
ColumnMatch match = canReferColumnToTable(identifier, db_and_table.table);
|
||||
if (match == ColumnMatch::NoMatch && identifier.isShort() && db_and_table.hasColumn(identifier.shortName()))
|
||||
match = ColumnMatch::ColumnName;
|
||||
return match;
|
||||
}
|
||||
|
||||
/// Strip qualificators from left side of column name.
|
||||
/// Example: 'database.table.name' -> 'name'.
|
||||
void IdentifierSemantic::setColumnShortName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
|
||||
|
@ -22,6 +22,7 @@ struct IdentifierSemantic
|
||||
enum class ColumnMatch
|
||||
{
|
||||
NoMatch,
|
||||
ColumnName, /// column qualified with column names list
|
||||
AliasedTableName, /// column qualified with table name (but table has an alias so its priority is lower than TableName)
|
||||
TableName, /// column qualified with table name
|
||||
DbAndTable, /// column qualified with database and table name
|
||||
@ -40,6 +41,9 @@ struct IdentifierSemantic
|
||||
static std::optional<String> extractNestedName(const ASTIdentifier & identifier, const String & table_name);
|
||||
|
||||
static ColumnMatch canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
|
||||
static ColumnMatch canReferColumnToTable(const ASTIdentifier & identifier, const TableWithColumnNames & db_and_table);
|
||||
static ColumnMatch canReferColumnToTable(const ASTIdentifier & identifier, const TableWithColumnNamesAndTypes & db_and_table);
|
||||
|
||||
static void setColumnShortName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
|
||||
static void setColumnLongName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
|
||||
static bool canBeAlias(const ASTIdentifier & identifier);
|
||||
@ -47,10 +51,12 @@ struct IdentifierSemantic
|
||||
static void coverName(ASTIdentifier &, const String & alias);
|
||||
static std::optional<ASTIdentifier> uncover(const ASTIdentifier & identifier);
|
||||
static std::optional<size_t> getMembership(const ASTIdentifier & identifier);
|
||||
static bool chooseTable(const ASTIdentifier &, const std::vector<DatabaseAndTableWithAlias> & tables, size_t & best_table_pos,
|
||||
bool ambiguous = false);
|
||||
static bool chooseTable(const ASTIdentifier &, const std::vector<TableWithColumnNames> & tables, size_t & best_table_pos,
|
||||
bool ambiguous = false);
|
||||
static std::optional<size_t> chooseTable(const ASTIdentifier &, const std::vector<DatabaseAndTableWithAlias> & tables,
|
||||
bool allow_ambiguous = false);
|
||||
static std::optional<size_t> chooseTable(const ASTIdentifier &, const std::vector<TableWithColumnNames> & tables,
|
||||
bool allow_ambiguous = false);
|
||||
static std::optional<size_t> chooseTable(const ASTIdentifier &, const std::vector<TableWithColumnNamesAndTypes> & tables,
|
||||
bool allow_ambiguous = false);
|
||||
|
||||
private:
|
||||
static bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & database, const String & table);
|
||||
|
@ -235,23 +235,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
throw Exception("Too deep subqueries. Maximum: " + settings.max_subquery_depth.toString(),
|
||||
ErrorCodes::TOO_DEEP_SUBQUERIES);
|
||||
|
||||
JoinedTables joined_tables(getSelectQuery());
|
||||
if (joined_tables.hasJoins())
|
||||
{
|
||||
CrossToInnerJoinVisitor::Data cross_to_inner;
|
||||
CrossToInnerJoinVisitor(cross_to_inner).visit(query_ptr);
|
||||
|
||||
JoinToSubqueryTransformVisitor::Data join_to_subs_data{*context};
|
||||
JoinToSubqueryTransformVisitor(join_to_subs_data).visit(query_ptr);
|
||||
|
||||
joined_tables.reset(getSelectQuery());
|
||||
}
|
||||
|
||||
max_streams = settings.max_threads;
|
||||
ASTSelectQuery & query = getSelectQuery();
|
||||
|
||||
const ASTPtr & left_table_expression = joined_tables.leftTableExpression();
|
||||
|
||||
bool has_input = input || input_pipe;
|
||||
if (input)
|
||||
{
|
||||
/// Read from prepared input.
|
||||
@ -262,35 +246,51 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
/// Read from prepared input.
|
||||
source_header = input_pipe->getHeader();
|
||||
}
|
||||
else if (joined_tables.isLeftTableSubquery())
|
||||
{
|
||||
/// Read from subquery.
|
||||
interpreter_subquery = std::make_unique<InterpreterSelectWithUnionQuery>(
|
||||
left_table_expression, getSubqueryContext(*context), options.subquery());
|
||||
|
||||
source_header = interpreter_subquery->getSampleBlock();
|
||||
}
|
||||
else if (!storage)
|
||||
{
|
||||
if (joined_tables.isLeftTableFunction())
|
||||
{
|
||||
/// Read from table function. propagate all settings from initSettings(),
|
||||
/// alternative is to call on current `context`, but that can potentially pollute it.
|
||||
storage = getSubqueryContext(*context).executeTableFunction(left_table_expression);
|
||||
}
|
||||
else
|
||||
storage = joined_tables.getLeftTableStorage(*context);
|
||||
}
|
||||
JoinedTables joined_tables(getSubqueryContext(*context), getSelectQuery());
|
||||
|
||||
if (!has_input && !storage)
|
||||
storage = joined_tables.getLeftTableStorage();
|
||||
|
||||
if (storage)
|
||||
{
|
||||
table_lock = storage->lockStructureForShare(false, context->getInitialQueryId());
|
||||
table_id = storage->getStorageID();
|
||||
|
||||
joined_tables.resolveTables(getSubqueryContext(*context), storage);
|
||||
}
|
||||
else
|
||||
joined_tables.resolveTables(getSubqueryContext(*context), source_header.getNamesAndTypesList());
|
||||
|
||||
if (has_input || !joined_tables.resolveTables())
|
||||
joined_tables.makeFakeTable(storage, source_header);
|
||||
|
||||
/// Rewrite JOINs
|
||||
if (!has_input && joined_tables.tablesCount() > 1)
|
||||
{
|
||||
CrossToInnerJoinVisitor::Data cross_to_inner{joined_tables.tablesWithColumns(), context->getCurrentDatabase()};
|
||||
CrossToInnerJoinVisitor(cross_to_inner).visit(query_ptr);
|
||||
|
||||
JoinToSubqueryTransformVisitor::Data join_to_subs_data{*context};
|
||||
JoinToSubqueryTransformVisitor(join_to_subs_data).visit(query_ptr);
|
||||
|
||||
joined_tables.reset(getSelectQuery());
|
||||
joined_tables.resolveTables();
|
||||
|
||||
if (storage && joined_tables.isLeftTableSubquery())
|
||||
{
|
||||
/// Rewritten with subquery. Free storage here locks here.
|
||||
storage = {};
|
||||
table_lock.release();
|
||||
table_id = StorageID::createEmpty();
|
||||
}
|
||||
}
|
||||
|
||||
if (!has_input)
|
||||
{
|
||||
interpreter_subquery = joined_tables.makeLeftTableSubquery(options.subquery());
|
||||
if (interpreter_subquery)
|
||||
source_header = interpreter_subquery->getSampleBlock();
|
||||
}
|
||||
|
||||
max_streams = settings.max_threads;
|
||||
ASTSelectQuery & query = getSelectQuery();
|
||||
|
||||
auto analyze = [&] (bool try_move_to_prewhere = true)
|
||||
{
|
||||
@ -330,11 +330,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
if (syntax_analyzer_result->rewrite_subqueries)
|
||||
{
|
||||
/// remake interpreter_subquery when PredicateOptimizer rewrites subqueries and main table is subquery
|
||||
if (joined_tables.isLeftTableSubquery())
|
||||
interpreter_subquery = std::make_unique<InterpreterSelectWithUnionQuery>(
|
||||
left_table_expression,
|
||||
getSubqueryContext(*context),
|
||||
options.subquery());
|
||||
interpreter_subquery = joined_tables.makeLeftTableSubquery(options.subquery());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -147,9 +147,8 @@ struct ColumnAliasesMatcher
|
||||
{
|
||||
bool last_table = false;
|
||||
{
|
||||
size_t best_table_pos = 0;
|
||||
if (IdentifierSemantic::chooseTable(*identifier, tables, best_table_pos))
|
||||
last_table = (best_table_pos + 1 == tables.size());
|
||||
if (auto best_table_pos = IdentifierSemantic::chooseTable(*identifier, tables))
|
||||
last_table = (*best_table_pos + 1 == tables.size());
|
||||
}
|
||||
|
||||
if (!last_table)
|
||||
@ -207,10 +206,9 @@ struct ColumnAliasesMatcher
|
||||
bool last_table = false;
|
||||
String long_name;
|
||||
|
||||
size_t table_pos = 0;
|
||||
if (IdentifierSemantic::chooseTable(node, data.tables, table_pos))
|
||||
if (auto table_pos = IdentifierSemantic::chooseTable(node, data.tables))
|
||||
{
|
||||
auto & table = data.tables[table_pos];
|
||||
auto & table = data.tables[*table_pos];
|
||||
IdentifierSemantic::setColumnLongName(node, table); /// table_name.column_name -> table_alias.column_name
|
||||
long_name = node.name;
|
||||
if (&table == &data.tables.back())
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Storages/StorageValues.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -33,8 +34,9 @@ void checkTablesWithColumns(const std::vector<T> & tables_with_columns, const Co
|
||||
|
||||
}
|
||||
|
||||
JoinedTables::JoinedTables(const ASTSelectQuery & select_query)
|
||||
: table_expressions(getTableExpressions(select_query))
|
||||
JoinedTables::JoinedTables(Context && context_, const ASTSelectQuery & select_query)
|
||||
: context(context_)
|
||||
, table_expressions(getTableExpressions(select_query))
|
||||
, left_table_expression(extractTableExpression(select_query, 0))
|
||||
, left_db_and_table(getDatabaseAndTable(select_query, 0))
|
||||
{}
|
||||
@ -49,9 +51,20 @@ bool JoinedTables::isLeftTableFunction() const
|
||||
return left_table_expression && left_table_expression->as<ASTFunction>();
|
||||
}
|
||||
|
||||
StoragePtr JoinedTables::getLeftTableStorage(Context & context)
|
||||
std::unique_ptr<InterpreterSelectWithUnionQuery> JoinedTables::makeLeftTableSubquery(const SelectQueryOptions & select_options)
|
||||
{
|
||||
StoragePtr storage;
|
||||
if (!isLeftTableSubquery())
|
||||
return {};
|
||||
return std::make_unique<InterpreterSelectWithUnionQuery>(left_table_expression, context, select_options);
|
||||
}
|
||||
|
||||
StoragePtr JoinedTables::getLeftTableStorage()
|
||||
{
|
||||
if (isLeftTableSubquery())
|
||||
return {};
|
||||
|
||||
if (isLeftTableFunction())
|
||||
return context.executeTableFunction(left_table_expression);
|
||||
|
||||
if (left_db_and_table)
|
||||
{
|
||||
@ -75,42 +88,36 @@ StoragePtr JoinedTables::getLeftTableStorage(Context & context)
|
||||
if (tmp_table_id.database_name == database_name && tmp_table_id.table_name == table_name)
|
||||
{
|
||||
/// Read from view source.
|
||||
storage = context.getViewSource();
|
||||
return context.getViewSource();
|
||||
}
|
||||
}
|
||||
|
||||
if (!storage)
|
||||
{
|
||||
/// Read from table. Even without table expression (implicit SELECT ... FROM system.one).
|
||||
storage = context.getTable(database_name, table_name);
|
||||
}
|
||||
|
||||
return storage;
|
||||
/// Read from table. Even without table expression (implicit SELECT ... FROM system.one).
|
||||
return context.getTable(database_name, table_name);
|
||||
}
|
||||
|
||||
void JoinedTables::resolveTables(const Context & context, StoragePtr storage)
|
||||
bool JoinedTables::resolveTables()
|
||||
{
|
||||
tables_with_columns = getDatabaseAndTablesWithColumns(table_expressions, context);
|
||||
checkTablesWithColumns(tables_with_columns, context);
|
||||
|
||||
if (tables_with_columns.empty())
|
||||
return !tables_with_columns.empty();
|
||||
}
|
||||
|
||||
void JoinedTables::makeFakeTable(StoragePtr storage, const Block & source_header)
|
||||
{
|
||||
if (storage)
|
||||
{
|
||||
const ColumnsDescription & storage_columns = storage->getColumns();
|
||||
tables_with_columns.emplace_back(DatabaseAndTableWithAlias{}, storage_columns.getOrdinary());
|
||||
|
||||
auto & table = tables_with_columns.back();
|
||||
table.addHiddenColumns(storage_columns.getMaterialized());
|
||||
table.addHiddenColumns(storage_columns.getAliases());
|
||||
table.addHiddenColumns(storage_columns.getVirtuals());
|
||||
}
|
||||
}
|
||||
|
||||
void JoinedTables::resolveTables(const Context & context, const NamesAndTypesList & source_columns)
|
||||
{
|
||||
tables_with_columns = getDatabaseAndTablesWithColumns(table_expressions, context);
|
||||
checkTablesWithColumns(tables_with_columns, context);
|
||||
|
||||
if (tables_with_columns.empty())
|
||||
tables_with_columns.emplace_back(DatabaseAndTableWithAlias{}, source_columns);
|
||||
else
|
||||
tables_with_columns.emplace_back(DatabaseAndTableWithAlias{}, source_header.getNamesAndTypesList());
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
|
||||
namespace DB
|
||||
@ -9,6 +10,7 @@ namespace DB
|
||||
|
||||
class ASTSelectQuery;
|
||||
class Context;
|
||||
struct SelectQueryOptions;
|
||||
|
||||
/// Joined tables' columns resolver.
|
||||
/// We want to get each table structure at most once per table occurance. Or even better once per table.
|
||||
@ -16,32 +18,30 @@ class Context;
|
||||
class JoinedTables
|
||||
{
|
||||
public:
|
||||
JoinedTables() = default;
|
||||
JoinedTables(const ASTSelectQuery & select_query);
|
||||
JoinedTables(Context && contex, const ASTSelectQuery & select_query);
|
||||
|
||||
void reset(const ASTSelectQuery & select_query)
|
||||
{
|
||||
*this = JoinedTables(select_query);
|
||||
*this = JoinedTables(std::move(context), select_query);
|
||||
}
|
||||
|
||||
StoragePtr getLeftTableStorage(Context & context);
|
||||
|
||||
/// Resolve columns or get from storage. It assumes storage is not nullptr.
|
||||
void resolveTables(const Context & context, StoragePtr storage);
|
||||
/// Resolve columns or get from source list.
|
||||
void resolveTables(const Context & context, const NamesAndTypesList & source_columns);
|
||||
StoragePtr getLeftTableStorage();
|
||||
bool resolveTables();
|
||||
void makeFakeTable(StoragePtr storage, const Block & source_header);
|
||||
|
||||
const std::vector<TableWithColumnNamesAndTypes> & tablesWithColumns() const { return tables_with_columns; }
|
||||
|
||||
bool isLeftTableSubquery() const;
|
||||
bool isLeftTableFunction() const;
|
||||
bool hasJoins() const { return table_expressions.size() > 1; }
|
||||
size_t tablesCount() const { return table_expressions.size(); }
|
||||
|
||||
const ASTPtr & leftTableExpression() const { return left_table_expression; }
|
||||
const String & leftTableDatabase() const { return database_name; }
|
||||
const String & leftTableName() const { return table_name; }
|
||||
|
||||
std::unique_ptr<InterpreterSelectWithUnionQuery> makeLeftTableSubquery(const SelectQueryOptions & select_options);
|
||||
|
||||
private:
|
||||
Context context;
|
||||
std::vector<const ASTTableExpression *> table_expressions;
|
||||
std::vector<TableWithColumnNamesAndTypes> tables_with_columns;
|
||||
|
||||
|
@ -30,7 +30,7 @@ static String wrongAliasMessage(const ASTPtr & ast, const ASTPtr & prev_ast, con
|
||||
}
|
||||
|
||||
|
||||
bool QueryAliasesMatcher::needChildVisit(ASTPtr & node, const ASTPtr &)
|
||||
bool QueryAliasesMatcher::needChildVisit(const ASTPtr & node, const ASTPtr &)
|
||||
{
|
||||
/// Don't descent into table functions and subqueries and special case for ArrayJoin.
|
||||
if (node->as<ASTTableExpression>() || node->as<ASTSelectWithUnionQuery>() || node->as<ASTArrayJoin>())
|
||||
@ -38,7 +38,7 @@ bool QueryAliasesMatcher::needChildVisit(ASTPtr & node, const ASTPtr &)
|
||||
return true;
|
||||
}
|
||||
|
||||
void QueryAliasesMatcher::visit(ASTPtr & ast, Data & data)
|
||||
void QueryAliasesMatcher::visit(const ASTPtr & ast, Data & data)
|
||||
{
|
||||
if (auto * s = ast->as<ASTSubquery>())
|
||||
visit(*s, ast, data);
|
||||
@ -81,8 +81,9 @@ void QueryAliasesMatcher::visit(const ASTArrayJoin &, const ASTPtr & ast, Data &
|
||||
/// set unique aliases for all subqueries. this is needed, because:
|
||||
/// 1) content of subqueries could change after recursive analysis, and auto-generated column names could become incorrect
|
||||
/// 2) result of different scalar subqueries can be cached inside expressions compilation cache and must have different names
|
||||
void QueryAliasesMatcher::visit(ASTSubquery & subquery, const ASTPtr & ast, Data & data)
|
||||
void QueryAliasesMatcher::visit(const ASTSubquery & const_subquery, const ASTPtr & ast, Data & data)
|
||||
{
|
||||
ASTSubquery & subquery = const_cast<ASTSubquery &>(const_subquery);
|
||||
Aliases & aliases = data.aliases;
|
||||
|
||||
static std::atomic_uint64_t subquery_index = 0;
|
||||
|
@ -15,19 +15,19 @@ struct ASTArrayJoin;
|
||||
class QueryAliasesMatcher
|
||||
{
|
||||
public:
|
||||
using Visitor = InDepthNodeVisitor<QueryAliasesMatcher, false>;
|
||||
using Visitor = ConstInDepthNodeVisitor<QueryAliasesMatcher, false>;
|
||||
|
||||
struct Data
|
||||
{
|
||||
Aliases & aliases;
|
||||
};
|
||||
|
||||
static void visit(ASTPtr & ast, Data & data);
|
||||
static bool needChildVisit(ASTPtr & node, const ASTPtr & child);
|
||||
static void visit(const ASTPtr & ast, Data & data);
|
||||
static bool needChildVisit(const ASTPtr & node, const ASTPtr & child);
|
||||
|
||||
private:
|
||||
static void visit(const ASTSelectQuery & select, const ASTPtr & ast, Data & data);
|
||||
static void visit(ASTSubquery & subquery, const ASTPtr & ast, Data & data);
|
||||
static void visit(const ASTSubquery & subquery, const ASTPtr & ast, Data & data);
|
||||
static void visit(const ASTArrayJoin &, const ASTPtr & ast, Data & data);
|
||||
static void visitOther(const ASTPtr & ast, Data & data);
|
||||
};
|
||||
|
@ -93,10 +93,10 @@ void TranslateQualifiedNamesMatcher::visit(ASTIdentifier & identifier, ASTPtr &,
|
||||
if (IdentifierSemantic::getColumnName(identifier))
|
||||
{
|
||||
String short_name = identifier.shortName();
|
||||
size_t table_pos = 0;
|
||||
bool allow_ambiguous = data.join_using_columns.count(short_name);
|
||||
if (IdentifierSemantic::chooseTable(identifier, data.tables, table_pos, allow_ambiguous))
|
||||
if (auto best_pos = IdentifierSemantic::chooseTable(identifier, data.tables, allow_ambiguous))
|
||||
{
|
||||
size_t table_pos = *best_pos;
|
||||
if (data.unknownColumn(table_pos, identifier))
|
||||
{
|
||||
String table_name = data.tables[table_pos].table.getQualifiedNamePrefix(false);
|
||||
|
@ -0,0 +1,14 @@
|
||||
1
|
||||
3
|
||||
5
|
||||
6
|
||||
0.0000
|
||||
9
|
||||
10
|
||||
12
|
||||
14
|
||||
0.00000000
|
||||
16
|
||||
18
|
||||
19
|
||||
0.0000
|
802
dbms/tests/queries/0_stateless/01095_tpch_like_smoke.sql
Normal file
802
dbms/tests/queries/0_stateless/01095_tpch_like_smoke.sql
Normal file
@ -0,0 +1,802 @@
|
||||
CREATE DATABASE IF NOT EXISTS tpch;
|
||||
USE tpch;
|
||||
|
||||
DROP TABLE IF EXISTS part;
|
||||
DROP TABLE IF EXISTS supplier;
|
||||
DROP TABLE IF EXISTS partsupp;
|
||||
DROP TABLE IF EXISTS customer;
|
||||
DROP TABLE IF EXISTS orders;
|
||||
DROP TABLE IF EXISTS lineitem;
|
||||
DROP TABLE IF EXISTS nation;
|
||||
DROP TABLE IF EXISTS region;
|
||||
|
||||
CREATE TABLE part
|
||||
(
|
||||
p_partkey Int32, -- PK
|
||||
p_name String, -- variable text, size 55
|
||||
p_mfgr FixedString(25),
|
||||
p_brand FixedString(10),
|
||||
p_type String, -- variable text, size 25
|
||||
p_size Int32, -- integer
|
||||
p_container FixedString(10),
|
||||
p_retailprice Decimal(18,2),
|
||||
p_comment String, -- variable text, size 23
|
||||
CONSTRAINT pk CHECK p_partkey >= 0,
|
||||
CONSTRAINT positive CHECK (p_size >= 0 AND p_retailprice >= 0)
|
||||
) engine = MergeTree ORDER BY (p_partkey);
|
||||
|
||||
CREATE TABLE supplier
|
||||
(
|
||||
s_suppkey Int32, -- PK
|
||||
s_name FixedString(25),
|
||||
s_address String, -- variable text, size 40
|
||||
s_nationkey Int32, -- FK n_nationkey
|
||||
s_phone FixedString(15),
|
||||
s_acctbal Decimal(18,2),
|
||||
s_comment String, -- variable text, size 101
|
||||
CONSTRAINT pk CHECK s_suppkey >= 0
|
||||
) engine = MergeTree ORDER BY (s_suppkey);
|
||||
|
||||
CREATE TABLE partsupp
|
||||
(
|
||||
ps_partkey Int32, -- PK(1), FK p_partkey
|
||||
ps_suppkey Int32, -- PK(2), FK s_suppkey
|
||||
ps_availqty Int32, -- integer
|
||||
ps_supplycost Decimal(18,2),
|
||||
ps_comment String, -- variable text, size 199
|
||||
CONSTRAINT pk CHECK ps_partkey >= 0,
|
||||
CONSTRAINT c1 CHECK (ps_availqty >= 0 AND ps_supplycost >= 0)
|
||||
) engine = MergeTree ORDER BY (ps_partkey, ps_suppkey);
|
||||
|
||||
CREATE TABLE customer
|
||||
(
|
||||
c_custkey Int32, -- PK
|
||||
c_name String, -- variable text, size 25
|
||||
c_address String, -- variable text, size 40
|
||||
c_nationkey Int32, -- FK n_nationkey
|
||||
c_phone FixedString(15),
|
||||
c_acctbal Decimal(18,2),
|
||||
c_mktsegment FixedString(10),
|
||||
c_comment String, -- variable text, size 117
|
||||
CONSTRAINT pk CHECK c_custkey >= 0
|
||||
) engine = MergeTree ORDER BY (c_custkey);
|
||||
|
||||
CREATE TABLE orders
|
||||
(
|
||||
o_orderkey Int32, -- PK
|
||||
o_custkey Int32, -- FK c_custkey
|
||||
o_orderstatus FixedString(1),
|
||||
o_totalprice Decimal(18,2),
|
||||
o_orderdate Date,
|
||||
o_orderpriority FixedString(15),
|
||||
o_clerk FixedString(15),
|
||||
o_shippriority Int32, -- integer
|
||||
o_comment String, -- variable text, size 79
|
||||
CONSTRAINT c1 CHECK o_totalprice >= 0
|
||||
) engine = MergeTree ORDER BY (o_orderdate, o_orderkey);
|
||||
|
||||
CREATE TABLE lineitem
|
||||
(
|
||||
l_orderkey Int32, -- PK(1), FK o_orderkey
|
||||
l_partkey Int32, -- FK ps_partkey
|
||||
l_suppkey Int32, -- FK ps_suppkey
|
||||
l_linenumber Int32, -- PK(2)
|
||||
l_quantity Decimal(18,2),
|
||||
l_extendedprice Decimal(18,2),
|
||||
l_discount Decimal(18,2),
|
||||
l_tax Decimal(18,2),
|
||||
l_returnflag FixedString(1),
|
||||
l_linestatus FixedString(1),
|
||||
l_shipdate Date,
|
||||
l_commitdate Date,
|
||||
l_receiptdate Date,
|
||||
l_shipinstruct FixedString(25),
|
||||
l_shipmode FixedString(10),
|
||||
l_comment String, -- variable text size 44
|
||||
CONSTRAINT c1 CHECK (l_quantity >= 0 AND l_extendedprice >= 0 AND l_tax >= 0 AND l_shipdate <= l_receiptdate)
|
||||
-- CONSTRAINT c2 CHECK (l_discount >= 0 AND l_discount <= 1)
|
||||
) engine = MergeTree ORDER BY (l_shipdate, l_receiptdate, l_orderkey, l_linenumber);
|
||||
|
||||
CREATE TABLE nation
|
||||
(
|
||||
n_nationkey Int32, -- PK
|
||||
n_name FixedString(25),
|
||||
n_regionkey Int32, -- FK r_regionkey
|
||||
n_comment String, -- variable text, size 152
|
||||
CONSTRAINT pk CHECK n_nationkey >= 0
|
||||
) Engine = MergeTree ORDER BY (n_nationkey);
|
||||
|
||||
CREATE TABLE region
|
||||
(
|
||||
r_regionkey Int32, -- PK
|
||||
r_name FixedString(25),
|
||||
r_comment String, -- variable text, size 152
|
||||
CONSTRAINT pk CHECK r_regionkey >= 0
|
||||
) engine = MergeTree ORDER BY (r_regionkey);
|
||||
|
||||
select 1;
|
||||
select
|
||||
l_returnflag,
|
||||
l_linestatus,
|
||||
sum(l_quantity) as sum_qty,
|
||||
sum(l_extendedprice) as sum_base_price,
|
||||
sum(l_extendedprice * (1 - l_discount)) as sum_disc_price,
|
||||
sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge,
|
||||
avg(l_quantity) as avg_qty,
|
||||
avg(l_extendedprice) as avg_price,
|
||||
avg(l_discount) as avg_disc,
|
||||
count(*) as count_order
|
||||
from
|
||||
lineitem
|
||||
where
|
||||
l_shipdate <= toDate('1998-12-01') - interval 90 day
|
||||
group by
|
||||
l_returnflag,
|
||||
l_linestatus
|
||||
order by
|
||||
l_returnflag,
|
||||
l_linestatus;
|
||||
|
||||
-- select 2; -- rewrite fail
|
||||
-- select
|
||||
-- s_acctbal,
|
||||
-- s_name,
|
||||
-- n_name,
|
||||
-- p_partkey,
|
||||
-- p_mfgr,
|
||||
-- s_address,
|
||||
-- s_phone,
|
||||
-- s_comment
|
||||
-- from
|
||||
-- part,
|
||||
-- supplier,
|
||||
-- partsupp,
|
||||
-- nation,
|
||||
-- region
|
||||
-- where
|
||||
-- p_partkey = ps_partkey
|
||||
-- and s_suppkey = ps_suppkey
|
||||
-- and p_size = 15
|
||||
-- and p_type like '%BRASS'
|
||||
-- and s_nationkey = n_nationkey
|
||||
-- and n_regionkey = r_regionkey
|
||||
-- and r_name = 'EUROPE'
|
||||
-- and ps_supplycost = (
|
||||
-- select
|
||||
-- min(ps_supplycost)
|
||||
-- from
|
||||
-- partsupp,
|
||||
-- supplier,
|
||||
-- nation,
|
||||
-- region
|
||||
-- where
|
||||
-- p_partkey = ps_partkey
|
||||
-- and s_suppkey = ps_suppkey
|
||||
-- and s_nationkey = n_nationkey
|
||||
-- and n_regionkey = r_regionkey
|
||||
-- and r_name = 'EUROPE'
|
||||
-- )
|
||||
-- order by
|
||||
-- s_acctbal desc,
|
||||
-- n_name,
|
||||
-- s_name,
|
||||
-- p_partkey
|
||||
-- limit 100;
|
||||
|
||||
select 3;
|
||||
select
|
||||
l_orderkey,
|
||||
sum(l_extendedprice * (1 - l_discount)) as revenue,
|
||||
o_orderdate,
|
||||
o_shippriority
|
||||
from
|
||||
customer,
|
||||
orders,
|
||||
lineitem
|
||||
where
|
||||
c_mktsegment = 'BUILDING'
|
||||
and c_custkey = o_custkey
|
||||
and l_orderkey = o_orderkey
|
||||
and o_orderdate < toDate('1995-03-15')
|
||||
and l_shipdate > toDate('1995-03-15')
|
||||
group by
|
||||
l_orderkey,
|
||||
o_orderdate,
|
||||
o_shippriority
|
||||
order by
|
||||
revenue desc,
|
||||
o_orderdate
|
||||
limit 10;
|
||||
|
||||
-- select 4;
|
||||
-- select
|
||||
-- o_orderpriority,
|
||||
-- count(*) as order_count
|
||||
-- from
|
||||
-- orders
|
||||
-- where
|
||||
-- o_orderdate >= toDate('1993-07-01')
|
||||
-- and o_orderdate < toDate('1993-07-01') + interval '3' month
|
||||
-- and exists (
|
||||
-- select
|
||||
-- *
|
||||
-- from
|
||||
-- lineitem
|
||||
-- where
|
||||
-- l_orderkey = o_orderkey
|
||||
-- and l_commitdate < l_receiptdate
|
||||
-- )
|
||||
-- group by
|
||||
-- o_orderpriority
|
||||
-- order by
|
||||
-- o_orderpriority;
|
||||
|
||||
select 5;
|
||||
select
|
||||
n_name,
|
||||
sum(l_extendedprice * (1 - l_discount)) as revenue
|
||||
from
|
||||
customer,
|
||||
orders,
|
||||
lineitem,
|
||||
supplier,
|
||||
nation,
|
||||
region
|
||||
where
|
||||
c_custkey = o_custkey
|
||||
and l_orderkey = o_orderkey
|
||||
and l_suppkey = s_suppkey
|
||||
and c_nationkey = s_nationkey
|
||||
and s_nationkey = n_nationkey
|
||||
and n_regionkey = r_regionkey
|
||||
and r_name = 'ASIA'
|
||||
and o_orderdate >= toDate('1994-01-01')
|
||||
and o_orderdate < toDate('1994-01-01') + interval '1' year
|
||||
group by
|
||||
n_name
|
||||
order by
|
||||
revenue desc;
|
||||
|
||||
select 6;
|
||||
select
|
||||
sum(l_extendedprice * l_discount) as revenue
|
||||
from
|
||||
lineitem
|
||||
where
|
||||
l_shipdate >= toDate('1994-01-01')
|
||||
and l_shipdate < toDate('1994-01-01') + interval '1' year
|
||||
and l_discount between toDecimal32(0.06, 2) - toDecimal32(0.01, 2)
|
||||
and toDecimal32(0.06, 2) + toDecimal32(0.01, 2)
|
||||
and l_quantity < 24;
|
||||
|
||||
-- select 7;
|
||||
-- select
|
||||
-- supp_nation,
|
||||
-- cust_nation,
|
||||
-- l_year,
|
||||
-- sum(volume) as revenue
|
||||
-- from
|
||||
-- (
|
||||
-- select
|
||||
-- n1.n_name as supp_nation,
|
||||
-- n2.n_name as cust_nation,
|
||||
-- extract(year from l_shipdate) as l_year,
|
||||
-- l_extendedprice * (1 - l_discount) as volume
|
||||
-- from
|
||||
-- supplier,
|
||||
-- lineitem,
|
||||
-- orders,
|
||||
-- customer,
|
||||
-- nation n1,
|
||||
-- nation n2
|
||||
-- where
|
||||
-- s_suppkey = l_suppkey
|
||||
-- and o_orderkey = l_orderkey
|
||||
-- and c_custkey = o_custkey
|
||||
-- and s_nationkey = n1.n_nationkey
|
||||
-- and c_nationkey = n2.n_nationkey
|
||||
-- and (
|
||||
-- (n1.n_name = 'FRANCE' and n2.n_name = 'GERMANY')
|
||||
-- or (n1.n_name = 'GERMANY' and n2.n_name = 'FRANCE')
|
||||
-- )
|
||||
-- and l_shipdate between toDate('1995-01-01') and toDate('1996-12-31')
|
||||
-- ) as shipping
|
||||
-- group by
|
||||
-- supp_nation,
|
||||
-- cust_nation,
|
||||
-- l_year
|
||||
-- order by
|
||||
-- supp_nation,
|
||||
-- cust_nation,
|
||||
-- l_year;
|
||||
|
||||
-- select 8;
|
||||
-- select
|
||||
-- o_year,
|
||||
-- sum(case
|
||||
-- when nation = 'BRAZIL' then volume
|
||||
-- else 0
|
||||
-- end) / sum(volume) as mkt_share
|
||||
-- from
|
||||
-- (
|
||||
-- select
|
||||
-- extract(year from o_orderdate) as o_year,
|
||||
-- l_extendedprice * (1 - l_discount) as volume,
|
||||
-- n2.n_name as nation
|
||||
-- from
|
||||
-- part,
|
||||
-- supplier,
|
||||
-- lineitem,
|
||||
-- orders,
|
||||
-- customer,
|
||||
-- nation n1,
|
||||
-- nation n2,
|
||||
-- region
|
||||
-- where
|
||||
-- p_partkey = l_partkey
|
||||
-- and s_suppkey = l_suppkey
|
||||
-- and l_orderkey = o_orderkey
|
||||
-- and o_custkey = c_custkey
|
||||
-- and c_nationkey = n1.n_nationkey
|
||||
-- and n1.n_regionkey = r_regionkey
|
||||
-- and r_name = 'AMERICA'
|
||||
-- and s_nationkey = n2.n_nationkey
|
||||
-- and o_orderdate between toDate('1995-01-01') and toDate('1996-12-31')
|
||||
-- and p_type = 'ECONOMY ANODIZED STEEL'
|
||||
-- ) as all_nations
|
||||
-- group by
|
||||
-- o_year
|
||||
-- order by
|
||||
-- o_year;
|
||||
|
||||
select 9;
|
||||
select
|
||||
nation,
|
||||
o_year,
|
||||
sum(amount) as sum_profit
|
||||
from
|
||||
(
|
||||
select
|
||||
n_name as nation,
|
||||
extract(year from o_orderdate) as o_year,
|
||||
l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount
|
||||
from
|
||||
part,
|
||||
supplier,
|
||||
lineitem,
|
||||
partsupp,
|
||||
orders,
|
||||
nation
|
||||
where
|
||||
s_suppkey = l_suppkey
|
||||
and ps_suppkey = l_suppkey
|
||||
and ps_partkey = l_partkey
|
||||
and p_partkey = l_partkey
|
||||
and o_orderkey = l_orderkey
|
||||
and s_nationkey = n_nationkey
|
||||
and p_name like '%green%'
|
||||
) as profit
|
||||
group by
|
||||
nation,
|
||||
o_year
|
||||
order by
|
||||
nation,
|
||||
o_year desc;
|
||||
|
||||
select 10;
|
||||
select
|
||||
c_custkey,
|
||||
c_name,
|
||||
sum(l_extendedprice * (1 - l_discount)) as revenue,
|
||||
c_acctbal,
|
||||
n_name,
|
||||
c_address,
|
||||
c_phone,
|
||||
c_comment
|
||||
from
|
||||
customer,
|
||||
orders,
|
||||
lineitem,
|
||||
nation
|
||||
where
|
||||
c_custkey = o_custkey
|
||||
and l_orderkey = o_orderkey
|
||||
and o_orderdate >= toDate('1993-10-01')
|
||||
and o_orderdate < toDate('1993-10-01') + interval '3' month
|
||||
and l_returnflag = 'R'
|
||||
and c_nationkey = n_nationkey
|
||||
group by
|
||||
c_custkey,
|
||||
c_name,
|
||||
c_acctbal,
|
||||
c_phone,
|
||||
n_name,
|
||||
c_address,
|
||||
c_comment
|
||||
order by
|
||||
revenue desc
|
||||
limit 20;
|
||||
|
||||
-- select 11; -- rewrite fail
|
||||
-- select
|
||||
-- ps_partkey,
|
||||
-- sum(ps_supplycost * ps_availqty) as value
|
||||
-- from
|
||||
-- partsupp,
|
||||
-- supplier,
|
||||
-- nation
|
||||
-- where
|
||||
-- ps_suppkey = s_suppkey
|
||||
-- and s_nationkey = n_nationkey
|
||||
-- and n_name = 'GERMANY'
|
||||
-- group by
|
||||
-- ps_partkey having
|
||||
-- sum(ps_supplycost * ps_availqty) > (
|
||||
-- select
|
||||
-- sum(ps_supplycost * ps_availqty) * 0.0100000000
|
||||
-- -- ^^^^^^^^^^^^
|
||||
-- -- The above constant needs to be adjusted according
|
||||
-- -- to the scale factor (SF): constant = 0.0001 / SF.
|
||||
-- from
|
||||
-- partsupp,
|
||||
-- supplier,
|
||||
-- nation
|
||||
-- where
|
||||
-- ps_suppkey = s_suppkey
|
||||
-- and s_nationkey = n_nationkey
|
||||
-- and n_name = 'GERMANY'
|
||||
-- )
|
||||
-- order by
|
||||
-- value desc;
|
||||
|
||||
select 12;
|
||||
select
|
||||
l_shipmode,
|
||||
sum(case
|
||||
when o_orderpriority = '1-URGENT'
|
||||
or o_orderpriority = '2-HIGH'
|
||||
then 1
|
||||
else 0
|
||||
end) as high_line_count,
|
||||
sum(case
|
||||
when o_orderpriority <> '1-URGENT'
|
||||
and o_orderpriority <> '2-HIGH'
|
||||
then 1
|
||||
else 0
|
||||
end) as low_line_count
|
||||
from
|
||||
orders,
|
||||
lineitem
|
||||
where
|
||||
o_orderkey = l_orderkey
|
||||
and l_shipmode in ('MAIL', 'SHIP')
|
||||
and l_commitdate < l_receiptdate
|
||||
and l_shipdate < l_commitdate
|
||||
and l_receiptdate >= toDate('1994-01-01')
|
||||
and l_receiptdate < toDate('1994-01-01') + interval '1' year
|
||||
group by
|
||||
l_shipmode
|
||||
order by
|
||||
l_shipmode;
|
||||
|
||||
-- select 13; -- rewrite fail
|
||||
-- select
|
||||
-- c_count,
|
||||
-- count(*) as custdist
|
||||
-- from
|
||||
-- (
|
||||
-- select
|
||||
-- c_custkey,
|
||||
-- count(o_orderkey)
|
||||
-- from
|
||||
-- customer left outer join orders on
|
||||
-- c_custkey = o_custkey
|
||||
-- and o_comment not like '%special%requests%'
|
||||
-- group by
|
||||
-- c_custkey
|
||||
-- ) as c_orders
|
||||
-- group by
|
||||
-- c_count
|
||||
-- order by
|
||||
-- custdist desc,
|
||||
-- c_count desc;
|
||||
|
||||
select 14;
|
||||
select
|
||||
toDecimal32(100.00, 2) * sum(case
|
||||
when p_type like 'PROMO%'
|
||||
then l_extendedprice * (1 - l_discount)
|
||||
else 0
|
||||
end) / (1 + sum(l_extendedprice * (1 - l_discount))) as promo_revenue
|
||||
from
|
||||
lineitem,
|
||||
part
|
||||
where
|
||||
l_partkey = p_partkey
|
||||
and l_shipdate >= toDate('1995-09-01')
|
||||
and l_shipdate < toDate('1995-09-01') + interval '1' month;
|
||||
|
||||
-- select 15;
|
||||
-- create view revenue0 as
|
||||
-- select
|
||||
-- l_suppkey,
|
||||
-- sum(l_extendedprice * (1 - l_discount))
|
||||
-- from
|
||||
-- lineitem
|
||||
-- where
|
||||
-- l_shipdate >= toDate('1996-01-01')
|
||||
-- and l_shipdate < toDate('1996-01-01') + interval '3' month
|
||||
-- group by
|
||||
-- l_suppkey;
|
||||
-- select
|
||||
-- s_suppkey,
|
||||
-- s_name,
|
||||
-- s_address,
|
||||
-- s_phone,
|
||||
-- total_revenue
|
||||
-- from
|
||||
-- supplier,
|
||||
-- revenue0
|
||||
-- where
|
||||
-- s_suppkey = supplier_no
|
||||
-- and total_revenue = (
|
||||
-- select
|
||||
-- max(total_revenue)
|
||||
-- from
|
||||
-- revenue0
|
||||
-- )
|
||||
-- order by
|
||||
-- s_suppkey;
|
||||
-- drop view revenue0;
|
||||
|
||||
select 16;
|
||||
select
|
||||
p_brand,
|
||||
p_type,
|
||||
p_size,
|
||||
count(distinct ps_suppkey) as supplier_cnt
|
||||
from
|
||||
partsupp,
|
||||
part
|
||||
where
|
||||
p_partkey = ps_partkey
|
||||
and p_brand <> 'Brand#45'
|
||||
and p_type not like 'MEDIUM POLISHED%'
|
||||
and p_size in (49, 14, 23, 45, 19, 3, 36, 9)
|
||||
and ps_suppkey not in (
|
||||
select
|
||||
s_suppkey
|
||||
from
|
||||
supplier
|
||||
where
|
||||
s_comment like '%Customer%Complaints%'
|
||||
)
|
||||
group by
|
||||
p_brand,
|
||||
p_type,
|
||||
p_size
|
||||
order by
|
||||
supplier_cnt desc,
|
||||
p_brand,
|
||||
p_type,
|
||||
p_size;
|
||||
|
||||
-- select 17;
|
||||
-- select
|
||||
-- sum(l_extendedprice) / 7.0 as avg_yearly
|
||||
-- from
|
||||
-- lineitem,
|
||||
-- part
|
||||
-- where
|
||||
-- p_partkey = l_partkey
|
||||
-- and p_brand = 'Brand#23'
|
||||
-- and p_container = 'MED BOX'
|
||||
-- and l_quantity < (
|
||||
-- select
|
||||
-- 0.2 * avg(l_quantity)
|
||||
-- from
|
||||
-- lineitem
|
||||
-- where
|
||||
-- l_partkey = p_partkey
|
||||
-- );
|
||||
|
||||
select 18;
|
||||
select
|
||||
c_name,
|
||||
c_custkey,
|
||||
o_orderkey,
|
||||
o_orderdate,
|
||||
o_totalprice,
|
||||
sum(l_quantity)
|
||||
from
|
||||
customer,
|
||||
orders,
|
||||
lineitem
|
||||
where
|
||||
o_orderkey in (
|
||||
select
|
||||
l_orderkey
|
||||
from
|
||||
lineitem
|
||||
group by
|
||||
l_orderkey having
|
||||
sum(l_quantity) > 300
|
||||
)
|
||||
and c_custkey = o_custkey
|
||||
and o_orderkey = l_orderkey
|
||||
group by
|
||||
c_name,
|
||||
c_custkey,
|
||||
o_orderkey,
|
||||
o_orderdate,
|
||||
o_totalprice
|
||||
order by
|
||||
o_totalprice desc,
|
||||
o_orderdate
|
||||
limit 100;
|
||||
|
||||
select 19;
|
||||
select
|
||||
sum(l_extendedprice* (1 - l_discount)) as revenue
|
||||
from
|
||||
lineitem,
|
||||
part
|
||||
where
|
||||
(
|
||||
p_partkey = l_partkey
|
||||
and p_brand = 'Brand#12'
|
||||
and p_container in ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')
|
||||
and l_quantity >= 1 and l_quantity <= 1 + 10
|
||||
and p_size between 1 and 5
|
||||
and l_shipmode in ('AIR', 'AIR REG')
|
||||
and l_shipinstruct = 'DELIVER IN PERSON'
|
||||
)
|
||||
or
|
||||
(
|
||||
p_partkey = l_partkey
|
||||
and p_brand = 'Brand#23'
|
||||
and p_container in ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')
|
||||
and l_quantity >= 10 and l_quantity <= 10 + 10
|
||||
and p_size between 1 and 10
|
||||
and l_shipmode in ('AIR', 'AIR REG')
|
||||
and l_shipinstruct = 'DELIVER IN PERSON'
|
||||
)
|
||||
or
|
||||
(
|
||||
p_partkey = l_partkey
|
||||
and p_brand = 'Brand#34'
|
||||
and p_container in ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')
|
||||
and l_quantity >= 20 and l_quantity <= 20 + 10
|
||||
and p_size between 1 and 15
|
||||
and l_shipmode in ('AIR', 'AIR REG')
|
||||
and l_shipinstruct = 'DELIVER IN PERSON'
|
||||
);
|
||||
|
||||
-- select 20;
|
||||
-- select
|
||||
-- s_name,
|
||||
-- s_address
|
||||
-- from
|
||||
-- supplier,
|
||||
-- nation
|
||||
-- where
|
||||
-- s_suppkey in (
|
||||
-- select
|
||||
-- ps_suppkey
|
||||
-- from
|
||||
-- partsupp
|
||||
-- where
|
||||
-- ps_partkey in (
|
||||
-- select
|
||||
-- p_partkey
|
||||
-- from
|
||||
-- part
|
||||
-- where
|
||||
-- p_name like 'forest%'
|
||||
-- )
|
||||
-- and ps_availqty > (
|
||||
-- select
|
||||
-- 0.5 * sum(l_quantity)
|
||||
-- from
|
||||
-- lineitem
|
||||
-- where
|
||||
-- l_partkey = ps_partkey
|
||||
-- and l_suppkey = ps_suppkey
|
||||
-- and l_shipdate >= toDate('1994-01-01')
|
||||
-- and l_shipdate < toDate('1994-01-01') + interval '1' year
|
||||
-- )
|
||||
-- )
|
||||
-- and s_nationkey = n_nationkey
|
||||
-- and n_name = 'CANADA'
|
||||
-- order by
|
||||
-- s_name;
|
||||
|
||||
-- select 21;
|
||||
-- select
|
||||
-- s_name,
|
||||
-- count(*) as numwait
|
||||
-- from
|
||||
-- supplier,
|
||||
-- lineitem l1,
|
||||
-- orders,
|
||||
-- nation
|
||||
-- where
|
||||
-- s_suppkey = l1.l_suppkey
|
||||
-- and o_orderkey = l1.l_orderkey
|
||||
-- and o_orderstatus = 'F'
|
||||
-- and l1.l_receiptdate > l1.l_commitdate
|
||||
-- and exists (
|
||||
-- select
|
||||
-- *
|
||||
-- from
|
||||
-- lineitem l2
|
||||
-- where
|
||||
-- l2.l_orderkey = l1.l_orderkey
|
||||
-- and l2.l_suppkey <> l1.l_suppkey
|
||||
-- )
|
||||
-- and not exists (
|
||||
-- select
|
||||
-- *
|
||||
-- from
|
||||
-- lineitem l3
|
||||
-- where
|
||||
-- l3.l_orderkey = l1.l_orderkey
|
||||
-- and l3.l_suppkey <> l1.l_suppkey
|
||||
-- and l3.l_receiptdate > l3.l_commitdate
|
||||
-- )
|
||||
-- and s_nationkey = n_nationkey
|
||||
-- and n_name = 'SAUDI ARABIA'
|
||||
-- group by
|
||||
-- s_name
|
||||
-- order by
|
||||
-- numwait desc,
|
||||
-- s_name
|
||||
-- limit 100;
|
||||
|
||||
-- select 22;
|
||||
-- select
|
||||
-- cntrycode,
|
||||
-- count(*) as numcust,
|
||||
-- sum(c_acctbal) as totacctbal
|
||||
-- from
|
||||
-- (
|
||||
-- select
|
||||
-- substring(c_phone from 1 for 2) as cntrycode,
|
||||
-- c_acctbal
|
||||
-- from
|
||||
-- customer
|
||||
-- where
|
||||
-- substring(c_phone from 1 for 2) in
|
||||
-- ('13', '31', '23', '29', '30', '18', '17')
|
||||
-- and c_acctbal > (
|
||||
-- select
|
||||
-- avg(c_acctbal)
|
||||
-- from
|
||||
-- customer
|
||||
-- where
|
||||
-- c_acctbal > 0.00
|
||||
-- and substring(c_phone from 1 for 2) in
|
||||
-- ('13', '31', '23', '29', '30', '18', '17')
|
||||
-- )
|
||||
-- and not exists (
|
||||
-- select
|
||||
-- *
|
||||
-- from
|
||||
-- orders
|
||||
-- where
|
||||
-- o_custkey = c_custkey
|
||||
-- )
|
||||
-- ) as custsale
|
||||
-- group by
|
||||
-- cntrycode
|
||||
-- order by
|
||||
-- cntrycode;
|
||||
|
||||
DROP TABLE part;
|
||||
DROP TABLE supplier;
|
||||
DROP TABLE partsupp;
|
||||
DROP TABLE customer;
|
||||
DROP TABLE orders;
|
||||
DROP TABLE lineitem;
|
||||
DROP TABLE nation;
|
||||
DROP TABLE region;
|
Loading…
Reference in New Issue
Block a user