Rework Buffer DirtyState with BackingImmutability

Having a single variable denoting the exact state of a buffer and the operations that could be performed on it was found to be too restrictive, it's now been expanded into an additional `BackingImmutability` variable but due to these two. We can no longer use atomics without significant additional complexity so all accesses to the state are now mediated through `stateMutex`, a mutex specifically designed for tracking the state.

While designing the system around `stateMutex` it was determined to be more efficient than atomics as it would enforce blocking far less than it would generally have been compared to if the regular atomic fallback of locking the main resource lock which is locked for significantly longer generally.

Co-authored-by: PixelyIon <pixelyion@protonmail.com>
This commit is contained in:
Billy Laws 2022-08-04 15:47:17 +05:30 committed by PixelyIon
parent 1af781c0a5
commit 04bcd7e580
No known key found for this signature in database
GPG Key ID: 11BC6C3201BC2C05
7 changed files with 174 additions and 92 deletions

View File

@ -15,13 +15,20 @@ namespace skyline::gpu {
alignedMirror = gpu.state.process->memory.CreateMirror(span<u8>{alignedData, alignedSize});
mirror = alignedMirror.subspan(static_cast<size_t>(guest->data() - alignedData), guest->size());
// We can't just capture `this` in the lambda since the lambda could exceed the lifetime of the buffer
std::weak_ptr<Buffer> weakThis{weak_from_this()};
// We can't just capture this in the lambda since the lambda could exceed the lifetime of the buffer
std::weak_ptr<Buffer> weakThis{shared_from_this()};
trapHandle = gpu.state.nce->TrapRegions(*guest, true, [weakThis] {
auto buffer{weakThis.lock()};
if (!buffer)
return;
std::scoped_lock lock{*buffer};
std::unique_lock stateLock{buffer->stateMutex};
if (buffer->AllCpuBackingWritesBlocked()) {
stateLock.unlock(); // If the lock isn't unlocked, a deadlock from threads waiting on the other lock can occur
// If this mutex would cause other callbacks to be blocked then we should block on this mutex in advance
std::scoped_lock lock{*buffer};
}
}, [weakThis] {
TRACE_EVENT("gpu", "Buffer::ReadTrap");
@ -29,11 +36,18 @@ namespace skyline::gpu {
if (!buffer)
return true;
std::unique_lock stateLock{buffer->stateMutex, std::try_to_lock};
if (!stateLock)
return false;
if (buffer->dirtyState != DirtyState::GpuDirty)
return true; // If state is already CPU dirty/Clean we don't need to do anything
std::unique_lock lock{*buffer, std::try_to_lock};
if (!lock)
return false;
SynchronizeGuest(true); // We can skip trapping since the caller will do it
buffer->SynchronizeGuest(true); // We can skip trapping since the caller will do it
return true;
}, [weakThis] {
TRACE_EVENT("gpu", "Buffer::WriteTrap");
@ -42,21 +56,28 @@ namespace skyline::gpu {
if (!buffer)
return true;
DirtyState expectedState{DirtyState::Clean};
if (buffer->dirtyState.compare_exchange_strong(expectedState, DirtyState::CpuDirty, std::memory_order_relaxed) || expectedState == DirtyState::CpuDirty)
return true; // If we can transition the buffer to CPU dirty (from Clean) or if it already is CPU dirty then we can just return, we only need to do the lock and corresponding sync if the buffer is GPU dirty
std::unique_lock stateLock{buffer->stateMutex, std::try_to_lock};
if (!stateLock)
return false;
if (!buffer->AllCpuBackingWritesBlocked() && buffer->dirtyState != DirtyState::GpuDirty) {
buffer->dirtyState = DirtyState::CpuDirty;
return true;
}
std::unique_lock lock{*buffer, std::try_to_lock};
if (!lock)
return false;
buffer->SynchronizeGuest(true, false, true); // We need to assume the buffer is dirty since we don't know what the guest is writing
buffer->WaitOnFence();
buffer->SynchronizeGuest(true); // We need to assume the buffer is dirty since we don't know what the guest is writing
buffer->dirtyState = DirtyState::CpuDirty;
return true;
});
}
Buffer::Buffer(GPU &gpu, GuestBuffer guest) : gpu{gpu}, backing{gpu.memory.AllocateBuffer(guest.size())}, guest{guest} {
SetupGuestMappings();
}
Buffer::Buffer(GPU &gpu, GuestBuffer guest) : gpu{gpu}, backing{gpu.memory.AllocateBuffer(guest.size())}, guest{guest} {}
Buffer::Buffer(GPU &gpu, vk::DeviceSize size) : gpu(gpu), backing(gpu.memory.AllocateBuffer(size)) {
dirtyState = DirtyState::Clean; // Since this is a host-only buffer it's always going to be clean
@ -75,19 +96,25 @@ namespace skyline::gpu {
if (!guest)
return;
auto currentState{dirtyState.load(std::memory_order_relaxed)};
do {
if (currentState == DirtyState::GpuDirty)
return;
} while (!dirtyState.compare_exchange_strong(currentState, DirtyState::GpuDirty, std::memory_order_relaxed));
std::scoped_lock lock{stateMutex}; // stateMutex is locked to prevent state changes at any point during this function
if (dirtyState == DirtyState::GpuDirty)
return;
else if (dirtyState == DirtyState::CpuDirty)
SynchronizeHost(true); // Will transition the Buffer to Clean
dirtyState = DirtyState::GpuDirty;
BlockAllCpuBackingWrites();
AdvanceSequence(); // The GPU will modify buffer contents so advance to the next sequence
gpu.state.nce->RetrapRegions(*trapHandle, false);
}
void Buffer::WaitOnFence() {
TRACE_EVENT("gpu", "Buffer::WaitOnFence");
std::scoped_lock lock{stateMutex};
if (cycle) {
cycle->Wait();
cycle = nullptr;
@ -95,6 +122,8 @@ namespace skyline::gpu {
}
bool Buffer::PollFence() {
std::scoped_lock lock{stateMutex};
if (!cycle)
return true;
@ -116,51 +145,52 @@ namespace skyline::gpu {
guest = {};
}
void Buffer::SynchronizeHost(bool rwTrap) {
void Buffer::SynchronizeHost(bool skipTrap) {
if (!guest)
return;
auto currentState{dirtyState.load(std::memory_order_relaxed)};
do {
if (currentState != DirtyState::CpuDirty || !guest)
return; // If the buffer has not been modified on the CPU, there is no need to synchronize it
} while (!dirtyState.compare_exchange_strong(currentState, rwTrap ? DirtyState::GpuDirty : DirtyState::Clean, std::memory_order_relaxed));
TRACE_EVENT("gpu", "Buffer::SynchronizeHost");
{
std::scoped_lock lock{stateMutex};
if (dirtyState != DirtyState::CpuDirty)
return;
dirtyState = DirtyState::Clean;
WaitOnFence();
}
AdvanceSequence(); // We are modifying GPU backing contents so advance to the next sequence
WaitOnFence();
if (!skipTrap)
gpu.state.nce->RetrapRegions(*trapHandle, true); // Trap any future CPU writes to this buffer, must be done before the memcpy so that any modifications during the copy are tracked
std::memcpy(backing.data(), mirror.data(), mirror.size());
gpu.state.nce->RetrapRegions(*trapHandle, !rwTrap); // Trap any future CPU reads (optionally) + writes to this buffer
}
bool Buffer::SynchronizeGuest(bool skipTrap, bool nonBlocking, bool setDirty) {
bool Buffer::SynchronizeGuest(bool skipTrap, bool nonBlocking) {
if (!guest)
return false;
auto currentState{dirtyState.load(std::memory_order_relaxed)};
do {
if (currentState == DirtyState::CpuDirty || (currentState == DirtyState::Clean && !setDirty))
return true; // If the buffer is synchronized (Clean/CpuDirty), there is no need to synchronize it
else if (currentState == DirtyState::GpuDirty && nonBlocking && !PollFence())
return false; // If the buffer is GPU dirty and the fence is not signalled then we can't block
} while (!dirtyState.compare_exchange_strong(currentState, setDirty ? DirtyState::CpuDirty : DirtyState::Clean, std::memory_order_relaxed));
TRACE_EVENT("gpu", "Buffer::SynchronizeGuest");
{
std::scoped_lock lock{stateMutex};
if (dirtyState != DirtyState::GpuDirty)
return true; // If the buffer is not dirty, there is no need to synchronize it
if (nonBlocking && !PollFence())
return false; // If the fence is not signalled and non-blocking behaviour is requested then bail out
WaitOnFence();
std::memcpy(mirror.data(), backing.data(), mirror.size());
dirtyState = DirtyState::Clean;
}
if (!skipTrap)
gpu.state.nce->RetrapRegions(*trapHandle, true);
if (setDirty && currentState == DirtyState::Clean)
return true; // If the texture was simply transitioned from Clean to CpuDirty, there is no need to synchronize it
if (!nonBlocking)
WaitOnFence();
std::memcpy(mirror.data(), backing.data(), mirror.size());
return true;
}
@ -173,6 +203,7 @@ namespace skyline::gpu {
}
void Buffer::Read(bool isFirstUsage, const std::function<void()> &flushHostCallback, span<u8> data, vk::DeviceSize offset) {
std::scoped_lock lock{stateMutex};
if (dirtyState == DirtyState::GpuDirty)
SynchronizeGuestImmediate(isFirstUsage, flushHostCallback);
@ -183,17 +214,18 @@ namespace skyline::gpu {
AdvanceSequence(); // We are modifying GPU backing contents so advance to the next sequence
everHadInlineUpdate = true;
// Perform a syncs in both directions to ensure correct ordering of writes
// We cannot have *ANY* state changes for the duration of this function, if the buffer became CPU dirty partway through the GPU writes would mismatch the CPU writes
std::scoped_lock lock{stateMutex};
// Syncs in both directions to ensure correct ordering of writes
if (dirtyState == DirtyState::CpuDirty)
SynchronizeHost();
else if (dirtyState == DirtyState::GpuDirty)
SynchronizeGuestImmediate(isFirstUsage, flushHostCallback);
// It's possible that the guest will arbitrarily modify the buffer contents on the CPU after the syncs and trigger the signal handler which would set the dirty state to CPU dirty, this is acceptable as there is no requirement to make writes visible immediately
std::memcpy(mirror.data() + offset, data.data(), data.size()); // Always copy to mirror since any CPU side reads will need the up-to-date contents
if (!usedByContext && PollFence())
if (!SequencedCpuBackingWritesBlocked() && PollFence())
// We can write directly to the backing as long as this resource isn't being actively used by a past workload (in the current context or another)
std::memcpy(backing.data() + offset, data.data(), data.size());
else
@ -222,6 +254,7 @@ namespace skyline::gpu {
}
span<u8> Buffer::GetReadOnlyBackingSpan(bool isFirstUsage, const std::function<void()> &flushHostCallback) {
std::scoped_lock lock{stateMutex};
if (dirtyState == DirtyState::GpuDirty)
SynchronizeGuestImmediate(isFirstUsage, flushHostCallback);
@ -243,7 +276,7 @@ namespace skyline::gpu {
void Buffer::unlock() {
tag = ContextTag{};
usedByContext = false;
backingImmutability = BackingImmutability::None;
mutex.unlock();
}
@ -284,8 +317,8 @@ namespace skyline::gpu {
BufferView::BufferView(std::shared_ptr<Buffer> buffer, const Buffer::BufferViewStorage *view) : bufferDelegate(std::make_shared<Buffer::BufferDelegate>(std::move(buffer), view)) {}
void BufferView::RegisterUsage(const std::shared_ptr<FenceCycle> &cycle, const std::function<void(const Buffer::BufferViewStorage &, const std::shared_ptr<Buffer> &)> &usageCallback) {
// Users of RegisterUsage expect the buffer contents to be sequenced as the guest GPU would be, so force any further writes in the current cycle to occur on the GPU
bufferDelegate->buffer->MarkGpuUsed();
// Users of RegisterUsage expect the buffer contents to be sequenced as the guest GPU would be, so force any further sequenced writes in the current cycle to occur on the GPU
bufferDelegate->buffer->BlockSequencedCpuBackingWrites();
usageCallback(*bufferDelegate->view, bufferDelegate->buffer);
if (!bufferDelegate->usageCallback) {
@ -306,8 +339,7 @@ namespace skyline::gpu {
// If megabuffering can't be enabled we have to do a GPU-side copy to ensure sequencing
bool gpuCopy{bufferDelegate->view->size > MegaBufferingDisableThreshold};
if (gpuCopy)
// This will force the host buffer contents to stay as is for the current cycle, requiring that write operations are instead sequenced on the GPU for the entire buffer
bufferDelegate->buffer->MarkGpuUsed();
bufferDelegate->buffer->BlockSequencedCpuBackingWrites();
bufferDelegate->buffer->Write(isFirstUsage, flushHostCallback, gpuCopyCallback, data, offset + bufferDelegate->view->offset);
}
@ -320,7 +352,7 @@ namespace skyline::gpu {
if (bufferDelegate->view->size > MegaBufferingDisableThreshold)
return 0;
auto[newSequence, sequenceSpan]{bufferDelegate->buffer->AcquireCurrentSequence()};
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

View File

@ -28,22 +28,27 @@ namespace skyline::gpu {
std::atomic<ContextTag> tag{}; //!< The tag associated with the last lock call
memory::Buffer backing;
std::optional<GuestBuffer> guest;
std::shared_ptr<FenceCycle> cycle{}; //!< A fence cycle for when any host operation mutating the buffer has completed, it must be waited on prior to any mutations to the backing
span<u8> mirror{}; //!< A contiguous mirror of all the guest mappings to allow linear access on the CPU
span<u8> alignedMirror{}; //!< The mirror mapping aligned to page size to reflect the full mapping
std::optional<nce::NCE::TrapHandle> trapHandle{}; //!< The handle of the traps for the guest mappings
enum class DirtyState {
Clean, //!< The CPU mappings are in sync with the GPU buffer
CpuDirty, //!< The CPU mappings have been modified but the GPU buffer is not up to date
GpuDirty, //!< The GPU buffer has been modified but the CPU mappings have not been updated
};
std::atomic<DirtyState> dirtyState{DirtyState::CpuDirty}; //!< The state of the CPU mappings with respect to the GPU buffer
static_assert(std::atomic<DirtyState>::is_always_lock_free);
} dirtyState{DirtyState::CpuDirty}; //!< The state of the CPU mappings with respect to the GPU buffer
enum class BackingImmutability {
None, //!< Backing can be freely written to and read from
SequencedWrites, //!< Sequenced writes must not modify the backing on the CPU due to it being read directly on the GPU, but non-sequenced writes can freely occur (SynchroniseHost etc)
AllWrites //!< No CPU writes to the backing can be performed, all must be sequenced on the GPU or delayed till this is no longer the case
} backingImmutability{}; //!< Describes how the buffer backing should be accessed by the current context
std::recursive_mutex stateMutex; //!< Synchronizes access to the dirty state and backing immutability
bool everHadInlineUpdate{}; //!< Whether the buffer has ever had an inline update since it was created, if this is set then megabuffering will be attempted by views to avoid the cost of inline GPU updates
bool usedByContext{}; //!< If this buffer is used by the current context, this determines if a buffer needs to be bound to the cycle it is locked by or not
public:
/**
* @brief Storage for all metadata about a specific view into the buffer, used to prevent redundant view creation and duplication of VkBufferView(s)
@ -122,7 +127,11 @@ namespace skyline::gpu {
void SetupGuestMappings();
public:
std::shared_ptr<FenceCycle> cycle{}; //!< A fence cycle for when any host operation mutating the buffer has completed, it must be waited on prior to any mutations to the backing
void UpdateCycle(const std::shared_ptr<FenceCycle> &newCycle) {
std::scoped_lock lock{stateMutex};
newCycle->ChainCycle(cycle);
cycle = newCycle;
}
constexpr vk::Buffer GetBacking() {
return backing.vkBuffer;
@ -159,9 +168,9 @@ namespace skyline::gpu {
void lock();
/**
* @brief Acquires an exclusive lock on the texture for the calling thread
* @param tag A tag to associate with the lock, future invocations with the same tag prior to the unlock will acquire the lock without waiting (0 is not a valid tag value and will disable tag behavior)
* @return If the lock was acquired by this call rather than having the same tag as the holder
* @brief Acquires an exclusive lock on the buffer for the calling thread
* @param tag A tag to associate with the lock, future invocations with the same tag prior to the unlock will acquire the lock without waiting (A default initialised tag will disable this behaviour)
* @return If the lock was acquired by this call as opposed to the buffer already being locked with the same tag
* @note All locks using the same tag **must** be from the same thread as it'll only have one corresponding unlock() call
*/
bool LockWithTag(ContextTag tag);
@ -186,22 +195,54 @@ namespace skyline::gpu {
void MarkGpuDirty();
/**
* @brief Marks the buffer as utilized by the current context, this will be reset on unlocking the buffer
* @brief Prevents sequenced writes to this buffer's backing from occuring on the CPU, forcing sequencing on the GPU instead for the duration of the context. Unsequenced writes such as those from the guest can still occur however.
* @note The buffer **must** be locked prior to calling this
* @note This is significantly different from MarkGpuDirty in that it doesn't imply that the buffer is written to on the GPU and only used on it, this eliminates the requirement to sync-back
*/
void MarkGpuUsed() {
usedByContext = true;
void BlockSequencedCpuBackingWrites() {
std::scoped_lock lock{stateMutex};
if (backingImmutability == BackingImmutability::None)
backingImmutability = BackingImmutability::SequencedWrites;
}
/**
* @return If this buffer has been utilized within the current context
* @note The buffer **must** be locked with a context prior to calling this
* @brief Prevents *any* writes to this buffer's backing from occuring on the CPU, forcing sequencing on the GPU instead for the duration of the context.
* @note The buffer **must** be locked prior to calling this
*/
bool UsedByContext() const {
return usedByContext;
void BlockAllCpuBackingWrites() {
std::scoped_lock lock{stateMutex};
backingImmutability = BackingImmutability::AllWrites;
}
/**
* @return If sequenced writes to the backing must not occur on the CPU
* @note The buffer **must** be locked prior to calling this
*/
bool SequencedCpuBackingWritesBlocked() {
std::scoped_lock lock{stateMutex};
return backingImmutability == BackingImmutability::SequencedWrites || backingImmutability == BackingImmutability::AllWrites;
}
/**
* @return If no writes to the backing are allowed to occur on the CPU
* @note The buffer **must** be locked prior to calling this
*/
bool AllCpuBackingWritesBlocked() {
std::scoped_lock lock{stateMutex};
return backingImmutability == BackingImmutability::AllWrites;
}
/**
* @return If the cycle needs to be attached to the buffer before ending the current context
* @note This is an alias for `SequencedCpuBackingWritesBlocked()` since this is only ever set when the backing is accessed on the GPU in some form
* @note The buffer **must** be locked prior to calling this
*/
bool RequiresCycleAttach() {
return SequencedCpuBackingWritesBlocked();
}
/**
* @note The buffer **must** be locked prior to calling this
*/
bool EverHadInlineUpdate() const {
return everHadInlineUpdate;
}
@ -228,20 +269,19 @@ namespace skyline::gpu {
/**
* @brief Synchronizes the host buffer with the guest
* @param rwTrap If true, the guest buffer will be read/write trapped rather than only being write trapped which is more efficient than calling MarkGpuDirty directly after
* @param skipTrap If true, setting up a CPU trap will be skipped
* @note The buffer **must** be locked prior to calling this
*/
void SynchronizeHost(bool rwTrap = false);
void SynchronizeHost(bool skipTrap = false);
/**
* @brief Synchronizes the guest buffer with the host buffer
* @param skipTrap If true, setting up a CPU trap will be skipped and the dirty state will be Clean/CpuDirty
* @param skipTrap If true, setting up a CPU trap will be skipped
* @param nonBlocking If true, the call will return immediately if the fence is not signalled, skipping the sync
* @param setDirty If true, the buffer will be marked as CpuDirty rather than Clean
* @return If the buffer's contents were successfully synchronized, this'll only be false on non-blocking operations or lack of a guest buffer
* @note The buffer **must** be locked prior to calling this
*/
bool SynchronizeGuest(bool skipTrap = false, bool nonBlocking = false, bool setDirty = false);
bool SynchronizeGuest(bool skipTrap = false, bool nonBlocking = false);
/**
* @brief Synchronizes the guest buffer with the host buffer immediately, flushing GPU work if necessary
@ -321,9 +361,9 @@ namespace skyline::gpu {
}
/**
* @brief Acquires an exclusive lock on the texture for the calling thread
* @param tag A tag to associate with the lock, future invocations with the same tag prior to the unlock will acquire the lock without waiting (0 is not a valid tag value and will disable tag behavior)
* @return If the lock was acquired without waiting (i.e. the tag was the same as the last lock)
* @brief Acquires an exclusive lock on the buffer for the calling thread
* @param tag A tag to associate with the lock, future invocations with the same tag prior to the unlock will acquire the lock without waiting (A default initialised tag will disable this behaviour)
* @return If the lock was acquired by this call as opposed to the buffer already being locked with the same tag
* @note All locks using the same tag **must** be from the same thread as it'll only have one corresponding unlock() call
*/
bool LockWithTag(ContextTag tag) const {

View File

@ -24,7 +24,7 @@ namespace skyline::gpu {
return mutex.try_lock();
}
BufferManager::LockedBuffer::LockedBuffer(std::shared_ptr<Buffer> pBuffer, ContextTag tag) : buffer{std::move(pBuffer)}, lock{tag, *buffer} {}
BufferManager::LockedBuffer::LockedBuffer(std::shared_ptr<Buffer> pBuffer, ContextTag tag) : buffer{std::move(pBuffer)}, lock{tag, *buffer}, stateLock(buffer->stateMutex) {}
Buffer *BufferManager::LockedBuffer::operator->() const {
return buffer.get();
@ -86,14 +86,20 @@ namespace skyline::gpu {
for (auto &srcBuffer : srcBuffers) {
// All newly created buffers that have this set are guaranteed to be attached in buffer FindOrCreate, attach will then lock the buffer without resetting this flag, which will only finally be reset when the lock is released
newBuffer->usedByContext |= srcBuffer->usedByContext;
if (newBuffer->backingImmutability == Buffer::BackingImmutability::None && srcBuffer->backingImmutability != Buffer::BackingImmutability::None)
newBuffer->backingImmutability = srcBuffer->backingImmutability;
else if (srcBuffer->backingImmutability == Buffer::BackingImmutability::AllWrites)
newBuffer->backingImmutability = Buffer::BackingImmutability::AllWrites;
newBuffer->everHadInlineUpdate |= srcBuffer->everHadInlineUpdate;
if (srcBuffer->cycle && newBuffer->cycle != srcBuffer->cycle)
// LockedBuffer also holds `stateMutex` so we can freely access this
if (srcBuffer->cycle && newBuffer->cycle != srcBuffer->cycle) {
if (newBuffer->cycle)
newBuffer->cycle->ChainCycle(srcBuffer->cycle);
else
newBuffer->cycle = srcBuffer->cycle;
}
if (srcBuffer->dirtyState == Buffer::DirtyState::GpuDirty) {
srcBuffer->WaitOnFence();
@ -102,10 +108,11 @@ namespace skyline::gpu {
copyBuffer(*newBuffer->guest, *srcBuffer->guest, newBuffer->mirror.data(), srcBuffer->backing.data());
else
newBuffer->MarkGpuDirty();
} else if (srcBuffer->usedByContext) {
} else if (srcBuffer->AllCpuBackingWritesBlocked()) {
if (srcBuffer->dirtyState == Buffer::DirtyState::CpuDirty)
Logger::Error("Buffer (0x{}-0x{}) is marked as CPU dirty while being utilized by the context, this is not valid", srcBuffer->guest->begin().base(), srcBuffer->guest->end().base());
// We need the backing to be stable so that any accesses within this context are sequenced correctly, we can't use the source mirror here either since buffer writes within this context will update the mirror on CPU and backing on GPU
Logger::Error("Buffer (0x{}-0x{}) is marked as CPU dirty while CPU backing writes are blocked, this is not valid", srcBuffer->guest->begin().base(), srcBuffer->guest->end().base());
// We need the backing to be stable so that any writes within this context are sequenced correctly, we can't use the source mirror here either since buffer writes within this context will update the mirror on CPU and backing on GPU
srcBuffer->WaitOnFence();
copyBuffer(*newBuffer->guest, *srcBuffer->guest, newBuffer->backing.data(), srcBuffer->backing.data());
}

View File

@ -39,6 +39,7 @@ namespace skyline::gpu {
struct LockedBuffer {
std::shared_ptr<Buffer> buffer;
ContextLock<Buffer> lock;
std::unique_lock<std::recursive_mutex> stateLock;
LockedBuffer(std::shared_ptr<Buffer> pBuffer, ContextTag tag);

View File

@ -302,10 +302,9 @@ namespace skyline::gpu::interconnect {
}
for (const auto &attachedBuffer : attachedBuffers) {
if (attachedBuffer->UsedByContext()) {
if (attachedBuffer->RequiresCycleAttach() ) {
cycle->AttachObject(attachedBuffer.buffer);
cycle->ChainCycle(attachedBuffer->cycle);
attachedBuffer->cycle = cycle;
attachedBuffer->UpdateCycle(cycle);
}
}
}

View File

@ -648,6 +648,9 @@ namespace skyline::gpu::interconnect {
Logger::Warn("GPU dirty buffer reads for attached buffers are unimplemented");
}, [&megaBuffer, &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()};
@ -662,7 +665,7 @@ namespace skyline::gpu::interconnect {
}, srcCpuBuf, dstOffset);
}
};
ConstantBuffer constantBufferSelector; //!< The constant buffer selector is used to bind a constant buffer to a stage or update data in it
ConstantBuffer constantBufferSelector{}; //!< The constant buffer selector is used to bind a constant buffer to a stage or update data in it
public:
void SetConstantBufferSelectorSize(u32 size) {
@ -1268,11 +1271,10 @@ namespace skyline::gpu::interconnect {
void BindPipelineConstantBuffer(maxwell3d::PipelineStage stage, bool enable, u32 index) {
auto &constantBuffer{pipelineStages[stage].constantBuffers[index]};
if (enable) {
if (enable)
constantBuffer = GetConstantBufferSelector().value();
} else {
else
constantBuffer = {};
}
}
/* Rasterizer State */

View File

@ -86,7 +86,8 @@ namespace skyline::gpu {
}
/**
* @return If this lock was the first lock on the resource from this context, this effectively means if it was the first usage since prior usages would have to lock the resource
* @return If this lock was the first lock on the resource from this context, this corresponds to it being the first usage since any past usages would need to lock
* @note This is an alias of OwnsLock() with a different name to be more semantically accurate
*/
constexpr bool IsFirstUsage() const {
return ownsLock;