mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
Fix default value in join types with non-zero default (close #18197)
This commit is contained in:
parent
e75b116466
commit
2f11fdf4df
@ -702,7 +702,7 @@ public:
|
||||
if (lazy_defaults_count)
|
||||
{
|
||||
for (size_t j = 0; j < right_indexes.size(); ++j)
|
||||
columns[j]->insertManyDefaults(lazy_defaults_count);
|
||||
JoinCommon::addDefaultValues(columns[j], type_name[j].first, lazy_defaults_count);
|
||||
lazy_defaults_count = 0;
|
||||
}
|
||||
}
|
||||
|
@ -398,7 +398,8 @@ bool joinEquals(const Block & left_block, const Block & right_block, const Block
|
||||
}
|
||||
|
||||
template <bool copy_left>
|
||||
void joinInequalsLeft(const Block & left_block, MutableColumns & left_columns, MutableColumns & right_columns,
|
||||
void joinInequalsLeft(const Block & left_block, MutableColumns & left_columns,
|
||||
const Block & right_block, MutableColumns & right_columns,
|
||||
size_t start, size_t end)
|
||||
{
|
||||
if (end <= start)
|
||||
@ -408,9 +409,10 @@ void joinInequalsLeft(const Block & left_block, MutableColumns & left_columns, M
|
||||
if constexpr (copy_left)
|
||||
copyLeftRange(left_block, left_columns, start, rows_to_add);
|
||||
|
||||
/// append nulls
|
||||
for (auto & column : right_columns)
|
||||
column->insertManyDefaults(rows_to_add);
|
||||
for (size_t i = 0; i < right_columns.size(); ++i)
|
||||
{
|
||||
JoinCommon::addDefaultValues(right_columns[i], right_block.getByPosition(i).type, rows_to_add);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@ -696,7 +698,7 @@ void MergeJoin::joinSortedBlock(Block & block, ExtraBlockPtr & not_processed)
|
||||
}
|
||||
|
||||
left_cursor.nextN(left_key_tail);
|
||||
joinInequalsLeft<is_all>(block, left_columns, right_columns, left_cursor.position(), left_cursor.end());
|
||||
joinInequalsLeft<is_all>(block, left_columns, right_columns_to_add, right_columns, left_cursor.position(), left_cursor.end());
|
||||
//left_cursor.nextN(left_cursor.end() - left_cursor.position());
|
||||
|
||||
changeLeftColumns(block, std::move(left_columns));
|
||||
@ -773,7 +775,7 @@ bool MergeJoin::leftJoin(MergeJoinCursor & left_cursor, const Block & left_block
|
||||
|
||||
Range range = left_cursor.getNextEqualRange(right_cursor);
|
||||
|
||||
joinInequalsLeft<is_all>(left_block, left_columns, right_columns, left_unequal_position, range.left_start);
|
||||
joinInequalsLeft<is_all>(left_block, left_columns, right_columns_to_add, right_columns, left_unequal_position, range.left_start);
|
||||
|
||||
if (range.empty())
|
||||
break;
|
||||
|
@ -276,6 +276,20 @@ void joinTotals(const Block & totals, const Block & columns_to_add, const Names
|
||||
}
|
||||
}
|
||||
|
||||
void addDefaultValues(MutableColumnPtr & column, const DataTypePtr & typ, size_t count)
|
||||
{
|
||||
if (typ->getTypeId() == column->getDataType())
|
||||
{
|
||||
column->insertManyDefaults(count);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto default_val = typ->getDefault();
|
||||
for (size_t i = 0; i < count; ++i)
|
||||
column->insert(default_val);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
@ -387,9 +401,24 @@ void NotJoined::correctLowcardAndNullability(MutableColumns & columns_right)
|
||||
|
||||
void NotJoined::addLeftColumns(Block & block, size_t rows_added) const
|
||||
{
|
||||
/// @note it's possible to make ColumnConst here and materialize it later
|
||||
for (size_t pos : column_indices_left)
|
||||
block.getByPosition(pos).column = block.getByPosition(pos).column->cloneResized(rows_added);
|
||||
{
|
||||
auto & col = block.getByPosition(pos);
|
||||
if (col.column->getDataType() == col.type->getTypeId())
|
||||
{
|
||||
col.column = col.column->cloneResized(rows_added);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto mut_col = col.column->cloneEmpty();
|
||||
mut_col->reserve(rows_added);
|
||||
|
||||
const auto default_value = col.type->getDefault();
|
||||
for (size_t i = 0; i < rows_added; ++i)
|
||||
mut_col->insert(default_value);
|
||||
col.column = std::move(mut_col);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void NotJoined::addRightColumns(Block & block, MutableColumns & columns_right) const
|
||||
|
@ -34,6 +34,8 @@ void checkTypesOfKeys(const Block & block_left, const Names & key_names_left, co
|
||||
void createMissedColumns(Block & block);
|
||||
void joinTotals(const Block & totals, const Block & columns_to_add, const Names & key_names_right, Block & block);
|
||||
|
||||
void addDefaultValues(MutableColumnPtr & column, const DataTypePtr & typ, size_t count);
|
||||
|
||||
}
|
||||
|
||||
/// Creates result from right table data in RIGHT and FULL JOIN when keys are not present in left table.
|
||||
|
36
tests/queries/0_stateless/01656_join_defaul_enum.reference
Normal file
36
tests/queries/0_stateless/01656_join_defaul_enum.reference
Normal file
@ -0,0 +1,36 @@
|
||||
1 First
|
||||
2 Second
|
||||
3 First
|
||||
1 First
|
||||
2 Second
|
||||
3 First
|
||||
4 Second
|
||||
2 Second
|
||||
4 Second
|
||||
2 Second
|
||||
4 Second
|
||||
1 First
|
||||
2 Second
|
||||
3 First
|
||||
1 First
|
||||
2 Second
|
||||
3 First
|
||||
4 Second
|
||||
1 First
|
||||
2 Second
|
||||
3 First
|
||||
1 First
|
||||
2 Second
|
||||
3 First
|
||||
4 Second
|
||||
2 Second
|
||||
4 Second
|
||||
2 Second
|
||||
4 Second
|
||||
1 First
|
||||
2 Second
|
||||
3 First
|
||||
1 First
|
||||
2 Second
|
||||
3 First
|
||||
4 Second
|
40
tests/queries/0_stateless/01656_join_defaul_enum.sql
Normal file
40
tests/queries/0_stateless/01656_join_defaul_enum.sql
Normal file
@ -0,0 +1,40 @@
|
||||
DROP DATABASE IF EXISTS test_01656;
|
||||
|
||||
CREATE DATABASE test_01656;
|
||||
USE test_01656;
|
||||
|
||||
DROP TABLE IF EXISTS table_key;
|
||||
DROP TABLE IF EXISTS table_with_enum;
|
||||
|
||||
CREATE TABLE table_key (keycol UInt16) ENGINE = MergeTree() ORDER BY (keycol) PARTITION BY tuple();
|
||||
|
||||
CREATE TABLE table_with_enum (keycol UInt16, enum_col Enum8('First' = 1,'Second' = 2))
|
||||
ENGINE = MergeTree() ORDER BY (keycol) PARTITION BY tuple();
|
||||
|
||||
INSERT INTO table_key VALUES (1), (2), (3);
|
||||
INSERT INTO table_with_enum VALUES (2, 'Second'), (4, 'Second');
|
||||
|
||||
SET join_algorithm = 'hash';
|
||||
|
||||
SELECT keycol, enum_col FROM table_key LEFT JOIN table_with_enum USING (keycol) ORDER BY keycol;
|
||||
SELECT keycol, enum_col FROM table_key FULL JOIN table_with_enum USING (keycol) ORDER BY keycol;
|
||||
SELECT keycol, enum_col FROM table_key RIGHT JOIN table_with_enum USING (keycol) ORDER BY keycol;
|
||||
|
||||
SELECT keycol, enum_col FROM table_with_enum LEFT JOIN table_key USING (keycol) ORDER BY keycol;
|
||||
SELECT keycol, enum_col FROM table_with_enum RIGHT JOIN table_key USING (keycol) ORDER BY keycol;
|
||||
SELECT keycol, enum_col FROM table_with_enum FULL JOIN table_key USING (keycol) ORDER BY keycol;
|
||||
|
||||
SET join_algorithm = 'partial_merge';
|
||||
|
||||
SELECT keycol, enum_col FROM table_key LEFT JOIN table_with_enum USING (keycol) ORDER BY keycol;
|
||||
SELECT keycol, enum_col FROM table_key FULL JOIN table_with_enum USING (keycol) ORDER BY keycol;
|
||||
SELECT keycol, enum_col FROM table_key RIGHT JOIN table_with_enum USING (keycol) ORDER BY keycol;
|
||||
|
||||
SELECT keycol, enum_col FROM table_with_enum LEFT JOIN table_key USING (keycol) ORDER BY keycol;
|
||||
SELECT keycol, enum_col FROM table_with_enum RIGHT JOIN table_key USING (keycol) ORDER BY keycol;
|
||||
SELECT keycol, enum_col FROM table_with_enum FULL JOIN table_key USING (keycol) ORDER BY keycol;
|
||||
|
||||
DROP TABLE IF EXISTS table_key;
|
||||
DROP TABLE IF EXISTS table_with_enum;
|
||||
|
||||
DROP DATABASE IF EXISTS test_01656;
|
Loading…
Reference in New Issue
Block a user