Merge branch 'master' into dictionaries_ddl_loader

This commit is contained in:
alesapin 2019-10-25 10:58:04 +03:00
commit 04ca163518
41 changed files with 773 additions and 294 deletions

View File

@ -180,7 +180,21 @@
<port>9000</port>
</replica>
</shard>
</test_cluster_two_shards_localhost>
</test_cluster_two_shards_localhost>
<test_cluster_two_shards>
<shard>
<replica>
<host>127.0.0.1</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>127.0.0.2</host>
<port>9000</port>
</replica>
</shard>
</test_cluster_two_shards>
<test_shard_localhost_secure>
<shard>
<replica>

View File

@ -29,7 +29,7 @@
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/QueryNormalizer.h>
#include <Interpreters/misc.h>
#include <Interpreters/ActionsVisitor.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/Set.h>

View File

@ -164,7 +164,8 @@ size_t CollectJoinOnKeysMatcher::getTableForIdentifiers(std::vector<const ASTIde
/// Column name could be cropped to a short form in TranslateQualifiedNamesVisitor.
/// In this case it saves membership in IdentifierSemantic.
size_t membership = IdentifierSemantic::getMembership(*identifier);
auto opt = IdentifierSemantic::getMembership(*identifier);
size_t membership = opt ? (*opt + 1) : 0;
if (!membership)
{

View File

@ -4,7 +4,7 @@
#include <Interpreters/CrossToInnerJoinVisitor.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/QueryNormalizer.h> // for functionIsInOperator
#include <Interpreters/misc.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTIdentifier.h>
@ -90,9 +90,12 @@ public:
using TypeToVisit = const ASTFunction;
CheckExpressionVisitorData(const std::vector<JoinedTable> & tables_)
: tables(tables_)
: joined_tables(tables_)
, ands_only(true)
{}
{
for (auto & joined : joined_tables)
tables.push_back(joined.table);
}
void visit(const ASTFunction & node, const ASTPtr & ast)
{
@ -156,7 +159,8 @@ public:
}
private:
const std::vector<JoinedTable> & tables;
const std::vector<JoinedTable> & joined_tables;
std::vector<DatabaseAndTableWithAlias> tables;
std::map<size_t, std::vector<ASTPtr>> asts_to_join_on;
bool ands_only;
@ -180,31 +184,16 @@ private:
/// @return table position to attach expression to or 0.
size_t checkIdentifiers(const ASTIdentifier & left, const ASTIdentifier & right)
{
/// {best_match, best_table_pos}
std::pair<size_t, size_t> left_best{0, 0};
std::pair<size_t, size_t> right_best{0, 0};
size_t left_table_pos = 0;
bool left_match = IdentifierSemantic::chooseTable(left, tables, left_table_pos);
for (size_t i = 0; i < tables.size(); ++i)
size_t right_table_pos = 0;
bool right_match = IdentifierSemantic::chooseTable(right, tables, right_table_pos);
if (left_match && right_match && (left_table_pos != right_table_pos))
{
size_t match = IdentifierSemantic::canReferColumnToTable(left, tables[i].table);
if (match > left_best.first)
{
left_best.first = match;
left_best.second = i;
}
match = IdentifierSemantic::canReferColumnToTable(right, tables[i].table);
if (match > right_best.first)
{
right_best.first = match;
right_best.second = i;
}
}
if (left_best.first && right_best.first && (left_best.second != right_best.second))
{
size_t table_pos = std::max(left_best.second, right_best.second);
if (tables[table_pos].canAttachOnExpression())
size_t table_pos = std::max(left_table_pos, right_table_pos);
if (joined_tables[table_pos].canAttachOnExpression())
return table_pos;
}
return 0;
@ -212,20 +201,10 @@ private:
size_t checkIdentifier(const ASTIdentifier & identifier)
{
size_t best_match = 0;
size_t best_table_pos = 0;
bool match = IdentifierSemantic::chooseTable(identifier, tables, best_table_pos);
for (size_t i = 0; i < tables.size(); ++i)
{
size_t match = IdentifierSemantic::canReferColumnToTable(identifier, tables[i].table);
if (match > best_match)
{
best_match = match;
best_table_pos = i;
}
}
if (best_match && tables[best_table_pos].canAttachOnExpression())
if (match && joined_tables[best_table_pos].canAttachOnExpression())
return best_table_pos;
return 0;
}

View File

@ -6,7 +6,7 @@
#include <Parsers/ASTExpressionList.h>
#include <Interpreters/Context.h>
#include <Interpreters/QueryNormalizer.h>
#include <Interpreters/misc.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/ExecuteScalarSubqueriesVisitor.h>
#include <Interpreters/addTypeConversionToAST.h>

View File

@ -54,7 +54,7 @@
#include <Parsers/queryToString.h>
#include <Interpreters/interpretSubquery.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/QueryNormalizer.h>
#include <Interpreters/misc.h>
#include <Interpreters/ActionsVisitor.h>

View File

@ -28,17 +28,9 @@ void FindIdentifierBestTableData::visit(ASTIdentifier & identifier, ASTPtr &)
}
else
{
// FIXME: make a better matcher using `names`?
size_t best_match = 0;
for (const auto & table_names : tables)
{
if (size_t match = IdentifierSemantic::canReferColumnToTable(identifier, table_names.first))
if (match > best_match)
{
best_match = match;
best_table = &table_names.first;
}
}
size_t best_table_pos = 0;
if (IdentifierSemantic::chooseTable(identifier, tables, best_table_pos))
best_table = &tables[best_table_pos].first;
}
identifier_table.emplace_back(&identifier, best_table);

View File

@ -5,6 +5,61 @@
namespace DB
{
namespace ErrorCodes
{
extern const int AMBIGUOUS_COLUMN_NAME;
}
namespace
{
const DatabaseAndTableWithAlias & extractTable(const DatabaseAndTableWithAlias & table)
{
return table;
}
const DatabaseAndTableWithAlias & extractTable(const TableWithColumnNames & table)
{
return table.first;
}
template <typename T>
IdentifierSemantic::ColumnMatch tryChooseTable(const ASTIdentifier & identifier, const std::vector<T> & tables,
size_t & best_table_pos, bool allow_ambiguous)
{
using ColumnMatch = IdentifierSemantic::ColumnMatch;
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]));
if (match != ColumnMatch::NoMatch)
{
if (match > best_match)
{
best_match = match;
best_table_pos = i;
same_match = 0;
}
else if (match == best_match)
++same_match;
}
}
if ((best_match != ColumnMatch::NoMatch) && same_match)
{
if (!allow_ambiguous)
throw Exception("Ambiguous column '" + identifier.name + "'", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
return ColumnMatch::Ambiguous;
}
return best_match;
}
}
std::optional<String> IdentifierSemantic::getColumnName(const ASTIdentifier & node)
{
if (!node.semantic->special)
@ -37,26 +92,36 @@ std::optional<String> IdentifierSemantic::getTableName(const ASTPtr & ast)
return {};
}
void IdentifierSemantic::setNeedLongName(ASTIdentifier & identifier, bool value)
{
identifier.semantic->need_long_name = value;
}
bool IdentifierSemantic::canBeAlias(const ASTIdentifier & identifier)
{
return identifier.semantic->can_be_alias;
}
void IdentifierSemantic::setMembership(ASTIdentifier & identifier, size_t table_no)
void IdentifierSemantic::setMembership(ASTIdentifier & identifier, size_t table_pos)
{
identifier.semantic->membership = table_no;
identifier.semantic->membership = table_pos;
identifier.semantic->can_be_alias = false;
}
size_t IdentifierSemantic::getMembership(const ASTIdentifier & identifier)
std::optional<size_t> IdentifierSemantic::getMembership(const ASTIdentifier & identifier)
{
return identifier.semantic->membership;
}
bool IdentifierSemantic::chooseTable(const ASTIdentifier & identifier, const std::vector<DatabaseAndTableWithAlias> & tables,
size_t & best_table_pos, bool ambiguous)
{
static constexpr auto no_match = IdentifierSemantic::ColumnMatch::NoMatch;
return tryChooseTable<DatabaseAndTableWithAlias>(identifier, tables, best_table_pos, ambiguous) != no_match;
}
bool IdentifierSemantic::chooseTable(const ASTIdentifier & identifier, const std::vector<TableWithColumnNames> & tables,
size_t & best_table_pos, bool ambiguous)
{
static constexpr auto no_match = IdentifierSemantic::ColumnMatch::NoMatch;
return tryChooseTable<TableWithColumnNames>(identifier, tables, best_table_pos, ambiguous) != no_match;
}
std::pair<String, String> IdentifierSemantic::extractDatabaseAndTable(const ASTIdentifier & identifier)
{
if (identifier.name_parts.size() > 2)
@ -84,24 +149,49 @@ bool IdentifierSemantic::doesIdentifierBelongTo(const ASTIdentifier & identifier
return false;
}
size_t IdentifierSemantic::canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
IdentifierSemantic::ColumnMatch IdentifierSemantic::canReferColumnToTable(const ASTIdentifier & identifier,
const DatabaseAndTableWithAlias & db_and_table)
{
/// database.table.column
if (doesIdentifierBelongTo(identifier, db_and_table.database, db_and_table.table))
return 2;
return ColumnMatch::DbAndTable;
/// table.column or alias.column.
if (doesIdentifierBelongTo(identifier, db_and_table.table) ||
doesIdentifierBelongTo(identifier, db_and_table.alias))
return 1;
/// alias.column
if (doesIdentifierBelongTo(identifier, db_and_table.alias))
return ColumnMatch::TableAlias;
return 0;
/// table.column
if (doesIdentifierBelongTo(identifier, db_and_table.table))
{
if (!db_and_table.alias.empty())
return ColumnMatch::AliasedTableName;
else
return ColumnMatch::TableName;
}
return ColumnMatch::NoMatch;
}
/// Checks that ast is ASTIdentifier and remove num_qualifiers_to_strip components from left.
/// Example: 'database.table.name' -> (num_qualifiers_to_strip = 2) -> 'name'.
void IdentifierSemantic::setColumnShortName(ASTIdentifier & identifier, size_t to_strip)
/// Strip qualificators from left side of column name.
/// Example: 'database.table.name' -> 'name'.
void IdentifierSemantic::setColumnShortName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
{
auto match = IdentifierSemantic::canReferColumnToTable(identifier, db_and_table);
size_t to_strip = 0;
switch (match)
{
case ColumnMatch::TableName:
case ColumnMatch::AliasedTableName:
case ColumnMatch::TableAlias:
to_strip = 1;
break;
case ColumnMatch::DbAndTable:
to_strip = 2;
break;
default:
break;
}
if (!to_strip)
return;
@ -117,18 +207,6 @@ void IdentifierSemantic::setColumnShortName(ASTIdentifier & identifier, size_t t
identifier.name.swap(new_name);
}
void IdentifierSemantic::setColumnNormalName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
{
size_t match = IdentifierSemantic::canReferColumnToTable(identifier, db_and_table);
setColumnShortName(identifier, match);
if (match)
identifier.semantic->can_be_alias = false;
if (identifier.semantic->need_long_name)
setColumnLongName(identifier, db_and_table);
}
void IdentifierSemantic::setColumnLongName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
{
String prefix = db_and_table.getQualifiedNamePrefix();
@ -141,16 +219,4 @@ void IdentifierSemantic::setColumnLongName(ASTIdentifier & identifier, const Dat
}
}
String IdentifierSemantic::columnNormalName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
{
ASTPtr copy = identifier.clone();
setColumnNormalName(copy->as<ASTIdentifier &>(), db_and_table);
return copy->getAliasOrColumnName();
}
String IdentifierSemantic::columnLongName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
{
return db_and_table.getQualifiedNamePrefix() + identifier.shortName();
}
}

View File

@ -1,5 +1,7 @@
#pragma once
#include <optional>
#include <Parsers/ASTIdentifier.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
@ -9,14 +11,23 @@ namespace DB
struct IdentifierSemanticImpl
{
bool special = false; /// for now it's 'not a column': tables, subselects and some special stuff like FORMAT
bool need_long_name = false;/// if column presents in multiple tables we need qualified names
bool can_be_alias = true; /// if it's a cropped name it could not be an alias
size_t membership = 0; /// table position in join (starting from 1) detected by qualifier or 0 if not detected.
std::optional<size_t> membership; /// table position in join
};
/// Static calss to manipulate IdentifierSemanticImpl via ASTIdentifier
struct IdentifierSemantic
{
enum class ColumnMatch
{
NoMatch,
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
TableAlias, /// column qualified with table alias
Ambiguous,
};
/// @returns name for column identifiers
static std::optional<String> getColumnName(const ASTIdentifier & node);
static std::optional<String> getColumnName(const ASTPtr & ast);
@ -26,20 +37,20 @@ struct IdentifierSemantic
static std::optional<String> getTableName(const ASTPtr & ast);
static std::pair<String, String> extractDatabaseAndTable(const ASTIdentifier & identifier);
static size_t canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static String columnNormalName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static String columnLongName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static void setColumnNormalName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static ColumnMatch canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static void setColumnShortName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static void setColumnLongName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static void setNeedLongName(ASTIdentifier & identifier, bool); /// if set setColumnNormalName makes qualified name
static bool canBeAlias(const ASTIdentifier & identifier);
static void setMembership(ASTIdentifier & identifier, size_t table_no);
static size_t getMembership(const ASTIdentifier & identifier);
static void setMembership(ASTIdentifier &, size_t table_no);
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);
private:
static bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & database, const String & table);
static bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & table);
static void setColumnShortName(ASTIdentifier & identifier, size_t match);
};
}

