Skip to content

Commit

Permalink
Merge pull request #48745 from Algunenano/arenita
Browse files Browse the repository at this point in the history
Don't count unused arena bytes as read_bytes in AggregateFunction types
  • Loading branch information
alexey-milovidov committed Apr 24, 2023
2 parents 93d6ddd + 868b4e4 commit dee0b78
Show file tree
Hide file tree
Showing 38 changed files with 156 additions and 79 deletions.
1 change: 1 addition & 0 deletions src/AggregateFunctions/AggregateFunctionDistinct.h
Expand Up @@ -9,6 +9,7 @@
#include <Common/HashTable/HashSet.h>
#include <Common/HashTable/HashMap.h>
#include <Common/SipHash.h>
#include <IO/ReadHelpersArena.h>


namespace DB
Expand Down
1 change: 1 addition & 0 deletions src/AggregateFunctions/AggregateFunctionGroupUniqArray.h
Expand Up @@ -4,6 +4,7 @@

#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadHelpersArena.h>

#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesNumber.h>
Expand Down
2 changes: 1 addition & 1 deletion src/AggregateFunctions/AggregateFunctionHistogram.h
Expand Up @@ -2,7 +2,6 @@

#include <base/sort.h>

#include <Common/Arena.h>
#include <Common/NaNUtils.h>

