gallium/u_threaded: implement asynchronous flushes

This requires out-of-band creation of fences, and will be signaled to
the pipe_context::flush implementation by a special TC_FLUSH_ASYNC flag.

v2:
- remove an incorrect assertion
- handle fence_server_sync for unsubmitted fences by
  relying on the improved cs_add_fence_dependency
- only implement asynchronous flushes on amdgpu

Reviewed-by: Marek Olšák <marek.olsak@amd.com>
diff --git a/src/gallium/auxiliary/util/u_threaded_context.c b/src/gallium/auxiliary/util/u_threaded_context.c
index 510201d..98d12ad 100644
--- a/src/gallium/auxiliary/util/u_threaded_context.c
+++ b/src/gallium/auxiliary/util/u_threaded_context.c
@@ -88,6 +88,8 @@
 
    tc_batch_check(batch);
 
+   assert(!batch->token);
+
    for (struct tc_call *iter = batch->call; iter != last;
         iter += iter->num_call_slots) {
       tc_assert(iter->sentinel == TC_SENTINEL);
@@ -108,6 +110,11 @@
    tc_debug_check(tc);
    p_atomic_add(&tc->num_offloaded_slots, next->num_total_call_slots);
 
+   if (next->token) {
+      next->token->tc = NULL;
+      tc_unflushed_batch_token_reference(&next->token, NULL);
+   }
+
    util_queue_add_job(&tc->queue, next, &next->fence, tc_batch_execute,
                       NULL);
    tc->last = tc->next;
@@ -179,6 +186,11 @@
 
    tc_debug_check(tc);
 
+   if (next->token) {
+      next->token->tc = NULL;
+      tc_unflushed_batch_token_reference(&next->token, NULL);
+   }
+
    /* .. and execute unflushed calls directly. */
    if (next->num_total_call_slots) {
       p_atomic_add(&tc->num_direct_slots, next->num_total_call_slots);
@@ -199,6 +211,24 @@
 #define tc_sync(tc) _tc_sync(tc, "", __func__)
 #define tc_sync_msg(tc, info) _tc_sync(tc, info, __func__)
 
+/**
+ * Call this from fence_finish for same-context fence waits of deferred fences
+ * that haven't been flushed yet.
+ *
+ * The passed pipe_context must be the one passed to pipe_screen::fence_finish,
+ * i.e., the wrapped one.
+ */
+void
+threaded_context_flush(struct pipe_context *_pipe,
+                       struct tc_unflushed_batch_token *token)
+{
+   struct threaded_context *tc = threaded_context(_pipe);
+
+   /* This is called from the state-tracker / application thread. */
+   if (token->tc && token->tc == tc)
+      tc_sync(token->tc);
+}
+
 static void
 tc_set_resource_reference(struct pipe_resource **dst, struct pipe_resource *src)
 {
@@ -1801,14 +1831,73 @@
  * draw, launch, clear, blit, copy, flush
  */
 
+struct tc_flush_payload {
+   struct pipe_fence_handle *fence;
+   unsigned flags;
+};
+
+static void
+tc_call_flush(struct pipe_context *pipe, union tc_payload *payload)
+{
+   struct tc_flush_payload *p = (struct tc_flush_payload *)payload;
+   struct pipe_screen *screen = pipe->screen;
+
+   pipe->flush(pipe, p->fence ? &p->fence : NULL, p->flags);
+   screen->fence_reference(screen, &p->fence, NULL);
+}
+
 static void
 tc_flush(struct pipe_context *_pipe, struct pipe_fence_handle **fence,
          unsigned flags)
 {
    struct threaded_context *tc = threaded_context(_pipe);
    struct pipe_context *pipe = tc->pipe;
+   struct pipe_screen *screen = pipe->screen;
    struct threaded_query *tq, *tmp;
+   bool async = flags & PIPE_FLUSH_DEFERRED;
 
+   if (flags & PIPE_FLUSH_ASYNC) {
+      struct tc_batch *last = &tc->batch_slots[tc->last];
+
+      /* Prefer to do the flush in the driver thread, but avoid the inter-thread
+       * communication overhead if the driver thread is currently idle and the
+       * caller is going to wait for the fence immediately anyway.
+       */
+      if (!(util_queue_fence_is_signalled(&last->fence) &&
+            (flags & PIPE_FLUSH_HINT_FINISH)))
+         async = true;
+   }
+
+   if (async && tc->create_fence) {
+      if (fence) {
+         struct tc_unflushed_batch_token *token = NULL;
+         struct tc_batch *next = &tc->batch_slots[tc->next];
+
+         if (!next->token) {
+            next->token = malloc(sizeof(*next->token));
+            if (!next->token)
+               goto out_of_memory;
+
+            pipe_reference_init(&next->token->ref, 1);
+            next->token->tc = tc;
+         }
+
+         screen->fence_reference(screen, fence, tc->create_fence(pipe, token));
+         if (!*fence)
+            goto out_of_memory;
+      }
+
+      struct tc_flush_payload *p =
+         tc_add_struct_typed_call(tc, TC_CALL_flush, tc_flush_payload);
+      p->fence = fence ? *fence : NULL;
+      p->flags = flags | TC_FLUSH_ASYNC;
+
+      if (!(flags & PIPE_FLUSH_DEFERRED))
+         tc_batch_flush(tc);
+      return;
+   }
+
+out_of_memory:
    if (!(flags & PIPE_FLUSH_DEFERRED)) {
       LIST_FOR_EACH_ENTRY_SAFE(tq, tmp, &tc->unflushed_queries, head_unflushed) {
          tq->flushed = true;
@@ -1816,7 +1905,6 @@
       }
    }
 
-   /* TODO: deferred flushes? */
    tc_sync_msg(tc, flags & PIPE_FLUSH_END_OF_FRAME ? "end of frame" :
                    flags & PIPE_FLUSH_DEFERRED ? "deferred fence" : "normal");
    pipe->flush(pipe, fence, flags);
@@ -2266,8 +2354,10 @@
    if (util_queue_is_initialized(&tc->queue)) {
       util_queue_destroy(&tc->queue);
 
-      for (unsigned i = 0; i < TC_MAX_BATCHES; i++)
+      for (unsigned i = 0; i < TC_MAX_BATCHES; i++) {
          util_queue_fence_destroy(&tc->batch_slots[i].fence);
+         assert(!tc->batch_slots[i].token);
+      }
    }
 
    slab_destroy_child(&tc->pool_transfers);
@@ -2298,6 +2388,7 @@
 threaded_context_create(struct pipe_context *pipe,
                         struct slab_parent_pool *parent_transfer_pool,
                         tc_replace_buffer_storage_func replace_buffer,
+                        tc_create_fence_func create_fence,
                         struct threaded_context **out)
 {
    struct threaded_context *tc;
@@ -2332,6 +2423,7 @@
 
    tc->pipe = pipe;
    tc->replace_buffer_storage = replace_buffer;
+   tc->create_fence = create_fence;
    tc->map_buffer_alignment =
       pipe->screen->get_param(pipe->screen, PIPE_CAP_MIN_MAP_BUFFER_ALIGNMENT);
    tc->base.priv = pipe; /* priv points to the wrapped driver context */
diff --git a/src/gallium/auxiliary/util/u_threaded_context.h b/src/gallium/auxiliary/util/u_threaded_context.h
index a9f79a6..ea815ed 100644
--- a/src/gallium/auxiliary/util/u_threaded_context.h
+++ b/src/gallium/auxiliary/util/u_threaded_context.h
@@ -108,6 +108,29 @@
  *    The threaded context uses its own buffer invalidation mechanism.
  *
  *
+ * Rules for fences
+ * ----------------
+ *
+ * Flushes will be executed asynchronously in the driver thread if a
+ * create_fence callback is provided. This affects fence semantics as follows.
+ *
+ * When the threaded context wants to perform an asynchronous flush, it will
+ * use the create_fence callback to pre-create the fence from the calling
+ * thread. This pre-created fence will be passed to pipe_context::flush
+ * together with the TC_FLUSH_ASYNC flag.
+ *
+ * The callback receives the unwrapped context as a parameter, but must use it
+ * in a thread-safe way because it is called from a non-driver thread.
+ *
+ * If the threaded_context does not immediately flush the current batch, the
+ * callback also receives a tc_unflushed_batch_token. If fence_finish is called
+ * on the returned fence in the context that created the fence,
+ * threaded_context_flush must be called.
+ *
+ * The driver must implement pipe_context::fence_server_sync properly, since
+ * the threaded context handles PIPE_FLUSH_ASYNC.
+ *
+ *
  * Additional requirements
  * -----------------------
  *
@@ -160,10 +183,14 @@
 
 #include "pipe/p_context.h"
 #include "pipe/p_state.h"
+#include "util/u_inlines.h"
 #include "util/u_queue.h"
 #include "util/u_range.h"
 #include "util/slab.h"
 
+struct threaded_context;
+struct tc_unflushed_batch_token;
+
 /* These are transfer flags sent to drivers. */
 /* Never infer whether it's safe to use unsychronized mappings: */
 #define TC_TRANSFER_MAP_NO_INFER_UNSYNCHRONIZED (1u << 29)
@@ -172,6 +199,10 @@
 /* transfer_map is called from a non-driver thread: */
 #define TC_TRANSFER_MAP_THREADED_UNSYNC      (1u << 31)
 
+/* Custom flush flags sent to drivers. */
+/* fence is pre-populated with a fence created by the create_fence callback */
+#define TC_FLUSH_ASYNC        (1u << 31)
+
 /* Size of the queue = number of batch slots in memory.
  * - 1 batch is always idle and records new commands
  * - 1 batch is being executed
@@ -204,6 +235,8 @@
 typedef void (*tc_replace_buffer_storage_func)(struct pipe_context *ctx,
                                                struct pipe_resource *dst,
                                                struct pipe_resource *src);
+typedef struct pipe_fence_handle *(*tc_create_fence_func)(struct pipe_context *ctx,
+                                                          struct tc_unflushed_batch_token *token);
 
 struct threaded_resource {
    struct pipe_resource b;
@@ -293,10 +326,21 @@
    union tc_payload payload;
 };
 
+/**
+ * A token representing an unflushed batch.
+ *
+ * See the general rules for fences for an explanation.
+ */
+struct tc_unflushed_batch_token {
+   struct pipe_reference ref;
+   struct threaded_context *tc;
+};
+
 struct tc_batch {
    struct pipe_context *pipe;
    unsigned sentinel;
    unsigned num_total_call_slots;
+   struct tc_unflushed_batch_token *token;
    struct util_queue_fence fence;
    struct tc_call call[TC_CALLS_PER_BATCH];
 };
@@ -306,6 +350,7 @@
    struct pipe_context *pipe;
    struct slab_child_pool pool_transfers;
    tc_replace_buffer_storage_func replace_buffer_storage;
+   tc_create_fence_func create_fence;
    unsigned map_buffer_alignment;
 
    struct list_head unflushed_queries;
@@ -330,8 +375,13 @@
 threaded_context_create(struct pipe_context *pipe,
                         struct slab_parent_pool *parent_transfer_pool,
                         tc_replace_buffer_storage_func replace_buffer,
+                        tc_create_fence_func create_fence,
                         struct threaded_context **out);
 
+void
+threaded_context_flush(struct pipe_context *_pipe,
+                       struct tc_unflushed_batch_token *token);
+
 static inline struct threaded_context *
 threaded_context(struct pipe_context *pipe)
 {
@@ -356,4 +406,13 @@
    return (struct threaded_transfer*)transfer;
 }
 
+static inline void
+tc_unflushed_batch_token_reference(struct tc_unflushed_batch_token **dst,
+                                   struct tc_unflushed_batch_token *src)
+{
+   if (pipe_reference((struct pipe_reference *)*dst, (struct pipe_reference *)src))
+      free(*dst);
+   *dst = src;
+}
+
 #endif
diff --git a/src/gallium/auxiliary/util/u_threaded_context_calls.h b/src/gallium/auxiliary/util/u_threaded_context_calls.h
index 546819a..1356c54 100644
--- a/src/gallium/auxiliary/util/u_threaded_context_calls.h
+++ b/src/gallium/auxiliary/util/u_threaded_context_calls.h
@@ -1,3 +1,4 @@
+CALL(flush)
 CALL(destroy_query)
 CALL(begin_query)
 CALL(end_query)
diff --git a/src/gallium/drivers/radeonsi/si_fence.c b/src/gallium/drivers/radeonsi/si_fence.c
index b416c47..701e8df 100644
--- a/src/gallium/drivers/radeonsi/si_fence.c
+++ b/src/gallium/drivers/radeonsi/si_fence.c
@@ -26,6 +26,7 @@
 
 #include "util/os_time.h"
 #include "util/u_memory.h"
+#include "util/u_queue.h"
 
 #include "si_pipe.h"
 
@@ -33,6 +34,8 @@
 	struct pipe_reference reference;
 	struct pipe_fence_handle *gfx;
 	struct pipe_fence_handle *sdma;
+	struct tc_unflushed_batch_token *tc_token;
+	struct util_queue_fence ready;
 
 	/* If the context wasn't flushed at fence creation, this is non-NULL. */
 	struct {
@@ -62,32 +65,48 @@
 	if (pipe_reference(&(*rdst)->reference, &rsrc->reference)) {
 		ws->fence_reference(&(*rdst)->gfx, NULL);
 		ws->fence_reference(&(*rdst)->sdma, NULL);
+		tc_unflushed_batch_token_reference(&(*rdst)->tc_token, NULL);
 		FREE(*rdst);
 	}
         *rdst = rsrc;
 }
 
+static struct si_multi_fence *si_create_multi_fence()
+{
+	struct si_multi_fence *fence = CALLOC_STRUCT(si_multi_fence);
+	if (!fence)
+		return NULL;
+
+	pipe_reference_init(&fence->reference, 1);
+	util_queue_fence_init(&fence->ready);
+
+	return fence;
+}
+
+struct pipe_fence_handle *si_create_fence(struct pipe_context *ctx,
+					  struct tc_unflushed_batch_token *tc_token)
+{
+	struct si_multi_fence *fence = si_create_multi_fence();
+	if (!fence)
+		return NULL;
+
+	util_queue_fence_reset(&fence->ready);
+	tc_unflushed_batch_token_reference(&fence->tc_token, tc_token);
+
+	return (struct pipe_fence_handle *)fence;
+}
+
 static void si_fence_server_sync(struct pipe_context *ctx,
 				 struct pipe_fence_handle *fence)
 {
 	struct r600_common_context *rctx = (struct r600_common_context *)ctx;
 	struct si_multi_fence *rfence = (struct si_multi_fence *)fence;
 
-	/* Only amdgpu needs to handle fence dependencies (for fence imports).
-	 * radeon synchronizes all rings by default and will not implement
-	 * fence imports.
-	 */
-	if (rctx->screen->info.drm_major == 2)
-		return;
+	util_queue_fence_wait(&rfence->ready);
 
-	/* Only imported fences need to be handled by fence_server_sync,
-	 * because the winsys handles synchronizations automatically for BOs
-	 * within the process.
-	 *
-	 * Simply skip unflushed fences here, and the winsys will drop no-op
-	 * dependencies (i.e. dependencies within the same ring).
-	 */
-	if (rfence->gfx_unflushed.ctx)
+	/* Unflushed fences from the same context are no-ops. */
+	if (rfence->gfx_unflushed.ctx &&
+	    rfence->gfx_unflushed.ctx == rctx)
 		return;
 
 	/* All unflushed commands will not start execution before
@@ -114,6 +133,30 @@
 	ctx = threaded_context_unwrap_sync(ctx);
 	rctx = ctx ? (struct r600_common_context*)ctx : NULL;
 
+	if (!util_queue_fence_is_signalled(&rfence->ready)) {
+		if (!timeout)
+			return false;
+
+		if (rfence->tc_token) {
+			/* Ensure that si_flush_from_st will be called for
+			 * this fence, but only if we're in the API thread
+			 * where the context is current.
+			 *
+			 * Note that the batch containing the flush may already
+			 * be in flight in the driver thread, so the fence
+			 * may not be ready yet when this call returns.
+			 */
+			threaded_context_flush(ctx, rfence->tc_token);
+		}
+
+		if (timeout == PIPE_TIMEOUT_INFINITE) {
+			util_queue_fence_wait(&rfence->ready);
+		} else {
+			if (!util_queue_fence_wait_timeout(&rfence->ready, abs_timeout))
+				return false;
+		}
+	}
+
 	if (rfence->sdma) {
 		if (!rws->fence_wait(rws, rfence->sdma, timeout))
 			return false;
@@ -160,11 +203,10 @@
 	if (!rscreen->info.has_sync_file)
 		return;
 
-	rfence = CALLOC_STRUCT(si_multi_fence);
+	rfence = si_create_multi_fence();
 	if (!rfence)
 		return;
 
-	pipe_reference_init(&rfence->reference, 1);
 	rfence->gfx = ws->fence_import_sync_file(ws, fd);
 	if (!rfence->gfx) {
 		FREE(rfence);
@@ -185,6 +227,8 @@
 	if (!rscreen->info.has_sync_file)
 		return -1;
 
+	util_queue_fence_wait(&rfence->ready);
+
 	/* Deferred fences aren't supported. */
 	assert(!rfence->gfx_unflushed.ctx);
 	if (rfence->gfx_unflushed.ctx)
@@ -260,15 +304,23 @@
 
 	/* Both engines can signal out of order, so we need to keep both fences. */
 	if (fence) {
-		struct si_multi_fence *multi_fence =
-				CALLOC_STRUCT(si_multi_fence);
-		if (!multi_fence) {
-			ws->fence_reference(&sdma_fence, NULL);
-			ws->fence_reference(&gfx_fence, NULL);
-			goto finish;
+		struct si_multi_fence *multi_fence;
+
+		if (flags & TC_FLUSH_ASYNC) {
+			multi_fence = (struct si_multi_fence *)*fence;
+			assert(multi_fence);
+		} else {
+			multi_fence = si_create_multi_fence();
+			if (!multi_fence) {
+				ws->fence_reference(&sdma_fence, NULL);
+				ws->fence_reference(&gfx_fence, NULL);
+				goto finish;
+			}
+
+			screen->fence_reference(screen, fence, NULL);
+			*fence = (struct pipe_fence_handle*)multi_fence;
 		}
 
-		multi_fence->reference.count = 1;
 		/* If both fences are NULL, fence_finish will always return true. */
 		multi_fence->gfx = gfx_fence;
 		multi_fence->sdma = sdma_fence;
@@ -278,8 +330,10 @@
 			multi_fence->gfx_unflushed.ib_index = rctx->num_gfx_cs_flushes;
 		}
 
-		screen->fence_reference(screen, fence, NULL);
-		*fence = (struct pipe_fence_handle*)multi_fence;
+		if (flags & TC_FLUSH_ASYNC) {
+			util_queue_fence_signal(&multi_fence->ready);
+			tc_unflushed_batch_token_reference(&multi_fence->tc_token, NULL);
+		}
 	}
 finish:
 	if (!(flags & PIPE_FLUSH_DEFERRED)) {
diff --git a/src/gallium/drivers/radeonsi/si_pipe.c b/src/gallium/drivers/radeonsi/si_pipe.c
index a299ab7..ef5cddd 100644
--- a/src/gallium/drivers/radeonsi/si_pipe.c
+++ b/src/gallium/drivers/radeonsi/si_pipe.c
@@ -397,8 +397,11 @@
 	if (sscreen->b.debug_flags & DBG_ALL_SHADERS)
 		return ctx;
 
+	/* Use asynchronous flushes only on amdgpu, since the radeon
+	 * implementation for fence_server_sync is incomplete. */
 	return threaded_context_create(ctx, &sscreen->b.pool_transfers,
 				       si_replace_buffer_storage,
+				       sscreen->b.info.drm_major >= 3 ? si_create_fence : NULL,
 				       &((struct si_context*)ctx)->b.tc);
 }
 
diff --git a/src/gallium/drivers/radeonsi/si_pipe.h b/src/gallium/drivers/radeonsi/si_pipe.h
index 84e8a4c..de089e3 100644
--- a/src/gallium/drivers/radeonsi/si_pipe.h
+++ b/src/gallium/drivers/radeonsi/si_pipe.h
@@ -604,6 +604,8 @@
 /* si_fence.c */
 void si_init_fence_functions(struct si_context *ctx);
 void si_init_screen_fence_functions(struct si_screen *screen);
+struct pipe_fence_handle *si_create_fence(struct pipe_context *ctx,
+					  struct tc_unflushed_batch_token *tc_token);
 
 /* si_hw_context.c */
 void si_destroy_saved_cs(struct si_saved_cs *scs);