View File

@ -147,7 +147,13 @@ struct ColumnAliasesMatcher
auto it = rev_aliases.find(long_name);
if (it == rev_aliases.end())
{
bool last_table = IdentifierSemantic::canReferColumnToTable(*identifier, tables.back());
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 (!last_table)
{
String alias = hide_prefix + long_name;
@ -202,17 +208,15 @@ struct ColumnAliasesMatcher
bool last_table = false;
String long_name;
for (auto & table : data.tables)
size_t table_pos = 0;
if (IdentifierSemantic::chooseTable(node, data.tables, table_pos))
{
if (IdentifierSemantic::canReferColumnToTable(node, table))
{
if (!long_name.empty())
throw Exception("Cannot refer column '" + node.name + "' to one table", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
IdentifierSemantic::setColumnLongName(node, table); /// table_name.column_name -> table_alias.column_name
long_name = node.name;
if (&table == &data.tables.back())
last_table = true;
}
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())
last_table = true;
}
if (long_name.empty())

View File

@ -0,0 +1,47 @@
#include <Poco/String.h>
#include <Interpreters/misc.h>
#include <Interpreters/MarkTableIdentifiersVisitor.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
namespace DB
{
bool MarkTableIdentifiersMatcher::needChildVisit(ASTPtr & node, const ASTPtr & child)
{
if (child->as<ASTSelectQuery>())
return false;
if (node->as<ASTTableExpression>())
return false;
return true;
}
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)
{
/// `IN t` can be specified, where t is a table, which is equivalent to `IN (SELECT * FROM t)`.
if (functionIsInOrGlobalInOperator(func.name))
{
auto & ast = func.arguments->children.at(1);
if (auto opt_name = tryGetIdentifierName(ast))
if (!data.aliases.count(*opt_name))
setIdentifierSpecial(ast);
}
}
}

View File

@ -0,0 +1,33 @@
#pragma once
#include <Parsers/IAST.h>
#include <Interpreters/Aliases.h>
#include <Interpreters/InDepthNodeVisitor.h>
namespace DB
{
class ASTFunction;
struct ASTTableExpression;
class MarkTableIdentifiersMatcher
{
public:
using Visitor = InDepthNodeVisitor<MarkTableIdentifiersMatcher, true>;
struct Data
{
const Aliases & aliases;
};
static bool needChildVisit(ASTPtr & node, const ASTPtr & child);
static void visit(ASTPtr & ast, Data & data);
private:
static void visit(ASTTableExpression & table, ASTPtr &, Data &);
static void visit(const ASTFunction & func, ASTPtr &, Data &);
};
using MarkTableIdentifiersVisitor = MarkTableIdentifiersMatcher::Visitor;
}

View File

@ -21,6 +21,7 @@
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/QueryNormalizer.h>
#include <Interpreters/QueryAliasesVisitor.h>
#include <Interpreters/MarkTableIdentifiersVisitor.h>
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
#include <Interpreters/FindIdentifierBestTableVisitor.h>
#include <Interpreters/ExtractFunctionDataVisitor.h>
@ -412,6 +413,9 @@ ASTs PredicateExpressionsOptimizer::getSelectQueryProjectionColumns(ASTPtr & ast
QueryAliasesVisitor::Data query_aliases_data{aliases};
QueryAliasesVisitor(query_aliases_data).visit(ast);
MarkTableIdentifiersVisitor::Data mark_tables_data{aliases};
MarkTableIdentifiersVisitor(mark_tables_data).visit(ast);
QueryNormalizer::Data normalizer_data(aliases, settings);
QueryNormalizer(normalizer_data).visit(ast);

View File

@ -9,7 +9,6 @@
#include <Parsers/ASTQueryParameter.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/typeid_cast.h>
#include <Common/quoteString.h>
namespace DB
@ -63,34 +62,6 @@ private:
};
void QueryNormalizer::visit(ASTFunction & node, const ASTPtr &, Data & data)
{
auto & aliases = data.aliases;
String & func_name = node.name;
ASTPtr & func_arguments = node.arguments;
/// `IN t` can be specified, where t is a table, which is equivalent to `IN (SELECT * FROM t)`.
if (functionIsInOrGlobalInOperator(func_name))
{
auto & ast = func_arguments->children.at(1);
if (auto opt_name = tryGetIdentifierName(ast))
if (!aliases.count(*opt_name))
setIdentifierSpecial(ast);
}
/// Special cases for count function.
String func_name_lowercase = Poco::toLower(func_name);
if (startsWith(func_name_lowercase, "count"))
{
/// Select implementation of countDistinct based on settings.
/// Important that it is done as query rewrite. It means rewritten query
/// will be sent to remote servers during distributed query execution,
/// and on all remote servers, function implementation will be same.
if (endsWith(func_name, "Distinct") && func_name_lowercase == "countdistinct")
func_name = data.settings.count_distinct_implementation;
}
}
void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data)
{
auto & current_asts = data.current_asts;
@ -144,16 +115,8 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data)
}
}
/// mark table identifiers as 'not columns'
void QueryNormalizer::visit(ASTTablesInSelectQueryElement & node, const ASTPtr &, Data & data)
{
/// mark table Identifiers as 'not a column'
if (node.table_expression)
{
auto & expr = node.table_expression->as<ASTTableExpression &>();
setIdentifierSpecial(expr.database_and_table_name);
}
/// normalize JOIN ON section
if (node.table_join)
{
@ -177,7 +140,6 @@ void QueryNormalizer::visit(ASTSelectQuery & select, const ASTPtr &, Data & data
if (needVisitChild(child))
visit(child, data);
#if 1 /// TODO: legacy?
/// If the WHERE clause or HAVING consists of a single alias, the reference must be replaced not only in children,
/// but also in where_expression and having_expression.
if (select.prewhere())
@ -186,7 +148,6 @@ void QueryNormalizer::visit(ASTSelectQuery & select, const ASTPtr &, Data & data
visit(select.refWhere(), data);
if (select.having())
visit(select.refHaving(), data);
#endif
}
/// Don't go into subqueries.
@ -243,9 +204,7 @@ void QueryNormalizer::visit(ASTPtr & ast, Data & data)
data.current_alias = my_alias;
}
if (auto * node_func = ast->as<ASTFunction>())
visit(*node_func, ast, data);
else if (auto * node_id = ast->as<ASTIdentifier>())
if (auto * node_id = ast->as<ASTIdentifier>())
visit(*node_id, ast, data);
else if (auto * node_tables = ast->as<ASTTablesInSelectQueryElement>())
visit(*node_tables, ast, data);

View File

@ -2,25 +2,13 @@
#include <map>
#include <Core/Names.h>
#include <Parsers/IAST.h>
#include <Interpreters/Aliases.h>
namespace DB
{
inline bool functionIsInOperator(const String & name)
{
return name == "in" || name == "notIn";
}
inline bool functionIsInOrGlobalInOperator(const String & name)
{
return functionIsInOperator(name) || name == "globalIn" || name == "globalNotIn";
}
class ASTSelectQuery;
class ASTFunction;
class ASTIdentifier;
struct ASTTablesInSelectQueryElement;
class Context;
@ -33,13 +21,11 @@ class QueryNormalizer
{
const UInt64 max_ast_depth;
const UInt64 max_expanded_ast_elements;
const String count_distinct_implementation;
template <typename T>
ExtractedSettings(const T & settings)
: max_ast_depth(settings.max_ast_depth),
max_expanded_ast_elements(settings.max_expanded_ast_elements),
count_distinct_implementation(settings.count_distinct_implementation)
max_expanded_ast_elements(settings.max_expanded_ast_elements)
{}
};
@ -80,7 +66,6 @@ private:
static void visit(ASTPtr & query, Data & data);
static void visit(ASTIdentifier &, ASTPtr &, Data &);
static void visit(ASTFunction &, const ASTPtr &, Data &);
static void visit(ASTTablesInSelectQueryElement &, const ASTPtr &, Data &);
static void visit(ASTSelectQuery &, const ASTPtr &, Data &);

View File

@ -9,6 +9,7 @@
#include <Interpreters/ArrayJoinedColumnsVisitor.h>
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
#include <Interpreters/Context.h>
#include <Interpreters/MarkTableIdentifiersVisitor.h>
#include <Interpreters/QueryNormalizer.h>
#include <Interpreters/ExecuteScalarSubqueriesVisitor.h>
#include <Interpreters/PredicateExpressionsOptimizer.h>
@ -72,6 +73,26 @@ namespace
using LogAST = DebugASTLog<false>; /// set to true to enable logs
/// Select implementation of countDistinct based on settings.
/// Important that it is done as query rewrite. It means rewritten query
/// will be sent to remote servers during distributed query execution,
/// and on all remote servers, function implementation will be same.
struct CustomizeFunctionsData
{
using TypeToVisit = ASTFunction;
const String & count_distinct;
void visit(ASTFunction & func, ASTPtr &)
{
if (Poco::toLower(func.name) == "countdistinct")
func.name = count_distinct;
}
};
using CustomizeFunctionsMatcher = OneTypeMatcher<CustomizeFunctionsData>;
using CustomizeFunctionsVisitor = InDepthNodeVisitor<CustomizeFunctionsMatcher, true>;
/// Add columns from storage to source_columns list.
void collectSourceColumns(const ColumnsDescription & columns, NamesAndTypesList & source_columns, bool add_virtuals)
@ -91,7 +112,10 @@ void collectSourceColumns(const ColumnsDescription & columns, NamesAndTypesList
}
}
std::vector<TableWithColumnNames> getTablesWithColumns(const ASTSelectQuery & select_query, const Context & context)
std::vector<TableWithColumnNames> getTablesWithColumns(const ASTSelectQuery & select_query, const Context & context,
const ASTTablesInSelectQueryElement * table_join_node,
NamesAndTypesList & columns_from_joined_table,
std::function<Names()> get_column_names)
{
std::vector<TableWithColumnNames> tables_with_columns = getDatabaseAndTablesWithColumnNames(select_query, context);
@ -104,6 +128,27 @@ std::vector<TableWithColumnNames> getTablesWithColumns(const ASTSelectQuery & se
ErrorCodes::ALIAS_REQUIRED);
}
TableWithColumnNames joined_table;
if (table_join_node)
{
const auto & joined_expression = table_join_node->table_expression->as<ASTTableExpression &>();
columns_from_joined_table = getNamesAndTypeListFromTableExpression(joined_expression, context);
joined_table.first = DatabaseAndTableWithAlias(joined_expression, context.getCurrentDatabase());
for (const auto & column : columns_from_joined_table)
joined_table.second.push_back(column.name);
}
/// If empty make table(s) with list of source and joined columns
if (tables_with_columns.empty())
{
tables_with_columns.emplace_back(DatabaseAndTableWithAlias{}, get_column_names());
if (!joined_table.second.empty())
tables_with_columns.emplace_back(std::move(joined_table));
}
return tables_with_columns;
}
@ -541,11 +586,16 @@ void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery & s
}
}
void replaceJoinedTable(const ASTTablesInSelectQueryElement* join)
void replaceJoinedTable(const ASTTablesInSelectQueryElement * join)
{
if (!join || !join->table_expression)
return;
/// TODO: Push down for CROSS JOIN is not OK [disabled]
const auto & table_join = join->table_join->as<ASTTableJoin &>();
if (table_join.kind == ASTTableJoin::Kind::Cross)
return;
auto & table_expr = join->table_expression->as<ASTTableExpression &>();
if (table_expr.database_and_table_name)
{
@ -805,41 +855,34 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
if (remove_duplicates)
renameDuplicatedColumns(select_query);
if (const ASTTablesInSelectQueryElement * node = select_query->join())
const ASTTablesInSelectQueryElement * table_join_node = select_query->join();
if (table_join_node)
{
if (!settings.any_join_distinct_right_table_keys)
checkJoin(node);
checkJoin(table_join_node);
if (settings.enable_optimize_predicate_expression)
replaceJoinedTable(node);
const auto & joined_expression = node->table_expression->as<ASTTableExpression &>();
DatabaseAndTableWithAlias table(joined_expression, context.getCurrentDatabase());
result.analyzed_join->columns_from_joined_table = getNamesAndTypeListFromTableExpression(joined_expression, context);
result.analyzed_join->deduplicateAndQualifyColumnNames(source_columns_set, table.getQualifiedNamePrefix());
replaceJoinedTable(table_join_node);
}
auto tables_with_columns = getTablesWithColumns(*select_query, context);
/// If empty make fake table with list of source and joined columns
if (tables_with_columns.empty())
auto get_column_names = [&]() -> Names
{
Names columns_list;
if (storage)
columns_list = storage->getColumns().getOrdinary().getNames();
else
{
columns_list.reserve(result.source_columns.size());
for (const auto & column : result.source_columns)
columns_list.emplace_back(column.name);
}
return storage->getColumns().getOrdinary().getNames();
for (auto & column : result.analyzed_join->getQualifiedColumnsSet())
columns_list.emplace_back(column);
Names columns;
columns.reserve(result.source_columns.size());
for (const auto & column : result.source_columns)
columns.push_back(column.name);
return columns;
};
tables_with_columns.emplace_back(DatabaseAndTableWithAlias{}, std::move(columns_list));
}
auto tables_with_columns = getTablesWithColumns(*select_query, context, table_join_node,
result.analyzed_join->columns_from_joined_table, get_column_names);
if (tables_with_columns.size() > 1)
result.analyzed_join->deduplicateAndQualifyColumnNames(
source_columns_set, tables_with_columns[1].first.getQualifiedNamePrefix());
translateQualifiedNames(query, *select_query, source_columns_set, std::move(tables_with_columns));
@ -850,6 +893,11 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
LogicalExpressionsOptimizer(select_query, settings.optimize_min_equality_disjunction_chain_length.value).perform();
}
{
CustomizeFunctionsVisitor::Data data{settings.count_distinct_implementation};
CustomizeFunctionsVisitor(data).visit(query);
}
/// Creates a dictionary `aliases`: alias -> ASTPtr
{
LogAST log;
@ -857,6 +905,12 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
QueryAliasesVisitor(query_aliases_data, log.stream()).visit(query);
}
/// Mark table ASTIdentifiers with not a column marker
{
MarkTableIdentifiersVisitor::Data data{result.aliases};
MarkTableIdentifiersVisitor(data).visit(query);
}
/// Common subexpression elimination. Rewrite rules.
{
QueryNormalizer::Data normalizer_data(result.aliases, context.getSettingsRef());

View File

@ -61,24 +61,20 @@ void TranslateQualifiedNamesMatcher::visit(ASTIdentifier & identifier, ASTPtr &,
{
if (IdentifierSemantic::getColumnName(identifier))
{
size_t best_table_pos = 0;
size_t best_match = 0;
for (size_t i = 0; i < data.tables.size(); ++i)
if (size_t match = IdentifierSemantic::canReferColumnToTable(identifier, data.tables[i].first))
if (match > best_match)
{
best_match = match;
best_table_pos = i;
}
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))
{
IdentifierSemantic::setMembership(identifier, table_pos);
if (best_match)
IdentifierSemantic::setMembership(identifier, best_table_pos + 1);
/// In case if column from the joined table are in source columns, change it's name to qualified.
if (best_table_pos && data.source_columns.count(identifier.shortName()))
IdentifierSemantic::setNeedLongName(identifier, true);
if (!data.tables.empty())
IdentifierSemantic::setColumnNormalName(identifier, data.tables[best_table_pos].first);
/// In case if column from the joined table are in source columns, change it's name to qualified.
auto & table = data.tables[table_pos].first;
if (table_pos && data.hasColumn(short_name))
IdentifierSemantic::setColumnLongName(identifier, table);
else
IdentifierSemantic::setColumnShortName(identifier, table);
}
}
}
@ -134,8 +130,10 @@ void TranslateQualifiedNamesMatcher::visit(ASTSelectQuery & select, const ASTPtr
Visitor(data).visit(select.refHaving());
}
static void addIdentifier(ASTs & nodes, const String & table_name, const String & column_name, AsteriskSemantic::RevertedAliasesPtr aliases)
static void addIdentifier(ASTs & nodes, const DatabaseAndTableWithAlias & table, const String & column_name,
AsteriskSemantic::RevertedAliasesPtr aliases)
{
String table_name = table.getQualifiedNamePrefix(false);
auto identifier = std::make_shared<ASTIdentifier>(std::vector<String>{table_name, column_name});
bool added = false;
@ -197,8 +195,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
{
if (first_table || !data.join_using_columns.count(column_name))
{
String table_name = table.getQualifiedNamePrefix(false);
addIdentifier(node.children, table_name, column_name, AsteriskSemantic::getAliases(*asterisk));
addIdentifier(node.children, table, column_name, AsteriskSemantic::getAliases(*asterisk));
}
}
@ -214,8 +211,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
{
if (asterisk_pattern->isColumnMatching(column_name) && (first_table || !data.join_using_columns.count(column_name)))
{
String table_name = table.getQualifiedNamePrefix(false);
addIdentifier(node.children, table_name, column_name, AsteriskSemantic::getAliases(*asterisk_pattern));
addIdentifier(node.children, table, column_name, AsteriskSemantic::getAliases(*asterisk_pattern));
}
}
@ -232,8 +228,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
{
for (const auto & column_name : table_columns)
{
String table_name = table.getQualifiedNamePrefix(false);
addIdentifier(node.children, table_name, column_name, AsteriskSemantic::getAliases(*qualified_asterisk));
addIdentifier(node.children, table, column_name, AsteriskSemantic::getAliases(*qualified_asterisk));
}
break;
}
@ -269,11 +264,13 @@ void TranslateQualifiedNamesMatcher::extractJoinUsingColumns(const ASTPtr ast, D
void RestoreQualifiedNamesData::visit(ASTIdentifier & identifier, ASTPtr & ast)
{
if (IdentifierSemantic::getColumnName(identifier) &&
IdentifierSemantic::getMembership(identifier))
if (IdentifierSemantic::getColumnName(identifier))
{
ast = identifier.clone();
ast->as<ASTIdentifier>()->restoreCompoundName();
if (IdentifierSemantic::getMembership(identifier))
{
ast = identifier.clone();
ast->as<ASTIdentifier>()->restoreCompoundName();
}
}
}

View File

@ -35,6 +35,10 @@ public:
, has_columns(has_columns_)
{}
bool hasColumn(const String & name) const { return source_columns.count(name); }
bool hasTable() const { return !tables.empty(); }
bool processAsterisks() const { return hasTable() && has_columns; }
static std::vector<TableWithColumnNames> tablesOnly(const std::vector<DatabaseAndTableWithAlias> & tables)
{
std::vector<TableWithColumnNames> tables_with_columns;
@ -44,8 +48,6 @@ public:
tables_with_columns.emplace_back(TableWithColumnNames{table, {}});
return tables_with_columns;
}
bool processAsterisks() const { return !tables.empty() && has_columns; }
};
static void visit(ASTPtr & ast, Data & data);

View File

@ -0,0 +1,16 @@
#pragma once
namespace DB
{
inline bool functionIsInOperator(const std::string & name)
{
return name == "in" || name == "notIn";
}
inline bool functionIsInOrGlobalInOperator(const std::string & name)
{
return functionIsInOperator(name) || name == "globalIn" || name == "globalNotIn";
}
}

View File

@ -34,10 +34,20 @@ ASTIdentifier::ASTIdentifier(const String & name_, std::vector<String> && name_p
, name_parts(name_parts_)
, semantic(std::make_shared<IdentifierSemanticImpl>())
{
if (name_parts.size() && name_parts[0] == "")
name_parts.erase(name_parts.begin());
if (name == "")
{
if (name_parts.size() == 2)
name = name_parts[0] + '.' + name_parts[1];
else if (name_parts.size() == 1)
name = name_parts[0];
}
}
ASTIdentifier::ASTIdentifier(std::vector<String> && name_parts_)
: ASTIdentifier(name_parts_.at(0) + '.' + name_parts_.at(1), std::move(name_parts_))
: ASTIdentifier("", std::move(name_parts_))
{}
void ASTIdentifier::setShortName(const String & new_name)

View File

@ -81,12 +81,27 @@ void DistributedBlockOutputStream::writePrefix()
void DistributedBlockOutputStream::write(const Block & block)
{
if (insert_sync)
writeSync(block);
else
writeAsync(block);
}
Block ordinary_block{ block };
/* They are added by the AddingDefaultBlockOutputStream, and we will get
* different number of columns eventually */
for (const auto & col : storage.getColumns().getMaterialized())
{
if (ordinary_block.has(col.name))
{
ordinary_block.erase(col.name);
LOG_DEBUG(log, storage.getTableName()
<< ": column " + col.name + " will be removed, "
<< "because it is MATERIALIZED");
}
}
if (insert_sync)
writeSync(ordinary_block);
else
writeAsync(ordinary_block);
}
void DistributedBlockOutputStream::writeAsync(const Block & block)
{

View File

@ -4,7 +4,7 @@
#include <Interpreters/SyntaxAnalyzer.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/QueryNormalizer.h>
#include <Interpreters/misc.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <Common/FieldVisitors.h>

View File

@ -2723,10 +2723,20 @@ void MergeTreeData::movePartitionToDisk(const ASTPtr & partition, const String &
if (!disk)
throw Exception("Disk " + name + " does not exists on policy " + storage_policy->getName(), ErrorCodes::UNKNOWN_DISK);
for (const auto & part : parts)
parts.erase(std::remove_if(parts.begin(), parts.end(), [&](auto part_ptr)
{
return part_ptr->disk->getName() == disk->getName();
}), parts.end());
if (parts.empty())
{
if (part->disk->getName() == disk->getName())
throw Exception("Part " + part->name + " already on disk " + name, ErrorCodes::UNKNOWN_DISK);
String no_parts_to_move_message;
if (moving_part)
no_parts_to_move_message = "Part '" + partition_id + "' is already on disk '" + disk->getName() + "'";
else
no_parts_to_move_message = "All parts of partition '" + partition_id + "' are already on disk '" + disk->getName() + "'";
throw Exception(no_parts_to_move_message, ErrorCodes::UNKNOWN_DISK);
}
if (!movePartsToSpace(parts, std::static_pointer_cast<const DiskSpace::Space>(disk)))
@ -2758,10 +2768,28 @@ void MergeTreeData::movePartitionToVolume(const ASTPtr & partition, const String
if (!volume)
throw Exception("Volume " + name + " does not exists on policy " + storage_policy->getName(), ErrorCodes::UNKNOWN_DISK);
for (const auto & part : parts)
for (const auto & disk : volume->disks)
if (part->disk->getName() == disk->getName())
throw Exception("Part " + part->name + " already on volume '" + name + "'", ErrorCodes::UNKNOWN_DISK);
parts.erase(std::remove_if(parts.begin(), parts.end(), [&](auto part_ptr)
{
for (const auto & disk : volume->disks)
{
if (part_ptr->disk->getName() == disk->getName())
{
return true;
}
}
return false;
}), parts.end());
if (parts.empty())
{
String no_parts_to_move_message;
if (moving_part)
no_parts_to_move_message = "Part '" + partition_id + "' is already on volume '" + volume->getName() + "'";
else
no_parts_to_move_message = "All parts of partition '" + partition_id + "' are already on volume '" + volume->getName() + "'";
throw Exception(no_parts_to_move_message, ErrorCodes::UNKNOWN_DISK);
}
if (!movePartsToSpace(parts, std::static_pointer_cast<const DiskSpace::Space>(volume)))
throw Exception("Cannot move parts because moves are manually disabled.", ErrorCodes::ABORTED);

View File

@ -1,5 +1,5 @@
#include <Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h>
#include <Interpreters/QueryNormalizer.h>
#include <Interpreters/misc.h>
#include <Interpreters/BloomFilterHash.h>
#include <Common/HashTable/ClearableHashMap.h>
#include <Storages/MergeTree/RPNBuilder.h>

View File

@ -8,7 +8,7 @@
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/SyntaxAnalyzer.h>
#include <Interpreters/QueryNormalizer.h>
#include <Interpreters/misc.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/RPNBuilder.h>
#include <Parsers/ASTIdentifier.h>

View File

@ -9,7 +9,7 @@
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/formatAST.h>
#include <Interpreters/QueryNormalizer.h>
#include <Interpreters/misc.h>
#include <Common/typeid_cast.h>
#include <DataTypes/NestedUtils.h>
#include <ext/map.h>

View File

@ -1,7 +1,6 @@
#include <Storages/StorageDistributed.h>
#include <DataStreams/OneBlockInputStream.h>
#include <DataStreams/materializeBlock.h>
#include <Databases/IDatabase.h>

View File

@ -74,6 +74,27 @@
</volumes>
</default_disk_with_external>
<!-- special policy for checking validation of `max_data_part_size` -->
<special_warning_policy>
<volumes>
<special_warning_zero_volume>
<disk>default</disk>
<max_data_part_size_bytes>0</max_data_part_size_bytes>
</special_warning_zero_volume>
<special_warning_default_volume>
<disk>external</disk>
</special_warning_default_volume>
<special_warning_small_volume>
<disk>jbod1</disk>
<max_data_part_size_bytes>1024</max_data_part_size_bytes>
</special_warning_small_volume>
<special_warning_big_volume>
<disk>jbod2</disk>
<max_data_part_size_bytes>1024000000</max_data_part_size_bytes>
</special_warning_big_volume>
</volumes>
</special_warning_policy>
</policies>
</storage_configuration>

View File

@ -1,8 +1,9 @@
import time
import json
import pytest
import random
import re
import string
import json
import time
from multiprocessing.dummy import Pool
from helpers.client import QueryRuntimeException
from helpers.cluster import ClickHouseCluster
@ -128,6 +129,38 @@ def test_system_tables(start_cluster):
"max_data_part_size": "20971520",
"move_factor": 0.1,
},
{
"policy_name": "special_warning_policy",
"volume_name": "special_warning_zero_volume",
"volume_priority": "1",
"disks": ["default"],
"max_data_part_size": "0",
"move_factor": 0.1,
},
{
"policy_name": "special_warning_policy",
"volume_name": "special_warning_default_volume",
"volume_priority": "2",
"disks": ["external"],
"max_data_part_size": "0",
"move_factor": 0.1,
},
{
"policy_name": "special_warning_policy",
"volume_name": "special_warning_small_volume",
"volume_priority": "3",
"disks": ["jbod1"],
"max_data_part_size": "1024",
"move_factor": 0.1,
},
{
"policy_name": "special_warning_policy",
"volume_name": "special_warning_big_volume",
"volume_priority": "4",
"disks": ["jbod2"],
"max_data_part_size": "1024000000",
"move_factor": 0.1,
},
]
clickhouse_policies_data = json.loads(node1.query("SELECT * FROM system.storage_policies WHERE policy_name != 'default' FORMAT JSON"))["data"]
@ -193,6 +226,28 @@ def get_random_string(length):
def get_used_disks_for_table(node, table_name):
return node.query("select disk_name from system.parts where table == '{}' and active=1 order by modification_time".format(table_name)).strip().split('\n')
def test_no_warning_about_zero_max_data_part_size(start_cluster):
def get_log(node):
return node.exec_in_container(["bash", "-c", "cat /var/log/clickhouse-server/clickhouse-server.log"])
for node in (node1, node2):
node.query("""
CREATE TABLE default.test_warning_table (
s String
) ENGINE = MergeTree
ORDER BY tuple()
SETTINGS storage_policy='small_jbod_with_external'
""")
node.query("""
DROP TABLE default.test_warning_table
""")
log = get_log(node)
assert not re.search("Warning.*Volume.*special_warning_zero_volume", log)
assert not re.search("Warning.*Volume.*special_warning_default_volume", log)
assert re.search("Warning.*Volume.*special_warning_small_volume", log)
assert not re.search("Warning.*Volume.*special_warning_big_volume", log)
@pytest.mark.parametrize("name,engine", [
("mt_on_jbod","MergeTree()"),
("replicated_mt_on_jbod","ReplicatedMergeTree('/clickhouse/replicated_mt_on_jbod', '1')",),
@ -462,7 +517,7 @@ def test_alter_move(start_cluster, name, engine):
node1.query("INSERT INTO {} VALUES(toDate('2019-04-10'), 42)".format(name))
node1.query("INSERT INTO {} VALUES(toDate('2019-04-11'), 43)".format(name))
used_disks = get_used_disks_for_table(node1, name)
assert all(d.startswith("jbod") for d in used_disks), "All writes shoud go to jbods"
assert all(d.startswith("jbod") for d in used_disks), "All writes should go to jbods"
first_part = node1.query("SELECT name FROM system.parts WHERE table = '{}' and active = 1 ORDER BY modification_time LIMIT 1".format(name)).strip()
@ -498,6 +553,91 @@ def test_alter_move(start_cluster, name, engine):
finally:
node1.query("DROP TABLE IF EXISTS {name}".format(name=name))
@pytest.mark.parametrize("volume_or_disk", [
"DISK",
"VOLUME"
])
def test_alter_move_half_of_partition(start_cluster, volume_or_disk):
name = "alter_move_half_of_partition"
engine = "MergeTree()"
try:
node1.query("""
CREATE TABLE {name} (
EventDate Date,
number UInt64
) ENGINE = {engine}
ORDER BY tuple()
PARTITION BY toYYYYMM(EventDate)
SETTINGS storage_policy='jbods_with_external'
""".format(name=name, engine=engine))
node1.query("SYSTEM STOP MERGES {}".format(name))
node1.query("INSERT INTO {} VALUES(toDate('2019-03-15'), 65)".format(name))
node1.query("INSERT INTO {} VALUES(toDate('2019-03-16'), 42)".format(name))
used_disks = get_used_disks_for_table(node1, name)
assert all(d.startswith("jbod") for d in used_disks), "All writes should go to jbods"
time.sleep(1)
parts = node1.query("SELECT name FROM system.parts WHERE table = '{}' and active = 1".format(name)).splitlines()
assert len(parts) == 2
node1.query("ALTER TABLE {} MOVE PART '{}' TO VOLUME 'external'".format(name, parts[0]))
disks = node1.query("SELECT disk_name FROM system.parts WHERE table = '{}' and name = '{}' and active = 1".format(name, parts[0])).splitlines()
assert disks == ["external"]
time.sleep(1)
node1.query("ALTER TABLE {} MOVE PARTITION 201903 TO {volume_or_disk} 'external'".format(name, volume_or_disk=volume_or_disk))
disks = node1.query("SELECT disk_name FROM system.parts WHERE table = '{}' and partition = '201903' and active = 1".format(name)).splitlines()
assert disks == ["external"]*2
assert node1.query("SELECT COUNT() FROM {}".format(name)) == "2\n"
finally:
node1.query("DROP TABLE IF EXISTS {name}".format(name=name))
@pytest.mark.parametrize("volume_or_disk", [
"DISK",
"VOLUME"
])
def test_alter_double_move_partition(start_cluster, volume_or_disk):
name = "alter_double_move_partition"
engine = "MergeTree()"
try:
node1.query("""
CREATE TABLE {name} (
EventDate Date,
number UInt64
) ENGINE = {engine}
ORDER BY tuple()
PARTITION BY toYYYYMM(EventDate)
SETTINGS storage_policy='jbods_with_external'
""".format(name=name, engine=engine))
node1.query("SYSTEM STOP MERGES {}".format(name))
node1.query("INSERT INTO {} VALUES(toDate('2019-03-15'), 65)".format(name))
node1.query("INSERT INTO {} VALUES(toDate('2019-03-16'), 42)".format(name))
used_disks = get_used_disks_for_table(node1, name)
assert all(d.startswith("jbod") for d in used_disks), "All writes should go to jbods"
time.sleep(1)
node1.query("ALTER TABLE {} MOVE PARTITION 201903 TO {volume_or_disk} 'external'".format(name, volume_or_disk=volume_or_disk))
disks = node1.query("SELECT disk_name FROM system.parts WHERE table = '{}' and partition = '201903' and active = 1".format(name)).splitlines()
assert disks == ["external"]*2
assert node1.query("SELECT COUNT() FROM {}".format(name)) == "2\n"
time.sleep(1)
with pytest.raises(QueryRuntimeException):
node1.query("ALTER TABLE {} MOVE PARTITION 201903 TO {volume_or_disk} 'external'".format(name, volume_or_disk=volume_or_disk))
finally:
node1.query("DROP TABLE IF EXISTS {name}".format(name=name))
def produce_alter_move(node, name):
move_type = random.choice(["PART", "PARTITION"])
if move_type == "PART":

View File

@ -56,26 +56,26 @@ comma nullable
1 1 1 1
2 2 1 2
cross
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE a = t2_00826.a
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826 ON a = t2_00826.a\nWHERE a = t2_00826.a
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE a = t2_00826.a
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON a = t2_00826.a\nWHERE a = t2_00826.a
cross nullable
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\n, \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE a = t2_00826.a
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826 ON a = t2_00826.a\nWHERE a = t2_00826.a
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\n, t2_00826\nWHERE a = t2_00826.a
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON a = t2_00826.a\nWHERE a = t2_00826.a
cross nullable vs not nullable
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE a = t2_00826.b
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826 ON a = t2_00826.b\nWHERE a = t2_00826.b
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE a = t2_00826.b
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON a = t2_00826.b\nWHERE a = t2_00826.b
cross self
SELECT \n a, \n b, \n y.a, \n y.b\nFROM t1_00826 AS x\nCROSS JOIN t1_00826 AS y\nWHERE (a = y.a) AND (b = y.b)
SELECT \n a, \n b, \n y.a, \n y.b\nFROM t1_00826 AS x\nALL INNER JOIN t1_00826 AS y ON (a = y.a) AND (b = y.b)\nWHERE (a = y.a) AND (b = y.b)
cross one table expr
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE a = b
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE a = b
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE a = b
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE a = b
cross multiple ands
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE (a = t2_00826.a) AND (b = t2_00826.b)
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND (b = t2_00826.b)
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE (a = t2_00826.a) AND (b = t2_00826.b)
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND (b = t2_00826.b)
cross and inside and
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b)))
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826 ON (a = t2_00826.a) AND (a = t2_00826.a) AND (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b)))
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b)))
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (a = t2_00826.a) AND (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b)))
cross split conjunction
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n WHERE b > 0\n) AS t2_00826\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) AND (a >= 1) AND (t2_00826.b > 0)
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n WHERE b > 0\n) AS t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) AND (a >= 1) AND (t2_00826.b > 0)
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) AND (a >= 1) AND (t2_00826.b > 0)
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) AND (a >= 1) AND (t2_00826.b > 0)

