Merge branch 'master' of github.com:yandex/ClickHouse

This commit is contained in:
Ivan Blinkov 2018-11-10 14:35:32 +01:00
commit a0650e6d09
72 changed files with 2664 additions and 1453 deletions

View File

@ -1,3 +1,26 @@
## ClickHouse release 18.14.13, 2018-11-08
### Исправления ошибок:
* Исправлена ошибка `Block structure mismatch in MergingSorted stream`. [#3162](https://github.com/yandex/ClickHouse/issues/3162)
* Исправлена работа запросов `ON CLUSTER` в случае, когда в конфигурации кластера включено шифрование (флаг `<secure>`). [#3465](https://github.com/yandex/ClickHouse/pull/3465)
* Исправлена ошибка при использовании `SAMPLE`, `PREWHERE` и столбцов-алиасов. [#3543](https://github.com/yandex/ClickHouse/pull/3543)
* Исправлена редкая ошибка `unknown compression method` при использовании настройки `min_bytes_to_use_direct_io`. [3544](https://github.com/yandex/ClickHouse/pull/3544)
### Улучшения производительности:
* Исправлена деградация производительности запросов с `GROUP BY` столбцов типа Int16, Date на процессорах AMD EPYC. [Игорь Лапко](https://github.com/yandex/ClickHouse/pull/3512)
* Исправлена деградация производительности при обработке длинных строк. [#3530](https://github.com/yandex/ClickHouse/pull/3530)
### Улучшения процесса сборки ClickHouse:
* Доработки для упрощения сборки в Arcadia. [#3475](https://github.com/yandex/ClickHouse/pull/3475), [#3535](https://github.com/yandex/ClickHouse/pull/3535)
## ClickHouse release 18.14.12, 2018-11-02
### Исправления ошибок:
* Исправлена ошибка при join-запросе двух неименованных подзапросов. [#3505](https://github.com/yandex/ClickHouse/pull/3505)
* Исправлена генерация пустой `WHERE`-части при запросах к внешним базам. [hotid](https://github.com/yandex/ClickHouse/pull/3477)
* Исправлена ошибка использования неправильной настройки таймаута в ODBC-словарях. [Marek Vavruša](https://github.com/yandex/ClickHouse/pull/3511)
## ClickHouse release 18.14.11, 2018-10-29
### Исправления ошибок:

View File

@ -420,7 +420,7 @@ protected:
void destroyElements()
{
if (!std::is_trivially_destructible_v<Cell>)
for (iterator it = begin(); it != end(); ++it)
for (iterator it = begin(), it_end = end(); it != it_end; ++it)
it.ptr->~Cell();
}
@ -445,12 +445,15 @@ protected:
Derived & operator++()
{
/// If iterator was pointed to ZeroValueStorage, move it to the beginning of the main buffer.
if (unlikely(ptr->isZero(*container)))
ptr = container->buf;
else
++ptr;
while (ptr < container->buf + container->grower.bufSize() && ptr->isZero(*container))
/// Skip empty cells in the main buffer.
auto buf_end = container->buf + container->grower.bufSize();
while (ptr < buf_end && ptr->isZero(*container))
++ptr;
return static_cast<Derived &>(*this);
@ -569,12 +572,15 @@ public:
return iteratorToZero();
const Cell * ptr = buf;
while (ptr < buf + grower.bufSize() && ptr->isZero(*this))
auto buf_end = buf + grower.bufSize();
while (ptr < buf_end && ptr->isZero(*this))
++ptr;
return const_iterator(this, ptr);
}
const_iterator cbegin() const { return begin(); }
iterator begin()
{
if (!buf)
@ -584,13 +590,15 @@ public:
return iteratorToZero();
Cell * ptr = buf;
while (ptr < buf + grower.bufSize() && ptr->isZero(*this))
auto buf_end = buf + grower.bufSize();
while (ptr < buf_end && ptr->isZero(*this))
++ptr;
return iterator(this, ptr);
}
const_iterator end() const { return const_iterator(this, buf + grower.bufSize()); }
const_iterator cend() const { return end(); }
iterator end() { return iterator(this, buf + grower.bufSize()); }
@ -811,9 +819,9 @@ public:
if (this->hasZero())
this->zeroValue()->write(wb);
for (size_t i = 0; i < grower.bufSize(); ++i)
if (!buf[i].isZero(*this))
buf[i].write(wb);
for (auto ptr = buf, buf_end = buf + grower.bufSize(); ptr < buf_end; ++ptr)
if (!ptr->isZero(*this))
ptr->write(wb);
}
void writeText(DB::WriteBuffer & wb) const
@ -827,12 +835,12 @@ public:
this->zeroValue()->writeText(wb);
}
for (size_t i = 0; i < grower.bufSize(); ++i)
for (auto ptr = buf, buf_end = buf + grower.bufSize(); ptr < buf_end; ++ptr)
{
if (!buf[i].isZero(*this))
if (!ptr->isZero(*this))
{
DB::writeChar(',', wb);
buf[i].writeText(wb);
ptr->writeText(wb);
}
}
}

View File

@ -395,17 +395,16 @@ void ZooKeeper::read(T & x)
}
void ZooKeeperRequest::write(WriteBuffer & out) const
{
/// Excessive copy to calculate length.
WriteBufferFromOwnString buf;
Coordination::write(xid, buf);
Coordination::write(getOpNum(), buf);
writeImpl(buf);
Coordination::write(buf.str(), out);
out.next();
}
void ZooKeeperRequest::write(WriteBuffer & out) const
{
/// Excessive copy to calculate length.
WriteBufferFromOwnString buf;
Coordination::write(xid, buf);
Coordination::write(getOpNum(), buf);
writeImpl(buf);
Coordination::write(buf.str(), out);
out.next();
}
struct ZooKeeperResponse : virtual Response

View File

@ -584,6 +584,10 @@ template <> struct NearestFieldType<UInt8> { using Type = UInt64; };
template <> struct NearestFieldType<UInt16> { using Type = UInt64; };
template <> struct NearestFieldType<UInt32> { using Type = UInt64; };
template <> struct NearestFieldType<UInt64> { using Type = UInt64; };
#ifdef __APPLE__
template <> struct NearestFieldType<time_t> { using Type = UInt64; };
template <> struct NearestFieldType<size_t> { using Type = UInt64; };
#endif
template <> struct NearestFieldType<DayNum> { using Type = UInt64; };
template <> struct NearestFieldType<UInt128> { using Type = UInt128; };
template <> struct NearestFieldType<UUID> { using Type = UInt128; };

View File

@ -115,7 +115,7 @@ bool ReadBufferAIO::nextImpl()
/// If the end of the file is just reached, do nothing else.
if (is_eof)
return true;
return bytes_read != 0;
/// Create an asynchronous request.
prepare();

View File

@ -49,9 +49,11 @@ namespace ErrorCodes
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
/// defined in ExpressionAnalyser.cpp
NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypesList & cols);
NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypesList & cols)
{
return std::find_if(cols.begin(), cols.end(),
[&](const NamesAndTypesList::value_type & val) { return val.name == name; });
}
void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool create_ordered_set,
const Context & context, const SizeLimits & size_limits, PreparedSets & prepared_sets)

View File

@ -1,46 +1,115 @@
#pragma once
#include <Common/typeid_cast.h>
#include <Parsers/ASTQueryWithTableAndOutput.h>
#include <Parsers/ASTRenameQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/DumpASTNode.h>
namespace DB
{
/// Visits AST nodes, add default database to DDLs if not set.
/// Visitors consist of functions with unified interface 'void visit(Casted & x, ASTPtr & y)', there x is y, successfully casted to Casted.
/// Both types and fuction could have const specifiers. The second argument is used by visitor to replaces AST node (y) if needed.
/// Visits AST nodes, add default database to tables if not set. There's different logic for DDLs and selects.
class AddDefaultDatabaseVisitor
{
public:
AddDefaultDatabaseVisitor(const String & default_database_)
: default_database(default_database_)
AddDefaultDatabaseVisitor(const String & database_name_, std::ostream * ostr_ = nullptr)
: database_name(database_name_),
visit_depth(0),
ostr(ostr_)
{}
void visit(ASTPtr & ast) const
void visitDDL(ASTPtr & ast) const
{
visitChildren(ast);
visitDDLChildren(ast);
if (!tryVisit<ASTQueryWithTableAndOutput>(ast) &&
!tryVisit<ASTRenameQuery>(ast))
if (!tryVisitDynamicCast<ASTQueryWithTableAndOutput>(ast) &&
!tryVisitDynamicCast<ASTRenameQuery>(ast))
{}
}
private:
const String default_database;
void visit(ASTQueryWithTableAndOutput * node, ASTPtr &) const
void visit(ASTPtr & ast) const
{
if (node->database.empty())
node->database = default_database;
if (!tryVisit<ASTSelectQuery>(ast) &&
!tryVisit<ASTSelectWithUnionQuery>(ast))
visitChildren(ast);
}
void visit(ASTRenameQuery * node, ASTPtr &) const
void visit(ASTSelectQuery & select) const
{
for (ASTRenameQuery::Element & elem : node->elements)
ASTPtr unused;
visit(select, unused);
}
void visit(ASTSelectWithUnionQuery & select) const
{
ASTPtr unused;
visit(select, unused);
}
private:
const String database_name;
mutable size_t visit_depth;
std::ostream * ostr;
void visit(ASTSelectWithUnionQuery & select, ASTPtr &) const
{
for (auto & child : select.list_of_selects->children)
tryVisit<ASTSelectQuery>(child);
}
void visit(ASTSelectQuery & select, ASTPtr &) const
{
if (select.tables)
tryVisit<ASTTablesInSelectQuery>(select.tables);
if (select.prewhere_expression)
visitChildren(select.prewhere_expression);
if (select.where_expression)
visitChildren(select.where_expression);
}
void visit(ASTTablesInSelectQuery & tables, ASTPtr &) const
{
for (auto & child : tables.children)
tryVisit<ASTTablesInSelectQueryElement>(child);
}
void visit(ASTTablesInSelectQueryElement & tables_element, ASTPtr &) const
{
if (tables_element.table_expression)
tryVisit<ASTTableExpression>(tables_element.table_expression);
}
void visit(ASTTableExpression & table_expression, ASTPtr &) const
{
if (table_expression.database_and_table_name)
{
if (elem.from.database.empty())
elem.from.database = default_database;
if (elem.to.database.empty())
elem.to.database = default_database;
tryVisit<ASTIdentifier>(table_expression.database_and_table_name);
if (table_expression.database_and_table_name->children.size() != 2)
throw Exception("Logical error: more than two components in table expression", ErrorCodes::LOGICAL_ERROR);
}
else if (table_expression.subquery)
tryVisit<ASTSubquery>(table_expression.subquery);
}
void visit(const ASTIdentifier & identifier, ASTPtr & ast) const
{
if (ast->children.empty())
ast = createDatabaseAndTableNode(database_name, identifier.name);
}
void visit(ASTSubquery & subquery, ASTPtr &) const
{
tryVisit<ASTSelectWithUnionQuery>(subquery.children[0]);
}
void visitChildren(ASTPtr & ast) const
@ -51,10 +120,46 @@ private:
template <typename T>
bool tryVisit(ASTPtr & ast) const
{
if (T * t = typeid_cast<T *>(ast.get()))
{
DumpASTNode dump(*ast, ostr, visit_depth, "addDefaultDatabaseName");
visit(*t, ast);
return true;
}
return false;
}
void visitDDL(ASTQueryWithTableAndOutput & node, ASTPtr &) const
{
if (node.database.empty())
node.database = database_name;
}
void visitDDL(ASTRenameQuery & node, ASTPtr &) const
{
for (ASTRenameQuery::Element & elem : node.elements)
{
if (elem.from.database.empty())
elem.from.database = database_name;
if (elem.to.database.empty())
elem.to.database = database_name;
}
}
void visitDDLChildren(ASTPtr & ast) const
{
for (auto & child : ast->children)
visitDDL(child);
}
template <typename T>
bool tryVisitDynamicCast(ASTPtr & ast) const
{
if (T * t = dynamic_cast<T *>(ast.get()))
{
visit(t, ast);
visitDDL(*t, ast);
return true;
}
return false;

View File

@ -1475,9 +1475,9 @@ void NO_INLINE Aggregator::mergeDataImpl(
Table & table_src,
Arena * arena) const
{
for (auto it = table_src.begin(); it != table_src.end(); ++it)
for (auto it = table_src.begin(), end = table_src.end(); it != end; ++it)
{
decltype(it) res_it;
typename Table::iterator res_it;
bool inserted;
table_dst.emplace(it->first, res_it, inserted, it.getHash());
@ -1512,9 +1512,9 @@ void NO_INLINE Aggregator::mergeDataNoMoreKeysImpl(
Table & table_src,
Arena * arena) const
{
for (auto it = table_src.begin(); it != table_src.end(); ++it)
for (auto it = table_src.begin(), end = table_src.end(); it != end; ++it)
{
decltype(it) res_it = table_dst.find(it->first, it.getHash());
typename Table::iterator res_it = table_dst.find(it->first, it.getHash());
AggregateDataPtr res_data = table_dst.end() == res_it
? overflows

View File

@ -0,0 +1,127 @@
#include <Interpreters/AnalyzedJoin.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Storages/IStorage.h>
namespace DB
{
ExpressionActionsPtr AnalyzedJoin::createJoinedBlockActions(
const JoinedColumnsList & columns_added_by_join,
const ASTSelectQuery * select_query_with_join,
const Context & context,
NameSet & required_columns_from_joined_table) const
{
if (!select_query_with_join)
return nullptr;
const ASTTablesInSelectQueryElement * join = select_query_with_join->join();
if (!join)
return nullptr;
const auto & join_params = static_cast<const ASTTableJoin &>(*join->table_join);
/// Create custom expression list with join keys from right table.
auto expression_list = std::make_shared<ASTExpressionList>();
ASTs & children = expression_list->children;
if (join_params.on_expression)
for (const auto & join_right_key : key_asts_right)
children.emplace_back(join_right_key);
NameSet required_columns_set(key_names_right.begin(), key_names_right.end());
for (const auto & joined_column : columns_added_by_join)
required_columns_set.insert(joined_column.name_and_type.name);
Names required_columns(required_columns_set.begin(), required_columns_set.end());
NamesAndTypesList source_column_names;
for (auto & column : columns_from_joined_table)
source_column_names.emplace_back(column.name_and_type);
ExpressionAnalyzer analyzer(expression_list, context, nullptr, source_column_names, required_columns);
auto joined_block_actions = analyzer.getActions(false);
auto required_action_columns = joined_block_actions->getRequiredColumns();
required_columns_from_joined_table.insert(required_action_columns.begin(), required_action_columns.end());
auto sample = joined_block_actions->getSampleBlock();
for (auto & column : key_names_right)
if (!sample.has(column))
required_columns_from_joined_table.insert(column);
for (auto & column : columns_added_by_join)
if (!sample.has(column.name_and_type.name))
required_columns_from_joined_table.insert(column.name_and_type.name);
return joined_block_actions;
}
const JoinedColumnsList & AnalyzedJoin::getColumnsFromJoinedTable(
const NameSet & source_columns, const Context & context, const ASTSelectQuery * select_query_with_join)
{
if (select_query_with_join && columns_from_joined_table.empty())
{
if (const ASTTablesInSelectQueryElement * node = select_query_with_join->join())
{
const auto & table_expression = static_cast<const ASTTableExpression &>(*node->table_expression);
DatabaseAndTableWithAlias table_name_with_alias(table_expression, context.getCurrentDatabase());
auto columns = getNamesAndTypeListFromTableExpression(table_expression, context);
for (auto & column : columns)
{
JoinedColumn joined_column(column, column.name);
if (source_columns.count(column.name))
{
auto qualified_name = table_name_with_alias.getQualifiedNamePrefix() + column.name;
joined_column.name_and_type.name = qualified_name;
}
/// We don't want to select duplicate columns from the joined subquery if they appear
if (std::find(columns_from_joined_table.begin(), columns_from_joined_table.end(), joined_column) == columns_from_joined_table.end())
columns_from_joined_table.push_back(joined_column);
}
}
}
return columns_from_joined_table;
}
NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpression & table_expression, const Context & context)
{
NamesAndTypesList names_and_type_list;
if (table_expression.subquery)
{
const auto & subquery = table_expression.subquery->children.at(0);
names_and_type_list = InterpreterSelectWithUnionQuery::getSampleBlock(subquery, context).getNamesAndTypesList();
}
else if (table_expression.table_function)
{
const auto table_function = table_expression.table_function;
auto query_context = const_cast<Context *>(&context.getQueryContext());
const auto & function_storage = query_context->executeTableFunction(table_function);
names_and_type_list = function_storage->getSampleBlockNonMaterialized().getNamesAndTypesList();
}
else if (table_expression.database_and_table_name)
{
const auto & identifier = static_cast<const ASTIdentifier &>(*table_expression.database_and_table_name);
DatabaseAndTableWithAlias database_table(identifier);
const auto & table = context.getTable(database_table.database, database_table.table);
names_and_type_list = table->getSampleBlockNonMaterialized().getNamesAndTypesList();
}
return names_and_type_list;
}
}

View File

@ -0,0 +1,79 @@
#pragma once
#include <Core/Names.h>
#include <Core/NamesAndTypes.h>
#include <Parsers/IAST.h>
#include <utility>
#include <memory>
namespace DB
{
class Context;
class ASTSelectQuery;
class ExpressionActions;
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
struct JoinedColumn
{
/// Column will be joined to block.
NameAndTypePair name_and_type;
/// original column name from joined source.
String original_name;
JoinedColumn(NameAndTypePair name_and_type_, String original_name_)
: name_and_type(std::move(name_and_type_)), original_name(std::move(original_name_)) {}
bool operator==(const JoinedColumn & o) const
{
return name_and_type == o.name_and_type && original_name == o.original_name;
}
};
using JoinedColumnsList = std::list<JoinedColumn>;
struct AnalyzedJoin
{
/// NOTE: So far, only one JOIN per query is supported.
/** Query of the form `SELECT expr(x) AS k FROM t1 ANY LEFT JOIN (SELECT expr(x) AS k FROM t2) USING k`
* The join is made by column k.
* During the JOIN,
* - in the "right" table, it will be available by alias `k`, since `Project` action for the subquery was executed.
* - in the "left" table, it will be accessible by the name `expr(x)`, since `Project` action has not been executed yet.
* You must remember both of these options.
*
* Query of the form `SELECT ... from t1 ANY LEFT JOIN (SELECT ... from t2) ON expr(t1 columns) = expr(t2 columns)`
* to the subquery will be added expression `expr(t2 columns)`.
* It's possible to use name `expr(t2 columns)`.
*/
Names key_names_left;
Names key_names_right; /// Duplicating names are qualified.
ASTs key_asts_left;
ASTs key_asts_right;
/// All columns which can be read from joined table. Duplicating names are qualified.
JoinedColumnsList columns_from_joined_table;
/// Columns from joined table which may be added to block.
/// It's columns_from_joined_table without duplicate columns and possibly modified types.
JoinedColumnsList available_joined_columns;
ExpressionActionsPtr createJoinedBlockActions(
const JoinedColumnsList & columns_added_by_join, /// Subset of available_joined_columns.
const ASTSelectQuery * select_query_with_join,
const Context & context,
NameSet & required_columns_from_joined_table /// Columns which will be used in query from joined table.
) const;
const JoinedColumnsList & getColumnsFromJoinedTable(const NameSet & source_columns,
const Context & context,
const ASTSelectQuery * select_query_with_join);
};
struct ASTTableExpression;
NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpression & table_expression, const Context & context);
}

View File

@ -1,8 +1,20 @@
#pragma once
#include <Core/Names.h>
#include <Common/typeid_cast.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <DataTypes/NestedUtils.h>
namespace DB
{
/// Visitors consist of functions with unified interface 'void visit(Casted & x, ASTPtr & y)', there x is y, successfully casted to Casted.
/// Both types and fuction could have const specifiers. The second argument is used by visitor to replaces AST node (y) if needed.
/// Fills the array_join_result_to_source: on which columns-arrays to replicate, and how to call them after that.
class ArrayJoinedColumnsVisitor
{
@ -27,48 +39,48 @@ private:
NameToNameMap & array_join_alias_to_name;
NameToNameMap & array_join_result_to_source;
void visit(const ASTTablesInSelectQuery *, ASTPtr &) const
void visit(const ASTTablesInSelectQuery &, ASTPtr &) const
{}
void visit(const ASTIdentifier * node, ASTPtr &) const
void visit(const ASTIdentifier & node, ASTPtr &) const
{
if (node->general())
{
auto splitted = Nested::splitName(node->name); /// ParsedParams, Key1
if (!node.general())
return;
if (array_join_alias_to_name.count(node->name))
{
/// ARRAY JOIN was written with an array column. Example: SELECT K1 FROM ... ARRAY JOIN ParsedParams.Key1 AS K1
array_join_result_to_source[node->name] = array_join_alias_to_name[node->name]; /// K1 -> ParsedParams.Key1
}
else if (array_join_alias_to_name.count(splitted.first) && !splitted.second.empty())
{
/// ARRAY JOIN was written with a nested table. Example: SELECT PP.KEY1 FROM ... ARRAY JOIN ParsedParams AS PP
array_join_result_to_source[node->name] /// PP.Key1 -> ParsedParams.Key1
= Nested::concatenateName(array_join_alias_to_name[splitted.first], splitted.second);
}
else if (array_join_name_to_alias.count(node->name))
{
/** Example: SELECT ParsedParams.Key1 FROM ... ARRAY JOIN ParsedParams.Key1 AS PP.Key1.
* That is, the query uses the original array, replicated by itself.
*/
array_join_result_to_source[ /// PP.Key1 -> ParsedParams.Key1
array_join_name_to_alias[node->name]] = node->name;
}
else if (array_join_name_to_alias.count(splitted.first) && !splitted.second.empty())
{
/** Example: SELECT ParsedParams.Key1 FROM ... ARRAY JOIN ParsedParams AS PP.
*/
array_join_result_to_source[ /// PP.Key1 -> ParsedParams.Key1
Nested::concatenateName(array_join_name_to_alias[splitted.first], splitted.second)] = node->name;
}
auto splitted = Nested::splitName(node.name); /// ParsedParams, Key1
if (array_join_alias_to_name.count(node.name))
{
/// ARRAY JOIN was written with an array column. Example: SELECT K1 FROM ... ARRAY JOIN ParsedParams.Key1 AS K1
array_join_result_to_source[node.name] = array_join_alias_to_name[node.name]; /// K1 -> ParsedParams.Key1
}
else if (array_join_alias_to_name.count(splitted.first) && !splitted.second.empty())
{
/// ARRAY JOIN was written with a nested table. Example: SELECT PP.KEY1 FROM ... ARRAY JOIN ParsedParams AS PP
array_join_result_to_source[node.name] /// PP.Key1 -> ParsedParams.Key1
= Nested::concatenateName(array_join_alias_to_name[splitted.first], splitted.second);
}
else if (array_join_name_to_alias.count(node.name))
{
/** Example: SELECT ParsedParams.Key1 FROM ... ARRAY JOIN ParsedParams.Key1 AS PP.Key1.
* That is, the query uses the original array, replicated by itself.
*/
array_join_result_to_source[ /// PP.Key1 -> ParsedParams.Key1
array_join_name_to_alias[node.name]] = node.name;
}
else if (array_join_name_to_alias.count(splitted.first) && !splitted.second.empty())
{
/** Example: SELECT ParsedParams.Key1 FROM ... ARRAY JOIN ParsedParams AS PP.
*/
array_join_result_to_source[ /// PP.Key1 -> ParsedParams.Key1
Nested::concatenateName(array_join_name_to_alias[splitted.first], splitted.second)] = node.name;
}
}
void visit(const ASTSubquery *, ASTPtr &) const
void visit(const ASTSubquery &, ASTPtr &) const
{}
void visit(const ASTSelectQuery *, ASTPtr &) const
void visit(const ASTSelectQuery &, ASTPtr &) const
{}
void visitChildren(ASTPtr & ast) const
@ -84,7 +96,7 @@ private:
{
if (const T * t = typeid_cast<const T *>(ast.get()))
{
visit(t, ast);
visit(*t, ast);
return true;
}
return false;

View File

@ -1427,6 +1427,15 @@ UInt16 Context::getTCPPort() const
return config.getInt("tcp_port");
}
std::optional<UInt16> Context::getTCPPortSecure() const
{
auto lock = getLock();
auto & config = getConfigRef();
if (config.has("tcp_port_secure"))
return config.getInt("tcp_port_secure");
return {};
}
std::shared_ptr<Cluster> Context::getCluster(const std::string & cluster_name) const
{

View File

@ -7,6 +7,7 @@
#include <mutex>
#include <thread>
#include <atomic>
#include <optional>
#include <Common/config.h>
#include <common/MultiVersion.h>
@ -277,6 +278,8 @@ public:
/// The port that the server listens for executing SQL queries.
UInt16 getTCPPort() const;
std::optional<UInt16> getTCPPortSecure() const;
/// Get query for the CREATE table.
ASTPtr getCreateTableQuery(const String & database_name, const String & table_name) const;
ASTPtr getCreateExternalTableQuery(const String & table_name) const;

View File

@ -300,7 +300,9 @@ bool DDLWorker::initAndCheckTask(const String & entry_name, String & out_reason)
bool host_in_hostlist = false;
for (const HostID & host : task->entry.hosts)
{
if (!host.isLocalAddress(context.getTCPPort()))
auto maybe_secure_port = context.getTCPPortSecure();
bool is_local_port = maybe_secure_port ? host.isLocalAddress(*maybe_secure_port) : host.isLocalAddress(context.getTCPPort());
if (!is_local_port)
continue;
if (host_in_hostlist)
@ -477,7 +479,8 @@ void DDLWorker::parseQueryAndResolveHost(DDLTask & task)
{
const Cluster::Address & address = shards[shard_num][replica_num];
if (isLocalAddress(address.getResolvedAddress(), context.getTCPPort()))
if (isLocalAddress(address.getResolvedAddress(), context.getTCPPort())
|| (context.getTCPPortSecure() && isLocalAddress(address.getResolvedAddress(), *context.getTCPPortSecure())))
{
if (found_via_resolving)
{
@ -562,6 +565,7 @@ void DDLWorker::processTask(DDLTask & task)
String finished_node_path = task.entry_path + "/finished/" + task.host_id_str;
auto code = zookeeper->tryCreate(active_node_path, "", zkutil::CreateMode::Ephemeral, dummy);
if (code == Coordination::ZOK || code == Coordination::ZNODEEXISTS)
{
// Ok
@ -943,7 +947,7 @@ void DDLWorker::run()
}
catch (...)
{
LOG_ERROR(log, "Unexpected error: " << getCurrentExceptionMessage(true) << ". Terminating.");
tryLogCurrentException(log, "Unexpected error, will terminate:");
return;
}
}
@ -1056,7 +1060,7 @@ public:
Cluster::Address::fromString(host_id, host, port);
if (status.code != 0 && first_exception == nullptr)
first_exception = std::make_unique<Exception>("There was an error on " + host + ": " + status.message, status.code);
first_exception = std::make_unique<Exception>("There was an error on [" + host + ":" + toString(port) + "]: " + status.message, status.code);
++num_hosts_finished;
@ -1208,7 +1212,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & cont
if (use_local_default_db)
{
AddDefaultDatabaseVisitor visitor(current_database);
visitor.visit(query_ptr);
visitor.visitDDL(query_ptr);
}
DDLLogEntry entry;

View File

@ -35,7 +35,7 @@ static ASTPtr addTypeConversion(std::unique_ptr<ASTLiteral> && ast, const String
return res;
}
void ExecuteScalarSubqueriesVisitor::visit(const ASTSubquery * subquery, ASTPtr & ast, const DumpASTNode &) const
void ExecuteScalarSubqueriesVisitor::visit(const ASTSubquery & subquery, ASTPtr & ast) const
{
Context subquery_context = context;
Settings subquery_settings = context.getSettings();
@ -43,7 +43,7 @@ void ExecuteScalarSubqueriesVisitor::visit(const ASTSubquery * subquery, ASTPtr
subquery_settings.extremes = 0;
subquery_context.setSettings(subquery_settings);
ASTPtr subquery_select = subquery->children.at(0);
ASTPtr subquery_select = subquery.children.at(0);
BlockIO res = InterpreterSelectWithUnionQuery(
subquery_select, subquery_context, {}, QueryProcessingStage::Complete, subquery_depth + 1).execute();
@ -76,14 +76,14 @@ void ExecuteScalarSubqueriesVisitor::visit(const ASTSubquery * subquery, ASTPtr
if (columns == 1)
{
auto lit = std::make_unique<ASTLiteral>((*block.safeGetByPosition(0).column)[0]);
lit->alias = subquery->alias;
lit->prefer_alias_to_column_name = subquery->prefer_alias_to_column_name;
lit->alias = subquery.alias;
lit->prefer_alias_to_column_name = subquery.prefer_alias_to_column_name;
ast = addTypeConversion(std::move(lit), block.safeGetByPosition(0).type->getName());
}
else
{
auto tuple = std::make_shared<ASTFunction>();
tuple->alias = subquery->alias;
tuple->alias = subquery.alias;
ast = tuple;
tuple->name = "tuple";
auto exp_list = std::make_shared<ASTExpressionList>();
@ -101,26 +101,26 @@ void ExecuteScalarSubqueriesVisitor::visit(const ASTSubquery * subquery, ASTPtr
}
void ExecuteScalarSubqueriesVisitor::visit(const ASTTableExpression *, ASTPtr &, const DumpASTNode &) const
void ExecuteScalarSubqueriesVisitor::visit(const ASTTableExpression &, ASTPtr &) const
{
/// Don't descend into subqueries in FROM section.
}
void ExecuteScalarSubqueriesVisitor::visit(const ASTFunction * func, ASTPtr & ast, const DumpASTNode &) const
void ExecuteScalarSubqueriesVisitor::visit(const ASTFunction & func, ASTPtr & ast) const
{
/// Don't descend into subqueries in arguments of IN operator.
/// But if an argument is not subquery, than deeper may be scalar subqueries and we need to descend in them.
if (functionIsInOrGlobalInOperator(func->name))
if (functionIsInOrGlobalInOperator(func.name))
{
for (auto & child : ast->children)
{
if (child != func->arguments)
if (child != func.arguments)
visit(child);
else
for (size_t i = 0, size = func->arguments->children.size(); i < size; ++i)
if (i != 1 || !typeid_cast<ASTSubquery *>(func->arguments->children[i].get()))
visit(func->arguments->children[i]);
for (size_t i = 0, size = func.arguments->children.size(); i < size; ++i)
if (i != 1 || !typeid_cast<ASTSubquery *>(func.arguments->children[i].get()))
visit(func.arguments->children[i]);
}
}
else

View File

@ -11,6 +11,8 @@ class ASTSubquery;
class ASTFunction;
struct ASTTableExpression;
/// Visitors consist of functions with unified interface 'void visit(Casted & x, ASTPtr & y)', there x is y, successfully casted to Casted.
/// Both types and fuction could have const specifiers. The second argument is used by visitor to replaces AST node (y) if needed.
/** Replace subqueries that return exactly one row
* ("scalar" subqueries) to the corresponding constants.
@ -40,11 +42,9 @@ public:
void visit(ASTPtr & ast) const
{
DumpASTNode dump(*ast, ostr, visit_depth, "executeScalarSubqueries");
if (!tryVisit<ASTSubquery>(ast, dump) &&
!tryVisit<ASTTableExpression>(ast, dump) &&
!tryVisit<ASTFunction>(ast, dump))
if (!tryVisit<ASTSubquery>(ast) &&
!tryVisit<ASTTableExpression>(ast) &&
!tryVisit<ASTFunction>(ast))
visitChildren(ast);
}
@ -54,9 +54,9 @@ private:
mutable size_t visit_depth;
std::ostream * ostr;
void visit(const ASTSubquery * subquery, ASTPtr & ast, const DumpASTNode & dump) const;
void visit(const ASTFunction * func, ASTPtr & ast, const DumpASTNode &) const;
void visit(const ASTTableExpression *, ASTPtr &, const DumpASTNode &) const;
void visit(const ASTSubquery & subquery, ASTPtr & ast) const;
void visit(const ASTFunction & func, ASTPtr & ast) const;
void visit(const ASTTableExpression &, ASTPtr &) const;
void visitChildren(ASTPtr & ast) const
{
@ -65,11 +65,12 @@ private:
}
template <typename T>
bool tryVisit(ASTPtr & ast, const DumpASTNode & dump) const
bool tryVisit(ASTPtr & ast) const
{
if (const T * t = typeid_cast<const T *>(ast.get()))
{
visit(t, ast, dump);
DumpASTNode dump(*ast, ostr, visit_depth, "executeScalarSubqueries");
visit(*t, ast);
return true;
}
return false;

View File

@ -271,8 +271,6 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings)
const std::string & name = projection[i].first;
const std::string & alias = projection[i].second;
ColumnWithTypeAndName column = sample_block.getByName(name);
if (column.column)
column.column = (*std::move(column.column)).mutate();
if (alias != "")
column.name = alias;
new_block.insert(std::move(column));
@ -485,8 +483,6 @@ void ExpressionAction::execute(Block & block, std::unordered_map<std::string, si
const std::string & name = projection[i].first;
const std::string & alias = projection[i].second;
ColumnWithTypeAndName column = block.getByName(name);
if (column.column)
column.column = (*std::move(column.column)).mutate();
if (alias != "")
column.name = alias;
new_block.insert(std::move(column));

File diff suppressed because it is too large Load Diff

View File

@ -3,6 +3,7 @@
#include <Interpreters/AggregateDescription.h>
#include <Interpreters/Settings.h>
#include <Interpreters/ActionsVisitor.h>
#include <Interpreters/SyntaxAnalyzer.h>
namespace DB
{
@ -55,9 +56,6 @@ struct ExpressionAnalyzerData
bool has_global_subqueries = false;
using Aliases = std::unordered_map<String, ASTPtr>;
Aliases aliases;
/// Which column is needed to be ARRAY-JOIN'ed to get the specified.
/// For example, for `SELECT s.v ... ARRAY JOIN a AS s` will get "s.v" -> "a.v".
NameToNameMap array_join_result_to_source;
@ -75,6 +73,19 @@ struct ExpressionAnalyzerData
/// Predicate optimizer overrides the sub queries
bool rewrite_subqueries = false;
/// Columns will be added to block by join.
JoinedColumnsList columns_added_by_join; /// Subset of analyzed_join.available_joined_columns
/// Actions which need to be calculated on joined block.
ExpressionActionsPtr joined_block_actions;
/// Columns which will be used in query from joined table. Duplicate names are qualified.
NameSet required_columns_from_joined_table;
/// Such columns will be copied from left join keys during join.
/// Example: select right from tab1 join tab2 on left + 1 = right
NameSet columns_added_by_join_from_right_keys;
protected:
ExpressionAnalyzerData(const NamesAndTypesList & source_columns_,
const Names & required_result_columns_,
@ -229,116 +240,17 @@ private:
size_t subquery_depth;
bool do_global; /// Do I need to prepare for execution global subqueries when analyzing the query.
struct AnalyzedJoin
{
/// NOTE: So far, only one JOIN per query is supported.
/** Query of the form `SELECT expr(x) AS k FROM t1 ANY LEFT JOIN (SELECT expr(x) AS k FROM t2) USING k`
* The join is made by column k.
* During the JOIN,
* - in the "right" table, it will be available by alias `k`, since `Project` action for the subquery was executed.
* - in the "left" table, it will be accessible by the name `expr(x)`, since `Project` action has not been executed yet.
* You must remember both of these options.
*
* Query of the form `SELECT ... from t1 ANY LEFT JOIN (SELECT ... from t2) ON expr(t1 columns) = expr(t2 columns)`
* to the subquery will be added expression `expr(t2 columns)`.
* It's possible to use name `expr(t2 columns)`.
*/
Names key_names_left;
Names key_names_right; /// Duplicating names are qualified.
ASTs key_asts_left;
ASTs key_asts_right;
struct JoinedColumn
{
/// Column will be joined to block.
NameAndTypePair name_and_type;
/// original column name from joined source.
String original_name;
JoinedColumn(const NameAndTypePair & name_and_type_, const String & original_name_)
: name_and_type(name_and_type_), original_name(original_name_) {}
bool operator==(const JoinedColumn & o) const
{
return name_and_type == o.name_and_type && original_name == o.original_name;
}
};
using JoinedColumnsList = std::list<JoinedColumn>;
/// All columns which can be read from joined table. Duplicating names are qualified.
JoinedColumnsList columns_from_joined_table;
/// Columns which will be used in query to the joined query. Duplicating names are qualified.
NameSet required_columns_from_joined_table;
/// Columns which will be added to block, possible including some columns from right join key.
JoinedColumnsList columns_added_by_join;
/// Such columns will be copied from left join keys during join.
NameSet columns_added_by_join_from_right_keys;
/// Actions which need to be calculated on joined block.
ExpressionActionsPtr joined_block_actions;
void createJoinedBlockActions(const NamesAndTypesList & source_columns,
const ASTSelectQuery * select_query_with_join,
const Context & context);
NamesAndTypesList getColumnsAddedByJoin() const;
const JoinedColumnsList & getColumnsFromJoinedTable(const NamesAndTypesList & source_columns,
const Context & context,
const ASTSelectQuery * select_query_with_join);
};
AnalyzedJoin analyzed_join;
/** Remove all unnecessary columns from the list of all available columns of the table (`columns`).
* At the same time, form a set of unknown columns (`unknown_required_source_columns`),
* as well as the columns added by JOIN (`columns_added_by_join`).
*/
void collectUsedColumns();
/** Find the columns that are obtained by JOIN.
*/
void collectJoinedColumns(NameSet & joined_columns);
/// Parse JOIN ON expression and collect ASTs for joined columns.
void collectJoinedColumnsFromJoinOnExpr();
/** For star nodes(`*`), expand them to a list of all columns.
* For literal nodes, substitute aliases.
*/
void normalizeTree();
/// Eliminates injective function calls and constant expressions from group by statement
void optimizeGroupBy();
/// Remove duplicate items from ORDER BY.
void optimizeOrderBy();
void optimizeLimitBy();
/// Remove duplicated columns from USING(...).
void optimizeUsing();
/// remove Function_if AST if condition is constant
void optimizeIfWithConstantCondition();
void optimizeIfWithConstantConditionImpl(ASTPtr & current_ast);
bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & value) const;
/// Replacing scalar subqueries with constant values.
void executeScalarSubqueries();
/// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables.
void initGlobalSubqueriesAndExternalTables();
/** Initialize InterpreterSelectQuery for a subquery in the GLOBAL IN/JOIN section,
* create a temporary table of type Memory and store it in the external_tables dictionary.
*/
void addExternalStorage(ASTPtr & subquery_or_table_name);
void getArrayJoinedColumns();
void addMultipleArrayJoinAction(ExpressionActionsPtr & actions) const;
void addJoinAction(ExpressionActionsPtr & actions, bool only_types) const;
@ -375,17 +287,6 @@ private:
void makeSetsForIndexImpl(const ASTPtr & node, const Block & sample_block);
/** Translate qualified names such as db.table.column, table.column, table_alias.column
* to unqualified names. This is done in a poor transitional way:
* only one ("main") table is supported. Ambiguity is not detected or resolved.
*/
void translateQualifiedNames();
/** Sometimes we have to calculate more columns in SELECT clause than will be returned from query.
* This is the case when we have DISTINCT or arrayJoin: we require more columns in SELECT even if we need less columns in result.
*/
void removeUnneededColumnsFromSelectClause();
bool isRemoteStorage() const;
};

View File

@ -25,11 +25,11 @@ private:
const Context & context;
Tables & external_tables;
void visit(const ASTIdentifier * node, ASTPtr &) const
void visit(const ASTIdentifier & node, ASTPtr &) const
{
if (node->special())
if (StoragePtr external_storage = context.tryGetExternalTable(node->name))
external_tables[node->name] = external_storage;
if (node.special())
if (StoragePtr external_storage = context.tryGetExternalTable(node.name))
external_tables[node.name] = external_storage;
}
template <typename T>
@ -37,7 +37,7 @@ private:
{
if (const T * t = typeid_cast<const T *>(ast.get()))
{
visit(t, ast);
visit(*t, ast);
return true;
}
return false;

View File

@ -3,6 +3,9 @@
namespace DB
{
/// Visitors consist of functions with unified interface 'void visit(Casted & x, ASTPtr & y)', there x is y, successfully casted to Casted.
/// Both types and fuction could have const specifiers. The second argument is used by visitor to replaces AST node (y) if needed.
/// Converts GLOBAL subqueries to external tables; Puts them into the external_tables dictionary: name -> StoragePtr.
class GlobalSubqueriesVisitor
{
@ -41,22 +44,22 @@ private:
bool & has_global_subqueries;
/// GLOBAL IN
void visit(ASTFunction * func, ASTPtr &) const
void visit(ASTFunction & func, ASTPtr &) const
{
if (func->name == "globalIn" || func->name == "globalNotIn")
if (func.name == "globalIn" || func.name == "globalNotIn")
{
addExternalStorage(func->arguments->children.at(1));
addExternalStorage(func.arguments->children.at(1));
has_global_subqueries = true;
}
}
/// GLOBAL JOIN
void visit(ASTTablesInSelectQueryElement * table_elem, ASTPtr &) const
void visit(ASTTablesInSelectQueryElement & table_elem, ASTPtr &) const
{
if (table_elem->table_join
&& static_cast<const ASTTableJoin &>(*table_elem->table_join).locality == ASTTableJoin::Locality::Global)
if (table_elem.table_join
&& static_cast<const ASTTableJoin &>(*table_elem.table_join).locality == ASTTableJoin::Locality::Global)
{
addExternalStorage(table_elem->table_expression);
addExternalStorage(table_elem.table_expression);
has_global_subqueries = true;
}
}
@ -66,12 +69,15 @@ private:
{
if (T * t = typeid_cast<T *>(ast.get()))
{
visit(t, ast);
visit(*t, ast);
return true;
}
return false;
}
/** Initialize InterpreterSelectQuery for a subquery in the GLOBAL IN/JOIN section,
* create a temporary table of type Memory and store it in the external_tables dictionary.
*/
void addExternalStorage(ASTPtr & subquery_or_table_name_or_table_expression) const
{
/// With nondistributed queries, creating temporary tables does not make sense.

View File

@ -30,6 +30,7 @@
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/InterpreterInsertQuery.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/AddDefaultDatabaseVisitor.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/NestedUtils.h>
@ -511,7 +512,10 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
create.to_database = current_database;
if (create.select && (create.is_view || create.is_materialized_view))
create.select->setDatabaseIfNeeded(current_database);
{
AddDefaultDatabaseVisitor visitor(current_database);
visitor.visit(*create.select);
}
Block as_select_sample;
if (create.select && (!create.attach || !create.columns))

View File

@ -323,6 +323,8 @@ InterpreterSelectQuery::AnalysisResult InterpreterSelectQuery::analyzeExpression
res.prewhere_info->remove_columns_actions = std::move(actions);
}
res.columns_to_remove_after_prewhere = std::move(columns_to_remove);
}
if (has_where)
res.remove_where_filter = chain.steps.at(where_step_num).can_remove_required_output.at(0);
@ -495,7 +497,7 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
throw Exception("Distributed on Distributed is not supported", ErrorCodes::NOT_IMPLEMENTED);
/** Read the data from Storage. from_stage - to what stage the request was completed in Storage. */
executeFetchColumns(from_stage, pipeline, expressions.prewhere_info);
executeFetchColumns(from_stage, pipeline, expressions.prewhere_info, expressions.columns_to_remove_after_prewhere);
LOG_TRACE(log, QueryProcessingStage::toString(from_stage) << " -> " << QueryProcessingStage::toString(to_stage));
}
@ -686,7 +688,8 @@ static void getLimitLengthAndOffset(ASTSelectQuery & query, size_t & length, siz
void InterpreterSelectQuery::executeFetchColumns(
QueryProcessingStage::Enum processing_stage, Pipeline & pipeline, const PrewhereInfoPtr & prewhere_info)
QueryProcessingStage::Enum processing_stage, Pipeline & pipeline,
const PrewhereInfoPtr & prewhere_info, const Names & columns_to_remove_after_prewhere)
{
const Settings & settings = context.getSettingsRef();
@ -751,11 +754,15 @@ void InterpreterSelectQuery::executeFetchColumns(
/// Columns which we will get after prewhere execution.
NamesAndTypesList additional_source_columns;
/// Add columns which will be added by prewhere (otherwise we will remove them in project action).
NameSet columns_to_remove(columns_to_remove_after_prewhere.begin(), columns_to_remove_after_prewhere.end());
for (const auto & column : prewhere_actions_result)
{
if (prewhere_info->remove_prewhere_column && column.name == prewhere_info->prewhere_column_name)
continue;
if (columns_to_remove.count(column.name))
continue;
required_columns_expr_list->children.emplace_back(std::make_shared<ASTIdentifier>(column.name));
additional_source_columns.emplace_back(column.name, column.type);
}
@ -1236,6 +1243,8 @@ void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline)
/// If there are several streams, then we merge them into one
if (pipeline.hasMoreThanOneStream())
{
unifyStreams(pipeline);
/** MergingSortedBlockInputStream reads the sources sequentially.
* To make the data on the remote servers prepared in parallel, we wrap it in AsynchronousBlockInputStream.
*/
@ -1290,16 +1299,7 @@ void InterpreterSelectQuery::executeUnion(Pipeline & pipeline)
/// If there are still several streams, then we combine them into one
if (pipeline.hasMoreThanOneStream())
{
/// Unify streams in case they have different headers.
auto first_header = pipeline.streams.at(0)->getHeader();
for (size_t i = 1; i < pipeline.streams.size(); ++i)
{
auto & stream = pipeline.streams[i];
auto header = stream->getHeader();
auto mode = ConvertingBlockInputStream::MatchColumnsMode::Name;
if (!blocksHaveEqualStructure(first_header, header))
stream = std::make_shared<ConvertingBlockInputStream>(context, stream, first_header, mode);
}
unifyStreams(pipeline);
pipeline.firstStream() = std::make_shared<UnionBlockInputStream<>>(pipeline.streams, pipeline.stream_with_non_joined_data, max_streams);
pipeline.stream_with_non_joined_data = nullptr;
@ -1429,6 +1429,23 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(Pipeline & pipeline
SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode));
}
void InterpreterSelectQuery::unifyStreams(Pipeline & pipeline)
{
if (pipeline.hasMoreThanOneStream())
{
/// Unify streams in case they have different headers.
auto first_header = pipeline.streams.at(0)->getHeader();
for (size_t i = 1; i < pipeline.streams.size(); ++i)
{
auto & stream = pipeline.streams[i];
auto header = stream->getHeader();
auto mode = ConvertingBlockInputStream::MatchColumnsMode::Name;
if (!blocksHaveEqualStructure(first_header, header))
stream = std::make_shared<ConvertingBlockInputStream>(context, stream, first_header, mode);
}
}
}
void InterpreterSelectQuery::ignoreWithTotals()
{

View File

@ -150,6 +150,9 @@ private:
/// Columns from the SELECT list, before renaming them to aliases.
Names selected_columns;
/// Columns will be removed after prewhere actions execution.
Names columns_to_remove_after_prewhere;
/// Do I need to perform the first part of the pipeline - running on remote servers during distributed processing.
bool first_stage = false;
/// Do I need to execute the second part of the pipeline - running on the initiating server during distributed processing.
@ -171,7 +174,8 @@ private:
/// dry_run - don't read from table, use empty header block instead.
void executeWithMultipleStreamsImpl(Pipeline & pipeline, const BlockInputStreamPtr & input, bool dry_run);
void executeFetchColumns(QueryProcessingStage::Enum processing_stage, Pipeline & pipeline, const PrewhereInfoPtr & prewhere_info);
void executeFetchColumns(QueryProcessingStage::Enum processing_stage, Pipeline & pipeline,
const PrewhereInfoPtr & prewhere_info, const Names & columns_to_remove_after_prewhere);
void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter);
void executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final);
@ -190,6 +194,9 @@ private:
void executeExtremes(Pipeline & pipeline);
void executeSubqueriesInSetsAndJoins(Pipeline & pipeline, std::unordered_map<String, SubqueryForSet> & subqueries_for_sets);
/// If pipeline has several streams with different headers, add ConvertingBlockInputStream to first header.
void unifyStreams(Pipeline & pipeline);
enum class Modificator
{
ROLLUP = 0,

View File

@ -302,8 +302,8 @@ ASTs PredicateExpressionsOptimizer::getSelectQueryProjectionColumns(ASTPtr & ast
{
/// first should normalize query tree.
std::unordered_map<String, ASTPtr> aliases;
QueryAliasesVisitor query_aliases_visitor;
query_aliases_visitor.visit(ast, aliases, 0);
QueryAliasesVisitor query_aliases_visitor(aliases);
query_aliases_visitor.visit(ast);
QueryNormalizer(ast, aliases, settings, {}, {}).perform();
ASTs projection_columns;

View File

@ -6,7 +6,6 @@
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/formatAST.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/DumpASTNode.h>
#include <IO/WriteHelpers.h>
namespace DB
@ -17,76 +16,85 @@ namespace ErrorCodes
extern const int MULTIPLE_EXPRESSIONS_FOR_ALIAS;
}
/// ignore_levels - aliases in how many upper levels of the subtree should be ignored.
/// For example, with ignore_levels=1 ast can not be put in the dictionary, but its children can.
void QueryAliasesVisitor::getQueryAliases(const ASTPtr & ast, Aliases & aliases, int ignore_levels) const
void QueryAliasesVisitor::visit(const ASTPtr & ast) const
{
DumpASTNode dump(*ast, ostr, visit_depth, "getQueryAliases");
/// Bottom-up traversal. We do not go into subqueries.
for (auto & child : ast->children)
visitChildren(ast);
if (!tryVisit<ASTSubquery>(ast))
{
int new_ignore_levels = std::max(0, ignore_levels - 1);
/// The top-level aliases in the ARRAY JOIN section have a special meaning, we will not add them
/// (skip the expression list itself and its children).
if (typeid_cast<ASTArrayJoin *>(ast.get()))
new_ignore_levels = 3;
/// Don't descent into table functions and subqueries.
if (!typeid_cast<ASTTableExpression *>(child.get())
&& !typeid_cast<ASTSelectWithUnionQuery *>(child.get()))
getQueryAliases(child, aliases, new_ignore_levels);
DumpASTNode dump(*ast, ostr, visit_depth, "getQueryAliases");
visitOther(ast);
}
if (ignore_levels > 0)
return;
getNodeAlias(ast, aliases, dump);
}
void QueryAliasesVisitor::getNodeAlias(const ASTPtr & ast, Aliases & aliases, const DumpASTNode & dump) const
/// The top-level aliases in the ARRAY JOIN section have a special meaning, we will not add them
/// (skip the expression list itself and its children).
void QueryAliasesVisitor::visit(const ASTArrayJoin &, const ASTPtr & ast) const
{
for (auto & child1 : ast->children)
for (auto & child2 : child1->children)
for (auto & child3 : child2->children)
visit(child3);
}
/// 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 QueryAliasesVisitor::visit(ASTSubquery & subquery, const ASTPtr & ast) const
{
static std::atomic_uint64_t subquery_index = 0;
if (subquery.alias.empty())
{
String alias;
do
{
alias = "_subquery" + std::to_string(++subquery_index);
}
while (aliases.count(alias));
subquery.setAlias(alias);
subquery.prefer_alias_to_column_name = true;
aliases[alias] = ast;
}
else
visitOther(ast);
}
void QueryAliasesVisitor::visitOther(const ASTPtr & ast) const
{
String alias = ast->tryGetAlias();
if (!alias.empty())
{
if (aliases.count(alias) && ast->getTreeHash() != aliases[alias]->getTreeHash())
{
std::stringstream message;
message << "Different expressions with the same alias " << backQuoteIfNeed(alias) << ":\n";
formatAST(*ast, message, false, true);
message << "\nand\n";
formatAST(*aliases[alias], message, false, true);
message << "\n";
throw Exception(message.str(), ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS);
}
throw Exception(wrongAliasMessage(ast, alias), ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS);
aliases[alias] = ast;
dump.print(visit_action, alias);
}
else if (auto subquery = typeid_cast<ASTSubquery *>(ast.get()))
}
void QueryAliasesVisitor::visitChildren(const ASTPtr & ast) const
{
for (auto & child : ast->children)
{
/// 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
if (subquery->alias.empty())
{
static std::atomic_uint64_t subquery_index = 1;
while (true)
{
alias = "_subquery" + std::to_string(subquery_index++);
if (!aliases.count(alias))
break;
}
subquery->setAlias(alias);
subquery->prefer_alias_to_column_name = true;
aliases[alias] = ast;
dump.print(visit_action, alias);
}
/// Don't descent into table functions and subqueries and special case for ArrayJoin.
if (!tryVisit<ASTTableExpression>(ast) &&
!tryVisit<ASTSelectWithUnionQuery>(ast) &&
!tryVisit<ASTArrayJoin>(ast))
visit(child);
}
}
String QueryAliasesVisitor::wrongAliasMessage(const ASTPtr & ast, const String & alias) const
{
std::stringstream message;
message << "Different expressions with the same alias " << backQuoteIfNeed(alias) << ":" << std::endl;
formatAST(*ast, message, false, true);
message << std::endl << "and" << std::endl;
formatAST(*aliases[alias], message, false, true);
message << std::endl;
return message.str();
}
}

View File

@ -1,34 +1,60 @@
#pragma once
#include <Common/typeid_cast.h>
#include <Parsers/DumpASTNode.h>
#include <unordered_map>
namespace DB
{
class ASTSelectWithUnionQuery;
class ASTSubquery;
struct ASTTableExpression;
struct ASTArrayJoin;
using Aliases = std::unordered_map<String, ASTPtr>;
/// Visitors consist of functions with unified interface 'void visit(Casted & x, ASTPtr & y)', there x is y, successfully casted to Casted.
/// Both types and fuction could have const specifiers. The second argument is used by visitor to replaces AST node (y) if needed.
/// Visits AST nodes and collect their aliases in one map (with links to source nodes).
class QueryAliasesVisitor
{
public:
QueryAliasesVisitor(std::ostream * ostr_ = nullptr)
: visit_depth(0),
QueryAliasesVisitor(Aliases & aliases_, std::ostream * ostr_ = nullptr)
: aliases(aliases_),
visit_depth(0),
ostr(ostr_)
{}
void visit(const ASTPtr & ast, Aliases & aliases, int ignore_levels = 0) const
{
getQueryAliases(ast, aliases, ignore_levels);
}
void visit(const ASTPtr & ast) const;
private:
static constexpr const char * visit_action = "addAlias";
Aliases & aliases;
mutable size_t visit_depth;
std::ostream * ostr;
void getQueryAliases(const ASTPtr & ast, Aliases & aliases, int ignore_levels) const;
void getNodeAlias(const ASTPtr & ast, Aliases & aliases, const DumpASTNode & dump) const;
void visit(const ASTTableExpression &, const ASTPtr & ) const {}
void visit(const ASTSelectWithUnionQuery &, const ASTPtr & ) const {}
void visit(ASTSubquery & subquery, const ASTPtr & ast) const;
void visit(const ASTArrayJoin &, const ASTPtr & ast) const;
void visitOther(const ASTPtr & ast) const;
void visitChildren(const ASTPtr & ast) const;
template <typename T>
bool tryVisit(const ASTPtr & ast) const
{
if (T * t = typeid_cast<T *>(ast.get()))
{
DumpASTNode dump(*ast, ostr, visit_depth, "getQueryAliases");
visit(*t, ast);
return true;
}
return false;
}
String wrongAliasMessage(const ASTPtr & ast, const String & alias) const;
};
}

View File

@ -8,6 +8,8 @@ namespace ErrorCodes
extern const int TYPE_MISMATCH;
}
/// Visitors consist of functions with unified interface 'void visit(Casted & x, ASTPtr & y)', there x is y, successfully casted to Casted.
/// Both types and fuction could have const specifiers. The second argument is used by visitor to replaces AST node (y) if needed.
/** Get a set of necessary columns to read from the table.
* In this case, the columns specified in ignored_names are considered unnecessary. And the ignored_names parameter can be modified.
@ -48,28 +50,28 @@ private:
const NameSet & available_joined_columns;
NameSet & required_joined_columns;
void visit(const ASTIdentifier * node, const ASTPtr &) const
void visit(const ASTIdentifier & node, const ASTPtr &) const
{
if (node->general()
&& !ignored_names.count(node->name)
&& !ignored_names.count(Nested::extractTableName(node->name)))
if (node.general()
&& !ignored_names.count(node.name)
&& !ignored_names.count(Nested::extractTableName(node.name)))
{
if (!available_joined_columns.count(node->name)
|| available_columns.count(node->name)) /// Read column from left table if has.
required_source_columns.insert(node->name);
if (!available_joined_columns.count(node.name)
|| available_columns.count(node.name)) /// Read column from left table if has.
required_source_columns.insert(node.name);
else
required_joined_columns.insert(node->name);
required_joined_columns.insert(node.name);
}
}
void visit(const ASTFunction * node, const ASTPtr & ast) const
void visit(const ASTFunction & node, const ASTPtr & ast) const
{
if (node->name == "lambda")
if (node.name == "lambda")
{
if (node->arguments->children.size() != 2)
if (node.arguments->children.size() != 2)
throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
ASTFunction * lambda_args_tuple = typeid_cast<ASTFunction *>(node->arguments->children.at(0).get());
ASTFunction * lambda_args_tuple = typeid_cast<ASTFunction *>(node.arguments->children.at(0).get());
if (!lambda_args_tuple || lambda_args_tuple->name != "tuple")
throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH);
@ -90,7 +92,7 @@ private:
}
}
visit(node->arguments->children.at(1));
visit(node.arguments->children.at(1));
for (size_t i = 0; i < added_ignored.size(); ++i)
ignored_names.erase(added_ignored[i]);
@ -100,7 +102,7 @@ private:
/// A special function `indexHint`. Everything that is inside it is not calculated
/// (and is used only for index analysis, see KeyCondition).
if (node->name == "indexHint")
if (node.name == "indexHint")
return;
visitChildren(ast);
@ -126,7 +128,7 @@ private:
{
if (const T * t = typeid_cast<const T *>(ast.get()))
{
visit(t, ast);
visit(*t, ast);
return true;
}
return false;

View File

@ -176,7 +176,7 @@ struct Settings
\
M(SettingBool, join_use_nulls, 0, "Use NULLs for non-joined rows of outer JOINs. If false, use default value of corresponding columns data type.") \
\
M(SettingJoinStrictness, join_default_strictness, JoinStrictness::Unspecified, "Set default strictness in JOIN query. Possible values: empty string, 'ANY', 'ALL'. If empty, query without strictness will throw exception.") \
M(SettingJoinStrictness, join_default_strictness, JoinStrictness::ALL, "Set default strictness in JOIN query. Possible values: empty string, 'ANY', 'ALL'. If empty, query without strictness will throw exception.") \
\
M(SettingUInt64, preferred_block_size_bytes, 1000000, "") \
\

View File

@ -0,0 +1,967 @@
#include <Interpreters/SyntaxAnalyzer.h>
#include <Interpreters/InJoinSubqueriesPreprocessor.h>
#include <Interpreters/LogicalExpressionsOptimizer.h>
#include <Interpreters/Settings.h>
#include <Interpreters/QueryAliasesVisitor.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/ArrayJoinedColumnsVisitor.h>
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
#include <Interpreters/Context.h>
#include <Interpreters/QueryNormalizer.h>
#include <Interpreters/ExecuteScalarSubqueriesVisitor.h>
#include <Interpreters/PredicateExpressionsOptimizer.h>
#include <Interpreters/ExternalDictionaries.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTOrderByElement.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/queryToString.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypeNullable.h>
#include <Common/typeid_cast.h>
#include <Core/NamesAndTypes.h>
#include <Storages/IStorage.h>
#include <IO/WriteHelpers.h>
#include <functional>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ALIAS_REQUIRED;
extern const int MULTIPLE_EXPRESSIONS_FOR_ALIAS;
extern const int EMPTY_NESTED_TABLE;
extern const int LOGICAL_ERROR;
extern const int INVALID_JOIN_ON_EXPRESSION;
}
namespace
{
using LogAST = DebugASTLog<false>; /// set to true to enable logs
using Aliases = std::unordered_map<String, ASTPtr>;
/// Add columns from storage to source_columns list.
NamesAndTypesList collectSourceColumns(NamesAndTypesList source_columns, ASTSelectQuery * select_query,
const Context & context, StoragePtr & storage);
/// Translate qualified names such as db.table.column, table.column, table_alias.column to unqualified names.
void translateQualifiedNames(ASTPtr & query, ASTSelectQuery * select_query,
const NameSet & source_columns, const Context & context);
/// For star nodes(`*`), expand them to a list of all columns. For literal nodes, substitute aliases.
void normalizeTree(
SyntaxAnalyzerResult & result,
const Names & source_columns,
const NameSet & source_columns_set,
const StoragePtr & storage,
const Context & context,
const ASTSelectQuery * select_query,
bool asterisk_left_columns_only);
/// Sometimes we have to calculate more columns in SELECT clause than will be returned from query.
/// This is the case when we have DISTINCT or arrayJoin: we require more columns in SELECT even if we need less columns in result.
void removeUnneededColumnsFromSelectClause(const ASTSelectQuery * select_query, const Names & required_result_columns);
/// Replacing scalar subqueries with constant values.
void executeScalarSubqueries(SyntaxAnalyzerResult & result, const ASTSelectQuery * select_query,
const Context & context, size_t subquery_depth);
/// Remove Function_if AST if condition is constant.
void optimizeIfWithConstantCondition(ASTPtr & current_ast, Aliases & aliases);
/// Eliminates injective function calls and constant expressions from group by statement.
void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_columns, const Context & context);
/// Remove duplicate items from ORDER BY.
void optimizeOrderBy(const ASTSelectQuery * select_query);
/// Remove duplicate items from LIMIT BY.
void optimizeLimitBy(const ASTSelectQuery * select_query);
/// Remove duplicated columns from USING(...).
void optimizeUsing(const ASTSelectQuery * select_query);
void getArrayJoinedColumns(SyntaxAnalyzerResult & result, const ASTSelectQuery * select_query,
const Names & source_columns, const NameSet & source_columns_set);
/// Parse JOIN ON expression and collect ASTs for joined columns.
void collectJoinedColumnsFromJoinOnExpr(AnalyzedJoin & analyzed_join, const ASTSelectQuery * select_query,
const NameSet & source_columns, const Context & context);
/// Find the columns that are obtained by JOIN.
void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery * select_query,
const NameSet & source_columns, const Context & context);
}
SyntaxAnalyzerResult SyntaxAnalyzer::analyze(
const ASTPtr & query,
NamesAndTypesList source_columns,
const Names & required_result_columns,
size_t subquery_depth) const
{
SyntaxAnalyzerResult result;
result.storage = storage;
result.query = query; // ->clone();
auto * select_query = typeid_cast<ASTSelectQuery *>(result.query.get());
result.source_columns = collectSourceColumns(std::move(source_columns), select_query, context, result.storage);
const auto & settings = context.getSettingsRef();
Names source_columns_list;
source_columns_list.reserve(result.source_columns.size());
for (const auto & type_name : result.source_columns)
source_columns_list.emplace_back(type_name.name);
NameSet source_columns_set(source_columns_list.begin(), source_columns_list.end());
translateQualifiedNames(result.query, select_query, source_columns_set, context);
/// Depending on the user's profile, check for the execution rights
/// distributed subqueries inside the IN or JOIN sections and process these subqueries.
InJoinSubqueriesPreprocessor(context).process(select_query);
/// Optimizes logical expressions.
LogicalExpressionsOptimizer(select_query, settings.optimize_min_equality_disjunction_chain_length.value).perform();
/// Creates a dictionary `aliases`: alias -> ASTPtr
{
LogAST log;
QueryAliasesVisitor query_aliases_visitor(result.aliases, log.stream());
query_aliases_visitor.visit(query);
}
/// Common subexpression elimination. Rewrite rules.
normalizeTree(result, source_columns_list, source_columns_set, result.storage,
context, select_query, settings.asterisk_left_columns_only != 0);
/// Remove unneeded columns according to 'required_result_columns'.
/// Leave all selected columns in case of DISTINCT; columns that contain arrayJoin function inside.
/// Must be after 'normalizeTree' (after expanding aliases, for aliases not get lost)
/// and before 'executeScalarSubqueries', 'analyzeAggregation', etc. to avoid excessive calculations.
removeUnneededColumnsFromSelectClause(select_query, required_result_columns);
/// Executing scalar subqueries - replacing them with constant values.
executeScalarSubqueries(result, select_query, context, subquery_depth);
/// Optimize if with constant condition after constants was substituted instead of sclalar subqueries.
optimizeIfWithConstantCondition(result.query, result.aliases);
/// GROUP BY injective function elimination.
optimizeGroupBy(select_query, source_columns_set, context);
/// Remove duplicate items from ORDER BY.
optimizeOrderBy(select_query);
// Remove duplicated elements from LIMIT BY clause.
optimizeLimitBy(select_query);
/// Remove duplicated columns from USING(...).
optimizeUsing(select_query);
/// array_join_alias_to_name, array_join_result_to_source.
getArrayJoinedColumns(result, select_query, source_columns_list, source_columns_set);
/// Push the predicate expression down to the subqueries.
result.rewrite_subqueries = PredicateExpressionsOptimizer(select_query, settings, context).optimize();
collectJoinedColumns(result.analyzed_join, select_query, source_columns_set, context);
return result;
}
namespace
{
void removeDuplicateColumns(NamesAndTypesList & columns)
{
std::set<String> names;
for (auto it = columns.begin(); it != columns.end();)
{
if (names.emplace(it->name).second)
++it;
else
columns.erase(it++);
}
}
NamesAndTypesList collectSourceColumns(NamesAndTypesList source_columns, ASTSelectQuery * select_query,
const Context & context, StoragePtr & storage)
{
if (!storage && select_query)
{
if (auto db_and_table = getDatabaseAndTable(*select_query, 0))
storage = context.tryGetTable(db_and_table->database, db_and_table->table);
}
if (storage)
{
auto physical_columns = storage->getColumns().getAllPhysical();
if (source_columns.empty())
source_columns.swap(physical_columns);
else
source_columns.insert(source_columns.end(), physical_columns.begin(), physical_columns.end());
if (select_query)
{
const auto & storage_aliases = storage->getColumns().aliases;
source_columns.insert(source_columns.end(), storage_aliases.begin(), storage_aliases.end());
}
}
removeDuplicateColumns(source_columns);
return source_columns;
}
void translateQualifiedNames(ASTPtr & query, ASTSelectQuery * select_query,
const NameSet & source_columns, const Context & context)
{
if (!select_query || !select_query->tables || select_query->tables->children.empty())
return;
std::vector<DatabaseAndTableWithAlias> tables = getDatabaseAndTables(*select_query, context.getCurrentDatabase());
LogAST log;
TranslateQualifiedNamesVisitor visitor(source_columns, tables, log.stream());
visitor.visit(query);
}
void normalizeTree(
SyntaxAnalyzerResult & result,
const Names & source_columns,
const NameSet & source_columns_set,
const StoragePtr & storage,
const Context & context,
const ASTSelectQuery * select_query,
bool asterisk_left_columns_only)
{
Names all_columns_name = storage ? storage->getColumns().ordinary.getNames() : source_columns;
if (!asterisk_left_columns_only)
{
auto columns_from_joined_table = result.analyzed_join.getColumnsFromJoinedTable(source_columns_set, context, select_query);
for (auto & column : columns_from_joined_table)
all_columns_name.emplace_back(column.name_and_type.name);
}
if (all_columns_name.empty())
throw Exception("An asterisk cannot be replaced with empty columns.", ErrorCodes::LOGICAL_ERROR);
TableNamesAndColumnNames table_names_and_column_names;
if (select_query && select_query->tables && !select_query->tables->children.empty())
{
std::vector<const ASTTableExpression *> tables_expression = getSelectTablesExpression(*select_query);
bool first = true;
for (const auto * table_expression : tables_expression)
{
DatabaseAndTableWithAlias table_name(*table_expression, context.getCurrentDatabase());
NamesAndTypesList names_and_types = getNamesAndTypeListFromTableExpression(*table_expression, context);
if (!first)
{
/// For joined tables qualify duplicating names.
for (auto & name_and_type : names_and_types)
if (source_columns_set.count(name_and_type.name))
name_and_type.name = table_name.getQualifiedNamePrefix() + name_and_type.name;
}
first = false;
table_names_and_column_names.emplace_back(std::pair(table_name, names_and_types.getNames()));
}
}
auto & settings = context.getSettingsRef();
QueryNormalizer(result.query, result.aliases, settings, all_columns_name, table_names_and_column_names).perform();
}
bool hasArrayJoin(const ASTPtr & ast)
{
if (const ASTFunction * function = typeid_cast<const ASTFunction *>(&*ast))
if (function->name == "arrayJoin")
return true;
for (const auto & child : ast->children)
if (!typeid_cast<ASTSelectQuery *>(child.get()) && hasArrayJoin(child))
return true;
return false;
}
void removeUnneededColumnsFromSelectClause(const ASTSelectQuery * select_query, const Names & required_result_columns)
{
if (!select_query)
return;
if (required_result_columns.empty())
return;
ASTs & elements = select_query->select_expression_list->children;
ASTs new_elements;
new_elements.reserve(elements.size());
/// Some columns may be queried multiple times, like SELECT x, y, y FROM table.
/// In that case we keep them exactly same number of times.
std::map<String, size_t> required_columns_with_duplicate_count;
for (const auto & name : required_result_columns)
++required_columns_with_duplicate_count[name];
for (const auto & elem : elements)
{
String name = elem->getAliasOrColumnName();
auto it = required_columns_with_duplicate_count.find(name);
if (required_columns_with_duplicate_count.end() != it && it->second)
{
new_elements.push_back(elem);
--it->second;
}
else if (select_query->distinct || hasArrayJoin(elem))
{
new_elements.push_back(elem);
}
}
elements = std::move(new_elements);
}
void executeScalarSubqueries(SyntaxAnalyzerResult & result, const ASTSelectQuery * select_query,
const Context & context, size_t subquery_depth)
{
LogAST log;
if (!select_query)
{
ExecuteScalarSubqueriesVisitor visitor(context, subquery_depth, log.stream());
visitor.visit(result.query);
}
else
{
for (auto & child : result.query->children)
{
/// Do not go to FROM, JOIN, UNION.
if (!typeid_cast<const ASTTableExpression *>(child.get())
&& !typeid_cast<const ASTSelectQuery *>(child.get()))
{
ExecuteScalarSubqueriesVisitor visitor(context, subquery_depth, log.stream());
visitor.visit(child);
}
}
}
}
bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & value)
{
/// numeric constant in condition
if (const ASTLiteral * literal = typeid_cast<ASTLiteral *>(condition.get()))
{
if (literal->value.getType() == Field::Types::Int64 ||
literal->value.getType() == Field::Types::UInt64)
{
value = literal->value.get<Int64>();
return true;
}
}
/// cast of numeric constant in condition to UInt8
if (const ASTFunction * function = typeid_cast<ASTFunction * >(condition.get()))
{
if (function->name == "CAST")
{
if (ASTExpressionList * expr_list = typeid_cast<ASTExpressionList *>(function->arguments.get()))
{
const ASTPtr & type_ast = expr_list->children.at(1);
if (const ASTLiteral * type_literal = typeid_cast<ASTLiteral *>(type_ast.get()))
{
if (type_literal->value.getType() == Field::Types::String &&
type_literal->value.get<std::string>() == "UInt8")
return tryExtractConstValueFromCondition(expr_list->children.at(0), value);
}
}
}
}
return false;
}
void optimizeIfWithConstantCondition(ASTPtr & current_ast, Aliases & aliases)
{
if (!current_ast)
return;
for (ASTPtr & child : current_ast->children)
{
auto * function_node = typeid_cast<ASTFunction *>(child.get());
if (!function_node || function_node->name != "if")
{
optimizeIfWithConstantCondition(child, aliases);
continue;
}
optimizeIfWithConstantCondition(function_node->arguments, aliases);
auto * args = typeid_cast<ASTExpressionList *>(function_node->arguments.get());
if (args->children.size() != 3)
throw Exception("Wrong number of arguments for function 'if' (" + toString(args->children.size()) + " instead of 3)",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
ASTPtr condition_expr = args->children[0];
ASTPtr then_expr = args->children[1];
ASTPtr else_expr = args->children[2];
bool condition;
if (tryExtractConstValueFromCondition(condition_expr, condition))
{
ASTPtr replace_ast = condition ? then_expr : else_expr;
ASTPtr child_copy = child;
String replace_alias = replace_ast->tryGetAlias();
String if_alias = child->tryGetAlias();
if (replace_alias.empty())
{
replace_ast->setAlias(if_alias);
child = replace_ast;
}
else
{
/// Only copy of one node is required here.
/// But IAST has only method for deep copy of subtree.
/// This can be a reason of performance degradation in case of deep queries.
ASTPtr replace_ast_deep_copy = replace_ast->clone();
replace_ast_deep_copy->setAlias(if_alias);
child = replace_ast_deep_copy;
}
if (!if_alias.empty())
{
auto alias_it = aliases.find(if_alias);
if (alias_it != aliases.end() && alias_it->second.get() == child_copy.get())
alias_it->second = child;
}
}
}
}
/** Calls to these functions in the GROUP BY statement would be
* replaced by their immediate argument.
*/
const std::unordered_set<String> injective_function_names
{
"negate",
"bitNot",
"reverse",
"reverseUTF8",
"toString",
"toFixedString",
"IPv4NumToString",
"IPv4StringToNum",
"hex",
"unhex",
"bitmaskToList",
"bitmaskToArray",
"tuple",
"regionToName",
"concatAssumeInjective",
};
const std::unordered_set<String> possibly_injective_function_names
{
"dictGetString",
"dictGetUInt8",
"dictGetUInt16",
"dictGetUInt32",
"dictGetUInt64",
"dictGetInt8",
"dictGetInt16",
"dictGetInt32",
"dictGetInt64",
"dictGetFloat32",
"dictGetFloat64",
"dictGetDate",
"dictGetDateTime"
};
void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_columns, const Context & context)
{
if (!(select_query && select_query->group_expression_list))
return;
const auto is_literal = [] (const ASTPtr & ast)
{
return typeid_cast<const ASTLiteral *>(ast.get());
};
auto & group_exprs = select_query->group_expression_list->children;
/// removes expression at index idx by making it last one and calling .pop_back()
const auto remove_expr_at_index = [&group_exprs] (const size_t idx)
{
if (idx < group_exprs.size() - 1)
std::swap(group_exprs[idx], group_exprs.back());
group_exprs.pop_back();
};
/// iterate over each GROUP BY expression, eliminate injective function calls and literals
for (size_t i = 0; i < group_exprs.size();)
{
if (const auto function = typeid_cast<ASTFunction *>(group_exprs[i].get()))
{
/// assert function is injective
if (possibly_injective_function_names.count(function->name))
{
/// do not handle semantic errors here
if (function->arguments->children.size() < 2)
{
++i;
continue;
}
const auto & dict_name = typeid_cast<const ASTLiteral &>(*function->arguments->children[0])
.value.safeGet<String>();
const auto & dict_ptr = context.getExternalDictionaries().getDictionary(dict_name);
const auto & attr_name = typeid_cast<const ASTLiteral &>(*function->arguments->children[1])
.value.safeGet<String>();
if (!dict_ptr->isInjective(attr_name))
{
++i;
continue;
}
}
else if (!injective_function_names.count(function->name))
{
++i;
continue;
}
/// copy shared pointer to args in order to ensure lifetime
auto args_ast = function->arguments;
/** remove function call and take a step back to ensure
* next iteration does not skip not yet processed data
*/
remove_expr_at_index(i);
/// copy non-literal arguments
std::remove_copy_if(
std::begin(args_ast->children), std::end(args_ast->children),
std::back_inserter(group_exprs), is_literal
);
}
else if (is_literal(group_exprs[i]))
{
remove_expr_at_index(i);
}
else
{
/// if neither a function nor literal - advance to next expression
++i;
}
}
if (group_exprs.empty())
{
/** You can not completely remove GROUP BY. Because if there were no aggregate functions, then it turns out that there will be no aggregation.
* Instead, leave `GROUP BY const`.
* Next, see deleting the constants in the analyzeAggregation method.
*/
/// You must insert a constant that is not the name of the column in the table. Such a case is rare, but it happens.
UInt64 unused_column = 0;
String unused_column_name = toString(unused_column);
while (source_columns.count(unused_column_name))
{
++unused_column;
unused_column_name = toString(unused_column);
}
select_query->group_expression_list = std::make_shared<ASTExpressionList>();
select_query->group_expression_list->children.emplace_back(std::make_shared<ASTLiteral>(UInt64(unused_column)));
}
}
void optimizeOrderBy(const ASTSelectQuery * select_query)
{
if (!(select_query && select_query->order_expression_list))
return;
/// Make unique sorting conditions.
using NameAndLocale = std::pair<String, String>;
std::set<NameAndLocale> elems_set;
ASTs & elems = select_query->order_expression_list->children;
ASTs unique_elems;
unique_elems.reserve(elems.size());
for (const auto & elem : elems)
{
String name = elem->children.front()->getColumnName();
const ASTOrderByElement & order_by_elem = typeid_cast<const ASTOrderByElement &>(*elem);
if (elems_set.emplace(name, order_by_elem.collation ? order_by_elem.collation->getColumnName() : "").second)
unique_elems.emplace_back(elem);
}
if (unique_elems.size() < elems.size())
elems = unique_elems;
}
void optimizeLimitBy(const ASTSelectQuery * select_query)
{
if (!(select_query && select_query->limit_by_expression_list))
return;
std::set<String> elems_set;
ASTs & elems = select_query->limit_by_expression_list->children;
ASTs unique_elems;
unique_elems.reserve(elems.size());
for (const auto & elem : elems)
{
if (elems_set.emplace(elem->getColumnName()).second)
unique_elems.emplace_back(elem);
}
if (unique_elems.size() < elems.size())
elems = unique_elems;
}
void optimizeUsing(const ASTSelectQuery * select_query)
{
if (!select_query)
return;
auto node = const_cast<ASTTablesInSelectQueryElement *>(select_query->join());
if (!node)
return;
auto table_join = static_cast<ASTTableJoin *>(&*node->table_join);
if (!(table_join && table_join->using_expression_list))
return;
ASTs & expression_list = table_join->using_expression_list->children;
ASTs uniq_expressions_list;
std::set<String> expressions_names;
for (const auto & expression : expression_list)
{
auto expression_name = expression->getAliasOrColumnName();
if (expressions_names.find(expression_name) == expressions_names.end())
{
uniq_expressions_list.push_back(expression);
expressions_names.insert(expression_name);
}
}
if (uniq_expressions_list.size() < expression_list.size())
expression_list = uniq_expressions_list;
}
void getArrayJoinedColumns(SyntaxAnalyzerResult & result, const ASTSelectQuery * select_query,
const Names & source_columns, const NameSet & source_columns_set)
{
if (select_query && select_query->array_join_expression_list())
{
ASTs & array_join_asts = select_query->array_join_expression_list()->children;
for (const auto & ast : array_join_asts)
{
const String nested_table_name = ast->getColumnName();
const String nested_table_alias = ast->getAliasOrColumnName();
if (nested_table_alias == nested_table_name && !typeid_cast<const ASTIdentifier *>(ast.get()))
throw Exception("No alias for non-trivial value in ARRAY JOIN: " + nested_table_name,
ErrorCodes::ALIAS_REQUIRED);
if (result.array_join_alias_to_name.count(nested_table_alias) || result.aliases.count(nested_table_alias))
throw Exception("Duplicate alias in ARRAY JOIN: " + nested_table_alias,
ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS);
result.array_join_alias_to_name[nested_table_alias] = nested_table_name;
result.array_join_name_to_alias[nested_table_name] = nested_table_alias;
}
{
ArrayJoinedColumnsVisitor visitor(result.array_join_name_to_alias,
result.array_join_alias_to_name,
result.array_join_result_to_source);
visitor.visit(result.query);
}
/// If the result of ARRAY JOIN is not used, it is necessary to ARRAY-JOIN any column,
/// to get the correct number of rows.
if (result.array_join_result_to_source.empty())
{
ASTPtr expr = select_query->array_join_expression_list()->children.at(0);
String source_name = expr->getColumnName();
String result_name = expr->getAliasOrColumnName();
/// This is an array.
if (!typeid_cast<ASTIdentifier *>(expr.get()) || source_columns_set.count(source_name))
{
result.array_join_result_to_source[result_name] = source_name;
}
else /// This is a nested table.
{
bool found = false;
for (const auto & column_name : source_columns)
{
auto splitted = Nested::splitName(column_name);
if (splitted.first == source_name && !splitted.second.empty())
{
result.array_join_result_to_source[Nested::concatenateName(result_name, splitted.second)] = column_name;
found = true;
break;
}
}
if (!found)
throw Exception("No columns in nested table " + source_name, ErrorCodes::EMPTY_NESTED_TABLE);
}
}
}
}
void collectJoinedColumnsFromJoinOnExpr(AnalyzedJoin & analyzed_join, const ASTSelectQuery * select_query,
const NameSet & source_columns, const Context & context)
{
const auto & tables = static_cast<const ASTTablesInSelectQuery &>(*select_query->tables);
const auto * left_tables_element = static_cast<const ASTTablesInSelectQueryElement *>(tables.children.at(0).get());
const auto * right_tables_element = select_query->join();
if (!left_tables_element || !right_tables_element)
return;
const auto & table_join = static_cast<const ASTTableJoin &>(*right_tables_element->table_join);
if (!table_join.on_expression)
return;
const auto & left_table_expression = static_cast<const ASTTableExpression &>(*left_tables_element->table_expression);
const auto & right_table_expression = static_cast<const ASTTableExpression &>(*right_tables_element->table_expression);
DatabaseAndTableWithAlias left_source_names(left_table_expression, context.getCurrentDatabase());
DatabaseAndTableWithAlias right_source_names(right_table_expression, context.getCurrentDatabase());
/// Stores examples of columns which are only from one table.
struct TableBelonging
{
const ASTIdentifier * example_only_from_left = nullptr;
const ASTIdentifier * example_only_from_right = nullptr;
};
/// Check all identifiers in ast and decide their possible table belonging.
/// Throws if there are two identifiers definitely from different tables.
std::function<TableBelonging(const ASTPtr &)> get_table_belonging;
get_table_belonging = [&](const ASTPtr & ast) -> TableBelonging
{
auto * identifier = typeid_cast<const ASTIdentifier *>(ast.get());
if (identifier)
{
if (identifier->general())
{
auto left_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, left_source_names);
auto right_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, right_source_names);
/// Assume that component from definite table if num_components is greater than for the other table.
if (left_num_components > right_num_components)
return {identifier, nullptr};
if (left_num_components < right_num_components)
return {nullptr, identifier};
}
return {};
}
TableBelonging table_belonging;
for (const auto & child : ast->children)
{
auto children_belonging = get_table_belonging(child);
if (!table_belonging.example_only_from_left)
table_belonging.example_only_from_left = children_belonging.example_only_from_left;
if (!table_belonging.example_only_from_right)
table_belonging.example_only_from_right = children_belonging.example_only_from_right;
}
if (table_belonging.example_only_from_left && table_belonging.example_only_from_right)
throw Exception("Invalid columns in JOIN ON section. Columns "
+ table_belonging.example_only_from_left->getAliasOrColumnName() + " and "
+ table_belonging.example_only_from_right->getAliasOrColumnName()
+ " are from different tables.", ErrorCodes::INVALID_JOIN_ON_EXPRESSION);
return table_belonging;
};
std::function<void(ASTPtr &, const DatabaseAndTableWithAlias &, bool)> translate_qualified_names;
translate_qualified_names = [&](ASTPtr & ast, const DatabaseAndTableWithAlias & source_names, bool right_table)
{
if (auto * identifier = typeid_cast<const ASTIdentifier *>(ast.get()))
{
if (identifier->general())
{
auto num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, source_names);
stripIdentifier(ast, num_components);
if (right_table && source_columns.count(ast->getColumnName()))
source_names.makeQualifiedName(ast);
}
return;
}
for (auto & child : ast->children)
translate_qualified_names(child, source_names, right_table);
};
const auto supported_syntax = " Supported syntax: JOIN ON Expr([table.]column, ...) = Expr([table.]column, ...) "
"[AND Expr([table.]column, ...) = Expr([table.]column, ...) ...]";
auto throwSyntaxException = [&](const String & msg)
{
throw Exception("Invalid expression for JOIN ON. " + msg + supported_syntax, ErrorCodes::INVALID_JOIN_ON_EXPRESSION);
};
/// For equal expression find out corresponding table for each part, translate qualified names and add asts to join keys.
auto add_columns_from_equals_expr = [&](const ASTPtr & expr)
{
auto * func_equals = typeid_cast<const ASTFunction *>(expr.get());
if (!func_equals || func_equals->name != "equals")
throwSyntaxException("Expected equals expression, got " + queryToString(expr) + ".");
ASTPtr left_ast = func_equals->arguments->children.at(0)->clone();
ASTPtr right_ast = func_equals->arguments->children.at(1)->clone();
auto left_table_belonging = get_table_belonging(left_ast);
auto right_table_belonging = get_table_belonging(right_ast);
bool can_be_left_part_from_left_table = left_table_belonging.example_only_from_right == nullptr;
bool can_be_left_part_from_right_table = left_table_belonging.example_only_from_left == nullptr;
bool can_be_right_part_from_left_table = right_table_belonging.example_only_from_right == nullptr;
bool can_be_right_part_from_right_table = right_table_belonging.example_only_from_left == nullptr;
auto add_join_keys = [&](ASTPtr & ast_to_left_table, ASTPtr & ast_to_right_table)
{
translate_qualified_names(ast_to_left_table, left_source_names, false);
translate_qualified_names(ast_to_right_table, right_source_names, true);
analyzed_join.key_asts_left.push_back(ast_to_left_table);
analyzed_join.key_names_left.push_back(ast_to_left_table->getColumnName());
analyzed_join.key_asts_right.push_back(ast_to_right_table);
analyzed_join.key_names_right.push_back(ast_to_right_table->getAliasOrColumnName());
};
/// Default variant when all identifiers may be from any table.
if (can_be_left_part_from_left_table && can_be_right_part_from_right_table)
add_join_keys(left_ast, right_ast);
else if (can_be_left_part_from_right_table && can_be_right_part_from_left_table)
add_join_keys(right_ast, left_ast);
else
{
auto * left_example = left_table_belonging.example_only_from_left ?
left_table_belonging.example_only_from_left :
left_table_belonging.example_only_from_right;
auto * right_example = right_table_belonging.example_only_from_left ?
right_table_belonging.example_only_from_left :
right_table_belonging.example_only_from_right;
auto left_name = queryToString(*left_example);
auto right_name = queryToString(*right_example);
auto expr_name = queryToString(expr);
throwSyntaxException("In expression " + expr_name + " columns " + left_name + " and " + right_name
+ " are from the same table but from different arguments of equal function.");
}
};
auto * func = typeid_cast<const ASTFunction *>(table_join.on_expression.get());
if (func && func->name == "and")
{
for (const auto & expr : func->arguments->children)
add_columns_from_equals_expr(expr);
}
else
add_columns_from_equals_expr(table_join.on_expression);
}
void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery * select_query,
const NameSet & source_columns, const Context & context)
{
if (!select_query)
return;
const ASTTablesInSelectQueryElement * node = select_query->join();
if (!node)
return;
const auto & table_join = static_cast<const ASTTableJoin &>(*node->table_join);
const auto & table_expression = static_cast<const ASTTableExpression &>(*node->table_expression);
DatabaseAndTableWithAlias joined_table_name(table_expression, context.getCurrentDatabase());
auto add_name_to_join_keys = [&](Names & join_keys, ASTs & join_asts, const ASTPtr & ast, bool right_table)
{
String name;
if (right_table)
{
name = ast->getAliasOrColumnName();
if (source_columns.count(name))
name = joined_table_name.getQualifiedNamePrefix() + name;
}
else
name = ast->getColumnName();
join_keys.push_back(name);
join_asts.push_back(ast);
};
if (table_join.using_expression_list)
{
auto & keys = typeid_cast<ASTExpressionList &>(*table_join.using_expression_list);
for (const auto & key : keys.children)
{
add_name_to_join_keys(analyzed_join.key_names_left, analyzed_join.key_asts_left, key, false);
add_name_to_join_keys(analyzed_join.key_names_right, analyzed_join.key_asts_right, key, true);
}
}
else if (table_join.on_expression)
collectJoinedColumnsFromJoinOnExpr(analyzed_join, select_query, source_columns, context);
auto & columns_from_joined_table = analyzed_join.getColumnsFromJoinedTable(source_columns, context, select_query);
NameSet joined_columns;
auto & settings = context.getSettingsRef();
for (auto & column : columns_from_joined_table)
{
auto & column_name = column.name_and_type.name;
auto & column_type = column.name_and_type.type;
auto & original_name = column.original_name;
{
if (joined_columns.count(column_name)) /// Duplicate columns in the subquery for JOIN do not make sense.
continue;
joined_columns.insert(column_name);
bool make_nullable = settings.join_use_nulls && (table_join.kind == ASTTableJoin::Kind::Left ||
table_join.kind == ASTTableJoin::Kind::Full);
auto type = make_nullable ? makeNullable(column_type) : column_type;
analyzed_join.available_joined_columns.emplace_back(NameAndTypePair(column_name, std::move(type)), original_name);
}
}
}
}
}

View File

@ -0,0 +1,66 @@
#pragma once
#include <Interpreters/AnalyzedJoin.h>
namespace DB
{
class IStorage;
using StoragePtr = std::shared_ptr<IStorage>;
struct SyntaxAnalyzerResult
{
ASTPtr query;
StoragePtr storage;
NamesAndTypesList source_columns;
/// Note: used only in tests.
using Aliases = std::unordered_map<String, ASTPtr>;
Aliases aliases;
/// Which column is needed to be ARRAY-JOIN'ed to get the specified.
/// For example, for `SELECT s.v ... ARRAY JOIN a AS s` will get "s.v" -> "a.v".
NameToNameMap array_join_result_to_source;
/// For the ARRAY JOIN section, mapping from the alias to the full column name.
/// For example, for `ARRAY JOIN [1,2] AS b` "b" -> "array(1,2)" will enter here.
NameToNameMap array_join_alias_to_name;
/// The backward mapping for array_join_alias_to_name.
NameToNameMap array_join_name_to_alias;
AnalyzedJoin analyzed_join;
/// Predicate optimizer overrides the sub queries
bool rewrite_subqueries = false;
};
/// AST syntax analysis.
/// Optimises AST tree and collect information for further expression analysis.
/// Result AST has the following invariants:
/// * all aliases are substituted
/// * qualified names are translated
/// * scalar subqueries are executed replaced with constants
/// * unneeded columns are removed from SELECT clause
/// * duplicated columns are removed from ORDER BY, LIMIT BY, USING(...).
/// Motivation:
/// * group most of the AST-changing operations in single place
/// * avoid AST rewriting in ExpressionAnalyzer
/// * decompose ExpressionAnalyzer
class SyntaxAnalyzer
{
public:
SyntaxAnalyzer(const Context & context, const StoragePtr & storage) : context(context), storage(storage) {}
SyntaxAnalyzerResult analyze(const ASTPtr & query,
NamesAndTypesList source_columns,
const Names & required_result_columns = {},
size_t subquery_depth = 0) const;
const Context & context;
StoragePtr storage;
};
}

View File

@ -1,6 +1,6 @@
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
#include <Core/NamesAndTypes.h>
#include <Core/Names.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTQualifiedAsterisk.h>
@ -8,6 +8,7 @@
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
namespace DB
{
@ -16,9 +17,9 @@ namespace ErrorCodes
extern const int UNKNOWN_IDENTIFIER;
}
void TranslateQualifiedNamesVisitor::visit(ASTIdentifier * identifier, ASTPtr & ast, const DumpASTNode & dump) const
void TranslateQualifiedNamesVisitor::visit(ASTIdentifier & identifier, ASTPtr & ast, const DumpASTNode & dump) const
{
if (identifier->general())
if (identifier.general())
{
/// Select first table name with max number of qualifiers which can be stripped.
size_t max_num_qualifiers_to_strip = 0;
@ -27,7 +28,7 @@ void TranslateQualifiedNamesVisitor::visit(ASTIdentifier * identifier, ASTPtr &
for (size_t table_pos = 0; table_pos < tables.size(); ++table_pos)
{
const auto & table = tables[table_pos];
auto num_qualifiers_to_strip = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, table);
auto num_qualifiers_to_strip = getNumComponentsToStripInOrderToTranslateQualifiedName(identifier, table);
if (num_qualifiers_to_strip > max_num_qualifiers_to_strip)
{
@ -38,12 +39,12 @@ void TranslateQualifiedNamesVisitor::visit(ASTIdentifier * identifier, ASTPtr &
if (max_num_qualifiers_to_strip)
{
dump.print(String("stripIdentifier ") + identifier->name, max_num_qualifiers_to_strip);
dump.print(String("stripIdentifier ") + identifier.name, max_num_qualifiers_to_strip);
stripIdentifier(ast, max_num_qualifiers_to_strip);
}
/// In case if column from the joined table are in source columns, change it's name to qualified.
if (best_table_pos && source_columns.contains(ast->getColumnName()))
if (best_table_pos && source_columns.count(ast->getColumnName()))
{
const DatabaseAndTableWithAlias & table = tables[best_table_pos];
table.makeQualifiedName(ast);
@ -52,7 +53,7 @@ void TranslateQualifiedNamesVisitor::visit(ASTIdentifier * identifier, ASTPtr &
}
}
void TranslateQualifiedNamesVisitor::visit(ASTQualifiedAsterisk *, ASTPtr & ast, const DumpASTNode &) const
void TranslateQualifiedNamesVisitor::visit(ASTQualifiedAsterisk &, ASTPtr & ast, const DumpASTNode &) const
{
if (ast->children.size() != 1)
throw Exception("Logical error: qualified asterisk must have exactly one child", ErrorCodes::LOGICAL_ERROR);
@ -88,23 +89,23 @@ void TranslateQualifiedNamesVisitor::visit(ASTQualifiedAsterisk *, ASTPtr & ast,
throw Exception("Unknown qualified identifier: " + ident->getAliasOrColumnName(), ErrorCodes::UNKNOWN_IDENTIFIER);
}
void TranslateQualifiedNamesVisitor::visit(ASTTableJoin * join, ASTPtr &, const DumpASTNode &) const
void TranslateQualifiedNamesVisitor::visit(ASTTableJoin & join, ASTPtr &, const DumpASTNode &) const
{
/// Don't translate on_expression here in order to resolve equation parts later.
if (join->using_expression_list)
visit(join->using_expression_list);
if (join.using_expression_list)
visit(join.using_expression_list);
}
void TranslateQualifiedNamesVisitor::visit(ASTSelectQuery * select, ASTPtr & ast, const DumpASTNode &) const
void TranslateQualifiedNamesVisitor::visit(ASTSelectQuery & select, ASTPtr & ast, const DumpASTNode &) const
{
/// If the WHERE clause or HAVING consists of a single quailified column, the reference must be translated not only in children,
/// but also in where_expression and having_expression.
if (select->prewhere_expression)
visit(select->prewhere_expression);
if (select->where_expression)
visit(select->where_expression);
if (select->having_expression)
visit(select->having_expression);
if (select.prewhere_expression)
visit(select.prewhere_expression);
if (select.where_expression)
visit(select.where_expression);
if (select.having_expression)
visit(select.having_expression);
visitChildren(ast);
}

View File

@ -5,24 +5,23 @@
#include <Common/typeid_cast.h>
#include <Parsers/DumpASTNode.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTQualifiedAsterisk.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
namespace DB
{
class ASTIdentifier;
class ASTQualifiedAsterisk;
class ASTSelectQuery;
struct ASTTableJoin;
class NamesAndTypesList;
/// Visitors consist of functions with unified interface 'void visit(Casted & x, ASTPtr & y)', there x is y, successfully casted to Casted.
/// Both types and fuction could have const specifiers. The second argument is used by visitor to replaces AST node (y) if needed.
/// It visits nodes, find columns (general identifiers and asterisks) and translate their names according to tables' names.
class TranslateQualifiedNamesVisitor
{
public:
TranslateQualifiedNamesVisitor(const NamesAndTypesList & source_columns_, const std::vector<DatabaseAndTableWithAlias> & tables_,
TranslateQualifiedNamesVisitor(const NameSet & source_columns_, const std::vector<DatabaseAndTableWithAlias> & tables_,
std::ostream * ostr_ = nullptr)
: source_columns(source_columns_),
tables(tables_),
@ -32,34 +31,33 @@ public:
void visit(ASTPtr & ast) const
{
DumpASTNode dump(*ast, ostr, visit_depth, "translateQualifiedNames");
if (!tryVisit<ASTIdentifier>(ast, dump) &&
!tryVisit<ASTQualifiedAsterisk>(ast, dump) &&
!tryVisit<ASTTableJoin>(ast, dump) &&
!tryVisit<ASTSelectQuery>(ast, dump))
if (!tryVisit<ASTIdentifier>(ast) &&
!tryVisit<ASTQualifiedAsterisk>(ast) &&
!tryVisit<ASTTableJoin>(ast) &&
!tryVisit<ASTSelectQuery>(ast))
visitChildren(ast); /// default: do nothing, visit children
}
private:
const NamesAndTypesList & source_columns;
const NameSet & source_columns;
const std::vector<DatabaseAndTableWithAlias> & tables;
mutable size_t visit_depth;
std::ostream * ostr;
void visit(ASTIdentifier * node, ASTPtr & ast, const DumpASTNode & dump) const;
void visit(ASTQualifiedAsterisk * node, ASTPtr & ast, const DumpASTNode & dump) const;
void visit(ASTTableJoin * node, ASTPtr & ast, const DumpASTNode & dump) const;
void visit(ASTSelectQuery * ast, ASTPtr &, const DumpASTNode & dump) const;
void visit(ASTIdentifier & node, ASTPtr & ast, const DumpASTNode & dump) const;
void visit(ASTQualifiedAsterisk & node, ASTPtr & ast, const DumpASTNode & dump) const;
void visit(ASTTableJoin & node, ASTPtr & ast, const DumpASTNode & dump) const;
void visit(ASTSelectQuery & ast, ASTPtr &, const DumpASTNode & dump) const;
void visitChildren(ASTPtr &) const;
template <typename T>
bool tryVisit(ASTPtr & ast, const DumpASTNode & dump) const
bool tryVisit(ASTPtr & ast) const
{
if (T * t = typeid_cast<T *>(ast.get()))
{
visit(t, ast, dump);
DumpASTNode dump(*ast, ostr, visit_depth, "translateQualifiedNames");
visit(*t, ast, dump);
return true;
}
return false;

View File

@ -8,7 +8,7 @@
#include <Parsers/parseQuery.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/SyntaxAnalyzer.h>
#include <Analyzers/CollectAliases.h>
#include <Analyzers/ExecuteTableFunctions.h>
@ -35,11 +35,9 @@ struct TestEntry
{
ASTPtr ast = parse(query);
ExpressionAnalyzer analyzer(ast, context, {}, source_columns, required_result_columns);
auto res = SyntaxAnalyzer(context, {}).analyze(ast, source_columns, required_result_columns);
const ExpressionAnalyzerData & data = analyzer.getAnalyzedData();
if (!checkAliases(data))
if (!checkAliases(res))
{
collectWithAnalysers(context, ast);
return false;
@ -49,9 +47,9 @@ struct TestEntry
}
private:
bool checkAliases(const ExpressionAnalyzerData & data)
bool checkAliases(const SyntaxAnalyzerResult & res)
{
for (const auto & alias : data.aliases)
for (const auto & alias : res.aliases)
{
const String & alias_name = alias.first;
if (expected_aliases.count(alias_name) == 0 ||

View File

@ -4,10 +4,8 @@
#include <Parsers/ASTAsterisk.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Common/typeid_cast.h>
#include "ASTSelectWithUnionQuery.h"
namespace DB
@ -311,44 +309,6 @@ const ASTTablesInSelectQueryElement * ASTSelectQuery::join() const
}
void ASTSelectQuery::setDatabaseIfNeeded(const String & database_name)
{
if (!tables)
return;
ASTTablesInSelectQuery & tables_in_select_query = static_cast<ASTTablesInSelectQuery &>(*tables);
for (auto & child : tables_in_select_query.children)
{
const auto & tables_element = static_cast<ASTTablesInSelectQueryElement &>(*child);
if (tables_element.table_expression)
{
const auto table_expression = static_cast<ASTTableExpression *>(tables_element.table_expression.get());
if (!table_expression->database_and_table_name && !table_expression->subquery)
continue;
if (table_expression->subquery)
{
const auto subquery = static_cast<const ASTSubquery *>(table_expression->subquery.get());
const auto select_with_union_query = static_cast<ASTSelectWithUnionQuery *>(subquery->children[0].get());
select_with_union_query->setDatabaseIfNeeded(database_name);
}
else if (table_expression->database_and_table_name->children.empty())
{
const ASTIdentifier & identifier = static_cast<const ASTIdentifier &>(*table_expression->database_and_table_name);
table_expression->database_and_table_name = createDatabaseAndTableNode(database_name, identifier.name);
}
else if (table_expression->database_and_table_name->children.size() != 2)
{
throw Exception("Logical error: more than two components in table expression", ErrorCodes::LOGICAL_ERROR);
}
}
}
}
void ASTSelectQuery::replaceDatabaseAndTable(const String & database_name, const String & table_name)
{
ASTTableExpression * table_expression = getFirstTableExpression(*this);

View File

@ -45,7 +45,6 @@ public:
const ASTTablesInSelectQueryElement * join() const;
bool array_join_is_left() const;
bool final() const;
void setDatabaseIfNeeded(const String & database_name);
void replaceDatabaseAndTable(const String & database_name, const String & table_name);
void addTableFunction(ASTPtr & table_function_ptr);

View File

@ -35,11 +35,4 @@ void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, F
}
}
void ASTSelectWithUnionQuery::setDatabaseIfNeeded(const String & database_name)
{
for (auto & child : list_of_selects->children)
typeid_cast<ASTSelectQuery &>(*child).setDatabaseIfNeeded(database_name);
}
}

View File

@ -17,8 +17,6 @@ public:
ASTPtr clone() const override;
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void setDatabaseIfNeeded(const String & database_name);
ASTPtr list_of_selects;
};

View File

@ -7,6 +7,7 @@
#include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/InterpreterDropQuery.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/AddDefaultDatabaseVisitor.h>
#include <Storages/StorageMaterializedView.h>
#include <Storages/StorageFactory.h>
@ -40,7 +41,8 @@ static void extractDependentTable(ASTSelectQuery & query, String & select_databa
if (db_and_table->database.empty())
{
db_and_table->database = select_database_name;
query.setDatabaseIfNeeded(select_database_name);
AddDefaultDatabaseVisitor visitor(select_database_name);
visitor.visit(query);
}
else
select_database_name = db_and_table->database;

View File

@ -87,7 +87,7 @@ def main(args):
os.environ.setdefault("CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL", server_logs_level)
if args.zookeeper is None:
code, out = commands.getstatusoutput(args.binary + "-extract-from-config --try --config " + args.configserver + ' --key zookeeper | grep . | wc -l')
code, out = commands.getstatusoutput(args.extract_from_config +" --try --config " + args.configserver + ' --key zookeeper | grep . | wc -l')
try:
if int(out) > 0:
args.zookeeper = True
@ -97,7 +97,7 @@ def main(args):
args.zookeeper = False
if args.shard is None:
code, out = commands.getstatusoutput(args.binary + "-extract-from-config --try --config " + args.configserver + ' --key listen_host | grep -E "127.0.0.2|::"')
code, out = commands.getstatusoutput(args.extract_from_config + " --try --config " + args.configserver + ' --key listen_host | grep -E "127.0.0.2|::"')
if out:
args.shard = True
else:
@ -332,6 +332,7 @@ if __name__ == '__main__':
parser.add_argument('--tmp', help = 'Path to tmp dir')
parser.add_argument('-b', '--binary', default = 'clickhouse', help = 'Main clickhouse binary')
parser.add_argument('-c', '--client', help = 'Client program')
parser.add_argument('--extract_from_config', help = 'extract-from-config program')
parser.add_argument('--configclient', help = 'Client config (if you use not default ports)')
parser.add_argument('--configserver', default= '/etc/clickhouse-server/config.xml', help = 'Preprocessed server config')
parser.add_argument('-o', '--output', help = 'Output xUnit compliant test report directory')
@ -366,7 +367,10 @@ if __name__ == '__main__':
args.tmp = args.queries
if args.client is None:
args.client = args.binary + '-client'
if os.access(args.binary + '-client', os.X_OK):
args.client = args.binary + '-client'
else:
args.client = args.binary + ' client'
if args.configclient:
args.client += ' --config-file=' + args.configclient
if os.getenv("CLICKHOUSE_HOST"):
@ -374,4 +378,10 @@ if __name__ == '__main__':
if os.getenv("CLICKHOUSE_PORT_TCP"):
args.client += ' --port=' + os.getenv("CLICKHOUSE_PORT_TCP")
if args.extract_from_config is None:
if os.access(args.binary + '-extract-from-config', os.X_OK):
args.extract_from_config = args.binary + '-extract-from-config'
else:
args.extract_from_config = args.binary + ' extract-from-config'
main(args)

View File

@ -10,10 +10,18 @@ DATA_DIR=${DATA_DIR:=`mktemp -d /tmp/clickhouse.test..XXXXX`}
DATA_DIR_PATTERN=${DATA_DIR_PATTERN:=/tmp/clickhouse} # path from config file, will be replaced to temporary
LOG_DIR=${LOG_DIR:=$DATA_DIR/log}
export CLICKHOUSE_BINARY=${CLICKHOUSE_BINARY:="clickhouse"}
[ -x "$ROOT_DIR/dbms/programs/${CLICKHOUSE_BINARY}-server" ] && BUILD_DIR=${BUILD_DIR:=$ROOT_DIR} # Build without separate build dir
BUILD_DIR=${BUILD_DIR:=$ROOT_DIR/build${BUILD_TYPE}}
[ -x "$CUR_DIR/clickhouse-server" ] && [ -x "${CUR_DIR}/${CLICKHOUSE_BINARY}-client" ] && BIN_DIR= # Allow run in /usr/bin
[ -x "$BUILD_DIR/dbms/programs/${CLICKHOUSE_BINARY}-server" ] && BIN_DIR=${BIN_DIR:=$BUILD_DIR/dbms/programs/}
( [ -x "$ROOT_DIR/dbms/programs/${CLICKHOUSE_BINARY}-server" ] || [ -x "$ROOT_DIR/dbms/programs/${CLICKHOUSE_BINARY}" ] ) && BUILD_DIR=${BUILD_DIR:=$ROOT_DIR} # Build without separate build dir
[ -d "$ROOT_DIR/build${BUILD_TYPE}" ] && BUILD_DIR=${BUILD_DIR:=$ROOT_DIR/build${BUILD_TYPE}}
BUILD_DIR=${BUILD_DIR:=$ROOT_DIR}
[ -x ${CLICKHOUSE_BINARY}-server" ] && [ -x ${CLICKHOUSE_BINARY}-client" ] && BIN_DIR= # Allow run in /usr/bin
( [ -x "$BUILD_DIR/dbms/programs/${CLICKHOUSE_BINARY}" ] || [ -x "$BUILD_DIR/dbms/programs/${CLICKHOUSE_BINARY}-server" ] ) && BIN_DIR=${BIN_DIR:=$BUILD_DIR/dbms/programs/}
[ -x "$BIN_DIR/${CLICKHOUSE_BINARY}-server" ] && CLICKHOUSE_SERVER=${CLICKHOUSE_SERVER:=$BIN_DIR/${CLICKHOUSE_BINARY}-server}
[ -x "$BIN_DIR/${CLICKHOUSE_BINARY}" ] && CLICKHOUSE_SERVER=${CLICKHOUSE_SERVER:=$BIN_DIR/${CLICKHOUSE_BINARY} server}
[ -x "$BIN_DIR/${CLICKHOUSE_BINARY}-client" ] && CLICKHOUSE_CLIENT=${CLICKHOUSE_CLIENT:=$BIN_DIR/${CLICKHOUSE_BINARY}-client}
[ -x "$BIN_DIR/${CLICKHOUSE_BINARY}" ] && CLICKHOUSE_CLIENT=${CLICKHOUSE_CLIENT:=$BIN_DIR/${CLICKHOUSE_BINARY} client}
[ -x "$BIN_DIR/${CLICKHOUSE_BINARY}-extract-from-config" ] && CLICKHOUSE_EXTRACT=${CLICKHOUSE_EXTRACT:=$BIN_DIR/${CLICKHOUSE_BINARY}-extract-from-config}
[ -x "$BIN_DIR/${CLICKHOUSE_BINARY}" ] && CLICKHOUSE_EXTRACT=${CLICKHOUSE_EXTRACT:=$BIN_DIR/${CLICKHOUSE_BINARY} extract-from-config}
[ -f "$CUR_DIR/server-test.xml" ] && CONFIG_DIR=${CONFIG_DIR=$CUR_DIR}/
CONFIG_CLIENT_DIR=${CONFIG_CLIENT_DIR=$CONFIG_DIR}
CONFIG_SERVER_DIR=${CONFIG_SERVER_DIR=$CONFIG_DIR}
@ -48,7 +56,7 @@ if [ "$DATA_DIR_PATTERN" != "$DATA_DIR" ]; then
cp $CLICKHOUSE_CONFIG_USERS $DATA_DIR/etc
fi
CLICKHOUSE_EXTRACT_CONFIG=${CLICKHOUSE_EXTRACT_CONFIG:="${BIN_DIR}${CLICKHOUSE_BINARY}-extract-from-config --config=$CLICKHOUSE_CONFIG"}
CLICKHOUSE_EXTRACT_CONFIG=${CLICKHOUSE_EXTRACT_CONFIG:="${CLICKHOUSE_EXTRACT} --config=$CLICKHOUSE_CONFIG"}
CLICKHOUSE_LOG=${CLICKHOUSE_LOG:=${LOG_DIR}clickhouse-server.log}
export CLICKHOUSE_PORT_TCP=${CLICKHOUSE_PORT_TCP:=`$CLICKHOUSE_EXTRACT_CONFIG --key=tcp_port || echo 9000`}
export CLICKHOUSE_PORT_HTTP=${CLICKHOUSE_PORT_HTTP:=`$CLICKHOUSE_EXTRACT_CONFIG --key=http_port || echo 8123`}
@ -58,8 +66,8 @@ export CLICKHOUSE_PORT_HTTPS=${CLICKHOUSE_PORT_HTTPS:=`$CLICKHOUSE_EXTRACT_CONFI
export CLICKHOUSE_ODBC_BRIDGE=${CLICKHOUSE_ODBC_BRIDGE:=`$CLICKHOUSE_EXTRACT_CONFIG --key=odbc_bridge.port || echo 9018`}
DHPARAM=`$CLICKHOUSE_EXTRACT_CONFIG --key=openSSL.server.dhParamsFile`
PRIVATEKEY=`${BIN_DIR}clickhouse-extract-from-config --config=$CLICKHOUSE_CONFIG --key=openSSL.server.privateKeyFile`
CERT=`${BIN_DIR}clickhouse-extract-from-config --config=$CLICKHOUSE_CONFIG --key=openSSL.server.certificateFile`
PRIVATEKEY=`$CLICKHOUSE_EXTRACT_CONFIG --key=openSSL.server.privateKeyFile`
CERT=`$CLICKHOUSE_EXTRACT_CONFIG --key=openSSL.server.certificateFile`
# Do not generate in case broken extract-config
[ -n "$DHPARAM" ] && openssl dhparam -out $DHPARAM 256
[ -n "$PRIVATEKEY" ] && [ -n "$CERT" ] && openssl req -subj "/CN=localhost" -new -newkey rsa:2048 -days 365 -nodes -x509 -keyout $PRIVATEKEY -out $CERT
@ -75,11 +83,11 @@ fi
# --remote_servers.test_shard_localhost_secure.shard.replica.port=$CLICKHOUSE_PORT_TCP_SECURE \
# --remote_servers.test_shard_localhost.shard.replica.port=$CLICKHOUSE_PORT_TCP \
VERSION=`${BIN_DIR}clickhouse-client --version-clean`
VERSION=`$CLICKHOUSE_CLIENT --version-clean`
# If run from compile dir - use in-place compile binary and headers
[ -n "$BIN_DIR" ] && INTERNAL_COMPILER_PARAMS="--compiler_executable_root=${INTERNAL_COMPILER_BIN_ROOT:=$BUILD_DIR/dbms/programs/} --compiler_headers=$BUILD_DIR/dbms/programs/clang/headers/$VERSION/ --compiler_headers_root=$BUILD_DIR/dbms/programs/clang/headers/$VERSION/"
$GDB ${BIN_DIR}clickhouse-server --config-file=$CLICKHOUSE_CONFIG --log=$CLICKHOUSE_LOG $TEST_SERVER_PARAMS -- \
$GDB $CLICKHOUSE_SERVER --config-file=$CLICKHOUSE_CONFIG --log=$CLICKHOUSE_LOG $TEST_SERVER_PARAMS -- \
--http_port=$CLICKHOUSE_PORT_HTTP \
--tcp_port=$CLICKHOUSE_PORT_TCP \
--https_port=$CLICKHOUSE_PORT_HTTPS \
@ -117,9 +125,9 @@ if [ -n "$*" ]; then
else
TEST_RUN=${TEST_RUN=1}
TEST_PERF=${TEST_PERF=1}
CLICKHOUSE_CLIENT_QUERY="${BIN_DIR}clickhouse-client --config ${CLICKHOUSE_CONFIG_CLIENT} --port $CLICKHOUSE_PORT_TCP -m -n -q"
CLICKHOUSE_CLIENT_QUERY="${CLICKHOUSE_CLIENT} --config ${CLICKHOUSE_CONFIG_CLIENT} --port $CLICKHOUSE_PORT_TCP -m -n -q"
$CLICKHOUSE_CLIENT_QUERY 'SELECT * from system.build_options; SELECT * FROM system.clusters;'
CLICKHOUSE_TEST="env PATH=$PATH:$BIN_DIR ${TEST_DIR}clickhouse-test --binary ${BIN_DIR}clickhouse --configclient $CLICKHOUSE_CONFIG_CLIENT --configserver $CLICKHOUSE_CONFIG --tmp $DATA_DIR/tmp --queries $QUERIES_DIR $TEST_OPT0 $TEST_OPT"
CLICKHOUSE_TEST="env PATH=$PATH:$BIN_DIR ${TEST_DIR}clickhouse-test --binary ${BIN_DIR}${CLICKHOUSE_BINARY} --configclient $CLICKHOUSE_CONFIG_CLIENT --configserver $CLICKHOUSE_CONFIG --tmp $DATA_DIR/tmp --queries $QUERIES_DIR $TEST_OPT0 $TEST_OPT"
CLICKHOUSE_PERFORMANCE_TEST="${BIN_DIR}clickhouse-performance-test --port $CLICKHOUSE_PORT_TCP --recursive $CUR_DIR/performance --skip-tags=long"
if [ "${TEST_RUN_STRESS}" ]; then
# Running test in parallel will fail some results (tests can create/fill/drop same tables)

View File

@ -806,7 +806,7 @@ def run_tests(args):
stderr_element = et.Element("system-err")
stderr_element.text = et.CDATA(stderr)
report_testcase.append(stderr_element)
print(stderr)
print(stderr.encode('utf-8'))
if 'Connection refused' in stderr or 'Attempt to read after eof' in stderr:
SERVER_DIED = True

View File

@ -0,0 +1,18 @@
<yandex>
<remote_servers>
<testcluster>
<shard>
<replica>
<host>node1</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>node2</host>
<port>9000</port>
</replica>
</shard>
</testcluster>
</remote_servers>
</yandex>

View File

@ -0,0 +1,50 @@
import time
import pytest
from contextlib import contextmanager
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'], with_zookeeper=True)
node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'], with_zookeeper=True)
#test reproducing issue https://github.com/yandex/ClickHouse/issues/3162
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
for node in (node1, node2):
node.query('''
CREATE TABLE local_test (
t UInt64,
date Date MATERIALIZED toDate(t/1000),
shard UInt64,
col1 String,
col2 String
) ENGINE = MergeTree
PARTITION BY toRelativeDayNum(date)
ORDER BY (t)
SETTINGS index_granularity=8192
''')
node.query('''
CREATE TABLE dist_test (
t UInt64,
shard UInt64,
date Date MATERIALIZED toDate(t/1000),
col1 String,
col2 String
) Engine = Distributed(testcluster, default, local_test, shard)
''')
yield cluster
finally:
cluster.shutdown()
def test(started_cluster):
node1.query("INSERT INTO dist_test (t, shard, col1, col2) VALUES (1000, 1, 'foo', 'bar'), (1000, 2, 'x', 'y')")
#time.sleep(3)
assert node1.query("SELECT col1, col2 FROM dist_test WHERE (t < 3600000) AND (col1 = 'foo') ORDER BY t ASC") == "foo\tbar\n"

View File

@ -0,0 +1,136 @@
<yandex>
<tcp_port_secure>9440</tcp_port_secure>
<remote_servers>
<!-- Main cluster -->
<cluster>
<shard>
<internal_replication>true</internal_replication>
<replica>
<host>ch1</host>
<port>9440</port>
<secure>1</secure>
</replica>
<replica>
<host>ch2</host>
<port>9440</port>
<secure>1</secure>
</replica>
</shard>
<shard>
<internal_replication>true</internal_replication>
<replica>
<host>ch3</host>
<port>9440</port>
<secure>1</secure>
</replica>
<replica>
<host>ch4</host>
<port>9440</port>
<secure>1</secure>
</replica>
</shard>
</cluster>
<!-- Cluster with specified default database -->
<cluster2>
<shard>
<internal_replication>true</internal_replication>
<replica>
<host>ch1</host>
<port>9440</port>
<secure>1</secure>
<default_database>default</default_database>
</replica>
<replica>
<host>ch2</host>
<port>9440</port>
<secure>1</secure>
<default_database>test2</default_database>
</replica>
</shard>
<shard>
<internal_replication>true</internal_replication>
<replica>
<host>ch3</host>
<port>9440</port>
<secure>1</secure>
<default_database>default</default_database>
</replica>
<replica>
<host>ch4</host>
<port>9440</port>
<secure>1</secure>
<default_database>test2</default_database>
</replica>
</shard>
</cluster2>
<!-- Cluster without replicas -->
<cluster_no_replicas>
<shard>
<internal_replication>false</internal_replication>
<replica>
<host>ch1</host>
<port>9440</port>
<secure>1</secure>
</replica>
</shard>
<shard>
<internal_replication>false</internal_replication>
<replica>
<host>ch2</host>
<port>9440</port>
<secure>1</secure>
</replica>
</shard>
<shard>
<internal_replication>false</internal_replication>
<replica>
<host>ch3</host>
<port>9440</port>
<secure>1</secure>
</replica>
</shard>
<shard>
<internal_replication>false</internal_replication>
<replica>
<host>ch4</host>
<port>9440</port>
<secure>1</secure>
</replica>
</shard>
</cluster_no_replicas>
<!-- Cluster without internal replication -->
<cluster_without_replication>
<shard>
<internal_replication>false</internal_replication>
<replica>
<host>ch1</host>
<port>9440</port>
<secure>1</secure>
</replica>
<replica>
<host>ch2</host>
<port>9440</port>
<secure>1</secure>
</replica>
</shard>
<shard>
<internal_replication>false</internal_replication>
<replica>
<host>ch3</host>
<port>9440</port>
<secure>1</secure>
</replica>
<replica>
<host>ch4</host>
<port>9440</port>
<secure>1</secure>
</replica>
</shard>
</cluster_without_replication>
</remote_servers>
</yandex>

View File

@ -0,0 +1,8 @@
<yandex>
<distributed_ddl>
<path>/clickhouse/task_queue/ddl</path>
<max_tasks_in_queue>10</max_tasks_in_queue>
<task_max_lifetime>3600</task_max_lifetime>
<cleanup_delay_period>5</cleanup_delay_period>
</distributed_ddl>
</yandex>

View File

@ -0,0 +1,5 @@
<yandex>
<macros>
<cluster>cluster_no_replicas</cluster>
</macros>
</yandex>

View File

@ -0,0 +1,14 @@
<yandex>
<!-- Query log. Used only for queries with setting log_queries = 1. -->
<query_log>
<!-- What table to insert data. If table is not exist, it will be created.
When query log structure is changed after system update,
then old table will be renamed and new table will be created automatically.
-->
<database>system</database>
<table>query_log</table>
<!-- Interval of flushing data. -->
<flush_interval_milliseconds>1000</flush_interval_milliseconds>
</query_log>
</yandex>

View File

@ -0,0 +1,17 @@
<yandex>
<openSSL>
<server>
<certificateFile>/etc/clickhouse-server/server.crt</certificateFile>
<privateKeyFile>/etc/clickhouse-server/server.key</privateKeyFile>
<verificationMode>none</verificationMode>
<cacheSessions>true</cacheSessions>
</server>
<client>
<cacheSessions>true</cacheSessions>
<verificationMode>none</verificationMode>
<invalidCertificateHandler>
<name>AcceptCertificateHandler</name>
</invalidCertificateHandler>
</client>
</openSSL>
</yandex>

View File

@ -0,0 +1,6 @@
<yandex>
<zookeeper>
<!-- Required for correct timing in current test case -->
<session_timeout_ms replace="1">3000</session_timeout_ms>
</zookeeper>
</yandex>

View File

@ -0,0 +1,8 @@
-----BEGIN DH PARAMETERS-----
MIIBCAKCAQEAua92DDli13gJ+//ZXyGaggjIuidqB0crXfhUlsrBk9BV1hH3i7fR
XGP9rUdk2ubnB3k2ejBStL5oBrkHm9SzUFSQHqfDjLZjKoUpOEmuDc4cHvX1XTR5
Pr1vf5cd0yEncJWG5W4zyUB8k++SUdL2qaeslSs+f491HBLDYn/h8zCgRbBvxhxb
9qeho1xcbnWeqkN6Kc9bgGozA16P9NLuuLttNnOblkH+lMBf42BSne/TWt3AlGZf
slKmmZcySUhF8aKfJnLKbkBCFqOtFRh8zBA9a7g+BT/lSANATCDPaAk1YVih2EKb
dpc3briTDbRsiqg2JKMI7+VdULY9bh3EawIBAg==
-----END DH PARAMETERS-----

View File

@ -0,0 +1,19 @@
-----BEGIN CERTIFICATE-----
MIIC/TCCAeWgAwIBAgIJANjx1QSR77HBMA0GCSqGSIb3DQEBCwUAMBQxEjAQBgNV
BAMMCWxvY2FsaG9zdDAgFw0xODA3MzAxODE2MDhaGA8yMjkyMDUxNDE4MTYwOFow
FDESMBAGA1UEAwwJbG9jYWxob3N0MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIB
CgKCAQEAs9uSo6lJG8o8pw0fbVGVu0tPOljSWcVSXH9uiJBwlZLQnhN4SFSFohfI
4K8U1tBDTnxPLUo/V1K9yzoLiRDGMkwVj6+4+hE2udS2ePTQv5oaMeJ9wrs+5c9T
4pOtlq3pLAdm04ZMB1nbrEysceVudHRkQbGHzHp6VG29Fw7Ga6YpqyHQihRmEkTU
7UCYNA+Vk7aDPdMS/khweyTpXYZimaK9f0ECU3/VOeG3fH6Sp2X6FN4tUj/aFXEj
sRmU5G2TlYiSIUMF2JPdhSihfk1hJVALrHPTU38SOL+GyyBRWdNcrIwVwbpvsvPg
pryMSNxnpr0AK0dFhjwnupIv5hJIOQIDAQABo1AwTjAdBgNVHQ4EFgQUjPLb3uYC
kcamyZHK4/EV8jAP0wQwHwYDVR0jBBgwFoAUjPLb3uYCkcamyZHK4/EV8jAP0wQw
DAYDVR0TBAUwAwEB/zANBgkqhkiG9w0BAQsFAAOCAQEAM/ocuDvfPus/KpMVD51j
4IdlU8R0vmnYLQ+ygzOAo7+hUWP5j0yvq4ILWNmQX6HNvUggCgFv9bjwDFhb/5Vr
85ieWfTd9+LTjrOzTw4avdGwpX9G+6jJJSSq15tw5ElOIFb/qNA9O4dBiu8vn03C
L/zRSXrARhSqTW5w/tZkUcSTT+M5h28+Lgn9ysx4Ff5vi44LJ1NnrbJbEAIYsAAD
+UA+4MBFKx1r6hHINULev8+lCfkpwIaeS8RL+op4fr6kQPxnULw8wT8gkuc8I4+L
P9gg/xDHB44T3ADGZ5Ib6O0DJaNiToO6rnoaaxs0KkotbvDWvRoxEytSbXKoYjYp
0g==
-----END CERTIFICATE-----

View File

@ -0,0 +1,28 @@
-----BEGIN PRIVATE KEY-----
MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQCz25KjqUkbyjyn
DR9tUZW7S086WNJZxVJcf26IkHCVktCeE3hIVIWiF8jgrxTW0ENOfE8tSj9XUr3L
OguJEMYyTBWPr7j6ETa51LZ49NC/mhox4n3Cuz7lz1Pik62WreksB2bThkwHWdus
TKxx5W50dGRBsYfMenpUbb0XDsZrpimrIdCKFGYSRNTtQJg0D5WTtoM90xL+SHB7
JOldhmKZor1/QQJTf9U54bd8fpKnZfoU3i1SP9oVcSOxGZTkbZOViJIhQwXYk92F
KKF+TWElUAusc9NTfxI4v4bLIFFZ01ysjBXBum+y8+CmvIxI3GemvQArR0WGPCe6
ki/mEkg5AgMBAAECggEATrbIBIxwDJOD2/BoUqWkDCY3dGevF8697vFuZKIiQ7PP
TX9j4vPq0DfsmDjHvAPFkTHiTQXzlroFik3LAp+uvhCCVzImmHq0IrwvZ9xtB43f
7Pkc5P6h1l3Ybo8HJ6zRIY3TuLtLxuPSuiOMTQSGRL0zq3SQ5DKuGwkz+kVjHXUN
MR2TECFwMHKQ5VLrC+7PMpsJYyOMlDAWhRfUalxC55xOXTpaN8TxNnwQ8K2ISVY5
212Jz/a4hn4LdwxSz3Tiu95PN072K87HLWx3EdT6vW4Ge5P/A3y+smIuNAlanMnu
plHBRtpATLiTxZt/n6npyrfQVbYjSH7KWhB8hBHtaQKBgQDh9Cq1c/KtqDtE0Ccr
/r9tZNTUwBE6VP+3OJeKdEdtsfuxjOCkS1oAjgBJiSDOiWPh1DdoDeVZjPKq6pIu
Mq12OE3Doa8znfCXGbkSzEKOb2unKZMJxzrz99kXt40W5DtrqKPNb24CNqTiY8Aa
CjtcX+3weat82VRXvph6U8ltMwKBgQDLxjiQQzNoY7qvg7CwJCjf9qq8jmLK766g
1FHXopqS+dTxDLM8eJSRrpmxGWJvNeNc1uPhsKsKgotqAMdBUQTf7rSTbt4MyoH5
bUcRLtr+0QTK9hDWMOOvleqNXha68vATkohWYfCueNsC60qD44o8RZAS6UNy3ENq
cM1cxqe84wKBgQDKkHutWnooJtajlTxY27O/nZKT/HA1bDgniMuKaz4R4Gr1PIez
on3YW3V0d0P7BP6PWRIm7bY79vkiMtLEKdiKUGWeyZdo3eHvhDb/3DCawtau8L2K
GZsHVp2//mS1Lfz7Qh8/L/NedqCQ+L4iWiPnZ3THjjwn3CoZ05ucpvrAMwKBgB54
nay039MUVq44Owub3KDg+dcIU62U+cAC/9oG7qZbxYPmKkc4oL7IJSNecGHA5SbU
2268RFdl/gLz6tfRjbEOuOHzCjFPdvAdbysanpTMHLNc6FefJ+zxtgk9sJh0C4Jh
vxFrw9nTKKzfEl12gQ1SOaEaUIO0fEBGbe8ZpauRAoGAMAlGV+2/K4ebvAJKOVTa
dKAzQ+TD2SJmeR1HZmKDYddNqwtZlzg3v4ZhCk4eaUmGeC1Bdh8MDuB3QQvXz4Dr
vOIP4UVaOr+uM+7TgAgVnP4/K6IeJGzUDhX93pmpWhODfdu/oojEKVcpCojmEmS1
KCBtmIrQLqzMpnBpLNuSY+Q=
-----END PRIVATE KEY-----

View File

@ -0,0 +1,8 @@
<yandex>
<profiles>
<!-- Default profile settings. -->
<default>
<log_queries>1</log_queries>
</default>
</profiles>
</yandex>

View File

@ -0,0 +1,16 @@
<yandex>
<users>
<restricted_user>
<password></password>
<profile>default</profile>
<quota>default</quota>
<networks>
<ip>::/0</ip>
</networks>
<allow_databases>
<database>db1</database>
</allow_databases>
</restricted_user>
</users>
</yandex>

View File

@ -0,0 +1,391 @@
import os
import os.path as p
import sys
import time
import datetime
import pytest
from contextlib import contextmanager
sys.path.insert(0, os.path.dirname(os.path.dirname(os.path.abspath(__file__))))
from helpers.cluster import ClickHouseCluster
from helpers.network import PartitionManager, PartitionManagerDisbaler
from helpers.test_tools import TSV
def check_all_hosts_sucesfully_executed(tsv_content, num_hosts=None):
if num_hosts is None:
num_hosts = len(cluster.instances)
M = TSV.toMat(tsv_content)
hosts = [(l[0], l[1]) for l in M] # (host, port)
codes = [l[2] for l in M]
messages = [l[3] for l in M]
assert len(hosts) == num_hosts and len(set(hosts)) == num_hosts, "\n" + tsv_content
assert len(set(codes)) == 1, "\n" + tsv_content
assert codes[0] == "0", "\n" + tsv_content
def ddl_check_query(instance, query, num_hosts=None):
contents = instance.query(query)
check_all_hosts_sucesfully_executed(contents, num_hosts)
return contents
def ddl_check_there_are_no_dublicates(instance):
rows = instance.query("SELECT max(c), argMax(q, c) FROM (SELECT lower(query) AS q, count() AS c FROM system.query_log WHERE type=2 AND q LIKE '/* ddl_entry=query-%' GROUP BY query)")
assert len(rows) > 0 and rows[0][0] == "1", "dublicates on {} {}, query {}".format(instance.name, instance.ip_address)
# Make retries in case of UNKNOWN_STATUS_OF_INSERT or zkutil::KeeperException errors
def insert_reliable(instance, query_insert):
for i in xrange(100):
try:
instance.query(query_insert)
return
except Exception as e:
last_exception = e
s = str(e)
if not (s.find('Unknown status, client must retry') >= 0 or s.find('zkutil::KeeperException')):
raise e
raise last_exception
TEST_REPLICATED_ALTERS=False # TODO: Check code and turn on
cluster = ClickHouseCluster(__file__)
def replace_domains_to_ip_addresses_in_cluster_config(instances_to_replace):
clusters_config = open(p.join(cluster.base_dir, 'configs/config.d/clusters.xml')).read()
for inst_name, inst in cluster.instances.items():
clusters_config = clusters_config.replace(inst_name, str(inst.ip_address))
for inst_name in instances_to_replace:
inst = cluster.instances[inst_name]
cluster.instances[inst_name].exec_in_container(['bash', '-c', 'echo "$NEW_CONFIG" > /etc/clickhouse-server/config.d/clusters.xml'], environment={"NEW_CONFIG": clusters_config}, privileged=True)
# print cluster.instances[inst_name].exec_in_container(['cat', "/etc/clickhouse-server/config.d/clusters.xml"])
def init_cluster(cluster):
try:
for i in xrange(4):
cluster.add_instance(
'ch{}'.format(i+1),
config_dir="configs",
macros={"layer": 0, "shard": i/2 + 1, "replica": i%2 + 1},
with_zookeeper=True)
cluster.start()
# Replace config files for testing ability to set host in DNS and IP formats
replace_domains_to_ip_addresses_in_cluster_config(['ch1', 'ch3'])
# Select sacrifice instance to test CONNECTION_LOSS and server fail on it
sacrifice = cluster.instances['ch4']
cluster.pm_random_drops = PartitionManager()
cluster.pm_random_drops._add_rule({'probability': 0.01, 'destination': sacrifice.ip_address, 'source_port': 2181, 'action': 'REJECT --reject-with tcp-reset'})
cluster.pm_random_drops._add_rule({'probability': 0.01, 'source': sacrifice.ip_address, 'destination_port': 2181, 'action': 'REJECT --reject-with tcp-reset'})
# Initialize databases and service tables
instance = cluster.instances['ch1']
ddl_check_query(instance, """
CREATE TABLE IF NOT EXISTS all_tables ON CLUSTER 'cluster_no_replicas'
(database String, name String, engine String, metadata_modification_time DateTime)
ENGINE = Distributed('cluster_no_replicas', 'system', 'tables')
""")
ddl_check_query(instance, "CREATE DATABASE IF NOT EXISTS test ON CLUSTER 'cluster'")
except Exception as e:
print e
raise
@pytest.fixture(scope="module")
def started_cluster():
try:
init_cluster(cluster)
yield cluster
instance = cluster.instances['ch1']
ddl_check_query(instance, "DROP DATABASE test ON CLUSTER 'cluster'")
ddl_check_query(instance, "DROP DATABASE IF EXISTS test2 ON CLUSTER 'cluster'")
# Check query log to ensure that DDL queries are not executed twice
time.sleep(1.5)
for instance in cluster.instances.values():
ddl_check_there_are_no_dublicates(instance)
cluster.pm_random_drops.heal_all()
finally:
cluster.shutdown()
def test_default_database(started_cluster):
instance = cluster.instances['ch3']
ddl_check_query(instance, "CREATE DATABASE IF NOT EXISTS test2 ON CLUSTER 'cluster' FORMAT TSV")
ddl_check_query(instance, "DROP TABLE IF EXISTS null ON CLUSTER 'cluster' FORMAT TSV")
ddl_check_query(instance, "CREATE TABLE null ON CLUSTER 'cluster2' (s String DEFAULT 'escape\t\nme') ENGINE = Null")
contents = instance.query("SELECT hostName() AS h, database FROM all_tables WHERE name = 'null' ORDER BY h")
assert TSV(contents) == TSV("ch1\tdefault\nch2\ttest2\nch3\tdefault\nch4\ttest2\n")
ddl_check_query(instance, "DROP TABLE IF EXISTS null ON CLUSTER cluster2")
ddl_check_query(instance, "DROP DATABASE IF EXISTS test2 ON CLUSTER 'cluster'")
def test_create_view(started_cluster):
instance = cluster.instances['ch3']
ddl_check_query(instance, "CREATE VIEW test.super_simple_view ON CLUSTER 'cluster' AS SELECT * FROM system.numbers FORMAT TSV")
ddl_check_query(instance, "CREATE MATERIALIZED VIEW test.simple_mat_view ON CLUSTER 'cluster' ENGINE = Memory AS SELECT * FROM system.numbers FORMAT TSV")
ddl_check_query(instance, "DROP TABLE test.simple_mat_view ON CLUSTER 'cluster' FORMAT TSV")
ddl_check_query(instance, "DROP TABLE IF EXISTS test.super_simple_view2 ON CLUSTER 'cluster' FORMAT TSV")
ddl_check_query(instance, "CREATE TABLE test.super_simple ON CLUSTER 'cluster' (i Int8) ENGINE = Memory")
ddl_check_query(instance, "RENAME TABLE test.super_simple TO test.super_simple2 ON CLUSTER 'cluster' FORMAT TSV")
ddl_check_query(instance, "DROP TABLE test.super_simple2 ON CLUSTER 'cluster'")
def test_on_server_fail(started_cluster):
instance = cluster.instances['ch1']
kill_instance = cluster.instances['ch2']
ddl_check_query(instance, "DROP TABLE IF EXISTS test.test_server_fail ON CLUSTER 'cluster'")
kill_instance.get_docker_handle().stop()
request = instance.get_query_request("CREATE TABLE test.test_server_fail ON CLUSTER 'cluster' (i Int8) ENGINE=Null", timeout=30)
kill_instance.get_docker_handle().start()
ddl_check_query(instance, "DROP TABLE IF EXISTS test.__nope__ ON CLUSTER 'cluster'")
# Check query itself
check_all_hosts_sucesfully_executed(request.get_answer())
# And check query artefacts
contents = instance.query("SELECT hostName() AS h FROM all_tables WHERE database='test' AND name='test_server_fail' ORDER BY h")
assert TSV(contents) == TSV("ch1\nch2\nch3\nch4\n")
ddl_check_query(instance, "DROP TABLE test.test_server_fail ON CLUSTER 'cluster'")
def _test_on_connection_losses(cluster, zk_timeout):
instance = cluster.instances['ch1']
kill_instance = cluster.instances['ch2']
with PartitionManager() as pm:
pm.drop_instance_zk_connections(kill_instance)
request = instance.get_query_request("DROP TABLE IF EXISTS test.__nope__ ON CLUSTER 'cluster'", timeout=10)
time.sleep(zk_timeout)
pm.restore_instance_zk_connections(kill_instance)
check_all_hosts_sucesfully_executed(request.get_answer())
def test_on_connection_loss(started_cluster):
_test_on_connection_losses(cluster, 1.5) # connection loss will occur only (3 sec ZK timeout in config)
def test_on_session_expired(started_cluster):
_test_on_connection_losses(cluster, 4) # session should be expired (3 sec ZK timeout in config)
def test_replicated_alters(started_cluster):
instance = cluster.instances['ch2']
ddl_check_query(instance, "DROP TABLE IF EXISTS merge_for_alter ON CLUSTER cluster")
ddl_check_query(instance, "DROP TABLE IF EXISTS all_merge_32 ON CLUSTER cluster")
ddl_check_query(instance, "DROP TABLE IF EXISTS all_merge_64 ON CLUSTER cluster")
if not TEST_REPLICATED_ALTERS:
return
# Temporarily disable random ZK packet drops, they might broke creation if ReplicatedMergeTree replicas
firewall_drops_rules = cluster.pm_random_drops.pop_rules()
ddl_check_query(instance, """
CREATE TABLE IF NOT EXISTS merge_for_alter ON CLUSTER cluster (p Date, i Int32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/hits', '{replica}', p, p, 1)
""")
ddl_check_query(instance, """
CREATE TABLE IF NOT EXISTS all_merge_32 ON CLUSTER cluster (p Date, i Int32)
ENGINE = Distributed(cluster, default, merge_for_alter, i)
""")
ddl_check_query(instance, """
CREATE TABLE IF NOT EXISTS all_merge_64 ON CLUSTER cluster (p Date, i Int64, s String)
ENGINE = Distributed(cluster, default, merge_for_alter, i)
""")
for i in xrange(4):
k = (i / 2) * 2
insert_reliable(cluster.instances['ch{}'.format(i + 1)], "INSERT INTO merge_for_alter (i) VALUES ({})({})".format(k, k+1))
assert TSV(instance.query("SELECT i FROM all_merge_32 ORDER BY i")) == TSV(''.join(['{}\n'.format(x) for x in xrange(4)]))
ddl_check_query(instance, "ALTER TABLE merge_for_alter ON CLUSTER cluster MODIFY COLUMN i Int64")
ddl_check_query(instance, "ALTER TABLE merge_for_alter ON CLUSTER cluster ADD COLUMN s DEFAULT toString(i)")
assert TSV(instance.query("SELECT i, s FROM all_merge_64 ORDER BY i")) == TSV(''.join(['{}\t{}\n'.format(x,x) for x in xrange(4)]))
for i in xrange(4):
k = (i / 2) * 2 + 4
insert_reliable(cluster.instances['ch{}'.format(i + 1)], "INSERT INTO merge_for_alter (p, i) VALUES (31, {})(31, {})".format(k, k+1))
assert TSV(instance.query("SELECT i, s FROM all_merge_64 ORDER BY i")) == TSV(''.join(['{}\t{}\n'.format(x,x) for x in xrange(8)]))
ddl_check_query(instance, "ALTER TABLE merge_for_alter ON CLUSTER cluster DETACH PARTITION 197002")
assert TSV(instance.query("SELECT i, s FROM all_merge_64 ORDER BY i")) == TSV(''.join(['{}\t{}\n'.format(x,x) for x in xrange(4)]))
ddl_check_query(instance, "DROP TABLE merge_for_alter ON CLUSTER cluster")
# Enable random ZK packet drops
cluster.pm_random_drops.push_rules(firewall_drops_rules)
ddl_check_query(instance, "DROP TABLE all_merge_32 ON CLUSTER cluster")
ddl_check_query(instance, "DROP TABLE all_merge_64 ON CLUSTER cluster")
def test_simple_alters(started_cluster):
instance = cluster.instances['ch2']
ddl_check_query(instance, "DROP TABLE IF EXISTS merge ON CLUSTER cluster_without_replication")
ddl_check_query(instance, "DROP TABLE IF EXISTS all_merge_32 ON CLUSTER cluster_without_replication")
ddl_check_query(instance, "DROP TABLE IF EXISTS all_merge_64 ON CLUSTER cluster_without_replication")
ddl_check_query(instance, """
CREATE TABLE IF NOT EXISTS merge ON CLUSTER cluster_without_replication (p Date, i Int32)
ENGINE = MergeTree(p, p, 1)
""")
ddl_check_query(instance, """
CREATE TABLE IF NOT EXISTS all_merge_32 ON CLUSTER cluster_without_replication (p Date, i Int32)
ENGINE = Distributed(cluster_without_replication, default, merge, i)
""")
ddl_check_query(instance, """
CREATE TABLE IF NOT EXISTS all_merge_64 ON CLUSTER cluster_without_replication (p Date, i Int64, s String)
ENGINE = Distributed(cluster_without_replication, default, merge, i)
""")
for i in xrange(4):
k = (i / 2) * 2
cluster.instances['ch{}'.format(i + 1)].query("INSERT INTO merge (i) VALUES ({})({})".format(k, k+1))
assert TSV(instance.query("SELECT i FROM all_merge_32 ORDER BY i")) == TSV(''.join(['{}\n'.format(x) for x in xrange(4)]))
time.sleep(5)
ddl_check_query(instance, "ALTER TABLE merge ON CLUSTER cluster_without_replication MODIFY COLUMN i Int64")
time.sleep(5)
ddl_check_query(instance, "ALTER TABLE merge ON CLUSTER cluster_without_replication ADD COLUMN s DEFAULT toString(i) FORMAT TSV")
assert TSV(instance.query("SELECT i, s FROM all_merge_64 ORDER BY i")) == TSV(''.join(['{}\t{}\n'.format(x,x) for x in xrange(4)]))
for i in xrange(4):
k = (i / 2) * 2 + 4
cluster.instances['ch{}'.format(i + 1)].query("INSERT INTO merge (p, i) VALUES (31, {})(31, {})".format(k, k+1))
assert TSV(instance.query("SELECT i, s FROM all_merge_64 ORDER BY i")) == TSV(''.join(['{}\t{}\n'.format(x,x) for x in xrange(8)]))
ddl_check_query(instance, "ALTER TABLE merge ON CLUSTER cluster_without_replication DETACH PARTITION 197002")
assert TSV(instance.query("SELECT i, s FROM all_merge_64 ORDER BY i")) == TSV(''.join(['{}\t{}\n'.format(x,x) for x in xrange(4)]))
ddl_check_query(instance, "DROP TABLE merge ON CLUSTER cluster_without_replication")
ddl_check_query(instance, "DROP TABLE all_merge_32 ON CLUSTER cluster_without_replication")
ddl_check_query(instance, "DROP TABLE all_merge_64 ON CLUSTER cluster_without_replication")
def test_macro(started_cluster):
instance = cluster.instances['ch2']
ddl_check_query(instance, "CREATE TABLE tab ON CLUSTER '{cluster}' (value UInt8) ENGINE = Memory")
for i in xrange(4):
insert_reliable(cluster.instances['ch{}'.format(i + 1)], "INSERT INTO tab VALUES ({})".format(i))
ddl_check_query(instance, "CREATE TABLE distr ON CLUSTER '{cluster}' (value UInt8) ENGINE = Distributed('{cluster}', 'default', 'tab', value % 4)")
assert TSV(instance.query("SELECT value FROM distr ORDER BY value")) == TSV('0\n1\n2\n3\n')
assert TSV( cluster.instances['ch3'].query("SELECT value FROM distr ORDER BY value")) == TSV('0\n1\n2\n3\n')
ddl_check_query(instance, "DROP TABLE IF EXISTS distr ON CLUSTER '{cluster}'")
ddl_check_query(instance, "DROP TABLE IF EXISTS tab ON CLUSTER '{cluster}'")
def test_implicit_macros(started_cluster):
# Temporarily disable random ZK packet drops, they might broke creation if ReplicatedMergeTree replicas
firewall_drops_rules = cluster.pm_random_drops.pop_rules()
instance = cluster.instances['ch2']
ddl_check_query(instance, "DROP DATABASE IF EXISTS test_db ON CLUSTER '{cluster}'")
ddl_check_query(instance, "CREATE DATABASE IF NOT EXISTS test_db ON CLUSTER '{cluster}'")
ddl_check_query(instance, """
CREATE TABLE IF NOT EXISTS test_db.test_macro ON CLUSTER '{cluster}' (p Date, i Int32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/{layer}-{shard}/{table}', '{replica}', p, p, 1)
""")
# Check that table was created at correct path in zookeeper
assert cluster.get_kazoo_client('zoo1').exists('/clickhouse/tables/test_db/0-1/test_macro') is not None
# Enable random ZK packet drops
cluster.pm_random_drops.push_rules(firewall_drops_rules)
def test_allowed_databases(started_cluster):
instance = cluster.instances['ch2']
instance.query("CREATE DATABASE IF NOT EXISTS db1 ON CLUSTER cluster")
instance.query("CREATE DATABASE IF NOT EXISTS db2 ON CLUSTER cluster")
instance.query("CREATE TABLE db1.t1 ON CLUSTER cluster (i Int8) ENGINE = Memory", settings={"user" : "restricted_user"})
with pytest.raises(Exception):
instance.query("CREATE TABLE db2.t2 ON CLUSTER cluster (i Int8) ENGINE = Memory", settings={"user" : "restricted_user"})
with pytest.raises(Exception):
instance.query("CREATE TABLE t3 ON CLUSTER cluster (i Int8) ENGINE = Memory", settings={"user" : "restricted_user"})
with pytest.raises(Exception):
instance.query("DROP DATABASE db2 ON CLUSTER cluster", settings={"user" : "restricted_user"})
instance.query("DROP DATABASE db1 ON CLUSTER cluster", settings={"user" : "restricted_user"})
def test_kill_query(started_cluster):
instance = cluster.instances['ch3']
ddl_check_query(instance, "KILL QUERY ON CLUSTER 'cluster' WHERE NOT elapsed FORMAT TSV")
def test_detach_query(started_cluster):
instance = cluster.instances['ch3']
ddl_check_query(instance, "DROP TABLE IF EXISTS test_attach ON CLUSTER cluster FORMAT TSV")
ddl_check_query(instance, "CREATE TABLE test_attach ON CLUSTER cluster (i Int8)ENGINE = Log")
ddl_check_query(instance, "DETACH TABLE test_attach ON CLUSTER cluster FORMAT TSV")
ddl_check_query(instance, "ATTACH TABLE test_attach ON CLUSTER cluster")
def test_optimize_query(started_cluster):
instance = cluster.instances['ch3']
ddl_check_query(instance, "DROP TABLE IF EXISTS test_optimize ON CLUSTER cluster FORMAT TSV")
ddl_check_query(instance, "CREATE TABLE test_optimize ON CLUSTER cluster (p Date, i Int32) ENGINE = MergeTree(p, p, 8192)")
ddl_check_query(instance, "OPTIMIZE TABLE test_optimize ON CLUSTER cluster FORMAT TSV")
def test_create_as_select(started_cluster):
instance = cluster.instances['ch2']
ddl_check_query(instance, "CREATE TABLE test_as_select ON CLUSTER cluster ENGINE = Memory AS (SELECT 1 AS x UNION ALL SELECT 2 AS x)")
assert TSV(instance.query("SELECT x FROM test_as_select ORDER BY x")) == TSV("1\n2\n")
ddl_check_query(instance, "DROP TABLE IF EXISTS test_as_select ON CLUSTER cluster")
if __name__ == '__main__':
with contextmanager(started_cluster)() as cluster:
for name, instance in cluster.instances.items():
print name, instance.ip_address
raw_input("Cluster created, press any key to destroy...")

View File

@ -19,7 +19,6 @@ if [ -z $tcp_port_secure ]; then
cat $CURDIR/00505_secure.reference
else
CLICKHOUSE_CLIENT_BINARY=${CLICKHOUSE_CLIENT_BINARY:="${CLICKHOUSE_BINARY}-client"}
if [[ $CLICKHOUSE_CLIENT != *"--port"* ]]; then
CLICKHOUSE_CLIENT_SECURE=${CLICKHOUSE_CLIENT_SECURE:="$CLICKHOUSE_CLIENT_BINARY $USE_CONFIG --secure --port=$CLICKHOUSE_PORT_TCP_SECURE"}

View File

@ -14,11 +14,8 @@ INSERT INTO test.a2 VALUES (1, 2);
INSERT INTO test.a2 VALUES (1, 3);
INSERT INTO test.a2 VALUES (1, 4);
SELECT a, b FROM test.a1 LEFT JOIN (SELECT a, b FROM test.a2) USING a ORDER BY b; -- { serverError 417 }
SELECT a, b FROM test.a1 LEFT JOIN (SELECT a, b FROM test.a2) USING a ORDER BY b SETTINGS join_default_strictness='ANY';
SELECT a, b FROM test.a1 LEFT JOIN (SELECT a, b FROM test.a2) USING a ORDER BY b SETTINGS join_default_strictness='ALL';
SELECT a, b FROM test.a1 LEFT JOIN (SELECT a, b FROM test.a2) USING a ORDER BY b; -- default SETTINGS join_default_strictness='ALL';
DROP TABLE IF EXISTS test.a1;
DROP TABLE IF EXISTS test.a2;

View File

@ -0,0 +1,6 @@
drop table if exists test.t;
create table test.t (date Date, counter UInt64, sampler UInt64, alias_col alias date + 1) engine = MergeTree(date, intHash32(sampler), (counter, date, intHash32(sampler)), 8192);
insert into test.t values ('2018-01-01', 1, 1);
select alias_col from test.t sample 1 / 2 where date = '2018-01-01' and counter = 1 and sampler = 1;
drop table if exists test.t;

View File

@ -3,7 +3,7 @@ DROP TABLE IF EXISTS test.arraytest;
CREATE TABLE test.arraytest ( created_date Date DEFAULT toDate(created_at), created_at DateTime DEFAULT now(), strings Array(String) DEFAULT emptyArrayString()) ENGINE = MergeTree(created_date, cityHash64(created_at), (created_date, cityHash64(created_at)), 8192);
INSERT INTO test.arraytest (created_at, strings) VALUES (now(), ['aaaaa', 'bbbbb', 'ccccc']);
INSERT INTO test.arraytest (created_at, strings) VALUES (now(), ['aaaaa', 'bbbbb', null]); -- { clientError 321 }
INSERT INTO test.arraytest (created_at, strings) VALUES (now(), ['aaaaa', 'bbbbb', null]); -- { clientError 53 }
SELECT strings from test.arraytest;

View File

@ -0,0 +1,7 @@
CREATE MATERIALIZED VIEW test.t_mv ( date Date, platform Enum8('a' = 0, 'b' = 1), app Enum8('a' = 0, 'b' = 1)) ENGINE = MergeTree ORDER BY date SETTINGS index_granularity = 8192 AS SELECT date, platform, app FROM test.t WHERE (app = (SELECT min(app) FROM test.u )) AND (platform = (SELECT min(platform) FROM test.v ))
2000-01-01 a a
2000-01-02 b b
2000-01-03 a a
2000-01-04 b b
2000-01-02 b b
2000-01-03 a a

View File

@ -0,0 +1,38 @@
USE test;
DROP TABLE IF EXISTS t;
DROP TABLE IF EXISTS t_mv;
DROP TABLE IF EXISTS u;
DROP TABLE IF EXISTS v;
CREATE TABLE t
(
date Date,
platform Enum8('a' = 0, 'b' = 1),
app Enum8('a' = 0, 'b' = 1)
) ENGINE = Memory;
CREATE TABLE u (app Enum8('a' = 0, 'b' = 1)) ENGINE = Memory;
CREATE TABLE v (platform Enum8('a' = 0, 'b' = 1)) ENGINE = Memory;
INSERT INTO u VALUES ('b');
INSERT INTO v VALUES ('b');
CREATE MATERIALIZED VIEW t_mv ENGINE = MergeTree ORDER BY date
AS SELECT date, platform, app FROM t
WHERE app = (SELECT min(app) from u) AND platform = (SELECT min(platform) from v);
SHOW CREATE TABLE test.t_mv FORMAT TabSeparatedRaw;
INSERT INTO t VALUES ('2000-01-01', 'a', 'a') ('2000-01-02', 'b', 'b');
INSERT INTO u VALUES ('a');
INSERT INTO v VALUES ('a');
INSERT INTO t VALUES ('2000-01-03', 'a', 'a') ('2000-01-04', 'b', 'b');
SELECT * FROM t ORDER BY date;
SELECT * FROM t_mv ORDER BY date;
DROP TABLE IF EXISTS t;
DROP TABLE IF EXISTS t_mv;

View File

@ -1,14 +1,22 @@
export CLICKHOUSE_BINARY=${CLICKHOUSE_BINARY:="clickhouse"}
export CLICKHOUSE_CLIENT=${CLICKHOUSE_CLIENT:="${CLICKHOUSE_BINARY}-client"}
[ -x "$CLICKHOUSE_BINARY-client" ] && CLICKHOUSE_CLIENT_BINARY=${CLICKHOUSE_CLIENT_BINARY:=$CLICKHOUSE_BINARY-client} && CLICKHOUSE_CLIENT=${CLICKHOUSE_CLIENT:="$CLICKHOUSE_CLIENT_BINARY"}
[ -x "$CLICKHOUSE_BINARY" ] && CLICKHOUSE_CLIENT_BINARY=${CLICKHOUSE_CLIENT_BINARY:=$CLICKHOUSE_BINARY client} && CLICKHOUSE_CLIENT=${CLICKHOUSE_CLIENT:="$CLICKHOUSE_CLIENT_BINARY"}
export CLICKHOUSE_CLIENT_BINARY=${CLICKHOUSE_CLIENT_BINARY:=$CLICKHOUSE_BINARY-client}
export CLICKHOUSE_CLIENT=${CLICKHOUSE_CLIENT:="$CLICKHOUSE_CLIENT_BINARY"}
export CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL:="warning"}
export CLICKHOUSE_CLIENT="${CLICKHOUSE_CLIENT} --send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL} ${CLICKHOUSE_CLIENT_OPT}"
[ -x "${CLICKHOUSE_BINARY}-local" ] && CLICKHOUSE_LOCAL=${CLICKHOUSE_LOCAL:="${CLICKHOUSE_BINARY}-local"}
[ -x "${CLICKHOUSE_BINARY}" ] && CLICKHOUSE_LOCAL=${CLICKHOUSE_LOCAL:="${CLICKHOUSE_BINARY} local"}
export CLICKHOUSE_LOCAL=${CLICKHOUSE_LOCAL:="${CLICKHOUSE_BINARY}-local"}
export CLICKHOUSE_CONFIG=${CLICKHOUSE_CONFIG:="/etc/clickhouse-server/config.xml"}
export CLICKHOUSE_CONFIG_CLIENT=${CLICKHOUSE_CONFIG_CLIENT:="/etc/clickhouse-client/config.xml"}
[ -x "${CLICKHOUSE_BINARY}-extract-from-config" ] && CLICKHOUSE_EXTRACT_CONFIG=${CLICKHOUSE_EXTRACT_CONFIG:="$CLICKHOUSE_BINARY-extract-from-config --config=$CLICKHOUSE_CONFIG"}
[ -x "${CLICKHOUSE_BINARY}" ] && CLICKHOUSE_EXTRACT_CONFIG=${CLICKHOUSE_EXTRACT_CONFIG:="$CLICKHOUSE_BINARY extract-from-config --config=$CLICKHOUSE_CONFIG"}
export CLICKHOUSE_EXTRACT_CONFIG=${CLICKHOUSE_EXTRACT_CONFIG:="$CLICKHOUSE_BINARY-extract-from-config --config=$CLICKHOUSE_CONFIG"}
export CLICKHOUSE_CONFIG_GREP=${CLICKHOUSE_CONFIG_GREP:="/etc/clickhouse-server/config-preprocessed.xml"}
export CLICKHOUSE_HOST=${CLICKHOUSE_HOST:="localhost"}

View File

@ -0,0 +1,34 @@
FROM ubuntu:18.04
RUN apt-get update -y \
&& env DEBIAN_FRONTEND=noninteractive \
apt-get install --yes --no-install-recommends \
bash \
tzdata \
fakeroot \
debhelper \
zookeeper \
zookeeperd \
expect \
python \
python-lxml \
python-termcolor \
python-requests \
curl \
sudo \
openssl \
netcat-openbsd \
telnet
ENV TZ=Europe/Moscow
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
COPY zookeeper.xml /etc/clickhouse-server/config.d/zookeeper.xml
COPY listen.xml /etc/clickhouse-server/config.d/listen.xml
CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \
dpkg -i package_folder/clickhouse-server_*.deb; \
dpkg -i package_folder/clickhouse-client_*.deb; \
dpkg -i package_folder/clickhouse-test_*.deb; \
service zookeeper start; sleep 5; \
service clickhouse-server start && sleep 5 && clickhouse-test --shard --zookeeper 2>&1 | tee test_output/test_result.txt

View File

@ -0,0 +1 @@
<yandex><listen_host>::</listen_host></yandex>

View File

@ -0,0 +1,16 @@
<yandex>
<zookeeper>
<node>
<host>localhost</host>
<port>2181</port>
</node>
<node>
<host>yandex.ru</host>
<port>2181</port>
</node>
<node>
<host>111.0.1.2</host>
<port>2181</port>
</node>
</zookeeper>
</yandex>

View File

@ -18,8 +18,8 @@ brew install cmake ninja gcc icu4c mariadb-connector-c openssl libtool gettext r
## Checkout ClickHouse Sources
```bash
git clone --recursive --depth=10 git@github.com:yandex/ClickHouse.git
# or: git clone --recursive --depth=10 https://github.com/yandex/ClickHouse.git
git clone --recursive git@github.com:yandex/ClickHouse.git
# or: git clone --recursive https://github.com/yandex/ClickHouse.git
cd ClickHouse
```

View File

@ -53,7 +53,7 @@ For a description of request parameters, see [request description](../../query_l
- `ORDER BY` — Primary key.
A tuple of columns or arbitrary expressions. Example: `ORDER BY (CounterID, EventDate)`.
If a sampling expression is used, the primary key must contain it. Example: `ORDER BY (CounerID, EventDate, intHash32(UserID))`.
If a sampling expression is used, the primary key must contain it. Example: `ORDER BY (CounterID, EventDate, intHash32(UserID))`.
- `PARTITION BY` — The [partitioning key](custom_partitioning_key.md#table_engines-custom_partitioning_key).

View File

@ -107,8 +107,6 @@ echo -e "\nCurrent version is $VERSION_STRING"
gen_changelog "$VERSION_STRING" "" "$AUTHOR" ""
$CURDIR/dbms/src/Storages/System/StorageSystemContributors.sh
if [ -z "$USE_PBUILDER" ] ; then
DEB_CC=${DEB_CC:=`which gcc-7 gcc-8 gcc | head -n1`}
DEB_CXX=${DEB_CXX:=`which g++-7 g++-8 g++ | head -n1`}