mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
BlockWithFlags removed from vdimir's change
This commit is contained in:
parent
6b6592fda7
commit
24e3e04203
@ -548,7 +548,7 @@ size_t HashJoin::getTotalRowCount() const
|
||||
if (data->type == Type::CROSS)
|
||||
{
|
||||
for (const auto & block : data->blocks)
|
||||
res += block.block.rows();
|
||||
res += block.rows();
|
||||
}
|
||||
else if (data->type != Type::DICT)
|
||||
{
|
||||
@ -568,7 +568,7 @@ size_t HashJoin::getTotalByteCount() const
|
||||
if (data->type == Type::CROSS)
|
||||
{
|
||||
for (const auto & block : data->blocks)
|
||||
res += block.block.bytes();
|
||||
res += block.bytes();
|
||||
}
|
||||
else if (data->type != Type::DICT)
|
||||
{
|
||||
@ -809,8 +809,8 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits)
|
||||
throw DB::Exception("addJoinedBlock called when HashJoin locked to prevent updates",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
BlockWithFlags & stored_block_with_flags = data->blocks.emplace_back(std::move(structured_block));
|
||||
Block * stored_block = &stored_block_with_flags.block;
|
||||
data->blocks.emplace_back(std::move(structured_block));
|
||||
Block * stored_block = &data->blocks.back();
|
||||
|
||||
if (rows)
|
||||
data->empty = false;
|
||||
@ -1694,10 +1694,8 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed)
|
||||
for (size_t left_row = start_left_row; left_row < rows_left; ++left_row)
|
||||
{
|
||||
size_t block_number = 0;
|
||||
for (const auto & block_wrapper : data->blocks)
|
||||
for (const Block & block_right : data->blocks)
|
||||
{
|
||||
const Block & block_right = block_wrapper.block;
|
||||
|
||||
++block_number;
|
||||
if (block_number < start_right_block)
|
||||
continue;
|
||||
@ -1964,11 +1962,6 @@ public:
|
||||
|
||||
size_t fillColumns(MutableColumns & columns_right) override
|
||||
{
|
||||
// if (multiple_disjuncts && parent.nullable_right_side)
|
||||
// {
|
||||
// JoinCommon::convertColumnsToNullable(columns_right);
|
||||
// }
|
||||
|
||||
size_t rows_added = 0;
|
||||
auto fill_callback = [&](auto, auto strictness, auto & map)
|
||||
{
|
||||
@ -1990,10 +1983,9 @@ private:
|
||||
const HashJoin & parent;
|
||||
UInt64 max_block_size;
|
||||
|
||||
|
||||
std::any position;
|
||||
std::optional<HashJoin::BlockNullmapList::const_iterator> nulls_position;
|
||||
std::optional<HashJoin::BlocksWithFlagsList::const_iterator> used_position;
|
||||
std::optional<BlocksList::const_iterator> used_position;
|
||||
|
||||
template <ASTTableJoin::Strictness STRICTNESS, typename Maps>
|
||||
size_t fillColumnsFromMap(const Maps & maps, MutableColumns & columns_keys_and_right)
|
||||
@ -2027,7 +2019,7 @@ private:
|
||||
|
||||
for (auto & it = *used_position; it != end && rows_added < max_block_size; ++it)
|
||||
{
|
||||
const Block & mapped_block = it->block;
|
||||
const Block & mapped_block = *it;
|
||||
|
||||
for (size_t row = 0; row < mapped_block.rows(); ++row)
|
||||
{
|
||||
|
@ -322,18 +322,6 @@ public:
|
||||
using RawBlockPtr = const Block *;
|
||||
using BlockNullmapList = std::deque<std::pair<RawBlockPtr, ColumnPtr>>;
|
||||
|
||||
struct BlockWithFlags
|
||||
{
|
||||
explicit BlockWithFlags(Block && block_)
|
||||
: block(std::move(block_))
|
||||
{}
|
||||
|
||||
Block block;
|
||||
bool empty() const { return !block; }
|
||||
};
|
||||
|
||||
using BlocksWithFlagsList = std::list<BlockWithFlags>;
|
||||
|
||||
struct RightTableData
|
||||
{
|
||||
Type type = Type::EMPTY;
|
||||
@ -341,7 +329,7 @@ public:
|
||||
|
||||
std::vector<MapsVariant> maps;
|
||||
Block sample_block; /// Block as it would appear in the BlockList
|
||||
BlocksWithFlagsList blocks;
|
||||
BlocksList blocks;
|
||||
BlockNullmapList blocks_nullmaps; /// Nullmaps for blocks of "right" table (if needed)
|
||||
|
||||
/// Additional data - strings for string keys and continuation elements of single-linked lists of references to rows.
|
||||
|
@ -51,7 +51,7 @@ bool JoinSwitcher::addJoinedBlock(const Block & block, bool)
|
||||
void JoinSwitcher::switchJoin()
|
||||
{
|
||||
std::shared_ptr<HashJoin::RightTableData> joined_data = static_cast<const HashJoin &>(*join).getJoinedData();
|
||||
HashJoin::BlocksWithFlagsList right_blocks = std::move(joined_data->blocks);
|
||||
BlocksList right_blocks = std::move(joined_data->blocks);
|
||||
|
||||
/// Destroy old join & create new one. Early destroy for memory saving.
|
||||
join = std::make_shared<MergeJoin>(table_join, right_sample_block);
|
||||
@ -62,20 +62,20 @@ void JoinSwitcher::switchJoin()
|
||||
if (!right_blocks.empty())
|
||||
{
|
||||
positions.reserve(right_sample_block.columns());
|
||||
const HashJoin::BlockWithFlags & tmp_block = *right_blocks.begin();
|
||||
const Block & tmp_block = *right_blocks.begin();
|
||||
for (const auto & sample_column : right_sample_block)
|
||||
{
|
||||
positions.emplace_back(tmp_block.block.getPositionByName(sample_column.name));
|
||||
positions.emplace_back(tmp_block.getPositionByName(sample_column.name));
|
||||
is_nullable.emplace_back(sample_column.type->isNullable());
|
||||
}
|
||||
}
|
||||
|
||||
for (HashJoin::BlockWithFlags & saved_block : right_blocks)
|
||||
for (Block & saved_block : right_blocks)
|
||||
{
|
||||
Block restored_block;
|
||||
for (size_t i = 0; i < positions.size(); ++i)
|
||||
{
|
||||
auto & column = saved_block.block.getByPosition(positions[i]);
|
||||
auto & column = saved_block.getByPosition(positions[i]);
|
||||
restored_block.insert(correctNullability(std::move(column), is_nullable[i]));
|
||||
}
|
||||
join->addJoinedBlock(restored_block);
|
||||
|
Loading…
Reference in New Issue
Block a user