View File

@ -1,9 +1,10 @@
SET enable_debug_queries = 1;
SET enable_optimize_predicate_expression = 0;
set allow_experimental_cross_to_join_conversion = 0;
select * from system.one cross join system.one;
select * from system.one l cross join system.one r;
set allow_experimental_cross_to_join_conversion = 1;
select * from system.one cross join system.one;
select * from system.one l cross join system.one r;
DROP TABLE IF EXISTS t1_00826;
DROP TABLE IF EXISTS t2_00826;

View File

@ -1,17 +1,17 @@
SELECT a\nFROM t1_00849\nCROSS JOIN \n(\n SELECT *\n FROM t2_00849\n) AS t2_00849
SELECT a\nFROM t1_00849\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00849\n) AS t2_00849 ON a = t2_00849.a\nWHERE a = t2_00849.a
SELECT a\nFROM t1_00849\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00849\n) AS t2_00849 ON b = t2_00849.b\nWHERE b = t2_00849.b
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n WHERE `--t1_00849.a` = `--t2_00849.a`\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t3_00849\n) AS t3_00849 ON `--t1_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = a)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b AS `--t1_00849.b`, \n t2_00849.a, \n t2_00849.b AS `--t2_00849.b`\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.b` = `--t2_00849.b`\n WHERE `--t1_00849.b` = `--t2_00849.b`\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t3_00849\n) AS t3_00849 ON `--t1_00849.b` = b\nWHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = b)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n WHERE `--t1_00849.a` = `--t2_00849.a`\n )\n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849 ON `--t1_00849.a` = `--t3_00849.a`\n WHERE (`--t1_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = `--t2_00849.a`)\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON `--t1_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = a)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n `--t1_00849.b`, \n `t2_00849.a`, \n `--t2_00849.b`, \n a, \n b AS `--t3_00849.b`\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b AS `--t1_00849.b`, \n t2_00849.a, \n t2_00849.b AS `--t2_00849.b`\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.b` = `--t2_00849.b`\n WHERE `--t1_00849.b` = `--t2_00849.b`\n )\n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849 ON `--t1_00849.b` = `--t3_00849.b`\n WHERE (`--t1_00849.b` = `--t3_00849.b`) AND (`--t1_00849.b` = `--t2_00849.b`)\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON `--t1_00849.b` = b\nWHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = `--t3_00849.b`) AND (`--t1_00849.b` = b)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t2_00849.a` = `--t1_00849.a`\n WHERE `--t2_00849.a` = `--t1_00849.a`\n )\n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849 ON `--t2_00849.a` = `--t3_00849.a`\n WHERE (`--t2_00849.a` = `--t3_00849.a`) AND (`--t2_00849.a` = `--t1_00849.a`)\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON `--t2_00849.a` = a\nWHERE (`--t2_00849.a` = `--t1_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t2_00849.a` = a)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849\n )\n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849 ON (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`)\n WHERE (`--t3_00849.a` = `--t2_00849.a`) AND (`--t3_00849.a` = `--t1_00849.a`)\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON `--t3_00849.a` = a\nWHERE (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`) AND (`--t3_00849.a` = a)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849\n )\n CROSS JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`)\nWHERE (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n WHERE `--t1_00849.a` = `--t2_00849.a`\n )\n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849 ON `--t2_00849.a` = `--t3_00849.a`\n WHERE (`--t2_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = `--t2_00849.a`)\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON `--t3_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t3_00849.a` = a)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `t2_00849.a`, \n `t2_00849.b`, \n a, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849\n )\n CROSS JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849\n)\nCROSS JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `t2_00849.a`, \n `t2_00849.b`, \n a, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849\n )\n CROSS JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849\n)\nCROSS JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n)\nCROSS JOIN \n(\n SELECT *\n FROM t3_00849\n) AS t3_00849
SELECT a\nFROM t1_00849\nCROSS JOIN t2_00849
SELECT a\nFROM t1_00849\nALL INNER JOIN t2_00849 ON a = t2_00849.a\nWHERE a = t2_00849.a
SELECT a\nFROM t1_00849\nALL INNER JOIN t2_00849 ON b = t2_00849.b\nWHERE b = t2_00849.b
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n)\nALL INNER JOIN t3_00849 ON `--t1_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = a)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b AS `--t1_00849.b`, \n t2_00849.a, \n t2_00849.b AS `--t2_00849.b`\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.b` = `--t2_00849.b`\n)\nALL INNER JOIN t3_00849 ON `--t1_00849.b` = b\nWHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = b)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n )\n ALL INNER JOIN t3_00849 ON `--t1_00849.a` = `--t3_00849.a`\n)\nALL INNER JOIN t4_00849 ON `--t1_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = a)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n `--t1_00849.b`, \n `t2_00849.a`, \n `--t2_00849.b`, \n a, \n b AS `--t3_00849.b`\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b AS `--t1_00849.b`, \n t2_00849.a, \n t2_00849.b AS `--t2_00849.b`\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.b` = `--t2_00849.b`\n )\n ALL INNER JOIN t3_00849 ON `--t1_00849.b` = `--t3_00849.b`\n)\nALL INNER JOIN t4_00849 ON `--t1_00849.b` = b\nWHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = `--t3_00849.b`) AND (`--t1_00849.b` = b)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t2_00849.a` = `--t1_00849.a`\n )\n ALL INNER JOIN t3_00849 ON `--t2_00849.a` = `--t3_00849.a`\n)\nALL INNER JOIN t4_00849 ON `--t2_00849.a` = a\nWHERE (`--t2_00849.a` = `--t1_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t2_00849.a` = a)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n )\n ALL INNER JOIN t3_00849 ON (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`)\n)\nALL INNER JOIN t4_00849 ON `--t3_00849.a` = a\nWHERE (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`) AND (`--t3_00849.a` = a)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n )\n CROSS JOIN t3_00849\n)\nALL INNER JOIN t4_00849 ON (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`)\nWHERE (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n )\n ALL INNER JOIN t3_00849 ON `--t2_00849.a` = `--t3_00849.a`\n)\nALL INNER JOIN t4_00849 ON `--t3_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t3_00849.a` = a)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `t2_00849.a`, \n `t2_00849.b`, \n a, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n )\n CROSS JOIN t3_00849\n)\nCROSS JOIN t4_00849
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `t2_00849.a`, \n `t2_00849.b`, \n a, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n )\n CROSS JOIN t3_00849\n)\nCROSS JOIN t4_00849
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n)\nCROSS JOIN t3_00849
SELECT * FROM t1, t2
1 1 1 1
1 1 1 \N

