mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 01:51:59 +00:00
Merge pull request #45485 from kitaisreal/remove-unnecessary-get-total-row-count-function-calls
Remove unnecessary getTotalRowCount function calls
This commit is contained in:
commit
7b48c75e82
@ -327,10 +327,11 @@ bool GraceHashJoin::addJoinedBlock(const Block & block, bool /*check_limits*/)
|
|||||||
bool GraceHashJoin::fitsInMemory() const
|
bool GraceHashJoin::fitsInMemory() const
|
||||||
{
|
{
|
||||||
/// One row can't be split, avoid loop
|
/// One row can't be split, avoid loop
|
||||||
if (hash_join->getTotalRowCount() < 2)
|
size_t total_row_count = hash_join->getTotalRowCount();
|
||||||
|
if (total_row_count < 2)
|
||||||
return true;
|
return true;
|
||||||
|
|
||||||
return table_join->sizeLimits().softCheck(hash_join->getTotalRowCount(), hash_join->getTotalByteCount());
|
return table_join->sizeLimits().softCheck(total_row_count, hash_join->getTotalByteCount());
|
||||||
}
|
}
|
||||||
|
|
||||||
GraceHashJoin::Buckets GraceHashJoin::rehashBuckets(size_t to_size)
|
GraceHashJoin::Buckets GraceHashJoin::rehashBuckets(size_t to_size)
|
||||||
|
@ -158,7 +158,8 @@ void DistinctSortedTransform::transform(Chunk & chunk)
|
|||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!set_size_limits.check(data.getTotalRowCount(), data.getTotalByteCount(), "DISTINCT", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED))
|
size_t data_total_row_count = data.getTotalRowCount();
|
||||||
|
if (!set_size_limits.check(data_total_row_count, data.getTotalByteCount(), "DISTINCT", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED))
|
||||||
{
|
{
|
||||||
stopReading();
|
stopReading();
|
||||||
chunk.clear();
|
chunk.clear();
|
||||||
@ -166,7 +167,7 @@ void DistinctSortedTransform::transform(Chunk & chunk)
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// Stop reading if we already reached the limit.
|
/// Stop reading if we already reached the limit.
|
||||||
if (limit_hint && data.getTotalRowCount() >= limit_hint)
|
if (limit_hint && data_total_row_count >= limit_hint)
|
||||||
stopReading();
|
stopReading();
|
||||||
|
|
||||||
prev_chunk.chunk = std::move(chunk);
|
prev_chunk.chunk = std::move(chunk);
|
||||||
|
@ -94,19 +94,20 @@ void DistinctTransform::transform(Chunk & chunk)
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// Just go to the next chunk if there isn't any new record in the current one.
|
/// Just go to the next chunk if there isn't any new record in the current one.
|
||||||
if (data.getTotalRowCount() == old_set_size)
|
size_t new_set_size = data.getTotalRowCount();
|
||||||
|
if (new_set_size == old_set_size)
|
||||||
return;
|
return;
|
||||||
|
|
||||||
if (!set_size_limits.check(data.getTotalRowCount(), data.getTotalByteCount(), "DISTINCT", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED))
|
if (!set_size_limits.check(new_set_size, data.getTotalByteCount(), "DISTINCT", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED))
|
||||||
return;
|
return;
|
||||||
|
|
||||||
for (auto & column : columns)
|
for (auto & column : columns)
|
||||||
column = column->filter(filter, -1);
|
column = column->filter(filter, -1);
|
||||||
|
|
||||||
chunk.setColumns(std::move(columns), data.getTotalRowCount() - old_set_size);
|
chunk.setColumns(std::move(columns), new_set_size - old_set_size);
|
||||||
|
|
||||||
/// Stop reading if we already reach the limit
|
/// Stop reading if we already reach the limit
|
||||||
if (limit_hint && data.getTotalRowCount() >= limit_hint)
|
if (limit_hint && new_set_size >= limit_hint)
|
||||||
{
|
{
|
||||||
stopReading();
|
stopReading();
|
||||||
return;
|
return;
|
||||||
|
@ -646,6 +646,8 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter(
|
|||||||
std::vector<size_t> key_position;
|
std::vector<size_t> key_position;
|
||||||
|
|
||||||
Columns columns = prepared_set->getSetElements();
|
Columns columns = prepared_set->getSetElements();
|
||||||
|
size_t prepared_set_total_row_count = prepared_set->getTotalRowCount();
|
||||||
|
|
||||||
for (const auto & elem : key_tuple_mapping)
|
for (const auto & elem : key_tuple_mapping)
|
||||||
{
|
{
|
||||||
bloom_filters.emplace_back();
|
bloom_filters.emplace_back();
|
||||||
@ -653,7 +655,8 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter(
|
|||||||
|
|
||||||
size_t tuple_idx = elem.tuple_index;
|
size_t tuple_idx = elem.tuple_index;
|
||||||
const auto & column = columns[tuple_idx];
|
const auto & column = columns[tuple_idx];
|
||||||
for (size_t row = 0; row < prepared_set->getTotalRowCount(); ++row)
|
|
||||||
|
for (size_t row = 0; row < prepared_set_total_row_count; ++row)
|
||||||
{
|
{
|
||||||
bloom_filters.back().emplace_back(params);
|
bloom_filters.back().emplace_back(params);
|
||||||
auto ref = column->getDataAt(row);
|
auto ref = column->getDataAt(row);
|
||||||
|
Loading…
Reference in New Issue
Block a user