BlockWithFlags removed from vdimir's change

This commit is contained in:
Ilya Golshtein 2021-08-18 16:28:15 +03:00
parent 6b6592fda7
commit 24e3e04203
3 changed files with 13 additions and 33 deletions

View File

@ -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)
{

View File

@ -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.

View File

@ -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);