View File

@ -1,4 +1,5 @@
SET enable_debug_queries = 1;
SET enable_optimize_predicate_expression = 0;
SET joined_subquery_requires_alias = 0;
DROP TABLE IF EXISTS t1_00849;

View File

@ -1,3 +1,14 @@
insert_distributed_sync=0
2018-08-01
2018-08-01
2018-08-01 2017-08-01
2018-08-01 2017-08-01
2018-08-01
2018-08-01 2017-08-01
insert_distributed_sync=1
2018-08-01
2018-08-01
2018-08-01 2017-08-01
2018-08-01 2017-08-01
2018-08-01
2018-08-01 2017-08-01

View File

@ -1,15 +1,42 @@
DROP TABLE IF EXISTS local_00952;
DROP TABLE IF EXISTS distributed_00952;
CREATE TABLE local_00952 (date Date, value Date MATERIALIZED toDate('2017-08-01')) ENGINE = MergeTree(date, date, 8192);
CREATE TABLE distributed_00952 AS local_00952 ENGINE = Distributed('test_shard_localhost', currentDatabase(), local_00952, rand());
--
-- insert_distributed_sync=0
--
SELECT 'insert_distributed_sync=0';
SET insert_distributed_sync=0;
SET insert_distributed_sync=1;
CREATE TABLE local_00952 (date Date, value Date MATERIALIZED toDate('2017-08-01')) ENGINE = MergeTree(date, date, 8192);
CREATE TABLE distributed_00952 AS local_00952 ENGINE = Distributed('test_cluster_two_shards', currentDatabase(), local_00952, rand());
INSERT INTO distributed_00952 VALUES ('2018-08-01');
SYSTEM FLUSH DISTRIBUTED distributed_00952;
SELECT * FROM distributed_00952;
SELECT date, value FROM distributed_00952;
SELECT * FROM local_00952;
SELECT date, value FROM local_00952;
DROP TABLE distributed_00952;
DROP TABLE local_00952;
--
-- insert_distributed_sync=1
--
SELECT 'insert_distributed_sync=1';
SET insert_distributed_sync=1;
CREATE TABLE local_00952 (date Date, value Date MATERIALIZED toDate('2017-08-01')) ENGINE = MergeTree(date, date, 8192);
CREATE TABLE distributed_00952 AS local_00952 ENGINE = Distributed('test_cluster_two_shards', currentDatabase(), local_00952, rand());
INSERT INTO distributed_00952 VALUES ('2018-08-01');
SELECT * FROM distributed_00952;
SELECT date, value FROM distributed_00952;
SELECT * FROM local_00952;
SELECT date, value FROM local_00952;
DROP TABLE distributed_00952;
DROP TABLE local_00952;

