fix inner and left join with duplicates [issue-4108]

This commit is contained in:
chertus 2019-01-29 15:38:53 +03:00
parent eb1feacf86
commit 9c6f71bb7e
7 changed files with 305 additions and 71 deletions

View File

@ -487,19 +487,19 @@ namespace
struct Adder<true, ASTTableJoin::Strictness::Any, Map>
{
static void addFound(const typename Map::const_iterator & it, size_t num_columns_to_add, MutableColumns & added_columns,
size_t i, IColumn::Filter * filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/,
size_t i, IColumn::Filter & filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/,
const std::vector<size_t> & right_indexes)
{
(*filter)[i] = 1;
filter[i] = 1;
for (size_t j = 0; j < num_columns_to_add; ++j)
added_columns[j]->insertFrom(*it->second.block->getByPosition(right_indexes[j]).column.get(), it->second.row_num);
}
static void addNotFound(size_t num_columns_to_add, MutableColumns & added_columns,
size_t i, IColumn::Filter * filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/)
size_t i, IColumn::Filter & filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/)
{
(*filter)[i] = 0;
filter[i] = 0;
for (size_t j = 0; j < num_columns_to_add; ++j)
added_columns[j]->insertDefault();
@ -510,19 +510,19 @@ namespace
struct Adder<false, ASTTableJoin::Strictness::Any, Map>
{
static void addFound(const typename Map::const_iterator & it, size_t num_columns_to_add, MutableColumns & added_columns,
size_t i, IColumn::Filter * filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/,
size_t i, IColumn::Filter & filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/,
const std::vector<size_t> & right_indexes)
{
(*filter)[i] = 1;
filter[i] = 1;
for (size_t j = 0; j < num_columns_to_add; ++j)
added_columns[j]->insertFrom(*it->second.block->getByPosition(right_indexes[j]).column.get(), it->second.row_num);
}
static void addNotFound(size_t /*num_columns_to_add*/, MutableColumns & /*added_columns*/,
size_t i, IColumn::Filter * filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/)
size_t i, IColumn::Filter & filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/)
{
(*filter)[i] = 0;
filter[i] = 0;
}
};
@ -530,10 +530,10 @@ namespace
struct Adder<fill_left, ASTTableJoin::Strictness::All, Map>
{
static void addFound(const typename Map::const_iterator & it, size_t num_columns_to_add, MutableColumns & added_columns,
size_t i, IColumn::Filter * filter, IColumn::Offset & current_offset, IColumn::Offsets * offsets,
size_t i, IColumn::Filter & filter, IColumn::Offset & current_offset, IColumn::Offsets * offsets,
const std::vector<size_t> & right_indexes)
{
(*filter)[i] = 1;
filter[i] = 1;
size_t rows_joined = 0;
for (auto current = &static_cast<const typename Map::mapped_type::Base_t &>(it->second); current != nullptr; current = current->next)
@ -549,9 +549,9 @@ namespace
}
static void addNotFound(size_t num_columns_to_add, MutableColumns & added_columns,
size_t i, IColumn::Filter * filter, IColumn::Offset & current_offset, IColumn::Offsets * offsets)
size_t i, IColumn::Filter & filter, IColumn::Offset & current_offset, IColumn::Offsets * offsets)
{
(*filter)[i] = 0;
filter[i] = 0;
if (!fill_left)
{
@ -571,10 +571,11 @@ namespace
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS, typename KeyGetter, typename Map, bool has_null_map>
void NO_INLINE joinBlockImplTypeCase(
const Map & map, size_t rows, const ColumnRawPtrs & key_columns, const Sizes & key_sizes,
MutableColumns & added_columns, ConstNullMapPtr null_map, std::unique_ptr<IColumn::Filter> & filter,
IColumn::Offset & current_offset, std::unique_ptr<IColumn::Offsets> & offsets_to_replicate,
MutableColumns & added_columns, ConstNullMapPtr null_map, IColumn::Filter & filter,
std::unique_ptr<IColumn::Offsets> & offsets_to_replicate,
const std::vector<size_t> & right_indexes)
{
IColumn::Offset current_offset = 0;
size_t keys_size = key_columns.size();
size_t num_columns_to_add = right_indexes.size();
@ -585,7 +586,7 @@ namespace
if (has_null_map && (*null_map)[i])
{
Adder<Join::KindTrait<KIND>::fill_left, STRICTNESS, Map>::addNotFound(
num_columns_to_add, added_columns, i, filter.get(), current_offset, offsets_to_replicate.get());
num_columns_to_add, added_columns, i, filter, current_offset, offsets_to_replicate.get());
}
else
{
@ -596,30 +597,40 @@ namespace
{
it->second.setUsed();
Adder<Join::KindTrait<KIND>::fill_left, STRICTNESS, Map>::addFound(
it, num_columns_to_add, added_columns, i, filter.get(), current_offset, offsets_to_replicate.get(), right_indexes);
it, num_columns_to_add, added_columns, i, filter, current_offset, offsets_to_replicate.get(), right_indexes);
}
else
Adder<Join::KindTrait<KIND>::fill_left, STRICTNESS, Map>::addNotFound(
num_columns_to_add, added_columns, i, filter.get(), current_offset, offsets_to_replicate.get());
num_columns_to_add, added_columns, i, filter, current_offset, offsets_to_replicate.get());
}
}
}
using BlockFilterData = std::pair<
std::unique_ptr<IColumn::Filter>,
std::unique_ptr<IColumn::Offsets>>;
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS, typename KeyGetter, typename Map>
void joinBlockImplType(
BlockFilterData joinBlockImplType(
const Map & map, size_t rows, const ColumnRawPtrs & key_columns, const Sizes & key_sizes,
MutableColumns & added_columns, ConstNullMapPtr null_map, std::unique_ptr<IColumn::Filter> & filter,
IColumn::Offset & current_offset, std::unique_ptr<IColumn::Offsets> & offsets_to_replicate,
const std::vector<size_t> & right_indexes)
MutableColumns & added_columns, ConstNullMapPtr null_map, const std::vector<size_t> & right_indexes)
{
std::unique_ptr<IColumn::Filter> filter = std::make_unique<IColumn::Filter>(rows);
std::unique_ptr<IColumn::Offsets> offsets_to_replicate;
if (STRICTNESS == ASTTableJoin::Strictness::All)
offsets_to_replicate = std::make_unique<IColumn::Offsets>(rows);
if (null_map)
joinBlockImplTypeCase<KIND, STRICTNESS, KeyGetter, Map, true>(
map, rows, key_columns, key_sizes, added_columns, null_map, filter,
current_offset, offsets_to_replicate, right_indexes);
map, rows, key_columns, key_sizes, added_columns, null_map, *filter,
offsets_to_replicate, right_indexes);
else
joinBlockImplTypeCase<KIND, STRICTNESS, KeyGetter, Map, false>(
map, rows, key_columns, key_sizes, added_columns, null_map, filter,
current_offset, offsets_to_replicate, right_indexes);
map, rows, key_columns, key_sizes, added_columns, null_map, *filter,
offsets_to_replicate, right_indexes);
return {std::move(filter), std::move(offsets_to_replicate)};
}
}
@ -705,27 +716,16 @@ void Join::joinBlockImpl(
}
}
size_t rows = block.rows();
std::unique_ptr<IColumn::Filter> filter;
bool filter_left_keys = (kind == ASTTableJoin::Kind::Inner || kind == ASTTableJoin::Kind::Right) && strictness == ASTTableJoin::Strictness::Any;
filter = std::make_unique<IColumn::Filter>(rows);
/// Used with ALL ... JOIN
IColumn::Offset current_offset = 0;
std::unique_ptr<IColumn::Offsets> offsets_to_replicate;
if (strictness == ASTTableJoin::Strictness::All)
offsets_to_replicate = std::make_unique<IColumn::Offsets>(rows);
switch (type)
{
#define M(TYPE) \
case Join::Type::TYPE: \
joinBlockImplType<KIND, STRICTNESS, typename KeyGetterForType<Join::Type::TYPE>::Type>(\
*maps_.TYPE, rows, key_columns, key_sizes, added_columns, null_map, \
filter, current_offset, offsets_to_replicate, right_indexes); \
std::tie(filter, offsets_to_replicate) = \
joinBlockImplType<KIND, STRICTNESS, typename KeyGetterForType<Join::Type::TYPE>::Type>(\
*maps_.TYPE, block.rows(), key_columns, key_sizes, added_columns, null_map, right_indexes); \
break;
APPLY_FOR_JOIN_VARIANTS(M)
#undef M
@ -738,47 +738,94 @@ void Join::joinBlockImpl(
for (size_t i = 0; i < added_columns_size; ++i)
block.insert(ColumnWithTypeAndName(std::move(added_columns[i]), added_type_name[i].first, added_type_name[i].second));
/// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones.
if (filter_left_keys)
for (size_t i = 0; i < existing_columns; ++i)
block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(*filter, -1);
if (!filter)
throw Exception("No data to filter columns", ErrorCodes::LOGICAL_ERROR);
ColumnUInt64::Ptr mapping;
/// Add join key columns from right block if they has different name.
for (size_t i = 0; i < key_names_right.size(); ++i)
if (strictness == ASTTableJoin::Strictness::Any)
{
auto & right_name = key_names_right[i];
auto & left_name = key_names_left[i];
if (needed_key_names_right.count(right_name) && !block.has(right_name))
if (kind == ASTTableJoin::Kind::Inner || kind == ASTTableJoin::Kind::Right)
{
const auto & col = block.getByName(left_name);
auto column = col.column;
if (!filter_left_keys)
/// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones.
for (size_t i = 0; i < existing_columns; ++i)
block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(*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)
{
if (!mapping)
auto & right_name = key_names_right[i];
auto & left_name = key_names_left[i];
if (needed_key_names_right.count(right_name) && !block.has(right_name))
{
auto mut_mapping = ColumnUInt64::create(column->size());
auto & data = mut_mapping->getData();
size_t size = column->size();
for (size_t j = 0; j < size; ++j)
data[j] = (*filter)[j] ? j : size;
mapping = std::move(mut_mapping);
const auto & col = block.getByName(left_name);
block.insert({col.column, col.type, right_name});
}
}
}
else
{
/// Add join key columns from right block if they has different name.
for (size_t i = 0; i < key_names_right.size(); ++i)
{
auto & right_name = key_names_right[i];
auto & left_name = key_names_left[i];
if (needed_key_names_right.count(right_name) && !block.has(right_name))
{
const auto & col = block.getByName(left_name);
auto & column = col.column;
MutableColumnPtr mut_column = column->cloneEmpty();
for (size_t col_no = 0; col_no < filter->size(); ++col_no)
{
if ((*filter)[col_no])
mut_column->insertFrom(*column, col_no);
else
mut_column->insertDefault();
}
block.insert({std::move(mut_column), col.type, right_name});
}
auto mut_column = (*std::move(column)).mutate();
mut_column->insertDefault();
column = mut_column->index(*mapping, 0);
}
block.insert({column, col.type, right_name});
}
}
/// If ALL ... JOIN - we replicate all the columns except the new ones.
if (offsets_to_replicate)
else
{
if (!offsets_to_replicate)
throw Exception("No data to filter columns", ErrorCodes::LOGICAL_ERROR);
/// Add join key columns from right block if they has different name.
for (size_t i = 0; i < key_names_right.size(); ++i)
{
auto & right_name = key_names_right[i];
auto & left_name = key_names_left[i];
if (needed_key_names_right.count(right_name) && !block.has(right_name))
{
const auto & col = block.getByName(left_name);
auto & column = col.column;
MutableColumnPtr mut_column = column->cloneEmpty();
size_t last_offset = 0;
for (size_t col_no = 0; col_no < column->size(); ++col_no)
{
if (size_t to_insert = (*offsets_to_replicate)[col_no] - last_offset)
{
if (!(*filter)[col_no])
mut_column->insertDefault();
else
for (size_t dup = 0; dup < to_insert; ++dup)
mut_column->insertFrom(*column, col_no);
}
last_offset = (*offsets_to_replicate)[col_no];
}
block.insert({std::move(mut_column), col.type, right_name});
}
}
/// If ALL ... JOIN - we replicate all the columns except the new ones.
for (size_t i = 0; i < existing_columns; ++i)
block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->replicate(*offsets_to_replicate);
}

View File

@ -0,0 +1,66 @@
inner
1 A 1 a
1 A 1 b
2 B 2 c
2 C 2 c
3 D 3 d
3 D 3 e
4 E 4 f
4 F 4 f
9 I 9 i
inner subs
1 A 1 a
1 A 1 b
2 B 2 c
2 C 2 c
3 D 3 d
3 D 3 e
4 E 4 f
4 F 4 f
9 I 9 i
inner expr
1 A 1 a
1 A 1 b
2 B 2 c
2 C 2 c
3 D 3 d
3 D 3 e
4 E 4 f
4 F 4 f
9 I 9 i
left
1 A 1 a
1 A 1 b
2 B 2 c
2 C 2 c
3 D 3 d
3 D 3 e
4 E 4 f
4 F 4 f
5 G 0
8 H 0
9 I 9 i
left subs
1 A 1 a
1 A 1 b
2 B 2 c
2 C 2 c
3 D 3 d
3 D 3 e
4 E 4 f
4 F 4 f
5 G 0
8 H 0
9 I 9 i
left expr
1 A 1 a
1 A 1 b
2 B 2 c
2 C 2 c
3 D 3 d
3 D 3 e
4 E 4 f
4 F 4 f
5 G 0
8 H 0
9 I 9 i

View File

@ -0,0 +1,40 @@
use test;
drop table if exists X;
drop table if exists Y;
create table X (id Int32, x_name String) engine Memory;
create table Y (id Int32, y_name String) engine Memory;
insert into X (id, x_name) values (1, 'A'), (2, 'B'), (2, 'C'), (3, 'D'), (4, 'E'), (4, 'F'), (5, 'G'), (8, 'H'), (9, 'I');
insert into Y (id, y_name) values (1, 'a'), (1, 'b'), (2, 'c'), (3, 'd'), (3, 'e'), (4, 'f'), (6, 'g'), (7, 'h'), (9, 'i');
select 'inner';
select X.*, Y.* from X inner join Y on X.id = Y.id;
select 'inner subs';
select s.*, j.* from (select * from X) as s inner join (select * from Y) as j on s.id = j.id;
select 'inner expr';
select X.*, Y.* from X inner join Y on (X.id + 1) = (Y.id + 1);
select 'left';
select X.*, Y.* from X left join Y on X.id = Y.id;
select 'left subs';
select s.*, j.* from (select * from X) as s left join (select * from Y) as j on s.id = j.id;
select 'left expr';
select X.*, Y.* from X left join Y on (X.id + 1) = (Y.id + 1);
--select 'right';
--select X.*, Y.* from X right join Y on X.id = Y.id order by id;
--select 'right subs';
--select s.*, j.* from (select * from X) as s right join (select * from Y) as j on s.id = j.id order by id;
--select 'right expr';
--select X.*, Y.* from X right join Y on (X.id + 1) = (Y.id + 1) order by id;
--select 'full';
--select X.*, Y.* from X full join Y on X.id = Y.id order by id;
--select 'full subs';
--select s.*, j.* from (select * from X) as s full join (select * from Y) as j on s.id = j.id order by id;
--select 'full expr';
--select X.*, Y.* from X full join Y on (X.id + 1) = (Y.id + 1) order by id;
drop table X;
drop table Y;

View File

@ -0,0 +1,44 @@
inner
1 A 1 a
1 A 1 b
2 B 2 c
2 C 2 c
3 D 3 d
3 D 3 e
4 E 4 f
4 F 4 f
9 I 9 i
inner subs
1 A 1 a
1 A 1 b
2 B 2 c
2 C 2 c
3 D 3 d
3 D 3 e
4 E 4 f
4 F 4 f
9 I 9 i
left
1 A 1 a
1 A 1 b
2 B 2 c
2 C 2 c
3 D 3 d
3 D 3 e
4 E 4 f
4 F 4 f
5 G 0
8 H 0
9 I 9 i
left subs
1 A 1 a
1 A 1 b
2 B 2 c
2 C 2 c
3 D 3 d
3 D 3 e
4 E 4 f
4 F 4 f
5 G 0
8 H 0
9 I 9 i

View File

@ -0,0 +1,32 @@
use test;
drop table if exists X;
drop table if exists Y;
create table X (id Int32, x_name String) engine Memory;
create table Y (id Int32, y_name String) engine Memory;
insert into X (id, x_name) values (1, 'A'), (2, 'B'), (2, 'C'), (3, 'D'), (4, 'E'), (4, 'F'), (5, 'G'), (8, 'H'), (9, 'I');
insert into Y (id, y_name) values (1, 'a'), (1, 'b'), (2, 'c'), (3, 'd'), (3, 'e'), (4, 'f'), (6, 'g'), (7, 'h'), (9, 'i');
select 'inner';
select X.*, Y.* from X inner join Y using id;
select 'inner subs';
select s.*, j.* from (select * from X) as s inner join (select * from Y) as j using id;
select 'left';
select X.*, Y.* from X left join Y using id;
select 'left subs';
select s.*, j.* from (select * from X) as s left join (select * from Y) as j using id;
--select 'right';
--select X.*, Y.* from X right join Y using id order by id;
--select 'right subs';
--select s.*, j.* from (select * from X) as s right join (select * from Y) as j using id order by id;
--select 'full';
--select X.*, Y.* from X full join Y using id order by id;
--select 'full subs';
--select s.*, j.* from (select * from X) as s full join (select * from Y) as j using id order by id;
drop table X;
drop table Y;

View File

@ -1,3 +1,7 @@
1 1 1 2
1 2 1 2
2 3 0 0
-
1 1 1 2
1 2 1 2
2 3 0 0

View File

@ -8,7 +8,8 @@ INSERT INTO test.a1 VALUES (1, 1), (1, 2), (2, 3);
INSERT INTO test.a2 VALUES (1, 2), (1, 3), (1, 4);
SELECT * FROM test.a1 as a left JOIN test.a2 as b on a.a=b.a ORDER BY b SETTINGS join_default_strictness='ANY';
SELECT '-';
SELECT a1.*, a2.* FROM test.a1 ANY LEFT JOIN test.a2 USING a ORDER BY b;
DROP TABLE IF EXISTS test.a1;
DROP TABLE IF EXISTS test.a2;