2020-04-07 09:48:47 +00:00
|
|
|
#include <Interpreters/TableJoin.h>
|
2018-11-02 18:53:23 +00:00
|
|
|
|
|
|
|
#include <Parsers/ASTExpressionList.h>
|
|
|
|
|
2019-09-09 19:43:37 +00:00
|
|
|
#include <Core/Settings.h>
|
2019-09-02 19:58:45 +00:00
|
|
|
#include <Core/Block.h>
|
2018-11-02 18:53:23 +00:00
|
|
|
|
2020-04-12 03:20:15 +00:00
|
|
|
#include <Common/StringUtils/StringUtils.h>
|
|
|
|
|
2019-09-02 19:58:45 +00:00
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
|
|
|
|
|
|
|
|
2018-11-02 18:53:23 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2020-05-04 20:15:38 +00:00
|
|
|
TableJoin::TableJoin(const Settings & settings, VolumeJBODPtr tmp_volume_)
|
2019-09-30 14:36:45 +00:00
|
|
|
: size_limits(SizeLimits{settings.max_rows_in_join, settings.max_bytes_in_join, settings.join_overflow_mode})
|
2019-10-15 16:31:49 +00:00
|
|
|
, default_max_bytes(settings.default_max_bytes_in_join)
|
2019-09-09 19:43:37 +00:00
|
|
|
, join_use_nulls(settings.join_use_nulls)
|
2020-01-17 11:45:43 +00:00
|
|
|
, max_joined_block_rows(settings.max_joined_block_size_rows)
|
2020-02-19 19:11:23 +00:00
|
|
|
, join_algorithm(settings.join_algorithm)
|
2019-09-26 18:51:17 +00:00
|
|
|
, partial_merge_join_optimizations(settings.partial_merge_join_optimizations)
|
2019-09-23 19:36:47 +00:00
|
|
|
, partial_merge_join_rows_in_right_blocks(settings.partial_merge_join_rows_in_right_blocks)
|
2020-06-16 20:13:18 +00:00
|
|
|
, partial_merge_join_left_table_buffer_bytes(settings.partial_merge_join_left_table_buffer_bytes)
|
2020-04-28 13:55:50 +00:00
|
|
|
, max_files_to_merge(settings.join_on_disk_max_files_to_merge)
|
|
|
|
, temporary_files_codec(settings.temporary_files_codec)
|
2020-01-19 14:26:28 +00:00
|
|
|
, tmp_volume(tmp_volume_)
|
2020-02-19 19:11:23 +00:00
|
|
|
{
|
|
|
|
if (settings.partial_merge_join)
|
|
|
|
join_algorithm = JoinAlgorithm::PREFER_PARTIAL_MERGE;
|
|
|
|
}
|
2019-09-09 19:43:37 +00:00
|
|
|
|
2020-05-20 09:29:23 +00:00
|
|
|
void TableJoin::resetCollected()
|
|
|
|
{
|
|
|
|
key_names_left.clear();
|
|
|
|
key_names_right.clear();
|
|
|
|
key_asts_left.clear();
|
|
|
|
key_asts_right.clear();
|
|
|
|
columns_from_joined_table.clear();
|
|
|
|
columns_added_by_join.clear();
|
|
|
|
original_names.clear();
|
|
|
|
renames.clear();
|
|
|
|
}
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
void TableJoin::addUsingKey(const ASTPtr & ast)
|
2019-02-13 15:18:02 +00:00
|
|
|
{
|
|
|
|
key_names_left.push_back(ast->getColumnName());
|
|
|
|
key_names_right.push_back(ast->getAliasOrColumnName());
|
|
|
|
|
|
|
|
key_asts_left.push_back(ast);
|
|
|
|
key_asts_right.push_back(ast);
|
2019-07-30 18:39:37 +00:00
|
|
|
|
|
|
|
auto & right_key = key_names_right.back();
|
|
|
|
if (renames.count(right_key))
|
|
|
|
right_key = renames[right_key];
|
2019-02-13 15:18:02 +00:00
|
|
|
}
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
void TableJoin::addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast)
|
2019-02-13 15:18:02 +00:00
|
|
|
{
|
|
|
|
key_names_left.push_back(left_table_ast->getColumnName());
|
|
|
|
key_names_right.push_back(right_table_ast->getAliasOrColumnName());
|
|
|
|
|
|
|
|
key_asts_left.push_back(left_table_ast);
|
|
|
|
key_asts_right.push_back(right_table_ast);
|
|
|
|
}
|
|
|
|
|
2019-05-13 18:58:15 +00:00
|
|
|
/// @return how many times right key appears in ON section.
|
2020-04-07 09:48:47 +00:00
|
|
|
size_t TableJoin::rightKeyInclusion(const String & name) const
|
2019-05-13 18:58:15 +00:00
|
|
|
{
|
2019-09-02 19:58:45 +00:00
|
|
|
if (hasUsing())
|
2019-05-13 18:58:15 +00:00
|
|
|
return 0;
|
|
|
|
|
|
|
|
size_t count = 0;
|
|
|
|
for (const auto & key_name : key_names_right)
|
|
|
|
if (name == key_name)
|
|
|
|
++count;
|
|
|
|
return count;
|
|
|
|
}
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
void TableJoin::deduplicateAndQualifyColumnNames(const NameSet & left_table_columns, const String & right_table_prefix)
|
2019-07-30 18:39:37 +00:00
|
|
|
{
|
|
|
|
NameSet joined_columns;
|
|
|
|
NamesAndTypesList dedup_columns;
|
|
|
|
|
|
|
|
for (auto & column : columns_from_joined_table)
|
|
|
|
{
|
|
|
|
if (joined_columns.count(column.name))
|
|
|
|
continue;
|
|
|
|
|
|
|
|
joined_columns.insert(column.name);
|
|
|
|
|
|
|
|
dedup_columns.push_back(column);
|
|
|
|
auto & inserted = dedup_columns.back();
|
|
|
|
|
2020-04-12 03:20:15 +00:00
|
|
|
/// Also qualify unusual column names - that does not look like identifiers.
|
|
|
|
|
|
|
|
if (left_table_columns.count(column.name) || !isValidIdentifierBegin(column.name.at(0)))
|
2019-07-30 18:39:37 +00:00
|
|
|
inserted.name = right_table_prefix + column.name;
|
|
|
|
|
|
|
|
original_names[inserted.name] = column.name;
|
|
|
|
if (inserted.name != column.name)
|
|
|
|
renames[column.name] = inserted.name;
|
|
|
|
}
|
|
|
|
|
|
|
|
columns_from_joined_table.swap(dedup_columns);
|
|
|
|
}
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
NameSet TableJoin::getQualifiedColumnsSet() const
|
2019-01-30 15:51:39 +00:00
|
|
|
{
|
2019-07-30 18:39:37 +00:00
|
|
|
NameSet out;
|
|
|
|
for (const auto & names : original_names)
|
|
|
|
out.insert(names.first);
|
|
|
|
return out;
|
2018-11-02 18:53:23 +00:00
|
|
|
}
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
NamesWithAliases TableJoin::getNamesWithAliases(const NameSet & required_columns) const
|
2019-07-30 18:39:37 +00:00
|
|
|
{
|
2019-09-04 16:20:02 +00:00
|
|
|
NamesWithAliases out;
|
2019-07-30 18:39:37 +00:00
|
|
|
for (const auto & column : required_columns)
|
2019-02-13 19:00:52 +00:00
|
|
|
{
|
2019-07-30 18:39:37 +00:00
|
|
|
auto it = original_names.find(column);
|
|
|
|
if (it != original_names.end())
|
2019-09-04 16:20:02 +00:00
|
|
|
out.emplace_back(it->second, it->first); /// {original_name, name}
|
2019-02-13 19:00:52 +00:00
|
|
|
}
|
2019-07-30 18:39:37 +00:00
|
|
|
return out;
|
2018-11-02 18:53:23 +00:00
|
|
|
}
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
ASTPtr TableJoin::leftKeysList() const
|
2019-09-02 19:58:45 +00:00
|
|
|
{
|
|
|
|
ASTPtr keys_list = std::make_shared<ASTExpressionList>();
|
|
|
|
keys_list->children = key_asts_left;
|
|
|
|
return keys_list;
|
|
|
|
}
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
ASTPtr TableJoin::rightKeysList() const
|
2019-09-02 19:58:45 +00:00
|
|
|
{
|
|
|
|
ASTPtr keys_list = std::make_shared<ASTExpressionList>();
|
|
|
|
if (hasOn())
|
|
|
|
keys_list->children = key_asts_right;
|
|
|
|
return keys_list;
|
|
|
|
}
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
Names TableJoin::requiredJoinedNames() const
|
2019-09-02 19:58:45 +00:00
|
|
|
{
|
|
|
|
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());
|
|
|
|
}
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
NameSet TableJoin::requiredRightKeys() const
|
2019-09-10 18:39:10 +00:00
|
|
|
{
|
2019-09-11 15:57:09 +00:00
|
|
|
NameSet required;
|
2019-09-10 18:39:10 +00:00
|
|
|
for (const auto & name : key_names_right)
|
2019-09-11 15:57:09 +00:00
|
|
|
for (const auto & column : columns_added_by_join)
|
|
|
|
if (name == column.name)
|
|
|
|
required.insert(name);
|
2019-09-10 18:39:10 +00:00
|
|
|
return required;
|
|
|
|
}
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
NamesWithAliases TableJoin::getRequiredColumns(const Block & sample, const Names & action_required_columns) const
|
2019-09-02 19:58:45 +00:00
|
|
|
{
|
2019-09-04 16:20:02 +00:00
|
|
|
NameSet required_columns(action_required_columns.begin(), action_required_columns.end());
|
|
|
|
|
|
|
|
for (auto & column : requiredJoinedNames())
|
2019-09-02 19:58:45 +00:00
|
|
|
if (!sample.has(column))
|
|
|
|
required_columns.insert(column);
|
|
|
|
|
2019-09-04 16:20:02 +00:00
|
|
|
return getNamesWithAliases(required_columns);
|
2019-09-02 19:58:45 +00:00
|
|
|
}
|
|
|
|
|
2020-04-13 17:03:11 +00:00
|
|
|
bool TableJoin::leftBecomeNullable(const DataTypePtr & column_type) const
|
|
|
|
{
|
|
|
|
return forceNullableLeft() && column_type->canBeInsideNullable();
|
|
|
|
}
|
|
|
|
|
|
|
|
bool TableJoin::rightBecomeNullable(const DataTypePtr & column_type) const
|
|
|
|
{
|
|
|
|
return forceNullableRight() && column_type->canBeInsideNullable();
|
|
|
|
}
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
void TableJoin::addJoinedColumn(const NameAndTypePair & joined_column)
|
2019-09-02 19:58:45 +00:00
|
|
|
{
|
2020-04-13 17:03:11 +00:00
|
|
|
if (rightBecomeNullable(joined_column.type))
|
|
|
|
columns_added_by_join.emplace_back(NameAndTypePair(joined_column.name, makeNullable(joined_column.type)));
|
2019-09-02 19:58:45 +00:00
|
|
|
else
|
|
|
|
columns_added_by_join.push_back(joined_column);
|
|
|
|
}
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
void TableJoin::addJoinedColumnsAndCorrectNullability(Block & sample_block) const
|
2019-09-02 19:58:45 +00:00
|
|
|
{
|
|
|
|
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;
|
|
|
|
|
2020-04-13 17:03:11 +00:00
|
|
|
if (leftBecomeNullable(col.type))
|
2019-09-02 19:58:45 +00:00
|
|
|
col.type = makeNullable(col.type);
|
|
|
|
}
|
|
|
|
|
|
|
|
for (const auto & col : columns_added_by_join)
|
|
|
|
{
|
|
|
|
auto res_type = col.type;
|
|
|
|
|
2020-04-13 17:03:11 +00:00
|
|
|
if (rightBecomeNullable(res_type))
|
2019-09-02 19:58:45 +00:00
|
|
|
res_type = makeNullable(res_type);
|
|
|
|
|
|
|
|
sample_block.insert(ColumnWithTypeAndName(nullptr, res_type, col.name));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
bool TableJoin::sameJoin(const TableJoin * x, const TableJoin * y)
|
2019-09-02 19:58:45 +00:00
|
|
|
{
|
|
|
|
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
|
2019-09-16 12:37:46 +00:00
|
|
|
&& x->columns_added_by_join == y->columns_added_by_join;
|
2019-09-03 14:36:02 +00:00
|
|
|
}
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
bool TableJoin::sameStrictnessAndKind(ASTTableJoin::Strictness strictness_, ASTTableJoin::Kind kind_) const
|
2020-02-02 16:12:48 +00:00
|
|
|
{
|
|
|
|
if (strictness_ == strictness() && kind_ == kind())
|
|
|
|
return true;
|
|
|
|
|
|
|
|
/// Compatibility: old ANY INNER == new SEMI LEFT
|
|
|
|
if (strictness_ == ASTTableJoin::Strictness::Semi && isLeft(kind_) &&
|
|
|
|
strictness() == ASTTableJoin::Strictness::RightAny && isInner(kind()))
|
|
|
|
return true;
|
|
|
|
if (strictness() == ASTTableJoin::Strictness::Semi && isLeft(kind()) &&
|
|
|
|
strictness_ == ASTTableJoin::Strictness::RightAny && isInner(kind_))
|
|
|
|
return true;
|
|
|
|
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
bool TableJoin::allowMergeJoin() const
|
2019-09-16 12:37:46 +00:00
|
|
|
{
|
2020-02-11 18:27:52 +00:00
|
|
|
bool is_any = (strictness() == ASTTableJoin::Strictness::Any);
|
|
|
|
bool is_all = (strictness() == ASTTableJoin::Strictness::All);
|
|
|
|
bool is_semi = (strictness() == ASTTableJoin::Strictness::Semi);
|
2019-09-18 19:12:53 +00:00
|
|
|
|
2020-02-11 18:27:52 +00:00
|
|
|
bool allow_merge_join = (isLeft(kind()) && (is_any || is_all || is_semi)) || (isInner(kind()) && is_all);
|
2020-02-17 17:08:31 +00:00
|
|
|
return allow_merge_join;
|
2019-09-25 19:17:32 +00:00
|
|
|
}
|
|
|
|
|
2020-04-09 20:00:57 +00:00
|
|
|
bool TableJoin::allowDictJoin(const String & dict_key, const Block & sample_block, Names & names, NamesAndTypesList & result_columns) const
|
|
|
|
{
|
2020-04-13 17:03:11 +00:00
|
|
|
/// Support ALL INNER, [ANY | ALL | SEMI | ANTI] LEFT
|
|
|
|
if (!isLeft(kind()) && !(isInner(kind()) && strictness() == ASTTableJoin::Strictness::All))
|
|
|
|
return false;
|
|
|
|
|
2020-04-09 20:00:57 +00:00
|
|
|
const Names & right_keys = keyNamesRight();
|
|
|
|
if (right_keys.size() != 1)
|
|
|
|
return false;
|
|
|
|
|
2020-04-22 06:01:33 +00:00
|
|
|
for (const auto & col : sample_block)
|
2020-04-09 20:00:57 +00:00
|
|
|
{
|
|
|
|
String original = original_names.find(col.name)->second;
|
|
|
|
if (col.name == right_keys[0])
|
|
|
|
{
|
|
|
|
if (original != dict_key)
|
|
|
|
return false; /// JOIN key != Dictionary key
|
|
|
|
continue; /// do not extract key column
|
|
|
|
}
|
|
|
|
|
|
|
|
names.push_back(original);
|
|
|
|
result_columns.push_back({col.name, col.type});
|
|
|
|
}
|
|
|
|
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2018-11-02 18:53:23 +00:00
|
|
|
}
|