View File

@ -0,0 +1,13 @@
0 0
0 0
0
0
0
0
┌─one.dummy─┬─A.dummy─┬─B.dummy─┐
│ 0 │ 0 │ 0 │
└───────────┴─────────┴─────────┘
┌─A.dummy─┬─one.dummy─┬─two.dummy─┐
│ 0 │ 0 │ 0 │
└─────────┴───────────┴───────────┘
0

View File

@ -0,0 +1,27 @@
select * from system.one cross join system.one; -- { serverError 352 }
select * from system.one cross join system.one r;
select * from system.one l cross join system.one;
select * from system.one left join system.one using dummy;
select dummy from system.one left join system.one using dummy;
USE system;
SELECT dummy FROM one AS A JOIN one ON A.dummy = one.dummy;
SELECT dummy FROM one JOIN one AS A ON A.dummy = one.dummy;
SELECT dummy FROM one l JOIN one r ON dummy = r.dummy; -- { serverError 352 }
SELECT dummy FROM one l JOIN one r ON l.dummy = dummy; -- { serverError 352 }
SELECT dummy FROM one l JOIN one r ON one.dummy = r.dummy; -- { serverError 352 }
SELECT dummy FROM one l JOIN one r ON l.dummy = one.dummy; -- { serverError 352 }
SELECT * from one
JOIN one A ON one.dummy = A.dummy
JOIN one B ON one.dummy = B.dummy
FORMAT PrettyCompact;
SELECT * from one A
JOIN system.one one ON A.dummy = one.dummy
JOIN system.one two ON A.dummy = two.dummy
FORMAT PrettyCompact;
-- SELECT one.dummy FROM one AS A FULL JOIN (SELECT 0 AS dymmy) AS one USING dummy;
SELECT one.dummy FROM one AS A JOIN (SELECT 0 AS dummy) B USING dummy;

