mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-17 20:02:05 +00:00
Merge remote-tracking branch 'upstream/master' into fix25
This commit is contained in:
commit
9672f5cf83
@ -21,6 +21,7 @@
|
|||||||
#include <Common/StringUtils/StringUtils.h>
|
#include <Common/StringUtils/StringUtils.h>
|
||||||
|
|
||||||
#include <common/phdr_cache.h>
|
#include <common/phdr_cache.h>
|
||||||
|
#include <ext/scope_guard.h>
|
||||||
|
|
||||||
|
|
||||||
/// Universal executable for various clickhouse applications
|
/// Universal executable for various clickhouse applications
|
||||||
@ -130,8 +131,19 @@ bool isClickhouseApp(const std::string & app_suffix, std::vector<char *> & argv)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/// This allows to implement assert to forbid initialization of a class in static constructors.
|
||||||
|
/// Usage:
|
||||||
|
///
|
||||||
|
/// extern bool inside_main;
|
||||||
|
/// class C { C() { assert(inside_main); } };
|
||||||
|
bool inside_main = false;
|
||||||
|
|
||||||
|
|
||||||
int main(int argc_, char ** argv_)
|
int main(int argc_, char ** argv_)
|
||||||
{
|
{
|
||||||
|
inside_main = true;
|
||||||
|
SCOPE_EXIT({ inside_main = false; });
|
||||||
|
|
||||||
/// Reset new handler to default (that throws std::bad_alloc)
|
/// Reset new handler to default (that throws std::bad_alloc)
|
||||||
/// It is needed because LLVM library clobbers it.
|
/// It is needed because LLVM library clobbers it.
|
||||||
std::set_new_handler(nullptr);
|
std::set_new_handler(nullptr);
|
||||||
|
@ -336,7 +336,9 @@ void FunctionArrayEnumerateRankedExtended<Derived>::executeMethodImpl(
|
|||||||
/// Skipping offsets if no data in this array
|
/// Skipping offsets if no data in this array
|
||||||
if (prev_off == off)
|
if (prev_off == off)
|
||||||
{
|
{
|
||||||
want_clear = true;
|
|
||||||
|
if (depth_to_look > 2)
|
||||||
|
want_clear = true;
|
||||||
|
|
||||||
if (depth_to_look >= 2)
|
if (depth_to_look >= 2)
|
||||||
{
|
{
|
||||||
|
@ -1,13 +1,18 @@
|
|||||||
#include <Interpreters/AnalyzedJoin.h>
|
#include <Interpreters/AnalyzedJoin.h>
|
||||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||||
|
#include <Interpreters/Join.h>
|
||||||
|
|
||||||
#include <Parsers/ASTExpressionList.h>
|
#include <Parsers/ASTExpressionList.h>
|
||||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||||
#include <Parsers/ASTSelectQuery.h>
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
|
|
||||||
|
#include <Core/Block.h>
|
||||||
#include <Storages/IStorage.h>
|
#include <Storages/IStorage.h>
|
||||||
|
|
||||||
|
#include <DataTypes/DataTypeNullable.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -26,7 +31,6 @@ void AnalyzedJoin::addUsingKey(const ASTPtr & ast)
|
|||||||
|
|
||||||
void AnalyzedJoin::addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast)
|
void AnalyzedJoin::addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast)
|
||||||
{
|
{
|
||||||
with_using = false;
|
|
||||||
key_names_left.push_back(left_table_ast->getColumnName());
|
key_names_left.push_back(left_table_ast->getColumnName());
|
||||||
key_names_right.push_back(right_table_ast->getAliasOrColumnName());
|
key_names_right.push_back(right_table_ast->getAliasOrColumnName());
|
||||||
|
|
||||||
@ -37,7 +41,7 @@ void AnalyzedJoin::addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast)
|
|||||||
/// @return how many times right key appears in ON section.
|
/// @return how many times right key appears in ON section.
|
||||||
size_t AnalyzedJoin::rightKeyInclusion(const String & name) const
|
size_t AnalyzedJoin::rightKeyInclusion(const String & name) const
|
||||||
{
|
{
|
||||||
if (with_using)
|
if (hasUsing())
|
||||||
return 0;
|
return 0;
|
||||||
|
|
||||||
size_t count = 0;
|
size_t count = 0;
|
||||||
@ -101,6 +105,120 @@ std::unordered_map<String, String> AnalyzedJoin::getOriginalColumnsMap(const Nam
|
|||||||
return out;
|
return out;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
ASTPtr AnalyzedJoin::leftKeysList() const
|
||||||
|
{
|
||||||
|
ASTPtr keys_list = std::make_shared<ASTExpressionList>();
|
||||||
|
keys_list->children = key_asts_left;
|
||||||
|
return keys_list;
|
||||||
|
}
|
||||||
|
|
||||||
|
ASTPtr AnalyzedJoin::rightKeysList() const
|
||||||
|
{
|
||||||
|
ASTPtr keys_list = std::make_shared<ASTExpressionList>();
|
||||||
|
if (hasOn())
|
||||||
|
keys_list->children = key_asts_right;
|
||||||
|
return keys_list;
|
||||||
|
}
|
||||||
|
|
||||||
|
Names AnalyzedJoin::requiredJoinedNames() const
|
||||||
|
{
|
||||||
|
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);
|
||||||
|
|
||||||
|
return Names(required_columns_set.begin(), required_columns_set.end());
|
||||||
|
}
|
||||||
|
|
||||||
|
void AnalyzedJoin::appendRequiredColumns(const Block & sample, NameSet & required_columns) const
|
||||||
|
{
|
||||||
|
for (auto & column : key_names_right)
|
||||||
|
if (!sample.has(column))
|
||||||
|
required_columns.insert(column);
|
||||||
|
|
||||||
|
for (auto & column : columns_added_by_join)
|
||||||
|
if (!sample.has(column.name))
|
||||||
|
required_columns.insert(column.name);
|
||||||
|
}
|
||||||
|
|
||||||
|
void AnalyzedJoin::addJoinedColumn(const NameAndTypePair & joined_column)
|
||||||
|
{
|
||||||
|
if (join_use_nulls && isLeftOrFull(table_join.kind))
|
||||||
|
{
|
||||||
|
auto type = joined_column.type->canBeInsideNullable() ? makeNullable(joined_column.type) : joined_column.type;
|
||||||
|
columns_added_by_join.emplace_back(NameAndTypePair(joined_column.name, std::move(type)));
|
||||||
|
}
|
||||||
|
else
|
||||||
|
columns_added_by_join.push_back(joined_column);
|
||||||
|
}
|
||||||
|
|
||||||
|
void AnalyzedJoin::addJoinedColumnsAndCorrectNullability(Block & sample_block) const
|
||||||
|
{
|
||||||
|
bool right_or_full_join = isRightOrFull(table_join.kind);
|
||||||
|
bool left_or_full_join = isLeftOrFull(table_join.kind);
|
||||||
|
|
||||||
|
for (auto & col : sample_block)
|
||||||
|
{
|
||||||
|
/// Materialize column.
|
||||||
|
/// Column is not empty if it is constant, but after Join all constants will be materialized.
|
||||||
|
/// So, we need remove constants from header.
|
||||||
|
if (col.column)
|
||||||
|
col.column = nullptr;
|
||||||
|
|
||||||
|
bool make_nullable = join_use_nulls && right_or_full_join;
|
||||||
|
|
||||||
|
if (make_nullable && col.type->canBeInsideNullable())
|
||||||
|
col.type = makeNullable(col.type);
|
||||||
|
}
|
||||||
|
|
||||||
|
for (const auto & col : columns_added_by_join)
|
||||||
|
{
|
||||||
|
auto res_type = col.type;
|
||||||
|
|
||||||
|
bool make_nullable = join_use_nulls && left_or_full_join;
|
||||||
|
|
||||||
|
if (!make_nullable)
|
||||||
|
{
|
||||||
|
/// Keys from right table are usually not stored in Join, but copied from the left one.
|
||||||
|
/// So, if left key is nullable, let's make right key nullable too.
|
||||||
|
/// Note: for some join types it's not needed and, probably, may be removed.
|
||||||
|
/// Note: changing this code, take into account the implementation in Join.cpp.
|
||||||
|
auto it = std::find(key_names_right.begin(), key_names_right.end(), col.name);
|
||||||
|
if (it != key_names_right.end())
|
||||||
|
{
|
||||||
|
auto pos = it - key_names_right.begin();
|
||||||
|
const auto & left_key_name = key_names_left[pos];
|
||||||
|
make_nullable = sample_block.getByName(left_key_name).type->isNullable();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (make_nullable && res_type->canBeInsideNullable())
|
||||||
|
res_type = makeNullable(res_type);
|
||||||
|
|
||||||
|
sample_block.insert(ColumnWithTypeAndName(nullptr, res_type, col.name));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
bool AnalyzedJoin::sameJoin(const AnalyzedJoin * x, const AnalyzedJoin * y)
|
||||||
|
{
|
||||||
|
if (!x && !y)
|
||||||
|
return true;
|
||||||
|
if (!x || !y)
|
||||||
|
return false;
|
||||||
|
|
||||||
|
return x->table_join.kind == y->table_join.kind
|
||||||
|
&& x->table_join.strictness == y->table_join.strictness
|
||||||
|
&& x->key_names_left == y->key_names_left
|
||||||
|
&& x->key_names_right == y->key_names_right
|
||||||
|
&& x->columns_added_by_join == y->columns_added_by_join;
|
||||||
|
}
|
||||||
|
|
||||||
|
JoinPtr AnalyzedJoin::makeHashJoin(const Block & sample_block, const SizeLimits & size_limits_for_join) const
|
||||||
|
{
|
||||||
|
auto join = std::make_shared<Join>(key_names_right, join_use_nulls, size_limits_for_join, table_join.kind, table_join.strictness);
|
||||||
|
join->setSampleBlock(sample_block);
|
||||||
|
return join;
|
||||||
|
}
|
||||||
|
|
||||||
NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpression & table_expression, const Context & context)
|
NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpression & table_expression, const Context & context)
|
||||||
{
|
{
|
||||||
NamesAndTypesList names_and_type_list;
|
NamesAndTypesList names_and_type_list;
|
||||||
|
@ -2,7 +2,8 @@
|
|||||||
|
|
||||||
#include <Core/Names.h>
|
#include <Core/Names.h>
|
||||||
#include <Core/NamesAndTypes.h>
|
#include <Core/NamesAndTypes.h>
|
||||||
#include <Parsers/IAST.h>
|
#include <Core/SettingsCommon.h>
|
||||||
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||||
|
|
||||||
#include <utility>
|
#include <utility>
|
||||||
#include <memory>
|
#include <memory>
|
||||||
@ -13,6 +14,10 @@ namespace DB
|
|||||||
class Context;
|
class Context;
|
||||||
class ASTSelectQuery;
|
class ASTSelectQuery;
|
||||||
struct DatabaseAndTableWithAlias;
|
struct DatabaseAndTableWithAlias;
|
||||||
|
class Block;
|
||||||
|
|
||||||
|
class Join;
|
||||||
|
using JoinPtr = std::shared_ptr<Join>;
|
||||||
|
|
||||||
struct AnalyzedJoin
|
struct AnalyzedJoin
|
||||||
{
|
{
|
||||||
@ -30,18 +35,19 @@ struct AnalyzedJoin
|
|||||||
|
|
||||||
private:
|
private:
|
||||||
friend class SyntaxAnalyzer;
|
friend class SyntaxAnalyzer;
|
||||||
friend struct SyntaxAnalyzerResult;
|
|
||||||
friend class ExpressionAnalyzer;
|
|
||||||
friend class SelectQueryExpressionAnalyzer;
|
|
||||||
|
|
||||||
Names key_names_left;
|
Names key_names_left;
|
||||||
Names key_names_right; /// Duplicating names are qualified.
|
Names key_names_right; /// Duplicating names are qualified.
|
||||||
ASTs key_asts_left;
|
ASTs key_asts_left;
|
||||||
ASTs key_asts_right;
|
ASTs key_asts_right;
|
||||||
bool with_using = true;
|
ASTTableJoin table_join;
|
||||||
|
bool join_use_nulls = false;
|
||||||
|
|
||||||
/// All columns which can be read from joined table. Duplicating names are qualified.
|
/// All columns which can be read from joined table. Duplicating names are qualified.
|
||||||
NamesAndTypesList columns_from_joined_table;
|
NamesAndTypesList columns_from_joined_table;
|
||||||
|
/// Columns will be added to block by JOIN. It's a subset of columns_from_joined_table with corrected Nullability
|
||||||
|
NamesAndTypesList columns_added_by_join;
|
||||||
|
|
||||||
/// Name -> original name. Names are the same as in columns_from_joined_table list.
|
/// Name -> original name. Names are the same as in columns_from_joined_table list.
|
||||||
std::unordered_map<String, String> original_names;
|
std::unordered_map<String, String> original_names;
|
||||||
/// Original name -> name. Only ranamed columns.
|
/// Original name -> name. Only ranamed columns.
|
||||||
@ -51,8 +57,8 @@ public:
|
|||||||
void addUsingKey(const ASTPtr & ast);
|
void addUsingKey(const ASTPtr & ast);
|
||||||
void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast);
|
void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast);
|
||||||
|
|
||||||
bool hasUsing() const { return with_using; }
|
bool hasUsing() const { return table_join.using_expression_list != nullptr; }
|
||||||
bool hasOn() const { return !with_using; }
|
bool hasOn() const { return !hasUsing(); }
|
||||||
|
|
||||||
NameSet getQualifiedColumnsSet() const;
|
NameSet getQualifiedColumnsSet() const;
|
||||||
NameSet getOriginalColumnsSet() const;
|
NameSet getOriginalColumnsSet() const;
|
||||||
@ -60,6 +66,22 @@ public:
|
|||||||
|
|
||||||
void deduplicateAndQualifyColumnNames(const NameSet & left_table_columns, const String & right_table_prefix);
|
void deduplicateAndQualifyColumnNames(const NameSet & left_table_columns, const String & right_table_prefix);
|
||||||
size_t rightKeyInclusion(const String & name) const;
|
size_t rightKeyInclusion(const String & name) const;
|
||||||
|
|
||||||
|
void appendRequiredColumns(const Block & sample, NameSet & required_columns) const;
|
||||||
|
void addJoinedColumn(const NameAndTypePair & joined_column);
|
||||||
|
void addJoinedColumnsAndCorrectNullability(Block & sample_block) const;
|
||||||
|
|
||||||
|
ASTPtr leftKeysList() const;
|
||||||
|
ASTPtr rightKeysList() const; /// For ON syntax only
|
||||||
|
|
||||||
|
Names requiredJoinedNames() const;
|
||||||
|
const Names & keyNamesLeft() const { return key_names_left; }
|
||||||
|
const NamesAndTypesList & columnsFromJoinedTable() const { return columns_from_joined_table; }
|
||||||
|
const NamesAndTypesList & columnsAddedByJoin() const { return columns_added_by_join; }
|
||||||
|
|
||||||
|
JoinPtr makeHashJoin(const Block & sample_block, const SizeLimits & size_limits_for_join) const;
|
||||||
|
|
||||||
|
static bool sameJoin(const AnalyzedJoin * x, const AnalyzedJoin * y);
|
||||||
};
|
};
|
||||||
|
|
||||||
struct ASTTableExpression;
|
struct ASTTableExpression;
|
||||||
|
@ -12,7 +12,6 @@
|
|||||||
#include <Functions/IFunction.h>
|
#include <Functions/IFunction.h>
|
||||||
#include <set>
|
#include <set>
|
||||||
#include <optional>
|
#include <optional>
|
||||||
#include <DataTypes/DataTypeNullable.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace ProfileEvents
|
namespace ProfileEvents
|
||||||
@ -45,7 +44,8 @@ Names ExpressionAction::getNeededColumns() const
|
|||||||
|
|
||||||
res.insert(res.end(), array_joined_columns.begin(), array_joined_columns.end());
|
res.insert(res.end(), array_joined_columns.begin(), array_joined_columns.end());
|
||||||
|
|
||||||
res.insert(res.end(), join_key_names_left.begin(), join_key_names_left.end());
|
if (join_params)
|
||||||
|
res.insert(res.end(), join_params->keyNamesLeft().begin(), join_params->keyNamesLeft().end());
|
||||||
|
|
||||||
for (const auto & column : projection)
|
for (const auto & column : projection)
|
||||||
res.push_back(column.first);
|
res.push_back(column.first);
|
||||||
@ -159,20 +159,12 @@ ExpressionAction ExpressionAction::arrayJoin(const NameSet & array_joined_column
|
|||||||
return a;
|
return a;
|
||||||
}
|
}
|
||||||
|
|
||||||
ExpressionAction ExpressionAction::ordinaryJoin(
|
ExpressionAction ExpressionAction::ordinaryJoin(std::shared_ptr<AnalyzedJoin> join_params, std::shared_ptr<const Join> hash_join)
|
||||||
const ASTTableJoin & join_params,
|
|
||||||
std::shared_ptr<const Join> join_,
|
|
||||||
const Names & join_key_names_left,
|
|
||||||
const Names & join_key_names_right,
|
|
||||||
const NamesAndTypesList & columns_added_by_join_)
|
|
||||||
{
|
{
|
||||||
ExpressionAction a;
|
ExpressionAction a;
|
||||||
a.type = JOIN;
|
a.type = JOIN;
|
||||||
a.join = std::move(join_);
|
a.join_params = join_params;
|
||||||
a.join_kind = join_params.kind;
|
a.join = hash_join;
|
||||||
a.join_key_names_left = join_key_names_left;
|
|
||||||
a.join_key_names_right = join_key_names_right;
|
|
||||||
a.columns_added_by_join = columns_added_by_join_;
|
|
||||||
return a;
|
return a;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -277,51 +269,7 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings,
|
|||||||
|
|
||||||
case JOIN:
|
case JOIN:
|
||||||
{
|
{
|
||||||
bool is_null_used_as_default = settings.join_use_nulls;
|
join_params->addJoinedColumnsAndCorrectNullability(sample_block);
|
||||||
bool right_or_full_join = isRightOrFull(join_kind);
|
|
||||||
bool left_or_full_join = isLeftOrFull(join_kind);
|
|
||||||
|
|
||||||
for (auto & col : sample_block)
|
|
||||||
{
|
|
||||||
/// Materialize column.
|
|
||||||
/// Column is not empty if it is constant, but after Join all constants will be materialized.
|
|
||||||
/// So, we need remove constants from header.
|
|
||||||
if (col.column)
|
|
||||||
col.column = nullptr;
|
|
||||||
|
|
||||||
bool make_nullable = is_null_used_as_default && right_or_full_join;
|
|
||||||
|
|
||||||
if (make_nullable && col.type->canBeInsideNullable())
|
|
||||||
col.type = makeNullable(col.type);
|
|
||||||
}
|
|
||||||
|
|
||||||
for (const auto & col : columns_added_by_join)
|
|
||||||
{
|
|
||||||
auto res_type = col.type;
|
|
||||||
|
|
||||||
bool make_nullable = is_null_used_as_default && left_or_full_join;
|
|
||||||
|
|
||||||
if (!make_nullable)
|
|
||||||
{
|
|
||||||
/// Keys from right table are usually not stored in Join, but copied from the left one.
|
|
||||||
/// So, if left key is nullable, let's make right key nullable too.
|
|
||||||
/// Note: for some join types it's not needed and, probably, may be removed.
|
|
||||||
/// Note: changing this code, take into account the implementation in Join.cpp.
|
|
||||||
auto it = std::find(join_key_names_right.begin(), join_key_names_right.end(), col.name);
|
|
||||||
if (it != join_key_names_right.end())
|
|
||||||
{
|
|
||||||
auto pos = it - join_key_names_right.begin();
|
|
||||||
const auto & left_key_name = join_key_names_left[pos];
|
|
||||||
make_nullable = sample_block.getByName(left_key_name).type->isNullable();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (make_nullable && res_type->canBeInsideNullable())
|
|
||||||
res_type = makeNullable(res_type);
|
|
||||||
|
|
||||||
sample_block.insert(ColumnWithTypeAndName(nullptr, res_type, col.name));
|
|
||||||
}
|
|
||||||
|
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -527,7 +475,7 @@ void ExpressionAction::execute(Block & block, bool dry_run) const
|
|||||||
|
|
||||||
case JOIN:
|
case JOIN:
|
||||||
{
|
{
|
||||||
join->joinBlock(block, join_key_names_left, columns_added_by_join);
|
join->joinBlock(block, *join_params);
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -645,9 +593,10 @@ std::string ExpressionAction::toString() const
|
|||||||
|
|
||||||
case JOIN:
|
case JOIN:
|
||||||
ss << "JOIN ";
|
ss << "JOIN ";
|
||||||
for (NamesAndTypesList::const_iterator it = columns_added_by_join.begin(); it != columns_added_by_join.end(); ++it)
|
for (NamesAndTypesList::const_iterator it = join_params->columnsAddedByJoin().begin();
|
||||||
|
it != join_params->columnsAddedByJoin().end(); ++it)
|
||||||
{
|
{
|
||||||
if (it != columns_added_by_join.begin())
|
if (it != join_params->columnsAddedByJoin().begin())
|
||||||
ss << ", ";
|
ss << ", ";
|
||||||
ss << it->name;
|
ss << it->name;
|
||||||
}
|
}
|
||||||
@ -1220,7 +1169,7 @@ BlockInputStreamPtr ExpressionActions::createStreamWithNonJoinedDataIfFullOrRigh
|
|||||||
for (const auto & action : actions)
|
for (const auto & action : actions)
|
||||||
if (action.join && isRightOrFull(action.join->getKind()))
|
if (action.join && isRightOrFull(action.join->getKind()))
|
||||||
return action.join->createStreamWithNonJoinedRows(
|
return action.join->createStreamWithNonJoinedRows(
|
||||||
source_header, action.join_key_names_left, action.columns_added_by_join, max_block_size);
|
source_header, *action.join_params, max_block_size);
|
||||||
|
|
||||||
return {};
|
return {};
|
||||||
}
|
}
|
||||||
@ -1267,7 +1216,7 @@ UInt128 ExpressionAction::ActionHash::operator()(const ExpressionAction & action
|
|||||||
hash.update(col);
|
hash.update(col);
|
||||||
break;
|
break;
|
||||||
case JOIN:
|
case JOIN:
|
||||||
for (const auto & col : action.columns_added_by_join)
|
for (const auto & col : action.join_params->columnsAddedByJoin())
|
||||||
hash.update(col.name);
|
hash.update(col.name);
|
||||||
break;
|
break;
|
||||||
case PROJECT:
|
case PROJECT:
|
||||||
@ -1326,9 +1275,7 @@ bool ExpressionAction::operator==(const ExpressionAction & other) const
|
|||||||
&& array_joined_columns == other.array_joined_columns
|
&& array_joined_columns == other.array_joined_columns
|
||||||
&& array_join_is_left == other.array_join_is_left
|
&& array_join_is_left == other.array_join_is_left
|
||||||
&& join == other.join
|
&& join == other.join
|
||||||
&& join_key_names_left == other.join_key_names_left
|
&& AnalyzedJoin::sameJoin(join_params.get(), other.join_params.get())
|
||||||
&& join_key_names_right == other.join_key_names_right
|
|
||||||
&& columns_added_by_join == other.columns_added_by_join
|
|
||||||
&& projection == other.projection
|
&& projection == other.projection
|
||||||
&& is_function_compiled == other.is_function_compiled;
|
&& is_function_compiled == other.is_function_compiled;
|
||||||
}
|
}
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
#include <Core/Settings.h>
|
#include <Core/Settings.h>
|
||||||
#include <DataStreams/IBlockStream_fwd.h>
|
#include <DataStreams/IBlockStream_fwd.h>
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
|
#include <Interpreters/AnalyzedJoin.h>
|
||||||
#include <Common/SipHash.h>
|
#include <Common/SipHash.h>
|
||||||
#include "config_core.h"
|
#include "config_core.h"
|
||||||
#include <unordered_map>
|
#include <unordered_map>
|
||||||
@ -104,11 +105,8 @@ public:
|
|||||||
bool unaligned_array_join = false;
|
bool unaligned_array_join = false;
|
||||||
|
|
||||||
/// For JOIN
|
/// For JOIN
|
||||||
|
std::shared_ptr<AnalyzedJoin> join_params = nullptr;
|
||||||
std::shared_ptr<const Join> join;
|
std::shared_ptr<const Join> join;
|
||||||
ASTTableJoin::Kind join_kind;
|
|
||||||
Names join_key_names_left;
|
|
||||||
Names join_key_names_right;
|
|
||||||
NamesAndTypesList columns_added_by_join;
|
|
||||||
|
|
||||||
/// For PROJECT.
|
/// For PROJECT.
|
||||||
NamesWithAliases projection;
|
NamesWithAliases projection;
|
||||||
@ -124,9 +122,7 @@ public:
|
|||||||
static ExpressionAction project(const Names & projected_columns_);
|
static ExpressionAction project(const Names & projected_columns_);
|
||||||
static ExpressionAction addAliases(const NamesWithAliases & aliased_columns_);
|
static ExpressionAction addAliases(const NamesWithAliases & aliased_columns_);
|
||||||
static ExpressionAction arrayJoin(const NameSet & array_joined_columns, bool array_join_is_left, const Context & context);
|
static ExpressionAction arrayJoin(const NameSet & array_joined_columns, bool array_join_is_left, const Context & context);
|
||||||
static ExpressionAction ordinaryJoin(const ASTTableJoin & join_params, std::shared_ptr<const Join> join_,
|
static ExpressionAction ordinaryJoin(std::shared_ptr<AnalyzedJoin> join_params, std::shared_ptr<const Join> hash_join);
|
||||||
const Names & join_key_names_left, const Names & join_key_names_right,
|
|
||||||
const NamesAndTypesList & columns_added_by_join_);
|
|
||||||
|
|
||||||
/// Which columns necessary to perform this action.
|
/// Which columns necessary to perform this action.
|
||||||
Names getNeededColumns() const;
|
Names getNeededColumns() const;
|
||||||
|
@ -29,7 +29,6 @@
|
|||||||
#include <Interpreters/PredicateExpressionsOptimizer.h>
|
#include <Interpreters/PredicateExpressionsOptimizer.h>
|
||||||
#include <Interpreters/ExternalDictionaries.h>
|
#include <Interpreters/ExternalDictionaries.h>
|
||||||
#include <Interpreters/Set.h>
|
#include <Interpreters/Set.h>
|
||||||
#include <Interpreters/Join.h>
|
|
||||||
|
|
||||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
|
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
|
||||||
@ -134,14 +133,8 @@ void ExpressionAnalyzer::analyzeAggregation()
|
|||||||
const ASTTablesInSelectQueryElement * join = select_query->join();
|
const ASTTablesInSelectQueryElement * join = select_query->join();
|
||||||
if (join)
|
if (join)
|
||||||
{
|
{
|
||||||
const auto & table_join = join->table_join->as<ASTTableJoin &>();
|
getRootActions(analyzedJoin().leftKeysList(), true, temp_actions);
|
||||||
if (table_join.using_expression_list)
|
addJoinAction(temp_actions);
|
||||||
getRootActions(table_join.using_expression_list, true, temp_actions);
|
|
||||||
if (table_join.on_expression)
|
|
||||||
for (const auto & key_ast : analyzedJoin().key_asts_left)
|
|
||||||
getRootActions(key_ast, true, temp_actions);
|
|
||||||
|
|
||||||
addJoinAction(table_join, temp_actions);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -298,7 +291,8 @@ void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node)
|
|||||||
{
|
{
|
||||||
NamesAndTypesList temp_columns = sourceColumns();
|
NamesAndTypesList temp_columns = sourceColumns();
|
||||||
temp_columns.insert(temp_columns.end(), array_join_columns.begin(), array_join_columns.end());
|
temp_columns.insert(temp_columns.end(), array_join_columns.begin(), array_join_columns.end());
|
||||||
temp_columns.insert(temp_columns.end(), columnsAddedByJoin().begin(), columnsAddedByJoin().end());
|
temp_columns.insert(temp_columns.end(),
|
||||||
|
analyzedJoin().columnsAddedByJoin().begin(), analyzedJoin().columnsAddedByJoin().end());
|
||||||
|
|
||||||
ExpressionActionsPtr temp_actions = std::make_shared<ExpressionActions>(temp_columns, context);
|
ExpressionActionsPtr temp_actions = std::make_shared<ExpressionActions>(temp_columns, context);
|
||||||
getRootActions(left_in_operand, true, temp_actions);
|
getRootActions(left_in_operand, true, temp_actions);
|
||||||
@ -412,22 +406,10 @@ bool SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & cha
|
|||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
static void appendRequiredColumns(
|
|
||||||
NameSet & required_columns, const Block & sample, const Names & key_names_right, const NamesAndTypesList & columns_added_by_join)
|
|
||||||
{
|
|
||||||
for (auto & column : key_names_right)
|
|
||||||
if (!sample.has(column))
|
|
||||||
required_columns.insert(column);
|
|
||||||
|
|
||||||
for (auto & column : columns_added_by_join)
|
|
||||||
if (!sample.has(column.name))
|
|
||||||
required_columns.insert(column.name);
|
|
||||||
}
|
|
||||||
|
|
||||||
/// It's possible to set nullptr as join for only_types mode
|
/// It's possible to set nullptr as join for only_types mode
|
||||||
void ExpressionAnalyzer::addJoinAction(const ASTTableJoin & join_params, ExpressionActionsPtr & actions, JoinPtr join) const
|
void ExpressionAnalyzer::addJoinAction(ExpressionActionsPtr & actions, JoinPtr join) const
|
||||||
{
|
{
|
||||||
actions->add(ExpressionAction::ordinaryJoin(join_params, std::move(join), analyzedJoin().key_names_left, analyzedJoin().key_names_right, columnsAddedByJoin()));
|
actions->add(ExpressionAction::ordinaryJoin(syntax->analyzed_join, join));
|
||||||
}
|
}
|
||||||
|
|
||||||
bool SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_types)
|
bool SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_types)
|
||||||
@ -438,16 +420,11 @@ bool SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, b
|
|||||||
|
|
||||||
SubqueryForSet & subquery_for_set = getSubqueryForJoin(*ast_join);
|
SubqueryForSet & subquery_for_set = getSubqueryForJoin(*ast_join);
|
||||||
|
|
||||||
ASTPtr left_keys_list = std::make_shared<ASTExpressionList>();
|
|
||||||
left_keys_list->children = analyzedJoin().key_asts_left;
|
|
||||||
|
|
||||||
initChain(chain, sourceColumns());
|
initChain(chain, sourceColumns());
|
||||||
ExpressionActionsChain::Step & step = chain.steps.back();
|
ExpressionActionsChain::Step & step = chain.steps.back();
|
||||||
|
|
||||||
auto & join_params = ast_join->table_join->as<ASTTableJoin &>();
|
getRootActions(analyzedJoin().leftKeysList(), only_types, step.actions);
|
||||||
|
addJoinAction(step.actions, subquery_for_set.join);
|
||||||
getRootActions(left_keys_list, only_types, step.actions);
|
|
||||||
addJoinAction(join_params, step.actions, subquery_for_set.join);
|
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
@ -524,11 +501,9 @@ void SelectQueryExpressionAnalyzer::makeHashJoin(const ASTTablesInSelectQueryEle
|
|||||||
Names action_columns = joined_block_actions->getRequiredColumns();
|
Names action_columns = joined_block_actions->getRequiredColumns();
|
||||||
NameSet required_columns(action_columns.begin(), action_columns.end());
|
NameSet required_columns(action_columns.begin(), action_columns.end());
|
||||||
|
|
||||||
auto & analyzed_join = analyzedJoin();
|
analyzedJoin().appendRequiredColumns(joined_block_actions->getSampleBlock(), required_columns);
|
||||||
appendRequiredColumns(
|
|
||||||
required_columns, joined_block_actions->getSampleBlock(), analyzed_join.key_names_right, columnsAddedByJoin());
|
|
||||||
|
|
||||||
auto original_map = analyzed_join.getOriginalColumnsMap(required_columns);
|
auto original_map = analyzedJoin().getOriginalColumnsMap(required_columns);
|
||||||
Names original_columns;
|
Names original_columns;
|
||||||
for (auto & pr : original_map)
|
for (auto & pr : original_map)
|
||||||
original_columns.push_back(pr.second);
|
original_columns.push_back(pr.second);
|
||||||
@ -542,29 +517,16 @@ void SelectQueryExpressionAnalyzer::makeHashJoin(const ASTTablesInSelectQueryEle
|
|||||||
joined_block_actions->execute(sample_block);
|
joined_block_actions->execute(sample_block);
|
||||||
|
|
||||||
/// TODO You do not need to set this up when JOIN is only needed on remote servers.
|
/// TODO You do not need to set this up when JOIN is only needed on remote servers.
|
||||||
auto & join_params = join_element.table_join->as<ASTTableJoin &>();
|
subquery_for_set.join = analyzedJoin().makeHashJoin(sample_block, settings.size_limits_for_join);
|
||||||
subquery_for_set.join = std::make_shared<Join>(analyzedJoin().key_names_right, settings.join_use_nulls,
|
|
||||||
settings.size_limits_for_join, join_params.kind, join_params.strictness);
|
|
||||||
subquery_for_set.join->setSampleBlock(sample_block);
|
|
||||||
subquery_for_set.joined_block_actions = joined_block_actions;
|
subquery_for_set.joined_block_actions = joined_block_actions;
|
||||||
}
|
}
|
||||||
|
|
||||||
ExpressionActionsPtr SelectQueryExpressionAnalyzer::createJoinedBlockActions() const
|
ExpressionActionsPtr SelectQueryExpressionAnalyzer::createJoinedBlockActions() const
|
||||||
{
|
{
|
||||||
/// Create custom expression list with join keys from right table.
|
ASTPtr expression_list = analyzedJoin().rightKeysList();
|
||||||
ASTPtr expression_list = std::make_shared<ASTExpressionList>();
|
Names required_columns = analyzedJoin().requiredJoinedNames();
|
||||||
ASTs & children = expression_list->children;
|
|
||||||
|
|
||||||
if (analyzedJoin().hasOn())
|
auto syntax_result = SyntaxAnalyzer(context).analyze(expression_list, analyzedJoin().columnsFromJoinedTable(), required_columns);
|
||||||
for (const auto & join_right_key : analyzedJoin().key_asts_right)
|
|
||||||
children.emplace_back(join_right_key);
|
|
||||||
|
|
||||||
NameSet required_columns_set(analyzedJoin().key_names_right.begin(), analyzedJoin().key_names_right.end());
|
|
||||||
for (const auto & joined_column : columnsAddedByJoin())
|
|
||||||
required_columns_set.insert(joined_column.name);
|
|
||||||
Names required_columns(required_columns_set.begin(), required_columns_set.end());
|
|
||||||
|
|
||||||
auto syntax_result = SyntaxAnalyzer(context).analyze(expression_list, analyzedJoin().columns_from_joined_table, required_columns);
|
|
||||||
return ExpressionAnalyzer(expression_list, syntax_result, context).getActions(true, false);
|
return ExpressionAnalyzer(expression_list, syntax_result, context).getActions(true, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -121,9 +121,8 @@ protected:
|
|||||||
SyntaxAnalyzerResultPtr syntax;
|
SyntaxAnalyzerResultPtr syntax;
|
||||||
|
|
||||||
const StoragePtr & storage() const { return syntax->storage; } /// The main table in FROM clause, if exists.
|
const StoragePtr & storage() const { return syntax->storage; } /// The main table in FROM clause, if exists.
|
||||||
const AnalyzedJoin & analyzedJoin() const { return syntax->analyzed_join; }
|
const AnalyzedJoin & analyzedJoin() const { return *syntax->analyzed_join; }
|
||||||
const NamesAndTypesList & sourceColumns() const { return syntax->required_source_columns; }
|
const NamesAndTypesList & sourceColumns() const { return syntax->required_source_columns; }
|
||||||
const NamesAndTypesList & columnsAddedByJoin() const { return syntax->columns_added_by_join; }
|
|
||||||
const std::vector<const ASTFunction *> & aggregates() const { return syntax->aggregates; }
|
const std::vector<const ASTFunction *> & aggregates() const { return syntax->aggregates; }
|
||||||
|
|
||||||
/// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables.
|
/// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables.
|
||||||
@ -131,7 +130,7 @@ protected:
|
|||||||
|
|
||||||
void addMultipleArrayJoinAction(ExpressionActionsPtr & actions, bool is_left) const;
|
void addMultipleArrayJoinAction(ExpressionActionsPtr & actions, bool is_left) const;
|
||||||
|
|
||||||
void addJoinAction(const ASTTableJoin & join_params, ExpressionActionsPtr & actions, JoinPtr join = {}) const;
|
void addJoinAction(ExpressionActionsPtr & actions, JoinPtr join = {}) const;
|
||||||
|
|
||||||
void getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts = false);
|
void getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts = false);
|
||||||
|
|
||||||
|
@ -10,6 +10,7 @@
|
|||||||
#include <DataTypes/DataTypeNullable.h>
|
#include <DataTypes/DataTypeNullable.h>
|
||||||
|
|
||||||
#include <Interpreters/Join.h>
|
#include <Interpreters/Join.h>
|
||||||
|
#include <Interpreters/AnalyzedJoin.h>
|
||||||
#include <Interpreters/joinDispatch.h>
|
#include <Interpreters/joinDispatch.h>
|
||||||
#include <Interpreters/NullableUtils.h>
|
#include <Interpreters/NullableUtils.h>
|
||||||
|
|
||||||
@ -1048,8 +1049,11 @@ void Join::joinGet(Block & block, const String & column_name) const
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void Join::joinBlock(Block & block, const Names & key_names_left, const NamesAndTypesList & columns_added_by_join) const
|
void Join::joinBlock(Block & block, const AnalyzedJoin & join_params) const
|
||||||
{
|
{
|
||||||
|
const Names & key_names_left = join_params.keyNamesLeft();
|
||||||
|
const NamesAndTypesList & columns_added_by_join = join_params.columnsAddedByJoin();
|
||||||
|
|
||||||
std::shared_lock lock(rwlock);
|
std::shared_lock lock(rwlock);
|
||||||
|
|
||||||
checkTypesOfKeys(block, key_names_left, sample_block_with_keys);
|
checkTypesOfKeys(block, key_names_left, sample_block_with_keys);
|
||||||
@ -1457,10 +1461,11 @@ private:
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
BlockInputStreamPtr Join::createStreamWithNonJoinedRows(const Block & left_sample_block, const Names & key_names_left,
|
BlockInputStreamPtr Join::createStreamWithNonJoinedRows(const Block & left_sample_block, const AnalyzedJoin & join_params,
|
||||||
const NamesAndTypesList & columns_added_by_join, UInt64 max_block_size) const
|
UInt64 max_block_size) const
|
||||||
{
|
{
|
||||||
return std::make_shared<NonJoinedBlockInputStream>(*this, left_sample_block, key_names_left, columns_added_by_join, max_block_size);
|
return std::make_shared<NonJoinedBlockInputStream>(*this, left_sample_block,
|
||||||
|
join_params.keyNamesLeft(), join_params.columnsAddedByJoin(), max_block_size);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -26,6 +26,8 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
struct AnalyzedJoin;
|
||||||
|
|
||||||
namespace JoinStuff
|
namespace JoinStuff
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -141,7 +143,7 @@ public:
|
|||||||
/** Join data from the map (that was previously built by calls to insertFromBlock) to the block with data from "left" table.
|
/** Join data from the map (that was previously built by calls to insertFromBlock) to the block with data from "left" table.
|
||||||
* Could be called from different threads in parallel.
|
* Could be called from different threads in parallel.
|
||||||
*/
|
*/
|
||||||
void joinBlock(Block & block, const Names & key_names_left, const NamesAndTypesList & columns_added_by_join) const;
|
void joinBlock(Block & block, const AnalyzedJoin & join_params) const;
|
||||||
|
|
||||||
/// Infer the return type for joinGet function
|
/// Infer the return type for joinGet function
|
||||||
DataTypePtr joinGetReturnType(const String & column_name) const;
|
DataTypePtr joinGetReturnType(const String & column_name) const;
|
||||||
@ -161,8 +163,8 @@ public:
|
|||||||
* Use only after all calls to joinBlock was done.
|
* Use only after all calls to joinBlock was done.
|
||||||
* left_sample_block is passed without account of 'use_nulls' setting (columns will be converted to Nullable inside).
|
* left_sample_block is passed without account of 'use_nulls' setting (columns will be converted to Nullable inside).
|
||||||
*/
|
*/
|
||||||
BlockInputStreamPtr createStreamWithNonJoinedRows(const Block & left_sample_block, const Names & key_names_left,
|
BlockInputStreamPtr createStreamWithNonJoinedRows(const Block & left_sample_block, const AnalyzedJoin & join_params,
|
||||||
const NamesAndTypesList & columns_added_by_join, UInt64 max_block_size) const;
|
UInt64 max_block_size) const;
|
||||||
|
|
||||||
/// Number of keys in all built JOIN maps.
|
/// Number of keys in all built JOIN maps.
|
||||||
size_t getTotalRowCount() const;
|
size_t getTotalRowCount() const;
|
||||||
|
@ -489,14 +489,13 @@ void getArrayJoinedColumns(ASTPtr & query, SyntaxAnalyzerResult & result, const
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_default_strictness, ASTTableJoin::Kind & join_kind)
|
void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_default_strictness, ASTTableJoin & out_table_join)
|
||||||
{
|
{
|
||||||
const ASTTablesInSelectQueryElement * node = select_query.join();
|
const ASTTablesInSelectQueryElement * node = select_query.join();
|
||||||
if (!node)
|
if (!node)
|
||||||
return;
|
return;
|
||||||
|
|
||||||
auto & table_join = const_cast<ASTTablesInSelectQueryElement *>(node)->table_join->as<ASTTableJoin &>();
|
auto & table_join = const_cast<ASTTablesInSelectQueryElement *>(node)->table_join->as<ASTTableJoin &>();
|
||||||
join_kind = table_join.kind;
|
|
||||||
|
|
||||||
if (table_join.strictness == ASTTableJoin::Strictness::Unspecified &&
|
if (table_join.strictness == ASTTableJoin::Strictness::Unspecified &&
|
||||||
table_join.kind != ASTTableJoin::Kind::Cross)
|
table_join.kind != ASTTableJoin::Kind::Cross)
|
||||||
@ -509,6 +508,8 @@ void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_defaul
|
|||||||
throw Exception("Expected ANY or ALL in JOIN section, because setting (join_default_strictness) is empty",
|
throw Exception("Expected ANY or ALL in JOIN section, because setting (join_default_strictness) is empty",
|
||||||
DB::ErrorCodes::EXPECTED_ALL_OR_ANY);
|
DB::ErrorCodes::EXPECTED_ALL_OR_ANY);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
out_table_join = table_join;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Find the columns that are obtained by JOIN.
|
/// Find the columns that are obtained by JOIN.
|
||||||
@ -609,8 +610,7 @@ std::vector<const ASTFunction *> getAggregates(const ASTPtr & query)
|
|||||||
/// Calculate which columns are required to execute the expression.
|
/// Calculate which columns are required to execute the expression.
|
||||||
/// Then, delete all other columns from the list of available columns.
|
/// Then, delete all other columns from the list of available columns.
|
||||||
/// After execution, columns will only contain the list of columns needed to read from the table.
|
/// After execution, columns will only contain the list of columns needed to read from the table.
|
||||||
void SyntaxAnalyzerResult::collectUsedColumns(const ASTPtr & query, const NamesAndTypesList & additional_source_columns,
|
void SyntaxAnalyzerResult::collectUsedColumns(const ASTPtr & query, const NamesAndTypesList & additional_source_columns)
|
||||||
bool make_joined_columns_nullable)
|
|
||||||
{
|
{
|
||||||
/// We caclulate required_source_columns with source_columns modifications and swap them on exit
|
/// We caclulate required_source_columns with source_columns modifications and swap them on exit
|
||||||
required_source_columns = source_columns;
|
required_source_columns = source_columns;
|
||||||
@ -637,8 +637,7 @@ void SyntaxAnalyzerResult::collectUsedColumns(const ASTPtr & query, const NamesA
|
|||||||
avaliable_columns.insert(name.name);
|
avaliable_columns.insert(name.name);
|
||||||
|
|
||||||
/// Add columns obtained by JOIN (if needed).
|
/// Add columns obtained by JOIN (if needed).
|
||||||
columns_added_by_join.clear();
|
for (const auto & joined_column : analyzed_join->columnsFromJoinedTable())
|
||||||
for (const auto & joined_column : analyzed_join.columns_from_joined_table)
|
|
||||||
{
|
{
|
||||||
auto & name = joined_column.name;
|
auto & name = joined_column.name;
|
||||||
if (avaliable_columns.count(name))
|
if (avaliable_columns.count(name))
|
||||||
@ -647,16 +646,9 @@ void SyntaxAnalyzerResult::collectUsedColumns(const ASTPtr & query, const NamesA
|
|||||||
if (required.count(name))
|
if (required.count(name))
|
||||||
{
|
{
|
||||||
/// Optimisation: do not add columns needed only in JOIN ON section.
|
/// Optimisation: do not add columns needed only in JOIN ON section.
|
||||||
if (columns_context.nameInclusion(name) > analyzed_join.rightKeyInclusion(name))
|
if (columns_context.nameInclusion(name) > analyzed_join->rightKeyInclusion(name))
|
||||||
{
|
analyzed_join->addJoinedColumn(joined_column);
|
||||||
if (make_joined_columns_nullable)
|
|
||||||
{
|
|
||||||
auto type = joined_column.type->canBeInsideNullable() ? makeNullable(joined_column.type) : joined_column.type;
|
|
||||||
columns_added_by_join.emplace_back(NameAndTypePair(joined_column.name, std::move(type)));
|
|
||||||
}
|
|
||||||
else
|
|
||||||
columns_added_by_join.push_back(joined_column);
|
|
||||||
}
|
|
||||||
required.erase(name);
|
required.erase(name);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -766,7 +758,7 @@ void SyntaxAnalyzerResult::collectUsedColumns(const ASTPtr & query, const NamesA
|
|||||||
if (columns_context.has_table_join)
|
if (columns_context.has_table_join)
|
||||||
{
|
{
|
||||||
ss << ", joined columns:";
|
ss << ", joined columns:";
|
||||||
for (const auto & column : analyzed_join.columns_from_joined_table)
|
for (const auto & column : analyzed_join->columnsFromJoinedTable())
|
||||||
ss << " '" << column.name << "'";
|
ss << " '" << column.name << "'";
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -798,15 +790,17 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
|
|||||||
storage = context.tryGetTable(db_and_table->database, db_and_table->table);
|
storage = context.tryGetTable(db_and_table->database, db_and_table->table);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
const auto & settings = context.getSettingsRef();
|
||||||
|
|
||||||
SyntaxAnalyzerResult result;
|
SyntaxAnalyzerResult result;
|
||||||
result.storage = storage;
|
result.storage = storage;
|
||||||
result.source_columns = source_columns_;
|
result.source_columns = source_columns_;
|
||||||
|
result.analyzed_join = std::make_shared<AnalyzedJoin>(); /// TODO: move to select_query logic
|
||||||
|
result.analyzed_join->join_use_nulls = settings.join_use_nulls;
|
||||||
|
|
||||||
collectSourceColumns(select_query, result.storage, result.source_columns);
|
collectSourceColumns(select_query, result.storage, result.source_columns);
|
||||||
NameSet source_columns_set = removeDuplicateColumns(result.source_columns);
|
NameSet source_columns_set = removeDuplicateColumns(result.source_columns);
|
||||||
|
|
||||||
const auto & settings = context.getSettingsRef();
|
|
||||||
|
|
||||||
Names source_columns_list;
|
Names source_columns_list;
|
||||||
source_columns_list.reserve(result.source_columns.size());
|
source_columns_list.reserve(result.source_columns.size());
|
||||||
for (const auto & type_name : result.source_columns)
|
for (const auto & type_name : result.source_columns)
|
||||||
@ -831,13 +825,13 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
|
|||||||
const auto & joined_expression = node->table_expression->as<ASTTableExpression &>();
|
const auto & joined_expression = node->table_expression->as<ASTTableExpression &>();
|
||||||
DatabaseAndTableWithAlias table(joined_expression, context.getCurrentDatabase());
|
DatabaseAndTableWithAlias table(joined_expression, context.getCurrentDatabase());
|
||||||
|
|
||||||
result.analyzed_join.columns_from_joined_table = getNamesAndTypeListFromTableExpression(joined_expression, context);
|
result.analyzed_join->columns_from_joined_table = getNamesAndTypeListFromTableExpression(joined_expression, context);
|
||||||
result.analyzed_join.deduplicateAndQualifyColumnNames(source_columns_set, table.getQualifiedNamePrefix());
|
result.analyzed_join->deduplicateAndQualifyColumnNames(source_columns_set, table.getQualifiedNamePrefix());
|
||||||
}
|
}
|
||||||
|
|
||||||
translateQualifiedNames(query, *select_query, context,
|
translateQualifiedNames(query, *select_query, context,
|
||||||
(storage ? storage->getColumns().getOrdinary().getNames() : source_columns_list), source_columns_set,
|
(storage ? storage->getColumns().getOrdinary().getNames() : source_columns_list), source_columns_set,
|
||||||
result.analyzed_join.getQualifiedColumnsSet());
|
result.analyzed_join->getQualifiedColumnsSet());
|
||||||
|
|
||||||
/// Rewrite IN and/or JOIN for distributed tables according to distributed_product_mode setting.
|
/// Rewrite IN and/or JOIN for distributed tables according to distributed_product_mode setting.
|
||||||
InJoinSubqueriesPreprocessor(context).visit(query);
|
InJoinSubqueriesPreprocessor(context).visit(query);
|
||||||
@ -872,7 +866,6 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
|
|||||||
/// Optimize if with constant condition after constants was substituted instead of scalar subqueries.
|
/// Optimize if with constant condition after constants was substituted instead of scalar subqueries.
|
||||||
OptimizeIfWithConstantConditionVisitor(result.aliases).visit(query);
|
OptimizeIfWithConstantConditionVisitor(result.aliases).visit(query);
|
||||||
|
|
||||||
bool make_joined_columns_nullable = false;
|
|
||||||
if (select_query)
|
if (select_query)
|
||||||
{
|
{
|
||||||
/// GROUP BY injective function elimination.
|
/// GROUP BY injective function elimination.
|
||||||
@ -893,15 +886,12 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
|
|||||||
/// Push the predicate expression down to the subqueries.
|
/// Push the predicate expression down to the subqueries.
|
||||||
result.rewrite_subqueries = PredicateExpressionsOptimizer(select_query, settings, context).optimize();
|
result.rewrite_subqueries = PredicateExpressionsOptimizer(select_query, settings, context).optimize();
|
||||||
|
|
||||||
ASTTableJoin::Kind join_kind = ASTTableJoin::Kind::Comma;
|
setJoinStrictness(*select_query, settings.join_default_strictness, result.analyzed_join->table_join);
|
||||||
setJoinStrictness(*select_query, settings.join_default_strictness, join_kind);
|
collectJoinedColumns(*result.analyzed_join, *select_query, source_columns_set, result.aliases);
|
||||||
make_joined_columns_nullable = settings.join_use_nulls && isLeftOrFull(join_kind);
|
|
||||||
|
|
||||||
collectJoinedColumns(result.analyzed_join, *select_query, source_columns_set, result.aliases);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
result.aggregates = getAggregates(query);
|
result.aggregates = getAggregates(query);
|
||||||
result.collectUsedColumns(query, additional_source_columns, make_joined_columns_nullable);
|
result.collectUsedColumns(query, additional_source_columns);
|
||||||
return std::make_shared<const SyntaxAnalyzerResult>(result);
|
return std::make_shared<const SyntaxAnalyzerResult>(result);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -15,13 +15,11 @@ class ASTFunction;
|
|||||||
struct SyntaxAnalyzerResult
|
struct SyntaxAnalyzerResult
|
||||||
{
|
{
|
||||||
StoragePtr storage;
|
StoragePtr storage;
|
||||||
AnalyzedJoin analyzed_join;
|
std::shared_ptr<AnalyzedJoin> analyzed_join;
|
||||||
|
|
||||||
NamesAndTypesList source_columns;
|
NamesAndTypesList source_columns;
|
||||||
/// Set of columns that are enough to read from the table to evaluate the expression. It does not include joined columns.
|
/// Set of columns that are enough to read from the table to evaluate the expression. It does not include joined columns.
|
||||||
NamesAndTypesList required_source_columns;
|
NamesAndTypesList required_source_columns;
|
||||||
/// Columns will be added to block by JOIN. It's a subset of analyzed_join.columns_from_joined_table with corrected Nullability
|
|
||||||
NamesAndTypesList columns_added_by_join;
|
|
||||||
|
|
||||||
Aliases aliases;
|
Aliases aliases;
|
||||||
std::vector<const ASTFunction *> aggregates;
|
std::vector<const ASTFunction *> aggregates;
|
||||||
@ -42,7 +40,7 @@ struct SyntaxAnalyzerResult
|
|||||||
/// Predicate optimizer overrides the sub queries
|
/// Predicate optimizer overrides the sub queries
|
||||||
bool rewrite_subqueries = false;
|
bool rewrite_subqueries = false;
|
||||||
|
|
||||||
void collectUsedColumns(const ASTPtr & query, const NamesAndTypesList & additional_source_columns, bool make_joined_columns_nullable);
|
void collectUsedColumns(const ASTPtr & query, const NamesAndTypesList & additional_source_columns);
|
||||||
Names requiredSourceColumns() const { return required_source_columns.getNames(); }
|
Names requiredSourceColumns() const { return required_source_columns.getNames(); }
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -13,6 +13,8 @@
|
|||||||
#include <Columns/ColumnString.h>
|
#include <Columns/ColumnString.h>
|
||||||
#include <Columns/ColumnNullable.h>
|
#include <Columns/ColumnNullable.h>
|
||||||
#include <Interpreters/castColumn.h>
|
#include <Interpreters/castColumn.h>
|
||||||
|
#include <algorithm>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -27,34 +29,28 @@ namespace DB
|
|||||||
extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN;
|
extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN;
|
||||||
extern const int THERE_IS_NO_COLUMN;
|
extern const int THERE_IS_NO_COLUMN;
|
||||||
}
|
}
|
||||||
const std::unordered_map<arrow::Type::type, std::shared_ptr<IDataType>> arrow_type_to_internal_type = {
|
|
||||||
//{arrow::Type::DECIMAL, std::make_shared<DataTypeDecimal>()},
|
|
||||||
{arrow::Type::UINT8, std::make_shared<DataTypeUInt8>()},
|
|
||||||
{arrow::Type::INT8, std::make_shared<DataTypeInt8>()},
|
|
||||||
{arrow::Type::UINT16, std::make_shared<DataTypeUInt16>()},
|
|
||||||
{arrow::Type::INT16, std::make_shared<DataTypeInt16>()},
|
|
||||||
{arrow::Type::UINT32, std::make_shared<DataTypeUInt32>()},
|
|
||||||
{arrow::Type::INT32, std::make_shared<DataTypeInt32>()},
|
|
||||||
{arrow::Type::UINT64, std::make_shared<DataTypeUInt64>()},
|
|
||||||
{arrow::Type::INT64, std::make_shared<DataTypeInt64>()},
|
|
||||||
{arrow::Type::HALF_FLOAT, std::make_shared<DataTypeFloat32>()},
|
|
||||||
{arrow::Type::FLOAT, std::make_shared<DataTypeFloat32>()},
|
|
||||||
{arrow::Type::DOUBLE, std::make_shared<DataTypeFloat64>()},
|
|
||||||
|
|
||||||
{arrow::Type::BOOL, std::make_shared<DataTypeUInt8>()},
|
static const std::initializer_list<std::pair<arrow::Type::type, const char *>> arrow_type_to_internal_type =
|
||||||
//{arrow::Type::DATE32, std::make_shared<DataTypeDate>()},
|
{
|
||||||
{arrow::Type::DATE32, std::make_shared<DataTypeDate>()},
|
{arrow::Type::UINT8, "UInt8"},
|
||||||
//{arrow::Type::DATE32, std::make_shared<DataTypeDateTime>()},
|
{arrow::Type::INT8, "Int8"},
|
||||||
{arrow::Type::DATE64, std::make_shared<DataTypeDateTime>()},
|
{arrow::Type::UINT16, "UInt16"},
|
||||||
{arrow::Type::TIMESTAMP, std::make_shared<DataTypeDateTime>()},
|
{arrow::Type::INT16, "Int16"},
|
||||||
//{arrow::Type::TIME32, std::make_shared<DataTypeDateTime>()},
|
{arrow::Type::UINT32, "UInt32"},
|
||||||
|
{arrow::Type::INT32, "Int32"},
|
||||||
|
{arrow::Type::UINT64, "UInt64"},
|
||||||
|
{arrow::Type::INT64, "Int64"},
|
||||||
|
{arrow::Type::HALF_FLOAT, "Float32"},
|
||||||
|
{arrow::Type::FLOAT, "Float32"},
|
||||||
|
{arrow::Type::DOUBLE, "Float64"},
|
||||||
|
|
||||||
|
{arrow::Type::BOOL, "UInt8"},
|
||||||
|
{arrow::Type::DATE32, "Date"},
|
||||||
|
{arrow::Type::DATE64, "DateTime"},
|
||||||
|
{arrow::Type::TIMESTAMP, "DateTime"},
|
||||||
|
|
||||||
{arrow::Type::STRING, std::make_shared<DataTypeString>()},
|
{arrow::Type::STRING, "String"},
|
||||||
{arrow::Type::BINARY, std::make_shared<DataTypeString>()},
|
{arrow::Type::BINARY, "String"},
|
||||||
//{arrow::Type::FIXED_SIZE_BINARY, std::make_shared<DataTypeString>()},
|
|
||||||
//{arrow::Type::UUID, std::make_shared<DataTypeString>()},
|
|
||||||
|
|
||||||
|
|
||||||
// TODO: add other types that are convertable to internal ones:
|
// TODO: add other types that are convertable to internal ones:
|
||||||
// 0. ENUM?
|
// 0. ENUM?
|
||||||
@ -253,7 +249,7 @@ namespace DB
|
|||||||
void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk &res, std::shared_ptr<arrow::Table> &table,
|
void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk &res, std::shared_ptr<arrow::Table> &table,
|
||||||
arrow::Status &read_status, const Block &header,
|
arrow::Status &read_status, const Block &header,
|
||||||
int &row_group_current, const Context &context, std::string format_name)
|
int &row_group_current, const Context &context, std::string format_name)
|
||||||
{
|
{
|
||||||
Columns columns_list;
|
Columns columns_list;
|
||||||
UInt64 num_rows = 0;
|
UInt64 num_rows = 0;
|
||||||
|
|
||||||
@ -308,15 +304,16 @@ namespace DB
|
|||||||
const auto decimal_type = static_cast<arrow::DecimalType *>(arrow_column->type().get());
|
const auto decimal_type = static_cast<arrow::DecimalType *>(arrow_column->type().get());
|
||||||
internal_nested_type = std::make_shared<DataTypeDecimal<Decimal128>>(decimal_type->precision(),
|
internal_nested_type = std::make_shared<DataTypeDecimal<Decimal128>>(decimal_type->precision(),
|
||||||
decimal_type->scale());
|
decimal_type->scale());
|
||||||
} else if (arrow_type_to_internal_type.find(arrow_type) != arrow_type_to_internal_type.end())
|
}
|
||||||
|
else if (auto internal_type_it = std::find_if(arrow_type_to_internal_type.begin(), arrow_type_to_internal_type.end(),
|
||||||
|
[=](auto && elem) { return elem.first == arrow_type; });
|
||||||
|
internal_type_it != arrow_type_to_internal_type.end())
|
||||||
{
|
{
|
||||||
internal_nested_type = arrow_type_to_internal_type.at(arrow_type);
|
internal_nested_type = DataTypeFactory::instance().get(internal_type_it->second);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
throw Exception
|
throw Exception{"The type \"" + arrow_column->type()->name() + "\" of an input column \"" + arrow_column->name()
|
||||||
{
|
|
||||||
"The type \"" + arrow_column->type()->name() + "\" of an input column \"" + arrow_column->name()
|
|
||||||
+ "\" is not supported for conversion from a " + format_name + " data format",
|
+ "\" is not supported for conversion from a " + format_name + " data format",
|
||||||
ErrorCodes::CANNOT_CONVERT_TYPE};
|
ErrorCodes::CANNOT_CONVERT_TYPE};
|
||||||
}
|
}
|
||||||
|
@ -21,34 +21,26 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream(
|
|||||||
header(header_), sync(sync_), skip_offsets(skip_offsets_),
|
header(header_), sync(sync_), skip_offsets(skip_offsets_),
|
||||||
already_written_offset_columns(already_written_offset_columns_)
|
already_written_offset_columns(already_written_offset_columns_)
|
||||||
{
|
{
|
||||||
|
serialization_states.reserve(header.columns());
|
||||||
|
WrittenOffsetColumns tmp_offset_columns;
|
||||||
|
IDataType::SerializeBinaryBulkSettings settings;
|
||||||
|
|
||||||
|
for (const auto & column_name : header.getNames())
|
||||||
|
{
|
||||||
|
const auto & col = header.getByName(column_name);
|
||||||
|
|
||||||
|
const auto columns = storage.getColumns();
|
||||||
|
addStreams(part_path, col.name, *col.type, columns.getCodecOrDefault(col.name, codec), 0, skip_offsets);
|
||||||
|
serialization_states.emplace_back(nullptr);
|
||||||
|
settings.getter = createStreamGetter(col.name, tmp_offset_columns, false);
|
||||||
|
col.type->serializeBinaryBulkStatePrefix(settings, serialization_states.back());
|
||||||
|
}
|
||||||
|
|
||||||
|
initSkipIndices();
|
||||||
}
|
}
|
||||||
|
|
||||||
void MergedColumnOnlyOutputStream::write(const Block & block)
|
void MergedColumnOnlyOutputStream::write(const Block & block)
|
||||||
{
|
{
|
||||||
if (!initialized)
|
|
||||||
{
|
|
||||||
column_streams.clear();
|
|
||||||
serialization_states.clear();
|
|
||||||
serialization_states.reserve(header.columns());
|
|
||||||
WrittenOffsetColumns tmp_offset_columns;
|
|
||||||
IDataType::SerializeBinaryBulkSettings settings;
|
|
||||||
|
|
||||||
for (const auto & column_name : header.getNames())
|
|
||||||
{
|
|
||||||
const auto & col = block.getByName(column_name);
|
|
||||||
|
|
||||||
const auto columns = storage.getColumns();
|
|
||||||
addStreams(part_path, col.name, *col.type, columns.getCodecOrDefault(col.name, codec), 0, skip_offsets);
|
|
||||||
serialization_states.emplace_back(nullptr);
|
|
||||||
settings.getter = createStreamGetter(col.name, tmp_offset_columns, false);
|
|
||||||
col.type->serializeBinaryBulkStatePrefix(settings, serialization_states.back());
|
|
||||||
}
|
|
||||||
|
|
||||||
initSkipIndices();
|
|
||||||
|
|
||||||
initialized = true;
|
|
||||||
}
|
|
||||||
|
|
||||||
std::set<String> skip_indexes_column_names_set;
|
std::set<String> skip_indexes_column_names_set;
|
||||||
for (const auto & index : skip_indices)
|
for (const auto & index : skip_indices)
|
||||||
std::copy(index->columns.cbegin(), index->columns.cend(),
|
std::copy(index->columns.cbegin(), index->columns.cend(),
|
||||||
@ -68,7 +60,6 @@ void MergedColumnOnlyOutputStream::write(const Block & block)
|
|||||||
if (!rows)
|
if (!rows)
|
||||||
return;
|
return;
|
||||||
|
|
||||||
|
|
||||||
size_t new_index_offset = 0;
|
size_t new_index_offset = 0;
|
||||||
size_t new_current_mark = 0;
|
size_t new_current_mark = 0;
|
||||||
WrittenOffsetColumns offset_columns = already_written_offset_columns;
|
WrittenOffsetColumns offset_columns = already_written_offset_columns;
|
||||||
@ -106,7 +97,8 @@ MergeTreeData::DataPart::Checksums MergedColumnOnlyOutputStream::writeSuffixAndG
|
|||||||
serialize_settings.getter = createStreamGetter(column.name, already_written_offset_columns, skip_offsets);
|
serialize_settings.getter = createStreamGetter(column.name, already_written_offset_columns, skip_offsets);
|
||||||
column.type->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[i]);
|
column.type->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[i]);
|
||||||
|
|
||||||
if (with_final_mark)
|
/// We wrote at least one row
|
||||||
|
if (with_final_mark && (index_offset != 0 || current_mark != 0))
|
||||||
writeFinalMark(column.name, column.type, offset_columns, skip_offsets, serialize_settings.path);
|
writeFinalMark(column.name, column.type, offset_columns, skip_offsets, serialize_settings.path);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -125,7 +117,6 @@ MergeTreeData::DataPart::Checksums MergedColumnOnlyOutputStream::writeSuffixAndG
|
|||||||
|
|
||||||
column_streams.clear();
|
column_streams.clear();
|
||||||
serialization_states.clear();
|
serialization_states.clear();
|
||||||
initialized = false;
|
|
||||||
|
|
||||||
return checksums;
|
return checksums;
|
||||||
}
|
}
|
||||||
|
@ -28,7 +28,6 @@ public:
|
|||||||
private:
|
private:
|
||||||
Block header;
|
Block header;
|
||||||
|
|
||||||
bool initialized = false;
|
|
||||||
bool sync;
|
bool sync;
|
||||||
bool skip_offsets;
|
bool skip_offsets;
|
||||||
|
|
||||||
|
@ -34,7 +34,8 @@ set the following environment variables:
|
|||||||
|
|
||||||
### Running with runner script
|
### Running with runner script
|
||||||
|
|
||||||
The only requirement is fresh docker configured docker.
|
The only requirement is fresh configured docker and
|
||||||
|
docker pull yandex/clickhouse-integration-tests-runner
|
||||||
|
|
||||||
Notes:
|
Notes:
|
||||||
* If you want to run integration tests without `sudo` you have to add your user to docker group `sudo usermod -aG docker $USER`. [More information](https://docs.docker.com/install/linux/linux-postinstall/) about docker configuration.
|
* If you want to run integration tests without `sudo` you have to add your user to docker group `sudo usermod -aG docker $USER`. [More information](https://docs.docker.com/install/linux/linux-postinstall/) about docker configuration.
|
||||||
|
@ -723,7 +723,8 @@ class ClickHouseInstance:
|
|||||||
os.mkdir(config_d_dir)
|
os.mkdir(config_d_dir)
|
||||||
os.mkdir(users_d_dir)
|
os.mkdir(users_d_dir)
|
||||||
|
|
||||||
shutil.copy(p.join(HELPERS_DIR, 'common_instance_config.xml'), config_d_dir)
|
# The file is named with 0_ prefix to be processed before other configuration overloads.
|
||||||
|
shutil.copy(p.join(HELPERS_DIR, '0_common_instance_config.xml'), config_d_dir)
|
||||||
|
|
||||||
# Generate and write macros file
|
# Generate and write macros file
|
||||||
macros = self.macros.copy()
|
macros = self.macros.copy()
|
||||||
|
@ -1,4 +0,0 @@
|
|||||||
*
|
|
||||||
!.gitignore
|
|
||||||
!source.tsv
|
|
||||||
!dictionary_preset*
|
|
@ -1,411 +0,0 @@
|
|||||||
import pytest
|
|
||||||
import os
|
|
||||||
import time
|
|
||||||
|
|
||||||
from helpers.cluster import ClickHouseCluster
|
|
||||||
from helpers.test_tools import TSV, assert_eq_with_retry
|
|
||||||
from generate_dictionaries import generate_structure, generate_dictionaries, DictionaryTestTable
|
|
||||||
|
|
||||||
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
|
||||||
|
|
||||||
cluster = None
|
|
||||||
instance = None
|
|
||||||
test_table = None
|
|
||||||
|
|
||||||
|
|
||||||
def get_status(dictionary_name):
|
|
||||||
return instance.query("SELECT status FROM system.dictionaries WHERE name='" + dictionary_name + "'").rstrip("\n")
|
|
||||||
|
|
||||||
|
|
||||||
def get_last_exception(dictionary_name):
|
|
||||||
return instance.query("SELECT last_exception FROM system.dictionaries WHERE name='" + dictionary_name + "'").rstrip("\n").replace("\\'", "'")
|
|
||||||
|
|
||||||
|
|
||||||
def get_loading_start_time(dictionary_name):
|
|
||||||
s = instance.query("SELECT loading_start_time FROM system.dictionaries WHERE name='" + dictionary_name + "'").rstrip("\n")
|
|
||||||
if s == "0000-00-00 00:00:00":
|
|
||||||
return None
|
|
||||||
return time.strptime(s, "%Y-%m-%d %H:%M:%S")
|
|
||||||
|
|
||||||
|
|
||||||
def get_loading_duration(dictionary_name):
|
|
||||||
return float(instance.query("SELECT loading_duration FROM system.dictionaries WHERE name='" + dictionary_name + "'"))
|
|
||||||
|
|
||||||
|
|
||||||
def replace_in_file_in_container(file_name, what, replace_with):
|
|
||||||
instance.exec_in_container('sed -i "s/' + what + '/' + replace_with + '/g" ' + file_name)
|
|
||||||
|
|
||||||
|
|
||||||
def setup_module(module):
|
|
||||||
global cluster
|
|
||||||
global instance
|
|
||||||
global test_table
|
|
||||||
|
|
||||||
structure = generate_structure()
|
|
||||||
dictionary_files = generate_dictionaries(os.path.join(SCRIPT_DIR, 'configs/dictionaries'), structure)
|
|
||||||
|
|
||||||
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs'))
|
|
||||||
instance = cluster.add_instance('instance', main_configs=dictionary_files)
|
|
||||||
test_table = DictionaryTestTable(os.path.join(SCRIPT_DIR, 'configs/dictionaries/source.tsv'))
|
|
||||||
|
|
||||||
|
|
||||||
@pytest.fixture(scope="module")
|
|
||||||
def started_cluster():
|
|
||||||
try:
|
|
||||||
cluster.start()
|
|
||||||
instance.query("CREATE DATABASE IF NOT EXISTS dict ENGINE=Dictionary")
|
|
||||||
test_table.create_clickhouse_source(instance)
|
|
||||||
for line in TSV(instance.query('select name from system.dictionaries')).lines:
|
|
||||||
print line,
|
|
||||||
|
|
||||||
# Create table `test.small_dict_source`
|
|
||||||
instance.query('''
|
|
||||||
drop table if exists test.small_dict_source;
|
|
||||||
create table test.small_dict_source (id UInt64, a String, b Int32, c Float64) engine=Log;
|
|
||||||
insert into test.small_dict_source values (0, 'water', 10, 1), (1, 'air', 40, 0.01), (2, 'earth', 100, 1.7);
|
|
||||||
''')
|
|
||||||
|
|
||||||
yield cluster
|
|
||||||
|
|
||||||
finally:
|
|
||||||
cluster.shutdown()
|
|
||||||
|
|
||||||
|
|
||||||
@pytest.fixture(params=[
|
|
||||||
# name, keys, use_parent
|
|
||||||
('clickhouse_hashed', ('id',), True),
|
|
||||||
('clickhouse_flat', ('id',), True),
|
|
||||||
('clickhouse_complex_integers_key_hashed', ('key0', 'key1'), False),
|
|
||||||
('clickhouse_complex_mixed_key_hashed', ('key0_str', 'key1'), False),
|
|
||||||
('clickhouse_range_hashed', ('id', 'StartDate', 'EndDate'), False),
|
|
||||||
],
|
|
||||||
ids=['clickhouse_hashed', 'clickhouse_flat',
|
|
||||||
'clickhouse_complex_integers_key_hashed',
|
|
||||||
'clickhouse_complex_mixed_key_hashed',
|
|
||||||
'clickhouse_range_hashed']
|
|
||||||
)
|
|
||||||
def dictionary_structure(started_cluster, request):
|
|
||||||
return request.param
|
|
||||||
|
|
||||||
|
|
||||||
def test_select_all(dictionary_structure):
|
|
||||||
name, keys, use_parent = dictionary_structure
|
|
||||||
query = instance.query
|
|
||||||
|
|
||||||
structure = test_table.get_structure_for_keys(keys, use_parent)
|
|
||||||
query('''
|
|
||||||
DROP TABLE IF EXISTS test.{0}
|
|
||||||
'''.format(name))
|
|
||||||
|
|
||||||
create_query = "CREATE TABLE test.{0} ({1}) engine = Dictionary({0})".format(name, structure)
|
|
||||||
TSV(query(create_query))
|
|
||||||
|
|
||||||
result = TSV(query('select * from test.{0}'.format(name)))
|
|
||||||
|
|
||||||
diff = test_table.compare_by_keys(keys, result.lines, use_parent, add_not_found_rows=True)
|
|
||||||
print test_table.process_diff(diff)
|
|
||||||
assert not diff
|
|
||||||
|
|
||||||
|
|
||||||
@pytest.fixture(params=[
|
|
||||||
# name, keys, use_parent
|
|
||||||
('clickhouse_cache', ('id',), True),
|
|
||||||
('clickhouse_complex_integers_key_cache', ('key0', 'key1'), False),
|
|
||||||
('clickhouse_complex_mixed_key_cache', ('key0_str', 'key1'), False)
|
|
||||||
],
|
|
||||||
ids=['clickhouse_cache', 'clickhouse_complex_integers_key_cache', 'clickhouse_complex_mixed_key_cache']
|
|
||||||
)
|
|
||||||
def cached_dictionary_structure(started_cluster, request):
|
|
||||||
return request.param
|
|
||||||
|
|
||||||
|
|
||||||
def test_select_all_from_cached(cached_dictionary_structure):
|
|
||||||
name, keys, use_parent = cached_dictionary_structure
|
|
||||||
query = instance.query
|
|
||||||
|
|
||||||
structure = test_table.get_structure_for_keys(keys, use_parent)
|
|
||||||
query('''
|
|
||||||
DROP TABLE IF EXISTS test.{0}
|
|
||||||
'''.format(name))
|
|
||||||
|
|
||||||
create_query = "CREATE TABLE test.{0} ({1}) engine = Dictionary({0})".format(name, structure)
|
|
||||||
TSV(query(create_query))
|
|
||||||
|
|
||||||
for i in range(4):
|
|
||||||
result = TSV(query('select * from test.{0}'.format(name)))
|
|
||||||
diff = test_table.compare_by_keys(keys, result.lines, use_parent, add_not_found_rows=False)
|
|
||||||
print test_table.process_diff(diff)
|
|
||||||
assert not diff
|
|
||||||
|
|
||||||
key = []
|
|
||||||
for key_name in keys:
|
|
||||||
if key_name.endswith('str'):
|
|
||||||
key.append("'" + str(i) + "'")
|
|
||||||
else:
|
|
||||||
key.append(str(i))
|
|
||||||
if len(key) == 1:
|
|
||||||
key = 'toUInt64(' + str(i) + ')'
|
|
||||||
else:
|
|
||||||
key = str('(' + ','.join(key) + ')')
|
|
||||||
query("select dictGetUInt8('{0}', 'UInt8_', {1})".format(name, key))
|
|
||||||
|
|
||||||
result = TSV(query('select * from test.{0}'.format(name)))
|
|
||||||
diff = test_table.compare_by_keys(keys, result.lines, use_parent, add_not_found_rows=True)
|
|
||||||
print test_table.process_diff(diff)
|
|
||||||
assert not diff
|
|
||||||
|
|
||||||
|
|
||||||
def test_null_value(started_cluster):
|
|
||||||
query = instance.query
|
|
||||||
|
|
||||||
assert TSV(query("select dictGetUInt8('clickhouse_cache', 'UInt8_', toUInt64(12121212))")) == TSV("1")
|
|
||||||
assert TSV(query("select dictGetString('clickhouse_cache', 'String_', toUInt64(12121212))")) == TSV("implicit-default")
|
|
||||||
assert TSV(query("select dictGetDate('clickhouse_cache', 'Date_', toUInt64(12121212))")) == TSV("2015-11-25")
|
|
||||||
|
|
||||||
# Check, that empty null_value interprets as default value
|
|
||||||
assert TSV(query("select dictGetUInt64('clickhouse_cache', 'UInt64_', toUInt64(12121212))")) == TSV("0")
|
|
||||||
assert TSV(query("select dictGetDateTime('clickhouse_cache', 'DateTime_', toUInt64(12121212))")) == TSV("0000-00-00 00:00:00")
|
|
||||||
|
|
||||||
|
|
||||||
def test_dictionary_dependency(started_cluster):
|
|
||||||
query = instance.query
|
|
||||||
|
|
||||||
# dictionaries_lazy_load == false, so these dictionary are not loaded.
|
|
||||||
assert get_status('dep_x') == 'NOT_LOADED'
|
|
||||||
assert get_status('dep_y') == 'NOT_LOADED'
|
|
||||||
assert get_status('dep_z') == 'NOT_LOADED'
|
|
||||||
|
|
||||||
# Dictionary 'dep_x' depends on 'dep_z', which depends on 'dep_y'.
|
|
||||||
# So they all should be loaded at once.
|
|
||||||
assert query("SELECT dictGetString('dep_x', 'a', toUInt64(1))") == "air\n"
|
|
||||||
assert get_status('dep_x') == 'LOADED'
|
|
||||||
assert get_status('dep_y') == 'LOADED'
|
|
||||||
assert get_status('dep_z') == 'LOADED'
|
|
||||||
|
|
||||||
# Other dictionaries should work too.
|
|
||||||
assert query("SELECT dictGetString('dep_y', 'a', toUInt64(1))") == "air\n"
|
|
||||||
assert query("SELECT dictGetString('dep_z', 'a', toUInt64(1))") == "air\n"
|
|
||||||
|
|
||||||
assert query("SELECT dictGetString('dep_x', 'a', toUInt64(3))") == "XX\n"
|
|
||||||
assert query("SELECT dictGetString('dep_y', 'a', toUInt64(3))") == "YY\n"
|
|
||||||
assert query("SELECT dictGetString('dep_z', 'a', toUInt64(3))") == "ZZ\n"
|
|
||||||
|
|
||||||
# Update the source table.
|
|
||||||
query("insert into test.small_dict_source values (3, 'fire', 30, 8)")
|
|
||||||
|
|
||||||
# Wait for dictionaries to be reloaded.
|
|
||||||
assert_eq_with_retry(instance, "SELECT dictHas('dep_y', toUInt64(3))", "1", sleep_time = 2, retry_count = 10)
|
|
||||||
assert query("SELECT dictGetString('dep_x', 'a', toUInt64(3))") == "XX\n"
|
|
||||||
assert query("SELECT dictGetString('dep_y', 'a', toUInt64(3))") == "fire\n"
|
|
||||||
assert query("SELECT dictGetString('dep_z', 'a', toUInt64(3))") == "ZZ\n"
|
|
||||||
|
|
||||||
# dep_x and dep_z are updated only when there `intDiv(count(), 4)` is changed.
|
|
||||||
query("insert into test.small_dict_source values (4, 'ether', 404, 0.001)")
|
|
||||||
assert_eq_with_retry(instance, "SELECT dictHas('dep_x', toUInt64(4))", "1", sleep_time = 2, retry_count = 10)
|
|
||||||
assert query("SELECT dictGetString('dep_x', 'a', toUInt64(3))") == "fire\n"
|
|
||||||
assert query("SELECT dictGetString('dep_y', 'a', toUInt64(3))") == "fire\n"
|
|
||||||
assert query("SELECT dictGetString('dep_z', 'a', toUInt64(3))") == "fire\n"
|
|
||||||
assert query("SELECT dictGetString('dep_x', 'a', toUInt64(4))") == "ether\n"
|
|
||||||
assert query("SELECT dictGetString('dep_y', 'a', toUInt64(4))") == "ether\n"
|
|
||||||
assert query("SELECT dictGetString('dep_z', 'a', toUInt64(4))") == "ether\n"
|
|
||||||
|
|
||||||
|
|
||||||
def test_reload_while_loading(started_cluster):
|
|
||||||
query = instance.query
|
|
||||||
|
|
||||||
# dictionaries_lazy_load == false, so this dictionary is not loaded.
|
|
||||||
assert get_status('longload') == "NOT_LOADED"
|
|
||||||
assert get_loading_duration('longload') == 0
|
|
||||||
|
|
||||||
# It's not possible to get a value from the dictionary within 1.0 second, so the following query fails by timeout.
|
|
||||||
assert query("SELECT dictGetInt32('longload', 'a', toUInt64(5))", timeout = 1, ignore_error = True) == ""
|
|
||||||
|
|
||||||
# The dictionary is now loading.
|
|
||||||
assert get_status('longload') == "LOADING"
|
|
||||||
start_time, duration = get_loading_start_time('longload'), get_loading_duration('longload')
|
|
||||||
assert duration > 0
|
|
||||||
|
|
||||||
time.sleep(0.5) # Still loading.
|
|
||||||
assert get_status('longload') == "LOADING"
|
|
||||||
prev_start_time, prev_duration = start_time, duration
|
|
||||||
start_time, duration = get_loading_start_time('longload'), get_loading_duration('longload')
|
|
||||||
assert start_time == prev_start_time
|
|
||||||
assert duration >= prev_duration
|
|
||||||
|
|
||||||
# SYSTEM RELOAD DICTIONARY should restart loading.
|
|
||||||
query("SYSTEM RELOAD DICTIONARY 'longload'")
|
|
||||||
assert get_status('longload') == "LOADING"
|
|
||||||
prev_start_time, prev_duration = start_time, duration
|
|
||||||
start_time, duration = get_loading_start_time('longload'), get_loading_duration('longload')
|
|
||||||
assert start_time > prev_start_time
|
|
||||||
assert duration < prev_duration
|
|
||||||
|
|
||||||
time.sleep(0.5) # Still loading.
|
|
||||||
assert get_status('longload') == "LOADING"
|
|
||||||
prev_start_time, prev_duration = start_time, duration
|
|
||||||
start_time, duration = get_loading_start_time('longload'), get_loading_duration('longload')
|
|
||||||
assert start_time == prev_start_time
|
|
||||||
assert duration >= prev_duration
|
|
||||||
|
|
||||||
# SYSTEM RELOAD DICTIONARIES should restart loading again.
|
|
||||||
query("SYSTEM RELOAD DICTIONARIES")
|
|
||||||
assert get_status('longload') == "LOADING"
|
|
||||||
prev_start_time, prev_duration = start_time, duration
|
|
||||||
start_time, duration = get_loading_start_time('longload'), get_loading_duration('longload')
|
|
||||||
assert start_time > prev_start_time
|
|
||||||
assert duration < prev_duration
|
|
||||||
|
|
||||||
# Changing the configuration file should restart loading one more time.
|
|
||||||
replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_longload.xml', 'sleep 100', 'sleep 0')
|
|
||||||
time.sleep(5) # Configuration files are reloaded once in 5 seconds.
|
|
||||||
|
|
||||||
# This time loading should finish quickly.
|
|
||||||
assert get_status('longload') == "LOADED"
|
|
||||||
assert query("SELECT dictGetInt32('longload', 'a', toUInt64(5))") == "6\n"
|
|
||||||
|
|
||||||
|
|
||||||
def test_reload_after_loading(started_cluster):
|
|
||||||
query = instance.query
|
|
||||||
|
|
||||||
assert query("SELECT dictGetInt32('cmd', 'a', toUInt64(7))") == "8\n"
|
|
||||||
assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "10\n"
|
|
||||||
|
|
||||||
# Change the dictionaries' data.
|
|
||||||
replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_cmd.xml', '8', '81')
|
|
||||||
replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_file.txt', '10', '101')
|
|
||||||
|
|
||||||
# SYSTEM RELOAD 'name' reloads only the specified dictionary.
|
|
||||||
query("SYSTEM RELOAD DICTIONARY 'cmd'")
|
|
||||||
assert query("SELECT dictGetInt32('cmd', 'a', toUInt64(7))") == "81\n"
|
|
||||||
assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "10\n"
|
|
||||||
|
|
||||||
query("SYSTEM RELOAD DICTIONARY 'file'")
|
|
||||||
assert query("SELECT dictGetInt32('cmd', 'a', toUInt64(7))") == "81\n"
|
|
||||||
assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "101\n"
|
|
||||||
|
|
||||||
# SYSTEM RELOAD DICTIONARIES reloads all loaded dictionaries.
|
|
||||||
replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_cmd.xml', '81', '82')
|
|
||||||
replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_file.txt', '101', '102')
|
|
||||||
query("SYSTEM RELOAD DICTIONARIES")
|
|
||||||
assert query("SELECT dictGetInt32('cmd', 'a', toUInt64(7))") == "82\n"
|
|
||||||
assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "102\n"
|
|
||||||
|
|
||||||
# Configuration files are reloaded and lifetimes are checked automatically once in 5 seconds.
|
|
||||||
replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_cmd.xml', '82', '83')
|
|
||||||
replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_file.txt', '102', '103')
|
|
||||||
time.sleep(5)
|
|
||||||
assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "103\n"
|
|
||||||
assert query("SELECT dictGetInt32('cmd', 'a', toUInt64(7))") == "83\n"
|
|
||||||
|
|
||||||
|
|
||||||
def test_reload_after_fail_by_system_reload(started_cluster):
|
|
||||||
query = instance.query
|
|
||||||
|
|
||||||
# dictionaries_lazy_load == false, so this dictionary is not loaded.
|
|
||||||
assert get_status("no_file") == "NOT_LOADED"
|
|
||||||
|
|
||||||
# We expect an error because the file source doesn't exist.
|
|
||||||
expected_error = "No such file"
|
|
||||||
assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file', 'a', toUInt64(9))")
|
|
||||||
assert get_status("no_file") == "FAILED"
|
|
||||||
|
|
||||||
# SYSTEM RELOAD should not change anything now, the status is still FAILED.
|
|
||||||
query("SYSTEM RELOAD DICTIONARY 'no_file'")
|
|
||||||
assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file', 'a', toUInt64(9))")
|
|
||||||
assert get_status("no_file") == "FAILED"
|
|
||||||
|
|
||||||
# Creating the file source makes the dictionary able to load.
|
|
||||||
instance.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/dictionaries/dictionary_preset_file.txt"), "/etc/clickhouse-server/config.d/dictionary_preset_no_file.txt")
|
|
||||||
query("SYSTEM RELOAD DICTIONARY 'no_file'")
|
|
||||||
query("SELECT dictGetInt32('no_file', 'a', toUInt64(9))") == "10\n"
|
|
||||||
assert get_status("no_file") == "LOADED"
|
|
||||||
|
|
||||||
# Removing the file source should not spoil the loaded dictionary.
|
|
||||||
instance.exec_in_container("rm /etc/clickhouse-server/config.d/dictionary_preset_no_file.txt")
|
|
||||||
query("SYSTEM RELOAD DICTIONARY 'no_file'")
|
|
||||||
query("SELECT dictGetInt32('no_file', 'a', toUInt64(9))") == "10\n"
|
|
||||||
assert get_status("no_file") == "LOADED"
|
|
||||||
|
|
||||||
|
|
||||||
def test_reload_after_fail_by_timer(started_cluster):
|
|
||||||
query = instance.query
|
|
||||||
|
|
||||||
# dictionaries_lazy_load == false, so this dictionary is not loaded.
|
|
||||||
assert get_status("no_file_2") == "NOT_LOADED"
|
|
||||||
|
|
||||||
# We expect an error because the file source doesn't exist.
|
|
||||||
expected_error = "No such file"
|
|
||||||
assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))")
|
|
||||||
assert get_status("no_file_2") == "FAILED"
|
|
||||||
|
|
||||||
# Passed time should not change anything now, the status is still FAILED.
|
|
||||||
time.sleep(6);
|
|
||||||
assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))")
|
|
||||||
assert get_status("no_file_2") == "FAILED"
|
|
||||||
|
|
||||||
# Creating the file source makes the dictionary able to load.
|
|
||||||
instance.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/dictionaries/dictionary_preset_file.txt"), "/etc/clickhouse-server/config.d/dictionary_preset_no_file_2.txt")
|
|
||||||
time.sleep(6);
|
|
||||||
query("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") == "10\n"
|
|
||||||
assert get_status("no_file_2") == "LOADED"
|
|
||||||
|
|
||||||
# Removing the file source should not spoil the loaded dictionary.
|
|
||||||
instance.exec_in_container("rm /etc/clickhouse-server/config.d/dictionary_preset_no_file_2.txt")
|
|
||||||
time.sleep(6);
|
|
||||||
query("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") == "10\n"
|
|
||||||
assert get_status("no_file_2") == "LOADED"
|
|
||||||
|
|
||||||
|
|
||||||
def test_reload_after_fail_in_cache_dictionary(started_cluster):
|
|
||||||
query = instance.query
|
|
||||||
query_and_get_error = instance.query_and_get_error
|
|
||||||
|
|
||||||
# Can't get a value from the cache dictionary because the source (table `test.xypairs`) doesn't respond.
|
|
||||||
expected_error = "Table test.xypairs doesn't exist"
|
|
||||||
assert expected_error in query_and_get_error("SELECT dictGetUInt64('cache_xypairs', 'y', toUInt64(1))")
|
|
||||||
assert get_status("cache_xypairs") == "LOADED"
|
|
||||||
assert expected_error in get_last_exception("cache_xypairs")
|
|
||||||
|
|
||||||
# Create table `test.xypairs`.
|
|
||||||
query('''
|
|
||||||
drop table if exists test.xypairs;
|
|
||||||
create table test.xypairs (x UInt64, y UInt64) engine=Log;
|
|
||||||
insert into test.xypairs values (1, 56), (3, 78);
|
|
||||||
''')
|
|
||||||
|
|
||||||
# Cache dictionary now works.
|
|
||||||
assert_eq_with_retry(instance, "SELECT dictGet('cache_xypairs', 'y', toUInt64(1))", "56", ignore_error=True)
|
|
||||||
query("SELECT dictGet('cache_xypairs', 'y', toUInt64(2))") == "0"
|
|
||||||
assert get_last_exception("cache_xypairs") == ""
|
|
||||||
|
|
||||||
# Drop table `test.xypairs`.
|
|
||||||
query('drop table if exists test.xypairs')
|
|
||||||
|
|
||||||
# Values are cached so we can get them.
|
|
||||||
query("SELECT dictGet('cache_xypairs', 'y', toUInt64(1))") == "56"
|
|
||||||
query("SELECT dictGet('cache_xypairs', 'y', toUInt64(2))") == "0"
|
|
||||||
assert get_last_exception("cache_xypairs") == ""
|
|
||||||
|
|
||||||
# But we can't get a value from the source table which isn't cached.
|
|
||||||
assert expected_error in query_and_get_error("SELECT dictGetUInt64('cache_xypairs', 'y', toUInt64(3))")
|
|
||||||
assert expected_error in get_last_exception("cache_xypairs")
|
|
||||||
|
|
||||||
# Passed time should not spoil the cache.
|
|
||||||
time.sleep(5);
|
|
||||||
query("SELECT dictGet('cache_xypairs', 'y', toUInt64(1))") == "56"
|
|
||||||
query("SELECT dictGet('cache_xypairs', 'y', toUInt64(2))") == "0"
|
|
||||||
assert expected_error in query_and_get_error("SELECT dictGetUInt64('cache_xypairs', 'y', toUInt64(3))")
|
|
||||||
assert expected_error in get_last_exception("cache_xypairs")
|
|
||||||
|
|
||||||
# Create table `test.xypairs` again with changed values.
|
|
||||||
query('''
|
|
||||||
drop table if exists test.xypairs;
|
|
||||||
create table test.xypairs (x UInt64, y UInt64) engine=Log;
|
|
||||||
insert into test.xypairs values (1, 57), (3, 79);
|
|
||||||
''')
|
|
||||||
|
|
||||||
# The cache dictionary returns new values now.
|
|
||||||
assert_eq_with_retry(instance, "SELECT dictGet('cache_xypairs', 'y', toUInt64(1))", "57")
|
|
||||||
query("SELECT dictGet('cache_xypairs', 'y', toUInt64(2))") == "0"
|
|
||||||
query("SELECT dictGet('cache_xypairs', 'y', toUInt64(3))") == "79"
|
|
||||||
assert get_last_exception("cache_xypairs") == ""
|
|
@ -0,0 +1,30 @@
|
|||||||
|
<?xml version="1.0"?>
|
||||||
|
<yandex>
|
||||||
|
<logger>
|
||||||
|
<level>trace</level>
|
||||||
|
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
|
||||||
|
<errorlog>/var/log/clickhouse-server/clickhouse-server.err.log</errorlog>
|
||||||
|
<size>1000M</size>
|
||||||
|
<count>10</count>
|
||||||
|
</logger>
|
||||||
|
|
||||||
|
<tcp_port>9000</tcp_port>
|
||||||
|
<listen_host>127.0.0.1</listen_host>
|
||||||
|
|
||||||
|
<openSSL>
|
||||||
|
<client>
|
||||||
|
<cacheSessions>true</cacheSessions>
|
||||||
|
<verificationMode>none</verificationMode>
|
||||||
|
<invalidCertificateHandler>
|
||||||
|
<name>AcceptCertificateHandler</name>
|
||||||
|
</invalidCertificateHandler>
|
||||||
|
</client>
|
||||||
|
</openSSL>
|
||||||
|
|
||||||
|
<max_concurrent_queries>500</max_concurrent_queries>
|
||||||
|
<mark_cache_size>5368709120</mark_cache_size>
|
||||||
|
<path>./clickhouse/</path>
|
||||||
|
<users_config>users.xml</users_config>
|
||||||
|
|
||||||
|
<dictionaries_config>/etc/clickhouse-server/config.d/*.xml</dictionaries_config>
|
||||||
|
</yandex>
|
@ -8,7 +8,7 @@
|
|||||||
<user>default</user>
|
<user>default</user>
|
||||||
<password></password>
|
<password></password>
|
||||||
<db>test</db>
|
<db>test</db>
|
||||||
<table>small_dict_source</table>
|
<table>elements</table>
|
||||||
</clickhouse>
|
</clickhouse>
|
||||||
</source>
|
</source>
|
||||||
<lifetime>5</lifetime>
|
<lifetime>5</lifetime>
|
@ -0,0 +1,76 @@
|
|||||||
|
import pytest
|
||||||
|
import os
|
||||||
|
from helpers.cluster import ClickHouseCluster
|
||||||
|
from helpers.test_tools import assert_eq_with_retry
|
||||||
|
|
||||||
|
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||||
|
DICTIONARY_FILES = ['configs/dictionaries/dep_x.xml', 'configs/dictionaries/dep_y.xml', 'configs/dictionaries/dep_z.xml']
|
||||||
|
|
||||||
|
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs'))
|
||||||
|
instance = cluster.add_instance('instance', main_configs=DICTIONARY_FILES)
|
||||||
|
|
||||||
|
|
||||||
|
@pytest.fixture(scope="module")
|
||||||
|
def started_cluster():
|
||||||
|
try:
|
||||||
|
cluster.start()
|
||||||
|
|
||||||
|
instance.query('''
|
||||||
|
CREATE DATABASE IF NOT EXISTS dict ENGINE=Dictionary;
|
||||||
|
CREATE DATABASE IF NOT EXISTS test;
|
||||||
|
DROP TABLE IF EXISTS test.elements;
|
||||||
|
CREATE TABLE test.elements (id UInt64, a String, b Int32, c Float64) ENGINE=Log;
|
||||||
|
INSERT INTO test.elements VALUES (0, 'water', 10, 1), (1, 'air', 40, 0.01), (2, 'earth', 100, 1.7);
|
||||||
|
''')
|
||||||
|
|
||||||
|
yield cluster
|
||||||
|
|
||||||
|
finally:
|
||||||
|
cluster.shutdown()
|
||||||
|
|
||||||
|
|
||||||
|
def get_status(dictionary_name):
|
||||||
|
return instance.query("SELECT status FROM system.dictionaries WHERE name='" + dictionary_name + "'").rstrip("\n")
|
||||||
|
|
||||||
|
|
||||||
|
def test_get_data(started_cluster):
|
||||||
|
query = instance.query
|
||||||
|
|
||||||
|
# dictionaries_lazy_load == false, so these dictionary are not loaded.
|
||||||
|
assert get_status('dep_x') == 'NOT_LOADED'
|
||||||
|
assert get_status('dep_y') == 'NOT_LOADED'
|
||||||
|
assert get_status('dep_z') == 'NOT_LOADED'
|
||||||
|
|
||||||
|
# Dictionary 'dep_x' depends on 'dep_z', which depends on 'dep_y'.
|
||||||
|
# So they all should be loaded at once.
|
||||||
|
assert query("SELECT dictGetString('dep_x', 'a', toUInt64(1))") == "air\n"
|
||||||
|
assert get_status('dep_x') == 'LOADED'
|
||||||
|
assert get_status('dep_y') == 'LOADED'
|
||||||
|
assert get_status('dep_z') == 'LOADED'
|
||||||
|
|
||||||
|
# Other dictionaries should work too.
|
||||||
|
assert query("SELECT dictGetString('dep_y', 'a', toUInt64(1))") == "air\n"
|
||||||
|
assert query("SELECT dictGetString('dep_z', 'a', toUInt64(1))") == "air\n"
|
||||||
|
|
||||||
|
assert query("SELECT dictGetString('dep_x', 'a', toUInt64(3))") == "XX\n"
|
||||||
|
assert query("SELECT dictGetString('dep_y', 'a', toUInt64(3))") == "YY\n"
|
||||||
|
assert query("SELECT dictGetString('dep_z', 'a', toUInt64(3))") == "ZZ\n"
|
||||||
|
|
||||||
|
# Update the source table.
|
||||||
|
query("INSERT INTO test.elements VALUES (3, 'fire', 30, 8)")
|
||||||
|
|
||||||
|
# Wait for dictionaries to be reloaded.
|
||||||
|
assert_eq_with_retry(instance, "SELECT dictHas('dep_y', toUInt64(3))", "1", sleep_time = 2, retry_count = 10)
|
||||||
|
assert query("SELECT dictGetString('dep_x', 'a', toUInt64(3))") == "XX\n"
|
||||||
|
assert query("SELECT dictGetString('dep_y', 'a', toUInt64(3))") == "fire\n"
|
||||||
|
assert query("SELECT dictGetString('dep_z', 'a', toUInt64(3))") == "ZZ\n"
|
||||||
|
|
||||||
|
# dep_x and dep_z are updated only when there `intDiv(count(), 4)` is changed.
|
||||||
|
query("INSERT INTO test.elements VALUES (4, 'ether', 404, 0.001)")
|
||||||
|
assert_eq_with_retry(instance, "SELECT dictHas('dep_x', toUInt64(4))", "1", sleep_time = 2, retry_count = 10)
|
||||||
|
assert query("SELECT dictGetString('dep_x', 'a', toUInt64(3))") == "fire\n"
|
||||||
|
assert query("SELECT dictGetString('dep_y', 'a', toUInt64(3))") == "fire\n"
|
||||||
|
assert query("SELECT dictGetString('dep_z', 'a', toUInt64(3))") == "fire\n"
|
||||||
|
assert query("SELECT dictGetString('dep_x', 'a', toUInt64(4))") == "ether\n"
|
||||||
|
assert query("SELECT dictGetString('dep_y', 'a', toUInt64(4))") == "ether\n"
|
||||||
|
assert query("SELECT dictGetString('dep_z', 'a', toUInt64(4))") == "ether\n"
|
@ -0,0 +1,113 @@
|
|||||||
|
<yandex>
|
||||||
|
<dictionary>
|
||||||
|
<name>cache</name>
|
||||||
|
|
||||||
|
<source>
|
||||||
|
<clickhouse>
|
||||||
|
<host>localhost</host>
|
||||||
|
<port>9000</port>
|
||||||
|
<user>default</user>
|
||||||
|
<password></password>
|
||||||
|
<db>test</db>
|
||||||
|
<table>source</table>
|
||||||
|
</clickhouse>
|
||||||
|
</source>
|
||||||
|
|
||||||
|
<lifetime>0</lifetime>
|
||||||
|
|
||||||
|
<layout>
|
||||||
|
<cache><size_in_cells>128</size_in_cells></cache>
|
||||||
|
</layout>
|
||||||
|
|
||||||
|
<structure>
|
||||||
|
<id>
|
||||||
|
<name>id</name>
|
||||||
|
</id>
|
||||||
|
|
||||||
|
<attribute>
|
||||||
|
<name>UInt8_</name>
|
||||||
|
<type>UInt8</type>
|
||||||
|
<null_value>1</null_value>
|
||||||
|
</attribute>
|
||||||
|
|
||||||
|
<attribute>
|
||||||
|
<name>UInt16_</name>
|
||||||
|
<type>UInt16</type>
|
||||||
|
<null_value>1</null_value>
|
||||||
|
</attribute>
|
||||||
|
|
||||||
|
<attribute>
|
||||||
|
<name>UInt32_</name>
|
||||||
|
<type>UInt32</type>
|
||||||
|
<null_value>1</null_value>
|
||||||
|
</attribute>
|
||||||
|
|
||||||
|
<attribute>
|
||||||
|
<name>UInt64_</name>
|
||||||
|
<type>UInt64</type>
|
||||||
|
<null_value></null_value>
|
||||||
|
</attribute>
|
||||||
|
|
||||||
|
<attribute>
|
||||||
|
<name>Int8_</name>
|
||||||
|
<type>Int8</type>
|
||||||
|
<null_value>-1</null_value>
|
||||||
|
</attribute>
|
||||||
|
|
||||||
|
<attribute>
|
||||||
|
<name>Int16_</name>
|
||||||
|
<type>Int16</type>
|
||||||
|
<null_value>-1</null_value>
|
||||||
|
</attribute>
|
||||||
|
|
||||||
|
<attribute>
|
||||||
|
<name>Int32_</name>
|
||||||
|
<type>Int32</type>
|
||||||
|
<null_value>-1</null_value>
|
||||||
|
</attribute>
|
||||||
|
|
||||||
|
<attribute>
|
||||||
|
<name>Int64_</name>
|
||||||
|
<type>Int64</type>
|
||||||
|
<null_value>-1</null_value>
|
||||||
|
</attribute>
|
||||||
|
|
||||||
|
<attribute>
|
||||||
|
<name>Float32_</name>
|
||||||
|
<type>Float32</type>
|
||||||
|
<null_value>2.71828</null_value>
|
||||||
|
</attribute>
|
||||||
|
|
||||||
|
<attribute>
|
||||||
|
<name>Float64_</name>
|
||||||
|
<type>Float64</type>
|
||||||
|
<null_value>2.71828</null_value>
|
||||||
|
</attribute>
|
||||||
|
|
||||||
|
<attribute>
|
||||||
|
<name>String_</name>
|
||||||
|
<type>String</type>
|
||||||
|
<null_value>implicit-default</null_value>
|
||||||
|
</attribute>
|
||||||
|
|
||||||
|
<attribute>
|
||||||
|
<name>Date_</name>
|
||||||
|
<type>Date</type>
|
||||||
|
<null_value>2015-11-25</null_value>
|
||||||
|
</attribute>
|
||||||
|
|
||||||
|
<attribute>
|
||||||
|
<name>DateTime_</name>
|
||||||
|
<type>DateTime</type>
|
||||||
|
<null_value></null_value>
|
||||||
|
</attribute>
|
||||||
|
|
||||||
|
<attribute>
|
||||||
|
<name>Parent</name>
|
||||||
|
<type>UInt64</type>
|
||||||
|
<hierarchical>true</hierarchical>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
</structure>
|
||||||
|
</dictionary>
|
||||||
|
</yandex>
|
@ -0,0 +1,23 @@
|
|||||||
|
<?xml version="1.0"?>
|
||||||
|
<yandex>
|
||||||
|
<profiles>
|
||||||
|
<default>
|
||||||
|
</default>
|
||||||
|
</profiles>
|
||||||
|
|
||||||
|
<users>
|
||||||
|
<default>
|
||||||
|
<password></password>
|
||||||
|
<networks incl="networks" replace="replace">
|
||||||
|
<ip>::/0</ip>
|
||||||
|
</networks>
|
||||||
|
<profile>default</profile>
|
||||||
|
<quota>default</quota>
|
||||||
|
</default>
|
||||||
|
</users>
|
||||||
|
|
||||||
|
<quotas>
|
||||||
|
<default>
|
||||||
|
</default>
|
||||||
|
</quotas>
|
||||||
|
</yandex>
|
45
dbms/tests/integration/test_dictionaries_null_value/test.py
Normal file
45
dbms/tests/integration/test_dictionaries_null_value/test.py
Normal file
@ -0,0 +1,45 @@
|
|||||||
|
import pytest
|
||||||
|
import os
|
||||||
|
from helpers.cluster import ClickHouseCluster
|
||||||
|
from helpers.test_tools import TSV, assert_eq_with_retry
|
||||||
|
|
||||||
|
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||||
|
DICTIONARY_FILES = ['configs/dictionaries/cache.xml']
|
||||||
|
|
||||||
|
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs'))
|
||||||
|
instance = cluster.add_instance('instance', main_configs=DICTIONARY_FILES)
|
||||||
|
|
||||||
|
|
||||||
|
@pytest.fixture(scope="module")
|
||||||
|
def started_cluster():
|
||||||
|
try:
|
||||||
|
cluster.start()
|
||||||
|
|
||||||
|
instance.query('''
|
||||||
|
CREATE DATABASE IF NOT EXISTS test;
|
||||||
|
DROP TABLE IF EXISTS test.source;
|
||||||
|
CREATE TABLE test.source (id UInt64, key0 UInt8, key0_str String, key1 UInt8,
|
||||||
|
StartDate Date, EndDate Date,
|
||||||
|
UInt8_ UInt8, UInt16_ UInt16, UInt32_ UInt32, UInt64_ UInt64,
|
||||||
|
Int8_ Int8, Int16_ Int16, Int32_ Int32, Int64_ Int64,
|
||||||
|
Float32_ Float32, Float64_ Float64,
|
||||||
|
String_ String,
|
||||||
|
Date_ Date, DateTime_ DateTime, Parent UInt64) ENGINE=Log;
|
||||||
|
''')
|
||||||
|
|
||||||
|
yield cluster
|
||||||
|
|
||||||
|
finally:
|
||||||
|
cluster.shutdown()
|
||||||
|
|
||||||
|
|
||||||
|
def test_null_value(started_cluster):
|
||||||
|
query = instance.query
|
||||||
|
|
||||||
|
assert query("select dictGetUInt8('cache', 'UInt8_', toUInt64(12121212))") == "1\n"
|
||||||
|
assert query("select dictGetString('cache', 'String_', toUInt64(12121212))") == "implicit-default\n"
|
||||||
|
assert query("select dictGetDate('cache', 'Date_', toUInt64(12121212))") == "2015-11-25\n"
|
||||||
|
|
||||||
|
# Check, that empty null_value interprets as default value
|
||||||
|
assert query("select dictGetUInt64('cache', 'UInt64_', toUInt64(12121212))") == "0\n"
|
||||||
|
assert query("select dictGetDateTime('cache', 'DateTime_', toUInt64(12121212))") == "0000-00-00 00:00:00\n"
|
@ -0,0 +1,30 @@
|
|||||||
|
<?xml version="1.0"?>
|
||||||
|
<yandex>
|
||||||
|
<logger>
|
||||||
|
<level>trace</level>
|
||||||
|
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
|
||||||
|
<errorlog>/var/log/clickhouse-server/clickhouse-server.err.log</errorlog>
|
||||||
|
<size>1000M</size>
|
||||||
|
<count>10</count>
|
||||||
|
</logger>
|
||||||
|
|
||||||
|
<tcp_port>9000</tcp_port>
|
||||||
|
<listen_host>127.0.0.1</listen_host>
|
||||||
|
|
||||||
|
<openSSL>
|
||||||
|
<client>
|
||||||
|
<cacheSessions>true</cacheSessions>
|
||||||
|
<verificationMode>none</verificationMode>
|
||||||
|
<invalidCertificateHandler>
|
||||||
|
<name>AcceptCertificateHandler</name>
|
||||||
|
</invalidCertificateHandler>
|
||||||
|
</client>
|
||||||
|
</openSSL>
|
||||||
|
|
||||||
|
<max_concurrent_queries>500</max_concurrent_queries>
|
||||||
|
<mark_cache_size>5368709120</mark_cache_size>
|
||||||
|
<path>./clickhouse/</path>
|
||||||
|
<users_config>users.xml</users_config>
|
||||||
|
|
||||||
|
<dictionaries_config>/etc/clickhouse-server/config.d/*.xml</dictionaries_config>
|
||||||
|
</yandex>
|
3
dbms/tests/integration/test_dictionaries_select_all/configs/dictionaries/.gitignore
vendored
Normal file
3
dbms/tests/integration/test_dictionaries_select_all/configs/dictionaries/.gitignore
vendored
Normal file
@ -0,0 +1,3 @@
|
|||||||
|
*
|
||||||
|
!.gitignore
|
||||||
|
!source.tsv
|
@ -0,0 +1,23 @@
|
|||||||
|
<?xml version="1.0"?>
|
||||||
|
<yandex>
|
||||||
|
<profiles>
|
||||||
|
<default>
|
||||||
|
</default>
|
||||||
|
</profiles>
|
||||||
|
|
||||||
|
<users>
|
||||||
|
<default>
|
||||||
|
<password></password>
|
||||||
|
<networks incl="networks" replace="replace">
|
||||||
|
<ip>::/0</ip>
|
||||||
|
</networks>
|
||||||
|
<profile>default</profile>
|
||||||
|
<quota>default</quota>
|
||||||
|
</default>
|
||||||
|
</users>
|
||||||
|
|
||||||
|
<quotas>
|
||||||
|
<default>
|
||||||
|
</default>
|
||||||
|
</quotas>
|
||||||
|
</yandex>
|
@ -12,13 +12,6 @@ types = [
|
|||||||
'Date', 'DateTime'
|
'Date', 'DateTime'
|
||||||
]
|
]
|
||||||
|
|
||||||
explicit_defaults = [
|
|
||||||
'42', '42', '42', '42',
|
|
||||||
'-42', '-42', '-42', '-42',
|
|
||||||
'1.5', '1.6',
|
|
||||||
"'explicit-default'",
|
|
||||||
"'2015-01-01'", "'2015-01-01 00:00:00'"
|
|
||||||
]
|
|
||||||
|
|
||||||
implicit_defaults = [
|
implicit_defaults = [
|
||||||
'1', '1', '1', '',
|
'1', '1', '1', '',
|
||||||
@ -182,9 +175,6 @@ def generate_dictionaries(path, structure):
|
|||||||
|
|
||||||
file_names = []
|
file_names = []
|
||||||
|
|
||||||
# Add ready dictionaries.
|
|
||||||
file_names.extend(glob.glob(os.path.join(path, "*dictionary_preset*")))
|
|
||||||
|
|
||||||
# Generate dictionaries.
|
# Generate dictionaries.
|
||||||
for (name, key_idx, has_parent), (source, layout) in zip(structure, sources_and_layouts):
|
for (name, key_idx, has_parent), (source, layout) in zip(structure, sources_and_layouts):
|
||||||
filename = os.path.join(path, 'dictionary_%s.xml' % name)
|
filename = os.path.join(path, 'dictionary_%s.xml' % name)
|
122
dbms/tests/integration/test_dictionaries_select_all/test.py
Normal file
122
dbms/tests/integration/test_dictionaries_select_all/test.py
Normal file
@ -0,0 +1,122 @@
|
|||||||
|
import pytest
|
||||||
|
import os
|
||||||
|
from helpers.cluster import ClickHouseCluster
|
||||||
|
from helpers.test_tools import TSV, assert_eq_with_retry
|
||||||
|
from generate_dictionaries import generate_structure, generate_dictionaries, DictionaryTestTable
|
||||||
|
|
||||||
|
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||||
|
|
||||||
|
cluster = None
|
||||||
|
instance = None
|
||||||
|
test_table = None
|
||||||
|
|
||||||
|
|
||||||
|
def setup_module(module):
|
||||||
|
global cluster
|
||||||
|
global instance
|
||||||
|
global test_table
|
||||||
|
|
||||||
|
structure = generate_structure()
|
||||||
|
dictionary_files = generate_dictionaries(os.path.join(SCRIPT_DIR, 'configs/dictionaries'), structure)
|
||||||
|
|
||||||
|
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs'))
|
||||||
|
instance = cluster.add_instance('instance', main_configs=dictionary_files)
|
||||||
|
test_table = DictionaryTestTable(os.path.join(SCRIPT_DIR, 'configs/dictionaries/source.tsv'))
|
||||||
|
|
||||||
|
|
||||||
|
@pytest.fixture(scope="module")
|
||||||
|
def started_cluster():
|
||||||
|
try:
|
||||||
|
cluster.start()
|
||||||
|
test_table.create_clickhouse_source(instance)
|
||||||
|
for line in TSV(instance.query('select name from system.dictionaries')).lines:
|
||||||
|
print line,
|
||||||
|
|
||||||
|
yield cluster
|
||||||
|
|
||||||
|
finally:
|
||||||
|
cluster.shutdown()
|
||||||
|
|
||||||
|
|
||||||
|
@pytest.fixture(params=[
|
||||||
|
# name, keys, use_parent
|
||||||
|
('clickhouse_hashed', ('id',), True),
|
||||||
|
('clickhouse_flat', ('id',), True),
|
||||||
|
('clickhouse_complex_integers_key_hashed', ('key0', 'key1'), False),
|
||||||
|
('clickhouse_complex_mixed_key_hashed', ('key0_str', 'key1'), False),
|
||||||
|
('clickhouse_range_hashed', ('id', 'StartDate', 'EndDate'), False),
|
||||||
|
],
|
||||||
|
ids=['clickhouse_hashed', 'clickhouse_flat',
|
||||||
|
'clickhouse_complex_integers_key_hashed',
|
||||||
|
'clickhouse_complex_mixed_key_hashed',
|
||||||
|
'clickhouse_range_hashed']
|
||||||
|
)
|
||||||
|
def dictionary_structure(started_cluster, request):
|
||||||
|
return request.param
|
||||||
|
|
||||||
|
|
||||||
|
def test_select_all(dictionary_structure):
|
||||||
|
name, keys, use_parent = dictionary_structure
|
||||||
|
query = instance.query
|
||||||
|
|
||||||
|
structure = test_table.get_structure_for_keys(keys, use_parent)
|
||||||
|
query('''
|
||||||
|
DROP TABLE IF EXISTS test.{0}
|
||||||
|
'''.format(name))
|
||||||
|
|
||||||
|
create_query = "CREATE TABLE test.{0} ({1}) engine = Dictionary({0})".format(name, structure)
|
||||||
|
TSV(query(create_query))
|
||||||
|
|
||||||
|
result = TSV(query('select * from test.{0}'.format(name)))
|
||||||
|
|
||||||
|
diff = test_table.compare_by_keys(keys, result.lines, use_parent, add_not_found_rows=True)
|
||||||
|
print test_table.process_diff(diff)
|
||||||
|
assert not diff
|
||||||
|
|
||||||
|
|
||||||
|
@pytest.fixture(params=[
|
||||||
|
# name, keys, use_parent
|
||||||
|
('clickhouse_cache', ('id',), True),
|
||||||
|
('clickhouse_complex_integers_key_cache', ('key0', 'key1'), False),
|
||||||
|
('clickhouse_complex_mixed_key_cache', ('key0_str', 'key1'), False)
|
||||||
|
],
|
||||||
|
ids=['clickhouse_cache', 'clickhouse_complex_integers_key_cache', 'clickhouse_complex_mixed_key_cache']
|
||||||
|
)
|
||||||
|
def cached_dictionary_structure(started_cluster, request):
|
||||||
|
return request.param
|
||||||
|
|
||||||
|
|
||||||
|
def test_select_all_from_cached(cached_dictionary_structure):
|
||||||
|
name, keys, use_parent = cached_dictionary_structure
|
||||||
|
query = instance.query
|
||||||
|
|
||||||
|
structure = test_table.get_structure_for_keys(keys, use_parent)
|
||||||
|
query('''
|
||||||
|
DROP TABLE IF EXISTS test.{0}
|
||||||
|
'''.format(name))
|
||||||
|
|
||||||
|
create_query = "CREATE TABLE test.{0} ({1}) engine = Dictionary({0})".format(name, structure)
|
||||||
|
TSV(query(create_query))
|
||||||
|
|
||||||
|
for i in range(4):
|
||||||
|
result = TSV(query('select * from test.{0}'.format(name)))
|
||||||
|
diff = test_table.compare_by_keys(keys, result.lines, use_parent, add_not_found_rows=False)
|
||||||
|
print test_table.process_diff(diff)
|
||||||
|
assert not diff
|
||||||
|
|
||||||
|
key = []
|
||||||
|
for key_name in keys:
|
||||||
|
if key_name.endswith('str'):
|
||||||
|
key.append("'" + str(i) + "'")
|
||||||
|
else:
|
||||||
|
key.append(str(i))
|
||||||
|
if len(key) == 1:
|
||||||
|
key = 'toUInt64(' + str(i) + ')'
|
||||||
|
else:
|
||||||
|
key = str('(' + ','.join(key) + ')')
|
||||||
|
query("select dictGetUInt8('{0}', 'UInt8_', {1})".format(name, key))
|
||||||
|
|
||||||
|
result = TSV(query('select * from test.{0}'.format(name)))
|
||||||
|
diff = test_table.compare_by_keys(keys, result.lines, use_parent, add_not_found_rows=True)
|
||||||
|
print test_table.process_diff(diff)
|
||||||
|
assert not diff
|
@ -0,0 +1,30 @@
|
|||||||
|
<?xml version="1.0"?>
|
||||||
|
<yandex>
|
||||||
|
<logger>
|
||||||
|
<level>trace</level>
|
||||||
|
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
|
||||||
|
<errorlog>/var/log/clickhouse-server/clickhouse-server.err.log</errorlog>
|
||||||
|
<size>1000M</size>
|
||||||
|
<count>10</count>
|
||||||
|
</logger>
|
||||||
|
|
||||||
|
<tcp_port>9000</tcp_port>
|
||||||
|
<listen_host>127.0.0.1</listen_host>
|
||||||
|
|
||||||
|
<openSSL>
|
||||||
|
<client>
|
||||||
|
<cacheSessions>true</cacheSessions>
|
||||||
|
<verificationMode>none</verificationMode>
|
||||||
|
<invalidCertificateHandler>
|
||||||
|
<name>AcceptCertificateHandler</name>
|
||||||
|
</invalidCertificateHandler>
|
||||||
|
</client>
|
||||||
|
</openSSL>
|
||||||
|
|
||||||
|
<max_concurrent_queries>500</max_concurrent_queries>
|
||||||
|
<mark_cache_size>5368709120</mark_cache_size>
|
||||||
|
<path>./clickhouse/</path>
|
||||||
|
<users_config>users.xml</users_config>
|
||||||
|
|
||||||
|
<dictionaries_config>/etc/clickhouse-server/config.d/*.xml</dictionaries_config>
|
||||||
|
</yandex>
|
@ -1,7 +1,7 @@
|
|||||||
<?xml version="1.0"?>
|
<?xml version="1.0"?>
|
||||||
<yandex>
|
<yandex>
|
||||||
<dictionary>
|
<dictionary>
|
||||||
<name>cmd</name>
|
<name>executable</name>
|
||||||
<source>
|
<source>
|
||||||
<executable>
|
<executable>
|
||||||
<command>echo '7\t8';</command>
|
<command>echo '7\t8';</command>
|
@ -4,7 +4,7 @@
|
|||||||
<name>file</name>
|
<name>file</name>
|
||||||
<source>
|
<source>
|
||||||
<file>
|
<file>
|
||||||
<path>/etc/clickhouse-server/config.d/dictionary_preset_file.txt</path>
|
<path>/etc/clickhouse-server/config.d/file.txt</path>
|
||||||
<format>TabSeparated</format>
|
<format>TabSeparated</format>
|
||||||
</file>
|
</file>
|
||||||
</source>
|
</source>
|
||||||
@ -21,7 +21,7 @@
|
|||||||
<name>no_file</name>
|
<name>no_file</name>
|
||||||
<source>
|
<source>
|
||||||
<file>
|
<file>
|
||||||
<path>/etc/clickhouse-server/config.d/dictionary_preset_no_file.txt</path>
|
<path>/etc/clickhouse-server/config.d/no_file.txt</path>
|
||||||
<format>TabSeparated</format>
|
<format>TabSeparated</format>
|
||||||
</file>
|
</file>
|
||||||
</source>
|
</source>
|
||||||
@ -38,7 +38,7 @@
|
|||||||
<name>no_file_2</name>
|
<name>no_file_2</name>
|
||||||
<source>
|
<source>
|
||||||
<file>
|
<file>
|
||||||
<path>/etc/clickhouse-server/config.d/dictionary_preset_no_file_2.txt</path>
|
<path>/etc/clickhouse-server/config.d/no_file_2.txt</path>
|
||||||
<format>TabSeparated</format>
|
<format>TabSeparated</format>
|
||||||
</file>
|
</file>
|
||||||
</source>
|
</source>
|
@ -1,7 +1,7 @@
|
|||||||
<?xml version="1.0"?>
|
<?xml version="1.0"?>
|
||||||
<yandex>
|
<yandex>
|
||||||
<dictionary>
|
<dictionary>
|
||||||
<name>longload</name>
|
<name>slow</name>
|
||||||
<source>
|
<source>
|
||||||
<executable>
|
<executable>
|
||||||
<command>sleep 100 && echo '5\t6';</command>
|
<command>sleep 100 && echo '5\t6';</command>
|
@ -0,0 +1,23 @@
|
|||||||
|
<?xml version="1.0"?>
|
||||||
|
<yandex>
|
||||||
|
<profiles>
|
||||||
|
<default>
|
||||||
|
</default>
|
||||||
|
</profiles>
|
||||||
|
|
||||||
|
<users>
|
||||||
|
<default>
|
||||||
|
<password></password>
|
||||||
|
<networks incl="networks" replace="replace">
|
||||||
|
<ip>::/0</ip>
|
||||||
|
</networks>
|
||||||
|
<profile>default</profile>
|
||||||
|
<quota>default</quota>
|
||||||
|
</default>
|
||||||
|
</users>
|
||||||
|
|
||||||
|
<quotas>
|
||||||
|
<default>
|
||||||
|
</default>
|
||||||
|
</quotas>
|
||||||
|
</yandex>
|
@ -0,0 +1,246 @@
|
|||||||
|
import pytest
|
||||||
|
import os
|
||||||
|
import time
|
||||||
|
from helpers.cluster import ClickHouseCluster
|
||||||
|
from helpers.test_tools import assert_eq_with_retry
|
||||||
|
|
||||||
|
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||||
|
DICTIONARY_FILES = ['configs/dictionaries/cache_xypairs.xml', 'configs/dictionaries/executable.xml', 'configs/dictionaries/file.xml', 'configs/dictionaries/file.txt', 'configs/dictionaries/slow.xml']
|
||||||
|
|
||||||
|
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs'))
|
||||||
|
instance = cluster.add_instance('instance', main_configs=DICTIONARY_FILES)
|
||||||
|
|
||||||
|
|
||||||
|
@pytest.fixture(scope="module")
|
||||||
|
def started_cluster():
|
||||||
|
try:
|
||||||
|
cluster.start()
|
||||||
|
instance.query("CREATE DATABASE IF NOT EXISTS test")
|
||||||
|
|
||||||
|
yield cluster
|
||||||
|
|
||||||
|
finally:
|
||||||
|
cluster.shutdown()
|
||||||
|
|
||||||
|
|
||||||
|
def get_status(dictionary_name):
|
||||||
|
return instance.query("SELECT status FROM system.dictionaries WHERE name='" + dictionary_name + "'").rstrip("\n")
|
||||||
|
|
||||||
|
|
||||||
|
def get_last_exception(dictionary_name):
|
||||||
|
return instance.query("SELECT last_exception FROM system.dictionaries WHERE name='" + dictionary_name + "'").rstrip("\n").replace("\\'", "'")
|
||||||
|
|
||||||
|
|
||||||
|
def get_loading_start_time(dictionary_name):
|
||||||
|
s = instance.query("SELECT loading_start_time FROM system.dictionaries WHERE name='" + dictionary_name + "'").rstrip("\n")
|
||||||
|
if s == "0000-00-00 00:00:00":
|
||||||
|
return None
|
||||||
|
return time.strptime(s, "%Y-%m-%d %H:%M:%S")
|
||||||
|
|
||||||
|
|
||||||
|
def get_loading_duration(dictionary_name):
|
||||||
|
return float(instance.query("SELECT loading_duration FROM system.dictionaries WHERE name='" + dictionary_name + "'"))
|
||||||
|
|
||||||
|
|
||||||
|
def replace_in_file_in_container(file_name, what, replace_with):
|
||||||
|
instance.exec_in_container('sed -i "s/' + what + '/' + replace_with + '/g" ' + file_name)
|
||||||
|
|
||||||
|
|
||||||
|
def test_reload_while_loading(started_cluster):
|
||||||
|
query = instance.query
|
||||||
|
|
||||||
|
# dictionaries_lazy_load == false, so this dictionary is not loaded.
|
||||||
|
assert get_status('slow') == "NOT_LOADED"
|
||||||
|
assert get_loading_duration('slow') == 0
|
||||||
|
|
||||||
|
# It's not possible to get a value from the dictionary within 1.0 second, so the following query fails by timeout.
|
||||||
|
assert query("SELECT dictGetInt32('slow', 'a', toUInt64(5))", timeout = 1, ignore_error = True) == ""
|
||||||
|
|
||||||
|
# The dictionary is now loading.
|
||||||
|
assert get_status('slow') == "LOADING"
|
||||||
|
start_time, duration = get_loading_start_time('slow'), get_loading_duration('slow')
|
||||||
|
assert duration > 0
|
||||||
|
|
||||||
|
time.sleep(0.5) # Still loading.
|
||||||
|
assert get_status('slow') == "LOADING"
|
||||||
|
prev_start_time, prev_duration = start_time, duration
|
||||||
|
start_time, duration = get_loading_start_time('slow'), get_loading_duration('slow')
|
||||||
|
assert start_time == prev_start_time
|
||||||
|
assert duration >= prev_duration
|
||||||
|
|
||||||
|
# SYSTEM RELOAD DICTIONARY should restart loading.
|
||||||
|
query("SYSTEM RELOAD DICTIONARY 'slow'")
|
||||||
|
assert get_status('slow') == "LOADING"
|
||||||
|
prev_start_time, prev_duration = start_time, duration
|
||||||
|
start_time, duration = get_loading_start_time('slow'), get_loading_duration('slow')
|
||||||
|
assert start_time > prev_start_time
|
||||||
|
assert duration < prev_duration
|
||||||
|
|
||||||
|
time.sleep(0.5) # Still loading.
|
||||||
|
assert get_status('slow') == "LOADING"
|
||||||
|
prev_start_time, prev_duration = start_time, duration
|
||||||
|
start_time, duration = get_loading_start_time('slow'), get_loading_duration('slow')
|
||||||
|
assert start_time == prev_start_time
|
||||||
|
assert duration >= prev_duration
|
||||||
|
|
||||||
|
# SYSTEM RELOAD DICTIONARIES should restart loading again.
|
||||||
|
query("SYSTEM RELOAD DICTIONARIES")
|
||||||
|
assert get_status('slow') == "LOADING"
|
||||||
|
prev_start_time, prev_duration = start_time, duration
|
||||||
|
start_time, duration = get_loading_start_time('slow'), get_loading_duration('slow')
|
||||||
|
assert start_time > prev_start_time
|
||||||
|
assert duration < prev_duration
|
||||||
|
|
||||||
|
# Changing the configuration file should restart loading one more time.
|
||||||
|
replace_in_file_in_container('/etc/clickhouse-server/config.d/slow.xml', 'sleep 100', 'sleep 0')
|
||||||
|
time.sleep(5) # Configuration files are reloaded once in 5 seconds.
|
||||||
|
|
||||||
|
# This time loading should finish quickly.
|
||||||
|
assert get_status('slow') == "LOADED"
|
||||||
|
assert query("SELECT dictGetInt32('slow', 'a', toUInt64(5))") == "6\n"
|
||||||
|
|
||||||
|
|
||||||
|
def test_reload_after_loading(started_cluster):
|
||||||
|
query = instance.query
|
||||||
|
|
||||||
|
assert query("SELECT dictGetInt32('executable', 'a', toUInt64(7))") == "8\n"
|
||||||
|
assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "10\n"
|
||||||
|
|
||||||
|
# Change the dictionaries' data.
|
||||||
|
replace_in_file_in_container('/etc/clickhouse-server/config.d/executable.xml', '8', '81')
|
||||||
|
replace_in_file_in_container('/etc/clickhouse-server/config.d/file.txt', '10', '101')
|
||||||
|
|
||||||
|
# SYSTEM RELOAD 'name' reloads only the specified dictionary.
|
||||||
|
query("SYSTEM RELOAD DICTIONARY 'executable'")
|
||||||
|
assert query("SELECT dictGetInt32('executable', 'a', toUInt64(7))") == "81\n"
|
||||||
|
assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "10\n"
|
||||||
|
|
||||||
|
query("SYSTEM RELOAD DICTIONARY 'file'")
|
||||||
|
assert query("SELECT dictGetInt32('executable', 'a', toUInt64(7))") == "81\n"
|
||||||
|
assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "101\n"
|
||||||
|
|
||||||
|
# SYSTEM RELOAD DICTIONARIES reloads all loaded dictionaries.
|
||||||
|
replace_in_file_in_container('/etc/clickhouse-server/config.d/executable.xml', '81', '82')
|
||||||
|
replace_in_file_in_container('/etc/clickhouse-server/config.d/file.txt', '101', '102')
|
||||||
|
query("SYSTEM RELOAD DICTIONARIES")
|
||||||
|
assert query("SELECT dictGetInt32('executable', 'a', toUInt64(7))") == "82\n"
|
||||||
|
assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "102\n"
|
||||||
|
|
||||||
|
# Configuration files are reloaded and lifetimes are checked automatically once in 5 seconds.
|
||||||
|
replace_in_file_in_container('/etc/clickhouse-server/config.d/executable.xml', '82', '83')
|
||||||
|
replace_in_file_in_container('/etc/clickhouse-server/config.d/file.txt', '102', '103')
|
||||||
|
time.sleep(5)
|
||||||
|
assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "103\n"
|
||||||
|
assert query("SELECT dictGetInt32('executable', 'a', toUInt64(7))") == "83\n"
|
||||||
|
|
||||||
|
|
||||||
|
def test_reload_after_fail_by_system_reload(started_cluster):
|
||||||
|
query = instance.query
|
||||||
|
|
||||||
|
# dictionaries_lazy_load == false, so this dictionary is not loaded.
|
||||||
|
assert get_status("no_file") == "NOT_LOADED"
|
||||||
|
|
||||||
|
# We expect an error because the file source doesn't exist.
|
||||||
|
expected_error = "No such file"
|
||||||
|
assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file', 'a', toUInt64(9))")
|
||||||
|
assert get_status("no_file") == "FAILED"
|
||||||
|
|
||||||
|
# SYSTEM RELOAD should not change anything now, the status is still FAILED.
|
||||||
|
query("SYSTEM RELOAD DICTIONARY 'no_file'")
|
||||||
|
assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file', 'a', toUInt64(9))")
|
||||||
|
assert get_status("no_file") == "FAILED"
|
||||||
|
|
||||||
|
# Creating the file source makes the dictionary able to load.
|
||||||
|
instance.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/dictionaries/file.txt"), "/etc/clickhouse-server/config.d/no_file.txt")
|
||||||
|
query("SYSTEM RELOAD DICTIONARY 'no_file'")
|
||||||
|
query("SELECT dictGetInt32('no_file', 'a', toUInt64(9))") == "10\n"
|
||||||
|
assert get_status("no_file") == "LOADED"
|
||||||
|
|
||||||
|
# Removing the file source should not spoil the loaded dictionary.
|
||||||
|
instance.exec_in_container("rm /etc/clickhouse-server/config.d/no_file.txt")
|
||||||
|
query("SYSTEM RELOAD DICTIONARY 'no_file'")
|
||||||
|
query("SELECT dictGetInt32('no_file', 'a', toUInt64(9))") == "10\n"
|
||||||
|
assert get_status("no_file") == "LOADED"
|
||||||
|
|
||||||
|
|
||||||
|
def test_reload_after_fail_by_timer(started_cluster):
|
||||||
|
query = instance.query
|
||||||
|
|
||||||
|
# dictionaries_lazy_load == false, so this dictionary is not loaded.
|
||||||
|
assert get_status("no_file_2") == "NOT_LOADED"
|
||||||
|
|
||||||
|
# We expect an error because the file source doesn't exist.
|
||||||
|
expected_error = "No such file"
|
||||||
|
assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))")
|
||||||
|
assert get_status("no_file_2") == "FAILED"
|
||||||
|
|
||||||
|
# Passed time should not change anything now, the status is still FAILED.
|
||||||
|
time.sleep(6);
|
||||||
|
assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))")
|
||||||
|
assert get_status("no_file_2") == "FAILED"
|
||||||
|
|
||||||
|
# Creating the file source makes the dictionary able to load.
|
||||||
|
instance.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/dictionaries/file.txt"), "/etc/clickhouse-server/config.d/no_file_2.txt")
|
||||||
|
time.sleep(6);
|
||||||
|
query("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") == "10\n"
|
||||||
|
assert get_status("no_file_2") == "LOADED"
|
||||||
|
|
||||||
|
# Removing the file source should not spoil the loaded dictionary.
|
||||||
|
instance.exec_in_container("rm /etc/clickhouse-server/config.d/no_file_2.txt")
|
||||||
|
time.sleep(6);
|
||||||
|
query("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") == "10\n"
|
||||||
|
assert get_status("no_file_2") == "LOADED"
|
||||||
|
|
||||||
|
|
||||||
|
def test_reload_after_fail_in_cache_dictionary(started_cluster):
|
||||||
|
query = instance.query
|
||||||
|
query_and_get_error = instance.query_and_get_error
|
||||||
|
|
||||||
|
# Can't get a value from the cache dictionary because the source (table `test.xypairs`) doesn't respond.
|
||||||
|
expected_error = "Table test.xypairs doesn't exist"
|
||||||
|
assert expected_error in query_and_get_error("SELECT dictGetUInt64('cache_xypairs', 'y', toUInt64(1))")
|
||||||
|
assert get_status("cache_xypairs") == "LOADED"
|
||||||
|
assert expected_error in get_last_exception("cache_xypairs")
|
||||||
|
|
||||||
|
# Create table `test.xypairs`.
|
||||||
|
query('''
|
||||||
|
DROP TABLE IF EXISTS test.xypairs;
|
||||||
|
CREATE TABLE test.xypairs (x UInt64, y UInt64) ENGINE=Log;
|
||||||
|
INSERT INTO test.xypairs VALUES (1, 56), (3, 78);
|
||||||
|
''')
|
||||||
|
|
||||||
|
# Cache dictionary now works.
|
||||||
|
assert_eq_with_retry(instance, "SELECT dictGet('cache_xypairs', 'y', toUInt64(1))", "56", ignore_error=True)
|
||||||
|
query("SELECT dictGet('cache_xypairs', 'y', toUInt64(2))") == "0"
|
||||||
|
assert get_last_exception("cache_xypairs") == ""
|
||||||
|
|
||||||
|
# Drop table `test.xypairs`.
|
||||||
|
query('DROP TABLE test.xypairs')
|
||||||
|
|
||||||
|
# Values are cached so we can get them.
|
||||||
|
query("SELECT dictGet('cache_xypairs', 'y', toUInt64(1))") == "56"
|
||||||
|
query("SELECT dictGet('cache_xypairs', 'y', toUInt64(2))") == "0"
|
||||||
|
assert get_last_exception("cache_xypairs") == ""
|
||||||
|
|
||||||
|
# But we can't get a value from the source table which isn't cached.
|
||||||
|
assert expected_error in query_and_get_error("SELECT dictGetUInt64('cache_xypairs', 'y', toUInt64(3))")
|
||||||
|
assert expected_error in get_last_exception("cache_xypairs")
|
||||||
|
|
||||||
|
# Passed time should not spoil the cache.
|
||||||
|
time.sleep(5);
|
||||||
|
query("SELECT dictGet('cache_xypairs', 'y', toUInt64(1))") == "56"
|
||||||
|
query("SELECT dictGet('cache_xypairs', 'y', toUInt64(2))") == "0"
|
||||||
|
assert expected_error in query_and_get_error("SELECT dictGetUInt64('cache_xypairs', 'y', toUInt64(3))")
|
||||||
|
assert expected_error in get_last_exception("cache_xypairs")
|
||||||
|
|
||||||
|
# Create table `test.xypairs` again with changed values.
|
||||||
|
query('''
|
||||||
|
CREATE TABLE test.xypairs (x UInt64, y UInt64) ENGINE=Log;
|
||||||
|
INSERT INTO test.xypairs VALUES (1, 57), (3, 79);
|
||||||
|
''')
|
||||||
|
|
||||||
|
# The cache dictionary returns new values now.
|
||||||
|
assert_eq_with_retry(instance, "SELECT dictGet('cache_xypairs', 'y', toUInt64(1))", "57")
|
||||||
|
query("SELECT dictGet('cache_xypairs', 'y', toUInt64(2))") == "0"
|
||||||
|
query("SELECT dictGet('cache_xypairs', 'y', toUInt64(3))") == "79"
|
||||||
|
assert get_last_exception("cache_xypairs") == ""
|
@ -0,0 +1,4 @@
|
|||||||
|
<?xml version="1.0"?>
|
||||||
|
<yandex>
|
||||||
|
<timezone>America/Los_Angeles</timezone>
|
||||||
|
</yandex>
|
17
dbms/tests/integration/test_timezone_config/test.py
Normal file
17
dbms/tests/integration/test_timezone_config/test.py
Normal file
@ -0,0 +1,17 @@
|
|||||||
|
import pytest
|
||||||
|
|
||||||
|
from helpers.cluster import ClickHouseCluster
|
||||||
|
|
||||||
|
cluster = ClickHouseCluster(__file__)
|
||||||
|
node = cluster.add_instance('node', main_configs=['configs/config.xml'])
|
||||||
|
|
||||||
|
@pytest.fixture(scope="module")
|
||||||
|
def start_cluster():
|
||||||
|
try:
|
||||||
|
cluster.start()
|
||||||
|
yield cluster
|
||||||
|
finally:
|
||||||
|
cluster.shutdown()
|
||||||
|
|
||||||
|
def test_check_timezone_config(start_cluster):
|
||||||
|
assert node.query("SELECT toDateTime(1111111111)") == "2005-03-17 17:58:31\n"
|
@ -278,3 +278,9 @@ a1,a2 12 [1,2]
|
|||||||
1 2019-06-06 1 4 2 1 5 1 [1,2] [1001,1002] [1,1]
|
1 2019-06-06 1 4 2 1 5 1 [1,2] [1001,1002] [1,1]
|
||||||
1 2019-06-06 1 4 2 1 5 0 [1,2] [1002,1003] [1,1]
|
1 2019-06-06 1 4 2 1 5 0 [1,2] [1002,1003] [1,1]
|
||||||
1 2019-06-06 1 4 2 1 6 0 [3] [2001] [1]
|
1 2019-06-06 1 4 2 1 6 0 [3] [2001] [1]
|
||||||
|
-- empty
|
||||||
|
[[1],[],[2]]
|
||||||
|
[[1],[],[2]]
|
||||||
|
[[1],[],[2],[],[3],[],[4],[],[5],[],[6],[],[7],[],[8],[],[9]]
|
||||||
|
[[],[1],[],[2],[],[3],[],[4],[],[5],[],[6],[],[7],[],[8]]
|
||||||
|
[[1],[2],[],[3]]
|
||||||
|
@ -305,3 +305,11 @@ ARRAY JOIN
|
|||||||
Test.PuidVal AS PuidValArr;
|
Test.PuidVal AS PuidValArr;
|
||||||
|
|
||||||
DROP TABLE arr_tests_visits;
|
DROP TABLE arr_tests_visits;
|
||||||
|
|
||||||
|
|
||||||
|
select '-- empty';
|
||||||
|
SELECT arrayEnumerateUniqRanked([['a'], [], ['a']]);
|
||||||
|
SELECT arrayEnumerateUniqRanked([[1], [], [1]]);
|
||||||
|
SELECT arrayEnumerateUniqRanked([[1], [], [1], [], [1], [], [1], [], [1], [], [1], [], [1], [], [1], [], [1]]);
|
||||||
|
SELECT arrayEnumerateUniqRanked([[], [1], [], [1], [], [1], [], [1], [], [1], [], [1], [], [1], [], [1]]);
|
||||||
|
SELECT arrayEnumerateUniqRanked([[1], [1], [], [1]]);
|
||||||
|
@ -5,6 +5,7 @@ insert into ttl_00933_2 values (toDateTime('2000-10-10 00:00:00'), 1);
|
|||||||
insert into ttl_00933_2 values (toDateTime('2000-10-10 00:00:00'), 2);
|
insert into ttl_00933_2 values (toDateTime('2000-10-10 00:00:00'), 2);
|
||||||
insert into ttl_00933_2 values (toDateTime('2100-10-10 00:00:00'), 3);
|
insert into ttl_00933_2 values (toDateTime('2100-10-10 00:00:00'), 3);
|
||||||
insert into ttl_00933_2 values (toDateTime('2100-10-10 00:00:00'), 4);
|
insert into ttl_00933_2 values (toDateTime('2100-10-10 00:00:00'), 4);
|
||||||
|
select sleep(0.7) format Null; -- wait if very fast merge happen
|
||||||
optimize table ttl_00933_2 final;
|
optimize table ttl_00933_2 final;
|
||||||
select a from ttl_00933_2 order by a;
|
select a from ttl_00933_2 order by a;
|
||||||
|
|
||||||
@ -15,6 +16,7 @@ insert into ttl_00933_2 values (toDateTime('2000-10-10 00:00:00'), 1, 100);
|
|||||||
insert into ttl_00933_2 values (toDateTime('2000-10-10 00:00:00'), 2, 200);
|
insert into ttl_00933_2 values (toDateTime('2000-10-10 00:00:00'), 2, 200);
|
||||||
insert into ttl_00933_2 values (toDateTime('2100-10-10 00:00:00'), 3, 300);
|
insert into ttl_00933_2 values (toDateTime('2100-10-10 00:00:00'), 3, 300);
|
||||||
insert into ttl_00933_2 values (toDateTime('2100-10-10 00:00:00'), 4, 400);
|
insert into ttl_00933_2 values (toDateTime('2100-10-10 00:00:00'), 4, 400);
|
||||||
|
select sleep(0.7) format Null; -- wait if very fast merge happen
|
||||||
optimize table ttl_00933_2 final;
|
optimize table ttl_00933_2 final;
|
||||||
select a, b from ttl_00933_2 order by a;
|
select a, b from ttl_00933_2 order by a;
|
||||||
|
|
||||||
@ -25,6 +27,7 @@ insert into ttl_00933_2 values (toDateTime('2000-10-10 00:00:00'), 1, 5);
|
|||||||
insert into ttl_00933_2 values (toDateTime('2000-10-10 00:00:00'), 2, 10);
|
insert into ttl_00933_2 values (toDateTime('2000-10-10 00:00:00'), 2, 10);
|
||||||
insert into ttl_00933_2 values (toDateTime('2100-10-10 00:00:00'), 3, 15);
|
insert into ttl_00933_2 values (toDateTime('2100-10-10 00:00:00'), 3, 15);
|
||||||
insert into ttl_00933_2 values (toDateTime('2100-10-10 00:00:00'), 4, 20);
|
insert into ttl_00933_2 values (toDateTime('2100-10-10 00:00:00'), 4, 20);
|
||||||
|
select sleep(0.7) format Null; -- wait if very fast merge happen
|
||||||
optimize table ttl_00933_2 final;
|
optimize table ttl_00933_2 final;
|
||||||
select a, b from ttl_00933_2 order by a;
|
select a, b from ttl_00933_2 order by a;
|
||||||
|
|
||||||
|
@ -0,0 +1,2 @@
|
|||||||
|
1
|
||||||
|
1
|
@ -0,0 +1,43 @@
|
|||||||
|
#!/usr/bin/env bash
|
||||||
|
|
||||||
|
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||||
|
. $CURDIR/../shell_config.sh
|
||||||
|
|
||||||
|
. $CURDIR/mergetree_mutations.lib
|
||||||
|
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS table_with_empty_part"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query="CREATE TABLE table_with_empty_part
|
||||||
|
(
|
||||||
|
id UInt64,
|
||||||
|
value UInt64
|
||||||
|
)
|
||||||
|
ENGINE = MergeTree()
|
||||||
|
ORDER BY id
|
||||||
|
PARTITION BY id
|
||||||
|
SETTINGS vertical_merge_algorithm_min_rows_to_activate=0, vertical_merge_algorithm_min_columns_to_activate=0
|
||||||
|
"
|
||||||
|
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query="INSERT INTO table_with_empty_part VALUES (1, 1)"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query="INSERT INTO table_with_empty_part VALUES (2, 2)"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query="ALTER TABLE table_with_empty_part DELETE WHERE id % 2 == 0"
|
||||||
|
|
||||||
|
sleep 0.5
|
||||||
|
|
||||||
|
mutation_id=`${CLICKHOUSE_CLIENT} --query="SELECT max(mutation_id) FROM system.mutations WHERE table='table_with_empty_part'"`
|
||||||
|
|
||||||
|
wait_for_mutation "table_with_empty_part" "$mutation_id"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT COUNT(DISTINCT value) FROM table_with_empty_part"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query="ALTER TABLE table_with_empty_part MODIFY COLUMN value Nullable(UInt64)"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT COUNT(distinct value) FROM table_with_empty_part"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query="OPTIMIZE TABLE table_with_empty_part FINAL"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS table_with_empty_part"
|
@ -0,0 +1 @@
|
|||||||
|
0000-00-00 1 bar_n_1 1
|
@ -0,0 +1,14 @@
|
|||||||
|
DROP TABLE IF EXISTS foo;
|
||||||
|
DROP TABLE IF EXISTS bar;
|
||||||
|
DROP TABLE IF EXISTS view_foo_bar;
|
||||||
|
|
||||||
|
create table foo (ddate Date, id Int64, n String) ENGINE = ReplacingMergeTree(ddate, (id), 8192);
|
||||||
|
create table bar (ddate Date, id Int64, n String, foo_id Int64) ENGINE = ReplacingMergeTree(ddate, (id), 8192);
|
||||||
|
insert into bar (id, n, foo_id) values (1, 'bar_n_1', 1);
|
||||||
|
create MATERIALIZED view view_foo_bar ENGINE = ReplacingMergeTree(ddate, (bar_id), 8192) as select ddate, bar_id, bar_n, foo_id, foo_n from (select ddate, id as bar_id, n as bar_n, foo_id from bar) any left join (select id as foo_id, n as foo_n from foo) using foo_id;
|
||||||
|
insert into bar (id, n, foo_id) values (1, 'bar_n_1', 1);
|
||||||
|
SELECT * FROM view_foo_bar;
|
||||||
|
|
||||||
|
DROP TABLE foo;
|
||||||
|
DROP TABLE bar;
|
||||||
|
DROP TABLE view_foo_bar;
|
@ -74,7 +74,7 @@ If `force_primary_key=1`, ClickHouse checks to see if the query has a primary ke
|
|||||||
|
|
||||||
## format_schema
|
## format_schema
|
||||||
|
|
||||||
This parameter is useful when you are using formats that require a schema definition, such as [Cap'n Proto](https://capnproto.org/), [Protobuf](https://developers.google.com/protocol-buffers/) or [Template](../../interfaces/formats.md#template-template). The value depends on the format.
|
This parameter is useful when you are using formats that require a schema definition, such as [Cap'n Proto](https://capnproto.org/), [Protobuf](https://developers.google.com/protocol-buffers/) or [Template](https://clickhouse.yandex/docs/en/interfaces/formats/#template). The value depends on the format.
|
||||||
|
|
||||||
## fsync_metadata
|
## fsync_metadata
|
||||||
|
|
||||||
|
@ -394,6 +394,33 @@ FROM
|
|||||||
└─────────┴─────────────────────┴───────┘
|
└─────────┴─────────────────────┴───────┘
|
||||||
```
|
```
|
||||||
|
|
||||||
|
Please note - block size affects the result. With each new block, the `runningDifference` state is reset.
|
||||||
|
|
||||||
|
``` sql
|
||||||
|
SELECT
|
||||||
|
number,
|
||||||
|
runningDifference(number + 1) AS diff
|
||||||
|
FROM numbers(100000)
|
||||||
|
WHERE diff != 1
|
||||||
|
┌─number─┬─diff─┐
|
||||||
|
│ 0 │ 0 │
|
||||||
|
└────────┴──────┘
|
||||||
|
┌─number─┬─diff─┐
|
||||||
|
│ 65536 │ 0 │
|
||||||
|
└────────┴──────┘
|
||||||
|
|
||||||
|
set max_block_size=100000 // default value is 65536!
|
||||||
|
|
||||||
|
SELECT
|
||||||
|
number,
|
||||||
|
runningDifference(number + 1) AS diff
|
||||||
|
FROM numbers(100000)
|
||||||
|
WHERE diff != 1
|
||||||
|
┌─number─┬─diff─┐
|
||||||
|
│ 0 │ 0 │
|
||||||
|
└────────┴──────┘
|
||||||
|
```
|
||||||
|
|
||||||
## runningDifferenceStartingWithFirstValue
|
## runningDifferenceStartingWithFirstValue
|
||||||
|
|
||||||
Same as for [runningDifference](./other_functions.md#other_functions-runningdifference), the difference is the value of the first row, returned the value of the first row, and each subsequent row returns the difference from the previous row.
|
Same as for [runningDifference](./other_functions.md#other_functions-runningdifference), the difference is the value of the first row, returned the value of the first row, and each subsequent row returns the difference from the previous row.
|
||||||
|
@ -72,7 +72,7 @@ ClickHouse применяет настройку в тех случаях, ко
|
|||||||
|
|
||||||
## format_schema
|
## format_schema
|
||||||
|
|
||||||
Параметр применяется в том случае, когда используются форматы, требующие определения схемы, например [Cap'n Proto](https://capnproto.org/), [Protobuf](https://developers.google.com/protocol-buffers/) или [Template](../../interfaces/formats.md#template-template). Значение параметра зависит от формата.
|
Параметр применяется в том случае, когда используются форматы, требующие определения схемы, например [Cap'n Proto](https://capnproto.org/), [Protobuf](https://developers.google.com/protocol-buffers/) или [Template](../../interfaces/formats.md#template). Значение параметра зависит от формата.
|
||||||
|
|
||||||
## fsync_metadata
|
## fsync_metadata
|
||||||
|
|
||||||
|
@ -369,6 +369,39 @@ FROM
|
|||||||
└─────────┴─────────────────────┴───────┘
|
└─────────┴─────────────────────┴───────┘
|
||||||
```
|
```
|
||||||
|
|
||||||
|
Обратите внимание — размер блока влияет на результат. С каждым новым блоком состояние `runningDifference` сбрасывается.
|
||||||
|
|
||||||
|
``` sql
|
||||||
|
SELECT
|
||||||
|
number,
|
||||||
|
runningDifference(number + 1) AS diff
|
||||||
|
FROM numbers(100000)
|
||||||
|
WHERE diff != 1
|
||||||
|
|
||||||
|
┌─number─┬─diff─┐
|
||||||
|
│ 0 │ 0 │
|
||||||
|
└────────┴──────┘
|
||||||
|
┌─number─┬─diff─┐
|
||||||
|
│ 65536 │ 0 │
|
||||||
|
└────────┴──────┘
|
||||||
|
|
||||||
|
|
||||||
|
set max_block_size=100000 // по умолчанию 65536!
|
||||||
|
|
||||||
|
SELECT
|
||||||
|
number,
|
||||||
|
runningDifference(number + 1) AS diff
|
||||||
|
FROM numbers(100000)
|
||||||
|
WHERE diff != 1
|
||||||
|
|
||||||
|
┌─number─┬─diff─┐
|
||||||
|
│ 0 │ 0 │
|
||||||
|
└────────┴──────┘
|
||||||
|
```
|
||||||
|
|
||||||
|
## runningDifferenceStartingWithFirstValue
|
||||||
|
То же, что и [runningDifference] (./other_functions.md # other_functions-runningdifference), но в первой строке возвращается значение первой строки, а не ноль.
|
||||||
|
|
||||||
## MACNumToString(num)
|
## MACNumToString(num)
|
||||||
Принимает число типа UInt64. Интерпретирует его, как MAC-адрес в big endian. Возвращает строку, содержащую соответствующий MAC-адрес в формате AA:BB:CC:DD:EE:FF (числа в шестнадцатеричной форме через двоеточие).
|
Принимает число типа UInt64. Интерпретирует его, как MAC-адрес в big endian. Возвращает строку, содержащую соответствующий MAC-адрес в формате AA:BB:CC:DD:EE:FF (числа в шестнадцатеричной форме через двоеточие).
|
||||||
|
|
||||||
|
@ -16,6 +16,7 @@
|
|||||||
#include <memory>
|
#include <memory>
|
||||||
#include <chrono>
|
#include <chrono>
|
||||||
#include <cstring>
|
#include <cstring>
|
||||||
|
#include <cassert>
|
||||||
#include <iostream>
|
#include <iostream>
|
||||||
|
|
||||||
#define DATE_LUT_MIN 0
|
#define DATE_LUT_MIN 0
|
||||||
@ -44,9 +45,16 @@ UInt8 getDayOfWeek(const cctz::civil_day & date)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
__attribute__((__weak__)) extern bool inside_main;
|
||||||
|
|
||||||
DateLUTImpl::DateLUTImpl(const std::string & time_zone_)
|
DateLUTImpl::DateLUTImpl(const std::string & time_zone_)
|
||||||
: time_zone(time_zone_)
|
: time_zone(time_zone_)
|
||||||
{
|
{
|
||||||
|
/// DateLUT should not be initialized in global constructors for the following reasons:
|
||||||
|
/// 1. It is too heavy.
|
||||||
|
if (&inside_main)
|
||||||
|
assert(inside_main);
|
||||||
|
|
||||||
size_t i = 0;
|
size_t i = 0;
|
||||||
time_t start_of_day = DATE_LUT_MIN;
|
time_t start_of_day = DATE_LUT_MIN;
|
||||||
|
|
||||||
|
@ -597,10 +597,12 @@ void BaseDaemon::initialize(Application & self)
|
|||||||
/// This must be done before any usage of DateLUT. In particular, before any logging.
|
/// This must be done before any usage of DateLUT. In particular, before any logging.
|
||||||
if (config().has("timezone"))
|
if (config().has("timezone"))
|
||||||
{
|
{
|
||||||
if (0 != setenv("TZ", config().getString("timezone").data(), 1))
|
const std::string timezone = config().getString("timezone");
|
||||||
|
if (0 != setenv("TZ", timezone.data(), 1))
|
||||||
throw Poco::Exception("Cannot setenv TZ variable");
|
throw Poco::Exception("Cannot setenv TZ variable");
|
||||||
|
|
||||||
tzset();
|
tzset();
|
||||||
|
DateLUT::setDefaultTimezone(timezone);
|
||||||
}
|
}
|
||||||
|
|
||||||
std::string log_path = config().getString("logger.log", "");
|
std::string log_path = config().getString("logger.log", "");
|
||||||
|
Loading…
Reference in New Issue
Block a user