Introduce chunked MegaBuffer allocation

After the introduction of workahead a system to hold a single large megabuffer per submission was implemented, this worked fine for most cases however when many submissions were flight at the same time memory usage would increase dramatically due to the amount of megabuffers needed. Since only one megabuffer was allowed per execution, it forced the buffer to be fairly large in order to accomodate the upper-bound, even further increasing memory usage.

This commit implements a system to fix the memory usage issue described above by allowing multiple megabuffers to be allocated per execution, as well as reuse across executions. Allocations now go through a global allocator object which chooses which chunk to allocate into on a per-allocation scale, if all are in use by the GPU another chunk will be allocated, that can then be reused for future allocations too. This reduces Hollow Knight megabuffer memory usage by a factor 4 and SMO by even more.
This commit is contained in:
PixelyIon 2022-08-07 02:59:33 +05:30
parent 99b5fc35c6
commit 5b7572a8b3
No known key found for this signature in database
GPG Key ID: 11BC6C3201BC2C05
12 changed files with 218 additions and 183 deletions

View File

@ -166,6 +166,7 @@ add_library(skyline SHARED
${source_DIR}/skyline/gpu/texture/texture.cpp
${source_DIR}/skyline/gpu/texture/layout.cpp
${source_DIR}/skyline/gpu/buffer.cpp
${source_DIR}/skyline/gpu/megabuffer.cpp
${source_DIR}/skyline/gpu/presentation_engine.cpp
${source_DIR}/skyline/gpu/shader_manager.cpp
${source_DIR}/skyline/gpu/cache/graphics_pipeline_cache.cpp

View File

@ -372,6 +372,7 @@ namespace skyline::gpu {
presentation(state, *this),
texture(*this),
buffer(*this),
megaBufferAllocator(*this),
descriptor(*this),
shader(state, *this),
graphicsPipelineCache(*this),

View File

@ -9,6 +9,7 @@
#include "gpu/presentation_engine.h"
#include "gpu/texture_manager.h"
#include "gpu/buffer_manager.h"
#include "gpu/megabuffer.h"
#include "gpu/descriptor_allocator.h"
#include "gpu/shader_manager.h"
#include "gpu/cache/graphics_pipeline_cache.h"
@ -44,6 +45,7 @@ namespace skyline::gpu {
TextureManager texture;
BufferManager buffer;
MegaBufferAllocator megaBufferAllocator;
DescriptorAllocator descriptor;
ShaderManager shader;

View File

@ -348,30 +348,30 @@ namespace skyline::gpu {
bufferDelegate->buffer->Write(isFirstUsage, flushHostCallback, gpuCopyCallback, data, offset + bufferDelegate->view->offset);
}
vk::DeviceSize BufferView::AcquireMegaBuffer(MegaBuffer &megaBuffer) const {
MegaBufferAllocator::Allocation BufferView::AcquireMegaBuffer(const std::shared_ptr<FenceCycle> &pCycle, MegaBufferAllocator &allocator) const {
if (!bufferDelegate->buffer->EverHadInlineUpdate())
// Don't megabuffer buffers that have never had inline updates since performance is only going to be harmed as a result of the constant copying and there wont be any benefit since there are no GPU inline updates that would be avoided
return 0;
return {};
if (bufferDelegate->view->size > MegaBufferingDisableThreshold)
return 0;
return {};
auto [newSequence, sequenceSpan]{bufferDelegate->buffer->AcquireCurrentSequence()};
if (!newSequence)
return 0; // If the sequence can't be acquired then the buffer is GPU dirty and we can't megabuffer
return {}; // If the sequence can't be acquired then the buffer is GPU dirty and we can't megabuffer
// If a copy of the view for the current sequence is already in megabuffer then we can just use that
if (newSequence == bufferDelegate->view->lastAcquiredSequence && bufferDelegate->view->megabufferOffset)
return bufferDelegate->view->megabufferOffset;
if (newSequence == bufferDelegate->view->lastAcquiredSequence && bufferDelegate->view->megaBufferAllocation)
return bufferDelegate->view->megaBufferAllocation;
// If the view is not in the megabuffer then we need to allocate a new copy
auto viewBackingSpan{sequenceSpan.subspan(bufferDelegate->view->offset, bufferDelegate->view->size)};
// TODO: we could optimise the alignment requirements here based on buffer usage
bufferDelegate->view->megabufferOffset = megaBuffer.Push(viewBackingSpan, true);
bufferDelegate->view->megaBufferAllocation = allocator.Push(pCycle, viewBackingSpan, true);
bufferDelegate->view->lastAcquiredSequence = newSequence;
return bufferDelegate->view->megabufferOffset; // Success!
return bufferDelegate->view->megaBufferAllocation; // Success!
}
span<u8> BufferView::GetReadOnlyBackingSpan(bool isFirstUsage, const std::function<void()> &flushHostCallback) {

View File

@ -8,6 +8,7 @@
#include <common/lockable_shared_ptr.h>
#include <nce.h>
#include <gpu/tag_allocator.h>
#include "megabuffer.h"
#include "memory_manager.h"
namespace skyline::gpu {
@ -60,7 +61,7 @@ namespace skyline::gpu {
// These are not accounted for in hash nor operator== since they are not an inherent property of the view, but they are required nonetheless for megabuffering on a per-view basis
mutable u64 lastAcquiredSequence{}; //!< The last sequence number for the attached buffer that the megabuffer copy of this view was acquired from, if this is equal to the current sequence of the attached buffer then the copy at `megabufferOffset` is still valid
mutable vk::DeviceSize megabufferOffset{}; //!< Offset of the current copy of the view in the megabuffer (if any), 0 if no copy exists and this is only valid if `lastAcquiredSequence` is equal to the current sequence of the attached buffer
mutable MegaBufferAllocator::Allocation megaBufferAllocation; //!< Allocation for the current copy of the view in the megabuffer (if any), 0 if no copy exists and this is only valid if `lastAcquiredSequence` is equal to the current sequence of the attached buffer
BufferViewStorage(vk::DeviceSize offset, vk::DeviceSize size, vk::Format format);
@ -421,10 +422,10 @@ namespace skyline::gpu {
/**
* @brief If megabuffering is beneficial for the current buffer, pushes its contents into the megabuffer and returns the offset of the pushed data
* @return The offset of the pushed buffer contents in the megabuffer, or 0 if megabuffering is not to be used
* @return The megabuffer allocation for the buffer, may be invalid if megabuffering is not beneficial
* @note The view **must** be locked prior to calling this
*/
vk::DeviceSize AcquireMegaBuffer(MegaBuffer &megaBuffer) const;
MegaBufferAllocator::Allocation AcquireMegaBuffer(const std::shared_ptr<FenceCycle> &pCycle, MegaBufferAllocator &allocator) const;
/**
* @return A span of the backing buffer contents

View File

@ -5,7 +5,7 @@
#include "buffer_manager.h"
namespace skyline::gpu {
BufferManager::BufferManager(GPU &gpu) : gpu(gpu) {}
BufferManager::BufferManager(GPU &gpu) : gpu{gpu} {}
bool BufferManager::BufferLessThan(const std::shared_ptr<Buffer> &it, u8 *pointer) {
return it->guest->begin().base() < pointer;
@ -209,78 +209,4 @@ namespace skyline::gpu {
return buffer->GetView(static_cast<vk::DeviceSize>(guestMapping.begin() - buffer->guest->begin()) + offset, size);
}
}
constexpr static vk::DeviceSize MegaBufferSize{100 * 1024 * 1024}; //!< Size in bytes of the megabuffer (100MiB)
BufferManager::MegaBufferSlot::MegaBufferSlot(GPU &gpu) : backing(gpu.memory.AllocateBuffer(MegaBufferSize)) {}
MegaBuffer::MegaBuffer(BufferManager::MegaBufferSlot &slot) : slot{&slot}, freeRegion{slot.backing.subspan(PAGE_SIZE)} {}
MegaBuffer::~MegaBuffer() {
if (slot)
slot->active.clear(std::memory_order_release);
}
MegaBuffer &MegaBuffer::operator=(MegaBuffer &&other) {
if (slot)
slot->active.clear(std::memory_order_release);
slot = other.slot;
freeRegion = other.freeRegion;
other.slot = nullptr;
return *this;
}
bool MegaBuffer::WasUsed() {
return freeRegion != slot->backing.subspan(PAGE_SIZE);
}
void MegaBuffer::ReplaceCycle(const std::shared_ptr<FenceCycle> &cycle) {
slot->cycle = cycle;
}
void MegaBuffer::Reset() {
freeRegion = slot->backing.subspan(PAGE_SIZE);
}
vk::Buffer MegaBuffer::GetBacking() const {
return slot->backing.vkBuffer;
}
vk::DeviceSize MegaBuffer::Push(span<u8> data, bool pageAlign) {
if (data.size() > freeRegion.size())
throw exception("Ran out of megabuffer space! Alloc size: 0x{:X}", data.size());
if (pageAlign) {
// If page aligned data was requested then align the free
auto alignedFreeBase{util::AlignUp(static_cast<size_t>(freeRegion.data() - slot->backing.data()), PAGE_SIZE)};
freeRegion = slot->backing.subspan(alignedFreeBase);
}
// Allocate space for data from the free region
auto resultSpan{freeRegion.subspan(0, data.size())};
resultSpan.copy_from(data);
// Move the free region along
freeRegion = freeRegion.subspan(data.size());
return static_cast<vk::DeviceSize>(resultSpan.data() - slot->backing.data());
}
MegaBuffer BufferManager::AcquireMegaBuffer(const std::shared_ptr<FenceCycle> &cycle) {
std::scoped_lock lock{megaBufferMutex};
for (auto &slot : megaBuffers) {
if (!slot.active.test_and_set(std::memory_order_acq_rel)) {
if (slot.cycle->Poll()) {
slot.cycle = cycle;
return {slot};
} else {
slot.active.clear(std::memory_order_release);
}
}
}
auto &megaBuffer{megaBuffers.emplace_back(gpu)};
megaBuffer.cycle = cycle;
return {megaBuffer};
}
}

View File

@ -7,8 +7,6 @@
#include "buffer.h"
namespace skyline::gpu {
class MegaBuffer;
/**
* @brief The Buffer Manager is responsible for maintaining a global view of buffers being mapped from the guest to the host, any lookups and creation of host buffer from equivalent guest buffer alongside reconciliation of any overlaps with existing textures
*/
@ -23,22 +21,6 @@ namespace skyline::gpu {
static constexpr size_t L2EntryGranularity{19}; //!< The amount of AS (in bytes) a single L2 PTE covers (512 KiB == 1 << 19)
SegmentTable<Buffer*, AddressSpaceSize, PageSizeBits, L2EntryGranularity> bufferTable; //!< A page table of all buffer mappings for O(1) lookups on full matches
std::mutex megaBufferMutex; //!< Synchronizes access to the allocated megabuffers
friend class MegaBuffer;
/**
* @brief A wrapper around a buffer which can be utilized as backing storage for a megabuffer and can track its state to avoid concurrent usage
*/
struct MegaBufferSlot {
std::atomic_flag active{true}; //!< If the megabuffer is currently being utilized, we want to construct a buffer as active
std::shared_ptr<FenceCycle> cycle; //!< The latest cycle on the fence, all waits must be performed through this
memory::Buffer backing; //!< The GPU buffer as the backing storage for the megabuffer
MegaBufferSlot(GPU &gpu);
};
/**
* @brief A wrapper around a Buffer which locks it with the specified ContextTag
*/
@ -86,8 +68,6 @@ namespace skyline::gpu {
static bool BufferLessThan(const std::shared_ptr<Buffer> &it, u8 *pointer);
public:
std::list<MegaBufferSlot> megaBuffers; //!< A pool of all allocated megabuffers, these are dynamically utilized
BufferManager(GPU &gpu);
/**
@ -114,56 +94,5 @@ namespace skyline::gpu {
* @note The buffer manager **must** be locked prior to calling this
*/
BufferView FindOrCreate(GuestBuffer guestMapping, ContextTag tag = {}, const std::function<void(std::shared_ptr<Buffer>, ContextLock<Buffer> &&)> &attachBuffer = {});
/**
* @return A dynamically allocated megabuffer which can be used to store buffer modifications allowing them to be replayed in-sequence on the GPU
* @note This object **must** be destroyed to be reclaimed by the manager and prevent a memory leak
* @note The buffer manager **doesn't** need to be locked prior to calling this
*/
MegaBuffer AcquireMegaBuffer(const std::shared_ptr<FenceCycle> &cycle);
};
/**
* @brief A simple linearly allocated GPU-side buffer used to temporarily store buffer modifications allowing them to be replayed in-sequence on the GPU
* @note This class is **not** thread-safe and any calls must be externally synchronized
*/
class MegaBuffer {
private:
BufferManager::MegaBufferSlot *slot;
span<u8> freeRegion; //!< The unallocated space in the megabuffer
public:
MegaBuffer(BufferManager::MegaBufferSlot &slot);
~MegaBuffer();
MegaBuffer &operator=(MegaBuffer &&other);
/**
* @return If any allocations into the megabuffer were done at the time of the call
*/
bool WasUsed();
/**
* @brief Replaces the cycle associated with the underlying megabuffer with the supplied cycle
* @note The megabuffer must **NOT** have any dependencies that aren't conveyed by the supplied cycle
*/
void ReplaceCycle(const std::shared_ptr<FenceCycle> &cycle);
/**
* @brief Resets the free region of the megabuffer to its initial state, data is left intact but may be overwritten
*/
void Reset();
/**
* @brief Returns the underlying Vulkan buffer for the megabuffer
*/
vk::Buffer GetBacking() const;
/**
* @brief Pushes data to the megabuffer and returns the offset at which it was written
* @param pageAlign Whether the pushed data should be page aligned in the megabuffer
*/
vk::DeviceSize Push(span<u8> data, bool pageAlign = false);
};
}

View File

@ -5,7 +5,7 @@
#include "command_executor.h"
namespace skyline::gpu::interconnect {
CommandExecutor::CommandExecutor(const DeviceState &state) : gpu{*state.gpu}, activeCommandBuffer{gpu.scheduler.AllocateCommandBuffer()}, cycle{activeCommandBuffer.GetFenceCycle()}, megaBuffer{gpu.buffer.AcquireMegaBuffer(cycle)}, tag{AllocateTag()} {}
CommandExecutor::CommandExecutor(const DeviceState &state) : gpu{*state.gpu}, activeCommandBuffer{gpu.scheduler.AllocateCommandBuffer()}, cycle{activeCommandBuffer.GetFenceCycle()}, tag{AllocateTag()} {}
CommandExecutor::~CommandExecutor() {
cycle->Cancel();
@ -23,6 +23,12 @@ namespace skyline::gpu::interconnect {
return gpu.buffer;
}
MegaBufferAllocator &CommandExecutor::AcquireMegaBufferAllocator() {
if (!megaBufferAllocatorLock)
megaBufferAllocatorLock.emplace(gpu.megaBufferAllocator);
return gpu.megaBufferAllocator;
}
bool CommandExecutor::CreateRenderPassWithSubpass(vk::Rect2D renderArea, span<TextureView *> inputAttachments, span<TextureView *> colorAttachments, TextureView *depthStencilAttachment) {
auto addSubpass{[&] {
renderPass->AddSubpass(inputAttachments, colorAttachments, depthStencilAttachment, gpu);
@ -314,12 +320,13 @@ namespace skyline::gpu::interconnect {
for (const auto &delegate : attachedBufferDelegates) {
delegate->usageCallback = nullptr;
delegate->view->megabufferOffset = 0;
delegate->view->megaBufferAllocation = {};
}
attachedBufferDelegates.clear();
attachedBuffers.clear();
bufferManagerLock.reset();
megaBufferAllocatorLock.reset();
}
void CommandExecutor::Submit() {
@ -331,10 +338,6 @@ namespace skyline::gpu::interconnect {
SubmitInternal();
activeCommandBuffer = gpu.scheduler.AllocateCommandBuffer();
cycle = activeCommandBuffer.GetFenceCycle();
if (megaBuffer.WasUsed())
megaBuffer = gpu.buffer.AcquireMegaBuffer(cycle);
else
megaBuffer.ReplaceCycle(cycle);
}
ResetInternal();
}
@ -347,8 +350,6 @@ namespace skyline::gpu::interconnect {
TRACE_EVENT("gpu", "CommandExecutor::SubmitWithFlush");
SubmitInternal();
cycle = activeCommandBuffer.Reset();
megaBuffer.ReplaceCycle(cycle);
megaBuffer.Reset();
}
ResetInternal();
}

View File

@ -5,6 +5,7 @@
#include <boost/container/stable_vector.hpp>
#include <unordered_set>
#include <gpu/megabuffer.h>
#include "command_nodes.h"
namespace skyline::gpu::interconnect {
@ -21,7 +22,8 @@ namespace skyline::gpu::interconnect {
size_t subpassCount{}; //!< The number of subpasses in the current render pass
std::optional<std::scoped_lock<TextureManager>> textureManagerLock; //!< The lock on the texture manager, this is locked for the duration of the command execution from the first usage inside an execution to the submission
std::optional<std::scoped_lock<BufferManager>> bufferManagerLock; //!< The lock on the buffer manager, this is locked for the duration of the command execution from the first usage inside an execution to the submission
std::optional<std::scoped_lock<BufferManager>> bufferManagerLock; //!< The lock on the buffer manager, see above for details
std::optional<std::scoped_lock<MegaBufferAllocator>> megaBufferAllocatorLock; //!< The lock on the megabuffer allocator, see above for details
/**
* @brief A wrapper of a Texture object that has been locked beforehand and must be unlocked afterwards
@ -96,7 +98,6 @@ namespace skyline::gpu::interconnect {
public:
std::shared_ptr<FenceCycle> cycle; //!< The fence cycle that this command executor uses to wait for the GPU to finish executing commands
MegaBuffer megaBuffer; //!< The megabuffer used to temporarily store buffer modifications allowing them to be replayed in-sequence on the GPU
ContextTag tag; //!< The tag associated with this command executor, any tagged resource locking must utilize this tag
CommandExecutor(const DeviceState &state);
@ -131,6 +132,12 @@ namespace skyline::gpu::interconnect {
*/
bool AttachBuffer(BufferView &view);
/**
* @return A reference to an instance of the megabuffer allocator which will be locked till execution
* @note Any access to the megabuffer allocator while recording commands **must** be done via this
*/
MegaBufferAllocator &AcquireMegaBufferAllocator();
/**
* @brief Attach the lifetime of a buffer view that's already locked to the command buffer
* @note The supplied buffer **must** be locked with the executor's tag

View File

@ -639,28 +639,27 @@ namespace skyline::gpu::interconnect {
* @note This must only be called when the GuestBuffer is resolved correctly
*/
template<typename T>
void Write(CommandExecutor &pExecutor, MegaBuffer &megaBuffer, span<T> buf, size_t dstOffset) {
void Write(CommandExecutor &pExecutor, MegaBufferAllocator &megaBufferAllocator, span<T> buf, size_t dstOffset) {
auto srcCpuBuf{buf.template cast<u8>()};
ContextLock lock{pExecutor.tag, view};
view.Write(lock.IsFirstUsage(), pExecutor.cycle, []() {
// TODO: see Read()
Logger::Warn("GPU dirty buffer reads for attached buffers are unimplemented");
}, [&megaBuffer, &pExecutor, srcCpuBuf, dstOffset, &view = this->view, &lock]() {
}, [&megaBufferAllocator, &pExecutor, srcCpuBuf, dstOffset, &view = this->view, &lock]() {
pExecutor.AttachLockedBufferView(view, std::move(lock));
// This will prevent any CPU accesses to backing for the duration of the usage
// ONLY in this specific case is it fine to access the backing buffer directly since the flag will be propagated with recreations
view->buffer->BlockAllCpuBackingWrites();
auto srcGpuOffset{megaBuffer.Push(srcCpuBuf)};
auto srcGpuBuf{megaBuffer.GetBacking()};
auto srcGpuAllocation{megaBufferAllocator.Push(pExecutor.cycle, srcCpuBuf)};
pExecutor.AddOutsideRpCommand([=](vk::raii::CommandBuffer &commandBuffer, const std::shared_ptr<FenceCycle> &, GPU &) {
vk::BufferCopy copyRegion{
.size = srcCpuBuf.size_bytes(),
.srcOffset = srcGpuOffset,
.srcOffset = srcGpuAllocation.offset,
.dstOffset = view->view->offset + dstOffset
};
commandBuffer.copyBuffer(srcGpuBuf, view->buffer->GetBacking(), copyRegion);
commandBuffer.copyBuffer(srcGpuAllocation.buffer, view->buffer->GetBacking(), copyRegion);
});
}, srcCpuBuf, dstOffset);
}
@ -742,7 +741,7 @@ namespace skyline::gpu::interconnect {
void ConstantBufferUpdate(std::vector<u32> data, u32 offset) {
auto constantBuffer{GetConstantBufferSelector().value()};
constantBuffer.Write<u32>(executor, executor.megaBuffer, data, offset);
constantBuffer.Write<u32>(executor, executor.AcquireMegaBufferAllocator(), data, offset);
}
/* Shader Program */
@ -1115,11 +1114,11 @@ namespace skyline::gpu::interconnect {
auto view{pipelineStage.constantBuffers[constantBuffer.index].view};
executor.AttachBuffer(view);
if (auto megaBufferOffset{view.AcquireMegaBuffer(executor.megaBuffer)}) {
if (auto megaBufferAllocation{view.AcquireMegaBuffer(executor.cycle, executor.AcquireMegaBufferAllocator())}) {
// If the buffer is megabuffered then since we don't get out data from the underlying buffer, rather the megabuffer which stays consistent throughout a single execution, we can skip registering usage
bufferDescriptors[bufferIndex] = vk::DescriptorBufferInfo{
.buffer = executor.megaBuffer.GetBacking(),
.offset = megaBufferOffset,
.buffer = megaBufferAllocation.buffer,
.offset = megaBufferAllocation.offset,
.range = view->view->size
};
} else {
@ -2832,10 +2831,10 @@ namespace skyline::gpu::interconnect {
boundIndexBuffer = std::make_shared<BoundIndexBuffer>();
boundIndexBuffer->type = indexBuffer.type;
if (auto megaBufferOffset{indexBufferView.AcquireMegaBuffer(executor.megaBuffer)}) {
if (auto megaBufferAllocation{indexBufferView.AcquireMegaBuffer(executor.cycle, executor.AcquireMegaBufferAllocator())}) {
// If the buffer is megabuffered then since we don't get out data from the underlying buffer, rather the megabuffer which stays consistent throughout a single execution, we can skip registering usage
boundIndexBuffer->handle = executor.megaBuffer.GetBacking();
boundIndexBuffer->offset = megaBufferOffset;
boundIndexBuffer->handle = megaBufferAllocation.buffer;
boundIndexBuffer->offset = megaBufferAllocation.offset;
} else {
indexBufferView.RegisterUsage(executor.cycle, [=](const Buffer::BufferViewStorage &view, const std::shared_ptr<Buffer> &buffer) {
boundIndexBuffer->handle = buffer->GetBacking();
@ -2873,10 +2872,10 @@ namespace skyline::gpu::interconnect {
auto &vertexBufferView{vertexBuffer->view};
executor.AttachBuffer(vertexBufferView);
if (auto megaBufferOffset{vertexBufferView.AcquireMegaBuffer(executor.megaBuffer)}) {
if (auto megaBufferAllocation{vertexBufferView.AcquireMegaBuffer(executor.cycle, executor.AcquireMegaBufferAllocator())}) {
// If the buffer is megabuffered then since we don't get out data from the underlying buffer, rather the megabuffer which stays consistent throughout a single execution, we can skip registering usage
boundVertexBuffers->handles[index] = executor.megaBuffer.GetBacking();
boundVertexBuffers->offsets[index] = megaBufferOffset;
boundVertexBuffers->handles[index] = megaBufferAllocation.buffer;
boundVertexBuffers->offsets[index] = megaBufferAllocation.offset;
} else {
vertexBufferView.RegisterUsage(executor.cycle, [handle = boundVertexBuffers->handles.data() + index, offset = boundVertexBuffers->offsets.data() + index](const Buffer::BufferViewStorage &view, const std::shared_ptr<Buffer> &buffer) {
*handle = buffer->GetBacking();

View File

@ -0,0 +1,77 @@
// SPDX-License-Identifier: MPL-2.0
// Copyright © 2021 Skyline Team and Contributors (https://github.com/skyline-emu/)
#include <gpu.h>
#include "megabuffer.h"
namespace skyline::gpu {
constexpr static vk::DeviceSize MegaBufferChunkSize{25 * 1024 * 1024}; //!< Size in bytes of a single megabuffer chunk (25MiB)
MegaBufferChunk::MegaBufferChunk(GPU &gpu) : backing{gpu.memory.AllocateBuffer(MegaBufferChunkSize)}, freeRegion{backing.subspan(PAGE_SIZE)} {}
bool MegaBufferChunk::TryReset() {
if (cycle && cycle->Poll(true)) {
freeRegion = backing.subspan(PAGE_SIZE);
cycle = nullptr;
return true;
}
return cycle == nullptr;
}
vk::Buffer MegaBufferChunk::GetBacking() const {
return backing.vkBuffer;
}
vk::DeviceSize MegaBufferChunk::Push(const std::shared_ptr<FenceCycle> &newCycle, span<u8> data, bool pageAlign) {
if (pageAlign) {
// If page aligned data was requested then align the free
auto alignedFreeBase{util::AlignUp(static_cast<size_t>(freeRegion.data() - backing.data()), PAGE_SIZE)};
freeRegion = backing.subspan(alignedFreeBase);
}
if (data.size() > freeRegion.size())
return 0;
if (cycle != newCycle) {
newCycle->ChainCycle(cycle);
cycle = newCycle;
}
// Allocate space for data from the free region
auto resultSpan{freeRegion.subspan(0, data.size())};
resultSpan.copy_from(data);
// Move the free region along
freeRegion = freeRegion.subspan(data.size());
return static_cast<vk::DeviceSize>(resultSpan.data() - backing.data());
}
MegaBufferAllocator::MegaBufferAllocator(GPU &gpu) : gpu{gpu}, activeChunk{chunks.emplace(chunks.end(), gpu)} {}
void MegaBufferAllocator::lock() {
mutex.lock();
}
void MegaBufferAllocator::unlock() {
mutex.unlock();
}
bool MegaBufferAllocator::try_lock() {
return mutex.try_lock();
}
MegaBufferAllocator::Allocation MegaBufferAllocator::Push(const std::shared_ptr<FenceCycle> &cycle, span<u8> data, bool pageAlign) {
if (vk::DeviceSize offset{activeChunk->Push(cycle, data, pageAlign)}; offset)
return {activeChunk->GetBacking(), offset};
activeChunk = ranges::find_if(chunks, [&](auto &chunk) { return chunk.TryReset(); });
if (activeChunk == chunks.end()) // If there are no chunks available, allocate a new one
activeChunk = chunks.emplace(chunks.end(), gpu);
if (vk::DeviceSize offset{activeChunk->Push(cycle, data, pageAlign)}; offset)
return {activeChunk->GetBacking(), offset};
else
throw exception("Failed to to allocate megabuffer space for size: 0x{:X}", data.size());
}
}

View File

@ -0,0 +1,91 @@
// SPDX-License-Identifier: MPL-2.0
// Copyright © 2021 Skyline Team and Contributors (https://github.com/skyline-emu/)
#pragma once
#include "memory_manager.h"
namespace skyline::gpu {
/**
* @brief A simple linearly allocated GPU-side buffer used to temporarily store buffer modifications allowing them to be replayed in-sequence on the GPU
* @note This class is **not** thread-safe and any calls must be externally synchronized
*/
class MegaBufferChunk {
private:
std::shared_ptr<FenceCycle> cycle; //!< Latest cycle this chunk has had allocations in
memory::Buffer backing; //!< The GPU buffer as the backing storage for the chunk
span<u8> freeRegion; //!< The unallocated space in the chunk
public:
MegaBufferChunk(GPU &gpu);
/**
* @brief If the chunk's cycle is is signalled, resets the free region of the megabuffer to its initial state, if it's not signalled the chunk must not be used
* @returns True if the chunk can be reused, false otherwise
*/
bool TryReset();
/**
* @brief Returns the underlying Vulkan buffer for the chunk
*/
vk::Buffer GetBacking() const;
/**
* @brief Pushes data to the chunk and returns the offset at which it was written
* @param pageAlign Whether the pushed data should be page aligned in the chunk
*/
vk::DeviceSize Push(const std::shared_ptr<FenceCycle> &newCycle, span<u8> data, bool pageAlign = false);
};
/**
* @brief Allocator for megabuffer chunks that takes the usage of resources on the GPU into account
* @note This class is not thread-safe and any calls must be externally synchronized
*/
class MegaBufferAllocator {
private:
GPU &gpu;
std::mutex mutex;
std::list<MegaBufferChunk> chunks; //!< A pool of all allocated megabuffer chunks, these are dynamically utilized
decltype(chunks)::iterator activeChunk; //!< Currently active chunk of the megabuffer which is being allocated into
public:
/**
* @brief A megabuffer chunk allocation
*/
struct Allocation {
vk::Buffer buffer; //!< The megabuffer chunk backing hat the allocation was made within
vk::DeviceSize offset; //!< The offset of the allocation in the chunk
operator bool() const {
return offset != 0;
}
};
MegaBufferAllocator(GPU &gpu);
/**
* @brief Acquires an exclusive lock on the allocator for the calling thread
* @note Naming is in accordance to the BasicLockable named requirement
*/
void lock();
/**
* @brief Relinquishes an existing lock on the allocator by the calling thread
* @note Naming is in accordance to the BasicLockable named requirement
*/
void unlock();
/**
* @brief Attempts to acquire an exclusive lock but returns immediately if it's captured by another thread
* @note Naming is in accordance to the Lockable named requirement
*/
bool try_lock();
/**
* @brief Pushes data to a megabuffer chunk and returns an structure describing the allocation
* @param pageAlign Whether the pushed data should be page aligned in the megabuffer
* @note The allocator *MUST* be locked before calling this function
*/
Allocation Push(const std::shared_ptr<FenceCycle> &cycle, span<u8> data, bool pageAlign = false);
};
}