mesa/glthread: switch to u_queue and redesign the batch management
This mirrors exactly how u_threaded_context works. If you understand this, you also understand u_threaded_context. Reviewed-by: Timothy Arceri <tarceri@itsqueeze.com>
This commit is contained in:
@@ -38,22 +38,12 @@
|
||||
#include "main/marshal_generated.h"
|
||||
#include "util/u_thread.h"
|
||||
|
||||
static void
|
||||
glthread_allocate_batch(struct gl_context *ctx)
|
||||
{
|
||||
struct glthread_state *glthread = ctx->GLThread;
|
||||
|
||||
/* TODO: handle memory allocation failure. */
|
||||
glthread->batch = malloc(sizeof(*glthread->batch));
|
||||
if (!glthread->batch)
|
||||
return;
|
||||
memset(glthread->batch, 0, offsetof(struct glthread_batch, buffer));
|
||||
}
|
||||
|
||||
static void
|
||||
glthread_unmarshal_batch(struct gl_context *ctx, struct glthread_batch *batch,
|
||||
const bool release_batch)
|
||||
glthread_unmarshal_batch(void *job, int thread_index)
|
||||
{
|
||||
struct glthread_batch *batch = (struct glthread_batch*)job;
|
||||
struct gl_context *ctx = batch->ctx;
|
||||
size_t pos = 0;
|
||||
|
||||
_glapi_set_dispatch(ctx->CurrentServerDispatch);
|
||||
@@ -62,57 +52,16 @@ glthread_unmarshal_batch(struct gl_context *ctx, struct glthread_batch *batch,
|
||||
pos += _mesa_unmarshal_dispatch_cmd(ctx, &batch->buffer[pos]);
|
||||
|
||||
assert(pos == batch->used);
|
||||
|
||||
if (release_batch)
|
||||
free(batch);
|
||||
else
|
||||
batch->used = 0;
|
||||
batch->used = 0;
|
||||
}
|
||||
|
||||
static void *
|
||||
glthread_worker(void *data)
|
||||
static void
|
||||
glthread_thread_initialization(void *job, int thread_index)
|
||||
{
|
||||
struct gl_context *ctx = data;
|
||||
struct glthread_state *glthread = ctx->GLThread;
|
||||
struct gl_context *ctx = (struct gl_context*)job;
|
||||
|
||||
ctx->Driver.SetBackgroundContext(ctx);
|
||||
_glapi_set_context(ctx);
|
||||
|
||||
u_thread_setname("mesa_glthread");
|
||||
|
||||
pthread_mutex_lock(&glthread->mutex);
|
||||
|
||||
while (true) {
|
||||
struct glthread_batch *batch;
|
||||
|
||||
/* Block (dropping the lock) until new work arrives for us. */
|
||||
while (!glthread->batch_queue && !glthread->shutdown) {
|
||||
pthread_cond_broadcast(&glthread->work_done);
|
||||
pthread_cond_wait(&glthread->new_work, &glthread->mutex);
|
||||
}
|
||||
|
||||
batch = glthread->batch_queue;
|
||||
|
||||
if (glthread->shutdown && !batch) {
|
||||
pthread_cond_broadcast(&glthread->work_done);
|
||||
pthread_mutex_unlock(&glthread->mutex);
|
||||
return NULL;
|
||||
}
|
||||
glthread->batch_queue = batch->next;
|
||||
if (glthread->batch_queue_tail == &batch->next)
|
||||
glthread->batch_queue_tail = &glthread->batch_queue;
|
||||
|
||||
glthread->busy = true;
|
||||
pthread_mutex_unlock(&glthread->mutex);
|
||||
|
||||
glthread_unmarshal_batch(ctx, batch, true);
|
||||
|
||||
pthread_mutex_lock(&glthread->mutex);
|
||||
glthread->busy = false;
|
||||
}
|
||||
|
||||
/* UNREACHED */
|
||||
return NULL;
|
||||
}
|
||||
|
||||
void
|
||||
@@ -123,24 +72,34 @@ _mesa_glthread_init(struct gl_context *ctx)
|
||||
if (!glthread)
|
||||
return;
|
||||
|
||||
ctx->MarshalExec = _mesa_create_marshal_table(ctx);
|
||||
if (!ctx->MarshalExec) {
|
||||
if (!util_queue_init(&glthread->queue, "glthread", MARSHAL_MAX_BATCHES - 2,
|
||||
1, 0)) {
|
||||
free(glthread);
|
||||
return;
|
||||
}
|
||||
|
||||
ctx->MarshalExec = _mesa_create_marshal_table(ctx);
|
||||
if (!ctx->MarshalExec) {
|
||||
util_queue_destroy(&glthread->queue);
|
||||
free(glthread);
|
||||
return;
|
||||
}
|
||||
|
||||
for (unsigned i = 0; i < MARSHAL_MAX_BATCHES; i++) {
|
||||
glthread->batches[i].ctx = ctx;
|
||||
util_queue_fence_init(&glthread->batches[i].fence);
|
||||
}
|
||||
|
||||
ctx->CurrentClientDispatch = ctx->MarshalExec;
|
||||
|
||||
pthread_mutex_init(&glthread->mutex, NULL);
|
||||
pthread_cond_init(&glthread->new_work, NULL);
|
||||
pthread_cond_init(&glthread->work_done, NULL);
|
||||
|
||||
glthread->batch_queue_tail = &glthread->batch_queue;
|
||||
ctx->GLThread = glthread;
|
||||
|
||||
glthread_allocate_batch(ctx);
|
||||
|
||||
pthread_create(&glthread->thread, NULL, glthread_worker, ctx);
|
||||
/* Execute the thread initialization function in the thread. */
|
||||
struct util_queue_fence fence;
|
||||
util_queue_fence_init(&fence);
|
||||
util_queue_add_job(&glthread->queue, ctx, &fence,
|
||||
glthread_thread_initialization, NULL);
|
||||
util_queue_fence_wait(&fence);
|
||||
util_queue_fence_destroy(&fence);
|
||||
}
|
||||
|
||||
void
|
||||
@@ -151,29 +110,11 @@ _mesa_glthread_destroy(struct gl_context *ctx)
|
||||
if (!glthread)
|
||||
return;
|
||||
|
||||
_mesa_glthread_flush_batch(ctx);
|
||||
_mesa_glthread_finish(ctx);
|
||||
util_queue_destroy(&glthread->queue);
|
||||
|
||||
pthread_mutex_lock(&glthread->mutex);
|
||||
glthread->shutdown = true;
|
||||
pthread_cond_broadcast(&glthread->new_work);
|
||||
pthread_mutex_unlock(&glthread->mutex);
|
||||
|
||||
/* Since this waits for the thread to exit, it means that all queued work
|
||||
* will have been completed.
|
||||
*/
|
||||
pthread_join(glthread->thread, NULL);
|
||||
|
||||
pthread_cond_destroy(&glthread->new_work);
|
||||
pthread_cond_destroy(&glthread->work_done);
|
||||
pthread_mutex_destroy(&glthread->mutex);
|
||||
|
||||
/* Due to the join above, there should be one empty batch allocated at this
|
||||
* point, and no batches queued.
|
||||
*/
|
||||
assert(!glthread->batch->used);
|
||||
assert(!glthread->batch->next);
|
||||
free(glthread->batch);
|
||||
assert(!glthread->batch_queue);
|
||||
for (unsigned i = 0; i < MARSHAL_MAX_BATCHES; i++)
|
||||
util_queue_fence_destroy(&glthread->batches[i].fence);
|
||||
|
||||
free(glthread);
|
||||
ctx->GLThread = NULL;
|
||||
@@ -196,19 +137,16 @@ _mesa_glthread_restore_dispatch(struct gl_context *ctx)
|
||||
}
|
||||
}
|
||||
|
||||
static void
|
||||
_mesa_glthread_flush_batch_locked(struct gl_context *ctx)
|
||||
void
|
||||
_mesa_glthread_flush_batch(struct gl_context *ctx)
|
||||
{
|
||||
struct glthread_state *glthread = ctx->GLThread;
|
||||
struct glthread_batch *batch = glthread->batch;
|
||||
|
||||
if (!batch->used)
|
||||
if (!glthread)
|
||||
return;
|
||||
|
||||
/* Immediately reallocate a new batch, since the next marshalled call would
|
||||
* just do it.
|
||||
*/
|
||||
glthread_allocate_batch(ctx);
|
||||
struct glthread_batch *next = &glthread->batches[glthread->next];
|
||||
if (!next->used)
|
||||
return;
|
||||
|
||||
/* Debug: execute the batch immediately from this thread.
|
||||
*
|
||||
@@ -216,32 +154,15 @@ _mesa_glthread_flush_batch_locked(struct gl_context *ctx)
|
||||
* need to restore it when it returns.
|
||||
*/
|
||||
if (false) {
|
||||
glthread_unmarshal_batch(ctx, batch, true);
|
||||
glthread_unmarshal_batch(next, 0);
|
||||
_glapi_set_dispatch(ctx->CurrentClientDispatch);
|
||||
return;
|
||||
}
|
||||
|
||||
*glthread->batch_queue_tail = batch;
|
||||
glthread->batch_queue_tail = &batch->next;
|
||||
pthread_cond_broadcast(&glthread->new_work);
|
||||
}
|
||||
|
||||
void
|
||||
_mesa_glthread_flush_batch(struct gl_context *ctx)
|
||||
{
|
||||
struct glthread_state *glthread = ctx->GLThread;
|
||||
struct glthread_batch *batch;
|
||||
|
||||
if (!glthread)
|
||||
return;
|
||||
|
||||
batch = glthread->batch;
|
||||
if (!batch->used)
|
||||
return;
|
||||
|
||||
pthread_mutex_lock(&glthread->mutex);
|
||||
_mesa_glthread_flush_batch_locked(ctx);
|
||||
pthread_mutex_unlock(&glthread->mutex);
|
||||
util_queue_add_job(&glthread->queue, next, &next->fence,
|
||||
glthread_unmarshal_batch, NULL);
|
||||
glthread->last = glthread->next;
|
||||
glthread->next = (glthread->next + 1) % MARSHAL_MAX_BATCHES;
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -254,7 +175,6 @@ void
|
||||
_mesa_glthread_finish(struct gl_context *ctx)
|
||||
{
|
||||
struct glthread_state *glthread = ctx->GLThread;
|
||||
|
||||
if (!glthread)
|
||||
return;
|
||||
|
||||
@@ -263,22 +183,21 @@ _mesa_glthread_finish(struct gl_context *ctx)
|
||||
* dri interface entrypoints), in which case we don't need to actually
|
||||
* synchronize against ourself.
|
||||
*/
|
||||
if (pthread_equal(pthread_self(), glthread->thread))
|
||||
if (u_thread_is_self(glthread->queue.threads[0]))
|
||||
return;
|
||||
|
||||
pthread_mutex_lock(&glthread->mutex);
|
||||
struct glthread_batch *last = &glthread->batches[glthread->last];
|
||||
struct glthread_batch *next = &glthread->batches[glthread->next];
|
||||
|
||||
if (!(glthread->batch_queue || glthread->busy)) {
|
||||
if (glthread->batch && glthread->batch->used) {
|
||||
struct _glapi_table *dispatch = _glapi_get_dispatch();
|
||||
glthread_unmarshal_batch(ctx, glthread->batch, false);
|
||||
_glapi_set_dispatch(dispatch);
|
||||
}
|
||||
} else {
|
||||
_mesa_glthread_flush_batch_locked(ctx);
|
||||
while (glthread->batch_queue || glthread->busy)
|
||||
pthread_cond_wait(&glthread->work_done, &glthread->mutex);
|
||||
if (!util_queue_fence_is_signalled(&last->fence))
|
||||
util_queue_fence_wait(&last->fence);
|
||||
|
||||
if (next->used) {
|
||||
/* Since glthread_unmarshal_batch changes the dispatch to direct,
|
||||
* restore it after it's done.
|
||||
*/
|
||||
struct _glapi_table *dispatch = _glapi_get_dispatch();
|
||||
glthread_unmarshal_batch(next, 0);
|
||||
_glapi_set_dispatch(dispatch);
|
||||
}
|
||||
|
||||
pthread_mutex_unlock(&glthread->mutex);
|
||||
}
|
||||
|
@@ -29,55 +29,50 @@
|
||||
/* Command size is a number of bytes stored in a short. */
|
||||
#define MARSHAL_MAX_CMD_SIZE 65535
|
||||
|
||||
/* The number of batch slots in memory.
|
||||
*
|
||||
* One batch is being executed, one batch is being filled, the rest are
|
||||
* waiting batches. There must be at least 1 slot for a waiting batch,
|
||||
* so the minimum number of batches is 3.
|
||||
*/
|
||||
#define MARSHAL_MAX_BATCHES 4
|
||||
|
||||
#include <inttypes.h>
|
||||
#include <stdbool.h>
|
||||
#include <pthread.h>
|
||||
#include "util/u_queue.h"
|
||||
|
||||
enum marshal_dispatch_cmd_id;
|
||||
|
||||
/** A single batch of commands queued up for execution. */
|
||||
struct glthread_batch
|
||||
{
|
||||
/** Batch fence for waiting for the execution to finish. */
|
||||
struct util_queue_fence fence;
|
||||
|
||||
/** The worker thread will access the context with this. */
|
||||
struct gl_context *ctx;
|
||||
|
||||
/** Amount of data used by batch commands, in bytes. */
|
||||
size_t used;
|
||||
|
||||
/** Data contained in the command buffer. */
|
||||
uint8_t buffer[MARSHAL_MAX_CMD_SIZE];
|
||||
};
|
||||
|
||||
struct glthread_state
|
||||
{
|
||||
/** The worker thread that asynchronously processes our GL commands. */
|
||||
pthread_t thread;
|
||||
/** Multithreaded queue. */
|
||||
struct util_queue queue;
|
||||
|
||||
/**
|
||||
* Mutex used for synchronizing between the main thread and the worker
|
||||
* thread.
|
||||
*/
|
||||
pthread_mutex_t mutex;
|
||||
/** The ring of batches in memory. */
|
||||
struct glthread_batch batches[MARSHAL_MAX_BATCHES];
|
||||
|
||||
/** Condvar used for waking the worker thread. */
|
||||
pthread_cond_t new_work;
|
||||
/** Index of the last submitted batch. */
|
||||
unsigned last;
|
||||
|
||||
/** Condvar used for waking the main thread. */
|
||||
pthread_cond_t work_done;
|
||||
|
||||
/** Used to tell the worker thread to quit */
|
||||
bool shutdown;
|
||||
|
||||
/** Indicates that the worker thread is currently processing a batch */
|
||||
bool busy;
|
||||
|
||||
/**
|
||||
* Singly-linked list of command batches that are awaiting execution by
|
||||
* a thread pool task. NULL if empty.
|
||||
*/
|
||||
struct glthread_batch *batch_queue;
|
||||
|
||||
/**
|
||||
* Tail pointer for appending batches to the end of batch_queue. If the
|
||||
* queue is empty, this points to batch_queue.
|
||||
*/
|
||||
struct glthread_batch **batch_queue_tail;
|
||||
|
||||
/**
|
||||
* Batch containing commands that are being prepared for insertion into
|
||||
* batch_queue. NULL if there are no such commands.
|
||||
*
|
||||
* Since this is only used by the main thread, it doesn't need the mutex to
|
||||
* be accessed.
|
||||
*/
|
||||
struct glthread_batch *batch;
|
||||
/** Index of the batch being filled and about to be submitted. */
|
||||
unsigned next;
|
||||
|
||||
/**
|
||||
* Tracks on the main thread side whether the current vertex array binding
|
||||
@@ -92,29 +87,6 @@ struct glthread_state
|
||||
bool element_array_is_vbo;
|
||||
};
|
||||
|
||||
/**
|
||||
* A single batch of commands queued up for later execution by a thread pool
|
||||
* task.
|
||||
*/
|
||||
struct glthread_batch
|
||||
{
|
||||
/**
|
||||
* Next batch of commands to execute after this batch, or NULL if this is
|
||||
* the last set of commands queued. Protected by ctx->Marshal.Mutex.
|
||||
*/
|
||||
struct glthread_batch *next;
|
||||
|
||||
/**
|
||||
* Amount of data used by batch commands, in bytes.
|
||||
*/
|
||||
size_t used;
|
||||
|
||||
/**
|
||||
* Data contained in the command buffer.
|
||||
*/
|
||||
uint8_t buffer[MARSHAL_MAX_CMD_SIZE];
|
||||
};
|
||||
|
||||
void _mesa_glthread_init(struct gl_context *ctx);
|
||||
void _mesa_glthread_destroy(struct gl_context *ctx);
|
||||
|
||||
|
@@ -53,15 +53,17 @@ _mesa_glthread_allocate_command(struct gl_context *ctx,
|
||||
size_t size)
|
||||
{
|
||||
struct glthread_state *glthread = ctx->GLThread;
|
||||
struct glthread_batch *next = &glthread->batches[glthread->next];
|
||||
struct marshal_cmd_base *cmd_base;
|
||||
const size_t aligned_size = ALIGN(size, 8);
|
||||
|
||||
if (unlikely(glthread->batch->used + size > MARSHAL_MAX_CMD_SIZE))
|
||||
if (unlikely(next->used + size > MARSHAL_MAX_CMD_SIZE)) {
|
||||
_mesa_glthread_flush_batch(ctx);
|
||||
next = &glthread->batches[glthread->next];
|
||||
}
|
||||
|
||||
cmd_base = (struct marshal_cmd_base *)
|
||||
&glthread->batch->buffer[glthread->batch->used];
|
||||
glthread->batch->used += aligned_size;
|
||||
cmd_base = (struct marshal_cmd_base *)&next->buffer[next->used];
|
||||
next->used += aligned_size;
|
||||
cmd_base->cmd_id = cmd_id;
|
||||
cmd_base->cmd_size = aligned_size;
|
||||
return cmd_base;
|
||||
|
Reference in New Issue
Block a user