Module: Mesa
Branch: main
Commit: 04bfe828db8187bdda755d8c55ec2f8fbf3ae298
URL:    
http://cgit.freedesktop.org/mesa/mesa/commit/?id=04bfe828db8187bdda755d8c55ec2f8fbf3ae298

Author: Paulo Zanoni <paulo.r.zan...@intel.com>
Date:   Fri Sep 29 11:17:32 2023 -0700

anv/sparse: allow sparse resouces to use TR-TT as its backend

TR-TT is a hardware feature supported by both i915.ko and xe.ko, which
means we can now finally have Sparse Resources on i915.ko and we also
have 2 options for xe.ko (and whatever is the best should be the
default).

In this patch we use batch commands to write the page tables and
forever keep them in device memory. We maintain a mirror of both the
L3 and and L2 tables because that helps us never having to read the
tables that are in device memory.

We still have some things to improve, but with this commit, workloads
that didn't work at all due to the lack of sparse resources should
at least run.

This is still all disabled by default in i915.ko, you can turn it on
by exporting ANV_SPARSE=1 before launching the applications. For
xe.ko, switch the default with ANV_SPARSE_USE_TRTT=1.

Reviewed-by: Lionel Landwerlin <lionel.g.landwer...@intel.com>
Signed-off-by: Paulo Zanoni <paulo.r.zan...@intel.com>
Part-of: <https://gitlab.freedesktop.org/mesa/mesa/-/merge_requests/25512>

---

 src/intel/vulkan/anv_batch_chain.c      |  37 +++++
 src/intel/vulkan/anv_device.c           |  59 ++++++-
 src/intel/vulkan/anv_gem_stubs.c        |   8 +
 src/intel/vulkan/anv_genX.h             |   4 +
 src/intel/vulkan/anv_kmd_backend.h      |   3 +
 src/intel/vulkan/anv_private.h          |  68 ++++++++
 src/intel/vulkan/anv_sparse.c           | 276 +++++++++++++++++++++++++++++++-
 src/intel/vulkan/genX_cmd_buffer.c      |  54 +++++++
 src/intel/vulkan/genX_init_state.c      |  56 +++++++
 src/intel/vulkan/i915/anv_batch_chain.c |  99 +++++++++++-
 src/intel/vulkan/i915/anv_batch_chain.h |   6 +
 src/intel/vulkan/i915/anv_kmd_backend.c |   1 +
 src/intel/vulkan/xe/anv_batch_chain.c   |  45 ++++++
 src/intel/vulkan/xe/anv_batch_chain.h   |   5 +
 src/intel/vulkan/xe/anv_kmd_backend.c   |   1 +
 15 files changed, 715 insertions(+), 7 deletions(-)

