GPU: Converge ShaderCompiler implementations

Part of #136993.

Share as much of the ShaderCompiler implementations as possible.
Remove the ShaderCompiler/ShaderCompilerGeneric split and make most of
its functions non virtual.
Move the `get_compiler` function from `Context` to `GPUBackend` and
creation/deletion to `GPUBackend::init/delete_resources`.
Add a `batch_cancel` function to `ShaderCompiler` (needed for the
GPUPass refactor).

As a nice extra, the multithreaded OpenGL compilation has become faster
too.
The barbershop materials + EEVEE static shaders have gone from 27s to
22s.

I have not observed any performance difference on Vulkan or Metal.

Pull Request: https://projects.blender.org/blender/blender/pulls/136676
This commit is contained in:
Miguel Pozo
2025-05-08 18:16:47 +02:00
parent 12decaf13c
commit 992e7c95a7
27 changed files with 435 additions and 1113 deletions

View File

@@ -127,7 +127,7 @@ bool ShaderModule::request_specializations(bool block_until_ready,
std::lock_guard lock(mutex_);
SpecializationBatchHandle specialization_handle = specialization_handles_.lookup_or_add_cb(
SpecializationBatchHandle &specialization_handle = specialization_handles_.lookup_or_add_cb(
{render_buffers_shadow_id, shadow_ray_count, shadow_ray_step_count}, [&]() {
Vector<ShaderSpecialization> specializations;
for (int i = 0; i < 3; i++) {

View File

@@ -44,6 +44,7 @@ const char *GPU_extension_get(int i);
int GPU_texture_size_with_limit(int res);
bool GPU_use_parallel_compilation();
int GPU_max_parallel_compilations();
bool GPU_stencil_clasify_buffer_workaround();
bool GPU_node_link_instancing_workaround();

View File

@@ -27,13 +27,19 @@ class GPUWorker {
std::atomic<bool> terminate_ = false;
public:
enum class ContextType {
/* Use the main GPU context on the worker threads. */
Main,
/* Use a different secondary GPU context for each worker thread. */
PerThread,
};
/**
* \param threads_count: Number of threads to span.
* \param share_context: If true, all threads will use the same secondary GPUContext,
* otherwise each thread will have its own unique GPUContext.
* \param context_type: The type of context each thread uses.
* \param run_cb: The callback function that will be called by a thread on `wake_up()`.
*/
GPUWorker(uint32_t threads_count, bool share_context, std::function<void()> run_cb);
GPUWorker(uint32_t threads_count, ContextType context_type, std::function<void()> run_cb);
~GPUWorker();
/* Wake up a single thread. */

View File

@@ -32,11 +32,6 @@ class DummyContext : public Context {
void flush() override {}
void finish() override {}
ShaderCompiler *get_compiler() override
{
return nullptr;
}
void memory_statistics_get(int * /*r_total_mem*/, int * /*r_free_mem*/) override {}
void debug_group_begin(const char * /*unused*/, int /*unused*/) override {}

View File

@@ -25,12 +25,16 @@ class IndexBuf;
class PixelBuffer;
class QueryPool;
class Shader;
class ShaderCompiler;
class Texture;
class UniformBuf;
class StorageBuf;
class VertBuf;
class GPUBackend {
protected:
ShaderCompiler *compiler_;
public:
virtual ~GPUBackend() = default;
@@ -41,6 +45,11 @@ class GPUBackend {
static GPUBackend *get();
ShaderCompiler *get_compiler()
{
return compiler_;
}
virtual void samplers_update() = 0;
virtual void compute_dispatch(int groups_x_len, int groups_y_len, int groups_z_len) = 0;
virtual void compute_dispatch_indirect(StorageBuf *indirect_buf) = 0;

View File

@@ -136,6 +136,11 @@ bool GPU_use_parallel_compilation()
return GCaps.max_parallel_compilations > 0;
}
int GPU_max_parallel_compilations()
{
return GCaps.max_parallel_compilations;
}
bool GPU_mip_render_workaround()
{
return GCaps.mip_render_workaround;

View File

@@ -101,8 +101,6 @@ class Context {
/* Will wait until the GPU has finished executing all command. */
virtual void finish() = 0;
virtual ShaderCompiler *get_compiler() = 0;
virtual void memory_statistics_get(int *r_total_mem, int *r_free_mem) = 0;
virtual void debug_group_begin(const char * /*name*/, int /*index*/){};

View File

@@ -266,7 +266,7 @@ GPUShader *GPU_shader_create_from_info(const GPUShaderCreateInfo *_info)
{
using namespace blender::gpu::shader;
const ShaderCreateInfo &info = *reinterpret_cast<const ShaderCreateInfo *>(_info);
return wrap(Context::get()->get_compiler()->compile(info, false));
return wrap(GPUBackend::get()->get_compiler()->compile(info, false));
}
std::string GPU_shader_preprocess_source(StringRefNull original)
@@ -294,7 +294,7 @@ GPUShader *GPU_shader_create_from_info_python(const GPUShaderCreateInfo *_info)
info.geometry_source_generated = GPU_shader_preprocess_source(info.geometry_source_generated);
info.compute_source_generated = GPU_shader_preprocess_source(info.compute_source_generated);
GPUShader *result = wrap(Context::get()->get_compiler()->compile(info, false));
GPUShader *result = wrap(GPUBackend::get()->get_compiler()->compile(info, false));
info.vertex_source_generated = vertex_source_original;
info.fragment_source_generated = fragment_source_original;
@@ -366,17 +366,17 @@ BatchHandle GPU_shader_batch_create_from_infos(Span<const GPUShaderCreateInfo *>
using namespace blender::gpu::shader;
Span<const ShaderCreateInfo *> &infos_ = reinterpret_cast<Span<const ShaderCreateInfo *> &>(
infos);
return Context::get()->get_compiler()->batch_compile(infos_);
return GPUBackend::get()->get_compiler()->batch_compile(infos_);
}
bool GPU_shader_batch_is_ready(BatchHandle handle)
{
return Context::get()->get_compiler()->batch_is_ready(handle);
return GPUBackend::get()->get_compiler()->batch_is_ready(handle);
}
Vector<GPUShader *> GPU_shader_batch_finalize(BatchHandle &handle)
{
Vector<Shader *> result = Context::get()->get_compiler()->batch_finalize(handle);
Vector<Shader *> result = GPUBackend::get()->get_compiler()->batch_finalize(handle);
return reinterpret_cast<Vector<GPUShader *> &>(result);
}
@@ -544,12 +544,12 @@ void GPU_shader_constant_bool(GPUShader *sh, const char *name, bool value)
SpecializationBatchHandle GPU_shader_batch_specializations(
blender::Span<ShaderSpecialization> specializations)
{
return Context::get()->get_compiler()->precompile_specializations(specializations);
return GPUBackend::get()->get_compiler()->precompile_specializations(specializations);
}
bool GPU_shader_batch_specializations_is_ready(SpecializationBatchHandle &handle)
{
return Context::get()->get_compiler()->specialization_batch_is_ready(handle);
return GPUBackend::get()->get_compiler()->specialization_batch_is_ready(handle);
}
/** \} */
@@ -819,7 +819,10 @@ Shader *ShaderCompiler::compile(const shader::ShaderCreateInfo &info, bool is_ba
using namespace blender::gpu::shader;
const_cast<ShaderCreateInfo &>(info).finalize();
GPU_debug_group_begin(GPU_DEBUG_SHADER_COMPILATION_GROUP);
if (Context::get()) {
/* Context can be null in Vulkan compilation threads. */
GPU_debug_group_begin(GPU_DEBUG_SHADER_COMPILATION_GROUP);
}
const std::string error = info.check_error();
if (!error.empty()) {
@@ -933,102 +936,193 @@ Shader *ShaderCompiler::compile(const shader::ShaderCreateInfo &info, bool is_ba
if (!shader->finalize(&info)) {
delete shader;
GPU_debug_group_end();
return nullptr;
shader = nullptr;
}
if (Context::get()) {
/* Context can be null in Vulkan compilation threads. */
GPU_debug_group_end();
}
GPU_debug_group_end();
return shader;
}
/** \} */
/* -------------------------------------------------------------------- */
/** \name ShaderCompilerGeneric
* \{ */
ShaderCompilerGeneric::ShaderCompilerGeneric()
ShaderCompiler::ShaderCompiler(uint32_t threads_count,
GPUWorker::ContextType context_type,
bool support_specializations)
{
support_specializations_ = support_specializations;
if (!GPU_use_main_context_workaround()) {
compilation_thread_ = std::make_unique<GPUWorker>(1, true, [this]() { this->run_thread(); });
compilation_worker_ = std::make_unique<GPUWorker>(
threads_count, context_type, [this]() { this->run_thread(); });
}
}
ShaderCompilerGeneric::~ShaderCompilerGeneric()
ShaderCompiler::~ShaderCompiler()
{
compilation_thread_.reset();
compilation_worker_.reset();
/* Ensure all the requested batches have been retrieved. */
BLI_assert(batches_.is_empty());
}
BatchHandle ShaderCompilerGeneric::batch_compile(Span<const shader::ShaderCreateInfo *> &infos)
Shader *ShaderCompiler::compile_shader(const shader::ShaderCreateInfo &info)
{
return compile(info, false);
}
BatchHandle ShaderCompiler::batch_compile(Span<const shader::ShaderCreateInfo *> &infos)
{
std::unique_lock lock(mutex_);
BatchHandle handle = next_batch_handle_++;
batches_.add(handle, std::make_unique<Batch>());
Batch *batch = batches_.lookup(handle).get();
Batch *batch = MEM_new<Batch>(__func__);
batch->infos = infos;
batch->shaders.reserve(infos.size());
if (compilation_thread_) {
compilation_queue_.push_back(batch);
lock.unlock();
compilation_thread_->wake_up();
BatchHandle handle = next_batch_handle_++;
batches_.add(handle, batch);
if (compilation_worker_) {
batch->shaders.resize(infos.size(), nullptr);
batch->pending_compilations = infos.size();
for (int i : infos.index_range()) {
compilation_queue_.push_back({batch, i});
compilation_worker_->wake_up();
}
}
else {
for (const shader::ShaderCreateInfo *info : infos) {
batch->shaders.append(compile(*info, false));
}
batch->is_ready = true;
}
return handle;
}
bool ShaderCompilerGeneric::batch_is_ready(BatchHandle handle)
void ShaderCompiler::batch_cancel(BatchHandle &handle)
{
std::lock_guard lock(mutex_);
bool is_ready = batches_.lookup(handle)->is_ready;
return is_ready;
Batch *batch = batches_.pop(handle);
for (ParallelWork &work : compilation_queue_) {
if (work.batch == batch) {
work = {};
batch->pending_compilations--;
}
}
compilation_queue_.erase(std::remove_if(compilation_queue_.begin(),
compilation_queue_.end(),
[](const ParallelWork &work) { return !work.batch; }));
if (batch->is_ready()) {
batch->free_shaders();
MEM_delete(batch);
}
else {
/* If it's currently compiling, the compilation thread makes the cleanup. */
batch->is_cancelled = true;
}
handle = 0;
}
Vector<Shader *> ShaderCompilerGeneric::batch_finalize(BatchHandle &handle)
bool ShaderCompiler::batch_is_ready(BatchHandle handle)
{
while (!batch_is_ready(handle)) {
BLI_time_sleep_ms(1);
std::lock_guard lock(mutex_);
return batches_.lookup(handle)->is_ready();
}
Vector<Shader *> ShaderCompiler::batch_finalize(BatchHandle &handle)
{
std::unique_lock lock(mutex_);
compilation_finished_notification_.wait(lock,
[&]() { return batches_.lookup(handle)->is_ready(); });
Batch *batch = batches_.pop(handle);
Vector<Shader *> shaders = std::move(batch->shaders);
MEM_delete(batch);
handle = 0;
return shaders;
}
SpecializationBatchHandle ShaderCompiler::precompile_specializations(
Span<ShaderSpecialization> specializations)
{
if (!compilation_worker_ || !support_specializations_) {
return 0;
}
std::lock_guard lock(mutex_);
Vector<Shader *> shaders = batches_.lookup(handle)->shaders;
batches_.pop(handle);
handle = 0;
return shaders;
Batch *batch = MEM_new<Batch>(__func__);
batch->specializations = specializations;
BatchHandle handle = next_batch_handle_++;
batches_.add(handle, batch);
batch->pending_compilations = specializations.size();
for (int i : specializations.index_range()) {
compilation_queue_.push_back({batch, i});
compilation_worker_->wake_up();
}
return handle;
}
void ShaderCompilerGeneric::run_thread()
bool ShaderCompiler::specialization_batch_is_ready(SpecializationBatchHandle &handle)
{
if (handle != 0 && batch_is_ready(handle)) {
std::lock_guard lock(mutex_);
Batch *batch = batches_.pop(handle);
MEM_delete(batch);
handle = 0;
}
return handle == 0;
}
void ShaderCompiler::run_thread()
{
while (true) {
Batch *batch = nullptr;
Batch *batch;
int shader_index;
{
std::unique_lock<std::mutex> lock(mutex_);
std::lock_guard lock(mutex_);
if (compilation_queue_.empty()) {
return;
}
batch = compilation_queue_.front();
ParallelWork &work = compilation_queue_.front();
batch = work.batch;
shader_index = work.shader_index;
compilation_queue_.pop_front();
}
/* Compile */
for (const shader::ShaderCreateInfo *info : batch->infos) {
batch->shaders.append(compile(*info, false));
if (!batch->is_specialization_batch()) {
batch->shaders[shader_index] = compile_shader(*batch->infos[shader_index]);
}
batch->is_ready = true;
else {
specialize_shader(batch->specializations[shader_index]);
}
{
std::lock_guard lock(mutex_);
batch->pending_compilations--;
if (batch->is_ready() && batch->is_cancelled) {
batch->free_shaders();
MEM_delete(batch);
}
}
compilation_finished_notification_.notify_all();
}
}

View File

@@ -84,8 +84,7 @@ class Shader {
Shader(const char *name);
virtual ~Shader();
/* `is_batch_compilation` is true when the shader is being compiled as part of a
* `GPU_shader_batch`. Backends that use the `ShaderCompilerGeneric` can ignore it. */
/* TODO: Remove `is_batch_compilation`. */
virtual void init(const shader::ShaderCreateInfo &info, bool is_batch_compilation) = 0;
virtual void vertex_shader_from_glsl(MutableSpan<StringRefNull> sources) = 0;
@@ -157,7 +156,6 @@ static inline const Shader *unwrap(const GPUShader *vert)
}
class ShaderCompiler {
protected:
struct Sources {
std::string vert;
std::string geom;
@@ -165,53 +163,73 @@ class ShaderCompiler {
std::string comp;
};
public:
virtual ~ShaderCompiler() = default;
Shader *compile(const shader::ShaderCreateInfo &info, bool is_batch_compilation);
virtual BatchHandle batch_compile(Span<const shader::ShaderCreateInfo *> &infos) = 0;
virtual bool batch_is_ready(BatchHandle handle) = 0;
virtual Vector<Shader *> batch_finalize(BatchHandle &handle) = 0;
virtual SpecializationBatchHandle precompile_specializations(
Span<ShaderSpecialization> /*specializations*/)
{
/* No-op. */
return 0;
};
virtual bool specialization_batch_is_ready(SpecializationBatchHandle &handle)
{
handle = 0;
return true;
};
};
/* Generic implementation used as fallback. */
class ShaderCompilerGeneric : public ShaderCompiler {
private:
struct Batch {
Vector<Shader *> shaders;
Vector<const shader::ShaderCreateInfo *> infos;
std::atomic_bool is_ready = false;
};
BatchHandle next_batch_handle_ = 1;
Map<BatchHandle, std::unique_ptr<Batch>> batches_;
std::mutex mutex_;
std::deque<Batch *> compilation_queue_;
std::unique_ptr<GPUWorker> compilation_thread_;
Vector<ShaderSpecialization> specializations;
std::atomic<int> pending_compilations = 0;
std::atomic<bool> is_cancelled = false;
bool is_specialization_batch()
{
return !specializations.is_empty();
}
bool is_ready()
{
BLI_assert(pending_compilations >= 0);
return pending_compilations == 0;
}
void free_shaders()
{
for (Shader *shader : shaders) {
if (shader) {
GPU_shader_free(wrap(shader));
}
}
shaders.clear();
}
};
Map<BatchHandle, Batch *> batches_;
std::mutex mutex_;
std::condition_variable compilation_finished_notification_;
struct ParallelWork {
Batch *batch = nullptr;
int shader_index = 0;
};
std::deque<ParallelWork> compilation_queue_;
std::unique_ptr<GPUWorker> compilation_worker_;
bool support_specializations_;
void run_thread();
public:
ShaderCompilerGeneric();
~ShaderCompilerGeneric() override;
BatchHandle next_batch_handle_ = 1;
BatchHandle batch_compile(Span<const shader::ShaderCreateInfo *> &infos) override;
bool batch_is_ready(BatchHandle handle) override;
Vector<Shader *> batch_finalize(BatchHandle &handle) override;
public:
ShaderCompiler(uint32_t threads_count = 1,
GPUWorker::ContextType context_type = GPUWorker::ContextType::PerThread,
bool support_specializations = false);
virtual ~ShaderCompiler();
Shader *compile(const shader::ShaderCreateInfo &info, bool is_batch_compilation);
virtual Shader *compile_shader(const shader::ShaderCreateInfo &info);
virtual void specialize_shader(ShaderSpecialization & /*specialization*/){};
BatchHandle batch_compile(Span<const shader::ShaderCreateInfo *> &infos);
void batch_cancel(BatchHandle &handle);
bool batch_is_ready(BatchHandle handle);
Vector<Shader *> batch_finalize(BatchHandle &handle);
SpecializationBatchHandle precompile_specializations(Span<ShaderSpecialization> specializations);
bool specialization_batch_is_ready(SpecializationBatchHandle &handle);
};
enum class Severity {

View File

@@ -6,17 +6,14 @@
namespace blender::gpu {
GPUWorker::GPUWorker(uint32_t threads_count, bool share_context, std::function<void()> run_cb)
GPUWorker::GPUWorker(uint32_t threads_count,
ContextType context_type,
std::function<void()> run_cb)
{
std::shared_ptr<GPUSecondaryContext> shared_context = nullptr;
if (share_context) {
shared_context = std::make_shared<GPUSecondaryContext>();
}
for (int i : IndexRange(threads_count)) {
UNUSED_VARS(i);
std::shared_ptr<GPUSecondaryContext> thread_context =
share_context ? shared_context : std::make_shared<GPUSecondaryContext>();
context_type == ContextType::PerThread ? std::make_shared<GPUSecondaryContext>() : nullptr;
threads_.append(std::make_unique<std::thread>([=]() { this->run(thread_context, run_cb); }));
}
}
@@ -32,7 +29,9 @@ GPUWorker::~GPUWorker()
void GPUWorker::run(std::shared_ptr<GPUSecondaryContext> context, std::function<void()> run_cb)
{
context->activate();
if (context) {
context->activate();
}
/* Loop until we get the terminate signal. */
while (!terminate_) {

View File

@@ -11,6 +11,7 @@
#include "BLI_vector.hh"
#include "gpu_backend.hh"
#include "gpu_shader_private.hh"
#include "mtl_capabilities.hh"
namespace blender::gpu {
@@ -40,15 +41,9 @@ class MTLBackend : public GPUBackend {
MTLBackend::platform_exit();
}
void init_resources() override
{
/* Create any resources with context active. */
}
void init_resources() override;
void delete_resources() override
{
/* Delete any resources with context active. */
}
void delete_resources() override;
static bool metal_is_supported();
static MTLBackend *get()

View File

@@ -41,6 +41,21 @@ thread_local int g_autoreleasepool_depth = 0;
/** \name Metal Backend
* \{ */
void MTLBackend::init_resources()
{
if (GPU_use_parallel_compilation()) {
compiler_ = MEM_new<MTLShaderCompiler>(__func__);
}
else {
compiler_ = MEM_new<ShaderCompiler>(__func__);
}
}
void MTLBackend::delete_resources()
{
MEM_delete(compiler_);
}
void MTLBackend::samplers_update(){
/* Placeholder -- Handled in MTLContext. */
};

View File

@@ -776,8 +776,6 @@ class MTLContext : public Context {
GPUVertFormat dummy_vertformat_[GPU_SAMPLER_TYPE_MAX];
VertBuf *dummy_verts_[GPU_SAMPLER_TYPE_MAX] = {nullptr};
ShaderCompiler *compiler;
public:
/* GPUContext interface. */
MTLContext(void *ghost_window, void *ghost_context);
@@ -793,11 +791,6 @@ class MTLContext : public Context {
void flush() override;
void finish() override;
ShaderCompiler *get_compiler() override
{
return compiler;
}
void memory_statistics_get(int *r_total_mem, int *r_free_mem) override;
static MTLContext *get()

View File

@@ -266,13 +266,6 @@ MTLContext::MTLContext(void *ghost_window, void *ghost_context)
/* Initialize samplers. */
this->sampler_state_cache_init();
if (GPU_use_parallel_compilation()) {
compiler = new MTLShaderCompiler();
}
else {
compiler = new ShaderCompilerGeneric();
}
}
MTLContext::~MTLContext()
@@ -381,8 +374,6 @@ MTLContext::~MTLContext()
if (this->device) {
[this->device release];
}
delete compiler;
}
void MTLContext::begin_frame()

View File

@@ -321,94 +321,12 @@ class MTLShader : public Shader {
MEM_CXX_CLASS_ALLOC_FUNCS("MTLShader");
};
class MTLParallelShaderCompiler {
private:
enum ParallelWorkType {
PARALLELWORKTYPE_UNSPECIFIED,
PARALLELWORKTYPE_COMPILE_SHADER,
PARALLELWORKTYPE_BAKE_PSO,
};
struct ParallelWork {
const shader::ShaderCreateInfo *info = nullptr;
class MTLShaderCompiler *shader_compiler = nullptr;
MTLShader *shader = nullptr;
Vector<Shader::Constants::Value> specialization_values;
ParallelWorkType work_type = PARALLELWORKTYPE_UNSPECIFIED;
bool is_ready = false;
};
struct Batch {
Vector<ParallelWork *> items;
bool is_ready = false;
};
std::mutex batch_mutex;
BatchHandle next_batch_handle = 1;
Map<BatchHandle, Batch> batches;
std::vector<std::thread> compile_threads;
volatile bool terminate_compile_threads;
std::condition_variable cond_var;
std::mutex queue_mutex;
std::deque<ParallelWork *> parallel_work_queue;
void parallel_compilation_thread_func(GPUContext *blender_gpu_context,
GHOST_ContextHandle ghost_gpu_context);
BatchHandle create_batch(size_t batch_size);
void add_item_to_batch(ParallelWork *work_item, BatchHandle batch_handle);
void add_parallel_item_to_queue(ParallelWork *add_parallel_item_to_queuework_item,
BatchHandle batch_handle);
std::atomic<int> ref_count = 1;
public:
MTLParallelShaderCompiler();
~MTLParallelShaderCompiler();
void create_compile_threads();
BatchHandle batch_compile(MTLShaderCompiler *shade_compiler,
Span<const shader::ShaderCreateInfo *> &infos);
bool batch_is_ready(BatchHandle handle);
Vector<Shader *> batch_finalize(BatchHandle &handle);
SpecializationBatchHandle precompile_specializations(Span<ShaderSpecialization> specializations);
bool specialization_batch_is_ready(SpecializationBatchHandle &handle);
void increment_ref_count()
{
ref_count++;
}
void decrement_ref_count()
{
BLI_assert(ref_count > 0);
ref_count--;
}
int get_ref_count()
{
return ref_count;
}
};
class MTLShaderCompiler : public ShaderCompiler {
private:
MTLParallelShaderCompiler *parallel_shader_compiler;
public:
MTLShaderCompiler();
virtual ~MTLShaderCompiler() override;
virtual BatchHandle batch_compile(Span<const shader::ShaderCreateInfo *> &infos) override;
virtual bool batch_is_ready(BatchHandle handle) override;
virtual Vector<Shader *> batch_finalize(BatchHandle &handle) override;
virtual SpecializationBatchHandle precompile_specializations(
Span<ShaderSpecialization> specializations) override;
virtual bool specialization_batch_is_ready(SpecializationBatchHandle &handle) override;
void release_parallel_shader_compiler();
Shader *compile_shader(const shader::ShaderCreateInfo &info) override;
void specialize_shader(ShaderSpecialization &specialization) override;
};
/* Vertex format conversion.

View File

@@ -1543,447 +1543,55 @@ MTLComputePipelineStateInstance *MTLShader::bake_compute_pipeline_state(
}
/** \} */
/* Since this is going to be compiling shaders in a multi-threaded fashion we
* don't want to create an instance per context as we want to restrict the
* number of simultaneous compilation threads to ensure system responsiveness.
* Hence the global shared instance. */
MTLParallelShaderCompiler *g_shared_parallel_shader_compiler = nullptr;
std::mutex g_shared_parallel_shader_compiler_mutex;
MTLParallelShaderCompiler *get_shared_parallel_shader_compiler()
{
std::scoped_lock lock(g_shared_parallel_shader_compiler_mutex);
if (!g_shared_parallel_shader_compiler) {
g_shared_parallel_shader_compiler = new MTLParallelShaderCompiler();
}
else {
g_shared_parallel_shader_compiler->increment_ref_count();
}
return g_shared_parallel_shader_compiler;
}
void release_shared_parallel_shader_compiler()
{
std::scoped_lock lock(g_shared_parallel_shader_compiler_mutex);
if (!g_shared_parallel_shader_compiler) {
return;
}
g_shared_parallel_shader_compiler->decrement_ref_count();
if (g_shared_parallel_shader_compiler->get_ref_count() == 0) {
delete g_shared_parallel_shader_compiler;
g_shared_parallel_shader_compiler = nullptr;
}
}
/* -------------------------------------------------------------------- */
/** \name MTLParallelShaderCompiler
* \{ */
MTLParallelShaderCompiler::MTLParallelShaderCompiler()
{
BLI_assert(GPU_use_parallel_compilation());
terminate_compile_threads = false;
}
MTLParallelShaderCompiler::~MTLParallelShaderCompiler()
{
/* Shutdown the compiler threads. */
terminate_compile_threads = true;
cond_var.notify_all();
for (auto &thread : compile_threads) {
thread.join();
}
/* Mark any unprocessed work items as ready so we can move
* them into a batch for cleanup. */
if (!parallel_work_queue.empty()) {
std::unique_lock<std::mutex> lock(queue_mutex);
while (!parallel_work_queue.empty()) {
ParallelWork *work_item = parallel_work_queue.front();
work_item->is_ready = true;
parallel_work_queue.pop_front();
}
}
/* Clean up any outstanding batches. */
for (BatchHandle handle : batches.keys()) {
Vector<Shader *> shaders = batch_finalize(handle);
/* Delete any shaders in the batch. */
for (Shader *shader : shaders) {
if (shader) {
delete shader;
}
}
}
BLI_assert(batches.is_empty());
}
void MTLParallelShaderCompiler::create_compile_threads()
{
std::unique_lock<std::mutex> lock(queue_mutex);
/* Return if the compilation threads already exist */
if (!compile_threads.empty()) {
return;
}
/* Limit to the number of compiler threads to (performance cores - 1) to
* leave one thread free for main thread/UI responsiveness */
const MTLCapabilities &capabilities = MTLBackend::get_capabilities();
int max_mtlcompiler_threads = capabilities.num_performance_cores - 1;
/* Save the main thread context */
GPUContext *main_thread_context = GPU_context_active_get();
MTLContext *metal_context = static_cast<MTLContext *>(unwrap(main_thread_context));
id<MTLDevice> metal_device = metal_context->device;
#if defined(MAC_OS_VERSION_13_3)
/* Clamp the number of threads if necessary. */
if (@available(macOS 13.3, *)) {
/* Check we've set the flag to allow more than 2 compile threads. */
BLI_assert(metal_device.shouldMaximizeConcurrentCompilation);
max_mtlcompiler_threads = MIN(int([metal_device maximumConcurrentCompilationTaskCount]),
max_mtlcompiler_threads);
}
#endif
/* GPU settings for context creation. */
GHOST_GPUSettings gpuSettings = {0};
gpuSettings.context_type = GHOST_kDrawingContextTypeMetal;
if (G.debug & G_DEBUG_GPU) {
gpuSettings.flags |= GHOST_gpuDebugContext;
}
gpuSettings.preferred_device.index = U.gpu_preferred_index;
gpuSettings.preferred_device.vendor_id = U.gpu_preferred_vendor_id;
gpuSettings.preferred_device.device_id = U.gpu_preferred_device_id;
/* Spawn the compiler threads. */
for (int i = 0; i < max_mtlcompiler_threads; i++) {
/* Grab the system handle. */
GHOST_SystemHandle ghost_system = reinterpret_cast<GHOST_SystemHandle>(
GPU_backend_ghost_system_get());
BLI_assert(ghost_system);
/* Create a Ghost GPU Context using the system handle. */
GHOST_ContextHandle ghost_gpu_context = GHOST_CreateGPUContext(ghost_system, gpuSettings);
/* Create a GPU context for the compile thread to use. */
GPUContext *per_thread_context = GPU_context_create(nullptr, ghost_gpu_context);
/* Restore the main thread context.
* (required as the above context creation also makes it active). */
GPU_context_active_set(main_thread_context);
/* Create a new thread */
compile_threads.push_back(std::thread([this, per_thread_context, ghost_gpu_context] {
this->parallel_compilation_thread_func(per_thread_context, ghost_gpu_context);
}));
}
}
void MTLParallelShaderCompiler::parallel_compilation_thread_func(
GPUContext *blender_gpu_context, GHOST_ContextHandle ghost_gpu_context)
{
/* Contexts can only be created on the main thread so we have to
* pass one in and make it active here */
GPU_context_active_set(blender_gpu_context);
MTLContext *metal_context = static_cast<MTLContext *>(unwrap(blender_gpu_context));
MTLShaderCompiler *shader_compiler = static_cast<MTLShaderCompiler *>(
metal_context->get_compiler());
/* This context is only for compilation, it does not need it's own instance of the compiler */
shader_compiler->release_parallel_shader_compiler();
/* Loop until we get the terminate signal */
while (!terminate_compile_threads) {
/* Grab the next shader off of the queue or wait... */
ParallelWork *work_item = nullptr;
{
std::unique_lock<std::mutex> lock(queue_mutex);
cond_var.wait(lock,
[&] { return terminate_compile_threads || !parallel_work_queue.empty(); });
if (terminate_compile_threads || parallel_work_queue.empty()) {
continue;
}
work_item = parallel_work_queue.front();
parallel_work_queue.pop_front();
}
/* Compile a shader */
if (work_item->work_type == PARALLELWORKTYPE_COMPILE_SHADER) {
BLI_assert(work_item->info);
const shader::ShaderCreateInfo *shader_info = work_item->info;
work_item->shader = static_cast<MTLShader *>(
work_item->shader_compiler->compile(*shader_info, true));
if (work_item->shader) {
/* Generate and cache any render PSOs if possible (typically materials only)
* (Finalize() will already bake a Compute PSO if possible) */
work_item->shader->warm_cache(-1);
}
}
/* Bake PSO */
else if (work_item->work_type == PARALLELWORKTYPE_BAKE_PSO) {
MTLShader *shader = work_item->shader;
/* Currently only support Compute */
BLI_assert(shader && shader->has_compute_shader_lib());
/* Create descriptor using these specialization constants. */
MTLComputePipelineStateDescriptor compute_pipeline_descriptor(
work_item->specialization_values);
shader->bake_compute_pipeline_state(metal_context, compute_pipeline_descriptor);
}
else {
BLI_assert(false);
}
work_item->is_ready = true;
}
GPU_context_discard(blender_gpu_context);
GHOST_SystemHandle ghost_system = reinterpret_cast<GHOST_SystemHandle>(
GPU_backend_ghost_system_get());
BLI_assert(ghost_system);
GHOST_DisposeGPUContext(ghost_system, ghost_gpu_context);
}
BatchHandle MTLParallelShaderCompiler::create_batch(size_t batch_size)
{
std::scoped_lock lock(batch_mutex);
BatchHandle batch_handle = next_batch_handle++;
batches.add(batch_handle, {});
Batch &batch = batches.lookup(batch_handle);
if (batch_size) {
batch.items.reserve(batch_size);
}
batch.is_ready = false;
shader_debug_printf("Created batch %llu\n", batch_handle);
return batch_handle;
}
void MTLParallelShaderCompiler::add_item_to_batch(ParallelWork *work_item,
BatchHandle batch_handle)
{
std::scoped_lock lock(batch_mutex);
Batch &batch = batches.lookup(batch_handle);
batch.items.append(work_item);
}
void MTLParallelShaderCompiler::add_parallel_item_to_queue(ParallelWork *work_item,
BatchHandle batch_handle)
{
shader_debug_printf("Request add shader work\n");
if (!terminate_compile_threads) {
/* Defer creation of compilation threads until required */
if (compile_threads.empty()) {
create_compile_threads();
}
add_item_to_batch(work_item, batch_handle);
std::lock_guard<std::mutex> lock(queue_mutex);
parallel_work_queue.push_back(work_item);
cond_var.notify_one();
}
}
BatchHandle MTLParallelShaderCompiler::batch_compile(MTLShaderCompiler *shader_compiler,
Span<const shader::ShaderCreateInfo *> &infos)
{
BLI_assert(GPU_use_parallel_compilation());
BatchHandle batch_handle = create_batch(infos.size());
shader_debug_printf("Batch compile %llu shaders (Batch = %llu)\n", infos.size(), batch_handle);
/* Have to finalize all shaderInfos *before* any parallel compilation as
* ShaderCreateInfo::finalize() is not thread safe */
for (const shader::ShaderCreateInfo *info : infos) {
const_cast<ShaderCreateInfo *>(info)->finalize();
}
for (const shader::ShaderCreateInfo *info : infos) {
ParallelWork *work_item = new ParallelWork;
work_item->info = info;
work_item->shader_compiler = shader_compiler;
work_item->is_ready = false;
work_item->shader = nullptr;
work_item->work_type = PARALLELWORKTYPE_COMPILE_SHADER;
add_parallel_item_to_queue(work_item, batch_handle);
}
return batch_handle;
}
bool MTLParallelShaderCompiler::batch_is_ready(BatchHandle handle)
{
std::scoped_lock lock(batch_mutex);
Batch &batch = batches.lookup(handle);
if (batch.is_ready) {
return true;
}
for (ParallelWork *item : batch.items) {
if (item->is_ready) {
continue;
}
else {
return false;
}
}
batch.is_ready = true;
shader_debug_printf("Batch %llu is now ready\n", handle);
return batch.is_ready;
}
Vector<Shader *> MTLParallelShaderCompiler::batch_finalize(BatchHandle &handle)
{
while (!batch_is_ready(handle)) {
BLI_time_sleep_ms(1);
}
std::scoped_lock lock(batch_mutex);
Batch batch = batches.pop(handle);
Vector<Shader *> result;
for (ParallelWork *item : batch.items) {
result.append(item->shader);
delete item;
}
handle = 0;
return result;
}
SpecializationBatchHandle MTLParallelShaderCompiler::precompile_specializations(
Span<ShaderSpecialization> specializations)
{
BLI_assert(GPU_use_parallel_compilation());
/* Zero indicates no batch was created */
SpecializationBatchHandle batch_handle = 0;
for (auto &specialization : specializations) {
MTLShader *sh = static_cast<MTLShader *>(unwrap(specialization.shader));
/* Specialization constants only take effect when we create the PSO.
* We don't have the relevant info to create a Render PSO Descriptor unless
* the shader has a has_parent_shader() but in that case it would (currently) be
* invalid to apply specialization constants. For those reasons we currently only
* support pre-compilation of Compute shaders.
* (technically we could call makeFunction but the benefit would likely be minimal) */
if (!sh->has_compute_shader_lib()) {
continue;
}
BLI_assert_msg(sh->is_valid(), "Shader must be finalized before precompiling specializations");
/* Defer batch creation until we have some work to do */
if (!batch_handle) {
batch_handle = create_batch(1);
}
ParallelWork *work_item = new ParallelWork;
work_item->info = nullptr;
work_item->is_ready = false;
work_item->shader = sh;
work_item->work_type = PARALLELWORKTYPE_BAKE_PSO;
/* Add the specialization constants to the work-item */
for (const SpecializationConstant &constant : specialization.constants) {
const ShaderInput *input = sh->interface->constant_get(constant.name.c_str());
BLI_assert_msg(input != nullptr, "The specialization constant doesn't exists");
work_item->specialization_values[input->location].u = constant.value.u;
}
sh->constants.is_dirty = true;
add_parallel_item_to_queue(work_item, batch_handle);
}
return batch_handle;
}
bool MTLParallelShaderCompiler::specialization_batch_is_ready(SpecializationBatchHandle &handle)
{
/* Check empty batch case where we have no handle */
if (!handle) {
return true;
}
std::scoped_lock lock(batch_mutex);
Batch &batch = batches.lookup(handle);
if (batch.is_ready) {
return true;
}
for (ParallelWork *item : batch.items) {
if (item->is_ready) {
continue;
}
else {
return false;
}
}
/* Handle is zeroed once the batch is ready */
handle = 0;
batch.is_ready = true;
shader_debug_printf("Specialization Batch %llu is now ready\n", handle);
return batch.is_ready;
}
/** \} */
/* -------------------------------------------------------------------- */
/** \name MTLShaderCompiler
* \{ */
MTLShaderCompiler::MTLShaderCompiler()
: ShaderCompiler(GPU_max_parallel_compilations(), GPUWorker::ContextType::PerThread, true)
{
parallel_shader_compiler = get_shared_parallel_shader_compiler();
BLI_assert(GPU_use_parallel_compilation());
}
MTLShaderCompiler::~MTLShaderCompiler()
Shader *MTLShaderCompiler::compile_shader(const shader::ShaderCreateInfo &info)
{
release_parallel_shader_compiler();
}
MTLShader *shader = static_cast<MTLShader *>(compile(info, true));
void MTLShaderCompiler::release_parallel_shader_compiler()
{
if (parallel_shader_compiler) {
release_shared_parallel_shader_compiler();
parallel_shader_compiler = nullptr;
if (shader) {
/* Generate and cache any render PSOs if possible (typically materials only)
* (Finalize() will already bake a Compute PSO if possible) */
shader->warm_cache(-1);
}
return shader;
}
BatchHandle MTLShaderCompiler::batch_compile(Span<const shader::ShaderCreateInfo *> &infos)
void MTLShaderCompiler::specialize_shader(ShaderSpecialization &specialization)
{
BLI_assert(parallel_shader_compiler);
return parallel_shader_compiler->batch_compile(this, infos);
}
bool MTLShaderCompiler::batch_is_ready(BatchHandle handle)
{
return parallel_shader_compiler->batch_is_ready(handle);
}
Vector<Shader *> MTLShaderCompiler::batch_finalize(BatchHandle &handle)
{
return parallel_shader_compiler->batch_finalize(handle);
}
SpecializationBatchHandle MTLShaderCompiler::precompile_specializations(
Span<ShaderSpecialization> specializations)
{
return parallel_shader_compiler->precompile_specializations(specializations);
}
MTLShader *shader = static_cast<MTLShader *>(unwrap(specialization.shader));
bool MTLShaderCompiler::specialization_batch_is_ready(SpecializationBatchHandle &handle)
{
return parallel_shader_compiler->specialization_batch_is_ready(handle);
BLI_assert_msg(shader->is_valid(),
"Shader must be finalized before precompiling specializations");
if (!shader->has_compute_shader_lib()) {
/* Currently only support Compute */
return;
}
Vector<Shader::Constants::Value> specialization_values(shader->interface->constant_len_);
for (const SpecializationConstant &constant : specialization.constants) {
const ShaderInput *input = shader->interface->constant_get(constant.name.c_str());
BLI_assert_msg(input != nullptr, "The specialization constant doesn't exists");
specialization_values[input->location].u = constant.value.u;
}
shader->constants.is_dirty = true;
/* Create descriptor using these specialization constants. */
MTLComputePipelineStateDescriptor compute_pipeline_descriptor(specialization_values);
MTLContext *metal_context = static_cast<MTLContext *>(Context::get());
shader->bake_compute_pipeline_state(metal_context, compute_pipeline_descriptor);
}
/** \} */

View File

@@ -40,8 +40,6 @@ class GLBackend : public GPUBackend {
renderdoc::api::Renderdoc renderdoc_;
#endif
ShaderCompiler *compiler_;
public:
GLBackend()
{
@@ -59,10 +57,10 @@ class GLBackend : public GPUBackend {
void init_resources() override
{
if (GPU_use_parallel_compilation()) {
compiler_ = new GLShaderCompiler();
compiler_ = MEM_new<GLShaderCompiler>(__func__);
}
else {
compiler_ = new ShaderCompilerGeneric();
compiler_ = MEM_new<ShaderCompiler>(__func__);
}
};
@@ -70,7 +68,7 @@ class GLBackend : public GPUBackend {
{
/* Delete any resources with context active. */
GLTexture::samplers_free();
delete compiler_;
MEM_delete(compiler_);
}
static GLBackend *get()
@@ -78,11 +76,6 @@ class GLBackend : public GPUBackend {
return static_cast<GLBackend *>(GPUBackend::get());
}
ShaderCompiler *get_compiler()
{
return compiler_;
}
void samplers_update() override
{
GLTexture::samplers_update();

View File

@@ -181,17 +181,6 @@ void GLContext::finish()
/** \} */
/* -------------------------------------------------------------------- */
/** \name ShaderCompiler
* \{ */
ShaderCompiler *GLContext::get_compiler()
{
return GLBackend::get()->get_compiler();
}
/** \} */
/* -------------------------------------------------------------------- */
/** \name Safe object deletion
*

View File

@@ -123,8 +123,6 @@ class GLContext : public Context {
void flush() override;
void finish() override;
ShaderCompiler *get_compiler() override;
void memory_statistics_get(int *r_total_mem, int *r_free_mem) override;
static GLContext *get()

View File

@@ -1687,18 +1687,15 @@ void GLCompilerWorker::compile(const GLSourcesBaked &sources)
compilation_start = BLI_time_now_seconds();
}
bool GLCompilerWorker::is_ready()
void GLCompilerWorker::block_until_ready()
{
BLI_assert(ELEM(state_, COMPILATION_REQUESTED, COMPILATION_READY));
if (state_ == COMPILATION_READY) {
return true;
return;
}
if (end_semaphore_->try_decrement()) {
state_ = COMPILATION_READY;
}
return state_ == COMPILATION_READY;
end_semaphore_->decrement();
state_ = COMPILATION_READY;
}
bool GLCompilerWorker::is_lost()
@@ -1706,7 +1703,8 @@ bool GLCompilerWorker::is_lost()
/* Use a timeout for hanged processes. */
float max_timeout_seconds = 30.0f;
return !subprocess_.is_running() ||
(BLI_time_now_seconds() - compilation_start) > max_timeout_seconds;
(state_ == COMPILATION_REQUESTED &&
(BLI_time_now_seconds() - compilation_start) > max_timeout_seconds);
}
bool GLCompilerWorker::load_program_binary(GLint program)
@@ -1742,8 +1740,6 @@ void GLCompilerWorker::release()
GLShaderCompiler::~GLShaderCompiler()
{
BLI_assert(batches.is_empty());
for (GLCompilerWorker *worker : workers_) {
delete worker;
}
@@ -1751,24 +1747,43 @@ GLShaderCompiler::~GLShaderCompiler()
GLCompilerWorker *GLShaderCompiler::get_compiler_worker(const GLSourcesBaked &sources)
{
auto try_get_compiler_worker = [&]() {
GLCompilerWorker *result = nullptr;
for (GLCompilerWorker *compiler : workers_) {
if (compiler->state_ == GLCompilerWorker::AVAILABLE) {
result = compiler;
break;
}
}
if (result) {
check_worker_is_lost(result);
}
if (!result && workers_.size() < GCaps.max_parallel_compilations) {
result = new GLCompilerWorker();
workers_.append(result);
}
return result;
};
std::lock_guard lock(workers_mutex_);
GLCompilerWorker *result = nullptr;
for (GLCompilerWorker *compiler : workers_) {
if (compiler->state_ == GLCompilerWorker::AVAILABLE) {
result = compiler;
while (true) {
if (result = try_get_compiler_worker()) {
BLI_time_sleep_ms(1);
break;
}
}
if (!result && workers_.size() < GCaps.max_parallel_compilations) {
result = new GLCompilerWorker();
workers_.append(result);
}
if (result) {
result->compile(sources);
}
result->compile(sources);
return result;
}
bool GLShaderCompiler::worker_is_lost(GLCompilerWorker *&worker)
bool GLShaderCompiler::check_worker_is_lost(GLCompilerWorker *&worker)
{
if (worker->is_lost()) {
std::cerr << "ERROR: Compilation subprocess lost\n";
@@ -1780,242 +1795,100 @@ bool GLShaderCompiler::worker_is_lost(GLCompilerWorker *&worker)
return worker == nullptr;
}
BatchHandle GLShaderCompiler::batch_compile(Span<const shader::ShaderCreateInfo *> &infos)
Shader *GLShaderCompiler::compile_shader(const shader::ShaderCreateInfo &info)
{
BLI_assert(GPU_use_parallel_compilation());
const_cast<ShaderCreateInfo *>(&info)->finalize();
GLShader *shader = static_cast<GLShader *>(compile(info, true));
GLSourcesBaked sources = shader->get_sources();
std::scoped_lock lock(mutex_);
BatchHandle handle = next_batch_handle++;
batches.add(handle, {});
Batch &batch = batches.lookup(handle);
batch.items.reserve(infos.size());
batch.is_ready = false;
for (const shader::ShaderCreateInfo *info : infos) {
const_cast<ShaderCreateInfo *>(info)->finalize();
batch.items.append({});
CompilationWork &item = batch.items.last();
item.info = info;
item.shader = static_cast<GLShader *>(compile(*info, true));
item.sources = item.shader->get_sources();
size_t required_size = item.sources.size();
item.do_async_compilation = required_size <= sizeof(ShaderSourceHeader::sources);
if (item.do_async_compilation) {
item.worker = get_compiler_worker(item.sources);
}
else {
delete item.shader;
item.sources = {};
}
size_t required_size = sources.size();
bool do_async_compilation = required_size <= sizeof(ShaderSourceHeader::sources);
if (!do_async_compilation) {
/* TODO: Can't reuse? */
delete shader;
return compile(info, false);
}
return handle;
GLCompilerWorker *worker = get_compiler_worker(sources);
if (!worker->load_program_binary(shader->program_active_->program_id) ||
!shader->post_finalize(&info))
{
/* Compilation failed, try to compile it locally. */
delete shader;
shader = nullptr;
}
worker->release();
if (!shader) {
return compile(info, false);
}
return shader;
}
bool GLShaderCompiler::batch_is_ready(BatchHandle handle)
void GLShaderCompiler::specialize_shader(ShaderSpecialization &specialization)
{
std::scoped_lock lock(mutex_);
static std::mutex mutex;
BLI_assert(batches.contains(handle));
Batch &batch = batches.lookup(handle);
if (batch.is_ready) {
return true;
}
GLShader *shader = static_cast<GLShader *>(unwrap(specialization.shader));
Vector<SpecializationConstant> &constants = specialization.constants;
batch.is_ready = true;
for (CompilationWork &item : batch.items) {
if (item.is_ready) {
continue;
auto program_get = [&]() -> GLShader::GLProgram * {
for (const SpecializationConstant &constant : constants) {
const ShaderInput *input = shader->interface->constant_get(constant.name.c_str());
BLI_assert_msg(input != nullptr, "The specialization constant doesn't exists");
shader->constants.values[input->location].u = constant.value.u;
}
if (!item.do_async_compilation) {
/* Compile it locally. */
item.shader = static_cast<GLShader *>(compile(*item.info, false));
item.is_ready = true;
continue;
shader->constants.is_dirty = true;
if (shader->program_cache_.contains(shader->constants.values)) {
return &shader->program_cache_.lookup(shader->constants.values);
}
return nullptr;
};
if (!item.worker) {
/* Try to acquire an available worker. */
item.worker = get_compiler_worker(item.sources);
}
else if (item.worker->is_ready()) {
/* Retrieve the binary compiled by the worker. */
if (!item.worker->load_program_binary(item.shader->program_active_->program_id) ||
!item.shader->post_finalize(item.info))
{
/* Compilation failed, try to compile it locally. */
delete item.shader;
item.shader = nullptr;
item.do_async_compilation = false;
}
else {
item.is_ready = true;
}
item.worker->release();
item.worker = nullptr;
}
else if (worker_is_lost(item.worker)) {
/* We lost the worker, try to compile it locally. */
delete item.shader;
item.shader = nullptr;
item.do_async_compilation = false;
}
auto program_release = [&]() {
/* Compilation failed, local compilation will be tried later on shader bind. */
GLShader::GLProgram *program = program_get();
glDeleteProgram(program->program_id);
program->program_id = 0;
shader->constants.is_dirty = true;
};
if (!item.is_ready) {
batch.is_ready = false;
}
}
GLSourcesBaked sources;
{
std::lock_guard lock(mutex);
return batch.is_ready;
}
Vector<Shader *> GLShaderCompiler::batch_finalize(BatchHandle &handle)
{
while (!batch_is_ready(handle)) {
BLI_time_sleep_ms(1);
}
std::scoped_lock lock(mutex_);
BLI_assert(batches.contains(handle));
Batch batch = batches.pop(handle);
Vector<Shader *> result;
for (CompilationWork &item : batch.items) {
result.append(item.shader);
}
handle = 0;
return result;
}
SpecializationBatchHandle GLShaderCompiler::precompile_specializations(
Span<ShaderSpecialization> specializations)
{
BLI_assert(GPU_use_parallel_compilation());
std::scoped_lock lock(mutex_);
SpecializationBatchHandle handle = next_batch_handle++;
specialization_queue.append({handle, specializations});
return handle;
}
GLShader::GLProgram *GLShaderCompiler::SpecializationWork::program_get()
{
for (const SpecializationConstant &constant : constants) {
const ShaderInput *input = shader->interface->constant_get(constant.name.c_str());
BLI_assert_msg(input != nullptr, "The specialization constant doesn't exists");
shader->constants.values[input->location].u = constant.value.u;
}
shader->constants.is_dirty = true;
if (shader->program_cache_.contains(shader->constants.values)) {
return &shader->program_cache_.lookup(shader->constants.values);
}
return nullptr;
}
void GLShaderCompiler::prepare_next_specialization_batch()
{
BLI_assert(current_specialization_batch.is_ready && !specialization_queue.is_empty());
SpecializationRequest &next = specialization_queue.first();
SpecializationBatch &batch = current_specialization_batch;
batch.handle = next.handle;
batch.is_ready = false;
Vector<SpecializationWork> &items = batch.items;
items.clear();
items.reserve(next.specializations.size());
for (auto &specialization : next.specializations) {
GLShader *sh = static_cast<GLShader *>(unwrap(specialization.shader));
items.append({});
SpecializationWork &item = items.last();
item.shader = sh;
item.constants = specialization.constants;
if (item.program_get()) {
/* Already compiled. */
items.pop_last();
continue;
if (program_get()) {
/*Already compiled*/
return;
}
/** WORKAROUND: Set async_compilation to true, so only the sources are generated. */
sh->async_compilation_ = true;
sh->program_get();
sh->async_compilation_ = false;
shader->async_compilation_ = true;
shader->program_get();
shader->async_compilation_ = false;
sources = shader->get_sources();
item.sources = sh->get_sources();
size_t required_size = item.sources.size();
item.do_async_compilation = required_size <= sizeof(ShaderSourceHeader::sources);
}
specialization_queue.remove(0);
}
bool GLShaderCompiler::specialization_batch_is_ready(SpecializationBatchHandle &handle)
{
std::scoped_lock lock(mutex_);
SpecializationBatch &batch = current_specialization_batch;
if (handle < batch.handle || (handle == batch.handle && batch.is_ready)) {
handle = 0;
return true;
}
if (batch.is_ready) {
prepare_next_specialization_batch();
}
bool is_ready = true;
for (SpecializationWork &item : batch.items) {
if (item.is_ready) {
continue;
}
if (!item.do_async_compilation) {
GLShader::GLProgram *program = item.program_get();
glDeleteProgram(program->program_id);
program->program_id = 0;
item.shader->constants.is_dirty = true;
item.is_ready = true;
continue;
}
if (item.worker == nullptr) {
/* Try to acquire an available worker. */
item.worker = get_compiler_worker(item.sources);
}
else if (item.worker->is_ready()) {
/* Retrieve the binary compiled by the worker. */
if (item.worker->load_program_binary(item.program_get()->program_id)) {
item.is_ready = true;
}
else {
/* Compilation failed, local compilation will be tried later on shader bind. */
item.do_async_compilation = false;
}
item.worker->release();
item.worker = nullptr;
}
else if (worker_is_lost(item.worker)) {
/* We lost the worker, local compilation will be tried later on shader bind. */
item.do_async_compilation = false;
}
if (!item.is_ready) {
is_ready = false;
size_t required_size = sources.size();
bool do_async_compilation = required_size <= sizeof(ShaderSourceHeader::sources);
if (!do_async_compilation) {
program_release();
return;
}
}
if (is_ready) {
batch.is_ready = true;
handle = 0;
GLCompilerWorker *worker = get_compiler_worker(sources);
worker->block_until_ready();
std::lock_guard lock(mutex);
if (!worker->load_program_binary(program_get()->program_id)) {
program_release();
}
return is_ready;
worker->release();
}
/** \} */

View File

@@ -16,6 +16,7 @@
#include "BLI_subprocess.hh"
#include "BLI_utility_mixins.hh"
#include "GPU_capabilities.hh"
#include "gpu_shader_create_info.hh"
#include "gpu_shader_private.hh"
@@ -233,14 +234,14 @@ class GLCompilerWorker {
/* The worker is not currently in use and can be acquired. */
AVAILABLE
};
eState state_ = AVAILABLE;
std::atomic<eState> state_ = AVAILABLE;
double compilation_start = 0;
GLCompilerWorker();
~GLCompilerWorker();
void compile(const GLSourcesBaked &sources);
bool is_ready();
void block_until_ready();
bool load_program_binary(GLint program);
void release();
@@ -250,77 +251,26 @@ class GLCompilerWorker {
class GLShaderCompiler : public ShaderCompiler {
private:
std::mutex mutex_;
Vector<GLCompilerWorker *> workers_;
struct CompilationWork {
const shader::ShaderCreateInfo *info = nullptr;
GLShader *shader = nullptr;
GLSourcesBaked sources;
GLCompilerWorker *worker = nullptr;
bool do_async_compilation = false;
bool is_ready = false;
};
struct Batch {
Vector<CompilationWork> items;
bool is_ready = false;
};
Map<BatchHandle, Batch> batches;
struct SpecializationRequest {
BatchHandle handle;
Vector<ShaderSpecialization> specializations;
};
Vector<SpecializationRequest> specialization_queue;
struct SpecializationWork {
GLShader *shader = nullptr;
Vector<shader::SpecializationConstant> constants;
GLSourcesBaked sources;
GLShader::GLProgram *program_get();
GLCompilerWorker *worker = nullptr;
bool do_async_compilation = false;
bool is_ready = false;
};
struct SpecializationBatch {
SpecializationBatchHandle handle = 0;
Vector<SpecializationWork> items;
bool is_ready = true;
};
SpecializationBatch current_specialization_batch;
void prepare_next_specialization_batch();
/* Shared across regular and specialization batches,
* to prevent the use of a wrong handle type. */
int64_t next_batch_handle = 1;
std::mutex workers_mutex_;
GLCompilerWorker *get_compiler_worker(const GLSourcesBaked &sources);
bool worker_is_lost(GLCompilerWorker *&worker);
bool check_worker_is_lost(GLCompilerWorker *&worker);
GLShader::GLProgram *specialization_program_get(ShaderSpecialization &specialization);
public:
GLShaderCompiler()
: ShaderCompiler(GPU_max_parallel_compilations(), GPUWorker::ContextType::PerThread, true){};
virtual ~GLShaderCompiler() override;
virtual BatchHandle batch_compile(Span<const shader::ShaderCreateInfo *> &infos) override;
virtual bool batch_is_ready(BatchHandle handle) override;
virtual Vector<Shader *> batch_finalize(BatchHandle &handle) override;
virtual SpecializationBatchHandle precompile_specializations(
Span<ShaderSpecialization> specializations) override;
virtual bool specialization_batch_is_ready(SpecializationBatchHandle &handle) override;
virtual Shader *compile_shader(const shader::ShaderCreateInfo &info) override;
virtual void specialize_shader(ShaderSpecialization &specialization) override;
};
#else
class GLShaderCompiler : public ShaderCompilerGeneric {};
class GLShaderCompiler : public ShaderCompiler {};
#endif

View File

@@ -14,6 +14,7 @@
#include "CLG_log.h"
#include "GPU_capabilities.hh"
#include "gpu_capabilities_private.hh"
#include "gpu_platform_private.hh"
@@ -447,8 +448,16 @@ void VKBackend::platform_exit()
}
}
void VKBackend::init_resources() {}
void VKBackend::delete_resources() {}
void VKBackend::init_resources()
{
compiler_ = MEM_new<ShaderCompiler>(
__func__, GPU_max_parallel_compilations(), GPUWorker::ContextType::Main);
}
void VKBackend::delete_resources()
{
MEM_delete(compiler_);
}
void VKBackend::samplers_update()
{

View File

@@ -31,7 +31,6 @@ class VKBackend : public GPUBackend {
#endif
public:
VKShaderCompiler shader_compiler;
/* Global instance to device handles. */
VKDevice device;

View File

@@ -182,11 +182,6 @@ TimelineValue VKContext::flush_render_graph(RenderGraphFlushFlags flags,
void VKContext::finish() {}
ShaderCompiler *VKContext::get_compiler()
{
return &VKBackend::get().shader_compiler;
}
void VKContext::memory_statistics_get(int *r_total_mem_kb, int *r_free_mem_kb)
{
const VKDevice &device = VKBackend::get().device;

View File

@@ -82,8 +82,6 @@ class VKContext : public Context, NonCopyable {
VkFence signal_fence = VK_NULL_HANDLE);
void finish() override;
ShaderCompiler *get_compiler() override;
void memory_statistics_get(int *r_total_mem_kb, int *r_free_mem_kb) override;
void debug_group_begin(const char *, int) override;

View File

@@ -20,12 +20,10 @@
#include "vk_shader_compiler.hh"
namespace blender::gpu {
std::optional<std::string> VKShaderCompiler::cache_dir;
static std::optional<std::string> cache_dir_get()
{
static std::optional<std::string> result;
if (!result.has_value()) {
static std::optional<std::string> result = []() -> std::optional<std::string> {
static char tmp_dir_buffer[FILE_MAX];
/* Shader builder doesn't return the correct appdir. */
if (!BKE_appdir_folder_caches(tmp_dir_buffer, sizeof(tmp_dir_buffer))) {
@@ -34,25 +32,12 @@ static std::optional<std::string> cache_dir_get()
std::string cache_dir = std::string(tmp_dir_buffer) + "vk-spirv-cache" + SEP_STR;
BLI_dir_create_recursive(cache_dir.c_str());
result = cache_dir;
}
return cache_dir;
}();
return result;
}
VKShaderCompiler::VKShaderCompiler()
{
task_pool_ = BLI_task_pool_create(nullptr, TASK_PRIORITY_HIGH);
cache_dir = cache_dir_get();
}
VKShaderCompiler::~VKShaderCompiler()
{
BLI_task_pool_work_and_wait(task_pool_);
BLI_task_pool_free(task_pool_);
task_pool_ = nullptr;
}
/* -------------------------------------------------------------------- */
/** \name SPIR-V disk cache
* \{ */
@@ -68,14 +53,13 @@ static bool read_spirv_from_disk(VKShaderModule &shader_module)
/* RenderDoc uses spirv shaders including debug information. */
return false;
}
if (!VKShaderCompiler::cache_dir.has_value()) {
if (!cache_dir_get().has_value()) {
return false;
}
shader_module.build_sources_hash();
std::string spirv_path = (*VKShaderCompiler::cache_dir) + SEP_STR + shader_module.sources_hash +
".spv";
std::string sidecar_path = (*VKShaderCompiler::cache_dir) + SEP_STR +
shader_module.sources_hash + ".sidecar.bin";
std::string spirv_path = (*cache_dir_get()) + SEP_STR + shader_module.sources_hash + ".spv";
std::string sidecar_path = (*cache_dir_get()) + SEP_STR + shader_module.sources_hash +
".sidecar.bin";
if (!BLI_exists(spirv_path.c_str()) || !BLI_exists(sidecar_path.c_str())) {
return false;
@@ -111,13 +95,12 @@ static void write_spirv_to_disk(VKShaderModule &shader_module)
if (G.debug & G_DEBUG_GPU_RENDERDOC) {
return;
}
if (!VKShaderCompiler::cache_dir.has_value()) {
if (!cache_dir_get().has_value()) {
return;
}
/* Write the spirv binary */
std::string spirv_path = (*VKShaderCompiler::cache_dir) + SEP_STR + shader_module.sources_hash +
".spv";
std::string spirv_path = (*cache_dir_get()) + SEP_STR + shader_module.sources_hash + ".spv";
size_t size = (shader_module.compilation_result.end() -
shader_module.compilation_result.begin()) *
sizeof(uint32_t);
@@ -126,20 +109,20 @@ static void write_spirv_to_disk(VKShaderModule &shader_module)
/* Write the sidecar */
SPIRVSidecar sidecar = {size};
std::string sidecar_path = (*VKShaderCompiler::cache_dir) + SEP_STR +
shader_module.sources_hash + ".sidecar.bin";
std::string sidecar_path = (*cache_dir_get()) + SEP_STR + shader_module.sources_hash +
".sidecar.bin";
fstream sidecar_file(sidecar_path, std::ios::binary | std::ios::out);
sidecar_file.write(reinterpret_cast<const char *>(&sidecar), sizeof(SPIRVSidecar));
}
void VKShaderCompiler::cache_dir_clear_old()
{
if (!cache_dir.has_value()) {
if (!cache_dir_get().has_value()) {
return;
}
direntry *entries = nullptr;
uint32_t dir_len = BLI_filelist_dir_contents(cache_dir->c_str(), &entries);
uint32_t dir_len = BLI_filelist_dir_contents(cache_dir_get()->c_str(), &entries);
for (int i : blender::IndexRange(dir_len)) {
direntry entry = entries[i];
if (S_ISDIR(entry.s.st_mode)) {
@@ -160,22 +143,6 @@ void VKShaderCompiler::cache_dir_clear_old()
/** \name Compilation
* \{ */
BatchHandle VKShaderCompiler::batch_compile(Span<const shader::ShaderCreateInfo *> &infos)
{
std::scoped_lock lock(mutex_);
BatchHandle handle = next_batch_handle_++;
VKBatch &batch = batches_.lookup_or_add_default(handle);
batch.shaders.reserve(infos.size());
for (const shader::ShaderCreateInfo *info : infos) {
Shader *shader = compile(*info, true);
batch.shaders.append(shader);
}
for (Shader *shader : batch.shaders) {
BLI_task_pool_push(task_pool_, run, shader, false, nullptr);
}
return handle;
}
static StringRef to_stage_name(shaderc_shader_kind stage)
{
switch (stage) {
@@ -236,72 +203,6 @@ bool VKShaderCompiler::compile_module(VKShader &shader,
return compile_ex(compiler, shader, stage, shader_module);
}
void VKShaderCompiler::run(TaskPool *__restrict /*pool*/, void *task_data)
{
VKShader &shader = *static_cast<VKShader *>(task_data);
shaderc::Compiler compiler;
bool has_not_succeeded = false;
if (!shader.vertex_module.is_ready) {
bool compilation_succeeded = compile_ex(
compiler, shader, shaderc_vertex_shader, shader.vertex_module);
has_not_succeeded |= !compilation_succeeded;
shader.vertex_module.is_ready = true;
}
if (!shader.geometry_module.is_ready) {
bool compilation_succeeded = compile_ex(
compiler, shader, shaderc_geometry_shader, shader.geometry_module);
has_not_succeeded |= !compilation_succeeded;
shader.geometry_module.is_ready = true;
}
if (!shader.fragment_module.is_ready) {
bool compilation_succeeded = compile_ex(
compiler, shader, shaderc_fragment_shader, shader.fragment_module);
has_not_succeeded |= !compilation_succeeded;
shader.fragment_module.is_ready = true;
}
if (!shader.compute_module.is_ready) {
bool compilation_succeeded = compile_ex(
compiler, shader, shaderc_compute_shader, shader.compute_module);
has_not_succeeded |= !compilation_succeeded;
shader.compute_module.is_ready = true;
}
if (has_not_succeeded) {
shader.compilation_failed = true;
}
shader.finalize_post();
/* Setting compilation finished needs to be the last step. It is used to detect if a compilation
* action of a batch has finished. See `VKShaderCompiler::batch_is_ready` */
shader.compilation_finished = true;
}
bool VKShaderCompiler::batch_is_ready(BatchHandle handle)
{
std::scoped_lock lock(mutex_);
BLI_assert(batches_.contains(handle));
VKBatch &batch = batches_.lookup(handle);
for (Shader *shader_ : batch.shaders) {
VKShader &shader = *unwrap(shader_);
if (!shader.is_ready()) {
return false;
}
}
return true;
}
Vector<Shader *> VKShaderCompiler::batch_finalize(BatchHandle &handle)
{
while (!batch_is_ready(handle)) {
BLI_time_sleep_ms(1);
}
std::scoped_lock lock(mutex_);
BLI_assert(batches_.contains(handle));
VKBatch batch = batches_.pop(handle);
handle = 0;
return batch.shaders;
}
/** \} */
} // namespace blender::gpu

View File

@@ -28,40 +28,12 @@ class VKShaderModule;
* Is used for both single threaded compilation by calling `VKShaderCompiler::compile_module` or
* batch based compilation.
*/
class VKShaderCompiler : public ShaderCompiler {
private:
std::mutex mutex_;
BatchHandle next_batch_handle_ = 1;
struct VKBatch {
Vector<Shader *> shaders;
};
Map<BatchHandle, VKBatch> batches_;
TaskPool *task_pool_ = nullptr;
class VKShaderCompiler {
public:
/**
* Cached path to the cache folder.
*
* GHOST and BKE_appdir are not thread safe. Storing the cache_dir locally to work around
* threading issues.
*/
static std::optional<std::string> cache_dir;
VKShaderCompiler();
virtual ~VKShaderCompiler();
BatchHandle batch_compile(Span<const shader::ShaderCreateInfo *> &infos) override;
bool batch_is_ready(BatchHandle handle) override;
Vector<Shader *> batch_finalize(BatchHandle &handle) override;
static bool compile_module(VKShader &shader,
shaderc_shader_kind stage,
VKShaderModule &shader_module);
static void cache_dir_clear_old();
private:
static void run(TaskPool *__restrict pool, void *task_data);
};
} // namespace blender::gpu