View File

@ -75,6 +75,20 @@
</replica>
</shard>
</test_shard_localhost>
<test_cluster_two_shards>
<shard>
<replica>
<host>127.0.0.1</host>
<port>59000</port>
</replica>
</shard>
<shard>
<replica>
<host>127.0.0.2</host>
<port>59000</port>
</replica>
</shard>
</test_cluster_two_shards>
<test_cluster_two_shards_localhost>
<shard>
<replica>

View File

@ -57,20 +57,28 @@ RUN apt-get update -y \
rename \
wget
# Build and install tools for cross-linking to Darwin
ENV CC=clang-8
ENV CXX=clang++-8
# libtapi is required to support .tbh format from recent MacOS SDKs
RUN git clone https://github.com/tpoechtrager/apple-libtapi.git
RUN cd apple-libtapi && INSTALLPREFIX=/cctools ./build.sh && ./install.sh
RUN rm -rf apple-libtapi
# Build and install tools for cross-linking to Darwin
RUN git clone https://github.com/tpoechtrager/cctools-port.git
RUN cd cctools-port/cctools && ./configure --prefix=/cctools --with-libtapi=/cctools --target=x86_64-apple-darwin && make install
RUN rm -rf cctools-port
# Download toolchain for Darwin
RUN mkdir -p /build/cmake/toolchain/darwin-x86_64
RUN wget https://github.com/phracker/MacOSX-SDKs/releases/download/10.14-beta4/MacOSX10.14.sdk.tar.xz
RUN tar xJf MacOSX10.14.sdk.tar.xz -C /cctools
RUN tar xJf MacOSX10.14.sdk.tar.xz -C /build/cmake/toolchain/darwin-x86_64 --strip-components=1
# Download toolchain for ARM
RUN mkdir -p /build/cmake/toolchain/linux-aarch64
RUN wget "https://developer.arm.com/-/media/Files/downloads/gnu-a/8.3-2019.03/binrel/gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz?revision=2e88a73f-d233-4f96-b1f4-d8b36e9bb0b9&la=en" -O gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz
RUN tar xJf gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz -C /build/cmake/toolchain/linux-aarch64 --strip-components=1
COPY build.sh /
CMD ["/bin/bash", "/build.sh"]