diff --git a/src/intel/vulkan/anv_batch_chain.c 
b/src/intel/vulkan/anv_batch_chain.c
index 60af0fa3157..895b3979c81 100644
--- a/src/intel/vulkan/anv_batch_chain.c
+++ b/src/intel/vulkan/anv_batch_chain.c
@@ -118,6 +118,10 @@ VkResult
 anv_reloc_list_add_bo_impl(struct anv_reloc_list *list,
                            struct anv_bo *target_bo)
 {
+   /* This can happen with sparse resources. */
+   if (!target_bo)
+      return VK_SUCCESS;
+
    uint32_t idx = target_bo->gem_handle;
    VkResult result = anv_reloc_list_grow_deps(list,
                                               (idx / BITSET_WORDBITS) + 1);
@@ -1693,6 +1697,39 @@ anv_queue_submit_simple_batch(struct anv_queue *queue,
    return result;
 }
 
+VkResult
+anv_queue_submit_trtt_batch(struct anv_queue *queue,
+                            struct anv_batch *batch)
+{
+   struct anv_device *device = queue->device;
+   VkResult result = VK_SUCCESS;
+
+   uint32_t batch_size = align(batch->next - batch->start, 8);
+
+   struct anv_bo *batch_bo;
+   result = anv_bo_pool_alloc(&device->batch_bo_pool, batch_size, &batch_bo);
+   if (result != VK_SUCCESS)
+      return result;
+
+   memcpy(batch_bo->map, batch->start, batch_size);
+#ifdef SUPPORT_INTEL_INTEGRATED_GPUS
+   if (device->physical->memory.need_flush)
+      intel_flush_range(batch_bo->map, batch_size);
+#endif
+
+   if (INTEL_DEBUG(DEBUG_BATCH)) {
+      intel_print_batch(queue->decoder, batch_bo->map, batch_bo->size,
+                        batch_bo->offset, false);
+   }
+
+   result = device->kmd_backend->execute_trtt_batch(queue, batch_bo,
+                                                    batch_size);
+
+   anv_bo_pool_free(&device->batch_bo_pool, batch_bo);
+
+   return result;
+}
+
 void
 anv_cmd_buffer_clflush(struct anv_cmd_buffer **cmd_buffers,
                        uint32_t num_cmd_buffers)
diff --git a/src/intel/vulkan/anv_device.c b/src/intel/vulkan/anv_device.c
index ea0dd5cd749..85028608366 100644
--- a/src/intel/vulkan/anv_device.c
+++ b/src/intel/vulkan/anv_device.c
@@ -1444,8 +1444,17 @@ anv_physical_device_try_create(struct vk_instance 
*vk_instance,
 
    device->uses_relocs = device->info.kmd_type != INTEL_KMD_TYPE_XE;
 
-   device->has_sparse = device->info.kmd_type == INTEL_KMD_TYPE_XE &&
-      debug_get_bool_option("ANV_SPARSE", true);
+   /* While xe.ko can use both vm_bind and TR-TT, i915.ko only has TR-TT. */
+   if (device->info.kmd_type == INTEL_KMD_TYPE_XE) {
+      device->has_sparse = true;
+      device->sparse_uses_trtt =
+         debug_get_bool_option("ANV_SPARSE_USE_TRTT", false);
+   } else {
+      device->has_sparse =
+         device->info.ver >= 12 &&
+         debug_get_bool_option("ANV_SPARSE", false);
+      device->sparse_uses_trtt = true;
+   }
 
    device->always_flush_cache = INTEL_DEBUG(DEBUG_STALL) ||
       driQueryOptionb(&instance->dri_options, "always_flush_cache");
@@ -1732,6 +1741,11 @@ void anv_GetPhysicalDeviceProperties(
    const bool has_sparse_or_fake = pdevice->instance->has_fake_sparse ||
                                    pdevice->has_sparse;
 
+   uint64_t sparse_addr_space_size =
+      !has_sparse_or_fake ? 0 :
+      pdevice->sparse_uses_trtt ? pdevice->va.trtt.size :
+      1ULL << 48;
+
    VkSampleCountFlags sample_counts =
       isl_device_get_sample_counts(&pdevice->isl_dev);
 
@@ -1749,7 +1763,7 @@ void anv_GetPhysicalDeviceProperties(
       .maxMemoryAllocationCount                 = UINT32_MAX,
       .maxSamplerAllocationCount                = 64 * 1024,
       .bufferImageGranularity                   = 1,
-      .sparseAddressSpaceSize                   = has_sparse_or_fake ? (1uLL 
<< 48) : 0,
+      .sparseAddressSpaceSize                   = sparse_addr_space_size,
       .maxBoundDescriptorSets                   = MAX_SETS,
       .maxPerStageDescriptorSamplers            = max_samplers,
       .maxPerStageDescriptorUniformBuffers      = 
MAX_PER_STAGE_DESCRIPTOR_UNIFORM_BUFFERS,
@@ -3083,6 +3097,33 @@ anv_device_destroy_context_or_vm(struct anv_device 
*device)
    }
 }
 
+static VkResult
+anv_device_init_trtt(struct anv_device *device)
+{
+   struct anv_trtt *trtt = &device->trtt;
+
+   if (pthread_mutex_init(&trtt->mutex, NULL) != 0)
+      return vk_error(device, VK_ERROR_INITIALIZATION_FAILED);
+
+   return VK_SUCCESS;
+}
+
+static void
+anv_device_finish_trtt(struct anv_device *device)
+{
+   struct anv_trtt *trtt = &device->trtt;
+
+   pthread_mutex_destroy(&trtt->mutex);
+
+   vk_free(&device->vk.alloc, trtt->l3_mirror);
+   vk_free(&device->vk.alloc, trtt->l2_mirror);
+
+   for (int i = 0; i < trtt->num_page_table_bos; i++)
+      anv_device_release_bo(device, trtt->page_table_bos[i]);
+
+   vk_free(&device->vk.alloc, trtt->page_table_bos);
+}
+
 VkResult anv_CreateDevice(
     VkPhysicalDevice                            physicalDevice,
     const VkDeviceCreateInfo*                   pCreateInfo,
@@ -3542,16 +3583,20 @@ VkResult anv_CreateDevice(
          goto fail_trivial_batch_bo_and_scratch_pool;
    }
 
-   result = anv_genX(device->info, init_device_state)(device);
+   result = anv_device_init_trtt(device);
    if (result != VK_SUCCESS)
       goto fail_btd_fifo_bo;
 
+   result = anv_genX(device->info, init_device_state)(device);
+   if (result != VK_SUCCESS)
+      goto fail_trtt;
+
    struct vk_pipeline_cache_create_info pcc_info = { };
    device->default_pipeline_cache =
       vk_pipeline_cache_create(&device->vk, &pcc_info, NULL);
    if (!device->default_pipeline_cache) {
       result = vk_error(device, VK_ERROR_OUT_OF_HOST_MEMORY);
-      goto fail_btd_fifo_bo;
+      goto fail_trtt;
    }
 
    /* Internal shaders need their own pipeline cache because, unlike the rest
@@ -3654,6 +3699,8 @@ VkResult anv_CreateDevice(
    vk_pipeline_cache_destroy(device->internal_cache, NULL);
  fail_default_pipeline_cache:
    vk_pipeline_cache_destroy(device->default_pipeline_cache, NULL);
+ fail_trtt:
+   anv_device_finish_trtt(device);
  fail_btd_fifo_bo:
    if (ANV_SUPPORT_RT && device->info->has_ray_tracing)
       anv_device_release_bo(device, device->btd_fifo_bo);
@@ -3754,6 +3801,8 @@ void anv_DestroyDevice(
    vk_pipeline_cache_destroy(device->internal_cache, NULL);
    vk_pipeline_cache_destroy(device->default_pipeline_cache, NULL);
 
+   anv_device_finish_trtt(device);
+
    if (ANV_SUPPORT_RT && device->info->has_ray_tracing)
       anv_device_release_bo(device, device->btd_fifo_bo);
 
diff --git a/src/intel/vulkan/anv_gem_stubs.c b/src/intel/vulkan/anv_gem_stubs.c
index f2b8103ea81..55f7a403ee7 100644
--- a/src/intel/vulkan/anv_gem_stubs.c
+++ b/src/intel/vulkan/anv_gem_stubs.c
@@ -65,6 +65,13 @@ stub_execute_simple_batch(struct anv_queue *queue, struct 
anv_bo *batch_bo,
    return VK_ERROR_UNKNOWN;
 }
 
+static VkResult
+stub_execute_trtt_batch(struct anv_queue *queue, struct anv_bo *batch_bo,
+                        uint32_t batch_size)
+{
+   return VK_ERROR_UNKNOWN;
+}
+
 static VkResult
 stub_queue_exec_locked(struct anv_queue *queue,
                        uint32_t wait_count,
@@ -190,6 +197,7 @@ const struct anv_kmd_backend *anv_stub_kmd_backend_get(void)
       .vm_bind_bo = stub_vm_bind_bo,
       .vm_unbind_bo = stub_vm_bind_bo,
       .execute_simple_batch = stub_execute_simple_batch,
+      .execute_trtt_batch = stub_execute_trtt_batch,
       .queue_exec_locked = stub_queue_exec_locked,
       .queue_exec_trace = stub_queue_exec_trace,
       .bo_alloc_flags_to_bo_flags = stub_bo_alloc_flags_to_bo_flags,
diff --git a/src/intel/vulkan/anv_genX.h b/src/intel/vulkan/anv_genX.h
index a9bd1134982..745a5d38907 100644
--- a/src/intel/vulkan/anv_genX.h
+++ b/src/intel/vulkan/anv_genX.h
@@ -283,3 +283,7 @@ genX(simple_shader_push_state_address)(struct 
anv_simple_shader *state,
 
 void
 genX(emit_simple_shader_end)(struct anv_simple_shader *state);
+
+VkResult genX(init_trtt_context_state)(struct anv_queue *queue);
+
+VkResult genX(write_trtt_entries)(struct anv_trtt_submission *submit);
diff --git a/src/intel/vulkan/anv_kmd_backend.h 
b/src/intel/vulkan/anv_kmd_backend.h
index ed860eba81f..5e3f508e49b 100644
--- a/src/intel/vulkan/anv_kmd_backend.h
+++ b/src/intel/vulkan/anv_kmd_backend.h
@@ -77,6 +77,9 @@ struct anv_kmd_backend {
                                     struct anv_bo *batch_bo,
                                     uint32_t batch_bo_size,
                                     bool is_companion_rcs_batch);
+   VkResult (*execute_trtt_batch)(struct anv_queue *queue,
+                                  struct anv_bo *batch_bo,
+                                  uint32_t batch_size);
    VkResult (*queue_exec_locked)(struct anv_queue *queue,
                                  uint32_t wait_count,
                                  const struct vk_sync_wait *waits,
diff --git a/src/intel/vulkan/anv_private.h b/src/intel/vulkan/anv_private.h
index 08f3ee3652d..52f1cd722c6 100644
--- a/src/intel/vulkan/anv_private.h
+++ b/src/intel/vulkan/anv_private.h
@@ -238,6 +238,22 @@ struct intel_perf_query_result;
 #define SO_BUFFER_INDEX_0_CMD 0x60
 #define anv_printflike(a, b) __attribute__((__format__(__printf__, a, b)))
 
+/* The TR-TT L1 page table entries may contain these values instead of actual
+ * pointers to indicate the regions are either NULL or invalid. We program
+ * these values to TR-TT registers, so we could change them, but it's super
+ * convenient to have the NULL value be 0 because everything is
+ * zero-initialized when allocated.
+ *
+ * Since we reserve these values for NULL/INVALID, then we can't use them as
+ * destinations for TR-TT address translation. Both values are shifted by 16
+ * bits, wich results in graphic addresses 0 and 64k. On Anv the first vma
+ * starts at 2MB, so we already don't use 0 and 64k for anything, so there's
+ * nothing really to reserve. We could instead just reserve random 64kb
+ * ranges from any of the non-TR-TT vmas and use their addresses.
+ */
+#define ANV_TRTT_L1_NULL_TILE_VAL 0
+#define ANV_TRTT_L1_INVALID_TILE_VAL 1
+
 static inline uint32_t
 align_down_npot_u32(uint32_t v, uint32_t a)
 {
@@ -695,6 +711,21 @@ struct anv_state_stream {
    struct util_dynarray all_blocks;
 };
 
+struct anv_trtt_bind {
+   uint64_t pte_addr;
+   uint64_t entry_addr;
+};
+
+struct anv_trtt_submission {
+   struct anv_queue *queue;
+
+   struct anv_trtt_bind *l3l2_binds;
+   struct anv_trtt_bind *l1_binds;
+
+   int l3l2_binds_len;
+   int l1_binds_len;
+};
+
 /* The block_pool functions exported for testing only.  The block pool should
  * only be used via a state pool (see below).
  */
@@ -912,6 +943,7 @@ struct anv_physical_device {
      * a vm_bind ioctl).
      */
     bool                                        has_sparse;
+    bool                                        sparse_uses_trtt;
 
     /** True if HW supports ASTC LDR */
     bool                                        has_astc_ldr;
@@ -1724,6 +1756,40 @@ struct anv_device {
      */
     VkCommandPool                               companion_rcs_cmd_pool;
 
+    struct anv_trtt {
+       pthread_mutex_t mutex;
+
+       /* Sometimes we need to run batches from places where we don't have a
+        * queue coming from the API, so we use this.
+        */
+       struct anv_queue *queue;
+
+       /* There's only one L3 table, so if l3_addr is zero that means we
+        * didn't initialize the TR-TT context yet (i.e., we're not using TR-TT
+        * yet in this context).
+        */
+       uint64_t l3_addr;
+
+       /* We don't want to access the page tables from the CPU, so just
+        * maintain a mirror that we can use.
+        */
+       uint64_t *l3_mirror;
+       uint64_t *l2_mirror;
+
+       /* We keep a dynamic list of page table bos, and each bo can store
+        * multiple page tables.
+        */
+       struct anv_bo **page_table_bos;
+       int num_page_table_bos;
+       int page_table_bos_capacity;
+
+       /* These are used to keep track of space available for more page tables
+        * within a bo.
+        */
+       struct anv_bo *cur_page_table_bo;
+       uint64_t next_page_table_bo_offset;
+    } trtt;
+
     /* This is true if the user ever bound a sparse resource to memory. This
      * is used for a workaround that makes every memoryBarrier flush more
      * things than it should. Many applications request for the sparse
@@ -1861,6 +1927,8 @@ VkResult anv_queue_submit(struct vk_queue *queue,
 VkResult anv_queue_submit_simple_batch(struct anv_queue *queue,
                                        struct anv_batch *batch,
                                        bool is_companion_rcs_batch);
+VkResult anv_queue_submit_trtt_batch(struct anv_queue *queue,
+                                     struct anv_batch *batch);
 
 void anv_queue_trace(struct anv_queue *queue, const char *label,
                      bool frame, bool begin);
diff --git a/src/intel/vulkan/anv_sparse.c b/src/intel/vulkan/anv_sparse.c
index 8e8024db1ab..3da57f35de8 100644
--- a/src/intel/vulkan/anv_sparse.c
+++ b/src/intel/vulkan/anv_sparse.c
@@ -276,6 +276,275 @@ anv_sparse_get_standard_image_block_shape(enum isl_format 
format,
    return vk_extent3d_el_to_px(block_shape, layout);
 }
 
+/* We really want to try to have all the page tables on as few BOs as possible
+ * to benefit from cache locality and to keep the i915.ko relocation lists
+ * small. On the other hand, we don't want to waste memory on unused space.
+ */
+#define ANV_TRTT_PAGE_TABLE_BO_SIZE (2 * 1024 * 1024)
+
+static VkResult
+trtt_make_page_table_bo(struct anv_device *device, struct anv_bo **bo)
+{
+   VkResult result;
+   struct anv_trtt *trtt = &device->trtt;
+
+   result = anv_device_alloc_bo(device, "trtt-page-table",
+                                ANV_TRTT_PAGE_TABLE_BO_SIZE, 0, 0, bo);
+   if (result != VK_SUCCESS)
+      return result;
+
+   if (trtt->num_page_table_bos < trtt->page_table_bos_capacity) {
+      trtt->page_table_bos[trtt->num_page_table_bos++] = *bo;
+   } else {
+
+      int new_capacity = MAX2(8, trtt->page_table_bos_capacity * 2);
+      struct anv_bo **new_page_table_bos =
+         vk_realloc(&device->vk.alloc, trtt->page_table_bos,
+                    new_capacity * sizeof(*trtt->page_table_bos), 8,
+                    VK_SYSTEM_ALLOCATION_SCOPE_DEVICE);
+      if (!new_page_table_bos) {
+         anv_device_release_bo(device, *bo);
+         return vk_error(device, VK_ERROR_OUT_OF_HOST_MEMORY);
+      }
+
+      new_page_table_bos[trtt->num_page_table_bos] = *bo;
+
+      trtt->page_table_bos = new_page_table_bos;
+      trtt->page_table_bos_capacity = new_capacity;
+      trtt->num_page_table_bos++;
+   }
+
+   trtt->cur_page_table_bo = *bo;
+   trtt->next_page_table_bo_offset = 0;
+
+   sparse_debug("new number of page table BOs: %d\n",
+                trtt->num_page_table_bos);
+
+   return VK_SUCCESS;
+}
+
+static VkResult
+trtt_get_page_table_bo(struct anv_device *device, struct anv_bo **bo,
+                       uint64_t *bo_addr)
+{
+   struct anv_trtt *trtt = &device->trtt;
+   VkResult result;
+
+   if (!trtt->cur_page_table_bo) {
+      result = trtt_make_page_table_bo(device, bo);
+      if (result != VK_SUCCESS)
+         return result;
+   }
+
+   *bo = trtt->cur_page_table_bo;
+   *bo_addr = trtt->cur_page_table_bo->offset +
+              trtt->next_page_table_bo_offset;
+
+   trtt->next_page_table_bo_offset += 4096;
+   if (trtt->next_page_table_bo_offset >= ANV_TRTT_PAGE_TABLE_BO_SIZE)
+      trtt->cur_page_table_bo = NULL;
+
+   return VK_SUCCESS;
+}
+
+static VkResult
+anv_trtt_init_context_state(struct anv_queue *queue)
+{
+   struct anv_device *device = queue->device;
+   struct anv_trtt *trtt = &device->trtt;
+
+   struct anv_bo *l3_bo;
+   VkResult result = trtt_get_page_table_bo(device, &l3_bo, &trtt->l3_addr);
+   if (result != VK_SUCCESS)
+      return result;
+
+   trtt->l3_mirror = vk_zalloc(&device->vk.alloc, 4096, 8,
+                                VK_SYSTEM_ALLOCATION_SCOPE_DEVICE);
+   if (!trtt->l3_mirror) {
+      result = vk_error(device, VK_ERROR_OUT_OF_HOST_MEMORY);
+      return result;
+   }
+
+   /* L3 has 512 entries, so we can have up to 512 L2 tables. */
+   trtt->l2_mirror = vk_zalloc(&device->vk.alloc, 512 * 4096, 8,
+                               VK_SYSTEM_ALLOCATION_SCOPE_DEVICE);
+   if (!trtt->l2_mirror) {
+      result = vk_error(device, VK_ERROR_OUT_OF_HOST_MEMORY);
+      goto fail_free_l3;
+   }
+
+   result = anv_genX(device->info, init_trtt_context_state)(queue);
+
+   return result;
+
+fail_free_l3:
+   vk_free(&device->vk.alloc, trtt->l3_mirror);
+   return result;
+}
+
+static void
+anv_trtt_bind_list_add_entry(struct anv_trtt_bind *binds, int *binds_len,
+                             uint64_t pte_addr, uint64_t entry_addr)
+{
+   binds[*binds_len] = (struct anv_trtt_bind) {
+      .pte_addr = pte_addr,
+      .entry_addr = entry_addr,
+   };
+   (*binds_len)++;
+}
+
+/* For L3 and L2 pages, null and invalid entries are indicated by bits 1 and 0
+ * respectively. For L1 entries, the hardware compares the addresses against
+ * what we program to the GFX_TRTT_NULL and GFX_TRTT_INVAL registers.
+ */
+#define ANV_TRTT_L3L2_NULL_ENTRY (1 << 1)
+#define ANV_TRTT_L3L2_INVALID_ENTRY (1 << 0)
+
+/* Adds elements to the anv_trtt_bind structs passed. This doesn't write the
+ * entries to the HW yet.
+ */
+static VkResult
+anv_trtt_bind_add(struct anv_device *device,
+                  uint64_t trtt_addr, uint64_t dest_addr,
+                  struct anv_trtt_submission *s)
+{
+   VkResult result = VK_SUCCESS;
+   struct anv_trtt *trtt = &device->trtt;
+   bool is_null_bind = dest_addr == ANV_TRTT_L1_NULL_TILE_VAL;
+
+   int l3_index = (trtt_addr >> 35) & 0x1FF;
+   int l2_index = (trtt_addr >> 26) & 0x1FF;
+   int l1_index = (trtt_addr >> 16) & 0x3FF;
+
+   uint64_t l2_addr = trtt->l3_mirror[l3_index];
+   if (l2_addr == ANV_TRTT_L3L2_NULL_ENTRY && is_null_bind) {
+      return VK_SUCCESS;
+   } else if (l2_addr == 0 || l2_addr == ANV_TRTT_L3L2_NULL_ENTRY) {
+      if (is_null_bind) {
+         trtt->l3_mirror[l3_index] = ANV_TRTT_L3L2_NULL_ENTRY;
+
+         anv_trtt_bind_list_add_entry(s->l3l2_binds, &s->l3l2_binds_len,
+               trtt->l3_addr + l3_index * sizeof(uint64_t),
+               ANV_TRTT_L3L2_NULL_ENTRY);
+
+         return VK_SUCCESS;
+      }
+
+      struct anv_bo *l2_bo;
+      result = trtt_get_page_table_bo(device, &l2_bo, &l2_addr);
+      if (result != VK_SUCCESS)
+         return result;
+
+      trtt->l3_mirror[l3_index] = l2_addr;
+
+      anv_trtt_bind_list_add_entry(s->l3l2_binds, &s->l3l2_binds_len,
+            trtt->l3_addr + l3_index * sizeof(uint64_t), l2_addr);
+   }
+   assert(l2_addr != 0 && l2_addr != ANV_TRTT_L3L2_NULL_ENTRY);
+
+   /* The first page in the l2_mirror corresponds to l3_index=0 and so on. */
+   uint64_t l1_addr = trtt->l2_mirror[l3_index * 512 + l2_index];
+   if (l1_addr == ANV_TRTT_L3L2_NULL_ENTRY && is_null_bind) {
+      return VK_SUCCESS;
+   } else if (l1_addr == 0 || l1_addr == ANV_TRTT_L3L2_NULL_ENTRY) {
+      if (is_null_bind) {
+         trtt->l2_mirror[l3_index * 512 + l2_index] =
+            ANV_TRTT_L3L2_NULL_ENTRY;
+
+         anv_trtt_bind_list_add_entry(s->l3l2_binds, &s->l3l2_binds_len,
+               l2_addr + l2_index * sizeof(uint64_t),
+               ANV_TRTT_L3L2_NULL_ENTRY);
+
+         return VK_SUCCESS;
+      }
+
+      struct anv_bo *l1_bo;
+      result = trtt_get_page_table_bo(device, &l1_bo, &l1_addr);
+      if (result != VK_SUCCESS)
+         return result;
+
+      trtt->l2_mirror[l3_index * 512 + l2_index] = l1_addr;
+
+      anv_trtt_bind_list_add_entry(s->l3l2_binds, &s->l3l2_binds_len,
+            l2_addr + l2_index * sizeof(uint64_t), l1_addr);
+   }
+   assert(l1_addr != 0 && l1_addr != ANV_TRTT_L3L2_NULL_ENTRY);
+
+   anv_trtt_bind_list_add_entry(s->l1_binds, &s->l1_binds_len,
+            l1_addr + l1_index * sizeof(uint32_t), dest_addr);
+
+   return VK_SUCCESS;
+}
+
+static VkResult
+anv_sparse_bind_trtt(struct anv_device *device, int num_vm_binds,
+                     struct anv_vm_bind *vm_binds)
+{
+   struct anv_trtt *trtt = &device->trtt;
+   VkResult result;
+
+   /* These capacities are conservative estimations. For L1 binds the
+    * number will match exactly unless we skip NULL binds due to L2 already
+    * being NULL. For L3/L2 things are harder to estimate, but the resulting
+    * numbers are so small that a little overestimation won't hurt.
+    *
+    * We have assertions below to catch estimation errors.
+    */
+   int l3l2_binds_capacity = 1;
+   int l1_binds_capacity = 0;
+   for (int b = 0; b < num_vm_binds; b++) {
+      int pages = vm_binds[b].size / (64 * 1024);
+      l1_binds_capacity += pages;
+      l3l2_binds_capacity += (pages / 1024 + 1) * 2;
+   }
+
+   STACK_ARRAY(struct anv_trtt_bind, l3l2_binds, l3l2_binds_capacity);
+   STACK_ARRAY(struct anv_trtt_bind, l1_binds, l1_binds_capacity);
+   struct anv_trtt_submission s = {
+      .queue = trtt->queue,
+      .l3l2_binds = l3l2_binds,
+      .l1_binds = l1_binds,
+      .l3l2_binds_len = 0,
+      .l1_binds_len = 0,
+   };
+
+   pthread_mutex_lock(&trtt->mutex);
+
+   if (!trtt->l3_addr)
+      anv_trtt_init_context_state(s.queue);
+
+   assert(trtt->l3_addr);
+
+   for (int b = 0; b < num_vm_binds; b++) {
+      for (size_t i = 0; i < vm_binds[b].size; i += 64 * 1024) {
+         uint64_t trtt_addr = vm_binds[b].address + i;
+         uint64_t dest_addr =
+            (vm_binds[b].op == ANV_VM_BIND && vm_binds[b].bo) ?
+               vm_binds[b].bo->offset + vm_binds[b].bo_offset + i :
+               ANV_TRTT_L1_NULL_TILE_VAL;
+
+         result = anv_trtt_bind_add(device, trtt_addr, dest_addr, &s);
+         if (result != VK_SUCCESS)
+            goto out;
+      }
+   }
+
+   assert(s.l3l2_binds_len <= l3l2_binds_capacity);
+   assert(s.l1_binds_len <= l1_binds_capacity);
+
+   sparse_debug("trtt_binds: num_vm_binds:%02d l3l2:%04d l1:%04d\n",
+                num_vm_binds, s.l3l2_binds_len, s.l1_binds_len);
+
+   if (s.l3l2_binds_len || s.l1_binds_len)
+      result = anv_genX(device->info, write_trtt_entries)(&s);
+
+out:
+   pthread_mutex_unlock(&trtt->mutex);
+   STACK_ARRAY_FINISH(l1_binds);
+   STACK_ARRAY_FINISH(l3l2_binds);
+   return result;
+}
+
 static VkResult
 anv_sparse_bind_vm_bind(struct anv_device *device, int num_binds,
                         struct anv_vm_bind *binds)
@@ -303,7 +572,9 @@ anv_sparse_bind(struct anv_device *device,
          dump_anv_vm_bind(device, sparse, &binds[b]);
    }
 
-   return anv_sparse_bind_vm_bind(device, num_binds, binds);
+   return device->physical->sparse_uses_trtt ?
+            anv_sparse_bind_trtt(device, num_binds, binds) :
+            anv_sparse_bind_vm_bind(device, num_binds, binds);
 }
 
 VkResult
@@ -316,6 +587,9 @@ anv_init_sparse_bindings(struct anv_device *device,
 {
    uint64_t size = align64(size_, ANV_SPARSE_BLOCK_SIZE);
 
+   if (device->physical->sparse_uses_trtt)
+      alloc_flags |= ANV_BO_ALLOC_TRTT;
+
    sparse->address = anv_vma_alloc(device, size, ANV_SPARSE_BLOCK_SIZE,
                                    alloc_flags,
                                    intel_48b_address(client_address),
diff --git a/src/intel/vulkan/genX_cmd_buffer.c 
b/src/intel/vulkan/genX_cmd_buffer.c
index 2e5387d8227..496b7ed8557 100644
--- a/src/intel/vulkan/genX_cmd_buffer.c
+++ b/src/intel/vulkan/genX_cmd_buffer.c
@@ -8426,3 +8426,57 @@ genX(cmd_buffer_end_companion_rcs_syncpoint)(struct 
anv_cmd_buffer *cmd_buffer,
    unreachable("Not implemented");
 #endif
 }
+
+VkResult
+genX(write_trtt_entries)(struct anv_trtt_submission *submit)
+{
+#if GFX_VER >= 12
+   struct anv_queue *queue = submit->queue;
+   size_t batch_size = submit->l3l2_binds_len * 20 +
+                       submit->l1_binds_len * 16 + 8;
+   STACK_ARRAY(uint32_t, cmds, batch_size);
+   struct anv_batch batch = {
+      .start = cmds,
+      .next = cmds,
+      .end = (void *)cmds + batch_size,
+   };
+
+   /* TODO: writes to contiguous addresses can be combined into a single big
+    * MI_STORE_DATA_IMM instruction.
+    */
+
+   for (int i = 0; i < submit->l3l2_binds_len; i++) {
+      bool is_last_write = submit->l1_binds_len == 0 &&
+                           i + 1 == submit->l3l2_binds_len;
+
+      anv_batch_emitn(&batch, 5, GENX(MI_STORE_DATA_IMM),
+         .ForceWriteCompletionCheck = is_last_write,
+         .StoreQword = true,
+         .Address = anv_address_from_u64(submit->l3l2_binds[i].pte_addr),
+         .ImmediateData = submit->l3l2_binds[i].entry_addr,
+      );
+   }
+
+   for (int i = 0; i < submit->l1_binds_len; i++) {
+      bool is_last_write = i + 1 == submit->l1_binds_len;
+
+      anv_batch_emit(&batch, GENX(MI_STORE_DATA_IMM), sdi) {
+         sdi.ForceWriteCompletionCheck = is_last_write;
+         sdi.Address = anv_address_from_u64(submit->l1_binds[i].pte_addr);
+         sdi.ImmediateData =
+            (submit->l1_binds[i].entry_addr >> 16) & 0xFFFFFFFF;
+      }
+   }
+
+   anv_batch_emit(&batch, GENX(MI_BATCH_BUFFER_END), bbe);
+
+   assert(batch.next <= batch.end);
+
+   VkResult result = anv_queue_submit_trtt_batch(queue, &batch);
+   STACK_ARRAY_FINISH(cmds);
+
+   return result;
+
+#endif
+   return VK_SUCCESS;
+}
diff --git a/src/intel/vulkan/genX_init_state.c 
b/src/intel/vulkan/genX_init_state.c
index 98d0c40a490..1586686fa99 100644
--- a/src/intel/vulkan/genX_init_state.c
+++ b/src/intel/vulkan/genX_init_state.c
@@ -606,6 +606,9 @@ init_render_queue_state(struct anv_queue *queue, bool 
is_companion_rcs_batch)
 
    assert(batch.next <= batch.end);
 
+   if (!device->trtt.queue)
+      device->trtt.queue = queue;
+
    return anv_queue_submit_simple_batch(queue, &batch, is_companion_rcs_batch);
 }
 
@@ -1205,3 +1208,56 @@ genX(apply_task_urb_workaround)(struct anv_cmd_buffer 
*cmd_buffer)
        WriteImmediateData, cmd_buffer->device->workaround_address, 0, 0);
 #endif
 }
+
+VkResult
+genX(init_trtt_context_state)(struct anv_queue *queue)
+{
+#if GFX_VER >= 12
+   struct anv_device *device = queue->device;
+   struct anv_trtt *trtt = &device->trtt;
+
+   uint32_t cmds[128];
+   struct anv_batch batch = {
+      .start = cmds,
+      .next = cmds,
+      .end = (void *)cmds + sizeof(cmds),
+   };
+
+   anv_batch_write_reg(&batch, GENX(GFX_TRTT_INVAL), trtt_inval) {
+      trtt_inval.InvalidTileDetectionValue = ANV_TRTT_L1_INVALID_TILE_VAL;
+   }
+   anv_batch_write_reg(&batch, GENX(GFX_TRTT_NULL), trtt_null) {
+      trtt_null.NullTileDetectionValue = ANV_TRTT_L1_NULL_TILE_VAL;
+   }
+   anv_batch_write_reg(&batch, GENX(GFX_TRTT_VA_RANGE), trtt_va_range) {
+      trtt_va_range.TRVAMaskValue = 0xF;
+      trtt_va_range.TRVADataValue = 0xF;
+   }
+
+   uint64_t l3_addr = trtt->l3_addr;
+   assert((l3_addr & 0xFFF) == 0);
+   anv_batch_write_reg(&batch, GENX(GFX_TRTT_L3_BASE_LOW), trtt_base_low) {
+      trtt_base_low.TRVAL3PointerLowerAddress =
+         (l3_addr & 0xFFFFF000) >> 12;
+   }
+   anv_batch_write_reg(&batch, GENX(GFX_TRTT_L3_BASE_HIGH),
+         trtt_base_high) {
+      trtt_base_high.TRVAL3PointerUpperAddress =
+         (l3_addr >> 32) & 0xFFFF;
+   }
+   /* Enabling TR-TT needs to be done after setting up the other registers.
+   */
+   anv_batch_write_reg(&batch, GENX(GFX_TRTT_CR), trtt_cr) {
+      trtt_cr.TRTTEnable = true;
+   }
+
+   anv_batch_emit(&batch, GENX(MI_BATCH_BUFFER_END), bbe);
+   assert(batch.next <= batch.end);
+
+   VkResult res = anv_queue_submit_simple_batch(queue, &batch, false);
+   if (res != VK_SUCCESS)
+      return res;
+
+#endif
+   return VK_SUCCESS;
+}
diff --git a/src/intel/vulkan/i915/anv_batch_chain.c 
b/src/intel/vulkan/i915/anv_batch_chain.c
index 925c40e09ca..1ea0697f2bd 100644
--- a/src/intel/vulkan/i915/anv_batch_chain.c
+++ b/src/intel/vulkan/i915/anv_batch_chain.c
@@ -338,6 +338,31 @@ get_context_and_exec_flags(struct anv_queue *queue,
                  device->context_id;
 }
 
+static VkResult
+anv_execbuf_add_trtt_bos(struct anv_device *device,
+                         struct anv_execbuf *execbuf)
+{
+   struct anv_trtt *trtt = &device->trtt;
+   VkResult result = VK_SUCCESS;
+
+   /* If l3_addr is zero we're not using TR-TT, there's no bo to add. */
+   if (!trtt->l3_addr)
+      return VK_SUCCESS;
+
+   pthread_mutex_lock(&trtt->mutex);
+
+   for (int i = 0; i < trtt->num_page_table_bos; i++) {
+      result = anv_execbuf_add_bo(device, execbuf, trtt->page_table_bos[i],
+                                  NULL, 0);
+      if (result != VK_SUCCESS)
+         goto out;
+   }
+
+out:
+   pthread_mutex_unlock(&trtt->mutex);
+   return result;
+}
+
 static VkResult
 setup_execbuf_for_cmd_buffers(struct anv_execbuf *execbuf,
                               struct anv_queue *queue,
@@ -401,7 +426,8 @@ setup_execbuf_for_cmd_buffers(struct anv_execbuf *execbuf,
       return result;
 
    /* Add the BOs for all user allocated memory objects because we can't
-    * track after binding updates of VK_EXT_descriptor_indexing.
+    * track after binding updates of VK_EXT_descriptor_indexing and due to how
+    * sparse resources work.
     */
    list_for_each_entry(struct anv_device_memory, mem,
                        &device->memory_objects, link) {
@@ -410,6 +436,10 @@ setup_execbuf_for_cmd_buffers(struct anv_execbuf *execbuf,
          return result;
    }
 
+   result = anv_execbuf_add_trtt_bos(device, execbuf);
+   if (result != VK_SUCCESS)
+      return result;
+
    /* Add all the private BOs from images because we can't track after binding
     * updates of VK_EXT_descriptor_indexing.
     */
@@ -954,6 +984,73 @@ fail:
    return result;
 }
 
+VkResult
+i915_execute_trtt_batch(struct anv_queue *queue, struct anv_bo *batch_bo,
+                        uint32_t batch_size)
+{
+   struct anv_device *device = queue->device;
+   struct anv_trtt *trtt = &device->trtt;
+   struct anv_execbuf execbuf = {
+      .alloc = &device->vk.alloc,
+      .alloc_scope = VK_SYSTEM_ALLOCATION_SCOPE_DEVICE,
+   };
+   VkResult result;
+
+   result = anv_execbuf_add_bo(device, &execbuf, device->workaround_bo, NULL,
+                               0);
+   if (result != VK_SUCCESS)
+      goto out;
+
+   for (int i = 0; i < trtt->num_page_table_bos; i++) {
+      result = anv_execbuf_add_bo(device, &execbuf, trtt->page_table_bos[i],
+                                  NULL, EXEC_OBJECT_WRITE);
+      if (result != VK_SUCCESS)
+         goto out;
+   }
+
+   result = anv_execbuf_add_bo(device, &execbuf, batch_bo, NULL, 0);
+   if (result != VK_SUCCESS)
+      goto out;
+
+   if (INTEL_DEBUG(DEBUG_SUBMIT))
+      anv_i915_debug_submit(&execbuf);
+
+   uint64_t exec_flags = 0;
+   uint32_t context_id;
+   get_context_and_exec_flags(queue, false, &exec_flags, &context_id);
+
+   execbuf.execbuf = (struct drm_i915_gem_execbuffer2) {
+      .buffers_ptr = (uintptr_t) execbuf.objects,
+      .buffer_count = execbuf.bo_count,
+      .batch_start_offset = 0,
+      .batch_len = batch_size,
+      .flags = I915_EXEC_HANDLE_LUT | I915_EXEC_NO_RELOC | exec_flags,
+      .rsvd1 = context_id,
+      .rsvd2 = 0,
+   };
+
+   int ret = queue->device->info->no_hw ? 0 :
+      anv_gem_execbuffer(device, &execbuf.execbuf);
+   if (ret) {
+      result = vk_device_set_lost(&device->vk,
+                                  "trtt anv_gem_execbuffer failed: %m");
+      goto out;
+   }
+
+   /* TODO: we can get rid of this wait once we can properly handle the buffer
+    * lifetimes.
+    */
+   result = anv_device_wait(device, batch_bo, INT64_MAX);
+   if (result != VK_SUCCESS) {
+      result = vk_device_set_lost(&device->vk,
+                                  "trtt anv_device_wait failed: %m");
+   }
+
+out:
+   anv_execbuf_finish(&execbuf);
+   return result;
+}
+
 VkResult
 i915_queue_exec_trace(struct anv_queue *queue,
                       struct anv_utrace_submit *submit)
diff --git a/src/intel/vulkan/i915/anv_batch_chain.h 
b/src/intel/vulkan/i915/anv_batch_chain.h
index f46f19c90bc..eff38ce2ee2 100644
--- a/src/intel/vulkan/i915/anv_batch_chain.h
+++ b/src/intel/vulkan/i915/anv_batch_chain.h
@@ -29,6 +29,7 @@
 
 #include "vk_sync.h"
 
+struct anv_device;
 struct anv_queue;
 struct anv_bo;
 struct anv_cmd_buffer;
@@ -41,6 +42,11 @@ i915_queue_exec_trace(struct anv_queue *queue,
 VkResult
 i915_execute_simple_batch(struct anv_queue *queue, struct anv_bo *batch_bo,
                           uint32_t batch_bo_size, bool is_companion_rcs_batch);
+
+VkResult
+i915_execute_trtt_batch(struct anv_queue *queue, struct anv_bo *batch_bo,
+                        uint32_t batch_size);
+
 VkResult
 i915_queue_exec_locked(struct anv_queue *queue,
                        uint32_t wait_count,
diff --git a/src/intel/vulkan/i915/anv_kmd_backend.c 
b/src/intel/vulkan/i915/anv_kmd_backend.c
index fe9be942ca8..887a9dbf6c5 100644
--- a/src/intel/vulkan/i915/anv_kmd_backend.c
+++ b/src/intel/vulkan/i915/anv_kmd_backend.c
@@ -277,6 +277,7 @@ anv_i915_kmd_backend_get(void)
       .vm_bind_bo = i915_vm_bind_bo,
       .vm_unbind_bo = i915_vm_bind_bo,
       .execute_simple_batch = i915_execute_simple_batch,
+      .execute_trtt_batch = i915_execute_trtt_batch,
       .queue_exec_locked = i915_queue_exec_locked,
       .queue_exec_trace = i915_queue_exec_trace,
       .bo_alloc_flags_to_bo_flags = i915_bo_alloc_flags_to_bo_flags,
diff --git a/src/intel/vulkan/xe/anv_batch_chain.c 
b/src/intel/vulkan/xe/anv_batch_chain.c
index baf04db3cb3..187be25e5ca 100644
--- a/src/intel/vulkan/xe/anv_batch_chain.c
+++ b/src/intel/vulkan/xe/anv_batch_chain.c
@@ -178,6 +178,51 @@ xe_exec_print_debug(struct anv_queue *queue, uint32_t 
cmd_buffer_count,
                                    is_companion_rcs_cmd_buffer);
 }
 
+VkResult
+xe_execute_trtt_batch(struct anv_queue *queue, struct anv_bo *batch_bo,
+                      uint32_t batch_size)
+{
+   struct anv_device *device = queue->device;
+   VkResult result = VK_SUCCESS;
+
+   uint32_t syncobj_handle;
+   if (drmSyncobjCreate(device->fd, 0, &syncobj_handle))
+      return vk_errorf(device, VK_ERROR_UNKNOWN, "Unable to create sync obj");
+
+   struct drm_xe_sync sync = {
+      .flags = DRM_XE_SYNC_SYNCOBJ | DRM_XE_SYNC_SIGNAL,
+      .handle = syncobj_handle,
+   };
+   struct drm_xe_exec exec = {
+      .exec_queue_id = queue->exec_queue_id,
+      .num_batch_buffer = 1,
+      .address = batch_bo->offset,
+      .num_syncs = 1,
+      .syncs = (uintptr_t)&sync,
+   };
+
+   if (!device->info->no_hw) {
+      if (intel_ioctl(device->fd, DRM_IOCTL_XE_EXEC, &exec)) {
+         result = vk_device_set_lost(&device->vk, "XE_EXEC failed: %m");
+         goto exec_error;
+      }
+   }
+
+   /* FIXME: we shouldn't need this wait, figure out a way to remove it. */
+   struct drm_syncobj_wait wait = {
+      .handles = (uintptr_t)&syncobj_handle,
+      .timeout_nsec = INT64_MAX,
+      .count_handles = 1,
+   };
+   if (intel_ioctl(device->fd, DRM_IOCTL_SYNCOBJ_WAIT, &wait))
+      result = vk_device_set_lost(&device->vk, "DRM_IOCTL_SYNCOBJ_WAIT failed: 
%m");
+
+exec_error:
+   drmSyncobjDestroy(device->fd, syncobj_handle);
+
+   return result;
+}
+
 VkResult
 xe_queue_exec_utrace_locked(struct anv_queue *queue,
                             struct anv_utrace_submit *utrace_submit)
diff --git a/src/intel/vulkan/xe/anv_batch_chain.h 
b/src/intel/vulkan/xe/anv_batch_chain.h
index f664f9673dd..d11dd11316a 100644
--- a/src/intel/vulkan/xe/anv_batch_chain.h
+++ b/src/intel/vulkan/xe/anv_batch_chain.h
@@ -28,6 +28,7 @@
 #include "vulkan/vulkan_core.h"
 #include "vk_sync.h"
 
+struct anv_device;
 struct anv_queue;
 struct anv_bo;
 struct anv_cmd_buffer;
@@ -38,6 +39,10 @@ VkResult
 xe_execute_simple_batch(struct anv_queue *queue, struct anv_bo *batch_bo,
                         uint32_t batch_bo_size, bool is_companion_rcs_batch);
 VkResult
+xe_execute_trtt_batch(struct anv_queue *queue, struct anv_bo *batch_bo,
+                      uint32_t batch_size);
+
+VkResult
 xe_queue_exec_locked(struct anv_queue *queue,
                      uint32_t wait_count,
                      const struct vk_sync_wait *waits,
diff --git a/src/intel/vulkan/xe/anv_kmd_backend.c 
b/src/intel/vulkan/xe/anv_kmd_backend.c
index 5063b19ec2c..a80bfb57c3b 100644
--- a/src/intel/vulkan/xe/anv_kmd_backend.c
+++ b/src/intel/vulkan/xe/anv_kmd_backend.c
@@ -220,6 +220,7 @@ anv_xe_kmd_backend_get(void)
       .vm_bind_bo = xe_vm_bind_bo,
       .vm_unbind_bo = xe_vm_unbind_bo,
       .execute_simple_batch = xe_execute_simple_batch,
+      .execute_trtt_batch = xe_execute_trtt_batch,
       .queue_exec_locked = xe_queue_exec_locked,
       .queue_exec_trace = xe_queue_exec_utrace_locked,
       .bo_alloc_flags_to_bo_flags = xe_bo_alloc_flags_to_bo_flags,


Reply via email to