提交 73dafaa2 编写于 作者: C chertus

better required-right-keys logic

上级 8afa48fa
......@@ -142,17 +142,13 @@ Names AnalyzedJoin::requiredJoinedNames() const
return Names(required_columns_set.begin(), required_columns_set.end());
}
std::unordered_map<String, DataTypePtr> AnalyzedJoin::requiredRightKeys() const
NameSet AnalyzedJoin::requiredRightKeys() const
{
NameSet right_keys;
NameSet required;
for (const auto & name : key_names_right)
right_keys.insert(name);
std::unordered_map<String, DataTypePtr> required;
for (const auto & column : columns_added_by_join)
if (right_keys.count(column.name))
required.insert({column.name, column.type});
for (const auto & column : columns_added_by_join)
if (name == column.name)
required.insert(name);
return required;
}
......
......@@ -92,7 +92,7 @@ public:
void deduplicateAndQualifyColumnNames(const NameSet & left_table_columns, const String & right_table_prefix);
size_t rightKeyInclusion(const String & name) const;
std::unordered_map<String, DataTypePtr> requiredRightKeys() const;
NameSet requiredRightKeys() const;
void addJoinedColumn(const NameAndTypePair & joined_column);
void addJoinedColumnsAndCorrectNullability(Block & sample_block) const;
......
......@@ -75,6 +75,7 @@ Join::Join(const AnalyzedJoin & join_options_, const Block & right_sample_block,
, kind(join_options_.kind())
, strictness(join_options_.strictness())
, key_names_right(join_options_.keyNamesRight())
, required_right_keys(join_options_.requiredRightKeys())
, use_nulls(join_options_.joinUseNulls())
, any_take_last_row(any_take_last_row_)
, log(&Logger::get("Join"))
......@@ -263,7 +264,7 @@ void Join::setSampleBlock(const Block & block)
if (!empty())
return;
ColumnRawPtrs key_columns = extractKeysForJoin(key_names_right, block, sample_block_with_keys, sample_block_with_columns_to_add);
ColumnRawPtrs key_columns = extractKeysForJoin(key_names_right, block, right_table_keys, sample_block_with_columns_to_add);
if (strictness == ASTTableJoin::Strictness::Asof)
{
......@@ -783,7 +784,7 @@ void Join::joinBlockImpl(
*/
ColumnsWithTypeAndName extras;
if constexpr (STRICTNESS == ASTTableJoin::Strictness::Asof)
extras.push_back(sample_block_with_keys.getByName(key_names_right.back()));
extras.push_back(right_table_keys.getByName(key_names_right.back()));
AddedColumns added(sample_block_with_columns_to_add, block_with_columns_to_add, block, blocklist_sample, extras);
std::unique_ptr<IColumn::Offsets> offsets_to_replicate;
......@@ -795,8 +796,6 @@ void Join::joinBlockImpl(
block.insert(added.moveColumn(i));
/// Filter & insert missing rows
auto right_keys = join_options.requiredRightKeys();
constexpr bool is_all_join = STRICTNESS == ASTTableJoin::Strictness::All;
constexpr bool inner_or_right = static_in_v<KIND, ASTTableJoin::Kind::Inner, ASTTableJoin::Kind::Right>;
constexpr bool left_or_full = static_in_v<KIND, ASTTableJoin::Kind::Left, ASTTableJoin::Kind::Full>;
......@@ -810,17 +809,16 @@ void Join::joinBlockImpl(
block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(row_filter, -1);
/// Add join key columns from right block if they has different name.
for (size_t i = 0; i < key_names_right.size(); ++i)
for (size_t i = 0; i < right_table_keys.columns(); ++i)
{
auto & right_name = key_names_right[i];
const auto & right_key = right_table_keys.getByPosition(i);
auto & left_name = key_names_left[i];
auto it = right_keys.find(right_name);
if (it != right_keys.end() && !block.has(right_name))
if (required_right_keys.count(right_key.name) && !block.has(right_key.name))
{
const auto & col = block.getByName(left_name);
bool is_nullable = it->second->isNullable();
block.insert(correctNullability({col.column, col.type, right_name}, is_nullable));
bool is_nullable = (use_nulls && left_or_full) || right_key.type->isNullable();
block.insert(correctNullability({col.column, col.type, right_key.name}, is_nullable));
}
}
}
......@@ -833,13 +831,12 @@ void Join::joinBlockImpl(
const IColumn::Filter & filter = null_map_filter.getData();
/// Add join key columns from right block if they has different name.
for (size_t i = 0; i < key_names_right.size(); ++i)
for (size_t i = 0; i < right_table_keys.columns(); ++i)
{
auto & right_name = key_names_right[i];
const auto & right_key = right_table_keys.getByPosition(i);
auto & left_name = key_names_left[i];
auto it = right_keys.find(right_name);
if (it != right_keys.end() && !block.has(right_name))
if (required_right_keys.count(right_key.name) && !block.has(right_key.name))
{
const auto & col = block.getByName(left_name);
ColumnPtr column = col.column->convertToFullColumnIfConst();
......@@ -854,11 +851,11 @@ void Join::joinBlockImpl(
mut_column->insertDefault();
}
bool is_nullable = (use_nulls && left_or_full) || it->second->isNullable();
block.insert(correctNullability({std::move(mut_column), col.type, right_name}, is_nullable, null_map_filter));
bool is_nullable = (use_nulls && left_or_full) || right_key.type->isNullable();
block.insert(correctNullability({std::move(mut_column), col.type, right_key.name}, is_nullable, null_map_filter));
if constexpr (is_all_join)
right_keys_to_replicate.push_back(block.getPositionByName(right_name));
right_keys_to_replicate.push_back(block.getPositionByName(right_key.name));
}
}
}
......@@ -992,7 +989,7 @@ void Join::joinGet(Block & block, const String & column_name) const
if (key_names_right.size() != 1)
throw Exception("joinGet only supports StorageJoin containing exactly one key", ErrorCodes::LOGICAL_ERROR);
checkTypeOfKey(block, sample_block_with_keys);
checkTypeOfKey(block, right_table_keys);
if (kind == ASTTableJoin::Kind::Left && strictness == ASTTableJoin::Strictness::Any)
{
......@@ -1009,7 +1006,7 @@ void Join::joinBlock(Block & block)
std::shared_lock lock(rwlock);
checkTypesOfKeys(block, key_names_left, sample_block_with_keys);
checkTypesOfKeys(block, key_names_left, right_table_keys);
if (joinDispatch(kind, strictness, maps, [&](auto kind_, auto strictness_, auto & map)
{
......@@ -1115,7 +1112,6 @@ public:
, max_block_size(max_block_size_)
{
const Names & key_names_left = parent_.join_options.keyNamesLeft();
std::unordered_map<String, DataTypePtr> required_right_keys = parent_.join_options.requiredRightKeys();
/** left_sample_block contains keys and "left" columns.
* result_sample_block - keys, "left" columns, and "right" columns.
......@@ -1135,10 +1131,9 @@ public:
const Block & right_sample_block = parent.sample_block_with_columns_to_add;
std::unordered_map<size_t, size_t> left_to_right_key_map;
makeResultSampleBlock(left_sample_block, right_sample_block, required_right_keys,
key_positions_left, left_to_right_key_map);
makeResultSampleBlock(left_sample_block, right_sample_block, key_positions_left, left_to_right_key_map);
auto nullability_changes = getNullabilityChanges(parent.sample_block_with_keys, result_sample_block,
auto nullability_changes = getNullabilityChanges(parent.right_table_keys, result_sample_block,
key_positions_left, left_to_right_key_map);
column_indices_left.reserve(left_sample_block.columns() - key_names_left.size());
......@@ -1204,7 +1199,6 @@ private:
void makeResultSampleBlock(const Block & left_sample_block, const Block & right_sample_block,
const std::unordered_map<String, DataTypePtr> & right_keys,
const std::vector<size_t> & key_positions_left,
std::unordered_map<size_t, size_t> & left_to_right_key_map)
{
......@@ -1223,22 +1217,19 @@ private:
result_sample_block.insert(src_column.cloneEmpty());
}
const auto & key_names_right = parent.key_names_right;
/// Add join key columns from right block if they has different name.
for (size_t i = 0; i < key_names_right.size(); ++i)
for (size_t i = 0; i < parent.right_table_keys.columns(); ++i)
{
auto & right_name = key_names_right[i];
const auto & right_key = parent.right_table_keys.getByPosition(i);
size_t left_key_pos = key_positions_left[i];
auto it = right_keys.find(right_name);
if (it != right_keys.end() && !result_sample_block.has(right_name))
if (parent.required_right_keys.count(right_key.name) && !result_sample_block.has(right_key.name))
{
const auto & col = result_sample_block.getByPosition(left_key_pos);
bool is_nullable = (parent.use_nulls && isFull(parent.kind)) || it->second->isNullable();
result_sample_block.insert(correctNullability({col.column, col.type, right_name}, is_nullable));
bool is_nullable = (parent.use_nulls && isFull(parent.kind)) || right_key.type->isNullable();
result_sample_block.insert(correctNullability({col.column, col.type, right_key.name}, is_nullable));
size_t right_key_pos = result_sample_block.getPositionByName(right_name);
size_t right_key_pos = result_sample_block.getPositionByName(right_key.name);
left_to_right_key_map[left_key_pos] = right_key_pos;
}
}
......@@ -1372,7 +1363,7 @@ private:
}
}
static std::unordered_set<size_t> getNullabilityChanges(const Block & sample_block_with_keys, const Block & out_block,
static std::unordered_set<size_t> getNullabilityChanges(const Block & right_table_keys, const Block & out_block,
const std::vector<size_t> & key_positions,
const std::unordered_map<size_t, size_t> & left_to_right_key_map)
{
......@@ -1387,7 +1378,7 @@ private:
key_pos = it->second;
const auto & dst = out_block.getByPosition(key_pos).column;
const auto & src = sample_block_with_keys.getByPosition(i).column;
const auto & src = right_table_keys.getByPosition(i).column;
if (dst->isNullable() != src->isNullable())
nullability_changes.insert(key_pos);
}
......
......@@ -280,8 +280,10 @@ private:
ASTTableJoin::Kind kind;
ASTTableJoin::Strictness strictness;
/// Names of key columns (columns for equi-JOIN) in "right" table (in the order they appear in USING clause).
/// Names of key columns in right-side table (in the order they appear in ON/USING clause). @note It could contain duplicates.
const Names key_names_right;
/// Names right-side table keys that are needed in result (would be attached after joined columns).
const NameSet required_right_keys;
/// Substitute NULLs for non-JOINed rows.
bool use_nulls;
......@@ -310,8 +312,8 @@ private:
/// Block with columns from the right-side table except key columns.
Block sample_block_with_columns_to_add;
/// Block with key columns in the same order they appear in the right-side table.
Block sample_block_with_keys;
/// Block with key columns in the same order they appear in the right-side table (duplicates appear once).
Block right_table_keys;
/// Block as it would appear in the BlockList
Block blocklist_sample;
......
......@@ -209,10 +209,10 @@ public:
for (size_t i = 0; i < sample_block.columns(); ++i)
{
auto & [_, type, name] = sample_block.getByPosition(i);
if (parent.sample_block_with_keys.has(name))
if (parent.right_table_keys.has(name))
{
key_pos = i;
column_with_null[i] = parent.sample_block_with_keys.getByName(name).type->isNullable();
column_with_null[i] = parent.right_table_keys.getByName(name).type->isNullable();
}
else
{
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册