#include <Columns/ColumnVector.h>
Expand All @@ -29,6 +28,7 @@
namespace DB
{
struct Settings;
class Arena;

namespace ErrorCodes
{
Expand Down
Expand Up @@ -6,7 +6,6 @@

#include <Columns/ColumnsNumber.h>

#include <Common/ArenaAllocator.h>
#include <Common/assert_cast.h>
#include <base/arithmeticOverflow.h>
#include <base/sort.h>
Expand Down
Expand Up @@ -5,7 +5,6 @@
#include <Columns/ColumnTuple.h>
#include <Common/Exception.h>
#include <Common/assert_cast.h>
#include <Common/ArenaAllocator.h>
#include <Common/PODArray_fwd.h>
#include <base/types.h>
#include <DataTypes/DataTypeNullable.h>
Expand Down
1 change: 0 additions & 1 deletion src/AggregateFunctions/AggregateFunctionMannWhitney.h
Expand Up @@ -6,7 +6,6 @@
#include <Columns/ColumnVector.h>
#include <Columns/ColumnTuple.h>
#include <Common/assert_cast.h>
#include <Common/ArenaAllocator.h>
#include <Common/PODArray_fwd.h>
#include <base/types.h>
#include <DataTypes/DataTypeArray.h>
Expand Down
2 changes: 0 additions & 2 deletions src/AggregateFunctions/AggregateFunctionRankCorrelation.h
Expand Up @@ -14,8 +14,6 @@
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeArray.h>

#include <Common/ArenaAllocator.h>

namespace DB
{
struct Settings;
Expand Down
1 change: 0 additions & 1 deletion src/AggregateFunctions/AggregateFunctionRetention.h
Expand Up @@ -8,7 +8,6 @@
#include <DataTypes/DataTypeArray.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Common/ArenaAllocator.h>
#include <base/range.h>
#include <bitset>

Expand Down
1 change: 1 addition & 0 deletions src/AggregateFunctions/AggregateFunctionTopK.h
Expand Up @@ -2,6 +2,7 @@

#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadHelpersArena.h>

#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesNumber.h>
Expand Down
1 change: 0 additions & 1 deletion src/AggregateFunctions/AggregateFunctionWindowFunnel.h
Expand Up @@ -6,7 +6,6 @@
#include <DataTypes/DataTypesNumber.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Common/ArenaAllocator.h>
#include <Common/assert_cast.h>

#include <AggregateFunctions/AggregateFunctionNull.h>
Expand Down
9 changes: 4 additions & 5 deletions src/Columns/ColumnAggregateFunction.cpp
Expand Up @@ -385,8 +385,7 @@ void ColumnAggregateFunction::updateHashFast(SipHash & hash) const
/// threads, so we can't know the size of these data.
size_t ColumnAggregateFunction::byteSize() const
{
return data.size() * sizeof(data[0])
+ (my_arena ? my_arena->size() : 0);
return data.size() * sizeof(data[0]) + (my_arena ? my_arena->usedBytes() : 0);
}

size_t ColumnAggregateFunction::byteSizeAt(size_t) const
Expand All @@ -395,11 +394,11 @@ size_t ColumnAggregateFunction::byteSizeAt(size_t) const
return sizeof(data[0]) + func->sizeOfData();
}

/// Like in byteSize(), the size is underestimated.
/// Similar to byteSize() the size is underestimated.
/// In this case it's also overestimated at the same time as it counts all the bytes allocated by the arena, used or not
size_t ColumnAggregateFunction::allocatedBytes() const
{
return data.allocated_bytes()
+ (my_arena ? my_arena->size() : 0);
return data.allocated_bytes() + (my_arena ? my_arena->allocatedBytes() : 0);
}

void ColumnAggregateFunction::protect()
Expand Down
27 changes: 17 additions & 10 deletions src/Common/Arena.h
Expand Up @@ -80,7 +80,8 @@ class Arena : private boost::noncopyable

/// Last contiguous MemoryChunk of memory.
MemoryChunk * head;
size_t size_in_bytes;
size_t allocated_bytes;
size_t used_bytes;
size_t page_size;

static size_t roundUpToPageSize(size_t s, size_t page_size)
Expand Down Expand Up @@ -119,17 +120,20 @@ class Arena : private boost::noncopyable
void NO_INLINE addMemoryChunk(size_t min_size)
{
head = new MemoryChunk(nextSize(min_size + pad_right), head);
size_in_bytes += head->size();
allocated_bytes += head->size();
}

friend class ArenaAllocator;
template <size_t> friend class AlignedArenaAllocator;

public:
explicit Arena(size_t initial_size_ = 4096, size_t growth_factor_ = 2, size_t linear_growth_threshold_ = 128 * 1024 * 1024)
: growth_factor(growth_factor_), linear_growth_threshold(linear_growth_threshold_),
head(new MemoryChunk(initial_size_, nullptr)), size_in_bytes(head->size()),
page_size(static_cast<size_t>(::getPageSize()))
: growth_factor(growth_factor_)
, linear_growth_threshold(linear_growth_threshold_)
, head(new MemoryChunk(initial_size_, nullptr))
, allocated_bytes(head->size())
, used_bytes(0)
, page_size(static_cast<size_t>(::getPageSize()))
{
}

Expand All @@ -141,6 +145,7 @@ class Arena : private boost::noncopyable
/// Get piece of memory, without alignment.
char * alloc(size_t size)
{
used_bytes += size;
if (unlikely(static_cast<std::ptrdiff_t>(size) > head->end - head->pos))
addMemoryChunk(size);

Expand All @@ -153,6 +158,7 @@ class Arena : private boost::noncopyable
/// Get piece of memory with alignment
char * alignedAlloc(size_t size, size_t alignment)
{
used_bytes += size;
do
{
void * head_pos = head->pos;
Expand Down Expand Up @@ -184,6 +190,7 @@ class Arena : private boost::noncopyable
*/
void * rollback(size_t size)
{
used_bytes -= size;
head->pos -= size;
ASAN_POISON_MEMORY_REGION(head->pos, size + pad_right);
return head->pos;
Expand Down Expand Up @@ -299,11 +306,11 @@ class Arena : private boost::noncopyable
return res;
}

/// Size of MemoryChunks in bytes.
size_t size() const
{
return size_in_bytes;
}
/// Size of all MemoryChunks in bytes.
size_t allocatedBytes() const { return allocated_bytes; }

/// Total space actually used (not counting padding or space unused by caller allocations) in all MemoryChunks in bytes.
size_t usedBytes() const { return used_bytes; }

/// Bad method, don't use it -- the MemoryChunks are not your business, the entire
/// purpose of the arena code is to manage them for you, so if you find
Expand Down
9 changes: 3 additions & 6 deletions src/Common/ArenaWithFreeLists.h
Expand Up @@ -107,10 +107,7 @@ class ArenaWithFreeLists : private Allocator<false>, private boost::noncopyable
}

/// Size of the allocated pool in bytes
size_t size() const
{
return pool.size();
}
size_t allocatedBytes() const { return pool.allocatedBytes(); }
};

class SynchronizedArenaWithFreeLists : private ArenaWithFreeLists
Expand All @@ -135,10 +132,10 @@ class SynchronizedArenaWithFreeLists : private ArenaWithFreeLists
}

/// Size of the allocated pool in bytes
size_t size() const
size_t allocatedBytes() const
{
std::lock_guard lock{mutex};
return ArenaWithFreeLists::size();
return ArenaWithFreeLists::allocatedBytes();
}
private:
mutable std::mutex mutex;
Expand Down
2 changes: 1 addition & 1 deletion src/Common/TLDListsHolder.h
Expand Up @@ -3,14 +3,14 @@
#include <base/defines.h>
#include <base/StringRef.h>
#include <Common/HashTable/StringHashMap.h>
#include <Common/Arena.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <mutex>
#include <string>
#include <unordered_map>

namespace DB
{
class Arena;

enum TLDType
{
Expand Down
6 changes: 3 additions & 3 deletions src/Common/examples/arena_with_free_lists.cpp
Expand Up @@ -270,7 +270,7 @@ int main(int argc, char ** argv)

watch.stop();
std::cerr
<< "Insert info arena. Bytes: " << arena.size()
<< "Insert info arena. Bytes: " << arena.allocatedBytes()
<< ", elapsed: " << watch.elapsedSeconds()
<< " (" << data.size() / watch.elapsedSeconds() << " elem/sec.,"
<< " " << sum_strings_size / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.)"
Expand Down Expand Up @@ -298,7 +298,7 @@ int main(int argc, char ** argv)

watch.stop();
std::cerr
<< "Randomly remove and insert elements. Bytes: " << arena.size()
<< "Randomly remove and insert elements. Bytes: " << arena.allocatedBytes()
<< ", elapsed: " << watch.elapsedSeconds()
<< " (" << data.size() / watch.elapsedSeconds() << " elem/sec.,"
<< " " << bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.)"
Expand Down Expand Up @@ -331,7 +331,7 @@ int main(int argc, char ** argv)

watch.stop();
std::cerr
<< "Filling cache. Bytes: " << arena.size()
<< "Filling cache. Bytes: " << arena.allocatedBytes()
<< ", elapsed: " << watch.elapsedSeconds()
<< " (" << data.size() / watch.elapsedSeconds() << " elem/sec.,"
<< " " << bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.)"
Expand Down
5 changes: 1 addition & 4 deletions src/Coordination/SnapshotableHashTable.h
Expand Up @@ -333,10 +333,7 @@ class SnapshotableHashTable
}
}