View File

@ -174,7 +174,7 @@ Changes already made by the mutation are not rolled back.
## OPTIMIZE {#misc_operations-optimize}
```sql
OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition] [FINAL]
OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition | PARTITION ID 'partition_id'] [FINAL]
```
This query tries to initialize an unscheduled merge of data parts for tables with a table engine from the [MergeTree](../operations/table_engines/mergetree.md) family. Other kinds of table engines aren't supported.
@ -182,7 +182,7 @@ This query tries to initialize an unscheduled merge of data parts for tables wit
When `OPTIMIZE` is used with the [ReplicatedMergeTree](../operations/table_engines/replication.md) family of table engines, ClickHouse creates a task for merging and waits for execution on all nodes (if the `replication_alter_partitions_sync` setting is enabled).
- If `OPTIMIZE` doesn't perform a merge for any reason, it doesn't notify the client. To enable notifications, use the [optimize_throw_if_noop](../operations/settings/settings.md#setting-optimize_throw_if_noop) setting.
- If you specify a `PARTITION`, only the specified partition is optimized.
- If you specify a `PARTITION`, only the specified partition is optimized. [How to set partition expression](alter.md#alter-how-to-specify-part-expr).
- If you specify `FINAL`, optimization is performed even when all the data is already in one part.
!!! warning "Warning"

View File

@ -173,7 +173,7 @@ KILL MUTATION WHERE database = 'default' AND table = 'table' AND mutation_id = '
## OPTIMIZE {#misc_operations-optimize}
```sql
OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition] [FINAL]
OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition | PARTITION ID 'partition_id'] [FINAL]
```
Запрос пытается запустить внеплановый мёрж кусков данных для таблиц семейства [MergeTree](../operations/table_engines/mergetree.md). Другие движки таблиц не поддерживаются.
@ -181,7 +181,7 @@ OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition] [FINAL]
Если `OPTIMIZE` применяется к таблицам семейства [ReplicatedMergeTree](../operations/table_engines/replication.md), ClickHouse создаёт задачу на мёрж и ожидает её исполнения на всех узлах (если активирована настройка `replication_alter_partitions_sync`).
- Если `OPTIMIZE` не выполняет мёрж по любой причине, ClickHouse не оповещает об этом клиента. Чтобы включить оповещения, используйте настройку [optimize_throw_if_noop](../operations/settings/settings.md#setting-optimize_throw_if_noop).
- Если указать `PARTITION`, то оптимизация выполняется только для указанной партиции.
- Если указать `PARTITION`, то оптимизация выполняется только для указанной партиции. [Как задавать имя партиции в запросах](alter.md#alter-how-to-specify-part-expr).
- Если указать `FINAL`, то оптимизация выполняется даже в том случае, если все данные уже лежат в одном куске.
!!! warning "Внимание"