Add an option to enable implicit device synchronization.

1. Auto-generate mutex locking code in DawnNative's ProcTable's
functions. Using a mutex owned by the related Device.
  - Unless the function/class has "no autolock" attribute (new) in
  dawn.json. In which cases, no locking code will be auto-generated.
  - Currently Instance, Adapter, Surface, Encoder classes and
  Reference/Release method have "no autolock".

2. Added Feature::ImplicitDeviceSynchronization to enable Device's
mutex.
  - If this feature is disabled, lock/unlock Device's mutex is no-op.
  Auto-generated locking code will have no effect. This is the default.
  - This approach is used instead of generating two sets of ProcTable
  because it's cleaner and the internal code doesn't need to care about
  whether there is a mutex being locked or not. Furthermore, if there
  were two sets of ProcTable, and user used dawnProcSetProcs() to set
  global ProcTable, it would affect all other modules using different
  Devices. Even though those modules don't need thread safety.

Bug: dawn:1662
Change-Id: I75f0d28959f333318e4159916b259131581f79f5
Reviewed-on: https://dawn-review.googlesource.com/c/dawn/+/119940
Kokoro: Kokoro <noreply+kokoro@google.com>
Commit-Queue: Quyen Le <lehoangquyen@chromium.org>
Reviewed-by: Austin Eng <enga@chromium.org>
This commit is contained in:
Le Hoang Quyen 2023-04-05 19:35:07 +00:00 committed by Dawn LUCI CQ
parent 76b73730c5
commit 7971bfe16f
24 changed files with 479 additions and 51 deletions

View File

