mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 01:51:59 +00:00
lower memory usage
This commit is contained in:
parent
4286634876
commit
9d3b4e5308
@ -337,7 +337,7 @@ ColumnPtr ColumnAggregateFunction::indexImpl(const PaddedPODArray<Type> & indexe
|
||||
assert(limit <= indexes.size());
|
||||
auto res = createView();
|
||||
|
||||
res->data.resize(limit);
|
||||
res->data.resize_exact(limit);
|
||||
for (size_t i = 0; i < limit; ++i)
|
||||
res->data[i] = data[indexes[i]];
|
||||
|
||||
@ -626,7 +626,7 @@ void ColumnAggregateFunction::getPermutation(PermutationSortDirection /*directio
|
||||
size_t /*limit*/, int /*nan_direction_hint*/, IColumn::Permutation & res) const
|
||||
{
|
||||
size_t s = data.size();
|
||||
res.resize(s);
|
||||
res.resize_exact(s);
|
||||
iota(res.data(), s, IColumn::Permutation::value_type(0));
|
||||
}
|
||||
|
||||
|
@ -109,7 +109,7 @@ MutableColumnPtr ColumnArray::cloneResized(size_t to_size) const
|
||||
offset = getOffsets().back();
|
||||
}
|
||||
|
||||
res->getOffsets().resize(to_size);
|
||||
res->getOffsets().resize_exact(to_size);
|
||||
for (size_t i = from_size; i < to_size; ++i)
|
||||
res->getOffsets()[i] = offset;
|
||||
}
|
||||
@ -427,6 +427,12 @@ void ColumnArray::reserve(size_t n)
|
||||
getData().reserve(n); /// The average size of arrays is not taken into account here. Or it is considered to be no more than 1.
|
||||
}
|
||||
|
||||
void ColumnArray::shrinkToFit()
|
||||
{
|
||||
getOffsets().shrink_to_fit();
|
||||
getData().shrinkToFit();
|
||||
}
|
||||
|
||||
void ColumnArray::ensureOwnership()
|
||||
{
|
||||
getData().ensureOwnership();
|
||||
@ -603,7 +609,7 @@ void ColumnArray::expand(const IColumn::Filter & mask, bool inverted)
|
||||
|
||||
ssize_t index = mask.size() - 1;
|
||||
ssize_t from = offsets_data.size() - 1;
|
||||
offsets_data.resize(mask.size());
|
||||
offsets_data.resize_exact(mask.size());
|
||||
UInt64 last_offset = offsets_data[from];
|
||||
while (index >= 0)
|
||||
{
|
||||
@ -831,7 +837,7 @@ ColumnPtr ColumnArray::indexImpl(const PaddedPODArray<T> & indexes, size_t limit
|
||||
auto res = ColumnArray::create(data->cloneEmpty());
|
||||
|
||||
Offsets & res_offsets = res->getOffsets();
|
||||
res_offsets.resize(limit);
|
||||
res_offsets.resize_exact(limit);
|
||||
size_t current_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < limit; ++i)
|
||||
|
@ -108,6 +108,7 @@ public:
|
||||
void updatePermutationWithCollation(const Collator & collator, PermutationSortDirection direction, PermutationSortStability stability,
|
||||
size_t limit, int nan_direction_hint, Permutation & res, EqualRanges& equal_ranges) const override;
|
||||
void reserve(size_t n) override;
|
||||
void shrinkToFit() override;
|
||||
void ensureOwnership() override;
|
||||
size_t byteSize() const override;
|
||||
size_t byteSizeAt(size_t n) const override;
|
||||
|
@ -128,7 +128,7 @@ MutableColumns ColumnConst::scatter(ColumnIndex num_columns, const Selector & se
|
||||
void ColumnConst::getPermutation(PermutationSortDirection /*direction*/, PermutationSortStability /*stability*/,
|
||||
size_t /*limit*/, int /*nan_direction_hint*/, Permutation & res) const
|
||||
{
|
||||
res.resize(s);
|
||||
res.resize_exact(s);
|
||||
iota(res.data(), s, IColumn::Permutation::value_type(0));
|
||||
}
|
||||
|
||||
|
@ -159,7 +159,7 @@ void ColumnDecimal<T>::getPermutation(IColumn::PermutationSortDirection directio
|
||||
};
|
||||
|
||||
size_t data_size = data.size();
|
||||
res.resize(data_size);
|
||||
res.resize_exact(data_size);
|
||||
|
||||
if (limit >= data_size)
|
||||
limit = 0;
|
||||
@ -318,7 +318,7 @@ MutableColumnPtr ColumnDecimal<T>::cloneResized(size_t size) const
|
||||
if (size > 0)
|
||||
{
|
||||
auto & new_col = static_cast<Self &>(*res);
|
||||
new_col.data.resize(size);
|
||||
new_col.data.resize_exact(size);
|
||||
|
||||
size_t count = std::min(this->size(), size);
|
||||
|
||||
|
@ -55,6 +55,7 @@ public:
|
||||
size_t allocatedBytes() const override { return data.allocated_bytes(); }
|
||||
void protect() override { data.protect(); }
|
||||
void reserve(size_t n) override { data.reserve(n); }
|
||||
void shrinkToFit() override { data.shrink_to_fit(); }
|
||||
|
||||
void insertFrom(const IColumn & src, size_t n) override { data.push_back(static_cast<const Self &>(src).getData()[n]); }
|
||||
void insertData(const char * src, size_t /*length*/) override;
|
||||
|
@ -39,7 +39,7 @@ MutableColumnPtr ColumnFixedString::cloneResized(size_t size) const
|
||||
if (size > 0)
|
||||
{
|
||||
auto & new_col = assert_cast<ColumnFixedString &>(*new_col_holder);
|
||||
new_col.chars.resize(size * n);
|
||||
new_col.chars.resize_exact(size * n);
|
||||
|
||||
size_t count = std::min(this->size(), size);
|
||||
memcpy(new_col.chars.data(), chars.data(), count * n * sizeof(chars[0]));
|
||||
|
@ -181,6 +181,11 @@ public:
|
||||
chars.reserve(n * size);
|
||||
}
|
||||
|
||||
void shrinkToFit() override
|
||||
{
|
||||
chars.shrink_to_fit();
|
||||
}
|
||||
|
||||
void resize(size_t size)
|
||||
{
|
||||
chars.resize(n * size);
|
||||
|
@ -159,6 +159,7 @@ public:
|
||||
}
|
||||
|
||||
void reserve(size_t n) override { idx.reserve(n); }
|
||||
void shrinkToFit() override { idx.shrinkToFit(); }
|
||||
|
||||
/// Don't count the dictionary size as it can be shared between different blocks.
|
||||
size_t byteSize() const override { return idx.getPositions()->byteSize(); }
|
||||
@ -295,6 +296,7 @@ public:
|
||||
|
||||
void popBack(size_t n) { positions->popBack(n); }
|
||||
void reserve(size_t n) { positions->reserve(n); }
|
||||
void shrinkToFit() { positions->shrinkToFit(); }
|
||||
|
||||
UInt64 getMaxPositionForCurrentType() const;
|
||||
|
||||
|
@ -229,6 +229,11 @@ void ColumnMap::reserve(size_t n)
|
||||
nested->reserve(n);
|
||||
}
|
||||
|
||||
void ColumnMap::shrinkToFit()
|
||||
{
|
||||
nested->shrinkToFit();
|
||||
}
|
||||
|
||||
void ColumnMap::ensureOwnership()
|
||||
{
|
||||
nested->ensureOwnership();
|
||||
|
@ -83,6 +83,7 @@ public:
|
||||
void updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability,
|
||||
size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const override;
|
||||
void reserve(size_t n) override;
|
||||
void shrinkToFit() override;
|
||||
void ensureOwnership() override;
|
||||
size_t byteSize() const override;
|
||||
size_t byteSizeAt(size_t n) const override;
|
||||
|
@ -97,7 +97,7 @@ MutableColumnPtr ColumnNullable::cloneResized(size_t new_size) const
|
||||
|
||||
if (new_size > 0)
|
||||
{
|
||||
new_null_map->getData().resize(new_size);
|
||||
new_null_map->getData().resize_exact(new_size);
|
||||
|
||||
size_t count = std::min(size(), new_size);
|
||||
memcpy(new_null_map->getData().data(), getNullMapData().data(), count * sizeof(getNullMapData()[0]));
|
||||
@ -678,6 +678,12 @@ void ColumnNullable::reserve(size_t n)
|
||||
getNullMapData().reserve(n);
|
||||
}
|
||||
|
||||
void ColumnNullable::shrinkToFit()
|
||||
{
|
||||
getNestedColumn().shrinkToFit();
|
||||
getNullMapData().shrink_to_fit();
|
||||
}
|
||||
|
||||
void ColumnNullable::ensureOwnership()
|
||||
{
|
||||
getNestedColumn().ensureOwnership();
|
||||
|
@ -109,6 +109,7 @@ public:
|
||||
void updatePermutationWithCollation(const Collator & collator, IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability,
|
||||
size_t limit, int null_direction_hint, Permutation & res, EqualRanges& equal_ranges) const override;
|
||||
void reserve(size_t n) override;
|
||||
void shrinkToFit() override;
|
||||
void ensureOwnership() override;
|
||||
size_t byteSize() const override;
|
||||
size_t byteSizeAt(size_t n) const override;
|
||||
|
@ -71,8 +71,8 @@ MutableColumnPtr ColumnString::cloneResized(size_t to_size) const
|
||||
/// Empty strings are just zero terminating bytes.
|
||||
|
||||
res->chars.resize_fill(res->chars.size() + to_size - from_size);
|
||||
res->offsets.resize_exact(to_size);
|
||||
|
||||
res->offsets.resize(to_size);
|
||||
for (size_t i = from_size; i < to_size; ++i)
|
||||
{
|
||||
++offset;
|
||||
@ -494,6 +494,11 @@ void ColumnString::reserve(size_t n)
|
||||
offsets.reserve(n);
|
||||
}
|
||||
|
||||
void ColumnString::shrinkToFit()
|
||||
{
|
||||
chars.shrink_to_fit();
|
||||
offsets.shrink_to_fit();
|
||||
}
|
||||
|
||||
void ColumnString::getExtremes(Field & min, Field & max) const
|
||||
{
|
||||
|
@ -259,6 +259,7 @@ public:
|
||||
ColumnPtr compress() const override;
|
||||
|
||||
void reserve(size_t n) override;
|
||||
void shrinkToFit() override;
|
||||
|
||||
void getExtremes(Field & min, Field & max) const override;
|
||||
|
||||
|
@ -444,6 +444,13 @@ void ColumnTuple::reserve(size_t n)
|
||||
getColumn(i).reserve(n);
|
||||
}
|
||||
|
||||
void ColumnTuple::shrinkToFit()
|
||||
{
|
||||
const size_t tuple_size = columns.size();
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
getColumn(i).shrinkToFit();
|
||||
}
|
||||
|
||||
void ColumnTuple::ensureOwnership()
|
||||
{
|
||||
const size_t tuple_size = columns.size();
|
||||
|
@ -91,6 +91,7 @@ public:
|
||||
void updatePermutationWithCollation(const Collator & collator, IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability,
|
||||
size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges& equal_ranges) const override;
|
||||
void reserve(size_t n) override;
|
||||
void shrinkToFit() override;
|
||||
void ensureOwnership() override;
|
||||
size_t byteSize() const override;
|
||||
size_t byteSizeAt(size_t n) const override;
|
||||
|
@ -237,7 +237,7 @@ void ColumnVector<T>::getPermutation(IColumn::PermutationSortDirection direction
|
||||
size_t limit, int nan_direction_hint, IColumn::Permutation & res) const
|
||||
{
|
||||
size_t data_size = data.size();
|
||||
res.resize(data_size);
|
||||
res.resize_exact(data_size);
|
||||
|
||||
if (data_size == 0)
|
||||
return;
|
||||
@ -424,7 +424,7 @@ MutableColumnPtr ColumnVector<T>::cloneResized(size_t size) const
|
||||
if (size > 0)
|
||||
{
|
||||
auto & new_col = static_cast<Self &>(*res);
|
||||
new_col.data.resize(size);
|
||||
new_col.data.resize_exact(size);
|
||||
|
||||
size_t count = std::min(this->size(), size);
|
||||
memcpy(new_col.data.data(), data.data(), count * sizeof(data[0]));
|
||||
@ -628,8 +628,8 @@ inline void doFilterAligned(const UInt8 *& filt_pos, const UInt8 *& filt_end_ali
|
||||
filt_pos += SIMD_ELEMENTS;
|
||||
data_pos += SIMD_ELEMENTS;
|
||||
}
|
||||
/// resize to the real size.
|
||||
res_data.resize(current_offset);
|
||||
/// Resize to the real size.
|
||||
res_data.resize_exact(current_offset);
|
||||
}
|
||||
)
|
||||
|
||||
|
@ -182,6 +182,11 @@ public:
|
||||
data.reserve(n);
|
||||
}
|
||||
|
||||
void shrinkToFit() override
|
||||
{
|
||||
data.shrink_to_fit();
|
||||
}
|
||||
|
||||
const char * getFamilyName() const override { return TypeName<T>.data(); }
|
||||
TypeIndex getDataType() const override { return TypeToTypeIndex<T>; }
|
||||
|
||||
|
@ -399,10 +399,7 @@ public:
|
||||
|
||||
/// Requests the removal of unused capacity.
|
||||
/// It is a non-binding request to reduce the capacity of the underlying container to its size.
|
||||
virtual MutablePtr shrinkToFit() const
|
||||
{
|
||||
return cloneResized(size());
|
||||
}
|
||||
virtual void shrinkToFit() {}
|
||||
|
||||
/// If we have another column as a source (owner of data), copy all data to ourself and reset source.
|
||||
virtual void ensureOwnership() {}
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Common/PODArray_fwd.h>
|
||||
#include <base/getPageSize.h>
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <cstdlib>
|
||||
#include <cstring>
|
||||
#include <cstddef>
|
||||
#include <cassert>
|
||||
@ -238,6 +239,12 @@ public:
|
||||
resize_assume_reserved(n);
|
||||
}
|
||||
|
||||
template <typename ... TAllocatorParams>
|
||||
void shrink_to_fit(TAllocatorParams &&... allocator_params)
|
||||
{
|
||||
realloc(PODArrayDetails::minimum_memory_for_elements(size(), ELEMENT_SIZE, pad_left, pad_right), std::forward<TAllocatorParams>(allocator_params)...);
|
||||
}
|
||||
|
||||
void resize_assume_reserved(const size_t n) /// NOLINT
|
||||
{
|
||||
c_end = c_start + PODArrayDetails::byte_size(n, ELEMENT_SIZE);
|
||||
|
@ -601,7 +601,7 @@ Block Block::shrinkToFit() const
|
||||
{
|
||||
Columns new_columns(data.size(), nullptr);
|
||||
for (size_t i = 0; i < data.size(); ++i)
|
||||
new_columns[i] = data[i].column->shrinkToFit();
|
||||
new_columns[i] = data[i].column->cloneResized(data[i].column->size());
|
||||
return cloneWithColumns(new_columns);
|
||||
}
|
||||
|
||||
|
@ -208,7 +208,7 @@ static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnSt
|
||||
data[offset - 1] = 0;
|
||||
}
|
||||
|
||||
data.resize(offset);
|
||||
data.resize_exact(offset);
|
||||
}
|
||||
|
||||
|
||||
|
@ -184,7 +184,11 @@ MergeTreeReadTask::BlockAndProgress MergeTreeReadTask::read(const BlockSizeParam
|
||||
|
||||
Block block;
|
||||
if (read_result.num_rows != 0)
|
||||
{
|
||||
for (const auto & column : read_result.columns)
|
||||
column->assumeMutableRef().shrinkToFit();
|
||||
block = sample_block.cloneWithColumns(read_result.columns);
|
||||
}
|
||||
|
||||
BlockAndProgress res = {
|
||||
.block = std::move(block),
|
||||
|
@ -140,6 +140,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource(
|
||||
|
||||
if (storage.supportsSubcolumns())
|
||||
options.withSubcolumns();
|
||||
|
||||
columns_for_reader = storage_snapshot->getColumnsByNames(options, columns_to_read);
|
||||
}
|
||||
else
|
||||
@ -156,6 +157,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource(
|
||||
read_settings.local_fs_method = LocalFSReadMethod::pread;
|
||||
if (read_with_direct_io)
|
||||
read_settings.direct_io_threshold = 1;
|
||||
|
||||
/// Configure throttling
|
||||
switch (type)
|
||||
{
|
||||
@ -224,7 +226,10 @@ try
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
{
|
||||
if (header.has(it->name))
|
||||
{
|
||||
columns[i]->assumeMutableRef().shrinkToFit();
|
||||
res_columns.emplace_back(std::move(columns[i]));
|
||||
}
|
||||
|
||||
++it;
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user