Put arena cleanup nodes on a separate chunked list instead of at the ends of arena blocks.

The motivation is (a) better data locality during SerialArena::CleanupList and (b) simplification of arena layout.

PiperOrigin-RevId: 631173641
pull/16758/head
Protobuf Team Bot 11 months ago committed by Copybara-Service
parent 24f27c3b88
commit f70d90b474
  1. 1
      src/google/protobuf/BUILD.bazel
  2. 191
      src/google/protobuf/arena.cc
  3. 86
      src/google/protobuf/arena_cleanup.h
  4. 42
      src/google/protobuf/arena_unittest.cc
  5. 113
      src/google/protobuf/serial_arena.h
  6. 2
      src/google/protobuf/thread_safe_arena.h

@ -1185,6 +1185,7 @@ cc_test(
}),
deps = [
":arena",
":arena_cleanup",
":cc_test_protos",
":lite_test_util",
":port",

@ -60,27 +60,43 @@ ArenaBlock* SentryArenaBlock() {
}
#endif
SizedPtr AllocateMemory(const AllocationPolicy* policy_ptr, size_t last_size,
size_t min_bytes) {
inline size_t AllocationSize(size_t last_size, size_t start_size,
size_t max_size) {
if (last_size == 0) return start_size;
// Double the current block size, up to a limit.
return std::min(2 * last_size, max_size);
}
SizedPtr AllocateMemory(const AllocationPolicy& policy, size_t size) {
if (policy.block_alloc == nullptr) {
return AllocateAtLeast(size);
}
return {policy.block_alloc(size), size};
}
SizedPtr AllocateBlock(const AllocationPolicy* policy_ptr, size_t last_size,
size_t min_bytes) {
AllocationPolicy policy; // default policy
if (policy_ptr) policy = *policy_ptr;
size_t size;
if (last_size != 0) {
// Double the current block size, up to a limit.
auto max_size = policy.max_block_size;
size = std::min(2 * last_size, max_size);
} else {
size = policy.start_block_size;
}
size_t size =
AllocationSize(last_size, policy.start_block_size, policy.max_block_size);
// Verify that min_bytes + kBlockHeaderSize won't overflow.
ABSL_CHECK_LE(min_bytes, std::numeric_limits<size_t>::max() -
SerialArena::kBlockHeaderSize);
size = std::max(size, SerialArena::kBlockHeaderSize + min_bytes);
if (policy.block_alloc == nullptr) {
return AllocateAtLeast(size);
}
return {policy.block_alloc(size), size};
return AllocateMemory(policy, size);
}
SizedPtr AllocateCleanupChunk(const AllocationPolicy* policy_ptr,
size_t last_size) {
constexpr size_t kStartSize = 64;
constexpr size_t kMaxSize = 4 << 10;
static_assert(kStartSize % sizeof(cleanup::CleanupNode) == 0, "");
const size_t size = AllocationSize(last_size, kStartSize, kMaxSize);
if (policy_ptr == nullptr) return AllocateAtLeast(size);
return AllocateMemory(*policy_ptr, size);
}
class GetDeallocator {
@ -102,6 +118,88 @@ class GetDeallocator {
} // namespace
namespace cleanup {
struct ChunkList::Chunk {
CleanupNode* First() { return reinterpret_cast<CleanupNode*>(this + 1); }
CleanupNode* Last() { return First() + Capacity() - 1; }
static size_t Capacity(size_t size) {
return (size - sizeof(Chunk)) / sizeof(CleanupNode);
}
size_t Capacity() const { return Capacity(size); }
Chunk* next;
size_t size;
// Cleanup nodes follow.
};
void ChunkList::AddFallback(void* elem, void (*destructor)(void*),
SerialArena& arena) {
ABSL_DCHECK_EQ(next_, limit_);
SizedPtr mem = AllocateCleanupChunk(arena.parent_.AllocPolicy(),
head_ == nullptr ? 0 : head_->size);
arena.AddSpaceAllocated(mem.n);
head_ = new (mem.p) Chunk{head_, mem.n};
next_ = head_->First();
prefetch_ptr_ = reinterpret_cast<char*>(next_);
limit_ = next_ + Chunk::Capacity(mem.n);
AddFromExisting(elem, destructor);
}
void ChunkList::Cleanup(const SerialArena& arena) {
Chunk* c = head_;
if (c == nullptr) return;
GetDeallocator deallocator(arena.parent_.AllocPolicy());
// Iterate backwards in order to destroy in the right order.
CleanupNode* it = next_ - 1;
while (true) {
CleanupNode* first = c->First();
// A prefetch distance of 8 here was chosen arbitrarily.
constexpr int kPrefetchDistance = 8;
CleanupNode* prefetch = it;
// Prefetch the first kPrefetchDistance nodes.
for (int i = 0; prefetch >= first && i < kPrefetchDistance;
--prefetch, ++i) {
prefetch->Prefetch();
}
// For the middle nodes, run destructor and prefetch the node
// kPrefetchDistance after the current one.
for (; prefetch >= first; --it, --prefetch) {
it->Destroy();
prefetch->Prefetch();
}
// Note: we could consider prefetching `next` chunk earlier.
absl::PrefetchToLocalCacheNta(c->next);
// Destroy the rest without prefetching.
for (; it >= first; --it) {
it->Destroy();
}
Chunk* next = c->next;
deallocator({c, c->size});
if (next == nullptr) return;
c = next;
it = c->Last();
};
}
std::vector<void*> ChunkList::PeekForTesting() {
std::vector<void*> ret;
Chunk* c = head_;
if (c == nullptr) return ret;
// Iterate backwards to match destruction order.
CleanupNode* it = next_ - 1;
while (true) {
CleanupNode* first = c->First();
for (; it >= first; --it) {
ret.push_back(it->elem);
}
c = c->next;
if (c == nullptr) return ret;
it = c->Last();
};
}
} // namespace cleanup
// It is guaranteed that this is constructed in `b`. IOW, this is not the first
// arena and `b` cannot be sentry.
SerialArena::SerialArena(ArenaBlock* b, ThreadSafeArena& parent)
@ -109,7 +207,6 @@ SerialArena::SerialArena(ArenaBlock* b, ThreadSafeArena& parent)
limit_{b->Limit()},
prefetch_ptr_(
b->Pointer(kBlockHeaderSize + ThreadSafeArena::kSerialArenaSize)),
prefetch_limit_(b->Limit()),
head_{b},
space_allocated_{b->size},
parent_{parent} {
@ -130,22 +227,7 @@ SerialArena::SerialArena(FirstSerialArena, ArenaBlock* b,
}
std::vector<void*> SerialArena::PeekCleanupListForTesting() {
std::vector<void*> res;
ArenaBlock* b = head();
if (b->IsSentry()) return res;
const auto peek_list = [&](char* pos, char* end) {
for (; pos != end; pos += cleanup::Size()) {
cleanup::PeekNode(pos, res);
}
};
peek_list(limit_, b->Limit());
for (b = b->next; b; b = b->next) {
peek_list(reinterpret_cast<char*>(b->cleanup_nodes), b->Limit());
}
return res;
return cleanup_list_.PeekForTesting();
}
std::vector<void*> ThreadSafeArena::PeekCleanupListForTesting() {
@ -223,25 +305,16 @@ void* SerialArena::AllocateFromStringBlockFallback() {
PROTOBUF_NOINLINE
void* SerialArena::AllocateAlignedWithCleanupFallback(
size_t n, size_t align, void (*destructor)(void*)) {
size_t required = AlignUpTo(n, align) + cleanup::Size();
size_t required = AlignUpTo(n, align);
AllocateNewBlock(required);
return AllocateAlignedWithCleanup(n, align, destructor);
}
PROTOBUF_NOINLINE
void SerialArena::AddCleanupFallback(void* elem, void (*destructor)(void*)) {
AllocateNewBlock(cleanup::Size());
AddCleanupFromExisting(elem, destructor);
}
void SerialArena::AllocateNewBlock(size_t n) {
size_t used = 0;
size_t wasted = 0;
ArenaBlock* old_head = head();
if (!old_head->IsSentry()) {
// Sync limit to block
old_head->cleanup_nodes = limit_;
// Record how much used in this block.
used = static_cast<size_t>(ptr() - old_head->Pointer(kBlockHeaderSize));
wasted = old_head->size - used - kBlockHeaderSize;
@ -253,7 +326,7 @@ void SerialArena::AllocateNewBlock(size_t n) {
// but with a CPU regression. The regression might have been an artifact of
// the microbenchmark.
auto mem = AllocateMemory(parent_.AllocPolicy(), old_head->size, n);
auto mem = AllocateBlock(parent_.AllocPolicy(), old_head->size, n);
AddSpaceAllocated(mem.n);
ThreadSafeArenaStats::RecordAllocateStats(parent_.arena_stats_.MutableStats(),
/*used=*/used,
@ -314,34 +387,6 @@ size_t SerialArena::FreeStringBlocks(StringBlock* string_block,
return deallocated;
}
void SerialArena::CleanupList() {
ArenaBlock* b = head();
if (b->IsSentry()) return;
b->cleanup_nodes = limit_;
do {
char* limit = b->Limit();
char* it = reinterpret_cast<char*>(b->cleanup_nodes);
ABSL_DCHECK(!b->IsSentry() || it == limit);
// A prefetch distance of 8 here was chosen arbitrarily.
char* prefetch = it;
int prefetch_dist = 8;
for (; prefetch < limit && --prefetch_dist; prefetch += cleanup::Size()) {
cleanup::PrefetchNode(prefetch);
}
for (; prefetch < limit;
it += cleanup::Size(), prefetch += cleanup::Size()) {
cleanup::DestroyNode(it);
cleanup::PrefetchNode(prefetch);
}
absl::PrefetchToLocalCacheNta(b->next);
for (; it < limit; it += cleanup::Size()) {
cleanup::DestroyNode(it);
}
b = b->next;
} while (b);
}
// Stores arrays of void* and SerialArena* instead of linked list of
// SerialArena* to speed up traversing all SerialArena. The cost of walk is non
// trivial when there are many nodes. Separately storing "ids" minimizes cache
@ -550,7 +595,7 @@ ArenaBlock* ThreadSafeArena::FirstBlock(void* buf, size_t size,
SizedPtr mem;
if (buf == nullptr || size < kBlockHeaderSize + kAllocPolicySize) {
mem = AllocateMemory(&policy, 0, kAllocPolicySize);
mem = AllocateBlock(&policy, 0, kAllocPolicySize);
} else {
mem = {buf, size};
// Record user-owned block.
@ -734,6 +779,8 @@ uint64_t ThreadSafeArena::Reset() {
// Have to do this in a first pass, because some of the destructors might
// refer to memory in other blocks.
CleanupList();
// Reset the first arena's cleanup list.
first_arena_.cleanup_list_ = cleanup::ChunkList();
// Discard all blocks except the first one. Whether it is user-provided or
// allocated, always reuse the first block for the first arena.
@ -913,7 +960,7 @@ SerialArena* ThreadSafeArena::GetSerialArenaFallback(size_t n) {
// have any blocks yet. So we'll allocate its first block now. It must be
// big enough to host SerialArena and the pending request.
serial = SerialArena::New(
AllocateMemory(alloc_policy_.get(), 0, n + kSerialArenaSize), *this);
AllocateBlock(alloc_policy_.get(), 0, n + kSerialArenaSize), *this);
AddSerialArena(id, serial);
}

@ -21,6 +21,9 @@
namespace google {
namespace protobuf {
namespace internal {
class SerialArena;
namespace cleanup {
// Helper function invoking the destructor of `object`
@ -33,44 +36,61 @@ void arena_destruct_object(void* object) {
// destroyed, and the function to destroy it (`destructor`)
// elem must be aligned at minimum on a 4 byte boundary.
struct CleanupNode {
// Optimization: performs a prefetch on the elem for the cleanup node. We
// explicitly use NTA prefetch here to avoid polluting remote caches: we are
// destroying these instances, there is no purpose for these cache lines to
// linger around in remote caches.
ABSL_ATTRIBUTE_ALWAYS_INLINE void Prefetch() {
// TODO: we should also prefetch the destructor code once
// processors support code prefetching.
absl::PrefetchToLocalCacheNta(elem);
}
// Destroys the object referenced by the cleanup node.
ABSL_ATTRIBUTE_ALWAYS_INLINE void Destroy() { destructor(elem); }
void* elem;
void (*destructor)(void*);
};
inline ABSL_ATTRIBUTE_ALWAYS_INLINE CleanupNode* ToCleanup(void* pos) {
return reinterpret_cast<CleanupNode*>(pos);
}
// Adds a cleanup entry at memory location `pos`.
inline ABSL_ATTRIBUTE_ALWAYS_INLINE void CreateNode(void* pos, void* elem,
// Manages the list of cleanup nodes in a chunked linked list. Chunks grow by
// factors of two up to a limit. Trivially destructible, but Cleanup() must be
// called before destruction.
class ChunkList {
public:
PROTOBUF_ALWAYS_INLINE void Add(void* elem, void (*destructor)(void*),
SerialArena& arena) {
if (PROTOBUF_PREDICT_TRUE(next_ < limit_)) {
AddFromExisting(elem, destructor);
return;
}
AddFallback(elem, destructor, arena);
}
// Runs all inserted cleanups and frees allocated chunks. Must be called
// before destruction.
void Cleanup(const SerialArena& arena);
private:
struct Chunk;
friend class internal::SerialArena;
void AddFallback(void* elem, void (*destructor)(void*), SerialArena& arena);
ABSL_ATTRIBUTE_ALWAYS_INLINE void AddFromExisting(void* elem,
void (*destructor)(void*)) {
CleanupNode n = {elem, destructor};
memcpy(pos, &n, sizeof(n));
}
// Optimization: performs a prefetch on the elem for the cleanup node at `pos`.
inline ABSL_ATTRIBUTE_ALWAYS_INLINE void PrefetchNode(void* pos) {
// We explicitly use NTA prefetch here to avoid polluting remote caches: we
// are destroying these instances, there is no purpose for these cache lines
// to linger around in remote caches.
absl::PrefetchToLocalCacheNta(ToCleanup(pos)->elem);
}
// Destroys the object referenced by the cleanup node.
inline ABSL_ATTRIBUTE_ALWAYS_INLINE void DestroyNode(void* pos) {
CleanupNode* cleanup = ToCleanup(pos);
cleanup->destructor(cleanup->elem);
}
// Append in `out` the pointer to the to-be-cleaned object in `pos`.
inline void PeekNode(void* pos, std::vector<void*>& out) {
out.push_back(ToCleanup(pos)->elem);
}
// Returns the required size for a cleanup node.
constexpr ABSL_ATTRIBUTE_ALWAYS_INLINE size_t Size() {
return sizeof(CleanupNode);
}
*next_++ = CleanupNode{elem, destructor};
}
// Returns the pointers to the to-be-cleaned objects.
std::vector<void*> PeekForTesting();
Chunk* head_ = nullptr;
CleanupNode* next_ = nullptr;
CleanupNode* limit_ = nullptr;
// Current prefetch position. Data from `next_` up to but not including
// `prefetch_ptr_` is software prefetched. Used in SerialArena prefetching.
const char* prefetch_ptr_ = nullptr;
};
} // namespace cleanup
} // namespace internal

@ -31,6 +31,7 @@
#include "absl/strings/string_view.h"
#include "absl/synchronization/barrier.h"
#include "absl/utility/utility.h"
#include "google/protobuf/arena_cleanup.h"
#include "google/protobuf/arena_test_util.h"
#include "google/protobuf/descriptor.h"
#include "google/protobuf/extension_set.h"
@ -1406,12 +1407,12 @@ TEST(ArenaTest, RepeatedFieldOnArena) {
// Preallocate an initial arena block to avoid mallocs during hooked region.
std::vector<char> arena_block(1024 * 1024);
Arena arena(arena_block.data(), arena_block.size());
const size_t initial_allocated_size = arena.SpaceAllocated();
{
internal::NoHeapChecker no_heap;
// Fill some repeated fields on the arena to test for leaks. Also verify no
// memory allocations.
// Fill some repeated fields on the arena to test for leaks. Also that the
// newly allocated memory is approximately the size of the cleanups for the
// repeated messages.
RepeatedField<int32_t> repeated_int32(&arena);
RepeatedPtrField<TestAllTypes> repeated_message(&arena);
for (int i = 0; i < 100; i++) {
@ -1432,10 +1433,14 @@ TEST(ArenaTest, RepeatedFieldOnArena) {
repeated_message.UnsafeArenaExtractSubrange(0, 5, extracted_messages);
EXPECT_EQ(&arena, repeated_message.Get(0).GetArena());
EXPECT_EQ(5, repeated_message.size());
// Upper bound of the size of the cleanups of new repeated messages.
const size_t upperbound_cleanup_size =
2 * 110 * sizeof(internal::cleanup::CleanupNode);
EXPECT_GT(initial_allocated_size + upperbound_cleanup_size,
arena.SpaceAllocated());
}
// Now, outside the scope of the NoHeapChecker, test ExtractSubrange's copying
// semantics.
// Now test ExtractSubrange's copying semantics.
{
RepeatedPtrField<TestAllTypes> repeated_message(&arena);
for (int i = 0; i < 100; i++) {
@ -1577,8 +1582,11 @@ TEST(ArenaTest, NoHeapAllocationsTest) {
Arena arena(options);
{
// We need to call Arena::Create before NoHeapChecker because the ArenaDtor
// allocates a new cleanup chunk.
TestAllTypes* message = Arena::Create<TestAllTypes>(&arena);
FillArenaAwareFields(message);
}
@ -1610,8 +1618,9 @@ TEST(ArenaTest, MessageLiteOnArena) {
initial_message.SerializeToString(&serialized);
{
MessageLite* generic_message = prototype->New(&arena);
EXPECT_TRUE(generic_message != nullptr);
EXPECT_EQ(&arena, generic_message->GetArena());
EXPECT_TRUE(generic_message->ParseFromString(serialized));
@ -1679,6 +1688,23 @@ TEST(ArenaTest, FirstArenaOverhead) {
}
TEST(ArenaTest, StartingBlockSize) {
Arena default_arena;
EXPECT_EQ(0, default_arena.SpaceAllocated());
// Allocate something to get starting block size.
Arena::CreateArray<char>(&default_arena, 1);
ArenaOptions options;
// First block size should be the default starting block size.
EXPECT_EQ(default_arena.SpaceAllocated(), options.start_block_size);
// Use a custom starting block size.
options.start_block_size *= 2;
Arena custom_arena(options);
Arena::CreateArray<char>(&custom_arena, 1);
EXPECT_EQ(custom_arena.SpaceAllocated(), options.start_block_size);
}
TEST(ArenaTest, BlockSizeDoubling) {
Arena arena;
EXPECT_EQ(0, arena.SpaceUsed());

@ -37,13 +37,10 @@ namespace internal {
// Arena blocks are variable length malloc-ed objects. The following structure
// describes the common header for all blocks.
struct ArenaBlock {
// For the sentry block with zero-size where ptr_, limit_, cleanup_nodes all
// point to "this".
constexpr ArenaBlock()
: next(nullptr), cleanup_nodes(this), size(0) {}
// For the sentry block with zero-size where ptr_/limit_ both point to `this`.
constexpr ArenaBlock() : next(nullptr), size(0) {}
ArenaBlock(ArenaBlock* next, size_t size)
: next(next), cleanup_nodes(nullptr), size(size) {
ArenaBlock(ArenaBlock* next, size_t size) : next(next), size(size) {
ABSL_DCHECK_GT(size, sizeof(ArenaBlock));
}
@ -56,7 +53,6 @@ struct ArenaBlock {
bool IsSentry() const { return size == 0; }
ArenaBlock* const next;
void* cleanup_nodes;
const size_t size;
// data follows
};
@ -86,7 +82,7 @@ class PROTOBUF_EXPORT SerialArena {
static constexpr size_t kBlockHeaderSize =
ArenaAlignDefault::Ceil(sizeof(ArenaBlock));
void CleanupList();
void CleanupList() { cleanup_list_.Cleanup(*this); }
uint64_t SpaceAllocated() const {
return space_allocated_.load(std::memory_order_relaxed);
}
@ -218,7 +214,7 @@ class PROTOBUF_EXPORT SerialArena {
*out = ret;
char* next = ret + n;
set_ptr(next);
MaybePrefetchForwards(next);
MaybePrefetchData(next);
return true;
}
@ -236,27 +232,23 @@ class PROTOBUF_EXPORT SerialArena {
n = ArenaAlignDefault::Ceil(n);
char* ret = ArenaAlignAs(align).CeilDefaultAligned(ptr());
// See the comment in MaybeAllocateAligned re uintptr_t.
if (PROTOBUF_PREDICT_FALSE(reinterpret_cast<uintptr_t>(ret) + n +
cleanup::Size() >
if (PROTOBUF_PREDICT_FALSE(reinterpret_cast<uintptr_t>(ret) + n >
reinterpret_cast<uintptr_t>(limit_))) {
return AllocateAlignedWithCleanupFallback(n, align, destructor);
}
PROTOBUF_UNPOISON_MEMORY_REGION(ret, n);
char* next = ret + n;
set_ptr(next);
AddCleanupFromExisting(ret, destructor);
AddCleanup(ret, destructor);
ABSL_DCHECK_GE(limit_, ptr());
MaybePrefetchForwards(next);
MaybePrefetchData(next);
return ret;
}
PROTOBUF_ALWAYS_INLINE
void AddCleanup(void* elem, void (*destructor)(void*)) {
size_t has = static_cast<size_t>(limit_ - ptr());
if (PROTOBUF_PREDICT_FALSE(cleanup::Size() > has)) {
return AddCleanupFallback(elem, destructor);
}
AddCleanupFromExisting(elem, destructor);
cleanup_list_.Add(elem, destructor, *this);
MaybePrefetchCleanup();
}
ABSL_ATTRIBUTE_RETURNS_NONNULL void* AllocateFromStringBlock();
@ -265,6 +257,7 @@ class PROTOBUF_EXPORT SerialArena {
private:
friend class ThreadSafeArena;
friend class cleanup::ChunkList;
// See comments for cached_blocks_.
struct CachedBlock {
@ -272,8 +265,8 @@ class PROTOBUF_EXPORT SerialArena {
CachedBlock* next;
};
static constexpr ptrdiff_t kPrefetchForwardsDegree = ABSL_CACHELINE_SIZE * 16;
static constexpr ptrdiff_t kPrefetchBackwardsDegree = ABSL_CACHELINE_SIZE * 6;
static constexpr ptrdiff_t kPrefetchDataDegree = ABSL_CACHELINE_SIZE * 16;
static constexpr ptrdiff_t kPrefetchCleanupDegree = ABSL_CACHELINE_SIZE * 6;
// Constructor is private as only New() should be used.
inline SerialArena(ArenaBlock* b, ThreadSafeArena& parent);
@ -285,59 +278,41 @@ class PROTOBUF_EXPORT SerialArena {
bool MaybeAllocateString(void*& p);
ABSL_ATTRIBUTE_RETURNS_NONNULL void* AllocateFromStringBlockFallback();
// Prefetch the next prefetch_degree bytes after `prefetch_ptr` and
// up to `limit`, if `next` is within prefetch_degree bytes of `prefetch_ptr`.
PROTOBUF_ALWAYS_INLINE
void AddCleanupFromExisting(void* elem, void (*destructor)(void*)) {
const size_t cleanup_size = cleanup::Size();
PROTOBUF_UNPOISON_MEMORY_REGION(limit_ - cleanup_size, cleanup_size);
limit_ -= cleanup_size;
MaybePrefetchBackwards(limit_);
ABSL_DCHECK_GE(limit_, ptr());
cleanup::CreateNode(limit_, elem, destructor);
}
// Prefetch the next kPrefetchForwardsDegree bytes after `prefetch_ptr_` and
// up to `prefetch_limit_`, if `next` is within kPrefetchForwardsDegree bytes
// of `prefetch_ptr_`.
PROTOBUF_ALWAYS_INLINE
void MaybePrefetchForwards(const char* next) {
ABSL_DCHECK(static_cast<const void*>(prefetch_ptr_) == nullptr ||
static_cast<const void*>(prefetch_ptr_) >= head());
if (PROTOBUF_PREDICT_TRUE(prefetch_ptr_ - next > kPrefetchForwardsDegree))
return;
if (PROTOBUF_PREDICT_TRUE(prefetch_ptr_ < prefetch_limit_)) {
const char* prefetch_ptr = std::max(next, prefetch_ptr_);
static const char* MaybePrefetchImpl(const ptrdiff_t prefetch_degree,
const char* next, const char* limit,
const char* prefetch_ptr) {
if (PROTOBUF_PREDICT_TRUE(prefetch_ptr - next > prefetch_degree))
return prefetch_ptr;
if (PROTOBUF_PREDICT_TRUE(prefetch_ptr < limit)) {
prefetch_ptr = std::max(next, prefetch_ptr);
ABSL_DCHECK(prefetch_ptr != nullptr);
const char* end =
std::min(prefetch_limit_, prefetch_ptr + ABSL_CACHELINE_SIZE * 16);
const char* end = std::min(limit, prefetch_ptr + prefetch_degree);
for (; prefetch_ptr < end; prefetch_ptr += ABSL_CACHELINE_SIZE) {
absl::PrefetchToLocalCacheForWrite(prefetch_ptr);
}
prefetch_ptr_ = prefetch_ptr;
}
return prefetch_ptr;
}
PROTOBUF_ALWAYS_INLINE
// Prefetch up to kPrefetchBackwardsDegree before `prefetch_limit_` and after
// `prefetch_ptr_`, if `limit` is within kPrefetchBackwardsDegree of
// `prefetch_limit_`.
void MaybePrefetchBackwards(const char* limit) {
ABSL_DCHECK(prefetch_limit_ == nullptr ||
static_cast<const void*>(prefetch_limit_) <=
static_cast<const void*>(head()->Limit()));
if (PROTOBUF_PREDICT_TRUE(limit - prefetch_limit_ >
kPrefetchBackwardsDegree))
return;
if (PROTOBUF_PREDICT_TRUE(prefetch_limit_ > prefetch_ptr_)) {
const char* prefetch_limit = std::min(limit, prefetch_limit_);
ABSL_DCHECK_NE(prefetch_limit, nullptr);
const char* end =
std::max(prefetch_ptr_, prefetch_limit - kPrefetchBackwardsDegree);
for (; prefetch_limit > end; prefetch_limit -= ABSL_CACHELINE_SIZE) {
absl::PrefetchToLocalCacheForWrite(prefetch_limit);
}
prefetch_limit_ = prefetch_limit;
}
void MaybePrefetchData(const char* next) {
ABSL_DCHECK(static_cast<const void*>(prefetch_ptr_) == nullptr ||
static_cast<const void*>(prefetch_ptr_) >= head());
prefetch_ptr_ =
MaybePrefetchImpl(kPrefetchDataDegree, next, limit_, prefetch_ptr_);
}
PROTOBUF_ALWAYS_INLINE
void MaybePrefetchCleanup() {
ABSL_DCHECK(static_cast<const void*>(cleanup_list_.prefetch_ptr_) ==
nullptr ||
static_cast<const void*>(cleanup_list_.prefetch_ptr_) >=
cleanup_list_.head_);
cleanup_list_.prefetch_ptr_ = MaybePrefetchImpl(
kPrefetchCleanupDegree, reinterpret_cast<char*>(cleanup_list_.next_),
reinterpret_cast<char*>(cleanup_list_.limit_),
cleanup_list_.prefetch_ptr_);
}
// Creates a new SerialArena inside mem using the remaining memory as for
@ -385,7 +360,6 @@ class PROTOBUF_EXPORT SerialArena {
set_ptr(ptr);
prefetch_ptr_ = ptr;
limit_ = limit;
prefetch_limit_ = limit;
}
void* AllocateAlignedFallback(size_t n);
@ -405,10 +379,11 @@ class PROTOBUF_EXPORT SerialArena {
// Limiting address up to which memory can be allocated from the head block.
char* limit_ = nullptr;
// Current prefetch positions. Data from `ptr_` up to but not including
// `prefetch_ptr_` is software prefetched. Similarly, data from `limit_` down
// to but not including `prefetch_limit_` is software prefetched.
// `prefetch_ptr_` is software prefetched.
const char* prefetch_ptr_ = nullptr;
const char* prefetch_limit_ = nullptr;
// Chunked linked list for managing cleanup for arena elements.
cleanup::ChunkList cleanup_list_;
// The active string block.
std::atomic<StringBlock*> string_block_{nullptr};

@ -20,6 +20,7 @@
#include "absl/synchronization/mutex.h"
#include "google/protobuf/arena_align.h"
#include "google/protobuf/arena_allocation_policy.h"
#include "google/protobuf/arena_cleanup.h"
#include "google/protobuf/arenaz_sampler.h"
#include "google/protobuf/port.h"
#include "google/protobuf/serial_arena.h"
@ -109,6 +110,7 @@ class PROTOBUF_EXPORT ThreadSafeArena {
friend class TcParser;
friend class SerialArena;
friend struct SerialArenaChunkHeader;
friend class cleanup::ChunkList;
static uint64_t GetNextLifeCycleId();
class SerialArenaChunk;

Loading…
Cancel
Save