[HIP] Implement event-backed semaphore and deferred queue (#16305)

Progress towards #15790 
diff --git a/experimental/hip/CMakeLists.txt b/experimental/hip/CMakeLists.txt
index 18c532c..818774d 100644
--- a/experimental/hip/CMakeLists.txt
+++ b/experimental/hip/CMakeLists.txt
@@ -25,6 +25,10 @@
     "api.h"
   SRCS
     "api.h"
+    "event_pool.c"
+    "event_pool.h"
+    "event_semaphore.c"
+    "event_semaphore.h"
     "graph_command_buffer.c"
     "graph_command_buffer.h"
     "hip_allocator.c"
@@ -40,10 +44,14 @@
     "native_executable.h"
     "nop_executable_cache.c"
     "nop_executable_cache.h"
+    "pending_queue_actions.c"
+    "pending_queue_actions.h"
     "pipeline_layout.c"
     "pipeline_layout.h"
     "stream_command_buffer.c"
     "stream_command_buffer.h"
+    "timepoint_pool.c"
+    "timepoint_pool.h"
   INCLUDES
     "${HIP_API_HEADERS_ROOT}"
   DEPS
@@ -52,11 +60,16 @@
     iree::base::core_headers
     iree::base::internal
     iree::base::internal::arena
+    iree::base::internal::atomic_slist
+    iree::base::internal::event_pool
+    iree::base::internal::synchronization
+    iree::base::internal::threading
     iree::base::internal::flatcc::parsing
     iree::hal
     iree::hal::utils::collective_batch
     iree::hal::utils::deferred_command_buffer
     iree::hal::utils::resource_set
+    iree::hal::utils::semaphore_base
     iree::schemas::rocm_executable_def_c_fbs
   COPTS
     "-D__HIP_PLATFORM_HCC__=1"
diff --git a/experimental/hip/dynamic_symbol_tables.h b/experimental/hip/dynamic_symbol_tables.h
index 78e8e2a..6ca17a1 100644
--- a/experimental/hip/dynamic_symbol_tables.h
+++ b/experimental/hip/dynamic_symbol_tables.h
@@ -19,6 +19,7 @@
                   const hipGraphNode_t *, size_t, const HIP_MEMCPY3D *,
                   hipCtx_t)
 IREE_HIP_PFN_DECL(hipEventCreate, hipEvent_t *)
+IREE_HIP_PFN_DECL(hipEventCreateWithFlags, hipEvent_t *, unsigned int)
 IREE_HIP_PFN_DECL(hipEventDestroy, hipEvent_t)
 IREE_HIP_PFN_DECL(hipEventElapsedTime, float *, hipEvent_t, hipEvent_t)
 IREE_HIP_PFN_DECL(hipEventQuery, hipEvent_t)
@@ -51,6 +52,7 @@
 IREE_HIP_PFN_DECL(hipHostRegister, void *, size_t, unsigned int)
 IREE_HIP_PFN_DECL(hipHostUnregister, void *)
 IREE_HIP_PFN_DECL(hipInit, unsigned int)
+IREE_HIP_PFN_DECL(hipLaunchHostFunc, hipStream_t, hipHostFn_t, void *)
 IREE_HIP_PFN_DECL(hipLaunchKernel, const void *, dim3, dim3, void **, size_t,
                   hipStream_t)
 IREE_HIP_PFN_DECL(hipMalloc, void **, size_t)
