mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Added 'join_use_nulls' setting to allow using NULLs as default values for non-joined rows in case of outer joins [#CLICKHOUSE-2].
This commit is contained in:
parent
b87c7936ed
commit
baf2e6e95b
@ -2238,7 +2238,8 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
|
||||
if (!subquery_for_set.join)
|
||||
{
|
||||
JoinPtr join = std::make_shared<Join>(
|
||||
join_key_names_left, join_key_names_right, settings.limits,
|
||||
join_key_names_left, join_key_names_right,
|
||||
settings.join_use_nulls, settings.limits,
|
||||
join_params.kind, join_params.strictness);
|
||||
|
||||
Names required_joined_columns(join_key_names_right.begin(), join_key_names_right.end());
|
||||
|
@ -2,6 +2,9 @@
|
||||
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
|
||||
#include <Interpreters/Join.h>
|
||||
#include <Interpreters/NullableUtils.h>
|
||||
@ -23,11 +26,12 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
Join::Join(const Names & key_names_left_, const Names & key_names_right_,
|
||||
Join::Join(const Names & key_names_left_, const Names & key_names_right_, bool use_nulls_,
|
||||
const Limits & limits, ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_)
|
||||
: kind(kind_), strictness(strictness_),
|
||||
key_names_left(key_names_left_),
|
||||
key_names_right(key_names_right_),
|
||||
use_nulls(use_nulls_),
|
||||
log(&Logger::get("Join")),
|
||||
max_rows(limits.max_rows_in_join),
|
||||
max_bytes(limits.max_bytes_in_join),
|
||||
@ -241,6 +245,18 @@ bool Join::checkSizeLimits() const
|
||||
}
|
||||
|
||||
|
||||
static void convertColumnToNullable(ColumnWithTypeAndName & column)
|
||||
{
|
||||
if (column.type->isNullable() || column.type->isNull())
|
||||
return;
|
||||
|
||||
column.type = std::make_shared<DataTypeNullable>(column.type);
|
||||
|
||||
column.column = std::make_shared<ColumnNullable>(column.column,
|
||||
std::make_shared<ColumnConstUInt8>(column.column->size(), 0)->convertToFullColumn());
|
||||
}
|
||||
|
||||
|
||||
void Join::setSampleBlock(const Block & block)
|
||||
{
|
||||
Poco::ScopedWriteRWLock lock(rwlock);
|
||||
@ -273,12 +289,19 @@ void Join::setSampleBlock(const Block & block)
|
||||
++pos;
|
||||
}
|
||||
|
||||
for (size_t i = 0, size = sample_block_with_columns_to_add.columns(); i < size; ++i)
|
||||
size_t num_columns_to_add = sample_block_with_columns_to_add.columns();
|
||||
|
||||
for (size_t i = 0; i < num_columns_to_add; ++i)
|
||||
{
|
||||
auto & column = sample_block_with_columns_to_add.getByPosition(i);
|
||||
if (!column.column)
|
||||
column.column = column.type->createColumn();
|
||||
}
|
||||
|
||||
/// In case of LEFT and FULL joins, if use_nulls, convert joined columns to Nullable.
|
||||
if (use_nulls && (kind == ASTTableJoin::Kind::Left || kind == ASTTableJoin::Kind::Full))
|
||||
for (size_t i = 0; i < num_columns_to_add; ++i)
|
||||
convertColumnToNullable(sample_block_with_keys.getByPosition(i));
|
||||
}
|
||||
|
||||
|
||||
@ -451,14 +474,25 @@ bool Join::insertFromBlock(const Block & block)
|
||||
stored_block->erase(stored_block->getPositionByName(name));
|
||||
}
|
||||
|
||||
size_t size = stored_block->columns();
|
||||
|
||||
/// Rare case, when joined columns are constant. To avoid code bloat, simply materialize them.
|
||||
for (size_t i = 0, size = stored_block->columns(); i < size; ++i)
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
ColumnPtr col = stored_block->safeGetByPosition(i).column;
|
||||
if (auto converted = col->convertToFullColumnIfConst())
|
||||
stored_block->safeGetByPosition(i).column = converted;
|
||||
}
|
||||
|
||||
/// In case of LEFT and FULL joins, if use_nulls, convert joined columns to Nullable.
|
||||
if (use_nulls && (kind == ASTTableJoin::Kind::Left || kind == ASTTableJoin::Kind::Full))
|
||||
{
|
||||
for (size_t i = getFullness(kind) ? keys_size : 0; i < size; ++i)
|
||||
{
|
||||
convertColumnToNullable(stored_block->getByPosition(i));
|
||||
}
|
||||
}
|
||||
|
||||
if (kind != ASTTableJoin::Kind::Cross)
|
||||
{
|
||||
/// Fill the hash table.
|
||||
@ -947,6 +981,11 @@ public:
|
||||
for (size_t i = 0; i < num_columns_right; ++i)
|
||||
column_numbers_keys_and_right.push_back(num_keys + num_columns_left + i);
|
||||
|
||||
/// If use_nulls, convert left columns to Nullable.
|
||||
if (parent.use_nulls)
|
||||
for (size_t i = 0; i < num_columns_left; ++i)
|
||||
convertColumnToNullable(result_sample_block.getByPosition(column_numbers_left[i]));
|
||||
|
||||
columns_left.resize(num_columns_left);
|
||||
columns_keys_and_right.resize(num_keys + num_columns_right);
|
||||
}
|
||||
|
@ -210,11 +210,19 @@ struct Limits;
|
||||
* NULLs never join to anything, even to each other.
|
||||
* During building of map, we just skip keys with NULL value of any component.
|
||||
* During joining, we simply treat rows with any NULLs in key as non joined.
|
||||
*
|
||||
* Default values for outer joins (LEFT, RIGHT, FULL):
|
||||
*
|
||||
* Behaviour is controlled by 'join_use_nulls' settings.
|
||||
* If it is false, we substitute (global) default value for the data type, for non-joined rows
|
||||
* (zero, empty string, etc. and NULL for Nullable data types).
|
||||
* If it is true, we always generate Nullable column and substitute NULLs for non-joined rows,
|
||||
* as in standard SQL.
|
||||
*/
|
||||
class Join
|
||||
{
|
||||
public:
|
||||
Join(const Names & key_names_left_, const Names & key_names_right_,
|
||||
Join(const Names & key_names_left_, const Names & key_names_right_, bool use_nulls_,
|
||||
const Limits & limits, ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_);
|
||||
|
||||
bool empty() { return type == Type::EMPTY; }
|
||||
@ -244,6 +252,7 @@ public:
|
||||
/** For RIGHT and FULL JOINs.
|
||||
* A stream that will contain default values from left table, joined with rows from right table, that was not joined before.
|
||||
* 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).
|
||||
*/
|
||||
BlockInputStreamPtr createStreamWithNonJoinedRows(Block & left_sample_block, size_t max_block_size) const;
|
||||
|
||||
@ -356,6 +365,9 @@ private:
|
||||
/// Names of key columns (columns for equi-JOIN) in "right" table.
|
||||
const Names key_names_right;
|
||||
|
||||
/// Substitute NULLs for non-JOINed rows.
|
||||
bool use_nulls;
|
||||
|
||||
/** Blocks of "right" table.
|
||||
*/
|
||||
BlocksList blocks;
|
||||
|
@ -258,7 +258,13 @@ struct Settings
|
||||
* will skip until next line and continue. \
|
||||
*/ \
|
||||
M(SettingUInt64, input_format_allow_errors_num, 0) \
|
||||
M(SettingFloat, input_format_allow_errors_ratio, 0)
|
||||
M(SettingFloat, input_format_allow_errors_ratio, 0) \
|
||||
\
|
||||
/** Use NULLs for non-joined rows of outer JOINs. \
|
||||
* If false, use default value of corresponding columns data type. \
|
||||
*/ \
|
||||
M(SettingBool, join_use_nulls, 0) \
|
||||
|
||||
|
||||
/// Possible limits for query execution.
|
||||
Limits limits;
|
||||
|
@ -40,7 +40,9 @@ StorageJoin::StorageJoin(
|
||||
"Key column (" + key + ") does not exist in table declaration.",
|
||||
ErrorCodes::NO_SUCH_COLUMN_IN_TABLE};
|
||||
|
||||
join = std::make_shared<Join>(key_names, key_names, Limits(), kind, strictness);
|
||||
/// NOTE StorageJoin doesn't use join_use_nulls setting.
|
||||
|
||||
join = std::make_shared<Join>(key_names, key_names, false /* use_nulls */, Limits(), kind, strictness);
|
||||
join->setSampleBlock(getSampleBlock());
|
||||
restore();
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user