@ -77,6 +77,7 @@
},
"adapter": {
"category": "object",
"no autolock": true,
"methods": [
{
"name": "get instance",
@ -579,6 +580,7 @@
},
"command encoder": {
"category": "object",
"no autolock": true,
"methods": [
{
"name": "finish",
@ -792,6 +794,7 @@
},
"compute pass encoder": {
"category": "object",
"no autolock": true,
"methods": [
{
"name": "insert debug marker",
@ -1203,10 +1206,12 @@
},
{
"name": "tick",
"no autolock": true,
"tags": ["dawn"]
},
{
"name": "set uncaptured error callback",
"no autolock": true,
"args": [
{"name": "callback", "type": "error callback"},
{"name": "userdata", "type": "void", "annotation": "*"}
@ -1214,6 +1219,7 @@
},
{
"name": "set logging callback",
"no autolock": true,
"tags": ["dawn"],
"args": [
{"name": "callback", "type": "logging callback"},
@ -1222,6 +1228,7 @@
},
{
"name": "set device lost callback",
"no autolock": true,
"args": [
{"name": "callback", "type": "device lost callback"},
{"name": "userdata", "type": "void", "annotation": "*"}
@ -1469,7 +1476,8 @@
{"value": 1003, "name": "dawn multi planar formats", "tags": ["dawn"]},
{"value": 1004, "name": "dawn native", "tags": ["dawn", "native"]},
{"value": 1005, "name": "chromium experimental dp4a", "tags": ["dawn"]},
{"value": 1006, "name": "timestamp query inside passes", "tags": ["dawn"]}
{"value": 1006, "name": "timestamp query inside passes", "tags": ["dawn"]},
{"value": 1007, "name": "implicit device synchronization", "tags": ["dawn", "native"]}
]
},
"filter mode": {
@ -1527,6 +1535,7 @@
},
"instance": {
"category": "object",
"no autolock": true,
"methods": [
{
"name": "create surface",
@ -1856,6 +1865,7 @@
"render bundle encoder": {
"category": "object",
"no autolock": true,
"methods": [
{
"name": "set pipeline",
@ -2027,6 +2037,7 @@
},
"render pass encoder": {
"category": "object",
"no autolock": true,
"methods": [
{
"name": "set pipeline",
@ -2482,6 +2493,7 @@
},
"surface": {
"category": "object",
"no autolock": true,
"methods": [
{
"name": "get preferred format",

View File

@ -76,6 +76,8 @@ A **record** is a list of **record members**, each of which is a dictionary with
- `"name"` a string
- `"return_type"` (default to no return type) a string that's the name of the return type.
- `"arguments"` a **record**, so an array of **record members**
- `"no autolock"`: a boolean flag (default is false) indicates that the method's generated code won't automatically do thread synchronization. This flag can only be true on device or device child objects currently.
- `"no autolock"`: a boolean flag (default is false) to indicate that the object's generated code won't automatically do thread synchronization. This will override individual method's `"no autolock"` flag. This flag can only be true on device or device child objects currently.
**`"constant"`**
- `"type"`: a string, the name of the base data type

View File

@ -189,8 +189,8 @@ class RecordMember:
self.handle_type = handle_type
Method = namedtuple('Method',
['name', 'return_type', 'arguments', 'json_data'])
Method = namedtuple(
'Method', ['name', 'return_type', 'arguments', 'autolock', 'json_data'])
class ObjectType(Type):
@ -337,11 +337,16 @@ def linked_record_members(json_data, types):
def link_object(obj, types):
# Disable method's autolock if obj's "no autolock" = True
obj_scoped_autolock_enabled = not obj.json_data.get('no autolock', False)
def make_method(json_data):
arguments = linked_record_members(json_data.get('args', []), types)
autolock_enabled = obj_scoped_autolock_enabled and not json_data.get(
'no autolock', False)
return Method(Name(json_data['name']),
types[json_data.get('returns',
'void')], arguments, json_data)
types[json_data.get('returns', 'void')], arguments,
autolock_enabled, json_data)
obj.methods = [make_method(m) for m in obj.json_data.get('methods', [])]
obj.methods.sort(key=lambda method: method.name.canonical_case())
@ -769,9 +774,9 @@ def as_formatType(typ):
def c_methods(params, typ):
return typ.methods + [
x for x in [
Method(Name('reference'), params['types']['void'], [],
Method(Name('reference'), params['types']['void'], [], False,
{'tags': ['dawn', 'emscripten']}),
Method(Name('release'), params['types']['void'], [],
Method(Name('release'), params['types']['void'], [], False,
{'tags': ['dawn', 'emscripten']}),
] if item_is_enabled(params['enabled_tags'], x.json_data)
and not item_is_disabled(params['disabled_tags'], x.json_data)

View File

@ -56,6 +56,17 @@ namespace {{native_namespace}} {
{% endif %}
{%- endfor-%}
{% if method.autolock %}
{% if type.name.get() != "device" %}
auto device = self->GetDevice();
{% else %}
auto device = self;
{% endif %}
auto deviceLock(device->GetScopedLock());
{% else %}
// This method is specified to not use AutoLock in json script.
{% endif %}
{% if method.return_type.name.canonical_case() != "void" %}
auto result =
{%- endif %}

View File

@ -246,6 +246,8 @@ if (is_win || is_linux || is_chromeos || is_mac || is_fuchsia || is_android) {
"Log.h",
"Math.cpp",
"Math.h",
"Mutex.cpp",
"Mutex.h",
"NSRef.h",
"NonCopyable.h",
"Numeric.h",

View File

@ -52,6 +52,8 @@ target_sources(dawn_common PRIVATE
"Log.h"
"Math.cpp"
"Math.h"
"Mutex.cpp"
"Mutex.h"
"NSRef.h"
"NonCopyable.h"
"Numeric.h"

49
src/dawn/common/Mutex.cpp Normal file
View File

@ -0,0 +1,49 @@
// Copyright 2023 The Dawn Authors
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include "dawn/common/Mutex.h"
namespace dawn {
Mutex::~Mutex() = default;
void Mutex::Lock() {
#if defined(DAWN_ENABLE_ASSERTS)
auto currentThread = std::this_thread::get_id();
ASSERT(mOwner.load(std::memory_order_acquire) != currentThread);
#endif // DAWN_ENABLE_ASSERTS
mNativeMutex.lock();
#if defined(DAWN_ENABLE_ASSERTS)
mOwner.store(currentThread, std::memory_order_release);
#endif // DAWN_ENABLE_ASSERTS
}
void Mutex::Unlock() {
#if defined(DAWN_ENABLE_ASSERTS)
ASSERT(IsLockedByCurrentThread());
mOwner.store(std::thread::id(), std::memory_order_release);
#endif // DAWN_ENABLE_ASSERTS
mNativeMutex.unlock();
}
bool Mutex::IsLockedByCurrentThread() {
#if defined(DAWN_ENABLE_ASSERTS)
return mOwner.load(std::memory_order_acquire) == std::this_thread::get_id();
#else
// This is not supported.
abort();
#endif
}
} // namespace dawn

72
src/dawn/common/Mutex.h Normal file
View File

@ -0,0 +1,72 @@
// Copyright 2023 The Dawn Authors
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#ifndef SRC_DAWN_COMMON_MUTEX_H_
#define SRC_DAWN_COMMON_MUTEX_H_
#include <atomic>
#include <mutex>
#include <thread>
#include <utility>
#include "dawn/common/Assert.h"
#include "dawn/common/NonCopyable.h"
#include "dawn/common/RefCounted.h"
namespace dawn {
class Mutex : public RefCounted, NonCopyable {
public:
template <typename MutexRef>
struct AutoLockBase : NonMovable {
AutoLockBase() = delete;
explicit AutoLockBase(MutexRef mutex) : mMutex(std::move(mutex)) {
if (mMutex != nullptr) {
mMutex->Lock();
}
}
~AutoLockBase() {
if (mMutex != nullptr) {
mMutex->Unlock();
}
}
private:
MutexRef mMutex;
};
// This scoped lock won't keep the mutex alive.
using AutoLock = AutoLockBase<Mutex*>;
// This scoped lock will keep the mutex alive until out of scope.
using AutoLockAndHoldRef = AutoLockBase<Ref<Mutex>>;
~Mutex() override;
void Lock();
void Unlock();
// This method is only enabled when DAWN_ENABLE_ASSERTS is turned on.
// Thus it should only be wrapped inside ASSERT() macro.
// i.e. ASSERT(mutex.IsLockedByCurrentThread())
bool IsLockedByCurrentThread();
private:
#if defined(DAWN_ENABLE_ASSERTS)
std::atomic<std::thread::id> mOwner;
#endif // DAWN_ENABLE_ASSERTS
std::mutex mNativeMutex;
};
} // namespace dawn
#endif

View File

@ -15,6 +15,11 @@
#include "dawn/common/RefCounted.h"
#include <cstddef>
#if defined(__has_feature)
#if __has_feature(thread_sanitizer)
#include <sanitizer/tsan_interface.h>
#endif
#endif
#include "dawn/common/Assert.h"
@ -68,6 +73,14 @@ bool RefCount::Decrement() {
// Note that on ARM64 this will generate a `dmb ish` instruction which is a global
// memory barrier, when an acquire load on mRefCount (using the `ldar` instruction)
// should be enough and could end up being faster.
// https://github.com/google/sanitizers/issues/1415 There is false positive bug in TSAN
// when using standalone fence.
#if defined(__has_feature)
#if __has_feature(thread_sanitizer)
__tsan_acquire(&mRefCount);
#endif
#endif
std::atomic_thread_fence(std::memory_order_acquire);
return true;
}
@ -95,6 +108,16 @@ void RefCounted::Release() {
}
}
void RefCounted::ReleaseAndLockBeforeDestroy() {
if (mRefCount.Decrement()) {
LockAndDeleteThis();
}
}
void RefCounted::DeleteThis() {
delete this;
}
void RefCounted::LockAndDeleteThis() {
DeleteThis();
}

View File

@ -46,18 +46,25 @@ class RefCounted {
uint64_t GetRefCountPayload() const;
void Reference();
// Release() is called by internal code, so it's assumed that there is already a thread
// synchronization in place for destruction.
void Release();
void APIReference() { Reference(); }
void APIRelease() { Release(); }
// APIRelease() can be called without any synchronization guarantees so we need to use a Release
// method that will call LockAndDeleteThis() on destruction.
void APIRelease() { ReleaseAndLockBeforeDestroy(); }
protected:
virtual ~RefCounted();
// A Derived class may override this if they require a custom deleter.
virtual void DeleteThis();
void ReleaseAndLockBeforeDestroy();
// A Derived class may override these if they require a custom deleter.
virtual void DeleteThis();
// This calls DeleteThis() by default.
virtual void LockAndDeleteThis();
private:
RefCount mRefCount;
};

View File

@ -42,6 +42,7 @@ MaybeError AdapterBase::Initialize() {
EnableFeature(Feature::DawnNative);
EnableFeature(Feature::DawnInternalUsages);
EnableFeature(Feature::ImplicitDeviceSynchronization);
InitializeSupportedFeaturesImpl();
DAWN_TRY_CONTEXT(

View File

@ -43,9 +43,14 @@ struct MapRequestTask : TrackTaskCallback {
private:
void FinishImpl() override {
ASSERT(mSerial != kMaxExecutionSerial);
TRACE_EVENT1(mPlatform, General, "Buffer::TaskInFlight::Finished", "serial",
uint64_t(mSerial));
{
// This is called from a callback, and no lock will be held by default. Hence, we need
// to lock the mutex now because mSerial might be changed by another thread.
auto deviceLock(buffer->GetDevice()->GetScopedLock());
ASSERT(mSerial != kMaxExecutionSerial);
TRACE_EVENT1(mPlatform, General, "Buffer::TaskInFlight::Finished", "serial",
uint64_t(mSerial));
}
buffer->CallbackOnMapRequestCompleted(id, WGPUBufferMapAsyncStatus_Success);
}
void HandleDeviceLossImpl() override {
@ -597,9 +602,14 @@ MaybeError BufferBase::ValidateUnmap() const {
void BufferBase::CallbackOnMapRequestCompleted(MapRequestID mapID,
WGPUBufferMapAsyncStatus status) {
if (mapID == mLastMapID && status == WGPUBufferMapAsyncStatus_Success &&
mState == BufferState::PendingMap) {
mState = BufferState::Mapped;
{
// This is called from a callback, and no lock will be held by default. Hence, we need to
// lock the mutex now because this will modify the buffer's states.
auto deviceLock(GetDevice()->GetScopedLock());
if (mapID == mLastMapID && status == WGPUBufferMapAsyncStatus_Success &&
mState == BufferState::PendingMap) {
mState = BufferState::Mapped;
}
}
auto cb = PrepareMappingCallback(mapID, status);

View File

@ -1047,14 +1047,21 @@ ResultOrError<std::function<void()>> CommandEncoder::ApplyRenderPassWorkarounds(
descriptor.dimension = wgpu::TextureDimension::e2D;
descriptor.mipLevelCount = 1;
// We are creating new resources. Need to lock the Device.
// TODO(crbug.com/dawn/1618): In future, all temp resources should be created at
// Command Submit time, so the locking would be removed from here at that point.
Ref<TextureBase> temporaryResolveTexture;
DAWN_TRY_ASSIGN(temporaryResolveTexture, device->CreateTexture(&descriptor));
TextureViewDescriptor viewDescriptor = {};
Ref<TextureViewBase> temporaryResolveView;
DAWN_TRY_ASSIGN(
temporaryResolveView,
device->CreateTextureView(temporaryResolveTexture.Get(), &viewDescriptor));
{
auto deviceLock(GetDevice()->GetScopedLock());
DAWN_TRY_ASSIGN(temporaryResolveTexture, device->CreateTexture(&descriptor));
TextureViewDescriptor viewDescriptor = {};
DAWN_TRY_ASSIGN(
temporaryResolveView,
device->CreateTextureView(temporaryResolveTexture.Get(), &viewDescriptor));
}
// Save the temporary and given render targets together for copying after
// the render pass ends.
@ -1200,6 +1207,11 @@ void CommandEncoder::APICopyBufferToTexture(const ImageCopyBuffer* source,
if (dst.aspect == Aspect::Depth &&
GetDevice()->IsToggleEnabled(Toggle::UseBlitForBufferToDepthTextureCopy)) {
// The below function might create new resources. Need to lock the Device.
// TODO(crbug.com/dawn/1618): In future, all temp resources should be created at
// Command Submit time, so the locking would be removed from here at that point.
auto deviceLock(GetDevice()->GetScopedLock());
DAWN_TRY_CONTEXT(
BlitBufferToDepth(GetDevice(), this, source->buffer, srcLayout, dst, *copySize),
"copying from %s to depth aspect of %s using blit workaround.", source->buffer,
@ -1207,6 +1219,11 @@ void CommandEncoder::APICopyBufferToTexture(const ImageCopyBuffer* source,
return {};
} else if (dst.aspect == Aspect::Stencil &&
GetDevice()->IsToggleEnabled(Toggle::UseBlitForBufferToStencilTextureCopy)) {
// The below function might create new resources. Need to lock the Device.
// TODO(crbug.com/dawn/1618): In future, all temp resources should be created at
// Command Submit time, so the locking would be removed from here at that point.
auto deviceLock(GetDevice()->GetScopedLock());
DAWN_TRY_CONTEXT(BlitBufferToStencil(GetDevice(), this, source->buffer, srcLayout,
dst, *copySize),
"copying from %s to stencil aspect of %s using blit workaround.",
@ -1375,6 +1392,11 @@ void CommandEncoder::APICopyTextureToTextureHelper(const ImageCopyTexture* sourc
// Use a blit to copy the depth aspect.
if (blitDepth) {
// This function might create new resources. Need to lock the Device.
// TODO(crbug.com/dawn/1618): In future, all temp resources should be created at
// Command Submit time, so the locking would be removed from here at that point.
auto deviceLock(GetDevice()->GetScopedLock());
DAWN_TRY_CONTEXT(BlitDepthToDepth(GetDevice(), this, src, dst, *copySize),
"copying depth aspect from %s to %s using blit workaround.",
source->texture, destination->texture);
@ -1586,6 +1608,9 @@ void CommandEncoder::APIWriteTimestamp(QuerySetBase* querySet, uint32_t queryInd
}
CommandBufferBase* CommandEncoder::APIFinish(const CommandBufferDescriptor* descriptor) {
// This function will create new object, need to lock the Device.
auto deviceLock(GetDevice()->GetScopedLock());
Ref<CommandBufferBase> commandBuffer;
if (GetDevice()->ConsumedError(Finish(descriptor), &commandBuffer)) {
return CommandBufferBase::MakeError(GetDevice());

View File

@ -238,6 +238,10 @@ ResultOrError<std::pair<Ref<BufferBase>, uint64_t>>
ComputePassEncoder::TransformIndirectDispatchBuffer(Ref<BufferBase> indirectBuffer,
uint64_t indirectOffset) {
DeviceBase* device = GetDevice();
// This function creates new resources, need to lock the Device.
// TODO(crbug.com/dawn/1618): In future, all temp resources should be created at Command Submit
// time, so the locking would be removed from here at that point.
auto deviceLock(GetDevice()->GetScopedLock());
const bool shouldDuplicateNumWorkgroups =
device->ShouldDuplicateNumWorkgroupsForDispatchIndirect(

View File

@ -92,6 +92,10 @@ struct DeviceBase::DeprecationWarnings {
};
namespace {
bool IsMutexLockedByCurrentThreadIfNeeded(const Ref<Mutex>& mutex) {
return mutex == nullptr || mutex->IsLockedByCurrentThread();
}
struct LoggingCallbackTask : CallbackTask {
public:
LoggingCallbackTask() = delete;
@ -277,6 +281,13 @@ MaybeError DeviceBase::Initialize(Ref<QueueBase> defaultQueue) {
CreateShaderModule(&descriptor));
}
if (HasFeature(Feature::ImplicitDeviceSynchronization)) {
mMutex = AcquireRef(new Mutex);
} else {
mMutex = nullptr;
}
// mAdapter is not set for mock test devices.
// TODO(crbug.com/dawn/1702): using a mock adapter could avoid the null checking.
if (mAdapter != nullptr) {
mAdapter->GetInstance()->AddDevice(this);
@ -286,30 +297,38 @@ MaybeError DeviceBase::Initialize(Ref<QueueBase> defaultQueue) {
}
void DeviceBase::WillDropLastExternalRef() {
// DeviceBase uses RefCountedWithExternalCount to break refcycles.
//
// DeviceBase holds multiple Refs to various API objects (pipelines, buffers, etc.) which are
// used to implement various device-level facilities. These objects are cached on the device,
// so we want to keep them around instead of making transient allocations. However, many of
// the objects also hold a Ref<Device> back to their parent device.
//
// In order to break this cycle and prevent leaks, when the application drops the last external
// ref and WillDropLastExternalRef is called, the device clears out any member refs to API
// objects that hold back-refs to the device - thus breaking any reference cycles.
//
// Currently, this is done by calling Destroy on the device to cease all in-flight work and
// drop references to internal objects. We may want to lift this in the future, but it would
// make things more complex because there might be pending tasks which hold a ref back to the
// device - either directly or indirectly. We would need to ensure those tasks don't create new
// reference cycles, and we would need to continuously try draining the pending tasks to clear
// out all remaining refs.
Destroy();
{
// This will be invoked by API side, so we need to lock.
// Note: we cannot hold the lock when flushing the callbacks so have to limit the scope of
// the lock.
auto deviceLock(GetScopedLock());
// DeviceBase uses RefCountedWithExternalCount to break refcycles.
//
// DeviceBase holds multiple Refs to various API objects (pipelines, buffers, etc.) which
// are used to implement various device-level facilities. These objects are cached on the
// device, so we want to keep them around instead of making transient allocations. However,
// many of the objects also hold a Ref<Device> back to their parent device.
//
// In order to break this cycle and prevent leaks, when the application drops the last
// external ref and WillDropLastExternalRef is called, the device clears out any member refs
// to API objects that hold back-refs to the device - thus breaking any reference cycles.
//
// Currently, this is done by calling Destroy on the device to cease all in-flight work and
// drop references to internal objects. We may want to lift this in the future, but it would
// make things more complex because there might be pending tasks which hold a ref back to
// the device - either directly or indirectly. We would need to ensure those tasks don't
// create new reference cycles, and we would need to continuously try draining the pending
// tasks to clear out all remaining refs.
Destroy();
}
// Flush last remaining callback tasks.
do {
mCallbackTaskManager->Flush();
FlushCallbackTaskQueue();
} while (!mCallbackTaskManager->IsEmpty());
auto deviceLock(GetScopedLock());
// Drop te device's reference to the queue. Because the application dropped the last external
// references, they can no longer get the queue from APIGetQueue().
mQueue = nullptr;
@ -570,7 +589,8 @@ void DeviceBase::APISetLoggingCallback(wgpu::LoggingCallback callback, void* use
// resetting) the resources pointed by such pointer may be freed. Flush all deferred
// callback tasks to guarantee we are never going to use the previous callback after
// this call.
mCallbackTaskManager->Flush();
FlushCallbackTaskQueue();
auto deviceLock(GetScopedLock());
if (IsLost()) {
return;
}
@ -584,7 +604,8 @@ void DeviceBase::APISetUncapturedErrorCallback(wgpu::ErrorCallback callback, voi
// resetting) the resources pointed by such pointer may be freed. Flush all deferred
// callback tasks to guarantee we are never going to use the previous callback after
// this call.
mCallbackTaskManager->Flush();
FlushCallbackTaskQueue();
auto deviceLock(GetScopedLock());
if (IsLost()) {
return;
}
@ -598,7 +619,8 @@ void DeviceBase::APISetDeviceLostCallback(wgpu::DeviceLostCallback callback, voi
// resetting) the resources pointed by such pointer may be freed. Flush all deferred
// callback tasks to guarantee we are never going to use the previous callback after
// this call.
mCallbackTaskManager->Flush();
FlushCallbackTaskQueue();
auto deviceLock(GetScopedLock());
if (IsLost()) {
return;
}
@ -851,6 +873,7 @@ Ref<RenderPipelineBase> DeviceBase::GetCachedRenderPipeline(
Ref<ComputePipelineBase> DeviceBase::AddOrGetCachedComputePipeline(
Ref<ComputePipelineBase> computePipeline) {
ASSERT(IsMutexLockedByCurrentThreadIfNeeded(mMutex));
auto [cachedPipeline, inserted] = mCaches->computePipelines.insert(computePipeline.Get());
if (inserted) {
computePipeline->SetIsCachedReference();
@ -862,6 +885,7 @@ Ref<ComputePipelineBase> DeviceBase::AddOrGetCachedComputePipeline(
Ref<RenderPipelineBase> DeviceBase::AddOrGetCachedRenderPipeline(
Ref<RenderPipelineBase> renderPipeline) {
ASSERT(IsMutexLockedByCurrentThreadIfNeeded(mMutex));
auto [cachedPipeline, inserted] = mCaches->renderPipelines.insert(renderPipeline.Get());
if (inserted) {
renderPipeline->SetIsCachedReference();
@ -1267,15 +1291,23 @@ bool DeviceBase::APITick() {
// Tick may trigger callbacks which drop a ref to the device itself. Hold a Ref to ourselves
// to avoid deleting |this| in the middle of this function call.
Ref<DeviceBase> self(this);
if (ConsumedError(Tick())) {
mCallbackTaskManager->Flush();
return false;
bool tickError;
{
// Note: we cannot hold the lock when flushing the callbacks so have to limit the scope of
// the lock here.
auto deviceLock(GetScopedLock());
tickError = ConsumedError(Tick());
}
// We have to check callback tasks in every APITick because it is not related to any global
// serials.
mCallbackTaskManager->Flush();
FlushCallbackTaskQueue();
if (tickError) {
return false;
}
auto deviceLock(GetScopedLock());
// We don't throw an error when device is lost. This allows pending callbacks to be
// executed even after the Device is lost/destroyed.
if (IsLost()) {
@ -1806,6 +1838,26 @@ void DeviceBase::ForceSetToggleForTesting(Toggle toggle, bool isEnabled) {
mToggles.ForceSet(toggle, isEnabled);
}
void DeviceBase::FlushCallbackTaskQueue() {
// Callbacks might cause re-entrances. Mutex shouldn't be locked. So we expect there is no
// locked mutex before entering this method.
ASSERT(mMutex == nullptr || !mMutex->IsLockedByCurrentThread());
Ref<CallbackTaskManager> callbackTaskManager;
{
// This is a data race with the assignment to InstanceBase's callback queue manager in
// WillDropLastExternalRef(). Need to protect with a lock and keep the old
// mCallbackTaskManager alive.
// TODO(crbug.com/dawn/752): In future, all devices should use InstanceBase's callback queue
// manager from the start. So we won't need to care about data race at that point.
auto deviceLock(GetScopedLock());
callbackTaskManager = mCallbackTaskManager;
}
callbackTaskManager->Flush();
}
const CombinedLimits& DeviceBase::GetLimits() const {
return mLimits;
}
@ -1836,7 +1888,15 @@ void DeviceBase::AddComputePipelineAsyncCallbackTask(
// CreateComputePipelineAsyncTaskImpl::Run() when the front-end pipeline cache is
// thread-safe.
ASSERT(mPipeline != nullptr);
mPipeline = mPipeline->GetDevice()->AddOrGetCachedComputePipeline(mPipeline);
{
// This is called inside a callback, and no lock will be held by default so we have
// to lock now to protect the cache.
// Note: we don't lock inside AddOrGetCachedComputePipeline() to avoid deadlock
// because many places calling that method might already have the lock held. For
// example, APICreateComputePipeline()
auto deviceLock(mPipeline->GetDevice()->GetScopedLock());
mPipeline = mPipeline->GetDevice()->AddOrGetCachedComputePipeline(mPipeline);
}
CreateComputePipelineAsyncCallbackTask::FinishImpl();
}
@ -1861,6 +1921,12 @@ void DeviceBase::AddRenderPipelineAsyncCallbackTask(Ref<RenderPipelineBase> pipe
// CreateRenderPipelineAsyncTaskImpl::Run() when the front-end pipeline cache is
// thread-safe.
if (mPipeline.Get() != nullptr) {
// This is called inside a callback, and no lock will be held by default so we have
// to lock now to protect the cache.
// Note: we don't lock inside AddOrGetCachedRenderPipeline() to avoid deadlock
// because many places calling that method might already have the lock held. For
// example, APICreateRenderPipeline()
auto deviceLock(mPipeline->GetDevice()->GetScopedLock());
mPipeline = mPipeline->GetDevice()->AddOrGetCachedRenderPipeline(mPipeline);
}
@ -1974,6 +2040,14 @@ MaybeError DeviceBase::CopyFromStagingToTexture(BufferBase* source,
return {};
}
Mutex::AutoLockAndHoldRef DeviceBase::GetScopedLockSafeForDelete() {
return Mutex::AutoLockAndHoldRef(mMutex);
}
Mutex::AutoLock DeviceBase::GetScopedLock() {
return Mutex::AutoLock(mMutex.Get());
}
IgnoreLazyClearCountScope::IgnoreLazyClearCountScope(DeviceBase* device)
: mDevice(device), mLazyClearCountForTesting(device->mLazyClearCountForTesting) {}

View File

@ -16,12 +16,12 @@
#define SRC_DAWN_NATIVE_DEVICE_H_
#include <memory>
#include <mutex>
#include <string>
#include <unordered_set>
#include <utility>
#include <vector>
#include "dawn/common/Mutex.h"
#include "dawn/native/CacheKey.h"
#include "dawn/native/Commands.h"
#include "dawn/native/ComputePipeline.h"
@ -424,6 +424,13 @@ class DeviceBase : public RefCountedWithExternalCount {
// method makes them to be submitted as soon as possbile in next ticks.
virtual void ForceEventualFlushOfCommands() = 0;
// It is guaranteed that the wrapped mutex will outlive the Device (if the Device is deleted
// before the AutoLockAndHoldRef).
[[nodiscard]] Mutex::AutoLockAndHoldRef GetScopedLockSafeForDelete();
// This lock won't guarantee the wrapped mutex will be alive if the Device is deleted before the
// AutoLock. It would crash if such thing happens.
[[nodiscard]] Mutex::AutoLock GetScopedLock();
// In the 'Normal' mode, currently recorded commands in the backend normally will be actually
// submitted in the next Tick. However in the 'Passive' mode, the submission will be postponed
// as late as possible, for example, until the client has explictly issued a submission.
@ -480,6 +487,7 @@ class DeviceBase : public RefCountedWithExternalCount {
virtual void SetLabelImpl();
virtual MaybeError TickImpl() = 0;
void FlushCallbackTaskQueue();
ResultOrError<Ref<BindGroupLayoutBase>> CreateEmptyBindGroupLayout();
@ -595,6 +603,9 @@ class DeviceBase : public RefCountedWithExternalCount {
std::unique_ptr<dawn::platform::WorkerTaskPool> mWorkerTaskPool;
std::string mLabel;
CacheKey mDeviceCacheKey;
// This pointer is non-null if Feature::ImplicitDeviceSynchronization is turned on.
Ref<Mutex> mMutex = nullptr;
};
ResultOrError<Ref<PipelineLayoutBase>> ValidateLayoutAndGetComputePipelineDescriptorWithDefaults(

View File

@ -87,6 +87,9 @@ void EncodingContext::HandleError(std::unique_ptr<ErrorData> error) {
mError = std::move(error);
}
} else {
// EncodingContext is unprotected from multiple threads by default, but this code will
// modify Device's internal states so we need to lock the device now.
auto deviceLock(mDevice->GetScopedLock());
mDevice->HandleError(std::move(error));
}
}

View File

@ -97,6 +97,11 @@ static constexpr FeatureEnumAndInfoList kFeatureNameAndInfoList = {{
"https://dawn.googlesource.com/dawn/+/refs/heads/main/docs/dawn/features/"
"dawn_native.md",
FeatureInfo::FeatureState::Stable}},
{Feature::ImplicitDeviceSynchronization,
{"implicit-device-sync",
"Public API methods (except encoding) will have implicit device synchronization. So they "
"will be safe to be used on multiple threads.",
"https://bugs.chromium.org/p/dawn/issues/detail?id=1662", FeatureInfo::FeatureState::Stable}},
}};
Feature FromAPIFeature(wgpu::FeatureName feature) {
@ -139,6 +144,8 @@ Feature FromAPIFeature(wgpu::FeatureName feature) {
return Feature::RG11B10UfloatRenderable;
case wgpu::FeatureName::BGRA8UnormStorage:
return Feature::BGRA8UnormStorage;
case wgpu::FeatureName::ImplicitDeviceSynchronization:
return Feature::ImplicitDeviceSynchronization;
}
return Feature::InvalidEnum;
}
@ -177,6 +184,8 @@ wgpu::FeatureName ToAPIFeature(Feature feature) {
return wgpu::FeatureName::RG11B10UfloatRenderable;
case Feature::BGRA8UnormStorage:
return wgpu::FeatureName::BGRA8UnormStorage;
case Feature::ImplicitDeviceSynchronization:
return wgpu::FeatureName::ImplicitDeviceSynchronization;
case Feature::EnumCount:
break;

View File

@ -45,6 +45,7 @@ enum class Feature {
DawnInternalUsages,
MultiPlanarFormats,
DawnNative,
ImplicitDeviceSynchronization,
EnumCount,
InvalidEnum = EnumCount,

View File

@ -96,6 +96,11 @@ void ApiObjectBase::DeleteThis() {
RefCounted::DeleteThis();
}
void ApiObjectBase::LockAndDeleteThis() {
auto deviceLock(GetDevice()->GetScopedLockSafeForDelete());
DeleteThis();
}
ApiObjectList* ApiObjectBase::GetObjectTrackingList() {
ASSERT(GetDevice() != nullptr);
return GetDevice()->GetObjectTrackingList(GetType());

View File

@ -107,6 +107,7 @@ class ApiObjectBase : public ObjectBase, public LinkNode<ApiObjectBase> {
// and they should ensure that their overriding versions call this underlying version
// somewhere.
void DeleteThis() override;
void LockAndDeleteThis() override;
// Returns the list where this object may be tracked for future destruction. This can be
// overrided to create hierarchical object tracking ownership:

View File

@ -293,6 +293,7 @@ dawn_test("dawn_unittests") {
"unittests/LimitsTests.cpp",
"unittests/LinkedListTests.cpp",
"unittests/MathTests.cpp",
"unittests/MutexTests.cpp",
"unittests/NumericTests.cpp",
"unittests/ObjectBaseTests.cpp",
"unittests/PerStageTests.cpp",

View File

@ -0,0 +1,97 @@
// Copyright 2023 The Dawn Authors
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include "dawn/common/Mutex.h"
#include "gtest/gtest.h"
namespace {
#if defined(DAWN_ENABLE_ASSERTS)
constexpr bool kAssertEnabled = true;
#else
constexpr bool kAssertEnabled = false;
#endif
} // namespace
class MutexTest : public ::testing::Test {
protected:
void SetUp() override {
// IsLockedByCurrentThread() requires DAWN_ENABLE_ASSERTS flag enabled.
if (!kAssertEnabled) {
GTEST_SKIP() << "DAWN_ENABLE_ASSERTS is not enabled";
}
}
dawn::Mutex mMutex;
};
// Simple Lock() then Unlock() test.
TEST_F(MutexTest, SimpleLockUnlock) {
mMutex.Lock();
EXPECT_TRUE(mMutex.IsLockedByCurrentThread());
mMutex.Unlock();
EXPECT_FALSE(mMutex.IsLockedByCurrentThread());
}
// Test AutoLock automatically locks the mutex and unlocks it when out of scope.
TEST_F(MutexTest, AutoLock) {
{
dawn::Mutex::AutoLock autoLock(&mMutex);
EXPECT_TRUE(mMutex.IsLockedByCurrentThread());
}
EXPECT_FALSE(mMutex.IsLockedByCurrentThread());
}
// Test AutoLockAndHoldRef will keep the mutex alive
TEST_F(MutexTest, AutoLockAndHoldRef) {
auto* mutex = new dawn::Mutex();
EXPECT_EQ(mutex->GetRefCountForTesting(), 1u);
{
dawn::Mutex::AutoLockAndHoldRef autoLock(mutex);
EXPECT_TRUE(mutex->IsLockedByCurrentThread());
EXPECT_EQ(mutex->GetRefCountForTesting(), 2u);
mutex->Release();
EXPECT_EQ(mutex->GetRefCountForTesting(), 1u);
}
}
using MutexDeathTest = MutexTest;
// Test that Unlock() call on unlocked mutex will cause assertion failure.
TEST_F(MutexDeathTest, UnlockWhenNotLocked) {
ASSERT_DEATH({ mMutex.Unlock(); }, "");
}
// Double Lock() calls should be cause assertion failure
TEST_F(MutexDeathTest, DoubleLockCalls) {
mMutex.Lock();
EXPECT_TRUE(mMutex.IsLockedByCurrentThread());
ASSERT_DEATH({ mMutex.Lock(); }, "");
mMutex.Unlock();
}
// Lock() then use AutoLock should cause assertion failure.
TEST_F(MutexDeathTest, LockThenAutoLock) {
mMutex.Lock();
EXPECT_TRUE(mMutex.IsLockedByCurrentThread());
ASSERT_DEATH({ dawn::Mutex::AutoLock autoLock(&mMutex); }, "");
mMutex.Unlock();
}
// Use AutoLock then call Lock() should cause assertion failure.
TEST_F(MutexDeathTest, AutoLockThenLock) {
dawn::Mutex::AutoLock autoLock(&mMutex);
EXPECT_TRUE(mMutex.IsLockedByCurrentThread());
ASSERT_DEATH({ mMutex.Lock(); }, "");
}

View File

@ -24,6 +24,7 @@ bool IsFeatureSupported(WGPUFeatureName feature) {
case WGPUFeatureName_Force32:
case WGPUFeatureName_DawnNative:
case WGPUFeatureName_DawnShaderFloat16: // Deprecated
case WGPUFeatureName_ImplicitDeviceSynchronization:
return false;
case WGPUFeatureName_Depth32FloatStencil8:
case WGPUFeatureName_TimestampQuery: