Remove unused *Locked methods from HashJoin, add comments

This commit is contained in:
vdimir 2021-02-24 19:19:04 +03:00
parent c006b41afc
commit 6aa4791be2
No known key found for this signature in database
GPG Key ID: F57B3E10A21DBB31
4 changed files with 28 additions and 28 deletions

View File

@ -421,22 +421,12 @@ bool HashJoin::empty() const
return data->type == Type::EMPTY;
}
size_t HashJoin::getTotalByteCount() const
{
return getTotalByteCountLocked();
}
size_t HashJoin::getTotalRowCount() const
{
return getTotalRowCountLocked();
}
bool HashJoin::alwaysReturnsEmptySet() const
{
return isInnerOrRight(getKind()) && data->empty && !overDictionary();
}
size_t HashJoin::getTotalRowCountLocked() const
size_t HashJoin::getTotalRowCount() const
{
size_t res = 0;
@ -453,7 +443,7 @@ size_t HashJoin::getTotalRowCountLocked() const
return res;
}
size_t HashJoin::getTotalByteCountLocked() const
size_t HashJoin::getTotalByteCount() const
{
size_t res = 0;
@ -674,8 +664,8 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits)
return true;
/// TODO: Do not calculate them every time
total_rows = getTotalRowCountLocked();
total_bytes = getTotalByteCountLocked();
total_rows = getTotalRowCount();
total_bytes = getTotalByteCount();
}
return table_join->sizeLimits().check(total_rows, total_bytes, "JOIN", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
@ -1249,7 +1239,7 @@ DataTypePtr HashJoin::joinGetCheckAndGetReturnType(const DataTypes & data_types,
template <typename Maps>
ColumnWithTypeAndName HashJoin::joinGetImpl(const Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const
{
// Assemble the key block with correct names.
/// Assemble the key block with correct names.
Block keys;
for (size_t i = 0; i < block.columns(); ++i)
{
@ -1264,8 +1254,8 @@ ColumnWithTypeAndName HashJoin::joinGetImpl(const Block & block, const Block & b
}
// TODO: return multiple columns as named tuple
// TODO: return array of values when strictness == ASTTableJoin::Strictness::All
/// 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
{
if ((strictness == ASTTableJoin::Strictness::Any || strictness == ASTTableJoin::Strictness::RightAny) &&

View File

@ -306,10 +306,6 @@ public:
struct RightTableData
{
/// Protect state for concurrent use in insertFromBlock and joinBlock.
/// @note that these methods could be called simultaneously only while use of StorageJoin.
// mutable std::shared_mutex rwlock;
Type type = Type::EMPTY;
bool empty = true;
@ -400,10 +396,6 @@ private:
static Type chooseMethod(const ColumnRawPtrs & key_columns, Sizes & key_sizes);
/// Call with already locked rwlock.
size_t getTotalRowCountLocked() const;
size_t getTotalByteCountLocked() const;
bool empty() const;
bool overDictionary() const;
};

View File

@ -109,9 +109,23 @@ void StorageJoin::insertBlock(const Block & block)
join->addJoinedBlock(block, true);
}
size_t StorageJoin::getSize() const { return join->getTotalRowCount(); }
std::optional<UInt64> StorageJoin::totalRows(const Settings &) const { return join->getTotalRowCount(); }
std::optional<UInt64> StorageJoin::totalBytes(const Settings &) const { return join->getTotalByteCount(); }
size_t StorageJoin::getSize() const
{
std::shared_lock<std::shared_mutex> lock(rwlock);
return join->getTotalRowCount();
}
std::optional<UInt64> StorageJoin::totalRows(const Settings &) const
{
std::shared_lock<std::shared_mutex> lock(rwlock);
return join->getTotalRowCount();
}
std::optional<UInt64> StorageJoin::totalBytes(const Settings &) const
{
std::shared_lock<std::shared_mutex> lock(rwlock);
return join->getTotalByteCount();
}
void registerStorageJoin(StorageFactory & factory)

View File

@ -69,6 +69,10 @@ private:
std::shared_ptr<TableJoin> table_join;
HashJoinPtr join;
/// Protect state for concurrent use in insertFromBlock and joinBlock.
/// Lock hold via HashJoin instance (or HashJoinHolder for joinGet)
/// during all query and block insertions.
mutable std::shared_mutex rwlock;
void insertBlock(const Block & block) override;