2019-07-04 12:12:39 +00:00
|
|
|
#include <any>
|
2020-04-21 19:01:34 +00:00
|
|
|
#include <limits>
|
2021-06-25 12:03:10 +00:00
|
|
|
#include <numeric>
|
2019-07-04 12:12:39 +00:00
|
|
|
|
2017-01-21 04:24:28 +00:00
|
|
|
#include <common/logger_useful.h>
|
|
|
|
|
2017-07-21 06:35:58 +00:00
|
|
|
#include <Columns/ColumnConst.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Columns/ColumnString.h>
|
2020-04-08 18:59:52 +00:00
|
|
|
#include <Columns/ColumnVector.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Columns/ColumnFixedString.h>
|
2017-04-04 06:52:39 +00:00
|
|
|
#include <Columns/ColumnNullable.h>
|
|
|
|
|
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
2020-04-08 18:59:52 +00:00
|
|
|
#include <DataTypes/DataTypeLowCardinality.h>
|
2014-06-12 02:31:30 +00:00
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
#include <Interpreters/HashJoin.h>
|
2019-09-18 12:46:57 +00:00
|
|
|
#include <Interpreters/join_common.h>
|
2020-04-07 09:48:47 +00:00
|
|
|
#include <Interpreters/TableJoin.h>
|
2019-07-02 14:38:31 +00:00
|
|
|
#include <Interpreters/joinDispatch.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/NullableUtils.h>
|
2020-04-08 18:59:52 +00:00
|
|
|
#include <Interpreters/DictionaryReader.h>
|
|
|
|
|
|
|
|
#include <Storages/StorageDictionary.h>
|
2017-03-30 14:09:24 +00:00
|
|
|
|
2018-02-19 21:01:46 +00:00
|
|
|
#include <DataStreams/materializeBlock.h>
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/ColumnNumbers.h>
|
2017-07-13 20:58:19 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2019-08-21 02:28:04 +00:00
|
|
|
#include <Common/assert_cast.h>
|
2014-06-12 02:31:30 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
2014-06-13 02:05:05 +00:00
|
|
|
|
2016-01-12 02:21:15 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2020-02-25 18:02:41 +00:00
|
|
|
extern const int NOT_IMPLEMENTED;
|
2020-06-02 20:26:41 +00:00
|
|
|
extern const int NO_SUCH_COLUMN_IN_TABLE;
|
|
|
|
extern const int INCOMPATIBLE_TYPE_OF_JOIN;
|
2019-08-05 14:03:14 +00:00
|
|
|
extern const int UNSUPPORTED_JOIN_KEYS;
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2020-07-16 08:56:45 +00:00
|
|
|
extern const int SYNTAX_ERROR;
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int SET_SIZE_LIMIT_EXCEEDED;
|
|
|
|
extern const int TYPE_MISMATCH;
|
2020-07-11 07:12:42 +00:00
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
2016-01-12 02:21:15 +00:00
|
|
|
}
|
|
|
|
|
2020-04-06 13:39:57 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
|
|
|
struct NotProcessedCrossJoin : public ExtraBlock
|
|
|
|
{
|
|
|
|
size_t left_position;
|
|
|
|
size_t right_block;
|
|
|
|
};
|
|
|
|
|
|
|
|
}
|
2019-03-27 13:37:11 +00:00
|
|
|
|
2021-02-04 14:46:36 +00:00
|
|
|
namespace JoinStuff
|
|
|
|
{
|
2021-02-08 11:38:31 +00:00
|
|
|
/// Version of `getUsed` with dynamic dispatch
|
2021-02-04 14:46:36 +00:00
|
|
|
bool JoinUsedFlags::getUsedSafe(size_t i) const
|
|
|
|
{
|
|
|
|
if (flags.empty())
|
|
|
|
return !need_flags;
|
|
|
|
return flags[i].load();
|
|
|
|
}
|
|
|
|
|
|
|
|
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS>
|
|
|
|
void JoinUsedFlags::reinit(size_t size)
|
|
|
|
{
|
|
|
|
if constexpr (MapGetter<KIND, STRICTNESS>::flagged)
|
|
|
|
{
|
|
|
|
assert(flags.size() <= size);
|
|
|
|
need_flags = true;
|
|
|
|
flags = std::vector<std::atomic_bool>(size);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
template <bool use_flags, bool multiple_disjuncts, typename FindResult>
|
|
|
|
void JoinUsedFlags::setUsed(const FindResult & f)
|
2021-02-04 14:46:36 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
if constexpr (use_flags)
|
|
|
|
{
|
|
|
|
/// Could be set simultaneously from different threads.
|
|
|
|
if constexpr (!multiple_disjuncts)
|
|
|
|
{
|
|
|
|
flags[f.getOffset()].store(true, std::memory_order_relaxed);
|
|
|
|
}
|
|
|
|
}
|
2021-02-04 14:46:36 +00:00
|
|
|
}
|
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
template <bool use_flags, bool multiple_disjuncts, typename FindResult>
|
|
|
|
bool JoinUsedFlags::getUsed(const FindResult & f)
|
|
|
|
{
|
|
|
|
if constexpr (use_flags)
|
|
|
|
{
|
|
|
|
return flags[f.getOffset()].load();
|
|
|
|
}
|
|
|
|
return true;
|
|
|
|
}
|
2021-02-04 14:46:36 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
template <bool use_flags, bool multiple_disjuncts, typename FindResult>
|
|
|
|
bool JoinUsedFlags::setUsedOnce(const FindResult & f)
|
2021-02-04 14:46:36 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
if constexpr (use_flags)
|
|
|
|
{
|
|
|
|
size_t off = f.getOffset();
|
|
|
|
/// fast check to prevent heavy CAS with seq_cst order
|
|
|
|
if (flags[off].load(std::memory_order_relaxed))
|
|
|
|
return false;
|
2021-02-04 14:46:36 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
bool expected = false;
|
|
|
|
return flags[off].compare_exchange_strong(expected, true);
|
|
|
|
}
|
|
|
|
return true;
|
2021-02-04 14:46:36 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-10-29 19:39:42 +00:00
|
|
|
static ColumnPtr filterWithBlanks(ColumnPtr src_column, const IColumn::Filter & filter, bool inverse_filter = false)
|
|
|
|
{
|
|
|
|
ColumnPtr column = src_column->convertToFullColumnIfConst();
|
|
|
|
MutableColumnPtr mut_column = column->cloneEmpty();
|
|
|
|
mut_column->reserve(column->size());
|
|
|
|
|
|
|
|
if (inverse_filter)
|
|
|
|
{
|
|
|
|
for (size_t row = 0; row < filter.size(); ++row)
|
|
|
|
{
|
|
|
|
if (filter[row])
|
|
|
|
mut_column->insertDefault();
|
|
|
|
else
|
|
|
|
mut_column->insertFrom(*column, row);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
for (size_t row = 0; row < filter.size(); ++row)
|
|
|
|
{
|
|
|
|
if (filter[row])
|
|
|
|
mut_column->insertFrom(*column, row);
|
|
|
|
else
|
|
|
|
mut_column->insertDefault();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return mut_column;
|
|
|
|
}
|
|
|
|
|
2019-03-27 13:37:11 +00:00
|
|
|
static ColumnWithTypeAndName correctNullability(ColumnWithTypeAndName && column, bool nullable)
|
|
|
|
{
|
|
|
|
if (nullable)
|
2019-10-29 19:39:42 +00:00
|
|
|
{
|
2019-09-11 18:03:21 +00:00
|
|
|
JoinCommon::convertColumnToNullable(column);
|
2019-10-29 19:39:42 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
/// We have to replace values masked by NULLs with defaults.
|
|
|
|
if (column.column)
|
2020-04-22 06:01:33 +00:00
|
|
|
if (const auto * nullable_column = checkAndGetColumn<ColumnNullable>(*column.column))
|
2019-10-29 19:39:42 +00:00
|
|
|
column.column = filterWithBlanks(column.column, nullable_column->getNullMapColumn().getData(), true);
|
|
|
|
|
|
|
|
JoinCommon::removeColumnNullability(column);
|
|
|
|
}
|
|
|
|
|
2019-03-27 13:37:11 +00:00
|
|
|
return std::move(column);
|
|
|
|
}
|
|
|
|
|
|
|
|
static ColumnWithTypeAndName correctNullability(ColumnWithTypeAndName && column, bool nullable, const ColumnUInt8 & negative_null_map)
|
|
|
|
{
|
|
|
|
if (nullable)
|
|
|
|
{
|
2021-05-15 14:09:51 +00:00
|
|
|
JoinCommon::convertColumnToNullable(column);
|
2020-03-09 00:28:05 +00:00
|
|
|
if (column.type->isNullable() && !negative_null_map.empty())
|
2019-03-27 13:37:11 +00:00
|
|
|
{
|
2020-05-14 08:30:18 +00:00
|
|
|
MutableColumnPtr mutable_column = IColumn::mutate(std::move(column.column));
|
2019-08-21 02:28:04 +00:00
|
|
|
assert_cast<ColumnNullable &>(*mutable_column).applyNegatedNullMap(negative_null_map);
|
2019-03-27 13:37:11 +00:00
|
|
|
column.column = std::move(mutable_column);
|
|
|
|
}
|
|
|
|
}
|
2019-10-29 19:39:42 +00:00
|
|
|
else
|
|
|
|
JoinCommon::removeColumnNullability(column);
|
|
|
|
|
2019-03-27 13:37:11 +00:00
|
|
|
return std::move(column);
|
|
|
|
}
|
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
static std::string formatKeysDebug(const NamesVector & key_names)
|
|
|
|
{
|
|
|
|
std::vector<std::string> res;
|
|
|
|
for (const auto & keys : key_names)
|
|
|
|
res.emplace_back(fmt::format("{}", fmt::join(keys, ", ")));
|
|
|
|
return fmt::format("{}", fmt::join(res, " | "));
|
|
|
|
}
|
2019-11-01 17:41:07 +00:00
|
|
|
|
2020-07-10 18:10:06 +00:00
|
|
|
HashJoin::HashJoin(std::shared_ptr<TableJoin> table_join_, const Block & right_sample_block_, bool any_take_last_row_)
|
|
|
|
: table_join(table_join_)
|
|
|
|
, kind(table_join->kind())
|
|
|
|
, strictness(table_join->strictness())
|
|
|
|
, key_names_right(table_join->keyNamesRight())
|
2021-06-25 12:03:10 +00:00
|
|
|
, key_names_left(table_join->keyNamesLeft())
|
2020-07-10 18:10:06 +00:00
|
|
|
, nullable_right_side(table_join->forceNullableRight())
|
|
|
|
, nullable_left_side(table_join->forceNullableLeft())
|
|
|
|
, any_take_last_row(any_take_last_row_)
|
|
|
|
, asof_inequality(table_join->getAsofInequality())
|
|
|
|
, data(std::make_shared<RightTableData>())
|
|
|
|
, right_sample_block(right_sample_block_)
|
|
|
|
, log(&Poco::Logger::get("HashJoin"))
|
2020-03-13 08:15:43 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
bool multiple_disjuncts = key_names_right.size() > 1;
|
2020-03-13 08:15:43 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
if (multiple_disjuncts)
|
|
|
|
{
|
|
|
|
// required_right_keys_sources concept does not work well if multiple disjuncts
|
|
|
|
sample_block_with_columns_to_add = right_table_keys = materializeBlock(right_sample_block);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
JoinCommon::splitAdditionalColumns(key_names_right, right_sample_block, right_table_keys, sample_block_with_columns_to_add);
|
|
|
|
required_right_keys = table_join->getRequiredRightKeys(right_table_keys, required_right_keys_sources);
|
|
|
|
}
|
2020-03-13 08:15:43 +00:00
|
|
|
|
2021-08-18 08:18:33 +00:00
|
|
|
LOG_DEBUG(log, "Right keys: [{}] (required: [{}]), left keys: [{}]",
|
2021-06-25 12:03:10 +00:00
|
|
|
formatKeysDebug(key_names_right),
|
2021-08-18 08:18:33 +00:00
|
|
|
fmt::join(required_right_keys.getNames(), ", "),
|
2021-06-25 12:03:10 +00:00
|
|
|
formatKeysDebug(key_names_left));
|
2021-08-18 08:18:33 +00:00
|
|
|
|
|
|
|
LOG_DEBUG(log, "Columns to add: [{}]", sample_block_with_columns_to_add.dumpStructure());
|
|
|
|
|
2020-07-10 18:10:06 +00:00
|
|
|
JoinCommon::removeLowCardinalityInplace(right_table_keys);
|
2021-06-25 12:03:10 +00:00
|
|
|
key_sizes.resize(key_names_right.size());
|
|
|
|
|
|
|
|
Type join_method = Type::EMPTY;
|
|
|
|
|
2020-07-10 18:10:06 +00:00
|
|
|
initRightBlockStructure(data->sample_block);
|
2020-03-13 08:15:43 +00:00
|
|
|
|
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
const size_t disjuncts_num = key_names_right.size();
|
2020-07-10 18:10:06 +00:00
|
|
|
JoinCommon::createMissedColumns(sample_block_with_columns_to_add);
|
2020-03-13 08:15:43 +00:00
|
|
|
|
2021-06-29 09:22:53 +00:00
|
|
|
if (table_join->getDictionaryReader())
|
2020-03-13 08:15:43 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
data->maps.resize(disjuncts_num);
|
2020-03-13 08:15:43 +00:00
|
|
|
}
|
2021-06-25 12:03:10 +00:00
|
|
|
condition_mask_column_name_left.resize(disjuncts_num);
|
|
|
|
condition_mask_column_name_right.resize(disjuncts_num);
|
|
|
|
|
|
|
|
if (nullable_right_side)
|
2020-03-13 08:15:43 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
JoinCommon::convertColumnsToNullable(sample_block_with_columns_to_add);
|
|
|
|
}
|
2020-07-10 18:10:06 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
for (size_t d = 0; d < disjuncts_num; ++d)
|
|
|
|
{
|
|
|
|
std::tie(condition_mask_column_name_left[d], condition_mask_column_name_right[d]) = table_join->joinConditionColumnNames(d);
|
|
|
|
ColumnRawPtrs key_columns = JoinCommon::extractKeysForJoin(right_table_keys, key_names_right[d]);
|
2020-07-10 18:10:06 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
if (table_join->dictionary_reader)
|
|
|
|
{
|
|
|
|
LOG_DEBUG(log, "Performing join over dict");
|
|
|
|
join_method = Type::DICT;
|
2020-03-13 08:15:43 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
std::get<MapsOne>(data->maps[d]).create(Type::DICT);
|
|
|
|
chooseMethod(key_columns, key_sizes[d]); /// init key_sizes
|
|
|
|
continue; // break ?
|
|
|
|
}
|
|
|
|
else if (strictness == ASTTableJoin::Strictness::Asof)
|
|
|
|
{
|
|
|
|
/// @note ASOF JOIN is not INNER. It's better avoid use of 'INNER ASOF' combination in messages.
|
|
|
|
/// In fact INNER means 'LEFT SEMI ASOF' while LEFT means 'LEFT OUTER ASOF'.
|
|
|
|
if (!isLeft(kind) && !isInner(kind))
|
|
|
|
throw Exception("Wrong ASOF JOIN type. Only ASOF and LEFT ASOF joins are supported", ErrorCodes::NOT_IMPLEMENTED);
|
|
|
|
|
|
|
|
if (key_columns.size() <= 1)
|
|
|
|
throw Exception("ASOF join needs at least one equi-join column", ErrorCodes::SYNTAX_ERROR);
|
|
|
|
|
|
|
|
if (right_table_keys.getByName(key_names_right[0].back()).type->isNullable())
|
|
|
|
throw Exception("ASOF join over right table Nullable column is not implemented", ErrorCodes::NOT_IMPLEMENTED);
|
|
|
|
|
|
|
|
size_t asof_size;
|
|
|
|
asof_type = AsofRowRefs::getTypeSize(*key_columns.back(), asof_size);
|
|
|
|
key_columns.pop_back();
|
|
|
|
|
|
|
|
/// this is going to set up the appropriate hash table for the direct lookup part of the join
|
|
|
|
/// However, this does not depend on the size of the asof join key (as that goes into the BST)
|
|
|
|
/// Therefore, add it back in such that it can be extracted appropriately from the full stored
|
|
|
|
/// key_columns and key_sizes
|
|
|
|
key_sizes[d].push_back(asof_size);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
/// Choose data structure to use for JOIN.
|
|
|
|
}
|
2014-06-18 20:08:31 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
auto current_join_method = chooseMethod(key_columns, key_sizes[d]);
|
|
|
|
if (join_method == Type::EMPTY)
|
|
|
|
{
|
|
|
|
join_method = current_join_method;
|
|
|
|
}
|
|
|
|
else if (join_method != current_join_method)
|
|
|
|
{
|
|
|
|
join_method = Type::hashed;
|
|
|
|
}
|
2020-07-10 18:10:06 +00:00
|
|
|
}
|
2021-06-25 12:03:10 +00:00
|
|
|
|
|
|
|
data->type = join_method;
|
|
|
|
if (join_method != Type::DICT)
|
2020-07-10 18:10:06 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
data->maps.resize(key_names_right.size());
|
|
|
|
|
|
|
|
for (size_t d = 0; d < disjuncts_num; ++d)
|
|
|
|
{
|
|
|
|
data_map_init(data->maps[d]);
|
|
|
|
}
|
2020-07-10 18:10:06 +00:00
|
|
|
}
|
|
|
|
}
|
2017-01-21 04:24:28 +00:00
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
HashJoin::Type HashJoin::chooseMethod(const ColumnRawPtrs & key_columns, Sizes & key_sizes)
|
2015-03-02 01:10:58 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t keys_size = key_columns.size();
|
|
|
|
|
|
|
|
if (keys_size == 0)
|
|
|
|
return Type::CROSS;
|
|
|
|
|
|
|
|
bool all_fixed = true;
|
|
|
|
size_t keys_bytes = 0;
|
|
|
|
key_sizes.resize(keys_size);
|
|
|
|
for (size_t j = 0; j < keys_size; ++j)
|
|
|
|
{
|
2017-12-09 10:14:45 +00:00
|
|
|
if (!key_columns[j]->isFixedAndContiguous())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
all_fixed = false;
|
|
|
|
break;
|
|
|
|
}
|
2017-12-09 10:14:45 +00:00
|
|
|
key_sizes[j] = key_columns[j]->sizeOfValueIfFixed();
|
2017-04-01 07:20:54 +00:00
|
|
|
keys_bytes += key_sizes[j];
|
|
|
|
}
|
|
|
|
|
2017-04-02 17:37:49 +00:00
|
|
|
/// If there is one numeric key that fits in 64 bits
|
2017-12-09 10:14:45 +00:00
|
|
|
if (keys_size == 1 && key_columns[0]->isNumeric())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-12-09 10:14:45 +00:00
|
|
|
size_t size_of_field = key_columns[0]->sizeOfValueIfFixed();
|
2017-04-01 07:20:54 +00:00
|
|
|
if (size_of_field == 1)
|
|
|
|
return Type::key8;
|
|
|
|
if (size_of_field == 2)
|
|
|
|
return Type::key16;
|
|
|
|
if (size_of_field == 4)
|
|
|
|
return Type::key32;
|
|
|
|
if (size_of_field == 8)
|
|
|
|
return Type::key64;
|
2017-11-09 23:37:57 +00:00
|
|
|
if (size_of_field == 16)
|
|
|
|
return Type::keys128;
|
2020-09-04 13:33:02 +00:00
|
|
|
if (size_of_field == 32)
|
|
|
|
return Type::keys256;
|
|
|
|
throw Exception("Logical error: numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32.", ErrorCodes::LOGICAL_ERROR);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2017-04-02 17:37:49 +00:00
|
|
|
/// If the keys fit in N bits, we will use a hash table for N-bit-packed keys
|
2017-04-01 07:20:54 +00:00
|
|
|
if (all_fixed && keys_bytes <= 16)
|
|
|
|
return Type::keys128;
|
|
|
|
if (all_fixed && keys_bytes <= 32)
|
|
|
|
return Type::keys256;
|
|
|
|
|
|
|
|
/// If there is single string key, use hash table of it's values.
|
2017-07-21 06:35:58 +00:00
|
|
|
if (keys_size == 1
|
|
|
|
&& (typeid_cast<const ColumnString *>(key_columns[0])
|
2019-08-21 02:28:04 +00:00
|
|
|
|| (isColumnConst(*key_columns[0]) && typeid_cast<const ColumnString *>(&assert_cast<const ColumnConst *>(key_columns[0])->getDataColumn()))))
|
2017-04-01 07:20:54 +00:00
|
|
|
return Type::key_string;
|
|
|
|
|
|
|
|
if (keys_size == 1 && typeid_cast<const ColumnFixedString *>(key_columns[0]))
|
|
|
|
return Type::key_fixed_string;
|
|
|
|
|
|
|
|
/// Otherwise, will use set of cryptographic hashes of unambiguously serialized values.
|
|
|
|
return Type::hashed;
|
2015-03-02 01:10:58 +00:00
|
|
|
}
|
|
|
|
|
2019-12-02 18:07:27 +00:00
|
|
|
template<typename KeyGetter, bool is_asof_join>
|
2019-03-28 18:35:50 +00:00
|
|
|
static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes)
|
2019-03-26 22:05:51 +00:00
|
|
|
{
|
2019-12-02 18:07:27 +00:00
|
|
|
if constexpr (is_asof_join)
|
2019-03-26 22:05:51 +00:00
|
|
|
{
|
2019-03-28 18:35:50 +00:00
|
|
|
auto key_column_copy = key_columns;
|
|
|
|
auto key_size_copy = key_sizes;
|
|
|
|
key_column_copy.pop_back();
|
|
|
|
key_size_copy.pop_back();
|
|
|
|
return KeyGetter(key_column_copy, key_size_copy, nullptr);
|
2019-03-20 16:58:28 +00:00
|
|
|
}
|
2019-03-28 18:35:50 +00:00
|
|
|
else
|
|
|
|
return KeyGetter(key_columns, key_sizes, nullptr);
|
2019-03-20 16:58:28 +00:00
|
|
|
}
|
|
|
|
|
2020-04-08 18:59:52 +00:00
|
|
|
class KeyGetterForDict
|
|
|
|
{
|
|
|
|
public:
|
2021-02-04 14:46:36 +00:00
|
|
|
using Mapped = RowRef;
|
|
|
|
using FindResult = ColumnsHashing::columns_hashing_impl::FindResultImpl<Mapped, true>;
|
2020-04-08 18:59:52 +00:00
|
|
|
|
2021-06-23 10:13:11 +00:00
|
|
|
KeyGetterForDict(const TableJoin & table_join, const ColumnRawPtrs & key_columns)
|
2020-04-08 18:59:52 +00:00
|
|
|
{
|
2021-06-29 09:22:53 +00:00
|
|
|
assert(table_join.getDictionaryReader());
|
|
|
|
table_join.getDictionaryReader()->readKeys(*key_columns[0], read_result, found, positions);
|
2020-04-08 18:59:52 +00:00
|
|
|
|
2021-06-23 10:13:11 +00:00
|
|
|
for (ColumnWithTypeAndName & column : read_result)
|
|
|
|
if (table_join.rightBecomeNullable(column.type))
|
|
|
|
JoinCommon::convertColumnToNullable(column);
|
|
|
|
}
|
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
FindResult findKey(const TableJoin & /* void * */, size_t row, const Arena &)
|
2021-06-23 10:13:11 +00:00
|
|
|
{
|
|
|
|
result.block = &read_result;
|
2020-04-13 17:03:11 +00:00
|
|
|
result.row_num = positions[row];
|
2021-02-04 14:46:36 +00:00
|
|
|
return FindResult(&result, found[row], 0);
|
2020-04-08 18:59:52 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
|
|
|
Block read_result;
|
|
|
|
Mapped result;
|
|
|
|
ColumnVector<UInt8>::Container found;
|
|
|
|
std::vector<size_t> positions;
|
|
|
|
};
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
template <HashJoin::Type type, typename Value, typename Mapped>
|
2019-01-24 14:56:04 +00:00
|
|
|
struct KeyGetterForTypeImpl;
|
2017-03-28 06:51:22 +00:00
|
|
|
|
2021-02-04 14:46:36 +00:00
|
|
|
constexpr bool use_offset = true;
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<HashJoin::Type::key8, Value, Mapped>
|
2019-01-24 14:56:04 +00:00
|
|
|
{
|
2021-02-04 14:46:36 +00:00
|
|
|
using Type = ColumnsHashing::HashMethodOneNumber<Value, Mapped, UInt8, false, use_offset>;
|
2019-01-24 14:56:04 +00:00
|
|
|
};
|
2020-04-07 09:48:47 +00:00
|
|
|
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<HashJoin::Type::key16, Value, Mapped>
|
2019-01-24 14:56:04 +00:00
|
|
|
{
|
2021-02-04 14:46:36 +00:00
|
|
|
using Type = ColumnsHashing::HashMethodOneNumber<Value, Mapped, UInt16, false, use_offset>;
|
2019-01-24 14:56:04 +00:00
|
|
|
};
|
2020-04-07 09:48:47 +00:00
|
|
|
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<HashJoin::Type::key32, Value, Mapped>
|
2019-01-24 14:56:04 +00:00
|
|
|
{
|
2021-02-04 14:46:36 +00:00
|
|
|
using Type = ColumnsHashing::HashMethodOneNumber<Value, Mapped, UInt32, false, use_offset>;
|
2019-01-24 14:56:04 +00:00
|
|
|
};
|
2020-04-07 09:48:47 +00:00
|
|
|
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<HashJoin::Type::key64, Value, Mapped>
|
2019-01-24 14:56:04 +00:00
|
|
|
{
|
2021-02-04 14:46:36 +00:00
|
|
|
using Type = ColumnsHashing::HashMethodOneNumber<Value, Mapped, UInt64, false, use_offset>;
|
2019-01-24 14:56:04 +00:00
|
|
|
};
|
2020-04-07 09:48:47 +00:00
|
|
|
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<HashJoin::Type::key_string, Value, Mapped>
|
2019-01-24 14:56:04 +00:00
|
|
|
{
|
2021-02-04 14:46:36 +00:00
|
|
|
using Type = ColumnsHashing::HashMethodString<Value, Mapped, true, false, use_offset>;
|
2019-01-24 14:56:04 +00:00
|
|
|
};
|
2020-04-07 09:48:47 +00:00
|
|
|
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<HashJoin::Type::key_fixed_string, Value, Mapped>
|
2019-01-24 14:56:04 +00:00
|
|
|
{
|
2021-02-04 14:46:36 +00:00
|
|
|
using Type = ColumnsHashing::HashMethodFixedString<Value, Mapped, true, false, use_offset>;
|
2019-01-24 14:56:04 +00:00
|
|
|
};
|
2020-04-07 09:48:47 +00:00
|
|
|
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<HashJoin::Type::keys128, Value, Mapped>
|
2019-01-24 14:56:04 +00:00
|
|
|
{
|
2021-02-04 14:46:36 +00:00
|
|
|
using Type = ColumnsHashing::HashMethodKeysFixed<Value, UInt128, Mapped, false, false, false, use_offset>;
|
2019-01-24 14:56:04 +00:00
|
|
|
};
|
2020-04-07 09:48:47 +00:00
|
|
|
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<HashJoin::Type::keys256, Value, Mapped>
|
2019-01-24 14:56:04 +00:00
|
|
|
{
|
2021-04-25 09:30:43 +00:00
|
|
|
using Type = ColumnsHashing::HashMethodKeysFixed<Value, UInt256, Mapped, false, false, false, use_offset>;
|
2019-01-24 14:56:04 +00:00
|
|
|
};
|
2020-04-07 09:48:47 +00:00
|
|
|
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<HashJoin::Type::hashed, Value, Mapped>
|
2019-01-24 14:56:04 +00:00
|
|
|
{
|
2021-02-04 14:46:36 +00:00
|
|
|
using Type = ColumnsHashing::HashMethodHashed<Value, Mapped, false, use_offset>;
|
2019-01-24 14:56:04 +00:00
|
|
|
};
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
template <HashJoin::Type type, typename Data>
|
2019-01-24 14:56:04 +00:00
|
|
|
struct KeyGetterForType
|
|
|
|
{
|
|
|
|
using Value = typename Data::value_type;
|
|
|
|
using Mapped_t = typename Data::mapped_type;
|
|
|
|
using Mapped = std::conditional_t<std::is_const_v<Data>, const Mapped_t, Mapped_t>;
|
|
|
|
using Type = typename KeyGetterForTypeImpl<type, Value, Mapped>::Type;
|
|
|
|
};
|
2017-03-28 06:51:22 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
void HashJoin::data_map_init(MapsVariant & map)
|
2014-06-18 20:08:31 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (kind == ASTTableJoin::Kind::Cross)
|
|
|
|
return;
|
2021-06-25 12:03:10 +00:00
|
|
|
joinDispatchInit(kind, strictness, map);
|
|
|
|
joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { map_.create(data->type); });
|
2014-06-18 20:08:31 +00:00
|
|
|
}
|
|
|
|
|
2021-01-04 23:49:31 +00:00
|
|
|
bool HashJoin::overDictionary() const
|
|
|
|
{
|
|
|
|
return data->type == Type::DICT;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool HashJoin::empty() const
|
|
|
|
{
|
|
|
|
return data->type == Type::EMPTY;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool HashJoin::alwaysReturnsEmptySet() const
|
2020-12-30 13:52:37 +00:00
|
|
|
{
|
2021-01-04 23:49:31 +00:00
|
|
|
return isInnerOrRight(getKind()) && data->empty && !overDictionary();
|
2020-12-30 13:52:37 +00:00
|
|
|
}
|
|
|
|
|
2021-02-24 16:19:04 +00:00
|
|
|
size_t HashJoin::getTotalRowCount() const
|
2020-12-30 13:52:37 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t res = 0;
|
|
|
|
|
2019-12-19 15:50:28 +00:00
|
|
|
if (data->type == Type::CROSS)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-12-19 15:50:28 +00:00
|
|
|
for (const auto & block : data->blocks)
|
2021-06-25 12:03:10 +00:00
|
|
|
res += block.block.rows();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2020-04-08 18:59:52 +00:00
|
|
|
else if (data->type != Type::DICT)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
for (const auto & map : data->maps)
|
|
|
|
{
|
|
|
|
joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { res += map_.getTotalRowCount(data->type); });
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return res;
|
2014-06-18 20:08:31 +00:00
|
|
|
}
|
|
|
|
|
2021-02-24 16:19:04 +00:00
|
|
|
size_t HashJoin::getTotalByteCount() const
|
2014-06-18 20:08:31 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t res = 0;
|
|
|
|
|
2019-12-19 15:50:28 +00:00
|
|
|
if (data->type == Type::CROSS)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-12-19 15:50:28 +00:00
|
|
|
for (const auto & block : data->blocks)
|
2021-06-25 12:03:10 +00:00
|
|
|
res += block.block.bytes();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2020-04-08 18:59:52 +00:00
|
|
|
else if (data->type != Type::DICT)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
for (const auto & map : data->maps)
|
|
|
|
{
|
|
|
|
joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { res += map_.getTotalByteCountImpl(data->type); });
|
|
|
|
}
|
2019-12-19 15:50:28 +00:00
|
|
|
res += data->pool.size();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return res;
|
2014-06-18 20:08:31 +00:00
|
|
|
}
|
|
|
|
|
2017-03-30 14:09:24 +00:00
|
|
|
namespace
|
|
|
|
{
|
2017-04-02 17:37:49 +00:00
|
|
|
/// Inserting an element into a hash table of the form `key -> reference to a string`, which will then be used by JOIN.
|
2017-04-01 07:20:54 +00:00
|
|
|
template <typename Map, typename KeyGetter>
|
2019-12-02 18:07:27 +00:00
|
|
|
struct Inserter
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-04-07 09:48:47 +00:00
|
|
|
static ALWAYS_INLINE void insertOne(const HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i,
|
2019-12-02 18:07:27 +00:00
|
|
|
Arena & pool)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-01-24 14:56:04 +00:00
|
|
|
auto emplace_result = key_getter.emplaceKey(map, i, pool);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-05-07 18:21:44 +00:00
|
|
|
if (emplace_result.isInserted() || join.anyTakeLastRow())
|
2019-01-24 14:56:04 +00:00
|
|
|
new (&emplace_result.getMapped()) typename Map::mapped_type(stored_block, i);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
static ALWAYS_INLINE void insertAll(const HashJoin &, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-01-24 14:56:04 +00:00
|
|
|
auto emplace_result = key_getter.emplaceKey(map, i, pool);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-01-24 14:56:04 +00:00
|
|
|
if (emplace_result.isInserted())
|
|
|
|
new (&emplace_result.getMapped()) typename Map::mapped_type(stored_block, i);
|
2017-04-01 07:20:54 +00:00
|
|
|
else
|
|
|
|
{
|
2019-05-14 14:40:43 +00:00
|
|
|
/// The first element of the list is stored in the value of the hash table, the rest in the pool.
|
2019-05-14 14:39:03 +00:00
|
|
|
emplace_result.getMapped().insert({stored_block, i}, pool);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
static ALWAYS_INLINE void insertAsof(HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool,
|
2020-08-03 23:11:39 +00:00
|
|
|
const IColumn & asof_column)
|
2019-03-20 16:58:28 +00:00
|
|
|
{
|
|
|
|
auto emplace_result = key_getter.emplaceKey(map, i, pool);
|
|
|
|
typename Map::mapped_type * time_series_map = &emplace_result.getMapped();
|
|
|
|
|
2020-08-03 23:11:39 +00:00
|
|
|
TypeIndex asof_type = *join.getAsofType();
|
2019-03-26 22:05:51 +00:00
|
|
|
if (emplace_result.isInserted())
|
2020-08-03 23:11:39 +00:00
|
|
|
time_series_map = new (time_series_map) typename Map::mapped_type(asof_type);
|
|
|
|
time_series_map->insert(asof_type, asof_column, stored_block, i);
|
2019-03-20 16:58:28 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
template <ASTTableJoin::Strictness STRICTNESS, typename KeyGetter, typename Map, bool has_null_map>
|
2021-02-04 14:46:36 +00:00
|
|
|
size_t NO_INLINE insertFromBlockImplTypeCase(
|
2020-04-07 09:48:47 +00:00
|
|
|
HashJoin & join, Map & map, size_t rows, const ColumnRawPtrs & key_columns,
|
2021-07-21 17:03:33 +00:00
|
|
|
const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2021-02-04 14:46:36 +00:00
|
|
|
[[maybe_unused]] constexpr bool mapped_one = std::is_same_v<typename Map::mapped_type, RowRef>;
|
2019-12-02 18:07:27 +00:00
|
|
|
constexpr bool is_asof_join = STRICTNESS == ASTTableJoin::Strictness::Asof;
|
|
|
|
|
2019-03-28 18:35:50 +00:00
|
|
|
const IColumn * asof_column [[maybe_unused]] = nullptr;
|
2019-12-02 18:07:27 +00:00
|
|
|
if constexpr (is_asof_join)
|
2020-08-03 23:11:39 +00:00
|
|
|
asof_column = key_columns.back();
|
2019-03-28 18:35:50 +00:00
|
|
|
|
2019-12-02 18:07:27 +00:00
|
|
|
auto key_getter = createKeyGetter<KeyGetter, is_asof_join>(key_columns, key_sizes);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
for (size_t i = 0; i < rows; ++i)
|
|
|
|
{
|
|
|
|
if (has_null_map && (*null_map)[i])
|
|
|
|
continue;
|
|
|
|
|
2021-07-21 17:03:33 +00:00
|
|
|
/// Check condition for right table from ON section
|
|
|
|
if (join_mask && !(*join_mask)[i])
|
|
|
|
continue;
|
|
|
|
|
2019-12-02 18:07:27 +00:00
|
|
|
if constexpr (is_asof_join)
|
2020-08-03 23:11:39 +00:00
|
|
|
Inserter<Map, KeyGetter>::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column);
|
2019-12-02 18:07:27 +00:00
|
|
|
else if constexpr (mapped_one)
|
2021-06-25 12:03:10 +00:00
|
|
|
{
|
2019-12-02 18:07:27 +00:00
|
|
|
Inserter<Map, KeyGetter>::insertOne(join, map, key_getter, stored_block, i, pool);
|
2021-06-25 12:03:10 +00:00
|
|
|
}
|
2019-03-29 21:20:23 +00:00
|
|
|
else
|
2021-06-25 12:03:10 +00:00
|
|
|
{
|
2019-12-02 18:07:27 +00:00
|
|
|
Inserter<Map, KeyGetter>::insertAll(join, map, key_getter, stored_block, i, pool);
|
2021-06-25 12:03:10 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2021-02-04 14:46:36 +00:00
|
|
|
return map.getBufferSizeInCells();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
template <ASTTableJoin::Strictness STRICTNESS, typename KeyGetter, typename Map>
|
2021-02-04 14:46:36 +00:00
|
|
|
size_t insertFromBlockImplType(
|
2020-04-07 09:48:47 +00:00
|
|
|
HashJoin & join, Map & map, size_t rows, const ColumnRawPtrs & key_columns,
|
2021-07-21 17:03:33 +00:00
|
|
|
const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
if (null_map)
|
2021-07-21 17:03:33 +00:00
|
|
|
return insertFromBlockImplTypeCase<STRICTNESS, KeyGetter, Map, true>(
|
|
|
|
join, map, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool);
|
2017-04-01 07:20:54 +00:00
|
|
|
else
|
2021-07-21 17:03:33 +00:00
|
|
|
return insertFromBlockImplTypeCase<STRICTNESS, KeyGetter, Map, false>(
|
|
|
|
join, map, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
template <ASTTableJoin::Strictness STRICTNESS, typename Maps>
|
2021-02-04 14:46:36 +00:00
|
|
|
size_t insertFromBlockImpl(
|
2020-04-07 09:48:47 +00:00
|
|
|
HashJoin & join, HashJoin::Type type, Maps & maps, size_t rows, const ColumnRawPtrs & key_columns,
|
2021-07-21 17:03:33 +00:00
|
|
|
const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
switch (type)
|
|
|
|
{
|
2021-02-04 14:46:36 +00:00
|
|
|
case HashJoin::Type::EMPTY: return 0;
|
|
|
|
case HashJoin::Type::CROSS: return 0; /// Do nothing. We have already saved block, and it is enough.
|
|
|
|
case HashJoin::Type::DICT: return 0; /// No one should call it with Type::DICT.
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
#define M(TYPE) \
|
2020-04-07 09:48:47 +00:00
|
|
|
case HashJoin::Type::TYPE: \
|
2021-02-04 14:46:36 +00:00
|
|
|
return insertFromBlockImplType<STRICTNESS, typename KeyGetterForType<HashJoin::Type::TYPE, std::remove_reference_t<decltype(*maps.TYPE)>>::Type>(\
|
2021-07-21 17:03:33 +00:00
|
|
|
join, *maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool); \
|
2017-04-01 07:20:54 +00:00
|
|
|
break;
|
|
|
|
APPLY_FOR_JOIN_VARIANTS(M)
|
|
|
|
#undef M
|
|
|
|
}
|
2021-02-08 11:38:31 +00:00
|
|
|
__builtin_unreachable();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2017-03-30 14:09:24 +00:00
|
|
|
}
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
void HashJoin::initRightBlockStructure(Block & saved_block_sample)
|
2019-11-01 17:41:07 +00:00
|
|
|
{
|
2020-02-18 12:41:23 +00:00
|
|
|
/// We could remove key columns for LEFT | INNER HashJoin but we should keep them for JoinSwitcher (if any).
|
2021-06-25 12:03:10 +00:00
|
|
|
bool save_key_columns = !table_join->forceHashJoin() || isRightOrFull(kind) || key_names_right.size() > 1;
|
2020-02-18 12:41:23 +00:00
|
|
|
if (save_key_columns)
|
2019-03-31 23:09:00 +00:00
|
|
|
{
|
2019-11-01 17:41:07 +00:00
|
|
|
saved_block_sample = right_table_keys.cloneEmpty();
|
|
|
|
}
|
|
|
|
else if (strictness == ASTTableJoin::Strictness::Asof)
|
|
|
|
{
|
|
|
|
/// Save ASOF key
|
|
|
|
saved_block_sample.insert(right_table_keys.safeGetByPosition(right_table_keys.columns() - 1));
|
|
|
|
}
|
2019-03-31 23:09:00 +00:00
|
|
|
|
2019-11-01 17:41:07 +00:00
|
|
|
/// Save non key columns
|
|
|
|
for (auto & column : sample_block_with_columns_to_add)
|
2021-06-25 12:03:10 +00:00
|
|
|
{
|
|
|
|
if (!saved_block_sample.findByName(column.name))
|
|
|
|
{
|
|
|
|
saved_block_sample.insert(column);
|
|
|
|
}
|
|
|
|
}
|
2019-03-31 23:09:00 +00:00
|
|
|
|
2019-11-03 19:33:59 +00:00
|
|
|
if (nullable_right_side)
|
2021-06-25 12:03:10 +00:00
|
|
|
{
|
2019-11-03 19:33:59 +00:00
|
|
|
JoinCommon::convertColumnsToNullable(saved_block_sample, (isFull(kind) ? right_table_keys.columns() : 0));
|
2021-06-25 12:03:10 +00:00
|
|
|
}
|
|
|
|
|
2019-11-01 17:41:07 +00:00
|
|
|
}
|
2019-03-31 23:09:00 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
HashJoin::BlockWithFlags HashJoin::structureRightBlock(const Block & block) const
|
2019-11-01 17:41:07 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
BlockWithFlags structured_block;
|
2020-04-22 06:01:33 +00:00
|
|
|
for (const auto & sample_column : savedBlockSample().getColumnsWithTypeAndName())
|
2019-11-03 19:33:59 +00:00
|
|
|
{
|
2019-12-17 15:02:42 +00:00
|
|
|
ColumnWithTypeAndName column = block.getByName(sample_column.name);
|
2019-11-03 19:33:59 +00:00
|
|
|
if (sample_column.column->isNullable())
|
|
|
|
JoinCommon::convertColumnToNullable(column);
|
2021-06-25 12:03:10 +00:00
|
|
|
structured_block.block.insert(column);
|
2019-03-31 23:09:00 +00:00
|
|
|
}
|
2019-11-01 17:41:07 +00:00
|
|
|
|
2019-12-03 13:31:52 +00:00
|
|
|
return structured_block;
|
2019-03-31 23:09:00 +00:00
|
|
|
}
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits)
|
2015-05-26 00:37:48 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (empty())
|
2020-04-07 09:48:47 +00:00
|
|
|
throw Exception("Logical error: HashJoin was not initialized", ErrorCodes::LOGICAL_ERROR);
|
2020-04-08 18:59:52 +00:00
|
|
|
if (overDictionary())
|
|
|
|
throw Exception("Logical error: insert into hash-map in HashJoin over dictionary", ErrorCodes::LOGICAL_ERROR);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-04-21 19:09:18 +00:00
|
|
|
/// RowRef::SizeT is uint32_t (not size_t) for hash table Cell memory efficiency.
|
2020-04-21 19:01:34 +00:00
|
|
|
/// It's possible to split bigger blocks and insert them by parts here. But it would be a dead code.
|
|
|
|
if (unlikely(source_block.rows() > std::numeric_limits<RowRef::SizeT>::max()))
|
|
|
|
throw Exception("Too many rows in right table block for HashJoin: " + toString(source_block.rows()), ErrorCodes::NOT_IMPLEMENTED);
|
|
|
|
|
2019-12-17 15:02:42 +00:00
|
|
|
/// There's no optimization for right side const columns. Remove constness if any.
|
|
|
|
Block block = materializeBlock(source_block);
|
|
|
|
size_t rows = block.rows();
|
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
size_t total_rows = 0;
|
|
|
|
size_t total_bytes = 0;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
// Collect all keys in all_key_names_right
|
|
|
|
// and lists of indexes in this vector for all disjuncts
|
|
|
|
Names all_key_names_right = key_names_right.front();
|
|
|
|
const size_t disjuncts_num = key_names_right.size();
|
|
|
|
std::vector<std::vector<size_t>> key_names_right_indexes(disjuncts_num);
|
|
|
|
key_names_right_indexes[0].resize(all_key_names_right.size());
|
|
|
|
std::iota(std::begin(key_names_right_indexes[0]), std::end(key_names_right_indexes[0]), 0);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
for (size_t d = 1; d < disjuncts_num; ++d)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
for (size_t i = 0; i < key_names_right[d].size(); ++i)
|
|
|
|
{
|
|
|
|
auto it = std::find(std::cbegin(all_key_names_right), std::cend(all_key_names_right), key_names_right[d][i]);
|
|
|
|
if (it == std::cend(all_key_names_right))
|
|
|
|
{
|
|
|
|
key_names_right_indexes[d].push_back(all_key_names_right.size());
|
|
|
|
all_key_names_right.push_back(key_names_right[d][i]);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
key_names_right_indexes[d].push_back(std::distance(std::cbegin(all_key_names_right), it));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
ColumnRawPtrs all_key_columns = JoinCommon::materializeColumnsInplace(block, all_key_names_right);
|
|
|
|
|
|
|
|
BlockWithFlags structured_block = structureRightBlock(block);
|
|
|
|
bool multiple_disjuncts = disjuncts_num > 1;
|
|
|
|
// if (nullable_right_side && multiple_disjuncts)
|
|
|
|
// {
|
|
|
|
// JoinCommon::convertColumnsToNullable(structured_block.block);
|
|
|
|
// }
|
|
|
|
std::vector<ColumnPtr> join_mask_col_vector(disjuncts_num);
|
|
|
|
// std::vector<const ColumnUInt8 &> join_mask_vector(disjuncts_num);
|
|
|
|
bool use_join_mask_col = false;
|
|
|
|
for (size_t d = 0; d < disjuncts_num; ++d)
|
|
|
|
{
|
|
|
|
join_mask_col_vector[d] = JoinCommon::getColumnAsMask(block, condition_mask_column_name_right[d]);
|
|
|
|
// join_mask_vector[d] = assert_cast<const ColumnUInt8 &>(*(join_mask_col_vector[d])).getData();
|
|
|
|
if (join_mask_col_vector[d])
|
|
|
|
use_join_mask_col = true;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2021-07-21 17:03:33 +00:00
|
|
|
|
|
|
|
/// Save blocks that do not hold conditions in ON section
|
|
|
|
ColumnUInt8::MutablePtr not_joined_map = nullptr;
|
2021-06-25 12:03:10 +00:00
|
|
|
if (isRightOrFull(kind) && use_join_mask_col)
|
2021-07-21 17:03:33 +00:00
|
|
|
{
|
|
|
|
/// Save rows that do not hold conditions
|
|
|
|
not_joined_map = ColumnUInt8::create(block.rows(), 0);
|
2021-06-25 12:03:10 +00:00
|
|
|
const size_t sz = assert_cast<const ColumnUInt8 &>(*(join_mask_col_vector[0])).getData().size();
|
|
|
|
for (size_t i = 0; i < sz; ++i)
|
2021-07-21 17:03:33 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
bool add_to_not_joined_map = true;
|
|
|
|
|
2021-07-21 17:03:33 +00:00
|
|
|
/// Condition hold, do not save row
|
2021-06-25 12:03:10 +00:00
|
|
|
for (size_t d = 0; d < disjuncts_num; ++d)
|
|
|
|
{
|
|
|
|
const auto & join_mask = assert_cast<const ColumnUInt8 &>(*(join_mask_col_vector[d])).getData();
|
|
|
|
if (join_mask[i])
|
|
|
|
{
|
|
|
|
add_to_not_joined_map = false;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
2021-07-21 17:03:33 +00:00
|
|
|
|
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
// !!!
|
|
|
|
// /// NULL key will be saved anyway because, do not save twice
|
|
|
|
// if (save_nullmap && (*null_map)[i])
|
|
|
|
// continue;
|
|
|
|
if (add_to_not_joined_map)
|
|
|
|
not_joined_map->getData()[i] = 1;
|
2021-07-21 17:03:33 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
std::vector<ConstNullMapPtr> null_map_vector;
|
|
|
|
Columns null_map_holder_vector;
|
2019-12-03 13:31:52 +00:00
|
|
|
|
2019-07-04 12:38:54 +00:00
|
|
|
{
|
2021-02-25 11:21:06 +00:00
|
|
|
if (storage_join_lock.mutex())
|
|
|
|
throw DB::Exception("addJoinedBlock called when HashJoin locked to prevent updates",
|
|
|
|
ErrorCodes::LOGICAL_ERROR);
|
2019-12-03 13:31:52 +00:00
|
|
|
|
2019-12-19 15:50:28 +00:00
|
|
|
data->blocks.emplace_back(std::move(structured_block));
|
2021-06-25 12:03:10 +00:00
|
|
|
BlockWithFlags & stored_block_with_flags = data->blocks.back();
|
|
|
|
Block * stored_block = &stored_block_with_flags.block;
|
|
|
|
stored_block_with_flags.flags = std::vector<std::atomic_bool>(stored_block->rows());
|
2019-12-03 13:31:52 +00:00
|
|
|
|
2019-12-03 14:30:51 +00:00
|
|
|
if (rows)
|
2019-12-19 15:50:28 +00:00
|
|
|
data->empty = false;
|
2019-10-12 10:06:07 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
bool save_a_nullmap = false;
|
|
|
|
|
|
|
|
for (size_t d = 0; d < disjuncts_num; ++d)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
ColumnRawPtrs key_columns(key_names_right_indexes[d].size());
|
|
|
|
std::transform(std::cbegin(key_names_right_indexes[d]), std::cend(key_names_right_indexes[d]), std::begin(key_columns), [&](size_t ind){return all_key_columns[ind];});
|
|
|
|
|
|
|
|
/// We will insert to the map only keys, where all components are not NULL.
|
|
|
|
|
|
|
|
null_map_vector.emplace_back();
|
|
|
|
null_map_holder_vector.push_back(extractNestedColumnsAndNullMap(key_columns, null_map_vector.back()));
|
|
|
|
|
|
|
|
|
|
|
|
/// If RIGHT or FULL save blocks with nulls for NonJoinedBlockInputStream
|
|
|
|
UInt8 save_nullmap = 0;
|
|
|
|
if (isRightOrFull(kind) && null_map_vector.back())
|
2019-12-03 14:30:51 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
for (size_t i = 0; !save_nullmap && i < null_map_vector.back()->size(); ++i)
|
|
|
|
save_nullmap |= (*null_map_vector.back())[i];
|
|
|
|
}
|
|
|
|
save_a_nullmap |= save_nullmap;
|
|
|
|
|
|
|
|
{
|
|
|
|
if (kind != ASTTableJoin::Kind::Cross)
|
|
|
|
{
|
|
|
|
joinDispatch(kind, strictness, data->maps[d], [&](auto kind_, auto strictness_, auto & map)
|
|
|
|
{
|
|
|
|
size_t size = insertFromBlockImpl<strictness_>(
|
|
|
|
*this, data->type, map, rows, key_columns, key_sizes[d], stored_block, null_map_vector.back(),
|
|
|
|
join_mask_col_vector[d] ? &assert_cast<const ColumnUInt8 &>(*join_mask_col_vector[d]).getData() : nullptr,
|
|
|
|
data->pool);
|
|
|
|
/// Number of buckets + 1 value from zero storage
|
|
|
|
|
|
|
|
if (!d)
|
|
|
|
{
|
|
|
|
used_flags.reinit<kind_, strictness_>(size + 1);
|
|
|
|
}
|
|
|
|
});
|
|
|
|
}
|
|
|
|
|
|
|
|
if (not_joined_map)
|
|
|
|
data->blocks_nullmaps.emplace_back(stored_block, std::move(not_joined_map));
|
2019-07-04 12:38:54 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
if (!check_limits)
|
|
|
|
return true;
|
2019-12-03 14:30:51 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
/// TODO: Do not calculate them every time
|
|
|
|
total_rows = getTotalRowCount();
|
|
|
|
total_bytes = getTotalByteCount();
|
|
|
|
}
|
|
|
|
}
|
2021-07-21 17:03:33 +00:00
|
|
|
|
2020-02-17 17:08:31 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
if (save_a_nullmap && !multiple_disjuncts)
|
|
|
|
{
|
|
|
|
data->blocks_nullmaps.emplace_back(stored_block, null_map_holder_vector[0]);
|
|
|
|
}
|
2019-07-04 12:38:54 +00:00
|
|
|
}
|
2019-07-03 19:06:34 +00:00
|
|
|
|
2019-12-03 14:30:51 +00:00
|
|
|
return table_join->sizeLimits().check(total_rows, total_bytes, "JOIN", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
|
2014-06-12 02:31:30 +00:00
|
|
|
}
|
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
using ColumnRawPtrsVector = std::vector<ColumnRawPtrs>;
|
|
|
|
using SizesVector = std::vector<Sizes>;
|
2019-03-20 14:41:35 +00:00
|
|
|
|
2019-03-20 14:49:05 +00:00
|
|
|
class AddedColumns
|
|
|
|
{
|
|
|
|
public:
|
2021-04-29 14:30:02 +00:00
|
|
|
struct TypeAndName
|
|
|
|
{
|
|
|
|
DataTypePtr type;
|
|
|
|
String name;
|
|
|
|
String qualified_name;
|
|
|
|
|
|
|
|
TypeAndName(DataTypePtr type_, const String & name_, const String & qualified_name_)
|
2021-06-15 08:34:53 +00:00
|
|
|
: type(type_), name(name_), qualified_name(qualified_name_)
|
|
|
|
{
|
|
|
|
}
|
2021-04-29 14:30:02 +00:00
|
|
|
};
|
2019-03-20 14:41:35 +00:00
|
|
|
|
2021-06-15 08:34:53 +00:00
|
|
|
AddedColumns(
|
|
|
|
const Block & block_with_columns_to_add,
|
|
|
|
const Block & block,
|
|
|
|
const Block & saved_block_sample,
|
|
|
|
const HashJoin & join,
|
2021-06-25 12:03:10 +00:00
|
|
|
const ColumnRawPtrsVector & key_columns_,
|
|
|
|
const SizesVector & key_sizes_,
|
|
|
|
const std::vector<ColumnPtr> & join_mask_column_,
|
2021-06-15 08:34:53 +00:00
|
|
|
bool is_asof_join,
|
|
|
|
bool is_join_get_)
|
2020-08-03 23:11:39 +00:00
|
|
|
: key_columns(key_columns_)
|
2019-11-06 19:39:52 +00:00
|
|
|
, key_sizes(key_sizes_)
|
|
|
|
, rows_to_add(block.rows())
|
2020-08-03 23:11:39 +00:00
|
|
|
, asof_type(join.getAsofType())
|
|
|
|
, asof_inequality(join.getAsofInequality())
|
2021-07-21 17:03:33 +00:00
|
|
|
, join_mask_column(join_mask_column_)
|
2021-06-15 08:34:53 +00:00
|
|
|
, is_join_get(is_join_get_)
|
2019-03-20 14:49:05 +00:00
|
|
|
{
|
2020-08-03 23:11:39 +00:00
|
|
|
size_t num_columns_to_add = block_with_columns_to_add.columns();
|
|
|
|
if (is_asof_join)
|
|
|
|
++num_columns_to_add;
|
2019-03-20 14:41:35 +00:00
|
|
|
|
2019-03-20 15:15:44 +00:00
|
|
|
columns.reserve(num_columns_to_add);
|
|
|
|
type_name.reserve(num_columns_to_add);
|
|
|
|
right_indexes.reserve(num_columns_to_add);
|
2019-03-20 14:41:35 +00:00
|
|
|
|
2020-04-22 06:01:33 +00:00
|
|
|
for (const auto & src_column : block_with_columns_to_add)
|
2019-03-20 15:15:44 +00:00
|
|
|
{
|
2021-05-10 13:39:32 +00:00
|
|
|
/// Column names `src_column.name` and `qualified_name` can differ for StorageJoin,
|
2021-04-29 14:30:02 +00:00
|
|
|
/// because it uses not qualified right block column names
|
|
|
|
auto qualified_name = join.getTableJoin().renamedRightColumnName(src_column.name);
|
2020-04-07 14:52:32 +00:00
|
|
|
/// Don't insert column if it's in left block
|
2021-04-29 14:30:02 +00:00
|
|
|
if (!block.has(qualified_name))
|
|
|
|
addColumn(src_column, qualified_name);
|
2019-03-31 21:14:43 +00:00
|
|
|
}
|
|
|
|
|
2020-08-03 23:11:39 +00:00
|
|
|
if (is_asof_join)
|
|
|
|
{
|
|
|
|
const ColumnWithTypeAndName & right_asof_column = join.rightAsofKeyColumn();
|
2021-04-29 14:30:02 +00:00
|
|
|
addColumn(right_asof_column, right_asof_column.name);
|
2021-06-25 12:03:10 +00:00
|
|
|
left_asof_key = key_columns.front().back();
|
2020-08-03 23:11:39 +00:00
|
|
|
}
|
2019-03-31 21:14:43 +00:00
|
|
|
|
2019-04-01 10:35:37 +00:00
|
|
|
for (auto & tn : type_name)
|
2021-04-29 14:30:02 +00:00
|
|
|
right_indexes.push_back(saved_block_sample.getPositionByName(tn.name));
|
2019-03-20 14:49:05 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-03-20 15:15:44 +00:00
|
|
|
size_t size() const { return columns.size(); }
|
|
|
|
|
2019-03-20 14:49:05 +00:00
|
|
|
ColumnWithTypeAndName moveColumn(size_t i)
|
|
|
|
{
|
2021-04-29 14:30:02 +00:00
|
|
|
return ColumnWithTypeAndName(std::move(columns[i]), type_name[i].type, type_name[i].qualified_name);
|
2019-03-20 14:49:05 +00:00
|
|
|
}
|
2019-03-20 14:41:35 +00:00
|
|
|
|
2019-11-08 16:13:43 +00:00
|
|
|
template <bool has_defaults>
|
2019-03-20 14:49:05 +00:00
|
|
|
void appendFromBlock(const Block & block, size_t row_num)
|
|
|
|
{
|
2019-11-08 16:13:43 +00:00
|
|
|
if constexpr (has_defaults)
|
|
|
|
applyLazyDefaults();
|
|
|
|
|
2021-06-15 08:34:53 +00:00
|
|
|
if (is_join_get)
|
|
|
|
{
|
|
|
|
/// If it's joinGetOrNull, we need to wrap not-nullable columns in StorageJoin.
|
|
|
|
for (size_t j = 0, size = right_indexes.size(); j < size; ++j)
|
|
|
|
{
|
|
|
|
const auto & column = *block.getByPosition(right_indexes[j]).column;
|
|
|
|
if (auto * nullable_col = typeid_cast<ColumnNullable *>(columns[j].get()); nullable_col && !column.isNullable())
|
|
|
|
nullable_col->insertFromNotNullable(column, row_num);
|
|
|
|
else
|
|
|
|
columns[j]->insertFrom(column, row_num);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
for (size_t j = 0, size = right_indexes.size(); j < size; ++j)
|
|
|
|
{
|
|
|
|
columns[j]->insertFrom(*block.getByPosition(right_indexes[j]).column, row_num);
|
|
|
|
}
|
|
|
|
}
|
2019-03-31 21:14:43 +00:00
|
|
|
}
|
2019-03-20 16:58:28 +00:00
|
|
|
|
2019-03-20 14:49:05 +00:00
|
|
|
void appendDefaultRow()
|
2019-03-20 14:41:35 +00:00
|
|
|
{
|
2019-11-06 19:39:52 +00:00
|
|
|
++lazy_defaults_count;
|
2019-03-20 14:49:05 +00:00
|
|
|
}
|
2019-03-20 14:41:35 +00:00
|
|
|
|
2019-11-06 19:39:52 +00:00
|
|
|
void applyLazyDefaults()
|
|
|
|
{
|
|
|
|
if (lazy_defaults_count)
|
|
|
|
{
|
2021-01-28 08:26:10 +00:00
|
|
|
for (size_t j = 0, size = right_indexes.size(); j < size; ++j)
|
2021-04-29 14:30:02 +00:00
|
|
|
JoinCommon::addDefaultValues(*columns[j], type_name[j].type, lazy_defaults_count);
|
2019-11-06 19:39:52 +00:00
|
|
|
lazy_defaults_count = 0;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-08-03 23:11:39 +00:00
|
|
|
TypeIndex asofType() const { return *asof_type; }
|
|
|
|
ASOF::Inequality asofInequality() const { return asof_inequality; }
|
|
|
|
const IColumn & leftAsofKey() const { return *left_asof_key; }
|
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
bool isRowFiltered(size_t i, size_t d)
|
|
|
|
{
|
|
|
|
if (join_mask_column[d])
|
|
|
|
{
|
2021-07-21 17:03:33 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
UInt8ColumnDataPtr jmc = &assert_cast<const ColumnUInt8 &>(*(join_mask_column[d])).getData();
|
|
|
|
return !(*jmc)[i];
|
|
|
|
}
|
|
|
|
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
const ColumnRawPtrsVector key_columns;
|
|
|
|
const SizesVector key_sizes;
|
2019-11-06 19:39:52 +00:00
|
|
|
size_t rows_to_add;
|
|
|
|
std::unique_ptr<IColumn::Offsets> offsets_to_replicate;
|
2020-08-03 23:11:39 +00:00
|
|
|
bool need_filter = false;
|
2021-06-25 12:03:10 +00:00
|
|
|
IColumn::Filter row_filter;
|
2019-11-06 19:39:52 +00:00
|
|
|
|
2019-03-31 19:03:57 +00:00
|
|
|
private:
|
2021-04-29 14:30:02 +00:00
|
|
|
std::vector<TypeAndName> type_name;
|
2019-03-31 19:03:57 +00:00
|
|
|
MutableColumns columns;
|
|
|
|
std::vector<size_t> right_indexes;
|
2019-11-06 19:39:52 +00:00
|
|
|
size_t lazy_defaults_count = 0;
|
2020-08-03 23:11:39 +00:00
|
|
|
/// for ASOF
|
|
|
|
std::optional<TypeIndex> asof_type;
|
|
|
|
ASOF::Inequality asof_inequality;
|
|
|
|
const IColumn * left_asof_key = nullptr;
|
2021-06-25 12:03:10 +00:00
|
|
|
std::vector<ColumnPtr> join_mask_column;
|
2021-06-15 08:34:53 +00:00
|
|
|
bool is_join_get;
|
2019-03-31 23:09:00 +00:00
|
|
|
|
2021-04-29 14:30:02 +00:00
|
|
|
void addColumn(const ColumnWithTypeAndName & src_column, const std::string & qualified_name)
|
2019-03-31 23:09:00 +00:00
|
|
|
{
|
|
|
|
columns.push_back(src_column.column->cloneEmpty());
|
|
|
|
columns.back()->reserve(src_column.column->size());
|
2021-04-29 14:30:02 +00:00
|
|
|
type_name.emplace_back(src_column.type, src_column.name, qualified_name);
|
2019-03-31 23:09:00 +00:00
|
|
|
}
|
2019-03-20 15:15:44 +00:00
|
|
|
};
|
2021-06-25 12:03:10 +00:00
|
|
|
using AddedColumnsV = std::vector<std::unique_ptr<AddedColumns>>;
|
|
|
|
|
|
|
|
namespace
|
|
|
|
{
|
|
|
|
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS>
|
|
|
|
struct JoinFeatures
|
|
|
|
{
|
|
|
|
static constexpr bool is_any_join = STRICTNESS == ASTTableJoin::Strictness::Any;
|
|
|
|
static constexpr bool is_all_join = STRICTNESS == ASTTableJoin::Strictness::All;
|
|
|
|
static constexpr bool is_asof_join = STRICTNESS == ASTTableJoin::Strictness::Asof;
|
|
|
|
static constexpr bool is_semi_join = STRICTNESS == ASTTableJoin::Strictness::Semi;
|
|
|
|
static constexpr bool is_anti_join = STRICTNESS == ASTTableJoin::Strictness::Anti;
|
|
|
|
|
|
|
|
static constexpr bool left = KIND == ASTTableJoin::Kind::Left;
|
|
|
|
static constexpr bool right = KIND == ASTTableJoin::Kind::Right;
|
|
|
|
static constexpr bool inner = KIND == ASTTableJoin::Kind::Inner;
|
|
|
|
static constexpr bool full = KIND == ASTTableJoin::Kind::Full;
|
|
|
|
|
|
|
|
static constexpr bool need_replication = is_all_join || (is_any_join && right) || (is_semi_join && right);
|
|
|
|
static constexpr bool need_filter = !need_replication && (inner || right || (is_semi_join && left) || (is_anti_join && left));
|
|
|
|
static constexpr bool add_missing = (left || full) && !is_semi_join;
|
|
|
|
|
|
|
|
static constexpr bool need_flags = MapGetter<KIND, STRICTNESS>::flagged;
|
|
|
|
|
|
|
|
};
|
|
|
|
|
|
|
|
template <bool multiple_disjuncts>
|
|
|
|
class KnownRowsHolder;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
// TODO: helper to clean, instead of recreating
|
|
|
|
|
|
|
|
template<>
|
|
|
|
class KnownRowsHolder<true>
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
using Type = std::pair<const Block*, DB::RowRef::SizeT>;
|
|
|
|
|
|
|
|
private:
|
|
|
|
static const size_t MAX_LINEAR = 16;
|
|
|
|
using LinearHolder = std::array<Type, MAX_LINEAR>;
|
|
|
|
using LogHolder = std::set<Type>;
|
|
|
|
using LogHolderPtr = std::unique_ptr<LogHolder>;
|
|
|
|
|
|
|
|
LinearHolder linh;
|
|
|
|
LogHolderPtr logh_ptr;
|
|
|
|
|
|
|
|
size_t items;
|
|
|
|
|
|
|
|
public:
|
|
|
|
KnownRowsHolder()
|
|
|
|
: items(0)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
template<class InputIt>
|
|
|
|
void add(InputIt from, InputIt to)
|
|
|
|
{
|
|
|
|
size_t new_items = std::distance(from, to);
|
|
|
|
if (items + new_items <= MAX_LINEAR)
|
|
|
|
{
|
|
|
|
std::copy(from, to, &linh[items]);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
if (items <= MAX_LINEAR)
|
|
|
|
{
|
|
|
|
logh_ptr = std::make_unique<LogHolder>();
|
|
|
|
logh_ptr->insert(std::cbegin(linh), std::cbegin(linh) + items);
|
|
|
|
}
|
|
|
|
logh_ptr->insert(from, to);
|
|
|
|
}
|
|
|
|
items += new_items;
|
|
|
|
}
|
|
|
|
|
|
|
|
template<class Needle>
|
|
|
|
bool isKnown(const Needle & needle)
|
|
|
|
{
|
|
|
|
return items <= MAX_LINEAR
|
|
|
|
? std::find(std::cbegin(linh), std::cbegin(linh) + items, needle) != std::cbegin(linh) + items
|
|
|
|
: logh_ptr->find(needle) != logh_ptr->end();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
template<>
|
|
|
|
class KnownRowsHolder<false>
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
template<class InputIt>
|
|
|
|
void add(InputIt, InputIt)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
template<class Needle>
|
|
|
|
static bool isKnown(const Needle &)
|
|
|
|
{
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
template <typename Map, bool add_missing, bool multiple_disjuncts>
|
|
|
|
void addFoundRowAll(const typename Map::mapped_type & mapped, AddedColumns & added, IColumn::Offset & current_offset, KnownRowsHolder<multiple_disjuncts> & known_rows [[maybe_unused]])
|
2019-03-20 14:49:05 +00:00
|
|
|
{
|
2019-12-02 11:45:21 +00:00
|
|
|
if constexpr (add_missing)
|
|
|
|
added.applyLazyDefaults();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
if constexpr (multiple_disjuncts)
|
2019-03-19 16:53:36 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
std::unique_ptr<std::vector<KnownRowsHolder<true>::Type>> new_known_rows_ptr;
|
|
|
|
|
|
|
|
for (auto it = mapped.begin(); it.ok(); ++it)
|
|
|
|
{
|
|
|
|
if (!known_rows.isKnown(std::make_pair(it->block, it->row_num)))
|
|
|
|
{
|
|
|
|
added.appendFromBlock<false>(*it->block, it->row_num);
|
|
|
|
++current_offset;
|
|
|
|
if (!new_known_rows_ptr)
|
|
|
|
{
|
|
|
|
new_known_rows_ptr = std::make_unique<std::vector<KnownRowsHolder<true>::Type>>();
|
|
|
|
}
|
|
|
|
new_known_rows_ptr->push_back(std::make_pair(it->block, it->row_num));
|
|
|
|
const HashJoin::BlockWithFlags * block_with_flags = reinterpret_cast<const HashJoin::BlockWithFlags*>(it->block);
|
|
|
|
|
|
|
|
block_with_flags->flags[it->row_num].store(true, std::memory_order_relaxed);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (new_known_rows_ptr)
|
|
|
|
{
|
|
|
|
known_rows.add(std::cbegin(*new_known_rows_ptr), std::cend(*new_known_rows_ptr));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
for (auto it = mapped.begin(); it.ok(); ++it)
|
|
|
|
{
|
|
|
|
added.appendFromBlock<false>(*it->block, it->row_num);
|
|
|
|
++current_offset;
|
|
|
|
}
|
2019-03-19 16:53:36 +00:00
|
|
|
}
|
2019-03-20 14:49:05 +00:00
|
|
|
};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-12-03 12:55:20 +00:00
|
|
|
template <bool add_missing, bool need_offset>
|
2019-03-20 14:49:05 +00:00
|
|
|
void addNotFoundRow(AddedColumns & added [[maybe_unused]], IColumn::Offset & current_offset [[maybe_unused]])
|
|
|
|
{
|
2019-11-08 16:13:43 +00:00
|
|
|
if constexpr (add_missing)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-03-20 14:49:05 +00:00
|
|
|
added.appendDefaultRow();
|
2019-12-03 12:55:20 +00:00
|
|
|
if constexpr (need_offset)
|
|
|
|
++current_offset;
|
2019-03-20 14:49:05 +00:00
|
|
|
}
|
|
|
|
}
|
2019-03-19 16:53:36 +00:00
|
|
|
|
2019-12-03 12:55:20 +00:00
|
|
|
template <bool need_filter>
|
|
|
|
void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unused]])
|
|
|
|
{
|
|
|
|
if constexpr (need_filter)
|
|
|
|
filter[pos] = 1;
|
|
|
|
}
|
|
|
|
|
2019-03-20 14:49:05 +00:00
|
|
|
/// Joins right table columns which indexes are present in right_indexes using specified map.
|
|
|
|
/// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS).
|
2021-06-25 12:03:10 +00:00
|
|
|
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS, typename KeyGetter, typename Map, bool need_filter, bool has_null_map, bool multiple_disjuncts>
|
2021-02-04 14:46:36 +00:00
|
|
|
NO_INLINE IColumn::Filter joinRightColumns(
|
2021-06-25 12:03:10 +00:00
|
|
|
std::vector<KeyGetter> && key_getter_vector,
|
|
|
|
const std::vector<const Map *> & mapv,
|
2021-02-04 14:46:36 +00:00
|
|
|
AddedColumns & added_columns,
|
2021-06-25 12:03:10 +00:00
|
|
|
const std::vector<ConstNullMapPtr> & null_map [[maybe_unused]],
|
2021-02-04 14:46:36 +00:00
|
|
|
JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]])
|
2019-03-20 14:49:05 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
JoinFeatures<KIND, STRICTNESS> jf;
|
2019-11-07 21:32:44 +00:00
|
|
|
|
2019-11-06 19:39:52 +00:00
|
|
|
size_t rows = added_columns.rows_to_add;
|
2019-12-03 12:55:20 +00:00
|
|
|
IColumn::Filter filter;
|
|
|
|
if constexpr (need_filter)
|
|
|
|
filter = IColumn::Filter(rows, 0);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-03-20 14:49:05 +00:00
|
|
|
Arena pool;
|
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
if constexpr (jf.need_replication)
|
2019-11-11 11:42:10 +00:00
|
|
|
added_columns.offsets_to_replicate = std::make_unique<IColumn::Offsets>(rows);
|
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
size_t disjunct_num = added_columns.key_columns.size();
|
|
|
|
|
|
|
|
// std::vector<KeyGetter> key_getter_vector;
|
|
|
|
|
|
|
|
// for (size_t d = 0; d < disjunct_num; ++d)
|
|
|
|
// {
|
|
|
|
// auto key_getter = createKeyGetter<KeyGetter, jf.is_asof_join>(added_columns.key_columns[d], added_columns.key_sizes[d]);
|
|
|
|
// key_getter_vector.push_back(std::move(key_getter));
|
|
|
|
// }
|
|
|
|
|
2019-03-20 16:58:28 +00:00
|
|
|
IColumn::Offset current_offset = 0;
|
|
|
|
|
2019-03-20 14:49:05 +00:00
|
|
|
for (size_t i = 0; i < rows; ++i)
|
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
bool right_row_found = false;
|
|
|
|
bool null_element_found = false;
|
|
|
|
|
|
|
|
KnownRowsHolder<multiple_disjuncts> known_rows;
|
|
|
|
size_t d = 0;
|
|
|
|
do
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
if constexpr (has_null_map)
|
2019-11-11 11:42:10 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
if (null_map[d] && (*null_map[d])[i])
|
|
|
|
{
|
|
|
|
null_element_found = true;
|
|
|
|
continue;
|
|
|
|
}
|
2019-11-11 11:42:10 +00:00
|
|
|
}
|
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
bool row_acceptable = !added_columns.isRowFiltered(i, d);
|
|
|
|
using FindResult = typename KeyGetter::FindResult;
|
|
|
|
auto find_result = row_acceptable ? key_getter_vector[d].findKey(*(mapv[d]), i, pool) : FindResult();
|
2019-03-20 14:49:05 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
if (find_result.isFound())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
right_row_found = true;
|
|
|
|
auto & mapped = find_result.getMapped();
|
|
|
|
if constexpr (jf.is_asof_join)
|
|
|
|
{
|
|
|
|
TypeIndex asof_type = added_columns.asofType();
|
|
|
|
ASOF::Inequality asof_inequality = added_columns.asofInequality();
|
|
|
|
const IColumn & left_asof_key = added_columns.leftAsofKey();
|
|
|
|
|
|
|
|
if (const RowRef * found = mapped.findAsof(asof_type, asof_inequality, left_asof_key, i))
|
|
|
|
{
|
|
|
|
setUsed<need_filter>(filter, i);
|
|
|
|
used_flags.template setUsed<jf.need_flags, multiple_disjuncts>(find_result);
|
|
|
|
added_columns.appendFromBlock<jf.add_missing>(*found->block, found->row_num);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
addNotFoundRow<jf.add_missing, jf.need_replication>(added_columns, current_offset);
|
|
|
|
}
|
|
|
|
else if constexpr (jf.is_all_join)
|
2019-03-30 21:30:21 +00:00
|
|
|
{
|
2019-12-03 12:55:20 +00:00
|
|
|
setUsed<need_filter>(filter, i);
|
2021-06-25 12:03:10 +00:00
|
|
|
used_flags.template setUsed<jf.need_flags, multiple_disjuncts>(find_result);
|
|
|
|
addFoundRowAll<Map, jf.add_missing>(mapped, added_columns, current_offset, known_rows);
|
2019-03-30 21:30:21 +00:00
|
|
|
}
|
2021-06-25 12:03:10 +00:00
|
|
|
else if constexpr ((jf.is_any_join || jf.is_semi_join) && jf.right)
|
|
|
|
{
|
|
|
|
/// Use first appeared left key + it needs left columns replication
|
|
|
|
bool used_once = used_flags.template setUsedOnce<jf.need_flags, multiple_disjuncts>(find_result);
|
|
|
|
|
|
|
|
if (used_once)
|
|
|
|
{
|
|
|
|
setUsed<need_filter>(filter, i);
|
|
|
|
addFoundRowAll<Map, jf.add_missing>(mapped, added_columns, current_offset, known_rows);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else if constexpr (jf.is_any_join && KIND == ASTTableJoin::Kind::Inner)
|
|
|
|
{
|
|
|
|
bool used_once = used_flags.template setUsedOnce<jf.need_flags, multiple_disjuncts>(find_result);
|
|
|
|
|
|
|
|
/// Use first appeared left key only
|
|
|
|
if (used_once)
|
|
|
|
{
|
|
|
|
setUsed<need_filter>(filter, i);
|
|
|
|
added_columns.appendFromBlock<jf.add_missing>(*mapped.block, mapped.row_num);
|
|
|
|
}
|
2021-02-04 14:46:36 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
else if constexpr (jf.is_any_join && jf.full)
|
|
|
|
{
|
|
|
|
/// TODO
|
|
|
|
}
|
|
|
|
else if constexpr (jf.is_anti_join)
|
|
|
|
{
|
|
|
|
if constexpr (jf.right && jf.need_flags)
|
|
|
|
used_flags.template setUsed<jf.need_flags, multiple_disjuncts>(find_result);
|
|
|
|
}
|
|
|
|
else /// ANY LEFT, SEMI LEFT, old ANY (RightAny)
|
2019-03-26 20:13:15 +00:00
|
|
|
{
|
2019-12-03 12:55:20 +00:00
|
|
|
setUsed<need_filter>(filter, i);
|
2021-06-25 12:03:10 +00:00
|
|
|
used_flags.template setUsed<jf.need_flags, multiple_disjuncts>(find_result);
|
|
|
|
added_columns.appendFromBlock<jf.add_missing>(*mapped.block, mapped.row_num);
|
|
|
|
if constexpr (multiple_disjuncts)
|
|
|
|
{
|
|
|
|
const HashJoin::BlockWithFlags * block_with_flags = reinterpret_cast<const HashJoin::BlockWithFlags*>(mapped.block);
|
|
|
|
block_with_flags->flags[mapped.row_num].store(true, std::memory_order_relaxed);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
if (jf.is_any_join)
|
|
|
|
{
|
|
|
|
break;
|
|
|
|
}
|
2019-03-20 16:58:28 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2021-06-25 12:03:10 +00:00
|
|
|
} while (multiple_disjuncts && ++d < disjunct_num);
|
|
|
|
|
|
|
|
if constexpr (has_null_map)
|
|
|
|
{
|
|
|
|
if (!right_row_found && null_element_found)
|
2019-11-11 11:42:10 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
addNotFoundRow<jf.add_missing, jf.need_replication>(added_columns, current_offset);
|
2021-02-04 14:46:36 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
if constexpr (jf.need_replication)
|
2019-11-07 21:32:44 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
(*added_columns.offsets_to_replicate)[i] = current_offset;
|
2019-03-20 16:58:28 +00:00
|
|
|
}
|
2021-06-25 12:03:10 +00:00
|
|
|
continue;
|
2019-11-11 11:42:10 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2021-06-25 12:03:10 +00:00
|
|
|
|
|
|
|
if (!right_row_found)
|
2019-12-02 18:07:27 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
if constexpr (jf.is_anti_join && jf.left)
|
2019-12-03 12:55:20 +00:00
|
|
|
setUsed<need_filter>(filter, i);
|
2021-06-25 12:03:10 +00:00
|
|
|
addNotFoundRow<jf.add_missing, jf.need_replication>(added_columns, current_offset);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
if constexpr (jf.need_replication)
|
|
|
|
{
|
2019-11-06 19:39:52 +00:00
|
|
|
(*added_columns.offsets_to_replicate)[i] = current_offset;
|
2021-06-25 12:03:10 +00:00
|
|
|
}
|
2019-03-19 16:53:36 +00:00
|
|
|
}
|
2019-01-29 12:38:53 +00:00
|
|
|
|
2019-11-06 19:39:52 +00:00
|
|
|
added_columns.applyLazyDefaults();
|
2019-11-11 11:42:10 +00:00
|
|
|
return filter;
|
2019-03-20 14:49:05 +00:00
|
|
|
}
|
2019-03-20 13:38:17 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS, typename KeyGetter, typename Map, bool need_filter, bool has_null_map>
|
|
|
|
IColumn::Filter joinRightColumnsSwitchMultipleDisjuncts(
|
|
|
|
std::vector<KeyGetter> && key_getter_vector,
|
|
|
|
const std::vector<const Map *> & mapv,
|
|
|
|
AddedColumns & added_columns,
|
|
|
|
const std::vector<ConstNullMapPtr> & null_map [[maybe_unused]],
|
|
|
|
JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]])
|
|
|
|
{
|
|
|
|
return mapv.size() > 1
|
|
|
|
? joinRightColumns<KIND, STRICTNESS, KeyGetter, Map, true, true, true>(std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, null_map, used_flags)
|
|
|
|
: joinRightColumns<KIND, STRICTNESS, KeyGetter, Map, true, true, false>(std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, null_map, used_flags);
|
|
|
|
}
|
|
|
|
|
2019-03-20 14:49:05 +00:00
|
|
|
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS, typename KeyGetter, typename Map>
|
2021-02-04 14:46:36 +00:00
|
|
|
IColumn::Filter joinRightColumnsSwitchNullability(
|
2021-06-25 12:03:10 +00:00
|
|
|
std::vector<KeyGetter> && key_getter_vector,
|
|
|
|
const std::vector<const Map *>/***/ & mapv,
|
|
|
|
AddedColumns & added_columns,
|
|
|
|
const std::vector<ConstNullMapPtr> & null_map,
|
|
|
|
JoinStuff::JoinUsedFlags & used_flags)
|
2019-03-20 14:49:05 +00:00
|
|
|
{
|
2019-12-03 12:55:20 +00:00
|
|
|
if (added_columns.need_filter)
|
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
if (!null_map.empty())
|
|
|
|
return joinRightColumnsSwitchMultipleDisjuncts<KIND, STRICTNESS, KeyGetter, Map, true, true>(std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, null_map, used_flags);
|
2019-12-03 12:55:20 +00:00
|
|
|
else
|
2021-06-25 12:03:10 +00:00
|
|
|
return joinRightColumnsSwitchMultipleDisjuncts<KIND, STRICTNESS, KeyGetter, Map, true, false>(std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, null_map, used_flags);
|
2019-12-03 12:55:20 +00:00
|
|
|
}
|
2019-03-20 14:49:05 +00:00
|
|
|
else
|
2019-12-03 12:55:20 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
if (!null_map.empty())
|
|
|
|
return joinRightColumnsSwitchMultipleDisjuncts<KIND, STRICTNESS, KeyGetter, Map, false, true>(std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, null_map, used_flags);
|
2019-12-03 12:55:20 +00:00
|
|
|
else
|
2021-06-25 12:03:10 +00:00
|
|
|
return joinRightColumnsSwitchMultipleDisjuncts<KIND, STRICTNESS, KeyGetter, Map, false, false>(std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, null_map, used_flags);
|
2019-12-03 12:55:20 +00:00
|
|
|
}
|
2019-03-20 14:49:05 +00:00
|
|
|
}
|
2019-03-19 16:53:36 +00:00
|
|
|
|
2019-03-20 14:49:05 +00:00
|
|
|
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS, typename Maps>
|
2021-02-04 14:46:36 +00:00
|
|
|
IColumn::Filter switchJoinRightColumns(
|
2021-06-25 12:03:10 +00:00
|
|
|
const std::vector<const Maps*> & mapv,
|
2021-06-23 10:13:11 +00:00
|
|
|
AddedColumns & added_columns,
|
|
|
|
HashJoin::Type type,
|
2021-06-25 12:03:10 +00:00
|
|
|
const std::vector<ConstNullMapPtr> & null_map,
|
2021-06-23 10:13:11 +00:00
|
|
|
JoinStuff::JoinUsedFlags & used_flags)
|
2019-03-20 14:49:05 +00:00
|
|
|
{
|
2021-06-23 10:13:11 +00:00
|
|
|
constexpr bool is_asof_join = STRICTNESS == ASTTableJoin::Strictness::Asof;
|
2019-03-20 14:49:05 +00:00
|
|
|
switch (type)
|
|
|
|
{
|
|
|
|
#define M(TYPE) \
|
2020-04-07 09:48:47 +00:00
|
|
|
case HashJoin::Type::TYPE: \
|
2021-06-25 12:03:10 +00:00
|
|
|
{ \
|
|
|
|
using AMapTypeVal = const typename std::remove_reference_t<decltype(Maps::TYPE)>::element_type; \
|
|
|
|
using KeyGetter = typename KeyGetterForType<HashJoin::Type::TYPE, AMapTypeVal>::Type; \
|
|
|
|
std::vector<const AMapTypeVal*> a_map_type_vector(mapv.size()); \
|
|
|
|
std::vector<KeyGetter> key_getter_vector; \
|
|
|
|
size_t disjunct_num = added_columns.key_columns.size(); \
|
|
|
|
for (size_t d = 0; d < disjunct_num; ++d) \
|
|
|
|
{ \
|
|
|
|
a_map_type_vector[d] = mapv[d]->TYPE.get(); \
|
|
|
|
key_getter_vector.push_back(std::move(createKeyGetter<KeyGetter, is_asof_join>(added_columns.key_columns[d], added_columns.key_sizes[d]))); \
|
|
|
|
} \
|
2021-06-23 10:13:11 +00:00
|
|
|
return joinRightColumnsSwitchNullability<KIND, STRICTNESS, KeyGetter>( \
|
2021-06-25 12:03:10 +00:00
|
|
|
std::move(key_getter_vector), a_map_type_vector, added_columns, null_map, used_flags); \
|
|
|
|
}
|
2019-03-20 14:49:05 +00:00
|
|
|
APPLY_FOR_JOIN_VARIANTS(M)
|
|
|
|
#undef M
|
|
|
|
|
|
|
|
default:
|
2019-08-05 14:03:14 +00:00
|
|
|
throw Exception("Unsupported JOIN keys. Type: " + toString(static_cast<UInt32>(type)), ErrorCodes::UNSUPPORTED_JOIN_KEYS);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2017-03-28 06:51:22 +00:00
|
|
|
}
|
|
|
|
|
2020-04-08 18:59:52 +00:00
|
|
|
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS>
|
2020-04-13 17:03:11 +00:00
|
|
|
IColumn::Filter dictionaryJoinRightColumns(const TableJoin & table_join, AddedColumns & added_columns, const ConstNullMapPtr & null_map)
|
2020-04-08 18:59:52 +00:00
|
|
|
{
|
|
|
|
if constexpr (KIND == ASTTableJoin::Kind::Left &&
|
|
|
|
(STRICTNESS == ASTTableJoin::Strictness::Any ||
|
|
|
|
STRICTNESS == ASTTableJoin::Strictness::Semi ||
|
|
|
|
STRICTNESS == ASTTableJoin::Strictness::Anti))
|
|
|
|
{
|
2021-06-23 10:13:11 +00:00
|
|
|
assert(added_columns.key_columns.size() == 1);
|
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
// JoinStuff::JoinUsedFlags flags;
|
|
|
|
// KeyGetterForDict key_getter(table_join, added_columns.key_columns);
|
|
|
|
// return joinRightColumnsSwitchNullability<KIND, STRICTNESS, KeyGetterForDict>(
|
|
|
|
// std::move(key_getter), nullptr, added_columns, null_map, flags);
|
|
|
|
std::vector<const TableJoin*> maps_vector;
|
|
|
|
maps_vector.push_back(&table_join);
|
|
|
|
|
|
|
|
std::vector<ConstNullMapPtr> null_maps_vector;
|
|
|
|
null_maps_vector.push_back(null_map);
|
|
|
|
|
2021-02-04 14:46:36 +00:00
|
|
|
JoinStuff::JoinUsedFlags flags;
|
2021-06-25 12:03:10 +00:00
|
|
|
std::vector<KeyGetterForDict> key_getter_vector;
|
|
|
|
key_getter_vector.push_back(KeyGetterForDict(table_join, added_columns.key_columns[0]));
|
|
|
|
// KeyGetterForDict key_getter(table_join, added_columns.key_columns);
|
|
|
|
return joinRightColumnsSwitchNullability<KIND, STRICTNESS, KeyGetterForDict>(std::move(key_getter_vector), maps_vector, added_columns, null_maps_vector, flags);
|
2020-04-08 18:59:52 +00:00
|
|
|
}
|
|
|
|
|
2021-09-14 12:05:09 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: {} {}", STRICTNESS, KIND);
|
2020-04-08 18:59:52 +00:00
|
|
|
}
|
|
|
|
|
2019-03-20 14:49:05 +00:00
|
|
|
} /// nameless
|
|
|
|
|
2016-07-22 20:39:28 +00:00
|
|
|
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS, typename Maps>
|
2021-06-25 12:03:10 +00:00
|
|
|
std::unique_ptr<AddedColumns> HashJoin::makeAddedColumns(
|
2018-11-30 14:49:35 +00:00
|
|
|
Block & block,
|
2021-06-25 12:03:10 +00:00
|
|
|
const NamesVector & key_names_left_vector,
|
2018-11-30 14:49:35 +00:00
|
|
|
const Block & block_with_columns_to_add,
|
2021-06-25 12:03:10 +00:00
|
|
|
const std::vector<const Maps*> & maps_,
|
2021-06-15 08:34:53 +00:00
|
|
|
bool is_join_get) const
|
2014-06-12 04:04:47 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
constexpr JoinFeatures<KIND, STRICTNESS> jf;
|
2019-12-02 18:07:27 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
ColumnRawPtrsVector left_key_columns_vector;
|
|
|
|
std::vector<ConstNullMapPtr> null_map_vector;
|
|
|
|
std::vector<ColumnPtr> null_map_holder_vector;
|
|
|
|
std::vector<Columns> materialized_keys_vector;
|
|
|
|
std::vector<ColumnPtr> join_mask_column_vector; /// Only rows where mask == true can be joined
|
2019-11-07 21:32:44 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
size_t disjunct = 0;
|
2019-11-07 21:32:44 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
for (const auto & key_names_left_part : key_names_left_vector)
|
|
|
|
{
|
|
|
|
/// Rare case, when keys are constant or low cardinality. To avoid code bloat, simply materialize them.
|
|
|
|
materialized_keys_vector.emplace_back(JoinCommon::materializeColumns(block, key_names_left_part));
|
|
|
|
ColumnRawPtrs left_key_columns = JoinCommon::getRawPointers(materialized_keys_vector.back());
|
|
|
|
left_key_columns_vector.push_back(std::move(left_key_columns));
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
/// Keys with NULL value in any column won't join to anything.
|
|
|
|
null_map_vector.emplace_back();
|
|
|
|
null_map_holder_vector.push_back(extractNestedColumnsAndNullMap(left_key_columns_vector.back(), null_map_vector.back()));
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
join_mask_column_vector.push_back(JoinCommon::getColumnAsMask(block, condition_mask_column_name_left[disjunct++]));
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-04-02 17:37:49 +00:00
|
|
|
/** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized.
|
|
|
|
* Because if they are constants, then in the "not joined" rows, they may have different values
|
|
|
|
* - default values, which can differ from the values of these constants.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2021-06-25 12:03:10 +00:00
|
|
|
if constexpr (jf.right || jf.full)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-09-12 12:59:53 +00:00
|
|
|
materializeBlockInplace(block);
|
2019-09-11 18:03:21 +00:00
|
|
|
|
2019-09-12 14:09:05 +00:00
|
|
|
if (nullable_left_side)
|
2019-09-11 18:03:21 +00:00
|
|
|
JoinCommon::convertColumnsToNullable(block);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2017-12-14 12:21:01 +00:00
|
|
|
/** For LEFT/INNER JOIN, the saved blocks do not contain keys.
|
|
|
|
* For FULL/RIGHT JOIN, the saved blocks contain keys;
|
|
|
|
* but they will not be used at this stage of joining (and will be in `AdderNonJoined`), and they need to be skipped.
|
2019-03-31 23:09:00 +00:00
|
|
|
* For ASOF, the last column is used as the ASOF column
|
2017-12-14 12:21:01 +00:00
|
|
|
*/
|
2021-06-25 12:03:10 +00:00
|
|
|
auto added_columns = std::make_unique<AddedColumns>(
|
2021-07-21 17:03:33 +00:00
|
|
|
block_with_columns_to_add,
|
2021-06-25 12:03:10 +00:00
|
|
|
block, savedBlockSample(),
|
2021-07-21 17:03:33 +00:00
|
|
|
*this,
|
2021-06-25 12:03:10 +00:00
|
|
|
left_key_columns_vector,
|
2021-07-21 17:03:33 +00:00
|
|
|
key_sizes,
|
2021-06-25 12:03:10 +00:00
|
|
|
join_mask_column_vector,
|
|
|
|
jf.is_asof_join,
|
2021-07-21 17:03:33 +00:00
|
|
|
is_join_get);
|
|
|
|
|
2019-12-03 12:55:20 +00:00
|
|
|
bool has_required_right_keys = (required_right_keys.columns() != 0);
|
2021-06-25 12:03:10 +00:00
|
|
|
added_columns->need_filter = jf.need_filter || has_required_right_keys;
|
|
|
|
|
|
|
|
added_columns->row_filter = overDictionary() ?
|
|
|
|
dictionaryJoinRightColumns<KIND, STRICTNESS>(*table_join, *added_columns, null_map_vector[0]):
|
|
|
|
switchJoinRightColumns<KIND, STRICTNESS>(maps_, *added_columns, data->type, null_map_vector, used_flags);
|
2019-03-20 12:08:38 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
for (size_t i = 0; i < added_columns->size(); ++i)
|
|
|
|
block.insert(added_columns->moveColumn(i));
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
return added_columns;
|
|
|
|
}
|
2019-03-27 13:37:11 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS>
|
|
|
|
void HashJoin::joinBlockImpl(
|
|
|
|
Block & block,
|
|
|
|
std::unique_ptr<AddedColumns> added_columns,
|
|
|
|
size_t existing_columns) const
|
|
|
|
{
|
|
|
|
JoinFeatures<KIND, STRICTNESS> jf;
|
|
|
|
bool has_required_right_keys = (required_right_keys.columns() != 0);
|
2019-07-31 20:19:52 +00:00
|
|
|
std::vector<size_t> right_keys_to_replicate [[maybe_unused]];
|
2018-07-24 10:00:53 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
if constexpr (jf.need_filter)
|
2019-07-31 20:19:52 +00:00
|
|
|
{
|
|
|
|
/// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones.
|
|
|
|
for (size_t i = 0; i < existing_columns; ++i)
|
2021-06-25 12:03:10 +00:00
|
|
|
block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(added_columns->row_filter, -1);
|
2018-07-24 10:00:53 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
/// Add join key columns from right block if needed
|
|
|
|
/// using value from left table because of equality
|
2019-11-01 17:41:07 +00:00
|
|
|
for (size_t i = 0; i < required_right_keys.columns(); ++i)
|
2018-07-24 10:00:53 +00:00
|
|
|
{
|
2019-11-01 17:41:07 +00:00
|
|
|
const auto & right_key = required_right_keys.getByPosition(i);
|
2021-06-25 12:03:10 +00:00
|
|
|
// renamed ???
|
|
|
|
if (!block.findByName(right_key.name))
|
|
|
|
{
|
|
|
|
const auto & left_name = required_right_keys_sources[i];
|
|
|
|
|
|
|
|
/// asof column is already in block.
|
|
|
|
if (jf.is_asof_join && right_key.name == key_names_right[0].back())
|
|
|
|
continue;
|
|
|
|
|
|
|
|
const auto & col = block.getByName(left_name);
|
|
|
|
bool is_nullable = nullable_right_side || right_key.type->isNullable();
|
|
|
|
auto right_col_name = getTableJoin().renamedRightColumnName(right_key.name);
|
|
|
|
ColumnWithTypeAndName right_col(col.column, col.type, right_col_name);
|
|
|
|
if (right_col.type->lowCardinality() != right_key.type->lowCardinality())
|
|
|
|
JoinCommon::changeLowCardinalityInplace(right_col);
|
|
|
|
right_col = correctNullability(std::move(right_col), is_nullable);
|
|
|
|
block.insert(right_col);
|
|
|
|
}
|
2019-01-29 12:38:53 +00:00
|
|
|
}
|
|
|
|
}
|
2019-12-03 12:55:20 +00:00
|
|
|
else if (has_required_right_keys)
|
2019-01-29 12:38:53 +00:00
|
|
|
{
|
2019-07-31 20:19:52 +00:00
|
|
|
/// Some trash to represent IColumn::Filter as ColumnUInt8 needed for ColumnNullable::applyNullMap()
|
|
|
|
auto null_map_filter_ptr = ColumnUInt8::create();
|
2019-08-21 02:28:04 +00:00
|
|
|
ColumnUInt8 & null_map_filter = assert_cast<ColumnUInt8 &>(*null_map_filter_ptr);
|
2021-06-25 12:03:10 +00:00
|
|
|
null_map_filter.getData().swap(added_columns->row_filter);
|
2019-07-31 20:19:52 +00:00
|
|
|
const IColumn::Filter & filter = null_map_filter.getData();
|
2019-10-29 19:39:42 +00:00
|
|
|
/// Add join key columns from right block if needed.
|
2019-11-01 17:41:07 +00:00
|
|
|
for (size_t i = 0; i < required_right_keys.columns(); ++i)
|
2019-01-29 12:38:53 +00:00
|
|
|
{
|
2019-11-01 17:41:07 +00:00
|
|
|
const auto & right_key = required_right_keys.getByPosition(i);
|
2021-06-25 12:03:10 +00:00
|
|
|
auto right_col_name = getTableJoin().renamedRightColumnName(right_key.name);
|
|
|
|
if (!block.findByName(right_col_name /*right_key.name*/))
|
|
|
|
{
|
|
|
|
const auto & left_name = required_right_keys_sources[i];
|
|
|
|
/// asof column is already in block.
|
|
|
|
if (jf.is_asof_join && right_key.name == key_names_right[0].back())
|
|
|
|
continue;
|
2018-09-28 13:06:09 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
const auto & col = block.getByName(left_name);
|
|
|
|
bool is_nullable = nullable_right_side || right_key.type->isNullable();
|
2020-11-03 11:28:28 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
ColumnPtr thin_column = filterWithBlanks(col.column, filter);
|
2018-09-28 13:06:09 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
ColumnWithTypeAndName right_col(thin_column, col.type, right_col_name);
|
|
|
|
if (right_col.type->lowCardinality() != right_key.type->lowCardinality())
|
|
|
|
JoinCommon::changeLowCardinalityInplace(right_col);
|
|
|
|
right_col = correctNullability(std::move(right_col), is_nullable, null_map_filter);
|
|
|
|
block.insert(right_col);
|
2021-05-15 11:39:13 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
if constexpr (jf.need_replication)
|
|
|
|
right_keys_to_replicate.push_back(block.getPositionByName(right_key.name));
|
|
|
|
}
|
2018-07-24 10:00:53 +00:00
|
|
|
}
|
2019-07-31 20:19:52 +00:00
|
|
|
}
|
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
if constexpr (jf.need_replication)
|
2019-07-31 20:19:52 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
std::unique_ptr<IColumn::Offsets> & offsets_to_replicate = added_columns->offsets_to_replicate;
|
2018-09-28 13:06:09 +00:00
|
|
|
|
2019-01-29 12:38:53 +00:00
|
|
|
/// If ALL ... JOIN - we replicate all the columns except the new ones.
|
2018-09-28 13:06:09 +00:00
|
|
|
for (size_t i = 0; i < existing_columns; ++i)
|
|
|
|
block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->replicate(*offsets_to_replicate);
|
2019-07-31 20:19:52 +00:00
|
|
|
|
|
|
|
/// Replicate additional right keys
|
|
|
|
for (size_t pos : right_keys_to_replicate)
|
2021-06-25 12:03:10 +00:00
|
|
|
{
|
2019-07-31 20:19:52 +00:00
|
|
|
block.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate);
|
2021-06-25 12:03:10 +00:00
|
|
|
}
|
|
|
|
|
2018-09-28 13:06:09 +00:00
|
|
|
}
|
2014-06-18 18:31:35 +00:00
|
|
|
}
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) const
|
2015-07-23 20:23:24 +00:00
|
|
|
{
|
2020-04-06 13:39:57 +00:00
|
|
|
size_t max_joined_block_rows = table_join->maxJoinedBlockRows();
|
|
|
|
size_t start_left_row = 0;
|
|
|
|
size_t start_right_block = 0;
|
2021-06-25 12:03:10 +00:00
|
|
|
|
2020-04-06 13:39:57 +00:00
|
|
|
if (not_processed)
|
|
|
|
{
|
|
|
|
auto & continuation = static_cast<NotProcessedCrossJoin &>(*not_processed);
|
|
|
|
start_left_row = continuation.left_position;
|
|
|
|
start_right_block = continuation.right_block;
|
|
|
|
not_processed.reset();
|
|
|
|
}
|
|
|
|
|
2017-12-15 19:17:15 +00:00
|
|
|
size_t num_existing_columns = block.columns();
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t num_columns_to_add = sample_block_with_columns_to_add.columns();
|
|
|
|
|
2020-04-06 13:39:57 +00:00
|
|
|
ColumnRawPtrs src_left_columns;
|
|
|
|
MutableColumns dst_columns;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
{
|
2020-04-06 13:39:57 +00:00
|
|
|
src_left_columns.reserve(num_existing_columns);
|
|
|
|
dst_columns.reserve(num_existing_columns + num_columns_to_add);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-04-06 13:39:57 +00:00
|
|
|
for (const ColumnWithTypeAndName & left_column : block)
|
|
|
|
{
|
|
|
|
src_left_columns.push_back(left_column.column.get());
|
|
|
|
dst_columns.emplace_back(src_left_columns.back()->cloneEmpty());
|
|
|
|
}
|
|
|
|
|
|
|
|
for (const ColumnWithTypeAndName & right_column : sample_block_with_columns_to_add)
|
|
|
|
dst_columns.emplace_back(right_column.column->cloneEmpty());
|
|
|
|
|
|
|
|
for (auto & dst : dst_columns)
|
|
|
|
dst->reserve(max_joined_block_rows);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2020-04-06 13:39:57 +00:00
|
|
|
size_t rows_left = block.rows();
|
|
|
|
size_t rows_added = 0;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-04-06 13:39:57 +00:00
|
|
|
for (size_t left_row = start_left_row; left_row < rows_left; ++left_row)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-04-06 13:39:57 +00:00
|
|
|
size_t block_number = 0;
|
2021-06-25 12:03:10 +00:00
|
|
|
for (const auto & block_wrapper : data->blocks)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
const Block & block_right = block_wrapper.block;
|
|
|
|
|
2020-04-06 13:39:57 +00:00
|
|
|
++block_number;
|
|
|
|
if (block_number < start_right_block)
|
|
|
|
continue;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t rows_right = block_right.rows();
|
2020-04-06 13:39:57 +00:00
|
|
|
rows_added += rows_right;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
for (size_t col_num = 0; col_num < num_existing_columns; ++col_num)
|
2020-04-06 13:39:57 +00:00
|
|
|
dst_columns[col_num]->insertManyFrom(*src_left_columns[col_num], left_row, rows_right);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
for (size_t col_num = 0; col_num < num_columns_to_add; ++col_num)
|
|
|
|
{
|
2020-04-06 13:39:57 +00:00
|
|
|
const IColumn & column_right = *block_right.getByPosition(col_num).column;
|
|
|
|
dst_columns[num_existing_columns + col_num]->insertRangeFrom(column_right, 0, rows_right);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
2020-04-06 13:39:57 +00:00
|
|
|
|
|
|
|
start_right_block = 0;
|
|
|
|
|
|
|
|
if (rows_added > max_joined_block_rows)
|
|
|
|
{
|
|
|
|
not_processed = std::make_shared<NotProcessedCrossJoin>(
|
|
|
|
NotProcessedCrossJoin{{block.cloneEmpty()}, left_row, block_number + 1});
|
|
|
|
not_processed->block.swap(block);
|
|
|
|
break;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2020-04-06 13:39:57 +00:00
|
|
|
for (const ColumnWithTypeAndName & src_column : sample_block_with_columns_to_add)
|
|
|
|
block.insert(src_column);
|
|
|
|
|
2017-12-15 19:17:15 +00:00
|
|
|
block = block.cloneWithColumns(std::move(dst_columns));
|
2015-07-23 20:23:24 +00:00
|
|
|
}
|
|
|
|
|
2020-07-11 07:12:42 +00:00
|
|
|
DataTypePtr HashJoin::joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const
|
2020-07-23 07:47:17 +00:00
|
|
|
{
|
2020-07-11 07:12:42 +00:00
|
|
|
size_t num_keys = data_types.size();
|
|
|
|
if (right_table_keys.columns() != num_keys)
|
|
|
|
throw Exception(
|
|
|
|
"Number of arguments for function joinGet" + toString(or_null ? "OrNull" : "")
|
|
|
|
+ " doesn't match: passed, should be equal to " + toString(num_keys),
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
for (size_t i = 0; i < num_keys; ++i)
|
|
|
|
{
|
|
|
|
const auto & left_type_origin = data_types[i];
|
|
|
|
const auto & [c2, right_type_origin, right_name] = right_table_keys.safeGetByPosition(i);
|
2021-02-24 08:44:51 +00:00
|
|
|
auto left_type = removeNullable(recursiveRemoveLowCardinality(left_type_origin));
|
|
|
|
auto right_type = removeNullable(recursiveRemoveLowCardinality(right_type_origin));
|
2020-07-11 07:12:42 +00:00
|
|
|
if (!left_type->equals(*right_type))
|
|
|
|
throw Exception(
|
|
|
|
"Type mismatch in joinGet key " + toString(i) + ": found type " + left_type->getName() + ", while the needed type is "
|
|
|
|
+ right_type->getName(),
|
|
|
|
ErrorCodes::TYPE_MISMATCH);
|
|
|
|
}
|
|
|
|
|
2018-11-30 14:49:35 +00:00
|
|
|
if (!sample_block_with_columns_to_add.has(column_name))
|
2020-06-02 20:26:41 +00:00
|
|
|
throw Exception("StorageJoin doesn't contain column " + column_name, ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
|
2020-07-11 07:12:42 +00:00
|
|
|
|
2020-04-12 03:18:21 +00:00
|
|
|
auto elem = sample_block_with_columns_to_add.getByName(column_name);
|
2020-04-07 14:52:32 +00:00
|
|
|
if (or_null)
|
2020-04-12 03:25:47 +00:00
|
|
|
elem.type = makeNullable(elem.type);
|
2020-04-12 03:18:21 +00:00
|
|
|
return elem.type;
|
2018-11-30 14:49:35 +00:00
|
|
|
}
|
|
|
|
|
2021-02-25 09:31:22 +00:00
|
|
|
/// TODO: return multiple columns as named tuple
|
|
|
|
/// TODO: return array of values when strictness == ASTTableJoin::Strictness::All
|
|
|
|
ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block_with_columns_to_add) const
|
2018-11-30 14:49:35 +00:00
|
|
|
{
|
2021-02-25 09:31:22 +00:00
|
|
|
bool is_valid = (strictness == ASTTableJoin::Strictness::Any || strictness == ASTTableJoin::Strictness::RightAny)
|
|
|
|
&& kind == ASTTableJoin::Kind::Left;
|
|
|
|
if (!is_valid)
|
|
|
|
throw Exception("joinGet only supports StorageJoin of type Left Any", ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN);
|
|
|
|
|
2021-02-24 16:19:04 +00:00
|
|
|
/// Assemble the key block with correct names.
|
2020-07-11 07:12:42 +00:00
|
|
|
Block keys;
|
|
|
|
for (size_t i = 0; i < block.columns(); ++i)
|
|
|
|
{
|
|
|
|
auto key = block.getByPosition(i);
|
2021-06-25 12:03:10 +00:00
|
|
|
key.name = key_names_right.front()[i];
|
2020-07-11 07:12:42 +00:00
|
|
|
keys.insert(std::move(key));
|
|
|
|
}
|
|
|
|
|
2021-02-25 09:42:20 +00:00
|
|
|
static_assert(!MapGetter<ASTTableJoin::Kind::Left, ASTTableJoin::Strictness::Any>::flagged,
|
|
|
|
"joinGet are not protected from hash table changes between block processing");
|
2021-06-25 12:03:10 +00:00
|
|
|
|
|
|
|
|
|
|
|
size_t existing_columns = block.columns();
|
|
|
|
|
|
|
|
std::vector<const MapsOne*> maps_vector;
|
|
|
|
maps_vector.push_back(&std::get<MapsOne>(data->maps[0]));
|
|
|
|
|
|
|
|
auto added_columns = makeAddedColumns<ASTTableJoin::Kind::Left, ASTTableJoin::Strictness::Any>(
|
|
|
|
keys, key_names_right, block_with_columns_to_add, maps_vector, /* is_join_get */ true);
|
|
|
|
|
|
|
|
|
2020-07-11 07:12:42 +00:00
|
|
|
joinBlockImpl<ASTTableJoin::Kind::Left, ASTTableJoin::Strictness::Any>(
|
2021-06-25 12:03:10 +00:00
|
|
|
keys, std::move(added_columns), existing_columns);
|
2020-07-11 07:12:42 +00:00
|
|
|
return keys.getByPosition(keys.columns() - 1);
|
2018-11-30 14:49:35 +00:00
|
|
|
}
|
|
|
|
|
2021-09-13 13:35:17 +00:00
|
|
|
void HashJoin::checkTypesOfKeys(const Block & block) const
|
|
|
|
{
|
|
|
|
JoinCommon::checkTypesOfKeys(block, table_join->keyNamesLeft(), right_table_keys, key_names_right);
|
|
|
|
}
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed)
|
2014-06-18 18:31:35 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
for (size_t i = 0; i < key_names_left.size(); ++i)
|
|
|
|
{
|
|
|
|
JoinCommon::checkTypesOfKeys(block, key_names_left[i], condition_mask_column_name_left[i],
|
|
|
|
right_sample_block, key_names_right[i], condition_mask_column_name_right[i]);
|
|
|
|
}
|
|
|
|
|
|
|
|
if (kind == ASTTableJoin::Kind::Cross)
|
|
|
|
{
|
|
|
|
joinBlockImplCross(block, not_processed);
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
else if (kind == ASTTableJoin::Kind::Right || kind == ASTTableJoin::Kind::Full)
|
|
|
|
{
|
|
|
|
materializeBlockInplace(block);
|
|
|
|
|
|
|
|
if (nullable_left_side)
|
|
|
|
JoinCommon::convertColumnsToNullable(block);
|
|
|
|
}
|
|
|
|
|
|
|
|
AddedColumnsV added_columns_v;
|
|
|
|
size_t existing_columns = block.columns();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-04-08 18:59:52 +00:00
|
|
|
if (overDictionary())
|
|
|
|
{
|
|
|
|
using Kind = ASTTableJoin::Kind;
|
|
|
|
using Strictness = ASTTableJoin::Strictness;
|
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
auto & map = std::get<MapsOne>(data->maps[0]);
|
|
|
|
std::vector<const std::decay_t<decltype(map)>*> maps_vector;
|
|
|
|
maps_vector.push_back(&map);
|
|
|
|
|
2020-04-08 18:59:52 +00:00
|
|
|
if (kind == Kind::Left)
|
|
|
|
{
|
|
|
|
switch (strictness)
|
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
case Strictness::Any:
|
|
|
|
case Strictness::All:
|
|
|
|
{
|
|
|
|
auto added_columns = makeAddedColumns<Kind::Left, Strictness::Any>(
|
|
|
|
block, key_names_left, sample_block_with_columns_to_add, maps_vector);
|
|
|
|
joinBlockImpl<Kind::Left, Strictness::Any>(block, std::move(added_columns), existing_columns);
|
2020-04-08 18:59:52 +00:00
|
|
|
break;
|
2021-06-25 12:03:10 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
case Strictness::Semi:
|
|
|
|
{
|
|
|
|
auto added_columns = makeAddedColumns<Kind::Left, Strictness::Semi>(
|
|
|
|
block, key_names_left, sample_block_with_columns_to_add, maps_vector);
|
|
|
|
joinBlockImpl<Kind::Left, Strictness::Semi>(block, std::move(added_columns), existing_columns);
|
2020-04-08 18:59:52 +00:00
|
|
|
break;
|
2021-06-25 12:03:10 +00:00
|
|
|
}
|
|
|
|
case Strictness::Anti:
|
|
|
|
{
|
|
|
|
auto added_columns = makeAddedColumns<Kind::Left, Strictness::Anti>(
|
|
|
|
block, key_names_left, sample_block_with_columns_to_add, maps_vector);
|
|
|
|
joinBlockImpl<Kind::Left, Strictness::Anti>(block, std::move(added_columns), existing_columns);
|
2020-04-08 18:59:52 +00:00
|
|
|
break;
|
2021-06-25 12:03:10 +00:00
|
|
|
}
|
|
|
|
default:
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: dictionary + {} {}", strictness, kind);
|
2020-04-08 18:59:52 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
else if (kind == Kind::Inner && strictness == Strictness::All)
|
2021-06-25 12:03:10 +00:00
|
|
|
{
|
|
|
|
auto added_columns = makeAddedColumns<Kind::Left, Strictness::Semi>(
|
|
|
|
block, key_names_left/*[0]*/, sample_block_with_columns_to_add, maps_vector);
|
|
|
|
joinBlockImpl<Kind::Left, Strictness::Semi>(block, std::move(added_columns), existing_columns);
|
|
|
|
}
|
|
|
|
|
2020-04-08 18:59:52 +00:00
|
|
|
else
|
2021-06-25 12:03:10 +00:00
|
|
|
throw throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: {} {}", strictness, kind);
|
2020-04-08 18:59:52 +00:00
|
|
|
}
|
2021-06-25 12:03:10 +00:00
|
|
|
else
|
|
|
|
{
|
|
|
|
|
|
|
|
// MapsVariantPtrVector maps_vector;
|
|
|
|
std::vector<const std::decay_t<decltype(data->maps[0])>* > maps_vector;
|
|
|
|
|
|
|
|
for (size_t i = 0; i < key_names_left.size(); ++i)
|
|
|
|
{
|
|
|
|
// JoinCommon::checkTypesOfKeys(block, key_names_left[i], condition_mask_column_name_left[i],
|
|
|
|
// right_table_keys, key_names_right[i], condition_mask_column_name_right[i]);
|
|
|
|
maps_vector.push_back(&data->maps[i]);
|
|
|
|
}
|
|
|
|
std::unique_ptr<AddedColumns> added_columns;
|
|
|
|
|
|
|
|
joinDispatch(kind, strictness, maps_vector, [&](auto kind_, auto strictness_, auto & maps_vector_)
|
|
|
|
{
|
|
|
|
added_columns = makeAddedColumns<kind_, strictness_>(block, key_names_left, sample_block_with_columns_to_add, maps_vector_);
|
|
|
|
});
|
|
|
|
|
|
|
|
if (joinDispatch(kind, strictness, data->maps[0], [&](auto kind_, auto strictness_, auto &)
|
2019-01-14 21:40:02 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
joinBlockImpl<kind_, strictness_>(block, std::move(added_columns), existing_columns);
|
2018-12-30 15:54:45 +00:00
|
|
|
}))
|
2021-06-25 12:03:10 +00:00
|
|
|
{
|
|
|
|
/// Joined
|
|
|
|
}
|
|
|
|
else
|
|
|
|
throw throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: {} {}", strictness, kind);
|
2019-01-14 21:40:02 +00:00
|
|
|
}
|
2014-06-12 04:04:47 +00:00
|
|
|
}
|
|
|
|
|
2015-04-17 08:46:06 +00:00
|
|
|
template <typename Mapped>
|
2019-12-02 18:07:27 +00:00
|
|
|
struct AdderNonJoined
|
2015-04-17 08:46:06 +00:00
|
|
|
{
|
2019-03-26 19:46:03 +00:00
|
|
|
static void add(const Mapped & mapped, size_t & rows_added, MutableColumns & columns_right)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2021-02-04 14:46:36 +00:00
|
|
|
constexpr bool mapped_asof = std::is_same_v<Mapped, AsofRowRefs>;
|
|
|
|
[[maybe_unused]] constexpr bool mapped_one = std::is_same_v<Mapped, RowRef>;
|
2019-11-07 21:32:44 +00:00
|
|
|
|
2019-12-02 18:07:27 +00:00
|
|
|
if constexpr (mapped_asof)
|
2019-07-31 20:19:52 +00:00
|
|
|
{
|
2019-12-02 18:07:27 +00:00
|
|
|
/// Do nothing
|
2019-07-31 20:19:52 +00:00
|
|
|
}
|
2019-12-02 18:07:27 +00:00
|
|
|
else if constexpr (mapped_one)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-01-31 18:10:16 +00:00
|
|
|
for (size_t j = 0; j < columns_right.size(); ++j)
|
2019-07-31 20:19:52 +00:00
|
|
|
{
|
2019-12-02 18:07:27 +00:00
|
|
|
const auto & mapped_column = mapped.block->getByPosition(j).column;
|
|
|
|
columns_right[j]->insertFrom(*mapped_column, mapped.row_num);
|
2019-07-31 20:19:52 +00:00
|
|
|
}
|
2018-11-29 15:44:12 +00:00
|
|
|
|
|
|
|
++rows_added;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2019-12-02 18:07:27 +00:00
|
|
|
else
|
|
|
|
{
|
|
|
|
for (auto it = mapped.begin(); it.ok(); ++it)
|
|
|
|
{
|
|
|
|
for (size_t j = 0; j < columns_right.size(); ++j)
|
|
|
|
{
|
|
|
|
const auto & mapped_column = it->block->getByPosition(j).column;
|
|
|
|
columns_right[j]->insertFrom(*mapped_column, it->row_num);
|
|
|
|
}
|
2015-04-17 08:46:06 +00:00
|
|
|
|
2019-12-02 18:07:27 +00:00
|
|
|
++rows_added;
|
|
|
|
}
|
|
|
|
}
|
2019-03-20 16:58:28 +00:00
|
|
|
}
|
|
|
|
};
|
2015-04-17 08:46:06 +00:00
|
|
|
|
2019-12-02 18:07:27 +00:00
|
|
|
|
2017-04-02 17:37:49 +00:00
|
|
|
/// Stream from not joined earlier rows of the right table.
|
2021-06-25 12:03:10 +00:00
|
|
|
template<bool multiple_disjuncts>
|
2021-08-17 13:30:01 +00:00
|
|
|
class NotJoinedHash final : public NotJoinedBlocks::RightColumnsFiller
|
2015-04-17 08:46:06 +00:00
|
|
|
{
|
|
|
|
public:
|
2021-08-09 14:30:37 +00:00
|
|
|
NotJoinedHash(const HashJoin & parent_, UInt64 max_block_size_)
|
|
|
|
: parent(parent_), max_block_size(max_block_size_)
|
2020-07-10 18:10:06 +00:00
|
|
|
{}
|
2015-04-17 08:46:06 +00:00
|
|
|
|
2021-08-09 14:30:37 +00:00
|
|
|
Block getEmptyBlock() override { return parent.savedBlockSample().cloneEmpty(); }
|
|
|
|
|
2021-08-06 14:15:11 +00:00
|
|
|
size_t fillColumns(MutableColumns & columns_right) override
|
2019-11-01 17:41:07 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
// if (multiple_disjuncts && parent.nullable_right_side)
|
|
|
|
// {
|
|
|
|
// JoinCommon::convertColumnsToNullable(columns_right);
|
|
|
|
// }
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
size_t rows_added = 0;
|
2019-07-03 19:06:34 +00:00
|
|
|
auto fill_callback = [&](auto, auto strictness, auto & map)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-11-01 17:41:07 +00:00
|
|
|
rows_added = fillColumnsFromMap<strictness>(map, columns_right);
|
2019-07-03 19:06:34 +00:00
|
|
|
};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
if (!joinDispatch(parent.kind, parent.strictness, parent.data->maps.front(), fill_callback))
|
2021-09-14 12:05:09 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown JOIN strictness '{}' (must be on of: ANY, ALL, ASOF)", parent.strictness);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
if constexpr (!multiple_disjuncts)
|
|
|
|
{
|
|
|
|
fillNullsFromBlocks(columns_right, rows_added);
|
|
|
|
}
|
|
|
|
|
2021-08-06 14:15:11 +00:00
|
|
|
return rows_added;
|
|
|
|
}
|
2021-08-06 11:54:04 +00:00
|
|
|
|
2021-08-06 14:15:11 +00:00
|
|
|
private:
|
|
|
|
const HashJoin & parent;
|
|
|
|
UInt64 max_block_size;
|
2021-08-06 11:54:04 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
|
2021-08-06 14:15:11 +00:00
|
|
|
std::any position;
|
|
|
|
std::optional<HashJoin::BlockNullmapList::const_iterator> nulls_position;
|
2021-06-25 12:03:10 +00:00
|
|
|
std::optional<HashJoin::BlocksWithFlagsList::const_iterator> used_position;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-07-03 19:06:34 +00:00
|
|
|
template <ASTTableJoin::Strictness STRICTNESS, typename Maps>
|
|
|
|
size_t fillColumnsFromMap(const Maps & maps, MutableColumns & columns_keys_and_right)
|
|
|
|
{
|
2019-12-19 15:50:28 +00:00
|
|
|
switch (parent.data->type)
|
2019-07-03 19:06:34 +00:00
|
|
|
{
|
|
|
|
#define M(TYPE) \
|
2020-04-07 09:48:47 +00:00
|
|
|
case HashJoin::Type::TYPE: \
|
2019-07-03 19:06:34 +00:00
|
|
|
return fillColumns<STRICTNESS>(*maps.TYPE, columns_keys_and_right);
|
|
|
|
APPLY_FOR_JOIN_VARIANTS(M)
|
|
|
|
#undef M
|
|
|
|
default:
|
2019-12-19 15:50:28 +00:00
|
|
|
throw Exception("Unsupported JOIN keys. Type: " + toString(static_cast<UInt32>(parent.data->type)),
|
2019-08-05 14:03:14 +00:00
|
|
|
ErrorCodes::UNSUPPORTED_JOIN_KEYS);
|
2019-07-03 19:06:34 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
__builtin_unreachable();
|
|
|
|
}
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
template <ASTTableJoin::Strictness STRICTNESS, typename Map>
|
2019-03-26 19:46:03 +00:00
|
|
|
size_t fillColumns(const Map & map, MutableColumns & columns_keys_and_right)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
size_t rows_added = 0;
|
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
if constexpr (multiple_disjuncts)
|
|
|
|
{
|
|
|
|
if (!used_position.has_value())
|
|
|
|
used_position = parent.data->blocks.begin();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
auto end = parent.data->blocks.end();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
for (auto & it = *used_position; it != end && rows_added < max_block_size; ++it)
|
|
|
|
{
|
|
|
|
const HashJoin::BlockWithFlags & block_with_flags = *it;
|
|
|
|
|
|
|
|
for (size_t row = 0; row < block_with_flags.flags.size(); ++row)
|
|
|
|
{
|
|
|
|
if (!block_with_flags.flags[row])
|
|
|
|
{
|
|
|
|
for (size_t colnum = 0; colnum < columns_keys_and_right.size(); ++colnum)
|
|
|
|
{
|
|
|
|
auto clmn = block_with_flags.block.getByPosition(colnum).column;
|
|
|
|
columns_keys_and_right[colnum]->insertFrom(*clmn, row);
|
|
|
|
}
|
|
|
|
|
|
|
|
++rows_added;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
using Mapped = typename Map::mapped_type;
|
|
|
|
using Iterator = typename Map::const_iterator;
|
2019-12-02 18:07:27 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
if (!position.has_value())
|
|
|
|
position = std::make_any<Iterator>(map.begin());
|
|
|
|
|
|
|
|
Iterator & it = std::any_cast<Iterator &>(position);
|
|
|
|
auto end = map.end();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
for (; it != end; ++it)
|
2018-01-05 16:12:35 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
const Mapped & mapped = it->getMapped();
|
|
|
|
|
|
|
|
size_t off = map.offsetInternal(it.getPtr());
|
|
|
|
if (parent.isUsed(off))
|
|
|
|
continue;
|
|
|
|
AdderNonJoined<Mapped>::add(mapped, rows_added, columns_keys_and_right);
|
|
|
|
|
|
|
|
|
|
|
|
if (rows_added >= max_block_size)
|
|
|
|
{
|
|
|
|
++it;
|
|
|
|
break;
|
|
|
|
}
|
2018-01-05 16:12:35 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return rows_added;
|
|
|
|
}
|
2019-03-29 18:07:22 +00:00
|
|
|
|
2019-07-03 19:06:34 +00:00
|
|
|
void fillNullsFromBlocks(MutableColumns & columns_keys_and_right, size_t & rows_added)
|
|
|
|
{
|
2019-07-04 12:12:39 +00:00
|
|
|
if (!nulls_position.has_value())
|
2019-12-19 15:50:28 +00:00
|
|
|
nulls_position = parent.data->blocks_nullmaps.begin();
|
2019-07-03 19:06:34 +00:00
|
|
|
|
2019-12-19 15:50:28 +00:00
|
|
|
auto end = parent.data->blocks_nullmaps.end();
|
2019-07-03 19:06:34 +00:00
|
|
|
|
2019-07-04 12:12:39 +00:00
|
|
|
for (auto & it = *nulls_position; it != end && rows_added < max_block_size; ++it)
|
2019-07-03 19:06:34 +00:00
|
|
|
{
|
2019-07-04 12:12:39 +00:00
|
|
|
const Block * block = it->first;
|
2019-08-21 02:28:04 +00:00
|
|
|
const NullMap & nullmap = assert_cast<const ColumnUInt8 &>(*it->second).getData();
|
2019-07-03 19:06:34 +00:00
|
|
|
|
|
|
|
for (size_t row = 0; row < nullmap.size(); ++row)
|
|
|
|
{
|
|
|
|
if (nullmap[row])
|
|
|
|
{
|
|
|
|
for (size_t col = 0; col < columns_keys_and_right.size(); ++col)
|
|
|
|
columns_keys_and_right[col]->insertFrom(*block->getByPosition(col).column, row);
|
|
|
|
++rows_added;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2015-04-17 08:46:06 +00:00
|
|
|
};
|
|
|
|
|
2021-08-17 13:30:01 +00:00
|
|
|
std::shared_ptr<NotJoinedBlocks> HashJoin::getNonJoinedBlocks(const Block & result_sample_block, UInt64 max_block_size) const
|
2015-04-17 08:46:06 +00:00
|
|
|
{
|
2019-12-02 18:07:27 +00:00
|
|
|
if (table_join->strictness() == ASTTableJoin::Strictness::Asof ||
|
2021-08-06 14:15:11 +00:00
|
|
|
table_join->strictness() == ASTTableJoin::Strictness::Semi ||
|
|
|
|
!isRightOrFull(table_join->kind()))
|
|
|
|
{
|
2019-12-02 18:07:27 +00:00
|
|
|
return {};
|
2021-08-06 14:15:11 +00:00
|
|
|
}
|
2021-06-25 12:03:10 +00:00
|
|
|
bool multiple_disjuncts = key_names_right.size() > 1;
|
|
|
|
|
|
|
|
if (multiple_disjuncts)
|
|
|
|
{
|
|
|
|
/// ... calculate `left_columns_count` ...
|
|
|
|
// throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "TODO");
|
|
|
|
size_t left_columns_count = result_sample_block.columns() - required_right_keys.columns() - sample_block_with_columns_to_add.columns();
|
|
|
|
auto non_joined = std::make_unique<NotJoinedHash<true>>(*this, max_block_size);
|
|
|
|
return std::make_shared<NotJoinedBlocks>(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap());
|
2019-12-02 18:07:27 +00:00
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
size_t left_columns_count = result_sample_block.columns() - required_right_keys.columns() - sample_block_with_columns_to_add.columns();
|
|
|
|
auto non_joined = std::make_unique<NotJoinedHash<false>>(*this, max_block_size);
|
|
|
|
return std::make_shared<NotJoinedBlocks>(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap());
|
|
|
|
}
|
2015-04-17 08:46:06 +00:00
|
|
|
}
|
|
|
|
|
2021-02-04 14:46:36 +00:00
|
|
|
void HashJoin::reuseJoinedData(const HashJoin & join)
|
|
|
|
{
|
|
|
|
data = join.data;
|
2021-04-28 17:32:12 +00:00
|
|
|
from_storage_join = true;
|
2021-06-25 12:03:10 +00:00
|
|
|
for (auto & map : data->maps)
|
2021-02-04 14:46:36 +00:00
|
|
|
{
|
2021-06-25 12:03:10 +00:00
|
|
|
joinDispatch(kind, strictness, map, [this](auto kind_, auto strictness_, auto & map_)
|
|
|
|
{
|
|
|
|
used_flags.reinit<kind_, strictness_>(map_.getBufferSizeInCells(data->type) + 1);
|
|
|
|
});
|
|
|
|
}
|
2021-02-04 14:46:36 +00:00
|
|
|
}
|
2014-06-12 02:31:30 +00:00
|
|
|
}
|