uint64_t keyArenaSize() const
{
return arena.size();
}
uint64_t keyArenaSize() const { return arena.allocatedBytes(); }

iterator begin() { return list.begin(); }
const_iterator begin() const { return list.cbegin(); }
Expand Down
2 changes: 1 addition & 1 deletion src/Dictionaries/CacheDictionaryStorage.h
Expand Up @@ -157,7 +157,7 @@ class CacheDictionaryStorage final : public ICacheDictionaryStorage
});
}

return arena.size() + sizeof(Cell) * configuration.max_size_in_cells + attributes_size_in_bytes;
return arena.allocatedBytes() + sizeof(Cell) * configuration.max_size_in_cells + attributes_size_in_bytes;
}

private:
Expand Down
3 changes: 2 additions & 1 deletion src/Dictionaries/DictionaryHelpers.h
@@ -1,6 +1,5 @@
#pragma once

#include <Common/Arena.h>
#include <Common/HashTable/HashMap.h>
#include <Columns/IColumn.h>
#include <Columns/ColumnDecimal.h>
Expand Down Expand Up @@ -29,6 +28,8 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}

class Arena;

/** Simple helper for getting default.
* Initialized with default value and default values column.
* If default values column is not null default value is taken from column.
Expand Down
2 changes: 1 addition & 1 deletion src/Dictionaries/FlatDictionary.cpp
Expand Up @@ -505,7 +505,7 @@ void FlatDictionary::calculateBytesAllocated()
bytes_allocated += hierarchical_index_bytes_allocated;
}

bytes_allocated += string_arena.size();
bytes_allocated += string_arena.allocatedBytes();
}

FlatDictionary::Attribute FlatDictionary::createAttribute(const DictionaryAttribute & dictionary_attribute)
Expand Down
2 changes: 1 addition & 1 deletion src/Dictionaries/HashedArrayDictionary.cpp
Expand Up @@ -797,7 +797,7 @@ void HashedArrayDictionary<dictionary_key_type>::calculateBytesAllocated()
bytes_allocated += hierarchical_index_bytes_allocated;
}

bytes_allocated += string_arena.size();
bytes_allocated += string_arena.allocatedBytes();
}

template <DictionaryKeyType dictionary_key_type>
Expand Down
2 changes: 1 addition & 1 deletion src/Dictionaries/HashedDictionary.cpp
Expand Up @@ -1022,7 +1022,7 @@ void HashedDictionary<dictionary_key_type, sparse, sharded>::calculateBytesAlloc
}

for (const auto & arena : string_arenas)
bytes_allocated += arena->size();
bytes_allocated += arena->allocatedBytes();
}

template <DictionaryKeyType dictionary_key_type, bool sparse, bool sharded>
Expand Down
2 changes: 1 addition & 1 deletion src/Dictionaries/IPAddressDictionary.cpp
Expand Up @@ -541,7 +541,7 @@ template <>
void IPAddressDictionary::addAttributeSize<String>(const Attribute & attribute)
{
addAttributeSize<StringRef>(attribute);
bytes_allocated += sizeof(Arena) + attribute.string_arena->size();
bytes_allocated += sizeof(Arena) + attribute.string_arena->allocatedBytes();
}

void IPAddressDictionary::calculateBytesAllocated()
Expand Down
3 changes: 2 additions & 1 deletion src/Dictionaries/IPAddressDictionary.h
Expand Up @@ -5,7 +5,6 @@
#include <variant>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Common/Arena.h>
#include <Common/HashTable/HashMap.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnVector.h>
Expand All @@ -18,6 +17,8 @@

namespace DB
{
class Arena;

class IPAddressDictionary final : public IDictionary
{
public:
Expand Down
2 changes: 1 addition & 1 deletion src/Dictionaries/RangeHashedDictionary.h
Expand Up @@ -726,7 +726,7 @@ void RangeHashedDictionary<dictionary_key_type>::calculateBytesAllocated()
if (update_field_loaded_block)
bytes_allocated += update_field_loaded_block->allocatedBytes();

bytes_allocated += string_arena.size();
bytes_allocated += string_arena.allocatedBytes();
}

template <DictionaryKeyType dictionary_key_type>
Expand Down
1 change: 0 additions & 1 deletion src/Dictionaries/RegExpTreeDictionary.h
Expand Up @@ -10,7 +10,6 @@

#include <Columns/IColumn.h>
#include <Columns/ColumnString.h>
#include <Common/Arena.h>
#include <Common/Exception.h>
#include <Common/HashTable/Hash.h>
#include <Common/HashTable/HashSet.h>
Expand Down
17 changes: 0 additions & 17 deletions src/IO/ReadHelpers.h
Expand Up @@ -28,7 +28,6 @@
#include <Common/Allocator.h>
#include <Common/Exception.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/Arena.h>
#include <Common/intExp.h>

#include <Formats/FormatSettings.h>
Expand Down Expand Up @@ -137,22 +136,6 @@ inline void readStringBinary(std::string & s, ReadBuffer & buf, size_t max_strin
buf.readStrict(s.data(), size);
}


inline StringRef readStringBinaryInto(Arena & arena, ReadBuffer & buf)
{
size_t size = 0;
readVarUInt(size, buf);

if (unlikely(size > DEFAULT_MAX_STRING_SIZE))
throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large string size.");

char * data = arena.alloc(size);
buf.readStrict(data, size);

return StringRef(data, size);
}


template <typename T>
void readVectorBinary(std::vector<T> & v, ReadBuffer & buf)
{
Expand Down

0 comments on commit dee0b78

Please sign in to comment.