diff --git a/experimental/hip/event_pool.c b/experimental/hip/event_pool.c
new file mode 100644
index 0000000..a26424b
--- /dev/null
+++ b/experimental/hip/event_pool.c
@@ -0,0 +1,308 @@
+// Copyright 2024 The IREE Authors
+//
+// Licensed under the Apache License v2.0 with LLVM Exceptions.
+// See https://llvm.org/LICENSE.txt for license information.
+// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
+
+#include "experimental/hip/event_pool.h"
+
+#include <stdbool.h>
+#include <stddef.h>
+#include <string.h>
+
+#include "experimental/hip/dynamic_symbols.h"
+#include "experimental/hip/status_util.h"
+#include "iree/base/api.h"
+#include "iree/base/internal/atomics.h"
+#include "iree/base/internal/synchronization.h"
+#include "iree/hal/api.h"
+
+//===----------------------------------------------------------------------===//
+// iree_hal_hip_event_t
+//===----------------------------------------------------------------------===//
+
+struct iree_hal_hip_event_t {
+  // A reference count used to manage resource lifetime. Its value range:
+  // * 1 - when inside the event pool and to be acquired;
+  // * >= 1 - when acquired outside of the event pool;
+  // * 0 - when before releasing back to the pool or destruction.
+  iree_atomic_ref_count_t ref_count;
+
+  // The allocator used to create the event.
+  iree_allocator_t host_allocator;
+  // The symbols used to create and destroy hipEvent_t objects.
+  const iree_hal_hip_dynamic_symbols_t* symbols;
+
+  // The event pool that owns this event. This cannot be NULL. We retain it to
+  // make sure the event outlive the pool.
+  iree_hal_hip_event_pool_t* pool;
+  // The underlying hipEvent_t object.
+  hipEvent_t hip_event;
+};
+
+hipEvent_t iree_hal_hip_event_handle(const iree_hal_hip_event_t* event) {
+  return event->hip_event;
+}
+
+static inline void iree_hal_hip_event_destroy(iree_hal_hip_event_t* event) {
+  iree_allocator_t host_allocator = event->host_allocator;
+  const iree_hal_hip_dynamic_symbols_t* symbols = event->symbols;
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  IREE_ASSERT_REF_COUNT_ZERO(&event->ref_count);
+  IREE_HIP_IGNORE_ERROR(symbols, hipEventDestroy(event->hip_event));
+  iree_allocator_free(host_allocator, event);
+
+  IREE_TRACE_ZONE_END(z0);
+}
+
+static inline iree_status_t iree_hal_hip_event_create(
+    const iree_hal_hip_dynamic_symbols_t* symbols,
+    iree_hal_hip_event_pool_t* pool, iree_allocator_t host_allocator,
+    iree_hal_hip_event_t** out_event) {
+  IREE_ASSERT_ARGUMENT(symbols);
+  IREE_ASSERT_ARGUMENT(pool);
+  IREE_ASSERT_ARGUMENT(out_event);
+  *out_event = NULL;
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  iree_hal_hip_event_t* event = NULL;
+  IREE_RETURN_AND_END_ZONE_IF_ERROR(
+      z0,
+      iree_allocator_malloc(host_allocator, sizeof(*event), (void**)&event));
+  iree_atomic_ref_count_init(&event->ref_count);  // -> 1
+  event->host_allocator = host_allocator;
+  event->symbols = symbols;
+  event->pool = pool;
+  event->hip_event = NULL;
+
+  iree_status_t status = IREE_HIP_RESULT_TO_STATUS(
+      symbols,
+      hipEventCreateWithFlags(&event->hip_event, hipEventDisableTiming),
+      "hipEventCreateWithFlags");
+  if (iree_status_is_ok(status)) {
+    *out_event = event;
+  } else {
+    iree_atomic_ref_count_dec(&event->ref_count);  // -> 0
+    iree_hal_hip_event_destroy(event);
+  }
+
+  IREE_TRACE_ZONE_END(z0);
+  return status;
+}
+
+void iree_hal_hip_event_retain(iree_hal_hip_event_t* event) {
+  iree_atomic_ref_count_inc(&event->ref_count);
+}
+
+static void iree_hal_hip_event_pool_release_event(
+    iree_hal_hip_event_pool_t* event_pool, iree_host_size_t event_count,
+    iree_hal_hip_event_t** events);
+
+void iree_hal_hip_event_release(iree_hal_hip_event_t* event) {
+  if (iree_atomic_ref_count_dec(&event->ref_count) == 1) {
+    iree_hal_hip_event_pool_t* pool = event->pool;
+    // Release back to the pool if the reference count becomes 0.
+    iree_hal_hip_event_pool_release_event(pool, 1, &event);
+    // Drop our reference to the pool itself when we return event to it.
+    iree_hal_hip_event_pool_release(pool);  // -1
+  }
+}
+
+//===----------------------------------------------------------------------===//
+// iree_hal_hip_event_pool_t
+//===----------------------------------------------------------------------===//
+
+struct iree_hal_hip_event_pool_t {
+  // A reference count used to manage resource lifetime.
+  iree_atomic_ref_count_t ref_count;
+
+  // The allocator used to create the event pool.
+  iree_allocator_t host_allocator;
+  // The symbols used to create and destroy hipEvent_t objects.
+  const iree_hal_hip_dynamic_symbols_t* symbols;
+
+  // Guards event related fields in the pool. We don't expect a performant
+  // program to frequently allocate events for synchronization purposes; the
+  // traffic to this pool should be low. So it should be fine to use mutex to
+  // guard here.
+  iree_slim_mutex_t event_mutex;
+
+  // Maximum number of event objects that will be maintained in the pool.
+  // More events may be allocated at any time, but they will be disposed
+  // directly when they are no longer needed.
+  iree_host_size_t available_capacity IREE_GUARDED_BY(event_mutex);
+  // Total number of currently available event objects.
+  iree_host_size_t available_count IREE_GUARDED_BY(event_mutex);
+  // The list of available_count event objects.
+  iree_hal_hip_event_t* available_list[] IREE_GUARDED_BY(event_mutex);
+};
+// + Additional inline allocation for holding events up to the capacity.
+
+static void iree_hal_hip_event_pool_free(iree_hal_hip_event_pool_t* event_pool);
+
+iree_status_t iree_hal_hip_event_pool_allocate(
+    const iree_hal_hip_dynamic_symbols_t* symbols,
+    iree_host_size_t available_capacity, iree_allocator_t host_allocator,
+    iree_hal_hip_event_pool_t** out_event_pool) {
+  IREE_ASSERT_ARGUMENT(symbols);
+  IREE_ASSERT_ARGUMENT(out_event_pool);
+  *out_event_pool = NULL;
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  iree_hal_hip_event_pool_t* event_pool = NULL;
+  iree_host_size_t total_size =
+      sizeof(*event_pool) +
+      available_capacity * sizeof(*event_pool->available_list);
+  IREE_RETURN_AND_END_ZONE_IF_ERROR(
+      z0,
+      iree_allocator_malloc(host_allocator, total_size, (void**)&event_pool));
+  iree_atomic_ref_count_init(&event_pool->ref_count);  // -> 1
+  event_pool->host_allocator = host_allocator;
+  event_pool->symbols = symbols;
+  iree_slim_mutex_initialize(&event_pool->event_mutex);
+  event_pool->available_capacity = available_capacity;
+  event_pool->available_count = 0;
+
+  iree_status_t status = iree_ok_status();
+  for (iree_host_size_t i = 0; i < available_capacity; ++i) {
+    status = iree_hal_hip_event_create(
+        symbols, event_pool, host_allocator,
+        &event_pool->available_list[event_pool->available_count++]);
+    if (!iree_status_is_ok(status)) break;
+  }
+
+  if (iree_status_is_ok(status)) {
+    *out_event_pool = event_pool;
+  } else {
+    iree_hal_hip_event_pool_free(event_pool);
+  }
+  IREE_TRACE_ZONE_END(z0);
+  return status;
+}
+
+static void iree_hal_hip_event_pool_free(
+    iree_hal_hip_event_pool_t* event_pool) {
+  iree_allocator_t host_allocator = event_pool->host_allocator;
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  for (iree_host_size_t i = 0; i < event_pool->available_count; ++i) {
+    iree_hal_hip_event_t* event = event_pool->available_list[i];
+    iree_atomic_ref_count_dec(&event->ref_count);  // -> 0
+    iree_hal_hip_event_destroy(event);
+  }
+  IREE_ASSERT_REF_COUNT_ZERO(&event_pool->ref_count);
+
+  iree_slim_mutex_deinitialize(&event_pool->event_mutex);
+  iree_allocator_free(host_allocator, event_pool);
+
+  IREE_TRACE_ZONE_END(z0);
+}
+
+void iree_hal_hip_event_pool_retain(iree_hal_hip_event_pool_t* event_pool) {
+  iree_atomic_ref_count_inc(&event_pool->ref_count);
+}
+
+void iree_hal_hip_event_pool_release(iree_hal_hip_event_pool_t* event_pool) {
+  if (iree_atomic_ref_count_dec(&event_pool->ref_count) == 1) {
+    iree_hal_hip_event_pool_free(event_pool);
+  }
+}
+
+iree_status_t iree_hal_hip_event_pool_acquire(
+    iree_hal_hip_event_pool_t* event_pool, iree_host_size_t event_count,
+    iree_hal_hip_event_t** out_events) {
+  IREE_ASSERT_ARGUMENT(event_pool);
+  if (!event_count) return iree_ok_status();
+  IREE_ASSERT_ARGUMENT(out_events);
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  // We'll try to get what we can from the pool and fall back to initializing
+  // new iree_hal_hip_event_t objects.
+  iree_host_size_t remaining_count = event_count;
+
+  // Try first to grab from the pool.
+  iree_slim_mutex_lock(&event_pool->event_mutex);
+  iree_host_size_t from_pool_count =
+      iree_min(event_pool->available_count, event_count);
+  if (from_pool_count > 0) {
+    iree_host_size_t pool_base_index =
+        event_pool->available_count - from_pool_count;
+    memcpy(out_events, &event_pool->available_list[pool_base_index],
+           from_pool_count * sizeof(*event_pool->available_list));
+    event_pool->available_count -= from_pool_count;
+    remaining_count -= from_pool_count;
+  }
+  iree_slim_mutex_unlock(&event_pool->event_mutex);
+
+  // Allocate the rest of the events.
+  if (remaining_count > 0) {
+    IREE_TRACE_ZONE_BEGIN_NAMED(z1, "event-pool-unpooled-acquire");
+    iree_status_t status = iree_ok_status();
+    for (iree_host_size_t i = 0; i < remaining_count; ++i) {
+      status = iree_hal_hip_event_create(event_pool->symbols, event_pool,
+                                         event_pool->host_allocator,
+                                         &out_events[from_pool_count + i]);
+      if (!iree_status_is_ok(status)) {
+        // Must release all events we've acquired so far.
+        iree_hal_hip_event_pool_release_event(event_pool, from_pool_count + i,
+                                              out_events);
+        IREE_TRACE_ZONE_END(z1);
+        IREE_TRACE_ZONE_END(z0);
+        return status;
+      }
+    }
+    IREE_TRACE_ZONE_END(z1);
+  }
+
+  // Retain a reference to a pool when we pass event to the caller. When the
+  // caller returns event back to the pool they'll release the reference.
+  for (iree_host_size_t i = 0; i < event_count; ++i) {
+    iree_hal_hip_event_pool_retain(out_events[i]->pool);  // +1
+  }
+
+  IREE_TRACE_ZONE_END(z0);
+  return iree_ok_status();
+}
+
+static void iree_hal_hip_event_pool_release_event(
+    iree_hal_hip_event_pool_t* event_pool, iree_host_size_t event_count,
+    iree_hal_hip_event_t** events) {
+  IREE_ASSERT_ARGUMENT(event_pool);
+  if (!event_count) return;
+  IREE_ASSERT_ARGUMENT(events);
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  // We'll try to release all we can back to the pool and then deinitialize
+  // the ones that won't fit.
+  iree_host_size_t remaining_count = event_count;
+
+  // Try first to release to the pool.
+  iree_slim_mutex_lock(&event_pool->event_mutex);
+  iree_host_size_t to_pool_count =
+      iree_min(event_pool->available_capacity - event_pool->available_count,
+               event_count);
+  if (to_pool_count > 0) {
+    for (iree_host_size_t i = 0; i < to_pool_count; ++i) {
+      IREE_ASSERT_REF_COUNT_ZERO(&events[i]->ref_count);
+      iree_hal_hip_event_retain(events[i]);  // -> 1
+    }
+    iree_host_size_t pool_base_index = event_pool->available_count;
+    memcpy(&event_pool->available_list[pool_base_index], events,
+           to_pool_count * sizeof(*event_pool->available_list));
+    event_pool->available_count += to_pool_count;
+    remaining_count -= to_pool_count;
+  }
+  iree_slim_mutex_unlock(&event_pool->event_mutex);
+
+  // Deallocate the rest of the events. We don't bother resetting them as we are
+  // getting rid of them.
+  if (remaining_count > 0) {
+    IREE_TRACE_ZONE_BEGIN_NAMED(z1, "event-pool-unpooled-release");
+    for (iree_host_size_t i = 0; i < remaining_count; ++i) {
+      iree_hal_hip_event_destroy(events[to_pool_count + i]);
+    }
+    IREE_TRACE_ZONE_END(z1);
+  }
+  IREE_TRACE_ZONE_END(z0);
+}
diff --git a/experimental/hip/event_pool.h b/experimental/hip/event_pool.h
new file mode 100644
index 0000000..682fbac
--- /dev/null
+++ b/experimental/hip/event_pool.h
@@ -0,0 +1,80 @@
+// Copyright 2024 The IREE Authors
+//
+// Licensed under the Apache License v2.0 with LLVM Exceptions.
+// See https://llvm.org/LICENSE.txt for license information.
+// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
+
+#ifndef IREE_EXPERIMENTAL_HIP_EVENT_POOL_H_
+#define IREE_EXPERIMENTAL_HIP_EVENT_POOL_H_
+
+#include "experimental/hip/dynamic_symbols.h"
+#include "iree/base/api.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif  // __cplusplus
+
+//===----------------------------------------------------------------------===//
+// iree_hal_hip_event_t
+//===----------------------------------------------------------------------===//
+
+// An struct that wraps a hipEvent_t object with a reference count for lifetime
+// management.
+//
+// iree_hal_hip_event_t objects cannot be directly created; they should be
+// acquired from the event pool and released back to the pool once done.
+//
+// Thread-safe; multiple threads may retain and release the same event.
+typedef struct iree_hal_hip_event_t iree_hal_hip_event_t;
+
+// Returns the underlying hipEvent_t handle behind |event|.
+hipEvent_t iree_hal_hip_event_handle(const iree_hal_hip_event_t* event);
+
+// Retains the given |event| by increasing its reference count.
+void iree_hal_hip_event_retain(iree_hal_hip_event_t* event);
+
+// Releases the given |event| by decreasing its reference count.
+//
+// |event| will be returned to its owning pool when the reference count is 0.
+void iree_hal_hip_event_release(iree_hal_hip_event_t* event);
+
+//===----------------------------------------------------------------------===//
+// iree_hal_hip_event_pool_t
+//===----------------------------------------------------------------------===//
+
+// A simple pool of iree_hal_event_t objects to recycle.
+//
+// Thread-safe; multiple threads may acquire and release events from the pool.
+typedef struct iree_hal_hip_event_pool_t iree_hal_hip_event_pool_t;
+
+// Allocates a new event pool with up to |available_capacity| events.
+//
+// Extra events requested beyond the capability are directly created and
+// destroyed without pooling.
+iree_status_t iree_hal_hip_event_pool_allocate(
+    const iree_hal_hip_dynamic_symbols_t* symbols,
+    iree_host_size_t available_capacity, iree_allocator_t host_allocator,
+    iree_hal_hip_event_pool_t** out_event_pool);
+
+// Retains the given |event_pool| by increasing its reference count.
+void iree_hal_hip_event_pool_retain(iree_hal_hip_event_pool_t* event_pool);
+
+// Releases the given |event_pool| by decreasing its reference count.
+//
+// Once the |event_pool|'s reference count becomes zero, it will be freed.
+void iree_hal_hip_event_pool_release(iree_hal_hip_event_pool_t* event_pool);
+
+// Acquires one or more events from the event pool.
+//
+// Each returned event have an initial reference count of 1. The returned
+// hipEvent_t objects may retain captured states of some queues from previous
+// uses; callers should record again to overwrite.
+iree_status_t iree_hal_hip_event_pool_acquire(
+    iree_hal_hip_event_pool_t* event_pool, iree_host_size_t event_count,
+    iree_hal_hip_event_t** out_events);
+
+#ifdef __cplusplus
+}  // extern "C"
+#endif  // __cplusplus
+
+#endif  // IREE_EXPERIMENTAL_HIP_EVENT_POOL_H_
diff --git a/experimental/hip/event_semaphore.c b/experimental/hip/event_semaphore.c
new file mode 100644
index 0000000..9a294b2
--- /dev/null
+++ b/experimental/hip/event_semaphore.c
@@ -0,0 +1,449 @@
+// Copyright 2024 The IREE Authors
+//
+// Licensed under the Apache License v2.0 with LLVM Exceptions.
+// See https://llvm.org/LICENSE.txt for license information.
+// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
+
+#include "experimental/hip/event_semaphore.h"
+
+#include "experimental/hip/dynamic_symbols.h"
+#include "experimental/hip/status_util.h"
+#include "experimental/hip/timepoint_pool.h"
+#include "iree/base/internal/synchronization.h"
+#include "iree/hal/utils/semaphore_base.h"
+
+typedef struct iree_hal_hip_semaphore_t {
+  // Abstract resource used for injecting reference counting and vtable;
+  // must be at offset 0.
+  iree_hal_semaphore_t base;
+
+  // The allocator used to create this semaphore.
+  iree_allocator_t host_allocator;
+  // The symbols used to issue HIP API calls.
+  const iree_hal_hip_dynamic_symbols_t* symbols;
+
+  // The timepoint pool to acquire timepoint objects.
+  iree_hal_hip_timepoint_pool_t* timepoint_pool;
+
+  // The list of pending queue actions that this semaphore need to advance on
+  // new signaled values.
+  iree_hal_hip_pending_queue_actions_t* pending_queue_actions;
+
+  // Guards value and status. We expect low contention on semaphores and since
+  // iree_slim_mutex_t is (effectively) just a CAS this keeps things simpler
+  // than trying to make the entire structure lock-free.
+  iree_slim_mutex_t mutex;
+
+  // Current signaled value. May be IREE_HAL_SEMAPHORE_FAILURE_VALUE to
+  // indicate that the semaphore has been signaled for failure and
+  // |failure_status| contains the error.
+  uint64_t current_value IREE_GUARDED_BY(mutex);
+
+  // OK or the status passed to iree_hal_semaphore_fail. Owned by the semaphore.
+  iree_status_t failure_status IREE_GUARDED_BY(mutex);
+} iree_hal_hip_semaphore_t;
+
+static const iree_hal_semaphore_vtable_t iree_hal_hip_semaphore_vtable;
+
+static iree_hal_hip_semaphore_t* iree_hal_hip_semaphore_cast(
+    iree_hal_semaphore_t* base_value) {
+  IREE_HAL_ASSERT_TYPE(base_value, &iree_hal_hip_semaphore_vtable);
+  return (iree_hal_hip_semaphore_t*)base_value;
+}
+
+iree_status_t iree_hal_hip_event_semaphore_create(
+    uint64_t initial_value, const iree_hal_hip_dynamic_symbols_t* symbols,
+    iree_hal_hip_timepoint_pool_t* timepoint_pool,
+    iree_hal_hip_pending_queue_actions_t* pending_queue_actions,
+    iree_allocator_t host_allocator, iree_hal_semaphore_t** out_semaphore) {
+  IREE_ASSERT_ARGUMENT(symbols);
+  IREE_ASSERT_ARGUMENT(timepoint_pool);
+  IREE_ASSERT_ARGUMENT(pending_queue_actions);
+  IREE_ASSERT_ARGUMENT(out_semaphore);
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  iree_hal_hip_semaphore_t* semaphore = NULL;
+  IREE_RETURN_AND_END_ZONE_IF_ERROR(
+      z0, iree_allocator_malloc(host_allocator, sizeof(*semaphore),
+                                (void**)&semaphore));
+
+  iree_hal_semaphore_initialize(&iree_hal_hip_semaphore_vtable,
+                                &semaphore->base);
+  semaphore->host_allocator = host_allocator;
+  semaphore->symbols = symbols;
+  semaphore->timepoint_pool = timepoint_pool;
+  semaphore->pending_queue_actions = pending_queue_actions;
+  iree_slim_mutex_initialize(&semaphore->mutex);
+  semaphore->current_value = initial_value;
+  semaphore->failure_status = iree_ok_status();
+
+  *out_semaphore = &semaphore->base;
+
+  IREE_TRACE_ZONE_END(z0);
+  return iree_ok_status();
+}
+
+static void iree_hal_hip_semaphore_destroy(
+    iree_hal_semaphore_t* base_semaphore) {
+  iree_hal_hip_semaphore_t* semaphore =
+      iree_hal_hip_semaphore_cast(base_semaphore);
+  iree_allocator_t host_allocator = semaphore->host_allocator;
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  iree_status_ignore(semaphore->failure_status);
+  iree_slim_mutex_deinitialize(&semaphore->mutex);
+
+  iree_hal_semaphore_deinitialize(&semaphore->base);
+  iree_allocator_free(host_allocator, semaphore);
+
+  IREE_TRACE_ZONE_END(z0);
+}
+
+static iree_status_t iree_hal_hip_semaphore_query(
+    iree_hal_semaphore_t* base_semaphore, uint64_t* out_value) {
+  iree_hal_hip_semaphore_t* semaphore =
+      iree_hal_hip_semaphore_cast(base_semaphore);
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  iree_slim_mutex_lock(&semaphore->mutex);
+
+  *out_value = semaphore->current_value;
+
+  iree_status_t status = iree_ok_status();
+  if (*out_value >= IREE_HAL_SEMAPHORE_FAILURE_VALUE) {
+    status = iree_status_clone(semaphore->failure_status);
+  }
+
+  iree_slim_mutex_unlock(&semaphore->mutex);
+
+  IREE_TRACE_ZONE_END(z0);
+  return status;
+}
+
+static iree_status_t iree_hal_hip_semaphore_signal(
+    iree_hal_semaphore_t* base_semaphore, uint64_t new_value) {
+  iree_hal_hip_semaphore_t* semaphore =
+      iree_hal_hip_semaphore_cast(base_semaphore);
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  iree_slim_mutex_lock(&semaphore->mutex);
+
+  if (new_value <= semaphore->current_value) {
+    uint64_t current_value IREE_ATTRIBUTE_UNUSED = semaphore->current_value;
+    iree_slim_mutex_unlock(&semaphore->mutex);
+    IREE_TRACE_ZONE_END(z0);
+    return iree_make_status(IREE_STATUS_OUT_OF_RANGE,
+                            "semaphore values must be monotonically "
+                            "increasing; current_value=%" PRIu64
+                            ", new_value=%" PRIu64,
+                            current_value, new_value);
+  }
+
+  semaphore->current_value = new_value;
+
+  iree_slim_mutex_unlock(&semaphore->mutex);
+
+  // Notify timepoints - note that this must happen outside the lock.
+  iree_hal_semaphore_notify(&semaphore->base, new_value, IREE_STATUS_OK);
+
+  // Advance the pending queue actions if possible. This also must happen
+  // outside the lock to avoid nesting.
+  iree_status_t status = iree_hal_hip_pending_queue_actions_issue(
+      semaphore->pending_queue_actions);
+
+  IREE_TRACE_ZONE_END(z0);
+  return status;
+}
+
+static void iree_hal_hip_semaphore_fail(iree_hal_semaphore_t* base_semaphore,
+                                        iree_status_t status) {
+  iree_hal_hip_semaphore_t* semaphore =
+      iree_hal_hip_semaphore_cast(base_semaphore);
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  const iree_status_code_t status_code = iree_status_code(status);
+
+  iree_slim_mutex_lock(&semaphore->mutex);
+
+  // Try to set our local status - we only preserve the first failure so only
+  // do this if we are going from a valid semaphore to a failed one.
+  if (!iree_status_is_ok(semaphore->failure_status)) {
+    // Previous status was not OK; drop our new status.
+    IREE_IGNORE_ERROR(status);
+    iree_slim_mutex_unlock(&semaphore->mutex);
+    IREE_TRACE_ZONE_END(z0);
+    return;
+  }
+
+  // Signal to our failure sentinel value.
+  semaphore->current_value = IREE_HAL_SEMAPHORE_FAILURE_VALUE;
+  semaphore->failure_status = status;
+
+  iree_slim_mutex_unlock(&semaphore->mutex);
+
+  // Notify timepoints - note that this must happen outside the lock.
+  iree_hal_semaphore_notify(&semaphore->base, IREE_HAL_SEMAPHORE_FAILURE_VALUE,
+                            status_code);
+  IREE_TRACE_ZONE_END(z0);
+}
+
+// Handles host wait timepoints on the host when the |semaphore| timeline
+// advances past the given |value|.
+//
+// Note that this callback is invoked by the a host thread.
+static iree_status_t iree_hal_hip_semaphore_timepoint_host_wait_callback(
+    void* user_data, iree_hal_semaphore_t* semaphore, uint64_t value,
+    iree_status_code_t status_code) {
+  IREE_TRACE_ZONE_BEGIN(z0);
+  iree_hal_hip_timepoint_t* timepoint = (iree_hal_hip_timepoint_t*)user_data;
+  iree_event_set(&timepoint->timepoint.host_wait);
+  IREE_TRACE_ZONE_END(z0);
+  return iree_ok_status();
+}
+
+// Acquires a timepoint to wait the timeline to reach at least the given
+// |min_value| from the host.
+static iree_status_t iree_hal_hip_semaphore_acquire_timepoint_host_wait(
+    iree_hal_hip_semaphore_t* semaphore, uint64_t min_value,
+    iree_timeout_t timeout, iree_hal_hip_timepoint_t** out_timepoint) {
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  IREE_RETURN_AND_END_ZONE_IF_ERROR(
+      z0, iree_hal_hip_timepoint_pool_acquire_host_wait(
+              semaphore->timepoint_pool, 1, out_timepoint));
+  // Initialize the timepoint with the value and callback, and connect it to
+  // this semaphore.
+  iree_hal_semaphore_acquire_timepoint(
+      &semaphore->base, min_value, timeout,
+      (iree_hal_semaphore_callback_t){
+          .fn = iree_hal_hip_semaphore_timepoint_host_wait_callback,
+          .user_data = *out_timepoint,
+      },
+      &(*out_timepoint)->base);
+
+  IREE_TRACE_ZONE_END(z0);
+  return iree_ok_status();
+}
+
+// Acquires an iree_hal_hip_event_t object to wait on the host for the
+// timeline to reach at least the given |min_value| on the device.
+// Returns true and writes to |out_event| if we can find such an event;
+// returns false otherwise.
+// The caller should release the |out_event| once done.
+static bool iree_hal_hip_semaphore_acquire_event_host_wait(
+    iree_hal_hip_semaphore_t* semaphore, uint64_t min_value,
+    iree_hal_hip_event_t** out_event) {
+  *out_event = NULL;
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  // Scan through the timepoint list and try to find a device event signal to
+  // wait on. We need to lock with the timepoint list mutex here.
+  iree_slim_mutex_lock(&semaphore->base.timepoint_mutex);
+  for (iree_hal_semaphore_timepoint_t* tp = semaphore->base.timepoint_list.head;
+       tp != NULL; tp = tp->next) {
+    iree_hal_hip_timepoint_t* signal_timepoint = (iree_hal_hip_timepoint_t*)tp;
+    if (signal_timepoint->kind == IREE_HAL_HIP_TIMEPOINT_KIND_DEVICE_SIGNAL &&
+        signal_timepoint->base.minimum_value >= min_value) {
+      *out_event = signal_timepoint->timepoint.device_signal;
+      iree_hal_hip_event_retain(*out_event);
+      break;
+    }
+  }
+  iree_slim_mutex_unlock(&semaphore->base.timepoint_mutex);
+
+  IREE_TRACE_ZONE_END(z0);
+  return *out_event != NULL;
+}
+
+static iree_status_t iree_hal_hip_semaphore_wait(
+    iree_hal_semaphore_t* base_semaphore, uint64_t value,
+    iree_timeout_t timeout) {
+  iree_hal_hip_semaphore_t* semaphore =
+      iree_hal_hip_semaphore_cast(base_semaphore);
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  iree_slim_mutex_lock(&semaphore->mutex);
+  if (!iree_status_is_ok(semaphore->failure_status)) {
+    // Fastest path: failed; return an error to tell callers to query for it.
+    iree_slim_mutex_unlock(&semaphore->mutex);
+    IREE_TRACE_ZONE_END(z0);
+    return iree_status_from_code(IREE_STATUS_ABORTED);
+  }
+  if (semaphore->current_value >= value) {
+    // Fast path: already satisfied.
+    iree_slim_mutex_unlock(&semaphore->mutex);
+    IREE_TRACE_ZONE_END(z0);
+    return iree_ok_status();
+  }
+  if (iree_timeout_is_immediate(timeout)) {
+    // Not satisfied but a poll, so can avoid the expensive wait handle work.
+    iree_slim_mutex_unlock(&semaphore->mutex);
+    IREE_TRACE_ZONE_END(z0);
+    return iree_status_from_code(IREE_STATUS_DEADLINE_EXCEEDED);
+  }
+  iree_slim_mutex_unlock(&semaphore->mutex);
+
+  iree_time_t deadline_ns = iree_timeout_as_deadline_ns(timeout);
+
+  // Slow path: try to see if we can have a device hipEvent_t to wait on. This
+  // should happen outside of the lock given that acquiring has its own internal
+  // locks. This is faster than waiting on a host timepoint.
+  iree_hal_hip_event_t* wait_event = NULL;
+  if (iree_hal_hip_semaphore_acquire_event_host_wait(semaphore, value,
+                                                     &wait_event)) {
+    IREE_HIP_RETURN_AND_END_ZONE_IF_ERROR(
+        z0, semaphore->symbols,
+        hipEventSynchronize(iree_hal_hip_event_handle(wait_event)),
+        "hipEventSynchronize");
+    iree_hal_hip_event_release(wait_event);
+    IREE_TRACE_ZONE_END(z0);
+    return iree_ok_status();
+  }
+
+  // Slow path: acquire a timepoint. This should happen outside of the lock to
+  // given that acquiring has its own internal locks.
+  iree_hal_hip_timepoint_t* timepoint = NULL;
+  iree_status_t status = iree_hal_hip_semaphore_acquire_timepoint_host_wait(
+      semaphore, value, timeout, &timepoint);
+  if (IREE_UNLIKELY(!iree_status_is_ok(status))) {
+    IREE_TRACE_ZONE_END(z0);
+    return status;
+  }
+
+  // Wait until the timepoint resolves.
+  // If satisfied the timepoint is automatically cleaned up and we are done. If
+  // the deadline is reached before satisfied then we have to clean it up.
+  status = iree_wait_one(&timepoint->timepoint.host_wait, deadline_ns);
+  if (!iree_status_is_ok(status)) {
+    iree_hal_semaphore_cancel_timepoint(&semaphore->base, &timepoint->base);
+  }
+  iree_hal_hip_timepoint_pool_release(semaphore->timepoint_pool, 1, &timepoint);
+  IREE_TRACE_ZONE_END(z0);
+  return status;
+}
+
+// Handles device signal timepoints on the host when the |semaphore| timeline
+// advances past the given |value|.
+//
+// Note that this callback is invoked by the a host thread after the HIP host
+// function callback function is triggered in the HIP driver.
+static iree_status_t iree_hal_hip_semaphore_timepoint_device_signal_callback(
+    void* user_data, iree_hal_semaphore_t* semaphore, uint64_t value,
+    iree_status_code_t status_code) {
+  IREE_TRACE_ZONE_BEGIN(z0);
+  iree_hal_hip_timepoint_t* timepoint = (iree_hal_hip_timepoint_t*)user_data;
+  // Just release the timepoint back to the pool. This will decrease the
+  // reference count of the underlying HIP event internally.
+  iree_hal_hip_timepoint_pool_release(timepoint->pool, 1, &timepoint);
+  IREE_TRACE_ZONE_END(z0);
+  return iree_ok_status();
+}
+
+// Acquires a timepoint to signal the timeline to the given |to_value| from the
+// device.
+iree_status_t iree_hal_hip_event_semaphore_acquire_timepoint_device_signal(
+    iree_hal_semaphore_t* base_semaphore, uint64_t to_value,
+    hipEvent_t* out_event) {
+  iree_hal_hip_semaphore_t* semaphore =
+      iree_hal_hip_semaphore_cast(base_semaphore);
+  iree_hal_hip_timepoint_t* signal_timepoint = NULL;
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  IREE_RETURN_AND_END_ZONE_IF_ERROR(
+      z0, iree_hal_hip_timepoint_pool_acquire_device_signal(
+              semaphore->timepoint_pool, 1, &signal_timepoint));
+
+  // Initialize the timepoint with the value and callback, and connect it to
+  // this semaphore.
+  iree_hal_semaphore_acquire_timepoint(
+      &semaphore->base, to_value, iree_infinite_timeout(),
+      (iree_hal_semaphore_callback_t){
+          .fn = iree_hal_hip_semaphore_timepoint_device_signal_callback,
+          .user_data = signal_timepoint,
+      },
+      &signal_timepoint->base);
+  iree_hal_hip_event_t* event = signal_timepoint->timepoint.device_signal;
+
+  // Scan through the timepoint list and update device wait timepoints to wait
+  // for this device signal when possible. We need to lock with the timepoint
+  // list mutex here.
+  iree_slim_mutex_lock(&semaphore->base.timepoint_mutex);
+  for (iree_hal_semaphore_timepoint_t* tp = semaphore->base.timepoint_list.head;
+       tp != NULL; tp = tp->next) {
+    iree_hal_hip_timepoint_t* wait_timepoint = (iree_hal_hip_timepoint_t*)tp;
+    if (wait_timepoint->kind == IREE_HAL_HIP_TIMEPOINT_KIND_DEVICE_WAIT &&
+        wait_timepoint->timepoint.device_wait == NULL &&
+        wait_timepoint->base.minimum_value <= to_value) {
+      iree_hal_hip_event_retain(event);
+      wait_timepoint->timepoint.device_wait = event;
+    }
+  }
+  iree_slim_mutex_unlock(&semaphore->base.timepoint_mutex);
+
+  *out_event = iree_hal_hip_event_handle(event);
+  IREE_TRACE_ZONE_END(z0);
+  return iree_ok_status();
+}
+
+// Handles device wait timepoints on the host when the |semaphore| timeline
+// advances past the given |value|.
+//
+// Note that this callback is invoked by the a host thread.
+static iree_status_t iree_hal_hip_semaphore_timepoint_device_wait_callback(
+    void* user_data, iree_hal_semaphore_t* semaphore, uint64_t value,
+    iree_status_code_t status_code) {
+  IREE_TRACE_ZONE_BEGIN(z0);
+  iree_hal_hip_timepoint_t* timepoint = (iree_hal_hip_timepoint_t*)user_data;
+  // Just release the timepoint back to the pool. This will decrease the
+  // reference count of the underlying HIP event internally.
+  iree_hal_hip_timepoint_pool_release(timepoint->pool, 1, &timepoint);
+  IREE_TRACE_ZONE_END(z0);
+  return iree_ok_status();
+}
+
+// Acquires a timepoint to wait the timeline to reach at least the given
+// |min_value| on the device.
+iree_status_t iree_hal_hip_event_semaphore_acquire_timepoint_device_wait(
+    iree_hal_semaphore_t* base_semaphore, uint64_t min_value,
+    hipEvent_t* out_event) {
+  iree_hal_hip_semaphore_t* semaphore =
+      iree_hal_hip_semaphore_cast(base_semaphore);
+  iree_hal_hip_timepoint_t* wait_timepoint = NULL;
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  IREE_RETURN_AND_END_ZONE_IF_ERROR(
+      z0, iree_hal_hip_timepoint_pool_acquire_device_wait(
+              semaphore->timepoint_pool, 1, &wait_timepoint));
+
+  // Initialize the timepoint with the value and callback, and connect it to
+  // this semaphore.
+  iree_hal_semaphore_acquire_timepoint(
+      &semaphore->base, min_value, iree_infinite_timeout(),
+      (iree_hal_semaphore_callback_t){
+          .fn = iree_hal_hip_semaphore_timepoint_device_wait_callback,
+          .user_data = wait_timepoint,
+      },
+      &wait_timepoint->base);
+
+  iree_hal_hip_event_t* wait_event = NULL;
+  if (iree_hal_hip_semaphore_acquire_event_host_wait(semaphore, min_value,
+                                                     &wait_event)) {
+    // We've found an existing signal timepoint to wait on; we don't need a
+    // standalone wait timepoint anymore. Decrease its refcount before
+    // overwriting it to return it back to the pool and retain the existing one.
+    iree_hal_hip_event_release(wait_timepoint->timepoint.device_wait);
+    wait_timepoint->timepoint.device_wait = wait_event;
+  }
+
+  *out_event = iree_hal_hip_event_handle(wait_timepoint->timepoint.device_wait);
+  IREE_TRACE_ZONE_END(z0);
+  return iree_ok_status();
+}
+
+static const iree_hal_semaphore_vtable_t iree_hal_hip_semaphore_vtable = {
+    .destroy = iree_hal_hip_semaphore_destroy,
+    .query = iree_hal_hip_semaphore_query,
+    .signal = iree_hal_hip_semaphore_signal,
+    .fail = iree_hal_hip_semaphore_fail,
+    .wait = iree_hal_hip_semaphore_wait,
+};
diff --git a/experimental/hip/event_semaphore.h b/experimental/hip/event_semaphore.h
new file mode 100644
index 0000000..44860f2
--- /dev/null
+++ b/experimental/hip/event_semaphore.h
@@ -0,0 +1,56 @@
+// Copyright 2024 The IREE Authors
+//
+// Licensed under the Apache License v2.0 with LLVM Exceptions.
+// See https://llvm.org/LICENSE.txt for license information.
+// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
+
+#ifndef IREE_EXPERIMENTAL_HIP_EVENT_SEMAPHORE_H_
+#define IREE_EXPERIMENTAL_HIP_EVENT_SEMAPHORE_H_
+
+#include <stdint.h>
+
+#include "experimental/hip/dynamic_symbols.h"
+#include "experimental/hip/pending_queue_actions.h"
+#include "experimental/hip/timepoint_pool.h"
+#include "iree/base/api.h"
+#include "iree/hal/api.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif  // __cplusplus
+
+// Creates an IREE HAL semaphore with the given |initial_value|.
+//
+// The HAL semaphore are backed by iree_event_t or hipEvent_t objects for
+// different timepoints along the timeline under the hood. Those timepoints will
+// be allocated from the |timepoint_pool|.
+//
+// This semaphore is meant to be used together with a pending queue actions; it
+// may advance the given |pending_queue_actions| if new values are signaled.
+//
+// Thread-safe; multiple threads may signal/wait values on the same semaphore.
+iree_status_t iree_hal_hip_event_semaphore_create(
+    uint64_t initial_value, const iree_hal_hip_dynamic_symbols_t* symbols,
+    iree_hal_hip_timepoint_pool_t* timepoint_pool,
+    iree_hal_hip_pending_queue_actions_t* pending_queue_actions,
+    iree_allocator_t host_allocator, iree_hal_semaphore_t** out_semaphore);
+
+// Acquires a timepoint to signal the timeline to the given |to_value| from the
+// device. The underlying HIP event is written into |out_event| for interacting
+// with HIP APIs.
+iree_status_t iree_hal_hip_event_semaphore_acquire_timepoint_device_signal(
+    iree_hal_semaphore_t* base_semaphore, uint64_t to_value,
+    hipEvent_t* out_event);
+
+// Acquires a timepoint to wait the timeline to reach at least the given
+// |min_value| on the device. The underlying HIP event is written into
+// |out_event| for interacting with HIP APIs.
+iree_status_t iree_hal_hip_event_semaphore_acquire_timepoint_device_wait(
+    iree_hal_semaphore_t* base_semaphore, uint64_t min_value,
+    hipEvent_t* out_event);
+
+#ifdef __cplusplus
+}  // extern "C"
+#endif  // __cplusplus
+
+#endif  // IREE_EXPERIMENTAL_HIP_EVENT_SEMAPHORE_H_
diff --git a/experimental/hip/pending_queue_actions.c b/experimental/hip/pending_queue_actions.c
new file mode 100644
index 0000000..08a8135
--- /dev/null
+++ b/experimental/hip/pending_queue_actions.c
@@ -0,0 +1,851 @@
+// Copyright 2023 The IREE Authors
+//
+// Licensed under the Apache License v2.0 with LLVM Exceptions.
+// See https://llvm.org/LICENSE.txt for license information.
+// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
+
+#include "experimental/hip/pending_queue_actions.h"
+
+#include <stdbool.h>
+#include <stddef.h>
+
+#include "experimental/hip/dynamic_symbols.h"
+#include "experimental/hip/event_semaphore.h"
+#include "experimental/hip/graph_command_buffer.h"
+#include "experimental/hip/hip_device.h"
+#include "experimental/hip/status_util.h"
+#include "iree/base/api.h"
+#include "iree/base/internal/arena.h"
+#include "iree/base/internal/atomic_slist.h"
+#include "iree/base/internal/synchronization.h"
+#include "iree/base/internal/threading.h"
+#include "iree/hal/api.h"
+#include "iree/hal/utils/deferred_command_buffer.h"
+#include "iree/hal/utils/resource_set.h"
+
+// The maximal number of hipEvent_t objects a command buffer can wait.
+#define IREE_HAL_HIP_MAX_WAIT_EVENT_COUNT 32
+
+//===----------------------------------------------------------------------===//
+// Queue action
+//===----------------------------------------------------------------------===//
+
+typedef enum iree_hal_hip_queue_action_kind_e {
+  IREE_HAL_HIP_QUEUE_ACTION_TYPE_EXECUTION,
+  // TODO: Add support for queue alloca and dealloca.
+} iree_hal_hip_queue_action_kind_t;
+
+// A pending queue action.
+//
+// Note that this struct does not have internal synchronization; it's expected
+// to work together with the pending action queue, which synchronizes accesses.
+typedef struct iree_hal_hip_queue_action_t {
+  // Intrusive doubly-linked list next entry pointer.
+  struct iree_hal_hip_queue_action_t* next;
+  // Intrusive doubly-linked list previous entry pointer.
+  struct iree_hal_hip_queue_action_t* prev;
+
+  // The owning pending actions queue. We use its allocators and pools.
+  // Retained to make sure it outlives the current action.
+  iree_hal_hip_pending_queue_actions_t* owning_actions;
+
+  // The callback to run after completing this action and before freeing
+  // all resources.
+  iree_hal_hip_pending_action_cleanup_callback_t cleanup_callback;
+  // User data to pass into the callback.
+  void* callback_user_data;
+
+  iree_hal_hip_queue_action_kind_t kind;
+  union {
+    struct {
+      iree_host_size_t count;
+      iree_hal_command_buffer_t** ptr;
+    } command_buffers;
+  } payload;
+
+  // The device from which to allocate HIP stream-based command buffers for
+  // applying deferred command buffers.
+  iree_hal_device_t* device;
+
+  // The stream to launch main GPU workload.
+  hipStream_t dispatch_hip_stream;
+  // The stream to launch HIP host function callbacks.
+  hipStream_t callback_hip_stream;
+
+  // Resource set to retain all associated resources by the payload.
+  iree_hal_resource_set_t* resource_set;
+
+  // Semaphore list to wait on for the payload to start on the GPU.
+  iree_hal_semaphore_list_t wait_semaphore_list;
+  // Semaphore list to signal after the payload completes on the GPU.
+  iree_hal_semaphore_list_t signal_semaphore_list;
+
+  // Scratch fields for analyzing whether actions are ready to issue.
+  hipEvent_t events[IREE_HAL_HIP_MAX_WAIT_EVENT_COUNT];
+  iree_host_size_t event_count;
+  bool is_pending;
+} iree_hal_hip_queue_action_t;
+
+//===----------------------------------------------------------------------===//
+// Queue action list
+//===----------------------------------------------------------------------===//
+
+typedef struct iree_hal_hip_queue_action_list_t {
+  iree_hal_hip_queue_action_t* head;
+  iree_hal_hip_queue_action_t* tail;
+} iree_hal_hip_queue_action_list_t;
+
+// Returns true if the action list is empty.
+static inline bool iree_hal_hip_queue_action_list_is_empty(
+    const iree_hal_hip_queue_action_list_t* list) {
+  return list->head == NULL;
+}
+
+// Pushes |action| on to the end of the given action |list|.
+static void iree_hal_hip_queue_action_list_push_back(
+    iree_hal_hip_queue_action_list_t* list,
+    iree_hal_hip_queue_action_t* action) {
+  if (list->tail) {
+    list->tail->next = action;
+  } else {
+    list->head = action;
+  }
+  action->next = NULL;
+  action->prev = list->tail;
+  list->tail = action;
+}
+
+// Erases |action| from |list|.
+static void iree_hal_hip_queue_action_list_erase(
+    iree_hal_hip_queue_action_list_t* list,
+    iree_hal_hip_queue_action_t* action) {
+  iree_hal_hip_queue_action_t* next = action->next;
+  iree_hal_hip_queue_action_t* prev = action->prev;
+  if (prev) {
+    prev->next = next;
+    action->prev = NULL;
+  } else {
+    list->head = next;
+  }
+  if (next) {
+    next->prev = prev;
+    action->next = NULL;
+  } else {
+    list->tail = prev;
+  }
+}
+
+// Takes all actions from |available_list| and moves them into |ready_list|.
+static void iree_hal_hip_queue_action_list_take_all(
+    iree_hal_hip_queue_action_list_t* available_list,
+    iree_hal_hip_queue_action_list_t* ready_list) {
+  IREE_ASSERT(available_list != ready_list);
+  ready_list->head = available_list->head;
+  ready_list->tail = available_list->tail;
+  available_list->head = NULL;
+  available_list->tail = NULL;
+}
+
+// Frees all actions in the given |list|.
+static void iree_hal_hip_queue_action_list_free_actions(
+    iree_allocator_t host_allocator, iree_hal_hip_queue_action_list_t* list) {
+  for (iree_hal_hip_queue_action_t* action = list->head; action != NULL;) {
+    iree_hal_hip_queue_action_t* next_action = action->next;
+    iree_allocator_free(host_allocator, action);
+    action = next_action;
+  }
+}
+
+//===----------------------------------------------------------------------===//
+// Ready-list processing
+//===----------------------------------------------------------------------===//
+
+// Ready action atomic slist entry struct.
+typedef struct iree_hal_hip_atomic_slist_entry_t {
+  iree_hal_hip_queue_action_t* ready_list_head;
+  iree_atomic_slist_intrusive_ptr_t slist_next;
+} iree_hal_hip_atomic_slist_entry_t;
+
+// Ready action atomic slist.
+IREE_TYPED_ATOMIC_SLIST_WRAPPER(iree_hal_hip_ready_action,
+                                iree_hal_hip_atomic_slist_entry_t,
+                                offsetof(iree_hal_hip_atomic_slist_entry_t,
+                                         slist_next));
+
+// The ready-list processing worker's working/exiting state.
+typedef enum iree_hal_hip_worker_state_e {
+  IREE_HAL_HIP_WORKER_STATE_IDLE_WAITING = 0,
+  IREE_HAL_HIP_WORKER_STATE_WORKLOAD_PENDING = 1,
+  IREE_HAL_HIP_WORKER_STATE_EXIT_REQUESTED = -1,
+  IREE_HAL_HIP_WORKER_STATE_EXIT_COMMITTED = -2,
+  IREE_HAL_HIP_WORKER_STATE_EXIT_ERROR = -3,
+} iree_hal_hip_worker_state_t;
+
+// The data structure needed by a ready-list processing worker thread to issue
+// ready actions to the GPU.
+//
+// This data structure is shared between the parent thread, which owns the
+// whole pending actions queue, and the worker thread; so proper synchronization
+// is needed to touch it from both sides.
+//
+// The parent thread should push a list of ready actions to ready_worklist,
+// update worker_state, and give state_notification accordingly.
+// The worker thread waits on the state_notification and checks worker_state,
+// and pops from the ready_worklist to process. The worker thread also monintors
+// worker_state and stops processing if requested by the parent thread.
+typedef struct iree_hal_hip_working_area_t {
+  // Notification from the parent thread to request worker state changes.
+  iree_notification_t state_notification;
+  // Notification to the parent thread to indicate the worker committed exiting.
+  iree_notification_t exit_notification;
+  iree_hal_hip_ready_action_slist_t ready_worklist;  // atomic
+  iree_atomic_int32_t worker_state;                  // atomic
+  iree_atomic_intptr_t error_code;                   // atomic
+  iree_allocator_t host_allocator;                   // const
+} iree_hal_hip_working_area_t;
+
+static void iree_hal_hip_working_area_initialize(
+    iree_allocator_t host_allocator,
+    iree_hal_hip_working_area_t* working_area) {
+  iree_notification_initialize(&working_area->state_notification);
+  iree_notification_initialize(&working_area->exit_notification);
+  iree_hal_hip_ready_action_slist_initialize(&working_area->ready_worklist);
+  iree_atomic_store_int32(&working_area->worker_state,
+                          IREE_HAL_HIP_WORKER_STATE_IDLE_WAITING,
+                          iree_memory_order_release);
+  iree_atomic_store_int32(&working_area->error_code, IREE_STATUS_OK,
+                          iree_memory_order_release);
+  working_area->host_allocator = host_allocator;
+}
+
+static void iree_hal_hip_working_area_deinitialize(
+    iree_hal_hip_working_area_t* working_area) {
+  iree_hal_hip_ready_action_slist_deinitialize(&working_area->ready_worklist);
+  iree_notification_deinitialize(&working_area->exit_notification);
+  iree_notification_deinitialize(&working_area->state_notification);
+}
+
+// The main function for the ready-list processing worker thread.
+static int iree_hal_hip_worker_execute(
+    iree_hal_hip_working_area_t* working_area);
+
+//===----------------------------------------------------------------------===//
+// Pending queue actions
+//===----------------------------------------------------------------------===//
+
+struct iree_hal_hip_pending_queue_actions_t {
+  // Abstract resource used for injecting reference counting and vtable;
+  // must be at offset 0.
+  iree_hal_resource_t resource;
+
+  // The allocator used to create the timepoint pool.
+  iree_allocator_t host_allocator;
+  // The block pool to allocate resource sets from.
+  iree_arena_block_pool_t* block_pool;
+
+  // The symbols used to create and destroy hipEvent_t objects.
+  const iree_hal_hip_dynamic_symbols_t* symbols;
+
+  // Non-recursive mutex guarding access to the action list.
+  iree_slim_mutex_t action_mutex;
+
+  // The double-linked list of pending actions.
+  iree_hal_hip_queue_action_list_t action_list IREE_GUARDED_BY(action_mutex);
+
+  // The worker thread that monitors incoming requests and issues ready actions
+  // to the GPU.
+  iree_thread_t* worker_thread;
+  // The worker's working area; data exchange place with the parent thread.
+  iree_hal_hip_working_area_t working_area;
+};
+
+static const iree_hal_resource_vtable_t
+    iree_hal_hip_pending_queue_actions_vtable;
+
+iree_status_t iree_hal_hip_pending_queue_actions_create(
+    const iree_hal_hip_dynamic_symbols_t* symbols,
+    iree_arena_block_pool_t* block_pool, iree_allocator_t host_allocator,
+    iree_hal_hip_pending_queue_actions_t** out_actions) {
+  IREE_ASSERT_ARGUMENT(symbols);
+  IREE_ASSERT_ARGUMENT(block_pool);
+  IREE_ASSERT_ARGUMENT(out_actions);
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  iree_hal_hip_pending_queue_actions_t* actions = NULL;
+  IREE_RETURN_AND_END_ZONE_IF_ERROR(
+      z0, iree_allocator_malloc(host_allocator, sizeof(*actions),
+                                (void**)&actions));
+  iree_hal_resource_initialize(&iree_hal_hip_pending_queue_actions_vtable,
+                               &actions->resource);
+  actions->host_allocator = host_allocator;
+  actions->block_pool = block_pool;
+  actions->symbols = symbols;
+  iree_slim_mutex_initialize(&actions->action_mutex);
+  memset(&actions->action_list, 0, sizeof(actions->action_list));
+
+  // Initialize the working area for the ready-list processing worker.
+  iree_hal_hip_working_area_t* working_area = &actions->working_area;
+  iree_hal_hip_working_area_initialize(host_allocator, working_area);
+
+  // Create the ready-list processing worker itself.
+  iree_thread_create_params_t params;
+  memset(&params, 0, sizeof(params));
+  params.name = IREE_SV("deferred_queue_worker");
+  params.create_suspended = false;
+  iree_status_t status = iree_thread_create(
+      (iree_thread_entry_t)iree_hal_hip_worker_execute, working_area, params,
+      actions->host_allocator, &actions->worker_thread);
+
+  if (iree_status_is_ok(status)) {
+    *out_actions = actions;
+  } else {
+    iree_hal_hip_pending_queue_actions_destroy((iree_hal_resource_t*)actions);
+  }
+
+  IREE_TRACE_ZONE_END(z0);
+  return iree_ok_status();
+}
+
+static iree_hal_hip_pending_queue_actions_t*
+iree_hal_hip_pending_queue_actions_cast(iree_hal_resource_t* base_value) {
+  return (iree_hal_hip_pending_queue_actions_t*)base_value;
+}
+
+static bool iree_hal_hip_worker_committed_exiting(
+    iree_hal_hip_working_area_t* working_area);
+
+void iree_hal_hip_pending_queue_actions_destroy(
+    iree_hal_resource_t* base_actions) {
+  iree_hal_hip_pending_queue_actions_t* actions =
+      iree_hal_hip_pending_queue_actions_cast(base_actions);
+  iree_allocator_t host_allocator = actions->host_allocator;
+  iree_hal_hip_working_area_t* working_area = &actions->working_area;
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  // Request the worker to exit.
+  iree_hal_hip_worker_state_t prev_state =
+      (iree_hal_hip_worker_state_t)iree_atomic_exchange_int32(
+          &working_area->worker_state, IREE_HAL_HIP_WORKER_STATE_EXIT_REQUESTED,
+          iree_memory_order_acq_rel);
+  iree_notification_post(&working_area->state_notification, IREE_ALL_WAITERS);
+
+  if (prev_state != IREE_HAL_HIP_WORKER_STATE_EXIT_ERROR) {
+    // Wait until the worker acknowledged exiting.
+    iree_notification_await(
+        &working_area->exit_notification,
+        (iree_condition_fn_t)iree_hal_hip_worker_committed_exiting,
+        working_area, iree_infinite_timeout());
+  }
+
+  // Now we can delete worker related resources.
+  iree_thread_release(actions->worker_thread);
+  iree_hal_hip_working_area_deinitialize(working_area);
+
+  iree_slim_mutex_deinitialize(&actions->action_mutex);
+  iree_hal_hip_queue_action_list_free_actions(host_allocator,
+                                              &actions->action_list);
+  iree_allocator_free(host_allocator, actions);
+
+  IREE_TRACE_ZONE_END(z0);
+}
+
+static const iree_hal_resource_vtable_t
+    iree_hal_hip_pending_queue_actions_vtable = {
+        .destroy = iree_hal_hip_pending_queue_actions_destroy,
+};
+
+// Copies of the given |in_list| to |out_list| to retain the command buffer
+// list.
+static iree_status_t iree_hal_hip_copy_command_buffer_list(
+    iree_host_size_t command_buffer_count,
+    iree_hal_command_buffer_t* const* in_list, iree_allocator_t host_allocator,
+    iree_hal_command_buffer_t*** out_list) {
+  *out_list = NULL;
+  if (!command_buffer_count) return iree_ok_status();
+
+  iree_host_size_t total_size = command_buffer_count * sizeof(*in_list);
+  IREE_RETURN_IF_ERROR(
+      iree_allocator_malloc(host_allocator, total_size, (void**)out_list));
+  memcpy((void*)*out_list, in_list, total_size);
+  return iree_ok_status();
+}
+
+// Frees the semaphore and value list inside |semaphore_list|.
+static void iree_hal_hip_free_command_buffer_list(
+    iree_allocator_t host_allocator,
+    iree_hal_command_buffer_t* const* command_buffer_list) {
+  iree_allocator_free(host_allocator, (void*)command_buffer_list);
+}
+
+// Copies of the given |in_list| to |out_list| to retain the semaphore and value
+// list.
+static iree_status_t iree_hal_hip_copy_semaphore_list(
+    iree_hal_semaphore_list_t in_list, iree_allocator_t host_allocator,
+    iree_hal_semaphore_list_t* out_list) {
+  memset(out_list, 0, sizeof(*out_list));
+  if (!in_list.count) return iree_ok_status();
+
+  out_list->count = in_list.count;
+  iree_host_size_t semaphore_size = in_list.count * sizeof(*in_list.semaphores);
+  IREE_RETURN_IF_ERROR(iree_allocator_malloc(host_allocator, semaphore_size,
+                                             (void**)&out_list->semaphores));
+  memcpy(out_list->semaphores, in_list.semaphores, semaphore_size);
+
+  iree_host_size_t value_size = in_list.count * sizeof(*in_list.payload_values);
+  IREE_RETURN_IF_ERROR(iree_allocator_malloc(
+      host_allocator, value_size, (void**)&out_list->payload_values));
+  memcpy(out_list->payload_values, in_list.payload_values, value_size);
+  return iree_ok_status();
+}
+
+// Frees the semaphore and value list inside |semaphore_list|.
+static void iree_hal_hip_free_semaphore_list(
+    iree_allocator_t host_allocator,
+    iree_hal_semaphore_list_t* semaphore_list) {
+  iree_allocator_free(host_allocator, semaphore_list->semaphores);
+  iree_allocator_free(host_allocator, semaphore_list->payload_values);
+}
+
+iree_status_t iree_hal_hip_pending_queue_actions_enqueue_execution(
+    iree_hal_device_t* device, hipStream_t dispatch_stream,
+    hipStream_t callback_stream, iree_hal_hip_pending_queue_actions_t* actions,
+    iree_hal_hip_pending_action_cleanup_callback_t cleanup_callback,
+    void* callback_user_data,
+    const iree_hal_semaphore_list_t wait_semaphore_list,
+    const iree_hal_semaphore_list_t signal_semaphore_list,
+    iree_host_size_t command_buffer_count,
+    iree_hal_command_buffer_t* const* command_buffers) {
+  IREE_ASSERT_ARGUMENT(actions);
+  IREE_ASSERT_ARGUMENT(command_buffer_count == 0 || command_buffers);
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  iree_hal_hip_queue_action_t* action = NULL;
+  IREE_RETURN_AND_END_ZONE_IF_ERROR(
+      z0, iree_allocator_malloc(actions->host_allocator, sizeof(*action),
+                                (void**)&action));
+
+  action->kind = IREE_HAL_HIP_QUEUE_ACTION_TYPE_EXECUTION;
+  action->cleanup_callback = cleanup_callback;
+  action->callback_user_data = callback_user_data;
+  action->device = device;
+  action->dispatch_hip_stream = dispatch_stream;
+  action->callback_hip_stream = callback_stream;
+  action->event_count = 0;
+  action->is_pending = true;
+
+  // Retain all command buffers and semaphores.
+  iree_hal_resource_set_t* resource_set = NULL;
+  iree_status_t status =
+      iree_hal_resource_set_allocate(actions->block_pool, &resource_set);
+  if (IREE_LIKELY(iree_status_is_ok(status))) {
+    status = iree_hal_resource_set_insert(resource_set, command_buffer_count,
+                                          command_buffers);
+  }
+  if (IREE_LIKELY(iree_status_is_ok(status))) {
+    status =
+        iree_hal_resource_set_insert(resource_set, wait_semaphore_list.count,
+                                     wait_semaphore_list.semaphores);
+  }
+  if (IREE_LIKELY(iree_status_is_ok(status))) {
+    status =
+        iree_hal_resource_set_insert(resource_set, signal_semaphore_list.count,
+                                     signal_semaphore_list.semaphores);
+  }
+
+  // Copy the command buffer list for later access.
+  // TODO: avoid host allocator malloc; use some pool for the allocation.
+  if (IREE_LIKELY(iree_status_is_ok(status))) {
+    action->payload.command_buffers.count = command_buffer_count;
+    status = iree_hal_hip_copy_command_buffer_list(
+        command_buffer_count, command_buffers, actions->host_allocator,
+        &action->payload.command_buffers.ptr);
+  }
+
+  // Copy the semaphore and value list for later access.
+  // TODO: avoid host allocator malloc; use some pool for the allocation.
+  if (IREE_LIKELY(iree_status_is_ok(status))) {
+    status = iree_hal_hip_copy_semaphore_list(wait_semaphore_list,
+                                              actions->host_allocator,
+                                              &action->wait_semaphore_list);
+  }
+  if (IREE_LIKELY(iree_status_is_ok(status))) {
+    status = iree_hal_hip_copy_semaphore_list(signal_semaphore_list,
+                                              actions->host_allocator,
+                                              &action->signal_semaphore_list);
+  }
+
+  if (IREE_LIKELY(iree_status_is_ok(status))) {
+    action->owning_actions = actions;
+    iree_hal_resource_retain(actions);
+
+    action->resource_set = resource_set;
+
+    iree_slim_mutex_lock(&actions->action_mutex);
+    iree_hal_hip_queue_action_list_push_back(&actions->action_list, action);
+    iree_slim_mutex_unlock(&actions->action_mutex);
+  } else {
+    iree_hal_hip_free_semaphore_list(actions->host_allocator,
+                                     &action->wait_semaphore_list);
+    iree_hal_hip_free_semaphore_list(actions->host_allocator,
+                                     &action->signal_semaphore_list);
+    iree_hal_hip_free_command_buffer_list(actions->host_allocator,
+                                          action->payload.command_buffers.ptr);
+    iree_hal_resource_set_free(resource_set);
+    iree_allocator_free(actions->host_allocator, action);
+  }
+
+  IREE_TRACE_ZONE_END(z0);
+  return status;
+}
+
+static void iree_hal_hip_pending_queue_actions_cleanup_execution(
+    iree_hal_hip_queue_action_t* action);
+
+// Releases resources after action completion on the GPU and advances timeline
+// and pending actions queue.
+//
+// This is the HIP host function callback to hipLaunchHostFunc, invoked by a
+// HIP driver thread.
+static void iree_hal_hip_execution_device_signal_host_callback(
+    void* user_data) {
+  IREE_TRACE_ZONE_BEGIN(z0);
+  iree_hal_hip_queue_action_t* action = (iree_hal_hip_queue_action_t*)user_data;
+  iree_hal_hip_pending_queue_actions_t* actions = action->owning_actions;
+  // Advance semaphore timelines by calling into the host signaling function.
+  IREE_IGNORE_ERROR(
+      iree_hal_semaphore_list_signal(action->signal_semaphore_list));
+  // Destroy the current action given its done now--this also frees all retained
+  // resources.
+  iree_hal_hip_pending_queue_actions_cleanup_execution(action);
+  // Try to release more pending actions to the GPU now.
+  IREE_IGNORE_ERROR(iree_hal_hip_pending_queue_actions_issue(actions));
+  IREE_TRACE_ZONE_END(z0);
+}
+
+// Issues the given kernel dispatch |action| to the GPU.
+static iree_status_t iree_hal_hip_pending_queue_actions_issue_execution(
+    iree_hal_hip_queue_action_t* action) {
+  IREE_ASSERT(action->is_pending == false);
+  const iree_hal_hip_dynamic_symbols_t* symbols =
+      action->owning_actions->symbols;
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  // No need to lock given that this action is already detched from the pending
+  // actions list; so only this thread is seeing it now.
+
+  // First wait all the device hipEvent_t in the dispatch stream.
+  for (iree_host_size_t i = 0; i < action->event_count; ++i) {
+    IREE_HIP_RETURN_AND_END_ZONE_IF_ERROR(
+        z0, symbols,
+        hipStreamWaitEvent(action->dispatch_hip_stream, action->events[i],
+                           /*flags=*/0),
+        "hipStreamWaitEvent");
+  }
+
+  // Then launch all command buffers to the dispatch stream.
+  for (iree_host_size_t i = 0; i < action->payload.command_buffers.count; ++i) {
+    iree_hal_command_buffer_t* command_buffer =
+        action->payload.command_buffers.ptr[i];
+    if (iree_hal_hip_graph_command_buffer_isa(command_buffer)) {
+      hipGraphExec_t exec = iree_hal_hip_graph_command_buffer_handle(
+          action->payload.command_buffers.ptr[i]);
+      IREE_HIP_RETURN_AND_END_ZONE_IF_ERROR(
+          z0, symbols, hipGraphLaunch(exec, action->dispatch_hip_stream),
+          "hipGraphLaunch");
+    } else {
+      iree_hal_command_buffer_t* stream_command_buffer = NULL;
+      iree_hal_command_buffer_mode_t mode =
+          IREE_HAL_COMMAND_BUFFER_MODE_ONE_SHOT |
+          IREE_HAL_COMMAND_BUFFER_MODE_ALLOW_INLINE_EXECUTION |
+          IREE_HAL_COMMAND_BUFFER_MODE_UNVALIDATED;
+      IREE_RETURN_AND_END_ZONE_IF_ERROR(
+          z0, iree_hal_hip_device_create_stream_command_buffer(
+                  action->device, mode, IREE_HAL_COMMAND_CATEGORY_ANY,
+                  /*binding_capacity=*/0, &stream_command_buffer));
+      IREE_RETURN_AND_END_ZONE_IF_ERROR(
+          z0, iree_hal_resource_set_insert(action->resource_set, 1,
+                                           &stream_command_buffer));
+      IREE_RETURN_AND_END_ZONE_IF_ERROR(
+          z0, iree_hal_deferred_command_buffer_apply(
+                  command_buffer, stream_command_buffer,
+                  iree_hal_buffer_binding_table_empty()));
+    }
+  }
+
+  // Last record hipEvent_t signals in the dispatch stream.
+  for (iree_host_size_t i = 0; i < action->signal_semaphore_list.count; ++i) {
+    // Grab a hipEvent_t for this semaphore value signaling.
+    hipEvent_t event = NULL;
+    IREE_RETURN_AND_END_ZONE_IF_ERROR(
+        z0, iree_hal_hip_event_semaphore_acquire_timepoint_device_signal(
+                action->signal_semaphore_list.semaphores[i],
+                action->signal_semaphore_list.payload_values[i], &event));
+
+    // Record the event signaling in the dispatch stream.
+    IREE_HIP_RETURN_AND_END_ZONE_IF_ERROR(
+        z0, symbols, hipEventRecord(event, action->dispatch_hip_stream),
+        "hipEventRecord");
+    // Let the callback stream to wait on the hipEvent_t.
+    IREE_HIP_RETURN_AND_END_ZONE_IF_ERROR(
+        z0, symbols,
+        hipStreamWaitEvent(action->callback_hip_stream, event, /*flags=*/0),
+        "hipStreamWaitEvent");
+  }
+
+  // Now launch a host function on the callback stream to advance the semaphore
+  // timeline.
+  IREE_HIP_RETURN_AND_END_ZONE_IF_ERROR(
+      z0, symbols,
+      hipLaunchHostFunc(action->callback_hip_stream,
+                        iree_hal_hip_execution_device_signal_host_callback,
+                        action),
+      "hipLaunchHostFunc");
+
+  IREE_TRACE_ZONE_END(z0);
+  return iree_ok_status();
+}
+
+// Releases resources after completing the given kernel dispatch |action|.
+static void iree_hal_hip_pending_queue_actions_cleanup_execution(
+    iree_hal_hip_queue_action_t* action) {
+  iree_hal_hip_pending_queue_actions_t* actions = action->owning_actions;
+  iree_allocator_t host_allocator = actions->host_allocator;
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  action->cleanup_callback(action->callback_user_data);
+
+  iree_hal_resource_set_free(action->resource_set);
+  iree_hal_hip_free_semaphore_list(host_allocator,
+                                   &action->wait_semaphore_list);
+  iree_hal_hip_free_semaphore_list(host_allocator,
+                                   &action->signal_semaphore_list);
+  iree_hal_resource_release(actions);
+
+  iree_allocator_free(host_allocator, action);
+
+  IREE_TRACE_ZONE_END(z0);
+}
+
+iree_status_t iree_hal_hip_pending_queue_actions_issue(
+    iree_hal_hip_pending_queue_actions_t* actions) {
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  iree_hal_hip_queue_action_list_t pending_list = {NULL, NULL};
+  iree_hal_hip_queue_action_list_t ready_list = {NULL, NULL};
+
+  iree_slim_mutex_lock(&actions->action_mutex);
+
+  if (iree_hal_hip_queue_action_list_is_empty(&actions->action_list)) {
+    iree_slim_mutex_unlock(&actions->action_mutex);
+    IREE_TRACE_ZONE_END(z0);
+    return iree_ok_status();
+  }
+
+  // Scan through the list and categorize actions into pending and ready lists.
+  iree_status_t status = iree_ok_status();
+  iree_hal_hip_queue_action_t* action = actions->action_list.head;
+  while (action) {
+    iree_hal_hip_queue_action_t* next_action = action->next;
+    action->next = NULL;
+
+    iree_host_size_t semaphore_count = action->wait_semaphore_list.count;
+    iree_hal_semaphore_t** semaphores = action->wait_semaphore_list.semaphores;
+    uint64_t* values = action->wait_semaphore_list.payload_values;
+
+    action->event_count = 0;
+    action->is_pending = false;
+
+    // Look at all wait semaphores.
+    for (iree_host_size_t i = 0; i < semaphore_count; ++i) {
+      // If this semaphore has already signaled past the desired value, we can
+      // just ignore it.
+      uint64_t value = 0;
+      status = iree_hal_semaphore_query(semaphores[i], &value);
+      if (IREE_UNLIKELY(!iree_status_is_ok(status))) break;
+      if (value >= values[i]) continue;
+
+      // Try to acquire a hipEvent_t from a device wait timepoint. If so, we can
+      // use that hipEvent_t to wait on the device. Otherwise, this action is
+      // still not ready.
+      hipEvent_t event = NULL;
+      status = iree_hal_hip_event_semaphore_acquire_timepoint_device_wait(
+          semaphores[i], values[i], &event);
+      if (IREE_UNLIKELY(!iree_status_is_ok(status))) break;
+      if (!event) {
+        // Clear the scratch fields.
+        action->event_count = 0;
+        action->is_pending = true;
+        break;
+      }
+      if (IREE_UNLIKELY(action->event_count >=
+                        IREE_HAL_HIP_MAX_WAIT_EVENT_COUNT)) {
+        status = iree_make_status(IREE_STATUS_RESOURCE_EXHAUSTED,
+                                  "exceeded max wait hipEvent_t limit");
+        break;
+      }
+      action->events[action->event_count++] = event;
+    }
+
+    if (IREE_UNLIKELY(!iree_status_is_ok(status))) break;
+
+    if (action->is_pending) {
+      iree_hal_hip_queue_action_list_push_back(&pending_list, action);
+    } else {
+      iree_hal_hip_queue_action_list_push_back(&ready_list, action);
+    }
+
+    action = next_action;
+  }
+
+  if (IREE_UNLIKELY(!iree_status_is_ok(status))) {
+    // Some error happened during processing the current action. Clear the
+    // scratch fields and put it back to the pending list so we don't leak.
+    action->event_count = 0;
+    action->is_pending = true;
+    iree_hal_hip_queue_action_list_push_back(&pending_list, action);
+  }
+
+  // Preserve pending timepoints.
+  actions->action_list = pending_list;
+
+  iree_slim_mutex_unlock(&actions->action_mutex);
+
+  iree_hal_hip_atomic_slist_entry_t* entry = NULL;
+  // TODO: avoid host allocator malloc; use some pool for the allocation.
+  if (iree_status_is_ok(status)) {
+    status = iree_allocator_malloc(actions->host_allocator, sizeof(*entry),
+                                   (void**)&entry);
+  }
+
+  if (IREE_UNLIKELY(!iree_status_is_ok(status))) {
+    // Release all actions in the ready list to avoid leaking.
+    iree_hal_hip_queue_action_list_free_actions(actions->host_allocator,
+                                                &ready_list);
+    IREE_TRACE_ZONE_END(z0);
+    return status;
+  }
+
+  // Now push the ready list to the worker and have it to issue the actions to
+  // the GPU.
+  entry->ready_list_head = ready_list.head;
+  iree_hal_hip_ready_action_slist_push(&actions->working_area.ready_worklist,
+                                       entry);
+  iree_hal_hip_worker_state_t prev_state =
+      (iree_hal_hip_worker_state_t)iree_atomic_exchange_int32(
+          &actions->working_area.worker_state,
+          IREE_HAL_HIP_WORKER_STATE_WORKLOAD_PENDING,
+          iree_memory_order_acq_rel);
+  iree_notification_post(&actions->working_area.state_notification,
+                         IREE_ALL_WAITERS);
+
+  // Handle potential error cases from the worker thread.
+  if (prev_state == IREE_HAL_HIP_WORKER_STATE_EXIT_ERROR) {
+    iree_status_code_t code = iree_atomic_load_int32(
+        &actions->working_area.error_code, iree_memory_order_acquire);
+    status = iree_status_from_code(code);
+  }
+
+  IREE_TRACE_ZONE_END(z0);
+  return status;
+}
+
+//===----------------------------------------------------------------------===//
+// Worker routines
+//===----------------------------------------------------------------------===//
+
+static bool iree_hal_hip_worker_has_incoming_request(
+    iree_hal_hip_working_area_t* working_area) {
+  iree_hal_hip_worker_state_t value = iree_atomic_load_int32(
+      &working_area->worker_state, iree_memory_order_acquire);
+  return value == IREE_HAL_HIP_WORKER_STATE_WORKLOAD_PENDING ||
+         value == IREE_HAL_HIP_WORKER_STATE_EXIT_REQUESTED;
+}
+
+static bool iree_hal_hip_worker_committed_exiting(
+    iree_hal_hip_working_area_t* working_area) {
+  return iree_atomic_load_int32(&working_area->worker_state,
+                                iree_memory_order_acquire) ==
+         IREE_HAL_HIP_WORKER_STATE_EXIT_COMMITTED;
+}
+
+// Processes all ready actions in the given |worklist|.
+static iree_status_t iree_hal_hip_worker_process_ready_list(
+    iree_allocator_t host_allocator,
+    iree_hal_hip_ready_action_slist_t* worklist) {
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  iree_status_t status = iree_ok_status();
+  do {
+    iree_hal_hip_atomic_slist_entry_t* entry =
+        iree_hal_hip_ready_action_slist_pop(worklist);
+    if (!entry) break;
+
+    // Process the current batch of ready actions.
+    iree_hal_hip_queue_action_t* action = entry->ready_list_head;
+    while (action) {
+      iree_hal_hip_queue_action_t* next_action = action->next;
+      action->next = NULL;
+
+      status = iree_hal_hip_pending_queue_actions_issue_execution(action);
+      if (!iree_status_is_ok(status)) break;
+      action->event_count = 0;
+
+      action = next_action;
+    }
+
+    iree_allocator_free(host_allocator, entry);
+  } while (iree_status_is_ok(status));
+
+  IREE_TRACE_ZONE_END(z0);
+  return status;
+}
+
+// The main function for the ready-list processing worker thread.
+static int iree_hal_hip_worker_execute(
+    iree_hal_hip_working_area_t* working_area) {
+  iree_hal_hip_ready_action_slist_t* worklist = &working_area->ready_worklist;
+
+  while (true) {
+    // Block waiting for incoming requests.
+    iree_notification_await(
+        &working_area->state_notification,
+        (iree_condition_fn_t)iree_hal_hip_worker_has_incoming_request,
+        working_area, iree_infinite_timeout());
+
+    // Check if we received request to stop processing and exit this thread.
+    bool should_exit = iree_atomic_load_int32(&working_area->worker_state,
+                                              iree_memory_order_acquire) ==
+                       IREE_HAL_HIP_WORKER_STATE_EXIT_REQUESTED;
+
+    iree_status_t status = iree_hal_hip_worker_process_ready_list(
+        working_area->host_allocator, worklist);
+    if (IREE_UNLIKELY(!iree_status_is_ok(status))) {
+      IREE_ASSERT(false && "error when processing ready list");
+      iree_atomic_store_int32(&working_area->error_code,
+                              iree_status_code(status),
+                              iree_memory_order_release);
+      iree_atomic_store_int32(&working_area->worker_state,
+                              IREE_HAL_HIP_WORKER_STATE_EXIT_ERROR,
+                              iree_memory_order_release);
+      iree_notification_post(&working_area->exit_notification,
+                             IREE_ALL_WAITERS);
+      return -1;
+    }
+
+    if (should_exit) {
+      // Signal that this thread is committed to exit.
+      iree_atomic_store_int32(&working_area->worker_state,
+                              IREE_HAL_HIP_WORKER_STATE_EXIT_COMMITTED,
+                              iree_memory_order_release);
+      iree_notification_post(&working_area->exit_notification,
+                             IREE_ALL_WAITERS);
+      return 0;
+    }
+
+    // Signal that this thread is done processing and now waiting for more.
+    iree_atomic_store_int32(&working_area->worker_state,
+                            IREE_HAL_HIP_WORKER_STATE_IDLE_WAITING,
+                            iree_memory_order_release);
+  }
+  return 0;
+}
diff --git a/experimental/hip/pending_queue_actions.h b/experimental/hip/pending_queue_actions.h
new file mode 100644
index 0000000..aeb900f
--- /dev/null
+++ b/experimental/hip/pending_queue_actions.h
@@ -0,0 +1,75 @@
+// Copyright 2024 The IREE Authors
+//
+// Licensed under the Apache License v2.0 with LLVM Exceptions.
+// See https://llvm.org/LICENSE.txt for license information.
+// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
+
+#ifndef IREE_EXPERIMENTAL_HIP_PENDING_QUEUE_ACTIONS_H_
+#define IREE_EXPERIMENTAL_HIP_PENDING_QUEUE_ACTIONS_H_
+
+#include "experimental/hip/dynamic_symbols.h"
+#include "iree/base/api.h"
+#include "iree/base/internal/arena.h"
+#include "iree/hal/api.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif  // __cplusplus
+
+// A data structure to manage pending queue actions (kernel launches and async
+// allocations).
+//
+// This is needed in order to satisfy queue action dependencies. IREE uses HAL
+// semaphore as the unified mechanism for synchronization directions including
+// host to host, host to device, devie to device, and device to host. Plus, it
+// allows wait before signal. These flexible capabilities are not all supported
+// by hipEvent_t objects. Therefore, we need supporting data structures to
+// implement them on top of hipEvent_t objects. Thus this pending queue actions.
+//
+// This buffers pending queue actions and their associated resources. It
+// provides an API to advance the wait list on demand--queue actions are
+// released to the GPU when all their wait semaphores are signaled past the
+// desired value, or we can have a hipEvent_t already recorded to some HIP
+// stream to wait on.
+//
+// Thread-safe; multiple threads may enqueue workloads.
+typedef struct iree_hal_hip_pending_queue_actions_t
+    iree_hal_hip_pending_queue_actions_t;
+
+// Creates a pending actions queue.
+iree_status_t iree_hal_hip_pending_queue_actions_create(
+    const iree_hal_hip_dynamic_symbols_t* symbols,
+    iree_arena_block_pool_t* block_pool, iree_allocator_t host_allocator,
+    iree_hal_hip_pending_queue_actions_t** out_actions);
+
+// Destroys the pending |actions| queue.
+void iree_hal_hip_pending_queue_actions_destroy(iree_hal_resource_t* actions);
+
+// Callback to execute user code after action completion but before resource
+// releasing.
+//
+// Data behind |user_data| must remain alive before the action is released.
+typedef void(IREE_API_PTR* iree_hal_hip_pending_action_cleanup_callback_t)(
+    void* user_data);
+
+// Enqueues the given list of |command_buffers| that waits on
+// |wait_semaphore_list| and signals |signal_semaphore_lsit|.
+iree_status_t iree_hal_hip_pending_queue_actions_enqueue_execution(
+    iree_hal_device_t* device, hipStream_t dispatch_stream,
+    hipStream_t callback_stream, iree_hal_hip_pending_queue_actions_t* actions,
+    iree_hal_hip_pending_action_cleanup_callback_t cleanup_callback,
+    void* callback_user_data,
+    const iree_hal_semaphore_list_t wait_semaphore_list,
+    const iree_hal_semaphore_list_t signal_semaphore_list,
+    iree_host_size_t command_buffer_count,
+    iree_hal_command_buffer_t* const* command_buffers);
+
+// Tries to scan the pending actions and release ready ones to the GPU.
+iree_status_t iree_hal_hip_pending_queue_actions_issue(
+    iree_hal_hip_pending_queue_actions_t* actions);
+
+#ifdef __cplusplus
+}  // extern "C"
+#endif  // __cplusplus
+
+#endif  // IREE_EXPERIMENTAL_HIP_PENDING_QUEUE_ACTIONS_H_
diff --git a/experimental/hip/timepoint_pool.c b/experimental/hip/timepoint_pool.c
new file mode 100644
index 0000000..444fe7f
--- /dev/null
+++ b/experimental/hip/timepoint_pool.c
@@ -0,0 +1,352 @@
+// Copyright 2024 The IREE Authors
+//
+// Licensed under the Apache License v2.0 with LLVM Exceptions.
+// See https://llvm.org/LICENSE.txt for license information.
+// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
+
+#include "experimental/hip/timepoint_pool.h"
+
+#include <stdbool.h>
+#include <stddef.h>
+#include <string.h>
+
+#include "experimental/hip/dynamic_symbols.h"
+#include "experimental/hip/event_pool.h"
+#include "experimental/hip/status_util.h"
+#include "iree/base/api.h"
+#include "iree/base/internal/atomics.h"
+#include "iree/base/internal/event_pool.h"
+#include "iree/base/internal/synchronization.h"
+#include "iree/hal/api.h"
+#include "iree/hal/utils/semaphore_base.h"
+
+//===----------------------------------------------------------------------===//
+// iree_hal_hip_timepoint_t
+//===----------------------------------------------------------------------===//
+
+static iree_status_t iree_hal_hip_timepoint_allocate(
+    iree_hal_hip_timepoint_pool_t* pool, iree_allocator_t host_allocator,
+    iree_hal_hip_timepoint_t** out_timepoint) {
+  IREE_ASSERT_ARGUMENT(pool);
+  IREE_ASSERT_ARGUMENT(out_timepoint);
+  *out_timepoint = NULL;
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  iree_hal_hip_timepoint_t* timepoint = NULL;
+  IREE_RETURN_AND_END_ZONE_IF_ERROR(
+      z0, iree_allocator_malloc(host_allocator, sizeof(*timepoint),
+                                (void**)&timepoint));
+  // iree_allocator_malloc zeros out the whole struct.
+  timepoint->host_allocator = host_allocator;
+  timepoint->pool = pool;
+
+  *out_timepoint = timepoint;
+
+  IREE_TRACE_ZONE_END(z0);
+  return iree_ok_status();
+}
+
+// Clears all data fields in the given |timepoint| except the original host
+// allocator and owning pool.
+static void iree_hal_hip_timepoint_clear(iree_hal_hip_timepoint_t* timepoint) {
+  iree_allocator_t host_allocator = timepoint->host_allocator;
+  iree_hal_hip_timepoint_pool_t* pool = timepoint->pool;
+  memset(timepoint, 0, sizeof(*timepoint));
+  timepoint->host_allocator = host_allocator;
+  timepoint->pool = pool;
+}
+
+static void iree_hal_hip_timepoint_free(iree_hal_hip_timepoint_t* timepoint) {
+  iree_allocator_t host_allocator = timepoint->host_allocator;
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  IREE_ASSERT(timepoint->kind == IREE_HAL_HIP_TIMEPOINT_KIND_NONE);
+  iree_allocator_free(host_allocator, timepoint);
+
+  IREE_TRACE_ZONE_END(z0);
+}
+
+//===----------------------------------------------------------------------===//
+// iree_hal_hip_timepoint_pool_t
+//===----------------------------------------------------------------------===//
+
+struct iree_hal_hip_timepoint_pool_t {
+  // The allocator used to create the timepoint pool.
+  iree_allocator_t host_allocator;
+
+  // The pool to acquire host events.
+  iree_event_pool_t* host_event_pool;
+  // The pool to acquire device events. Internally synchronized.
+  iree_hal_hip_event_pool_t* device_event_pool;
+
+  // Note that the above pools are internally synchronized; so we don't and
+  // shouldn't use the following mutex to guard access to them.
+
+  // Guards timepoint related fields this pool. We don't expect a performant
+  // program to frequently allocate timepoints for synchronization purposes; the
+  // traffic to this pool should be low. So it should be fine to use mutex to
+  // guard here.
+  iree_slim_mutex_t timepoint_mutex;
+
+  // Maximum number of timepoint objects that will be maintained in the pool.
+  // More timepoints may be allocated at any time, but they will be disposed
+  // directly when they are no longer needed.
+  iree_host_size_t available_capacity IREE_GUARDED_BY(timepoint_mutex);
+  // Total number of currently available timepoint objects.
+  iree_host_size_t available_count IREE_GUARDED_BY(timepoint_mutex);
+  // The list of available_count timepoint objects.
+  iree_hal_hip_timepoint_t* available_list[] IREE_GUARDED_BY(timepoint_mutex);
+};
+// + Additional inline allocation for holding timepoints up to the capacity.
+
+iree_status_t iree_hal_hip_timepoint_pool_allocate(
+    iree_event_pool_t* host_event_pool,
+    iree_hal_hip_event_pool_t* device_event_pool,
+    iree_host_size_t available_capacity, iree_allocator_t host_allocator,
+    iree_hal_hip_timepoint_pool_t** out_timepoint_pool) {
+  IREE_ASSERT_ARGUMENT(host_event_pool);
+  IREE_ASSERT_ARGUMENT(device_event_pool);
+  IREE_ASSERT_ARGUMENT(out_timepoint_pool);
+  *out_timepoint_pool = NULL;
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  iree_hal_hip_timepoint_pool_t* timepoint_pool = NULL;
+  iree_host_size_t total_size =
+      sizeof(*timepoint_pool) +
+      available_capacity * sizeof(*timepoint_pool->available_list);
+  IREE_RETURN_AND_END_ZONE_IF_ERROR(
+      z0, iree_allocator_malloc(host_allocator, total_size,
+                                (void**)&timepoint_pool));
+  timepoint_pool->host_allocator = host_allocator;
+  timepoint_pool->host_event_pool = host_event_pool;
+  timepoint_pool->device_event_pool = device_event_pool;
+
+  iree_slim_mutex_initialize(&timepoint_pool->timepoint_mutex);
+  timepoint_pool->available_capacity = available_capacity;
+  timepoint_pool->available_count = 0;
+
+  iree_status_t status = iree_ok_status();
+  for (iree_host_size_t i = 0; i < available_capacity; ++i) {
+    status = iree_hal_hip_timepoint_allocate(
+        timepoint_pool, host_allocator,
+        &timepoint_pool->available_list[timepoint_pool->available_count++]);
+    if (!iree_status_is_ok(status)) break;
+  }
+
+  if (iree_status_is_ok(status)) {
+    *out_timepoint_pool = timepoint_pool;
+  } else {
+    iree_hal_hip_timepoint_pool_free(timepoint_pool);
+  }
+  IREE_TRACE_ZONE_END(z0);
+  return status;
+}
+
+void iree_hal_hip_timepoint_pool_free(
+    iree_hal_hip_timepoint_pool_t* timepoint_pool) {
+  iree_allocator_t host_allocator = timepoint_pool->host_allocator;
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  for (iree_host_size_t i = 0; i < timepoint_pool->available_count; ++i) {
+    iree_hal_hip_timepoint_free(timepoint_pool->available_list[i]);
+  }
+  iree_slim_mutex_deinitialize(&timepoint_pool->timepoint_mutex);
+  iree_allocator_free(host_allocator, timepoint_pool);
+
+  IREE_TRACE_ZONE_END(z0);
+}
+
+// Acquires |timepoint_count| timepoints from the given |timepoint_pool|.
+// The |out_timepoints| needs to be further initialized with proper kind and
+// payload values.
+static iree_status_t iree_hal_hip_timepoint_pool_acquire_internal(
+    iree_hal_hip_timepoint_pool_t* timepoint_pool,
+    iree_host_size_t timepoint_count,
+    iree_hal_hip_timepoint_t** out_timepoints) {
+  IREE_ASSERT_ARGUMENT(timepoint_pool);
+  if (!timepoint_count) return iree_ok_status();
+  IREE_ASSERT_ARGUMENT(out_timepoints);
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  // We'll try to get what we can from the pool and fall back to initializing
+  // new iree_hal_hip_timepoint_t objects.
+  iree_host_size_t remaining_count = timepoint_count;
+
+  // Try first to grab from the pool.
+  iree_slim_mutex_lock(&timepoint_pool->timepoint_mutex);
+  iree_host_size_t from_pool_count =
+      iree_min(timepoint_pool->available_count, timepoint_count);
+  if (from_pool_count > 0) {
+    iree_host_size_t pool_base_index =
+        timepoint_pool->available_count - from_pool_count;
+    memcpy(out_timepoints, &timepoint_pool->available_list[pool_base_index],
+           from_pool_count * sizeof(*timepoint_pool->available_list));
+    timepoint_pool->available_count -= from_pool_count;
+    remaining_count -= from_pool_count;
+  }
+  iree_slim_mutex_unlock(&timepoint_pool->timepoint_mutex);
+
+  // Allocate the rest of the timepoints.
+  if (remaining_count > 0) {
+    IREE_TRACE_ZONE_BEGIN_NAMED(z1, "timepoint-pool-unpooled-acquire");
+    iree_status_t status = iree_ok_status();
+    for (iree_host_size_t i = 0; i < remaining_count; ++i) {
+      status = iree_hal_hip_timepoint_allocate(
+          timepoint_pool, timepoint_pool->host_allocator,
+          &out_timepoints[from_pool_count + i]);
+      if (!iree_status_is_ok(status)) {
+        // Must release all timepoints we've acquired so far.
+        iree_hal_hip_timepoint_pool_release(timepoint_pool, from_pool_count + i,
+                                            out_timepoints);
+        IREE_TRACE_ZONE_END(z1);
+        IREE_TRACE_ZONE_END(z0);
+        return status;
+      }
+    }
+    IREE_TRACE_ZONE_END(z1);
+  }
+
+  IREE_TRACE_ZONE_END(z0);
+  return iree_ok_status();
+}
+
+iree_status_t iree_hal_hip_timepoint_pool_acquire_host_wait(
+    iree_hal_hip_timepoint_pool_t* timepoint_pool,
+    iree_host_size_t timepoint_count,
+    iree_hal_hip_timepoint_t** out_timepoints) {
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  // Acquire host events to wrap up. This should happen before acquiring the
+  // timepoints to avoid nested locks.
+  iree_event_t* host_events = iree_alloca(
+      timepoint_count * sizeof((*out_timepoints)->timepoint.host_wait));
+  IREE_RETURN_AND_END_ZONE_IF_ERROR(
+      z0, iree_event_pool_acquire(timepoint_pool->host_event_pool,
+                                  timepoint_count, host_events));
+
+  IREE_RETURN_AND_END_ZONE_IF_ERROR(
+      z0, iree_hal_hip_timepoint_pool_acquire_internal(
+              timepoint_pool, timepoint_count, out_timepoints));
+  for (iree_host_size_t i = 0; i < timepoint_count; ++i) {
+    out_timepoints[i]->kind = IREE_HAL_HIP_TIMEPOINT_KIND_HOST_WAIT;
+    out_timepoints[i]->timepoint.host_wait = host_events[i];
+  }
+
+  IREE_TRACE_ZONE_END(z0);
+  return iree_ok_status();
+}
+
+iree_status_t iree_hal_hip_timepoint_pool_acquire_device_signal(
+    iree_hal_hip_timepoint_pool_t* timepoint_pool,
+    iree_host_size_t timepoint_count,
+    iree_hal_hip_timepoint_t** out_timepoints) {
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  // Acquire device events to wrap up. This should happen before acquiring the
+  // timepoints to avoid nested locks.
+  iree_hal_hip_event_t** device_events = iree_alloca(
+      timepoint_count * sizeof((*out_timepoints)->timepoint.device_signal));
+  IREE_RETURN_AND_END_ZONE_IF_ERROR(
+      z0, iree_hal_hip_event_pool_acquire(timepoint_pool->device_event_pool,
+                                          timepoint_count, device_events));
+
+  IREE_RETURN_AND_END_ZONE_IF_ERROR(
+      z0, iree_hal_hip_timepoint_pool_acquire_internal(
+              timepoint_pool, timepoint_count, out_timepoints));
+  for (iree_host_size_t i = 0; i < timepoint_count; ++i) {
+    out_timepoints[i]->kind = IREE_HAL_HIP_TIMEPOINT_KIND_DEVICE_SIGNAL;
+    out_timepoints[i]->timepoint.device_signal = device_events[i];
+  }
+
+  IREE_TRACE_ZONE_END(z0);
+  return iree_ok_status();
+}
+
+iree_status_t iree_hal_hip_timepoint_pool_acquire_device_wait(
+    iree_hal_hip_timepoint_pool_t* timepoint_pool,
+    iree_host_size_t timepoint_count,
+    iree_hal_hip_timepoint_t** out_timepoints) {
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  // Acquire device events to wrap up. This should happen before acquiring the
+  // timepoints to avoid nested locks.
+  iree_hal_hip_event_t** device_events = iree_alloca(
+      timepoint_count * sizeof((*out_timepoints)->timepoint.device_wait));
+  IREE_RETURN_AND_END_ZONE_IF_ERROR(
+      z0, iree_hal_hip_event_pool_acquire(timepoint_pool->device_event_pool,
+                                          timepoint_count, device_events));
+
+  IREE_RETURN_AND_END_ZONE_IF_ERROR(
+      z0, iree_hal_hip_timepoint_pool_acquire_internal(
+              timepoint_pool, timepoint_count, out_timepoints));
+  for (iree_host_size_t i = 0; i < timepoint_count; ++i) {
+    out_timepoints[i]->kind = IREE_HAL_HIP_TIMEPOINT_KIND_DEVICE_WAIT;
+    out_timepoints[i]->timepoint.device_wait = device_events[i];
+  }
+
+  IREE_TRACE_ZONE_END(z0);
+  return iree_ok_status();
+}
+
+void iree_hal_hip_timepoint_pool_release(
+    iree_hal_hip_timepoint_pool_t* timepoint_pool,
+    iree_host_size_t timepoint_count, iree_hal_hip_timepoint_t** timepoints) {
+  IREE_ASSERT_ARGUMENT(timepoint_pool);
+  if (!timepoint_count) return;
+  IREE_ASSERT_ARGUMENT(timepoints);
+  IREE_TRACE_ZONE_BEGIN(z0);
+
+  // Release the wrapped host/device events. This should happen before acquiring
+  // the timepoint pool's lock given that the host/device event pool its
+  // internal lock too.
+  // TODO: Release in batch to avoid lock overhead from separate calls.
+  for (iree_host_size_t i = 0; i < timepoint_count; ++i) {
+    switch (timepoints[i]->kind) {
+      case IREE_HAL_HIP_TIMEPOINT_KIND_HOST_WAIT:
+        iree_event_pool_release(timepoint_pool->host_event_pool, 1,
+                                &timepoints[i]->timepoint.host_wait);
+        break;
+      case IREE_HAL_HIP_TIMEPOINT_KIND_DEVICE_SIGNAL:
+        iree_hal_hip_event_release(timepoints[i]->timepoint.device_signal);
+        break;
+      case IREE_HAL_HIP_TIMEPOINT_KIND_DEVICE_WAIT:
+        iree_hal_hip_event_release(timepoints[i]->timepoint.device_wait);
+        break;
+      default:
+        break;
+    }
+  }
+
+  // We'll try to release all we can back to the pool and then deinitialize
+  // the ones that won't fit.
+  iree_host_size_t remaining_count = timepoint_count;
+
+  // Try first to release to the pool.
+  iree_slim_mutex_lock(&timepoint_pool->timepoint_mutex);
+  iree_host_size_t to_pool_count = iree_min(
+      timepoint_pool->available_capacity - timepoint_pool->available_count,
+      timepoint_count);
+  if (to_pool_count > 0) {
+    for (iree_host_size_t i = 0; i < to_pool_count; ++i) {
+      iree_hal_hip_timepoint_clear(timepoints[i]);
+    }
+    iree_host_size_t pool_base_index = timepoint_pool->available_count;
+    memcpy(&timepoint_pool->available_list[pool_base_index], timepoints,
+           to_pool_count * sizeof(*timepoint_pool->available_list));
+    timepoint_pool->available_count += to_pool_count;
+    remaining_count -= to_pool_count;
+  }
+  iree_slim_mutex_unlock(&timepoint_pool->timepoint_mutex);
+
+  // Deallocate the rest of the timepoints. We don't bother resetting them as we
+  // are getting rid of them.
+  if (remaining_count > 0) {
+    IREE_TRACE_ZONE_BEGIN_NAMED(z1, "timepoint-pool-unpooled-release");
+    for (iree_host_size_t i = 0; i < remaining_count; ++i) {
+      iree_hal_hip_timepoint_clear(timepoints[to_pool_count + i]);
+      iree_hal_hip_timepoint_free(timepoints[to_pool_count + i]);
+    }
+    IREE_TRACE_ZONE_END(z1);
+  }
+  IREE_TRACE_ZONE_END(z0);
+}
diff --git a/experimental/hip/timepoint_pool.h b/experimental/hip/timepoint_pool.h
new file mode 100644
index 0000000..a4e4e5a
--- /dev/null
+++ b/experimental/hip/timepoint_pool.h
@@ -0,0 +1,119 @@
+// Copyright 2024 The IREE Authors
+//
+// Licensed under the Apache License v2.0 with LLVM Exceptions.
+// See https://llvm.org/LICENSE.txt for license information.
+// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
+
+#ifndef IREE_EXPERIMENTAL_HIP_TIMEPOINT_POOL_H_
+#define IREE_EXPERIMENTAL_HIP_TIMEPOINT_POOL_H_
+
+#include "experimental/hip/event_pool.h"
+#include "iree/base/api.h"
+#include "iree/base/internal/event_pool.h"
+#include "iree/hal/utils/semaphore_base.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif  // __cplusplus
+
+//===----------------------------------------------------------------------===//
+// iree_hal_hip_timepoint_t
+//===----------------------------------------------------------------------===//
+
+// Forward declaration of the timepoint pool.
+typedef struct iree_hal_hip_timepoint_pool_t iree_hal_hip_timepoint_pool_t;
+
+// An enum to identify the timepoint kind in iree_hal_hip_timepoint_t objects.
+typedef enum iree_hal_hip_timepoint_kind_e {
+  // None; for uninitialized timepoint objects.
+  IREE_HAL_HIP_TIMEPOINT_KIND_NONE = 0,
+  // A timepoint waited by the host.
+  IREE_HAL_HIP_TIMEPOINT_KIND_HOST_WAIT,
+  // A timepoint signaled by the device.
+  IREE_HAL_HIP_TIMEPOINT_KIND_DEVICE_SIGNAL,
+  // A timepoint waited by the device.
+  IREE_HAL_HIP_TIMEPOINT_KIND_DEVICE_WAIT,
+} iree_hal_hip_timepoint_kind_t;
+
+// An object that wraps a host iree_event_t or device iree_hal_hip_event_t to
+// represent wait/signal of a timepoint on a timeline.
+//
+// iree_hal_hip_timepoint_t objects cannot be directly created; it should be
+// acquired from the timeline pool and released back to the pool once done.
+//
+// Thread-compatible; a timepoint is typically only accessed by one thread.
+typedef struct iree_hal_hip_timepoint_t {
+  // Base timepoint structure providing intrusive linked list pointers and
+  // timepoint callback mechanisms.
+  iree_hal_semaphore_timepoint_t base;
+
+  // The allocator used to create the timepoint.
+  iree_allocator_t host_allocator;
+
+  // The timepoint pool that owns this timepoint.
+  iree_hal_hip_timepoint_pool_t* pool;
+
+  iree_hal_hip_timepoint_kind_t kind;
+  union {
+    iree_event_t host_wait;
+    iree_hal_hip_event_t* device_signal;
+    // The device event to wait. NULL means no device event available to wait
+    // for this timepoint at the moment.
+    iree_hal_hip_event_t* device_wait;
+  } timepoint;
+} iree_hal_hip_timepoint_t;
+
+//===----------------------------------------------------------------------===//
+// iree_hal_hip_timepoint_pool_t
+//===----------------------------------------------------------------------===//
+
+// A simple pool of iree_hal_hip_timepoint_t objects to recycle.
+//
+// Thread-safe; multiple threads may acquire and release timepoints from the
+// pool.
+typedef struct iree_hal_hip_timepoint_pool_t iree_hal_hip_timepoint_pool_t;
+
+// Allocates a new timepoint pool with up to |available_capacity| timepoints.
+//
+// Extra timepoint requests beyond the capability are directly created and
+// destroyed without pooling.
+iree_status_t iree_hal_hip_timepoint_pool_allocate(
+    iree_event_pool_t* host_event_pool,
+    iree_hal_hip_event_pool_t* device_event_pool,
+    iree_host_size_t available_capacity, iree_allocator_t host_allocator,
+    iree_hal_hip_timepoint_pool_t** out_timepoint_pool);
+
+// Deallocates a timepoint pool and destroys all timepoints.
+//
+// All timepoints that were acquired from the pool must have already been
+// released back to it prior to deallocation.
+void iree_hal_hip_timepoint_pool_free(
+    iree_hal_hip_timepoint_pool_t* timepoint_pool);
+
+// Acquires one or more timepoints from the timepoint pool.
+//
+// |out_timepoints| are owned by the caller and must be kept live until the
+// timepoints have been reached, or cancelled by the caller.
+iree_status_t iree_hal_hip_timepoint_pool_acquire_host_wait(
+    iree_hal_hip_timepoint_pool_t* timepoint_pool,
+    iree_host_size_t timepoint_count,
+    iree_hal_hip_timepoint_t** out_timepoints);
+iree_status_t iree_hal_hip_timepoint_pool_acquire_device_signal(
+    iree_hal_hip_timepoint_pool_t* timepoint_pool,
+    iree_host_size_t timepoint_count,
+    iree_hal_hip_timepoint_t** out_timepoints);
+iree_status_t iree_hal_hip_timepoint_pool_acquire_device_wait(
+    iree_hal_hip_timepoint_pool_t* timepoint_pool,
+    iree_host_size_t timepoint_count,
+    iree_hal_hip_timepoint_t** out_timepoints);
+
+// Releases one or more timepoints back to the timepoint pool.
+void iree_hal_hip_timepoint_pool_release(
+    iree_hal_hip_timepoint_pool_t* timepoint_pool,
+    iree_host_size_t timepoint_count, iree_hal_hip_timepoint_t** timepoints);
+
+#ifdef __cplusplus
+}  // extern "C"
+#endif  // __cplusplus
+
+#endif  // IREE_EXPERIMENTAL_HIP_TIMEPOINT_POOL_H_