Adding a new iree/hal/local/ path as the local CPU driver.
This subsumes the existing iree/hal/host/ and will allow us to share the
same device for all local execution - including mixing and matching
executables in different forms (AOT, vmla, JIT, etc) within the same
device. It also adds the (initial, unoptimized) task-system based
command buffer and semaphore implementations.
diff --git a/iree/base/atomic_slist.c b/iree/base/atomic_slist.c
index c495c7e..025eeba 100644
--- a/iree/base/atomic_slist.c
+++ b/iree/base/atomic_slist.c
@@ -59,7 +59,10 @@
iree_atomic_slist_entry_t* iree_atomic_slist_pop(iree_atomic_slist_t* list) {
iree_slim_mutex_lock(&list->mutex);
iree_atomic_slist_entry_t* entry = list->head;
- list->head = entry ? entry->next : NULL;
+ if (entry != NULL) {
+ list->head = entry->next;
+ entry->next = NULL;
+ }
iree_slim_mutex_unlock(&list->mutex);
return entry;
}
diff --git a/iree/hal/cts/semaphore_test.cc b/iree/hal/cts/semaphore_test.cc
index 3eae1f3..d2a31fa 100644
--- a/iree/hal/cts/semaphore_test.cc
+++ b/iree/hal/cts/semaphore_test.cc
@@ -281,11 +281,11 @@
}
TEST_P(SemaphoreTest, SubmitWithWait) {
+ // Empty command buffer.
iree_hal_command_buffer_t* command_buffer;
IREE_ASSERT_OK(iree_hal_command_buffer_create(
device_, IREE_HAL_COMMAND_BUFFER_MODE_ONE_SHOT,
IREE_HAL_COMMAND_CATEGORY_DISPATCH, &command_buffer));
-
IREE_ASSERT_OK(iree_hal_command_buffer_begin(command_buffer));
IREE_ASSERT_OK(iree_hal_command_buffer_end(command_buffer));
@@ -294,11 +294,11 @@
iree_hal_semaphore_t* wait_semaphore;
iree_hal_semaphore_t* signal_semaphore;
IREE_ASSERT_OK(iree_hal_semaphore_create(device_, 0ull, &wait_semaphore));
- IREE_ASSERT_OK(iree_hal_semaphore_create(device_, 0ull, &signal_semaphore));
+ IREE_ASSERT_OK(iree_hal_semaphore_create(device_, 100ull, &signal_semaphore));
iree_hal_semaphore_t* wait_semaphore_ptrs[] = {wait_semaphore};
iree_hal_semaphore_t* signal_semaphore_ptrs[] = {signal_semaphore};
uint64_t wait_payload_values[] = {1ull};
- uint64_t signal_payload_values[] = {1ull};
+ uint64_t signal_payload_values[] = {101ull};
submission_batch.wait_semaphores.count = IREE_ARRAYSIZE(wait_semaphore_ptrs);
submission_batch.wait_semaphores.semaphores = wait_semaphore_ptrs;
submission_batch.wait_semaphores.payload_values = wait_payload_values;
@@ -317,12 +317,12 @@
// Work shouldn't start until the wait semaphore reaches its payload value.
uint64_t value;
IREE_ASSERT_OK(iree_hal_semaphore_query(signal_semaphore, &value));
- EXPECT_EQ(0ull, value);
+ EXPECT_EQ(100ull, value);
// Signal the wait semaphore, work should begin and complete.
IREE_ASSERT_OK(iree_hal_semaphore_signal(wait_semaphore, 1ull));
IREE_ASSERT_OK(iree_hal_semaphore_wait_with_deadline(
- signal_semaphore, 1ull, IREE_TIME_INFINITE_FUTURE));
+ signal_semaphore, 101ull, IREE_TIME_INFINITE_FUTURE));
iree_hal_command_buffer_release(command_buffer);
iree_hal_semaphore_release(wait_semaphore);
diff --git a/iree/hal/local/BUILD b/iree/hal/local/BUILD
index d768f38..82c6416 100644
--- a/iree/hal/local/BUILD
+++ b/iree/hal/local/BUILD
@@ -38,3 +38,84 @@
"//iree/base:synchronization",
],
)
+
+# TODO(benvanik): move into base/? may be useful for other backends or for other
+# parts of the system (like modules handling IO/RPC).
+cc_library(
+ name = "event_pool",
+ srcs = ["event_pool.c"],
+ hdrs = ["event_pool.h"],
+ deps = [
+ "//iree/base:api",
+ "//iree/base:core_headers",
+ "//iree/base:synchronization",
+ "//iree/base:tracing",
+ "//iree/base:wait_handle",
+ ],
+)
+
+cc_library(
+ name = "executable_library",
+ hdrs = ["executable_library.h"],
+)
+
+cc_library(
+ name = "local",
+ srcs = [
+ "executable_loader.c",
+ "local_descriptor_set.c",
+ "local_descriptor_set_layout.c",
+ "local_executable.c",
+ "local_executable_cache.c",
+ "local_executable_layout.c",
+ ],
+ hdrs = [
+ "executable_loader.h",
+ "local_descriptor_set.h",
+ "local_descriptor_set_layout.h",
+ "local_executable.h",
+ "local_executable_cache.h",
+ "local_executable_layout.h",
+ ],
+ deps = [
+ ":executable_library",
+ "//iree/base:api",
+ "//iree/base:core_headers",
+ "//iree/base:tracing",
+ "//iree/hal:api",
+ ],
+)
+
+cc_library(
+ name = "task_driver",
+ srcs = [
+ "task_command_buffer.c",
+ "task_device.c",
+ "task_driver.c",
+ "task_event.c",
+ "task_queue.c",
+ "task_queue_state.c",
+ "task_semaphore.c",
+ ],
+ hdrs = [
+ "task_command_buffer.h",
+ "task_device.h",
+ "task_driver.h",
+ "task_event.h",
+ "task_queue.h",
+ "task_queue_state.h",
+ "task_semaphore.h",
+ ],
+ deps = [
+ ":arena",
+ ":event_pool",
+ ":local",
+ "//iree/base:api",
+ "//iree/base:core_headers",
+ "//iree/base:synchronization",
+ "//iree/base:tracing",
+ "//iree/base:wait_handle",
+ "//iree/hal:api",
+ "//iree/task",
+ ],
+)
diff --git a/iree/hal/local/CMakeLists.txt b/iree/hal/local/CMakeLists.txt
index 486e1e6..3eb8267 100644
--- a/iree/hal/local/CMakeLists.txt
+++ b/iree/hal/local/CMakeLists.txt
@@ -28,3 +28,86 @@
iree::base::synchronization
PUBLIC
)
+
+iree_cc_library(
+ NAME
+ event_pool
+ HDRS
+ "event_pool.h"
+ SRCS
+ "event_pool.c"
+ DEPS
+ iree::base::api
+ iree::base::core_headers
+ iree::base::synchronization
+ iree::base::tracing
+ iree::base::wait_handle
+ PUBLIC
+)
+
+iree_cc_library(
+ NAME
+ executable_library
+ HDRS
+ "executable_library.h"
+ PUBLIC
+)
+
+iree_cc_library(
+ NAME
+ local
+ HDRS
+ "executable_loader.h"
+ "local_descriptor_set.h"
+ "local_descriptor_set_layout.h"
+ "local_executable.h"
+ "local_executable_cache.h"
+ "local_executable_layout.h"
+ SRCS
+ "executable_loader.c"
+ "local_descriptor_set.c"
+ "local_descriptor_set_layout.c"
+ "local_executable.c"
+ "local_executable_cache.c"
+ "local_executable_layout.c"
+ DEPS
+ ::executable_library
+ iree::base::api
+ iree::base::core_headers
+ iree::base::tracing
+ iree::hal::api
+ PUBLIC
+)
+
+iree_cc_library(
+ NAME
+ task_driver
+ HDRS
+ "task_command_buffer.h"
+ "task_device.h"
+ "task_driver.h"
+ "task_event.h"
+ "task_queue.h"
+ "task_queue_state.h"
+ "task_semaphore.h"
+ SRCS
+ "task_command_buffer.c"
+ "task_device.c"
+ "task_driver.c"
+ "task_event.c"
+ "task_queue.c"
+ "task_queue_state.c"
+ "task_semaphore.c"
+ DEPS
+ ::arena
+ ::event_pool
+ ::local
+ iree::base::api
+ iree::base::core_headers
+ iree::base::synchronization
+ iree::base::tracing
+ iree::base::wait_handle
+ iree::hal::api
+ iree::task
+ PUBLIC
+)
diff --git a/iree/hal/local/arena.c b/iree/hal/local/arena.c
index 4ef3c60..7b9529a 100644
--- a/iree/hal/local/arena.c
+++ b/iree/hal/local/arena.c
@@ -62,8 +62,12 @@
// case we may end up allocating a block when perhaps we didn't need to but
// that's fine - it's just one block and the contention means there's likely
// to be a need for more anyway.
- IREE_RETURN_IF_ERROR(iree_allocator_malloc(
- block_pool->block_allocator, block_pool->total_block_size, &block));
+ uint8_t* block_base = NULL;
+ IREE_RETURN_IF_ERROR(iree_allocator_malloc(block_pool->block_allocator,
+ block_pool->total_block_size,
+ (void**)&block_base));
+ block = (iree_arena_block_t*)(block_base + (block_pool->total_block_size -
+ sizeof(iree_arena_block_t)));
}
block->next = NULL;
@@ -174,7 +178,7 @@
iree_allocator_t iree_arena_allocator(iree_arena_allocator_t* arena) {
iree_allocator_t v = {
.self = arena,
- .alloc = (iree_allocator_alloc_fn_t)iree_arena_allocate,
+ .alloc = (iree_allocator_alloc_fn_t)iree_arena_allocate_thunk,
.free = NULL,
};
return v;
diff --git a/iree/hal/local/event_pool.c b/iree/hal/local/event_pool.c
new file mode 100644
index 0000000..0a6c1dd
--- /dev/null
+++ b/iree/hal/local/event_pool.c
@@ -0,0 +1,172 @@
+// Copyright 2021 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "iree/hal/local/event_pool.h"
+
+#include "iree/base/debugging.h"
+#include "iree/base/synchronization.h"
+#include "iree/base/tracing.h"
+
+struct iree_hal_local_event_pool_s {
+ // Allocator used to create the event pool.
+ iree_allocator_t host_allocator;
+ // Guards the pool. Since this pool is used to get operating system-level
+ // event objects that will be signaled and waited on using syscalls it's got
+ // relatively low contention: callers are rate limited by how fast they can
+ // signal and wait on the events they get.
+ iree_slim_mutex_t mutex;
+ // Maximum number of events that will be maintained in the pool. More events
+ // may be allocated at any time but when they are no longer needed they will
+ // be disposed directly.
+ iree_host_size_t available_capacity;
+ // Total number of available
+ iree_host_size_t available_count;
+ // Dense left-aligned list of available_count events.
+ iree_event_t available_list[];
+};
+
+iree_status_t iree_hal_local_event_pool_allocate(
+ iree_host_size_t available_capacity, iree_allocator_t host_allocator,
+ iree_hal_local_event_pool_t** out_event_pool) {
+ IREE_ASSERT_ARGUMENT(out_event_pool);
+ *out_event_pool = NULL;
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ iree_hal_local_event_pool_t* event_pool = NULL;
+ iree_host_size_t total_size =
+ sizeof(*event_pool) +
+ available_capacity * sizeof(event_pool->available_list[0]);
+ IREE_RETURN_AND_END_ZONE_IF_ERROR(
+ z0,
+ iree_allocator_malloc(host_allocator, total_size, (void**)&event_pool));
+ event_pool->host_allocator = host_allocator;
+ 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_event_initialize(
+ /*initial_state=*/false,
+ &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_local_event_pool_free(event_pool);
+ }
+ IREE_TRACE_ZONE_END(z0);
+ return status;
+}
+
+void iree_hal_local_event_pool_free(iree_hal_local_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_event_deinitialize(&event_pool->available_list[i]);
+ }
+ iree_slim_mutex_deinitialize(&event_pool->mutex);
+ iree_allocator_free(host_allocator, event_pool);
+
+ IREE_TRACE_ZONE_END(z0);
+}
+
+iree_status_t iree_hal_local_event_pool_acquire(
+ iree_hal_local_event_pool_t* event_pool, iree_host_size_t event_count,
+ iree_event_t* out_events) {
+ IREE_ASSERT_ARGUMENT(event_pool);
+ if (!event_count) return iree_ok_status();
+ IREE_ASSERT_ARGUMENT(out_events);
+
+ // We'll try to get what we can from the pool and fall back to initializing
+ // new events.
+ iree_host_size_t remaining_count = event_count;
+
+ // Try first to grab from the pool.
+ iree_slim_mutex_lock(&event_pool->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(iree_event_t));
+ event_pool->available_count -= from_pool_count;
+ remaining_count -= from_pool_count;
+ }
+ iree_slim_mutex_unlock(&event_pool->mutex);
+
+ // Allocate the rest of the events.
+ if (remaining_count > 0) {
+ IREE_TRACE_ZONE_BEGIN(z0);
+ iree_status_t status = iree_ok_status();
+ for (iree_host_size_t i = 0; i < remaining_count; ++i) {
+ status = iree_event_initialize(/*initial_state=*/false,
+ &out_events[from_pool_count + i]);
+ if (!iree_status_is_ok(status)) {
+ // Must release all events we've acquired so far.
+ iree_hal_local_event_pool_release(event_pool, from_pool_count + i,
+ out_events);
+ IREE_TRACE_ZONE_END(z0);
+ return status;
+ }
+ }
+ IREE_TRACE_ZONE_END(z0);
+ }
+
+ return iree_ok_status();
+}
+
+void iree_hal_local_event_pool_release(iree_hal_local_event_pool_t* event_pool,
+ iree_host_size_t event_count,
+ iree_event_t* events) {
+ IREE_ASSERT_ARGUMENT(event_pool);
+ if (!event_count) return;
+ IREE_ASSERT_ARGUMENT(events);
+
+ // 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.
+ // Note that we reset the events we add back to the pool so that they are
+ // ready to be acquired again.
+ iree_slim_mutex_lock(&event_pool->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) {
+ iree_host_size_t pool_base_index = event_pool->available_count;
+ for (iree_host_size_t i = 0; i < to_pool_count; ++i) {
+ iree_event_reset(&events[i]);
+ }
+ memcpy(&event_pool->available_list[pool_base_index], events,
+ to_pool_count * sizeof(iree_event_t));
+ event_pool->available_count += to_pool_count;
+ remaining_count -= to_pool_count;
+ }
+ iree_slim_mutex_unlock(&event_pool->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(z0);
+ for (iree_host_size_t i = 0; i < remaining_count; ++i) {
+ iree_event_deinitialize(&events[to_pool_count + i]);
+ }
+ IREE_TRACE_ZONE_END(z0);
+ }
+}
diff --git a/iree/hal/local/event_pool.h b/iree/hal/local/event_pool.h
new file mode 100644
index 0000000..6a4dd9c
--- /dev/null
+++ b/iree/hal/local/event_pool.h
@@ -0,0 +1,57 @@
+// Copyright 2021 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef IREE_HAL_LOCAL_EVENT_POOL_H_
+#define IREE_HAL_LOCAL_EVENT_POOL_H_
+
+#include "iree/base/api.h"
+#include "iree/base/wait_handle.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif // __cplusplus
+
+// A simple pool of iree_event_ts to recycle.
+//
+// Thread-safe; multiple threads may acquire and release events from the pool.
+typedef struct iree_hal_local_event_pool_s iree_hal_local_event_pool_t;
+
+// Allocates a new event pool with up to |available_capacity| events.
+iree_status_t iree_hal_local_event_pool_allocate(
+ iree_host_size_t available_capacity, iree_allocator_t host_allocator,
+ iree_hal_local_event_pool_t** out_event_pool);
+
+// Deallocates an event pool and destroys all events.
+// All events that were acquired from the pool must have already been released
+// back to it prior to deallocation.
+void iree_hal_local_event_pool_free(iree_hal_local_event_pool_t* event_pool);
+
+// Acquires one or more events from the event pool.
+// The returned events will be unsignaled and ready for use. Callers may set and
+// reset the events as much as they want prior to releasing them back to the
+// pool with iree_hal_local_event_pool_release.
+iree_status_t iree_hal_local_event_pool_acquire(
+ iree_hal_local_event_pool_t* event_pool, iree_host_size_t event_count,
+ iree_event_t* out_events);
+
+// Releases one or more events back to the block pool.
+void iree_hal_local_event_pool_release(iree_hal_local_event_pool_t* event_pool,
+ iree_host_size_t event_count,
+ iree_event_t* events);
+
+#ifdef __cplusplus
+} // extern "C"
+#endif // __cplusplus
+
+#endif // IREE_HAL_LOCAL_EVENT_POOL_H_
diff --git a/iree/hal/local/executable_library.h b/iree/hal/local/executable_library.h
new file mode 100644
index 0000000..a8ec8b2
--- /dev/null
+++ b/iree/hal/local/executable_library.h
@@ -0,0 +1,147 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef IREE_HAL_LOCAL_EXECUTABLE_LIBRARY_H_
+#define IREE_HAL_LOCAL_EXECUTABLE_LIBRARY_H_
+
+// NOTE: this file is designed to be a standalone header: it is embedded in the
+// compiler and must not take any dependences on the runtime HAL code.
+// Changes here will require changes to the compiler and must be versioned as if
+// this was a schema: backwards-incompatible changes require version bumps or
+// the ability to feature-detect at runtime.
+
+#include <stddef.h>
+#include <stdint.h>
+
+//===----------------------------------------------------------------------===//
+// Versioning and interface querying
+//===----------------------------------------------------------------------===//
+
+// Known valid version values.
+enum iree_hal_executable_library_version_e {
+ // iree_hal_executable_library_v0_t is used as the API communication
+ // structure.
+ IREE_HAL_EXECUTABLE_LIBRARY_VERSION_0 = 0u,
+};
+typedef uint32_t iree_hal_executable_library_version_t;
+
+// The latest version of the library API; can be used to populate the
+// iree_hal_executable_library_header_t::version when building libraries.
+#define IREE_HAL_EXECUTABLE_LIBRARY_LATEST_VERSION \
+ IREE_HAL_EXECUTABLE_LIBRARY_VERSION_0
+
+// A header present at the top of all versions of the library API used by the
+// runtime to ensure version compatibility.
+typedef struct {
+ // Version of the API this library was built with, which was likely the value
+ // of IREE_HAL_EXECUTABLE_LIBRARY_LATEST_VERSION.
+ iree_hal_executable_library_version_t version;
+
+ // Name used for logging/diagnostics.
+ const char* name;
+} iree_hal_executable_library_header_t;
+
+// Exported function from dynamic libraries for querying library information.
+// The provided |max_version| is the maximum version the caller supports;
+// callees must return NULL if their lowest available version is greater
+// than the max version supported by the caller.
+typedef const iree_hal_executable_library_header_t* (
+ *iree_hal_executable_library_query_fn_t)(
+ iree_hal_executable_library_version_t max_version);
+
+// Function name exported from dynamic libraries (pass to dlsym).
+#define IREE_HAL_EXECUTABLE_LIBRARY_EXPORT_NAME \
+ "iree_hal_executable_library_query"
+
+//===----------------------------------------------------------------------===//
+// IREE_HAL_EXECUTABLE_LIBRARY_VERSION_0
+//===----------------------------------------------------------------------===//
+
+// Read-only per-dispatch state passed to each tile in a dispatch.
+typedef struct {
+ uint32_t reserved;
+} iree_hal_executable_dispatch_state_v0_t;
+
+typedef union {
+ struct {
+ uint32_t x;
+ uint32_t y;
+ uint32_t z;
+ };
+ uint32_t value[3];
+} iree_hal_vec3_t;
+
+#if defined(_MSC_VER)
+typedef __declspec(
+ align(16)) const uint32_t* iree_hal_executable_push_constants_ptr_t;
+#else
+typedef const uint32_t* iree_hal_executable_push_constants_ptr_t
+ __attribute__((align_value(16)));
+#endif // MSVC
+
+typedef void* iree_hal_executable_binding_ptr_t;
+
+// Function signature of exported executable entry points.
+// The same |state| is passed to all tiles in a dispatch, with other arguments
+// such as |workgroup_id| varying per-tile (counting to the |workgroup_count|).
+// Each tile represents |workgroup_size| local invocations in the global
+// |workgroup_count| grid.
+//
+// 0 or more push constants are available at |push_constants| with the count
+// being determined by the sidechannel information provided by the compiler.
+//
+// The |bindings| list is a dense set of pointers to I/O data with the count and
+// ordering determined by the compiler.
+typedef void (*iree_hal_executable_dispatch_v0_t)(
+ const iree_hal_executable_dispatch_state_v0_t* state,
+ const iree_hal_vec3_t* workgroup_id, const iree_hal_vec3_t* workgroup_size,
+ const iree_hal_vec3_t* workgroup_count,
+ const iree_hal_executable_push_constants_ptr_t push_constants,
+ const iree_hal_executable_binding_ptr_t* bindings);
+
+// Structure used for v0 library interfaces.
+// The entire structure is designed to be read-only and able to live embedded in
+// the binary .rdata section.
+//
+// Implementations may still choose to heap allocate this structure and modify
+// at runtime so long as they observe the thread-safety guarantees. For example,
+// a JIT may default all entry_points to JIT thunk functions and then swap them
+// out for the translated function pointers.
+typedef struct {
+ // Version/metadata header. Will have a version of
+ // IREE_HAL_EXECUTABLE_LIBRARY_VERSION_0.
+ const iree_hal_executable_library_header_t* header;
+
+ // The total number of entry points available in the library. Bounds all of
+ // the tables below.
+ uint32_t entry_point_count;
+
+ // Table of export function entry points matching the ordinals defined during
+ // library generation. The runtime will use this table to map the ordinals to
+ // function pointers for execution.
+ const iree_hal_executable_dispatch_v0_t* entry_points;
+
+ // Optional table of export function entry point names 1:1 with entry_points.
+ // These names are only used for tracing/debugging and can be omitted to save
+ // binary size.
+ const char** entry_point_names;
+
+ // Optional table of entry point tags that describe the entry point in a
+ // human-readable format useful for verbose logging. The string values, when
+ // present, may be attached to tracing/debugging events related to the entry
+ // point.
+ const char** entry_point_tags;
+} iree_hal_executable_library_v0_t;
+
+#endif // IREE_HAL_LOCAL_EXECUTABLE_LIBRARY_H_
diff --git a/iree/hal/local/executable_loader.c b/iree/hal/local/executable_loader.c
new file mode 100644
index 0000000..b3a4005
--- /dev/null
+++ b/iree/hal/local/executable_loader.c
@@ -0,0 +1,60 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "iree/hal/local/executable_loader.h"
+
+void iree_hal_executable_loader_initialize(
+ const void* vtable, iree_hal_executable_loader_t* out_base_loader) {
+ iree_atomic_ref_count_init(&out_base_loader->ref_count);
+ out_base_loader->vtable = vtable;
+}
+
+void iree_hal_executable_loader_retain(
+ iree_hal_executable_loader_t* executable_loader) {
+ if (IREE_LIKELY(executable_loader)) {
+ iree_atomic_ref_count_inc(&executable_loader->ref_count);
+ }
+}
+
+void iree_hal_executable_loader_release(
+ iree_hal_executable_loader_t* executable_loader) {
+ if (IREE_LIKELY(executable_loader) &&
+ iree_atomic_ref_count_dec(&executable_loader->ref_count) == 1) {
+ executable_loader->vtable->destroy(executable_loader);
+ }
+}
+
+bool iree_hal_executable_loader_query_support(
+ iree_hal_executable_loader_t* executable_loader,
+ iree_hal_executable_format_t executable_format,
+ iree_hal_executable_caching_mode_t caching_mode) {
+ IREE_ASSERT_ARGUMENT(executable_loader);
+ return executable_loader->vtable->query_support(
+ executable_loader, executable_format, caching_mode);
+}
+
+iree_status_t iree_hal_executable_loader_try_load(
+ iree_hal_executable_loader_t* executable_loader,
+ iree_hal_executable_layout_t* executable_layout,
+ iree_hal_executable_format_t executable_format,
+ iree_hal_executable_caching_mode_t caching_mode,
+ iree_const_byte_span_t executable_data,
+ iree_hal_executable_t** out_executable) {
+ IREE_ASSERT_ARGUMENT(executable_loader);
+ IREE_ASSERT_ARGUMENT(executable_data.data);
+ IREE_ASSERT_ARGUMENT(out_executable);
+ return executable_loader->vtable->try_load(
+ executable_loader, executable_layout, executable_format, caching_mode,
+ executable_data, out_executable);
+}
diff --git a/iree/hal/local/executable_loader.h b/iree/hal/local/executable_loader.h
new file mode 100644
index 0000000..7c18ff5
--- /dev/null
+++ b/iree/hal/local/executable_loader.h
@@ -0,0 +1,116 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef IREE_HAL_LOCAL_EXECUTABLE_LOADER_H_
+#define IREE_HAL_LOCAL_EXECUTABLE_LOADER_H_
+
+#include <stdbool.h>
+#include <stdint.h>
+
+#include "iree/base/api.h"
+#include "iree/base/atomics.h"
+#include "iree/hal/api.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif // __cplusplus
+
+//===----------------------------------------------------------------------===//
+// iree_hal_executable_loader_t
+//===----------------------------------------------------------------------===//
+
+typedef struct iree_hal_executable_loader_vtable_s
+ iree_hal_executable_loader_vtable_t;
+
+// Interface for compiled executable loader implementations.
+// A loader may be as simple as something that resolves function pointers in the
+// local executable for statically linked executables or as complex as a custom
+// relocatable ELF loader. Loaders are registered and persist for each device
+// they are attached to and may keep internal caches or memoize resources shared
+// by multiple loaded executables.
+//
+// Thread-safe - multiple threads may load executables (including the *same*
+// executable) simultaneously.
+typedef struct {
+ iree_atomic_ref_count_t ref_count;
+ const iree_hal_executable_loader_vtable_t* vtable;
+} iree_hal_executable_loader_t;
+
+// Initializes the base iree_hal_executable_loader_t type.
+// Called by subclasses upon allocating their loader.
+void iree_hal_executable_loader_initialize(
+ const void* vtable, iree_hal_executable_loader_t* out_base_loader);
+
+// Retains the given |executable_loader| for the caller.
+void iree_hal_executable_loader_retain(
+ iree_hal_executable_loader_t* executable_loader);
+
+// Releases the given |executable_loader| from the caller.
+void iree_hal_executable_loader_release(
+ iree_hal_executable_loader_t* executable_loader);
+
+// Returns true if the loader can load executables of the given
+// |executable_format|. Note that loading may still fail if the executable uses
+// features not available on the current host or runtime.
+bool iree_hal_executable_loader_query_support(
+ iree_hal_executable_loader_t* executable_loader,
+ iree_hal_executable_format_t executable_format,
+ iree_hal_executable_caching_mode_t caching_mode);
+
+// Tries loading the |executable_data| provided in the given
+// |executable_format|. May fail even if the executable is valid if it requires
+// features not supported by the current host or runtime (such as available
+// architectures, imports, etc).
+//
+// Depending on loader ability the |caching_mode| is used to enable certain
+// features such as instrumented profiling. Not all formats support these
+// features and cooperation of both the compiler producing the executables and
+// the runtime loader and system are required.
+//
+// Returns IREE_STATUS_CANCELLED when the loader cannot load the file in the
+// given format.
+iree_status_t iree_hal_executable_loader_try_load(
+ iree_hal_executable_loader_t* executable_loader,
+ iree_hal_executable_layout_t* executable_layout,
+ iree_hal_executable_format_t executable_format,
+ iree_hal_executable_caching_mode_t caching_mode,
+ iree_const_byte_span_t executable_data,
+ iree_hal_executable_t** out_executable);
+
+//===----------------------------------------------------------------------===//
+// iree_hal_executable_loader_t implementation details
+//===----------------------------------------------------------------------===//
+
+typedef struct iree_hal_executable_loader_vtable_s {
+ void(IREE_API_PTR* destroy)(iree_hal_executable_loader_t* executable_loader);
+
+ bool(IREE_API_PTR* query_support)(
+ iree_hal_executable_loader_t* executable_loader,
+ iree_hal_executable_format_t executable_format,
+ iree_hal_executable_caching_mode_t caching_mode);
+
+ iree_status_t(IREE_API_PTR* try_load)(
+ iree_hal_executable_loader_t* executable_loader,
+ iree_hal_executable_layout_t* executable_layout,
+ iree_hal_executable_format_t executable_format,
+ iree_hal_executable_caching_mode_t caching_mode,
+ iree_const_byte_span_t executable_data,
+ iree_hal_executable_t** out_executable);
+} iree_hal_executable_loader_vtable_t;
+
+#ifdef __cplusplus
+} // extern "C"
+#endif // __cplusplus
+
+#endif // IREE_HAL_LOCAL_EXECUTABLE_LOADER_H_
diff --git a/iree/hal/local/local_descriptor_set.c b/iree/hal/local/local_descriptor_set.c
new file mode 100644
index 0000000..34ce270
--- /dev/null
+++ b/iree/hal/local/local_descriptor_set.c
@@ -0,0 +1,86 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "iree/hal/local/local_descriptor_set.h"
+
+#include "iree/base/tracing.h"
+
+static const iree_hal_descriptor_set_vtable_t
+ iree_hal_local_descriptor_set_vtable;
+
+iree_status_t iree_hal_local_descriptor_set_create(
+ iree_hal_descriptor_set_layout_t* base_layout,
+ iree_host_size_t binding_count,
+ const iree_hal_descriptor_set_binding_t* bindings,
+ iree_hal_descriptor_set_t** out_descriptor_set) {
+ IREE_ASSERT_ARGUMENT(base_layout);
+ IREE_ASSERT_ARGUMENT(!binding_count || bindings);
+ IREE_ASSERT_ARGUMENT(out_descriptor_set);
+ *out_descriptor_set = NULL;
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ iree_hal_local_descriptor_set_layout_t* local_layout =
+ iree_hal_local_descriptor_set_layout_cast(base_layout);
+ IREE_ASSERT_ARGUMENT(local_layout);
+
+ iree_hal_local_descriptor_set_t* descriptor_set = NULL;
+ iree_host_size_t total_size =
+ sizeof(*descriptor_set) +
+ binding_count * sizeof(*descriptor_set->bindings);
+ iree_status_t status = iree_allocator_malloc(
+ local_layout->host_allocator, total_size, (void**)&descriptor_set);
+ if (iree_status_is_ok(status)) {
+ iree_hal_resource_initialize(&iree_hal_local_descriptor_set_vtable,
+ &descriptor_set->resource);
+ descriptor_set->layout = local_layout;
+ iree_hal_descriptor_set_layout_retain(base_layout);
+ descriptor_set->binding_count = binding_count;
+ memcpy(descriptor_set->bindings, bindings,
+ binding_count * sizeof(iree_hal_descriptor_set_binding_t));
+ for (iree_host_size_t i = 0; i < descriptor_set->binding_count; ++i) {
+ iree_hal_buffer_retain(descriptor_set->bindings[i].buffer);
+ }
+ *out_descriptor_set = (iree_hal_descriptor_set_t*)descriptor_set;
+ }
+
+ IREE_TRACE_ZONE_END(z0);
+ return status;
+}
+
+static void iree_hal_local_descriptor_set_destroy(
+ iree_hal_descriptor_set_t* base_descriptor_set) {
+ iree_hal_local_descriptor_set_t* descriptor_set =
+ (iree_hal_local_descriptor_set_t*)base_descriptor_set;
+ iree_allocator_t host_allocator = descriptor_set->layout->host_allocator;
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ for (iree_host_size_t i = 0; i < descriptor_set->binding_count; ++i) {
+ iree_hal_buffer_release(descriptor_set->bindings[i].buffer);
+ }
+ iree_hal_descriptor_set_layout_release(
+ (iree_hal_descriptor_set_layout_t*)descriptor_set->layout);
+ iree_allocator_free(host_allocator, descriptor_set);
+
+ IREE_TRACE_ZONE_END(z0);
+}
+
+iree_hal_local_descriptor_set_t* iree_hal_local_descriptor_set_cast(
+ iree_hal_descriptor_set_t* base_value) {
+ return (iree_hal_local_descriptor_set_t*)base_value;
+}
+
+static const iree_hal_descriptor_set_vtable_t
+ iree_hal_local_descriptor_set_vtable = {
+ .destroy = iree_hal_local_descriptor_set_destroy,
+};
diff --git a/iree/hal/local/local_descriptor_set.h b/iree/hal/local/local_descriptor_set.h
new file mode 100644
index 0000000..1032008
--- /dev/null
+++ b/iree/hal/local/local_descriptor_set.h
@@ -0,0 +1,45 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef IREE_HAL_LOCAL_LOCAL_DESCRIPTOR_SET_H_
+#define IREE_HAL_LOCAL_LOCAL_DESCRIPTOR_SET_H_
+
+#include "iree/base/api.h"
+#include "iree/hal/api.h"
+#include "iree/hal/local/local_descriptor_set_layout.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif // __cplusplus
+
+typedef struct {
+ iree_hal_resource_t resource;
+ iree_hal_local_descriptor_set_layout_t* layout;
+ iree_host_size_t binding_count;
+ iree_hal_descriptor_set_binding_t bindings[];
+} iree_hal_local_descriptor_set_t;
+
+iree_status_t iree_hal_local_descriptor_set_create(
+ iree_hal_descriptor_set_layout_t* layout, iree_host_size_t binding_count,
+ const iree_hal_descriptor_set_binding_t* bindings,
+ iree_hal_descriptor_set_t** out_descriptor_set);
+
+iree_hal_local_descriptor_set_t* iree_hal_local_descriptor_set_cast(
+ iree_hal_descriptor_set_t* base_value);
+
+#ifdef __cplusplus
+} // extern "C"
+#endif // __cplusplus
+
+#endif // IREE_HAL_LOCAL_LOCAL_DESCRIPTOR_SET_H_
diff --git a/iree/hal/local/local_descriptor_set_layout.c b/iree/hal/local/local_descriptor_set_layout.c
new file mode 100644
index 0000000..4d2af6a
--- /dev/null
+++ b/iree/hal/local/local_descriptor_set_layout.c
@@ -0,0 +1,80 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "iree/hal/local/local_descriptor_set_layout.h"
+
+#include "iree/base/tracing.h"
+
+static const iree_hal_descriptor_set_layout_vtable_t
+ iree_hal_local_descriptor_set_layout_vtable;
+
+iree_status_t iree_hal_local_descriptor_set_layout_create(
+ iree_hal_descriptor_set_layout_usage_type_t usage_type,
+ iree_host_size_t binding_count,
+ const iree_hal_descriptor_set_layout_binding_t* bindings,
+ iree_allocator_t host_allocator,
+ iree_hal_descriptor_set_layout_t** out_descriptor_set_layout) {
+ IREE_ASSERT_ARGUMENT(!binding_count || bindings);
+ IREE_ASSERT_ARGUMENT(out_descriptor_set_layout);
+ *out_descriptor_set_layout = NULL;
+ if (binding_count > IREE_HAL_LOCAL_MAX_DESCRIPTOR_BINDING_COUNT) {
+ return iree_make_status(
+ IREE_STATUS_INVALID_ARGUMENT, "binding count %zu over the limit of %d",
+ binding_count, IREE_HAL_LOCAL_MAX_DESCRIPTOR_BINDING_COUNT);
+ }
+
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ iree_hal_local_descriptor_set_layout_t* layout = NULL;
+ iree_host_size_t total_size =
+ sizeof(*layout) + binding_count * sizeof(*layout->bindings);
+ iree_status_t status =
+ iree_allocator_malloc(host_allocator, total_size, (void**)&layout);
+ if (iree_status_is_ok(status)) {
+ iree_hal_resource_initialize(&iree_hal_local_descriptor_set_layout_vtable,
+ &layout->resource);
+ layout->host_allocator = host_allocator;
+ layout->usage_type = usage_type;
+ layout->binding_count = binding_count;
+ memcpy(layout->bindings, bindings,
+ binding_count * sizeof(iree_hal_descriptor_set_layout_binding_t));
+ *out_descriptor_set_layout = (iree_hal_descriptor_set_layout_t*)layout;
+ }
+
+ IREE_TRACE_ZONE_END(z0);
+ return status;
+}
+
+static void iree_hal_local_descriptor_set_layout_destroy(
+ iree_hal_descriptor_set_layout_t* base_layout) {
+ iree_hal_local_descriptor_set_layout_t* layout =
+ (iree_hal_local_descriptor_set_layout_t*)base_layout;
+ iree_allocator_t host_allocator = layout->host_allocator;
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ iree_allocator_free(host_allocator, layout);
+
+ IREE_TRACE_ZONE_END(z0);
+}
+
+iree_hal_local_descriptor_set_layout_t*
+iree_hal_local_descriptor_set_layout_cast(
+ iree_hal_descriptor_set_layout_t* base_value) {
+ return (iree_hal_local_descriptor_set_layout_t*)base_value;
+}
+
+static const iree_hal_descriptor_set_layout_vtable_t
+ iree_hal_local_descriptor_set_layout_vtable = {
+ .destroy = iree_hal_local_descriptor_set_layout_destroy,
+};
diff --git a/iree/hal/local/local_descriptor_set_layout.h b/iree/hal/local/local_descriptor_set_layout.h
new file mode 100644
index 0000000..3ee7dc7
--- /dev/null
+++ b/iree/hal/local/local_descriptor_set_layout.h
@@ -0,0 +1,50 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef IREE_HAL_LOCAL_LOCAL_DESCRIPTOR_SET_LAYOUT_H_
+#define IREE_HAL_LOCAL_LOCAL_DESCRIPTOR_SET_LAYOUT_H_
+
+#include "iree/base/api.h"
+#include "iree/hal/api.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif // __cplusplus
+
+#define IREE_HAL_LOCAL_MAX_DESCRIPTOR_BINDING_COUNT 32
+
+typedef struct {
+ iree_hal_resource_t resource;
+ iree_allocator_t host_allocator;
+ iree_hal_descriptor_set_layout_usage_type_t usage_type;
+ iree_host_size_t binding_count;
+ iree_hal_descriptor_set_layout_binding_t bindings[];
+} iree_hal_local_descriptor_set_layout_t;
+
+iree_status_t iree_hal_local_descriptor_set_layout_create(
+ iree_hal_descriptor_set_layout_usage_type_t usage_type,
+ iree_host_size_t binding_count,
+ const iree_hal_descriptor_set_layout_binding_t* bindings,
+ iree_allocator_t host_allocator,
+ iree_hal_descriptor_set_layout_t** out_descriptor_set_layout);
+
+iree_hal_local_descriptor_set_layout_t*
+iree_hal_local_descriptor_set_layout_cast(
+ iree_hal_descriptor_set_layout_t* base_value);
+
+#ifdef __cplusplus
+} // extern "C"
+#endif // __cplusplus
+
+#endif // IREE_HAL_LOCAL_LOCAL_DESCRIPTOR_SET_LAYOUT_H_
diff --git a/iree/hal/local/local_executable.c b/iree/hal/local/local_executable.c
new file mode 100644
index 0000000..616c3d7
--- /dev/null
+++ b/iree/hal/local/local_executable.c
@@ -0,0 +1,45 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "iree/hal/local/local_executable.h"
+
+void iree_hal_local_executable_initialize(
+ const iree_hal_local_executable_vtable_t* vtable,
+ iree_hal_local_executable_layout_t* layout,
+ iree_hal_local_executable_t* out_base_executable) {
+ iree_hal_resource_initialize(vtable, &out_base_executable->resource);
+ out_base_executable->layout = layout;
+ iree_hal_executable_layout_retain((iree_hal_executable_layout_t*)layout);
+}
+
+void iree_hal_local_executable_deinitialize(
+ iree_hal_local_executable_t* base_executable) {
+ iree_hal_executable_layout_release(
+ (iree_hal_executable_layout_t*)base_executable->layout);
+}
+
+iree_hal_local_executable_t* iree_hal_local_executable_cast(
+ iree_hal_executable_t* base_value) {
+ return (iree_hal_local_executable_t*)base_value;
+}
+
+iree_status_t iree_hal_local_executable_issue_call(
+ iree_hal_local_executable_t* executable, iree_host_size_t ordinal,
+ const iree_hal_local_executable_call_t* call) {
+ IREE_ASSERT_ARGUMENT(executable);
+ IREE_ASSERT_ARGUMENT(call);
+ return ((const iree_hal_local_executable_vtable_t*)
+ executable->resource.vtable)
+ ->issue_call(executable, ordinal, call);
+}
diff --git a/iree/hal/local/local_executable.h b/iree/hal/local/local_executable.h
new file mode 100644
index 0000000..a3e2acf
--- /dev/null
+++ b/iree/hal/local/local_executable.h
@@ -0,0 +1,69 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef IREE_HAL_LOCAL_LOCAL_EXECUTABLE_H_
+#define IREE_HAL_LOCAL_LOCAL_EXECUTABLE_H_
+
+#include "iree/base/api.h"
+#include "iree/hal/api.h"
+#include "iree/hal/local/executable_library.h"
+#include "iree/hal/local/local_executable_layout.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif // __cplusplus
+
+typedef struct {
+ const iree_hal_executable_dispatch_state_v0_t* state;
+ iree_hal_vec3_t workgroup_id;
+ iree_hal_vec3_t workgroup_size;
+ iree_hal_vec3_t workgroup_count;
+ iree_hal_executable_push_constants_ptr_t push_constants;
+ const iree_hal_executable_binding_ptr_t* bindings;
+ const iree_host_size_t* binding_lengths;
+} iree_hal_local_executable_call_t;
+
+typedef struct {
+ iree_hal_resource_t resource;
+ iree_hal_local_executable_layout_t* layout;
+} iree_hal_local_executable_t;
+
+typedef struct {
+ iree_hal_executable_vtable_t base;
+
+ iree_status_t(IREE_API_PTR* issue_call)(
+ iree_hal_local_executable_t* executable, iree_host_size_t ordinal,
+ const iree_hal_local_executable_call_t* call);
+} iree_hal_local_executable_vtable_t;
+
+void iree_hal_local_executable_initialize(
+ const iree_hal_local_executable_vtable_t* vtable,
+ iree_hal_local_executable_layout_t* layout,
+ iree_hal_local_executable_t* out_base_executable);
+
+void iree_hal_local_executable_deinitialize(
+ iree_hal_local_executable_t* base_executable);
+
+iree_hal_local_executable_t* iree_hal_local_executable_cast(
+ iree_hal_executable_t* base_value);
+
+iree_status_t iree_hal_local_executable_issue_call(
+ iree_hal_local_executable_t* executable, iree_host_size_t ordinal,
+ const iree_hal_local_executable_call_t* call);
+
+#ifdef __cplusplus
+} // extern "C"
+#endif // __cplusplus
+
+#endif // IREE_HAL_LOCAL_LOCAL_EXECUTABLE_H_
diff --git a/iree/hal/local/local_executable_cache.c b/iree/hal/local/local_executable_cache.c
new file mode 100644
index 0000000..e18d73f
--- /dev/null
+++ b/iree/hal/local/local_executable_cache.c
@@ -0,0 +1,138 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "iree/hal/local/local_executable_cache.h"
+
+#include "iree/base/tracing.h"
+
+typedef struct {
+ iree_hal_resource_t resource;
+ iree_allocator_t host_allocator;
+ iree_string_view_t identifier;
+ iree_host_size_t loader_count;
+ iree_hal_executable_loader_t* loaders[];
+} iree_hal_local_executable_cache_t;
+
+static const iree_hal_executable_cache_vtable_t
+ iree_hal_local_executable_cache_vtable;
+
+iree_status_t iree_hal_local_executable_cache_create(
+ iree_string_view_t identifier, iree_host_size_t loader_count,
+ iree_hal_executable_loader_t** loaders, iree_allocator_t host_allocator,
+ iree_hal_executable_cache_t** out_executable_cache) {
+ IREE_ASSERT_ARGUMENT(!loader_count || loaders);
+ IREE_ASSERT_ARGUMENT(out_executable_cache);
+ *out_executable_cache = NULL;
+
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ iree_hal_local_executable_cache_t* executable_cache = NULL;
+ iree_host_size_t total_size =
+ sizeof(*executable_cache) +
+ loader_count * sizeof(*executable_cache->loaders) + identifier.size;
+ iree_status_t status = iree_allocator_malloc(host_allocator, total_size,
+ (void**)&executable_cache);
+ if (iree_status_is_ok(status)) {
+ iree_hal_resource_initialize(&iree_hal_local_executable_cache_vtable,
+ &executable_cache->resource);
+ executable_cache->host_allocator = host_allocator;
+ iree_string_view_append_to_buffer(
+ identifier, &executable_cache->identifier,
+ (char*)executable_cache + total_size - identifier.size);
+
+ executable_cache->loader_count = loader_count;
+ for (iree_host_size_t i = 0; i < executable_cache->loader_count; ++i) {
+ executable_cache->loaders[i] = loaders[i];
+ iree_hal_executable_loader_retain(executable_cache->loaders[i]);
+ }
+
+ *out_executable_cache = (iree_hal_executable_cache_t*)executable_cache;
+ }
+
+ IREE_TRACE_ZONE_END(z0);
+ return status;
+}
+
+static void iree_hal_local_executable_cache_destroy(
+ iree_hal_executable_cache_t* base_executable_cache) {
+ iree_hal_local_executable_cache_t* executable_cache =
+ (iree_hal_local_executable_cache_t*)base_executable_cache;
+ iree_allocator_t host_allocator = executable_cache->host_allocator;
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ for (iree_host_size_t i = 0; i < executable_cache->loader_count; ++i) {
+ iree_hal_executable_loader_release(executable_cache->loaders[i]);
+ }
+ iree_allocator_free(host_allocator, executable_cache);
+
+ IREE_TRACE_ZONE_END(z0);
+}
+
+static bool iree_hal_local_executable_cache_can_prepare_format(
+ iree_hal_executable_cache_t* base_executable_cache,
+ iree_hal_executable_format_t format) {
+ iree_hal_local_executable_cache_t* executable_cache =
+ (iree_hal_local_executable_cache_t*)base_executable_cache;
+ for (iree_host_size_t i = 0; i < executable_cache->loader_count; ++i) {
+ if (iree_hal_executable_loader_query_support(
+ executable_cache->loaders[i], format,
+ IREE_HAL_EXECUTABLE_CACHING_MODE_DEFAULT)) {
+ return true;
+ }
+ }
+ return false;
+}
+
+static iree_status_t iree_hal_local_executable_cache_prepare_executable(
+ iree_hal_executable_cache_t* base_executable_cache,
+ iree_hal_executable_layout_t* executable_layout,
+ iree_hal_executable_caching_mode_t caching_mode,
+ iree_const_byte_span_t executable_data,
+ iree_hal_executable_t** out_executable) {
+ iree_hal_local_executable_cache_t* executable_cache =
+ (iree_hal_local_executable_cache_t*)base_executable_cache;
+ for (iree_host_size_t i = 0; i < executable_cache->loader_count; ++i) {
+ // TODO(benvanik): pass executable format through from the HAL.
+ // if (iree_hal_executable_loader_query_support(
+ // executable_cache->loaders[i], executable_format,
+ // IREE_HAL_EXECUTABLE_CACHING_MODE_DEFAULT)) {
+ // return iree_hal_executable_loader_try_load(
+ // executable_cache->loaders[i], executable_layout,
+ // executable_format, caching_mode, executable_data,
+ // out_executable);
+ // }
+ iree_status_t status = iree_hal_executable_loader_try_load(
+ executable_cache->loaders[i], executable_layout,
+ /*executable_format=*/0, caching_mode, executable_data, out_executable);
+ if (iree_status_is_ok(status)) {
+ // Executable was successfully loaded.
+ return status;
+ } else if (!iree_status_is_cancelled(status)) {
+ // Error beyond just the try failing due to unsupported formats.
+ return status;
+ }
+ }
+ return iree_make_status(
+ IREE_STATUS_NOT_FOUND,
+ "no executable loader registered for the given file format");
+}
+
+static const iree_hal_executable_cache_vtable_t
+ iree_hal_local_executable_cache_vtable = {
+ .destroy = iree_hal_local_executable_cache_destroy,
+ .can_prepare_format =
+ iree_hal_local_executable_cache_can_prepare_format,
+ .prepare_executable =
+ iree_hal_local_executable_cache_prepare_executable,
+};
diff --git a/iree/hal/local/local_executable_cache.h b/iree/hal/local/local_executable_cache.h
new file mode 100644
index 0000000..09f5024
--- /dev/null
+++ b/iree/hal/local/local_executable_cache.h
@@ -0,0 +1,43 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef IREE_HAL_LOCAL_LOCAL_EXECUTABLE_CACHE_H_
+#define IREE_HAL_LOCAL_LOCAL_EXECUTABLE_CACHE_H_
+
+#include "iree/base/api.h"
+#include "iree/hal/api.h"
+#include "iree/hal/local/executable_loader.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif // __cplusplus
+
+// TODO(benvanik): when we refactor executable caches this can become something
+// more specialized; like nop_executable_cache (does nothing but pass through)
+// or inproc_lru_executable_cache (simple in-memory LRU of recent executables).
+//
+// We can also set this up so they share storage. Ideally a JIT'ed executable in
+// one device is the same JIT'ed executable in another, and in multi-tenant
+// situations we're likely to want that isolation _and_ sharing.
+
+iree_status_t iree_hal_local_executable_cache_create(
+ iree_string_view_t identifier, iree_host_size_t loader_count,
+ iree_hal_executable_loader_t** loaders, iree_allocator_t host_allocator,
+ iree_hal_executable_cache_t** out_executable_cache);
+
+#ifdef __cplusplus
+} // extern "C"
+#endif // __cplusplus
+
+#endif // IREE_HAL_LOCAL_LOCAL_EXECUTABLE_CACHE_H_
diff --git a/iree/hal/local/local_executable_layout.c b/iree/hal/local/local_executable_layout.c
new file mode 100644
index 0000000..ebbe34b
--- /dev/null
+++ b/iree/hal/local/local_executable_layout.c
@@ -0,0 +1,110 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "iree/hal/local/local_executable_layout.h"
+
+#include "iree/base/tracing.h"
+#include "iree/hal/local/local_descriptor_set_layout.h"
+
+static const iree_hal_executable_layout_vtable_t
+ iree_hal_local_executable_layout_vtable;
+
+iree_status_t iree_hal_local_executable_layout_create(
+ iree_host_size_t set_layout_count,
+ iree_hal_descriptor_set_layout_t** set_layouts,
+ iree_host_size_t push_constants, iree_allocator_t host_allocator,
+ iree_hal_executable_layout_t** out_executable_layout) {
+ IREE_ASSERT_ARGUMENT(!set_layout_count || set_layouts);
+ IREE_ASSERT_ARGUMENT(out_executable_layout);
+ *out_executable_layout = NULL;
+ if (set_layout_count > IREE_HAL_LOCAL_MAX_DESCRIPTOR_SET_COUNT) {
+ return iree_make_status(IREE_STATUS_INVALID_ARGUMENT,
+ "set layout count %zu over the limit of %d",
+ set_layout_count,
+ IREE_HAL_LOCAL_MAX_DESCRIPTOR_SET_COUNT);
+ }
+ if (push_constants > IREE_HAL_LOCAL_MAX_PUSH_CONSTANT_COUNT) {
+ return iree_make_status(IREE_STATUS_INVALID_ARGUMENT,
+ "push constant count %zu over the limit of %d",
+ push_constants,
+ IREE_HAL_LOCAL_MAX_PUSH_CONSTANT_COUNT);
+ }
+
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ iree_host_size_t total_size =
+ sizeof(iree_hal_local_executable_layout_t) +
+ set_layout_count * sizeof(iree_hal_descriptor_set_layout_t*);
+
+ iree_hal_local_executable_layout_t* layout = NULL;
+ iree_status_t status =
+ iree_allocator_malloc(host_allocator, total_size, (void**)&layout);
+ if (iree_status_is_ok(status)) {
+ iree_hal_resource_initialize(&iree_hal_local_executable_layout_vtable,
+ &layout->resource);
+ layout->host_allocator = host_allocator;
+ layout->push_constants = push_constants;
+ layout->dynamic_binding_count = 0;
+ layout->used_bindings = 0;
+ layout->set_layout_count = set_layout_count;
+ for (iree_host_size_t i = 0; i < set_layout_count; ++i) {
+ layout->set_layouts[i] = set_layouts[i];
+ iree_hal_descriptor_set_layout_retain(layout->set_layouts[i]);
+
+ iree_hal_local_descriptor_set_layout_t* local_set_layout =
+ iree_hal_local_descriptor_set_layout_cast(set_layouts[i]);
+ for (iree_host_size_t j = 0; j < local_set_layout->binding_count; ++j) {
+ const iree_hal_descriptor_set_layout_binding_t* binding =
+ &local_set_layout->bindings[j];
+ layout->used_bindings |=
+ 1ull << (i * IREE_HAL_LOCAL_MAX_DESCRIPTOR_BINDING_COUNT + j);
+ switch (binding->type) {
+ case IREE_HAL_DESCRIPTOR_TYPE_STORAGE_BUFFER_DYNAMIC:
+ case IREE_HAL_DESCRIPTOR_TYPE_UNIFORM_BUFFER_DYNAMIC:
+ ++layout->dynamic_binding_count;
+ break;
+ }
+ }
+ }
+ *out_executable_layout = (iree_hal_executable_layout_t*)layout;
+ }
+
+ IREE_TRACE_ZONE_END(z0);
+ return status;
+}
+
+static void iree_hal_local_executable_layout_destroy(
+ iree_hal_executable_layout_t* base_layout) {
+ iree_hal_local_executable_layout_t* layout =
+ (iree_hal_local_executable_layout_t*)base_layout;
+ iree_allocator_t host_allocator = layout->host_allocator;
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ for (iree_host_size_t i = 0; i < layout->set_layout_count; ++i) {
+ iree_hal_descriptor_set_layout_release(layout->set_layouts[i]);
+ }
+ iree_allocator_free(host_allocator, layout);
+
+ IREE_TRACE_ZONE_END(z0);
+}
+
+iree_hal_local_executable_layout_t* iree_hal_local_executable_layout_cast(
+ iree_hal_executable_layout_t* base_value) {
+ return (iree_hal_local_executable_layout_t*)base_value;
+}
+
+static const iree_hal_executable_layout_vtable_t
+ iree_hal_local_executable_layout_vtable = {
+ .destroy = iree_hal_local_executable_layout_destroy,
+};
diff --git a/iree/hal/local/local_executable_layout.h b/iree/hal/local/local_executable_layout.h
new file mode 100644
index 0000000..801236a
--- /dev/null
+++ b/iree/hal/local/local_executable_layout.h
@@ -0,0 +1,53 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef IREE_HAL_LOCAL_LOCAL_EXECUTABLE_LAYOUT_H_
+#define IREE_HAL_LOCAL_LOCAL_EXECUTABLE_LAYOUT_H_
+
+#include "iree/base/api.h"
+#include "iree/hal/api.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif // __cplusplus
+
+#define IREE_HAL_LOCAL_MAX_DESCRIPTOR_SET_COUNT 2
+#define IREE_HAL_LOCAL_MAX_PUSH_CONSTANT_COUNT 64
+
+typedef uint64_t iree_hal_local_binding_mask_t;
+
+typedef struct {
+ iree_hal_resource_t resource;
+ iree_allocator_t host_allocator;
+ iree_host_size_t push_constants;
+ iree_host_size_t dynamic_binding_count;
+ iree_hal_local_binding_mask_t used_bindings;
+ iree_host_size_t set_layout_count;
+ iree_hal_descriptor_set_layout_t* set_layouts[];
+} iree_hal_local_executable_layout_t;
+
+iree_status_t iree_hal_local_executable_layout_create(
+ iree_host_size_t set_layout_count,
+ iree_hal_descriptor_set_layout_t** set_layouts,
+ iree_host_size_t push_constants, iree_allocator_t host_allocator,
+ iree_hal_executable_layout_t** out_executable_layout);
+
+iree_hal_local_executable_layout_t* iree_hal_local_executable_layout_cast(
+ iree_hal_executable_layout_t* base_value);
+
+#ifdef __cplusplus
+} // extern "C"
+#endif // __cplusplus
+
+#endif // IREE_HAL_LOCAL_LOCAL_EXECUTABLE_LAYOUT_H_
diff --git a/iree/hal/local/task_command_buffer.c b/iree/hal/local/task_command_buffer.c
new file mode 100644
index 0000000..c879923
--- /dev/null
+++ b/iree/hal/local/task_command_buffer.c
@@ -0,0 +1,864 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "iree/hal/local/task_command_buffer.h"
+
+#include "iree/base/debugging.h"
+#include "iree/base/tracing.h"
+#include "iree/hal/local/local_descriptor_set_layout.h"
+#include "iree/hal/local/local_executable.h"
+#include "iree/hal/local/local_executable_layout.h"
+#include "iree/task/list.h"
+#include "iree/task/submission.h"
+#include "iree/task/task.h"
+
+//===----------------------------------------------------------------------===//
+// iree_hal_task_command_buffer_t
+//===----------------------------------------------------------------------===//
+
+// iree/task/-based command buffer.
+// We track a minimal amount of state here and incrementally build out the task
+// DAG that we can submit to the task system directly. There's no intermediate
+// data structures and we produce the iree_task_ts directly. In the steady state
+// all allocations are served from a shared per-device block pool with no
+// additional allocations required during recording or execution. That means our
+// command buffer here is essentially just a builder for the task system types
+// and manager of the lifetime of the tasks.
+typedef struct {
+ iree_hal_resource_t resource;
+
+ iree_hal_device_t* device;
+ iree_task_scope_t* scope;
+ iree_hal_command_buffer_mode_t mode;
+ iree_hal_command_category_t allowed_categories;
+
+ // Arena used for all allocations; references the shared device block pool.
+ iree_arena_allocator_t arena;
+
+ // One or more tasks at the root of the command buffer task DAG.
+ // These tasks are all able to execute concurrently and will be the initial
+ // ready task set in the submission.
+ iree_task_list_t root_tasks;
+
+ // One or more tasks at the leaves of the DAG.
+ // Only once all these tasks have completed execution will the command buffer
+ // be considered completed as a whole.
+ //
+ // An empty list indicates that root_tasks are also the leaves.
+ iree_task_list_t leaf_tasks;
+
+ // TODO(benvanik): move this out of the struct and allocate from the arena -
+ // we only need this during recording and it's ~4KB of waste otherwise.
+ // State tracked within the command buffer during recording only.
+ struct {
+ // The last global barrier that was inserted, if any.
+ // The barrier is allocated and inserted into the DAG when requested but the
+ // actual barrier dependency list is only allocated and set on flushes.
+ // This lets us allocate the appropriately sized barrier task list from the
+ // arena even though when the barrier is recorded we don't yet know what
+ // other tasks we'll be emitting as we walk the command stream.
+ iree_task_barrier_t* open_barrier;
+
+ // The number of tasks in the open barrier (|open_tasks|), used to quickly
+ // allocate storage for the task list without needing to walk the list.
+ iree_host_size_t open_task_count;
+
+ // All execution tasks emitted that must execute after |open_barrier|.
+ iree_task_list_t open_tasks;
+
+ // A flattened list of all available descriptor set bindings.
+ // As descriptor sets are pushed/bound the bindings will be updated to
+ // represent the fully-translated binding data pointer.
+ // TODO(benvanik): support proper mapping semantics and track the
+ // iree_hal_buffer_mapping_t and map/unmap where appropriate.
+ iree_hal_executable_binding_ptr_t
+ bindings[IREE_HAL_LOCAL_MAX_DESCRIPTOR_SET_COUNT *
+ IREE_HAL_LOCAL_MAX_DESCRIPTOR_BINDING_COUNT];
+ iree_device_size_t
+ binding_lengths[IREE_HAL_LOCAL_MAX_DESCRIPTOR_SET_COUNT *
+ IREE_HAL_LOCAL_MAX_DESCRIPTOR_BINDING_COUNT];
+
+ // All available push constants updated each time push_constants is called.
+ // Reset only with the command buffer and otherwise will maintain its values
+ // during recording to allow for partial push_constants updates.
+ uint32_t push_constants[IREE_HAL_LOCAL_MAX_PUSH_CONSTANT_COUNT];
+ } state;
+} iree_hal_task_command_buffer_t;
+
+static const iree_hal_command_buffer_vtable_t
+ iree_hal_task_command_buffer_vtable;
+
+iree_status_t iree_hal_task_command_buffer_create(
+ iree_hal_device_t* device, iree_task_scope_t* scope,
+ iree_hal_command_buffer_mode_t mode,
+ iree_hal_command_category_t command_categories,
+ iree_arena_block_pool_t* block_pool,
+ iree_hal_command_buffer_t** out_command_buffer) {
+ IREE_ASSERT_ARGUMENT(device);
+ IREE_ASSERT_ARGUMENT(out_command_buffer);
+ *out_command_buffer = NULL;
+ if (mode != IREE_HAL_COMMAND_BUFFER_MODE_ONE_SHOT) {
+ // If we want reuse we'd need to support duplicating the task DAG after
+ // recording or have some kind of copy-on-submit behavior that does so if
+ // a command buffer is submitted for execution twice. Allowing for the same
+ // command buffer to be enqueued multiple times would be fine so long as
+ // execution doesn't overlap (`cmdbuf|cmdbuf` vs
+ // `cmdbuf -> semaphore -> cmdbuf`) though we'd still need to be careful
+ // that we did the enqueuing and reset of the task structures at the right
+ // times. Definitely something that'll be useful in the future... but not
+ // today :)
+ return iree_make_status(IREE_STATUS_UNIMPLEMENTED,
+ "only one-shot command buffer usage is supported");
+ }
+
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ iree_hal_task_command_buffer_t* command_buffer = NULL;
+ iree_status_t status =
+ iree_allocator_malloc(iree_hal_device_host_allocator(device),
+ sizeof(*command_buffer), (void**)&command_buffer);
+ if (iree_status_is_ok(status)) {
+ iree_hal_resource_initialize(&iree_hal_task_command_buffer_vtable,
+ &command_buffer->resource);
+ command_buffer->device = device;
+ command_buffer->scope = scope;
+ command_buffer->mode = mode;
+ command_buffer->allowed_categories = command_categories;
+ iree_arena_initialize(block_pool, &command_buffer->arena);
+ iree_task_list_initialize(&command_buffer->root_tasks);
+ iree_task_list_initialize(&command_buffer->leaf_tasks);
+ memset(&command_buffer->state, 0, sizeof(command_buffer->state));
+ *out_command_buffer = (iree_hal_command_buffer_t*)command_buffer;
+ }
+
+ IREE_TRACE_ZONE_END(z0);
+ return status;
+}
+
+static void iree_hal_task_command_buffer_reset(
+ iree_hal_task_command_buffer_t* command_buffer) {
+ memset(&command_buffer->state, 0, sizeof(command_buffer->state));
+ iree_task_list_discard(&command_buffer->leaf_tasks);
+ iree_task_list_discard(&command_buffer->root_tasks);
+ iree_arena_reset(&command_buffer->arena);
+}
+
+static void iree_hal_task_command_buffer_destroy(
+ iree_hal_command_buffer_t* base_command_buffer) {
+ iree_hal_task_command_buffer_t* command_buffer =
+ (iree_hal_task_command_buffer_t*)base_command_buffer;
+ iree_allocator_t host_allocator =
+ iree_hal_device_host_allocator(command_buffer->device);
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ iree_hal_task_command_buffer_reset(command_buffer);
+ iree_arena_deinitialize(&command_buffer->arena);
+ iree_allocator_free(host_allocator, command_buffer);
+
+ IREE_TRACE_ZONE_END(z0);
+}
+
+static iree_hal_command_category_t
+iree_hal_task_command_buffer_allowed_categories(
+ const iree_hal_command_buffer_t* base_command_buffer) {
+ return ((const iree_hal_task_command_buffer_t*)base_command_buffer)
+ ->allowed_categories;
+}
+
+//===----------------------------------------------------------------------===//
+// iree_hal_task_command_buffer_t recording
+//===----------------------------------------------------------------------===//
+
+static iree_status_t iree_hal_task_command_buffer_flush_tasks(
+ iree_hal_task_command_buffer_t* command_buffer);
+
+static iree_status_t iree_hal_task_command_buffer_begin(
+ iree_hal_command_buffer_t* base_command_buffer) {
+ iree_hal_task_command_buffer_t* command_buffer =
+ iree_hal_task_command_buffer_cast(base_command_buffer);
+ iree_hal_task_command_buffer_reset(command_buffer);
+ return iree_ok_status();
+}
+
+static iree_status_t iree_hal_task_command_buffer_end(
+ iree_hal_command_buffer_t* base_command_buffer) {
+ iree_hal_task_command_buffer_t* command_buffer =
+ iree_hal_task_command_buffer_cast(base_command_buffer);
+
+ // Flush any open barriers.
+ IREE_RETURN_IF_ERROR(
+ iree_hal_task_command_buffer_flush_tasks(command_buffer));
+
+ // Move the tasks from the leaf list (tail) to the root list (head) if this
+ // was the first set of tasks recorded.
+ if (iree_task_list_is_empty(&command_buffer->root_tasks) &&
+ !iree_task_list_is_empty(&command_buffer->leaf_tasks)) {
+ iree_task_list_move(&command_buffer->leaf_tasks,
+ &command_buffer->root_tasks);
+ }
+
+ return iree_ok_status();
+}
+
+// Flushes all open tasks to the previous barrier and prepares for more
+// recording. The root tasks are also populated here when required as this is
+// the one place where we can see both halves of the most recent synchronization
+// event: those tasks recorded prior (if any) and the task that marks the set of
+// tasks that will be recorded after (if any).
+static iree_status_t iree_hal_task_command_buffer_flush_tasks(
+ iree_hal_task_command_buffer_t* command_buffer) {
+ iree_task_barrier_t* open_barrier = command_buffer->state.open_barrier;
+ if (open_barrier != NULL) {
+ // There is an open barrier we need to fixup the fork out to all of the open
+ // tasks that were recorded after it.
+ iree_task_t* task_head =
+ iree_task_list_front(&command_buffer->state.open_tasks);
+ iree_host_size_t dependent_task_count =
+ command_buffer->state.open_task_count;
+ if (dependent_task_count == 1) {
+ // Special-case: only one open task so we can avoid the additional barrier
+ // overhead by reusing the completion task.
+ iree_task_set_completion_task(&open_barrier->header, task_head);
+ } else if (dependent_task_count > 1) {
+ // Allocate the list of tasks we'll stash back on the previous barrier.
+ // Since we couldn't know at the time how many tasks would end up in the
+ // barrier we had to defer it until now.
+ iree_task_t** dependent_tasks = NULL;
+ IREE_RETURN_IF_ERROR(iree_arena_allocate(
+ &command_buffer->arena, dependent_task_count * sizeof(iree_task_t*),
+ (void**)&dependent_tasks));
+ iree_task_t* task = task_head;
+ for (iree_host_size_t i = 0; i < dependent_task_count; ++i) {
+ dependent_tasks[i] = task;
+ task = task->next_task;
+ }
+ iree_task_barrier_set_dependent_tasks(open_barrier, dependent_task_count,
+ dependent_tasks);
+ }
+ }
+ command_buffer->state.open_barrier = NULL;
+
+ // Move the open tasks to the tail as they represent the first half of the
+ // *next* barrier that will be inserted.
+ if (command_buffer->state.open_task_count > 0) {
+ iree_task_list_move(&command_buffer->state.open_tasks,
+ &command_buffer->leaf_tasks);
+ command_buffer->state.open_task_count = 0;
+ }
+
+ return iree_ok_status();
+}
+
+// Emits a global barrier, splitting execution into all prior recorded tasks
+// and all subsequent recorded tasks. This is currently the critical piece that
+// limits our concurrency: changing to fine-grained barriers (via barrier
+// buffers or events) will allow more work to overlap at the cost of more brain
+// to build out the proper task graph.
+static iree_status_t iree_hal_task_command_buffer_emit_global_barrier(
+ iree_hal_task_command_buffer_t* command_buffer) {
+ // Flush open tasks to the previous barrier. This resets our state such that
+ // we can assign the new open barrier and start recording tasks for it.
+ // Previous tasks will be moved into the leaf_tasks list.
+ IREE_RETURN_IF_ERROR(
+ iree_hal_task_command_buffer_flush_tasks(command_buffer));
+
+ // Allocate the new open barrier.
+ // As we are recording forward we can't yet assign the dependent tasks (the
+ // second half of the synchronization domain) and instead are just inserting
+ // it so we can setup the join from previous tasks (the first half of the
+ // synchronization domain).
+ iree_task_barrier_t* barrier = NULL;
+ IREE_RETURN_IF_ERROR(iree_arena_allocate(&command_buffer->arena,
+ sizeof(*barrier), (void**)&barrier));
+ iree_task_barrier_initialize_empty(command_buffer->scope, barrier);
+
+ // If there were previous tasks then join them to the barrier.
+ for (iree_task_t* task = iree_task_list_front(&command_buffer->leaf_tasks);
+ task != NULL; task = task->next_task) {
+ iree_task_set_completion_task(task, &barrier->header);
+ }
+
+ // Move the tasks from the leaf list (tail) to the root list (head) if this
+ // was the first set of tasks recorded.
+ if (iree_task_list_is_empty(&command_buffer->root_tasks) &&
+ !iree_task_list_is_empty(&command_buffer->leaf_tasks)) {
+ iree_task_list_move(&command_buffer->leaf_tasks,
+ &command_buffer->root_tasks);
+ }
+
+ // Reset the tail of the command buffer to the barrier. This leaves us in a
+ // consistent state if the recording ends immediate after this (the barrier
+ // will be the last task).
+ iree_task_list_initialize(&command_buffer->leaf_tasks);
+ iree_task_list_push_back(&command_buffer->leaf_tasks, &barrier->header);
+
+ // NOTE: all new tasks emitted will be executed after this barrier.
+ command_buffer->state.open_barrier = barrier;
+ command_buffer->state.open_task_count = 0;
+
+ return iree_ok_status();
+}
+
+// Emits a the given execution |task| into the current open synchronization
+// scope (after state.open_barrier and before the next barrier).
+static iree_status_t iree_hal_task_command_buffer_emit_execution_task(
+ iree_hal_task_command_buffer_t* command_buffer, iree_task_t* task) {
+ if (command_buffer->state.open_barrier == NULL) {
+ // If there is no open barrier then we are at the head and going right into
+ // the task DAG.
+ iree_task_list_push_back(&command_buffer->leaf_tasks, task);
+ } else {
+ // Append to the open task list that will be flushed to the open barrier.
+ iree_task_list_push_back(&command_buffer->state.open_tasks, task);
+ ++command_buffer->state.open_task_count;
+ }
+ return iree_ok_status();
+}
+
+//===----------------------------------------------------------------------===//
+// iree_hal_task_command_buffer_t execution
+//===----------------------------------------------------------------------===//
+
+iree_status_t iree_hal_task_command_buffer_issue(
+ iree_hal_command_buffer_t* base_command_buffer,
+ iree_hal_task_queue_state_t* queue_state, iree_task_t* retire_task,
+ iree_arena_allocator_t* arena, iree_task_submission_t* pending_submission) {
+ iree_hal_task_command_buffer_t* command_buffer =
+ iree_hal_task_command_buffer_cast(base_command_buffer);
+
+ // If the command buffer is empty (valid!) then we are a no-op.
+ bool has_root_tasks = !iree_task_list_is_empty(&command_buffer->root_tasks);
+ if (!has_root_tasks) {
+ return iree_ok_status();
+ }
+
+ bool has_leaf_tasks = !iree_task_list_is_empty(&command_buffer->leaf_tasks);
+ if (has_leaf_tasks) {
+ // Chain the retire task onto the leaf tasks as their completion indicates
+ // that all commands have completed.
+ for (iree_task_t* task = command_buffer->leaf_tasks.head; task != NULL;
+ task = task->next_task) {
+ iree_task_set_completion_task(task, retire_task);
+ }
+ } else {
+ // If we have no leaf tasks it means that this is a single layer DAG and
+ // after the root tasks complete the entire command buffer has completed.
+ for (iree_task_t* task = command_buffer->root_tasks.head; task != NULL;
+ task = task->next_task) {
+ iree_task_set_completion_task(task, retire_task);
+ }
+ }
+
+ // Enqueue all root tasks that are ready to run immediately.
+ // After this all of the command buffer tasks are owned by the submission and
+ // we need to ensure the command buffer doesn't try to discard them.
+ iree_task_submission_enqueue_list(pending_submission,
+ &command_buffer->root_tasks);
+ iree_task_list_initialize(&command_buffer->leaf_tasks);
+
+ return iree_ok_status();
+}
+
+//===----------------------------------------------------------------------===//
+// iree_hal_command_buffer_execution_barrier
+//===----------------------------------------------------------------------===//
+
+static iree_status_t iree_hal_task_command_buffer_execution_barrier(
+ iree_hal_command_buffer_t* base_command_buffer,
+ iree_hal_execution_stage_t source_stage_mask,
+ iree_hal_execution_stage_t target_stage_mask,
+ iree_host_size_t memory_barrier_count,
+ const iree_hal_memory_barrier_t* memory_barriers,
+ iree_host_size_t buffer_barrier_count,
+ const iree_hal_buffer_barrier_t* buffer_barriers) {
+ iree_hal_task_command_buffer_t* command_buffer =
+ (iree_hal_task_command_buffer_t*)base_command_buffer;
+
+ // TODO(benvanik): actual DAG construction. Right now we are just doing simple
+ // global barriers each time and forcing a join-fork point.
+ return iree_hal_task_command_buffer_emit_global_barrier(command_buffer);
+}
+
+//===----------------------------------------------------------------------===//
+// iree_hal_command_buffer_signal_event
+//===----------------------------------------------------------------------===//
+
+static iree_status_t iree_hal_task_command_buffer_signal_event(
+ iree_hal_command_buffer_t* base_command_buffer, iree_hal_event_t* event,
+ iree_hal_execution_stage_t source_stage_mask) {
+ // TODO(#4518): implement events. For now we just insert global barriers.
+ return iree_ok_status();
+}
+
+//===----------------------------------------------------------------------===//
+// iree_hal_command_buffer_reset_event
+//===----------------------------------------------------------------------===//
+
+static iree_status_t iree_hal_task_command_buffer_reset_event(
+ iree_hal_command_buffer_t* base_command_buffer, iree_hal_event_t* event,
+ iree_hal_execution_stage_t source_stage_mask) {
+ // TODO(#4518): implement events. For now we just insert global barriers.
+ return iree_ok_status();
+}
+
+//===----------------------------------------------------------------------===//
+// iree_hal_command_buffer_wait_events
+//===----------------------------------------------------------------------===//
+
+static iree_status_t iree_hal_task_command_buffer_wait_events(
+ iree_hal_command_buffer_t* base_command_buffer,
+ iree_host_size_t event_count, const iree_hal_event_t** events,
+ iree_hal_execution_stage_t source_stage_mask,
+ iree_hal_execution_stage_t target_stage_mask,
+ iree_host_size_t memory_barrier_count,
+ const iree_hal_memory_barrier_t* memory_barriers,
+ iree_host_size_t buffer_barrier_count,
+ const iree_hal_buffer_barrier_t* buffer_barriers) {
+ iree_hal_task_command_buffer_t* command_buffer =
+ iree_hal_task_command_buffer_cast(base_command_buffer);
+ // TODO(#4518): implement events. For now we just insert global barriers.
+ return iree_hal_task_command_buffer_emit_global_barrier(command_buffer);
+}
+
+//===----------------------------------------------------------------------===//
+// iree_hal_command_buffer_discard_buffer
+//===----------------------------------------------------------------------===//
+
+static iree_status_t iree_hal_task_command_buffer_discard_buffer(
+ iree_hal_command_buffer_t* base_command_buffer, iree_hal_buffer_t* buffer) {
+ return iree_ok_status();
+}
+
+//===----------------------------------------------------------------------===//
+// iree_hal_command_buffer_fill_buffer
+//===----------------------------------------------------------------------===//
+// NOTE: for large fills we could dispatch this as tiles for parallelism.
+// We'd want to do some measurement for when it's worth it; filling a 200KB
+// buffer: maybe not, filling a 200MB buffer: yeah.
+
+typedef struct {
+ iree_task_call_t task;
+ iree_hal_buffer_t* target_buffer;
+ iree_device_size_t target_offset;
+ iree_device_size_t length;
+ uint32_t pattern_length;
+ uint8_t pattern[8];
+} iree_hal_cmd_fill_buffer_t;
+
+static iree_status_t iree_hal_cmd_fill_buffer(
+ uintptr_t user_context, iree_task_t* task,
+ iree_task_submission_t* pending_submission) {
+ const iree_hal_cmd_fill_buffer_t* cmd =
+ (const iree_hal_cmd_fill_buffer_t*)user_context;
+ IREE_TRACE_ZONE_BEGIN(z0);
+ iree_status_t status =
+ iree_hal_buffer_fill(cmd->target_buffer, cmd->target_offset, cmd->length,
+ cmd->pattern, cmd->pattern_length);
+ IREE_TRACE_ZONE_END(z0);
+ return status;
+}
+
+static iree_status_t iree_hal_task_command_buffer_fill_buffer(
+ iree_hal_command_buffer_t* base_command_buffer,
+ iree_hal_buffer_t* target_buffer, iree_device_size_t target_offset,
+ iree_device_size_t length, const void* pattern,
+ iree_host_size_t pattern_length) {
+ iree_hal_task_command_buffer_t* command_buffer =
+ (iree_hal_task_command_buffer_t*)base_command_buffer;
+
+ iree_hal_cmd_fill_buffer_t* cmd = NULL;
+ IREE_RETURN_IF_ERROR(
+ iree_arena_allocate(&command_buffer->arena, sizeof(*cmd), (void**)&cmd));
+
+ iree_task_call_initialize(
+ command_buffer->scope,
+ iree_task_make_call_closure(iree_hal_cmd_fill_buffer, (uintptr_t)cmd),
+ &cmd->task);
+ cmd->target_buffer = target_buffer;
+ cmd->target_offset = target_offset;
+ cmd->length = length;
+ memcpy(cmd->pattern, pattern, pattern_length);
+ cmd->pattern_length = pattern_length;
+
+ return iree_hal_task_command_buffer_emit_execution_task(command_buffer,
+ &cmd->task.header);
+}
+
+//===----------------------------------------------------------------------===//
+// iree_hal_command_buffer_update_buffer
+//===----------------------------------------------------------------------===//
+
+typedef struct {
+ iree_task_call_t task;
+ iree_hal_buffer_t* target_buffer;
+ iree_device_size_t target_offset;
+ iree_device_size_t length;
+ uint8_t source_buffer[];
+} iree_hal_cmd_update_buffer_t;
+
+static iree_status_t iree_hal_cmd_update_buffer(
+ uintptr_t user_context, iree_task_t* task,
+ iree_task_submission_t* pending_submission) {
+ const iree_hal_cmd_update_buffer_t* cmd =
+ (const iree_hal_cmd_update_buffer_t*)user_context;
+ IREE_TRACE_ZONE_BEGIN(z0);
+ iree_status_t status = iree_hal_buffer_write_data(
+ cmd->target_buffer, cmd->target_offset, cmd->source_buffer, cmd->length);
+ IREE_TRACE_ZONE_END(z0);
+ return status;
+}
+
+static iree_status_t iree_hal_task_command_buffer_update_buffer(
+ iree_hal_command_buffer_t* base_command_buffer, const void* source_buffer,
+ iree_host_size_t source_offset, iree_hal_buffer_t* target_buffer,
+ iree_device_size_t target_offset, iree_device_size_t length) {
+ iree_hal_task_command_buffer_t* command_buffer =
+ (iree_hal_task_command_buffer_t*)base_command_buffer;
+
+ iree_host_size_t total_cmd_size =
+ sizeof(iree_hal_cmd_update_buffer_t) + length;
+
+ iree_hal_cmd_update_buffer_t* cmd = NULL;
+ IREE_RETURN_IF_ERROR(iree_arena_allocate(&command_buffer->arena,
+ total_cmd_size, (void**)&cmd));
+
+ iree_task_call_initialize(
+ command_buffer->scope,
+ iree_task_make_call_closure(iree_hal_cmd_update_buffer, (uintptr_t)cmd),
+ &cmd->task);
+ cmd->target_buffer = (iree_hal_buffer_t*)target_buffer;
+ cmd->target_offset = target_offset;
+ cmd->length = length;
+
+ memcpy(cmd->source_buffer, (const uint8_t*)source_buffer + source_offset,
+ cmd->length);
+
+ return iree_hal_task_command_buffer_emit_execution_task(command_buffer,
+ &cmd->task.header);
+}
+
+//===----------------------------------------------------------------------===//
+// iree_hal_command_buffer_copy_buffer
+//===----------------------------------------------------------------------===//
+// NOTE: for large copies we could dispatch this as tiles for parallelism.
+// We'd want to do some measurement for when it's worth it; copying a 200KB
+// buffer: maybe not, copying a 200MB buffer: yeah.
+
+typedef struct {
+ iree_task_call_t task;
+ iree_hal_buffer_t* source_buffer;
+ iree_device_size_t source_offset;
+ iree_hal_buffer_t* target_buffer;
+ iree_device_size_t target_offset;
+ iree_device_size_t length;
+} iree_hal_cmd_copy_buffer_t;
+
+static iree_status_t iree_hal_cmd_copy_buffer(
+ uintptr_t user_context, iree_task_t* task,
+ iree_task_submission_t* pending_submission) {
+ const iree_hal_cmd_copy_buffer_t* cmd =
+ (const iree_hal_cmd_copy_buffer_t*)user_context;
+ IREE_TRACE_ZONE_BEGIN(z0);
+ iree_status_t status = iree_hal_buffer_copy_data(
+ cmd->source_buffer, cmd->source_offset, cmd->target_buffer,
+ cmd->target_offset, cmd->length);
+ IREE_TRACE_ZONE_END(z0);
+ return status;
+}
+
+static iree_status_t iree_hal_task_command_buffer_copy_buffer(
+ iree_hal_command_buffer_t* base_command_buffer,
+ iree_hal_buffer_t* source_buffer, iree_device_size_t source_offset,
+ iree_hal_buffer_t* target_buffer, iree_device_size_t target_offset,
+ iree_device_size_t length) {
+ iree_hal_task_command_buffer_t* command_buffer =
+ (iree_hal_task_command_buffer_t*)base_command_buffer;
+
+ iree_hal_cmd_copy_buffer_t* cmd = NULL;
+ IREE_RETURN_IF_ERROR(
+ iree_arena_allocate(&command_buffer->arena, sizeof(*cmd), (void**)&cmd));
+
+ iree_task_call_initialize(
+ command_buffer->scope,
+ iree_task_make_call_closure(iree_hal_cmd_copy_buffer, (uintptr_t)cmd),
+ &cmd->task);
+ cmd->source_buffer = (iree_hal_buffer_t*)source_buffer;
+ cmd->source_offset = source_offset;
+ cmd->target_buffer = (iree_hal_buffer_t*)target_buffer;
+ cmd->target_offset = target_offset;
+ cmd->length = length;
+
+ return iree_hal_task_command_buffer_emit_execution_task(command_buffer,
+ &cmd->task.header);
+}
+
+//===----------------------------------------------------------------------===//
+// iree_hal_command_buffer_push_constants
+//===----------------------------------------------------------------------===//
+// NOTE: command buffer state change only; enqueues no tasks.
+
+static iree_status_t iree_hal_task_command_buffer_push_constants(
+ iree_hal_command_buffer_t* base_command_buffer,
+ iree_hal_executable_layout_t* executable_layout, iree_host_size_t offset,
+ const void* values, iree_host_size_t values_length) {
+ iree_hal_task_command_buffer_t* command_buffer =
+ (iree_hal_task_command_buffer_t*)base_command_buffer;
+
+ if (IREE_UNLIKELY(offset + values_length >=
+ IREE_HAL_LOCAL_MAX_PUSH_CONSTANT_COUNT)) {
+ return iree_make_status(IREE_STATUS_INVALID_ARGUMENT,
+ "push constant range %zu (length=%zu) out of range",
+ offset, values_length);
+ }
+
+ memcpy(&command_buffer->state.push_constants[offset], values,
+ values_length * sizeof(uint32_t));
+
+ return iree_ok_status();
+}
+
+//===----------------------------------------------------------------------===//
+// iree_hal_command_buffer_push_descriptor_set
+//===----------------------------------------------------------------------===//
+// NOTE: command buffer state change only; enqueues no tasks.
+
+static iree_status_t iree_hal_task_command_buffer_push_descriptor_set(
+ iree_hal_command_buffer_t* base_command_buffer,
+ iree_hal_executable_layout_t* executable_layout, uint32_t set,
+ iree_host_size_t binding_count,
+ const iree_hal_descriptor_set_binding_t* bindings) {
+ iree_hal_task_command_buffer_t* command_buffer =
+ (iree_hal_task_command_buffer_t*)base_command_buffer;
+
+ if (IREE_UNLIKELY(set >= IREE_HAL_LOCAL_MAX_DESCRIPTOR_SET_COUNT)) {
+ return iree_make_status(IREE_STATUS_INVALID_ARGUMENT,
+ "set %u out of bounds", set);
+ }
+
+ iree_hal_local_executable_layout_t* local_executable_layout =
+ iree_hal_local_executable_layout_cast(executable_layout);
+ iree_hal_local_descriptor_set_layout_t* local_set_layout =
+ iree_hal_local_descriptor_set_layout_cast(
+ local_executable_layout->set_layouts[set]);
+
+ iree_host_size_t binding_base =
+ set * IREE_HAL_LOCAL_MAX_DESCRIPTOR_BINDING_COUNT;
+ for (iree_host_size_t i = 0; i < binding_count; ++i) {
+ if (IREE_UNLIKELY(bindings[i].binding >=
+ IREE_HAL_LOCAL_MAX_DESCRIPTOR_BINDING_COUNT)) {
+ return iree_make_status(IREE_STATUS_INVALID_ARGUMENT,
+ "buffer binding index out of bounds");
+ }
+ iree_host_size_t binding_ordinal = binding_base + bindings[i].binding;
+
+ // TODO(benvanik): track mapping so we can properly map/unmap/flush/etc.
+ iree_hal_buffer_mapping_t buffer_mapping;
+ IREE_RETURN_IF_ERROR(iree_hal_buffer_map_range(
+ bindings[i].buffer, local_set_layout->bindings[binding_ordinal].access,
+ bindings[i].offset, bindings[i].length, &buffer_mapping));
+ command_buffer->state.bindings[binding_ordinal] =
+ buffer_mapping.contents.data;
+ command_buffer->state.binding_lengths[binding_ordinal] =
+ buffer_mapping.contents.data_length;
+ }
+
+ return iree_ok_status();
+}
+
+//===----------------------------------------------------------------------===//
+// iree_hal_command_buffer_bind_descriptor_set
+//===----------------------------------------------------------------------===//
+// NOTE: command buffer state change only; enqueues no tasks.
+
+static iree_status_t iree_hal_task_command_buffer_bind_descriptor_set(
+ iree_hal_command_buffer_t* base_command_buffer,
+ iree_hal_executable_layout_t* executable_layout, uint32_t set,
+ iree_hal_descriptor_set_t* descriptor_set,
+ iree_host_size_t dynamic_offset_count,
+ const iree_device_size_t* dynamic_offsets) {
+ return iree_make_status(IREE_STATUS_UNIMPLEMENTED,
+ "descriptor set binding not yet implemented");
+}
+
+//===----------------------------------------------------------------------===//
+// iree_hal_command_buffer_dispatch
+//===----------------------------------------------------------------------===//
+
+typedef struct {
+ iree_task_dispatch_t task;
+ iree_hal_local_executable_t* executable;
+ iree_host_size_t ordinal;
+ iree_hal_executable_binding_ptr_t* restrict bindings;
+ iree_device_size_t* restrict binding_lengths;
+ uint32_t* restrict push_constants;
+} iree_hal_cmd_dispatch_t;
+
+static iree_status_t iree_hal_cmd_dispatch_tile(
+ uintptr_t user_context, const iree_task_tile_context_t* tile_context,
+ iree_task_submission_t* pending_submission) {
+ const iree_hal_cmd_dispatch_t* cmd =
+ (const iree_hal_cmd_dispatch_t*)user_context;
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ iree_hal_executable_dispatch_state_v0_t state;
+ // TODO(benvanik): wire up device state (imports, etc) and cache on the
+ // command buffer for reuse across all tiles.
+
+ iree_hal_local_executable_call_t call = {
+ .state = &state,
+ .push_constants = cmd->push_constants,
+ .bindings = cmd->bindings,
+ .binding_lengths = cmd->binding_lengths,
+ };
+ memcpy(call.workgroup_id.value, tile_context->workgroup_xyz,
+ sizeof(iree_hal_vec3_t));
+ memcpy(call.workgroup_size.value, tile_context->workgroup_size,
+ sizeof(iree_hal_vec3_t));
+ memcpy(call.workgroup_count.value, tile_context->workgroup_count,
+ sizeof(iree_hal_vec3_t));
+ iree_status_t status = iree_hal_local_executable_issue_call(
+ cmd->executable, cmd->ordinal, &call);
+
+ IREE_TRACE_ZONE_END(z0);
+ return status;
+}
+
+static iree_status_t iree_hal_task_command_buffer_build_dispatch(
+ iree_hal_command_buffer_t* base_command_buffer,
+ iree_hal_executable_t* executable, int32_t entry_point,
+ uint32_t workgroup_x, uint32_t workgroup_y, uint32_t workgroup_z,
+ iree_hal_cmd_dispatch_t** out_cmd) {
+ iree_hal_task_command_buffer_t* command_buffer =
+ (iree_hal_task_command_buffer_t*)base_command_buffer;
+
+ iree_hal_local_executable_t* local_executable =
+ iree_hal_local_executable_cast(executable);
+ iree_host_size_t push_constant_count =
+ local_executable->layout->push_constants;
+ iree_hal_local_binding_mask_t used_binding_mask =
+ local_executable->layout->used_bindings;
+ iree_host_size_t used_binding_count =
+ iree_math_count_ones_u64(used_binding_mask);
+
+ iree_hal_cmd_dispatch_t* cmd = NULL;
+ iree_host_size_t total_cmd_size =
+ sizeof(*cmd) + push_constant_count * sizeof(uint32_t) +
+ used_binding_count * sizeof(iree_hal_executable_binding_ptr_t) +
+ used_binding_count * sizeof(iree_device_size_t);
+ IREE_RETURN_IF_ERROR(iree_arena_allocate(&command_buffer->arena,
+ total_cmd_size, (void**)&cmd));
+
+ cmd->executable = local_executable;
+ cmd->ordinal = entry_point;
+
+ uint32_t workgroup_count[3] = {workgroup_x, workgroup_y, workgroup_z};
+ // TODO(benvanik): expose on API or keep fixed on executable.
+ uint32_t workgroup_size[3] = {1, 1, 1};
+ iree_task_dispatch_initialize(command_buffer->scope,
+ iree_task_make_dispatch_closure(
+ iree_hal_cmd_dispatch_tile, (uintptr_t)cmd),
+ workgroup_size, workgroup_count, &cmd->task);
+
+ // Copy only the push constant range used by the executable.
+ uint8_t* cmd_ptr = (uint8_t*)cmd + sizeof(*cmd);
+ cmd->push_constants = (uint32_t*)cmd_ptr;
+ memcpy(cmd->push_constants, command_buffer->state.push_constants,
+ push_constant_count * sizeof(*cmd->push_constants));
+ cmd_ptr += push_constant_count * sizeof(*cmd->push_constants);
+
+ // Produce the dense binding list based on the declared bindings used.
+ // This allows us to change the descriptor sets and bindings counts supported
+ // in the HAL independent of any executable as each executable just gets the
+ // flat dense list and doesn't care about our descriptor set stuff.
+ //
+ // Note that we are just directly setting the binding data pointers here with
+ // no ownership/retaining/etc - it's part of the HAL contract that buffers are
+ // kept valid for the duration they may be in use.
+ cmd->bindings = (iree_hal_executable_binding_ptr_t*)cmd_ptr;
+ cmd_ptr += used_binding_count * sizeof(*cmd->bindings);
+ cmd->binding_lengths = (iree_device_size_t*)cmd_ptr;
+ cmd_ptr += used_binding_count * sizeof(*cmd->binding_lengths);
+ iree_host_size_t binding_base = 0;
+ for (iree_host_size_t i = 0; i < used_binding_count; ++i) {
+ int mask_offset = iree_math_count_trailing_zeros_u64(used_binding_mask);
+ int binding_ordinal = binding_base + mask_offset;
+ binding_base += mask_offset + 1;
+ used_binding_mask = used_binding_mask >> (mask_offset + 1);
+ cmd->bindings[i] = command_buffer->state.bindings[binding_ordinal];
+ cmd->binding_lengths[i] =
+ command_buffer->state.binding_lengths[binding_ordinal];
+ if (!cmd->bindings[i]) {
+ return iree_make_status(IREE_STATUS_FAILED_PRECONDITION,
+ "(flat) binding %d is NULL", binding_ordinal);
+ }
+ }
+
+ *out_cmd = cmd;
+ return iree_hal_task_command_buffer_emit_execution_task(command_buffer,
+ &cmd->task.header);
+}
+
+static iree_status_t iree_hal_task_command_buffer_dispatch(
+ iree_hal_command_buffer_t* base_command_buffer,
+ iree_hal_executable_t* executable, int32_t entry_point,
+ uint32_t workgroup_x, uint32_t workgroup_y, uint32_t workgroup_z) {
+ iree_hal_cmd_dispatch_t* cmd = NULL;
+ return iree_hal_task_command_buffer_build_dispatch(
+ base_command_buffer, executable, entry_point, workgroup_x, workgroup_y,
+ workgroup_z, &cmd);
+}
+
+static iree_status_t iree_hal_task_command_buffer_dispatch_indirect(
+ iree_hal_command_buffer_t* base_command_buffer,
+ iree_hal_executable_t* executable, int32_t entry_point,
+ iree_hal_buffer_t* workgroups_buffer,
+ iree_device_size_t workgroups_offset) {
+ // TODO(benvanik): track mapping so we can properly map/unmap/flush/etc.
+ iree_hal_buffer_mapping_t buffer_mapping;
+ IREE_RETURN_IF_ERROR(iree_hal_buffer_map_range(
+ workgroups_buffer, IREE_HAL_MEMORY_ACCESS_READ, workgroups_offset,
+ 3 * sizeof(uint32_t), &buffer_mapping));
+
+ iree_hal_cmd_dispatch_t* cmd = NULL;
+ IREE_RETURN_IF_ERROR(iree_hal_task_command_buffer_build_dispatch(
+ base_command_buffer, executable, entry_point, 0, 0, 0, &cmd));
+ cmd->task.workgroup_count.ptr = (const uint32_t*)buffer_mapping.contents.data;
+ cmd->task.header.flags |= IREE_TASK_FLAG_DISPATCH_INDIRECT;
+ return iree_ok_status();
+}
+
+//===----------------------------------------------------------------------===//
+// iree_hal_command_buffer_vtable_t
+//===----------------------------------------------------------------------===//
+
+static const iree_hal_command_buffer_vtable_t
+ iree_hal_task_command_buffer_vtable = {
+ .destroy = iree_hal_task_command_buffer_destroy,
+ .allowed_categories = iree_hal_task_command_buffer_allowed_categories,
+ .begin = iree_hal_task_command_buffer_begin,
+ .end = iree_hal_task_command_buffer_end,
+ .execution_barrier = iree_hal_task_command_buffer_execution_barrier,
+ .signal_event = iree_hal_task_command_buffer_signal_event,
+ .reset_event = iree_hal_task_command_buffer_reset_event,
+ .wait_events = iree_hal_task_command_buffer_wait_events,
+ .discard_buffer = iree_hal_task_command_buffer_discard_buffer,
+ .fill_buffer = iree_hal_task_command_buffer_fill_buffer,
+ .update_buffer = iree_hal_task_command_buffer_update_buffer,
+ .copy_buffer = iree_hal_task_command_buffer_copy_buffer,
+ .push_constants = iree_hal_task_command_buffer_push_constants,
+ .push_descriptor_set = iree_hal_task_command_buffer_push_descriptor_set,
+ .bind_descriptor_set = iree_hal_task_command_buffer_bind_descriptor_set,
+ .dispatch = iree_hal_task_command_buffer_dispatch,
+ .dispatch_indirect = iree_hal_task_command_buffer_dispatch_indirect,
+};
diff --git a/iree/hal/local/task_command_buffer.h b/iree/hal/local/task_command_buffer.h
new file mode 100644
index 0000000..ece7388
--- /dev/null
+++ b/iree/hal/local/task_command_buffer.h
@@ -0,0 +1,60 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef IREE_HAL_LOCAL_TASK_COMMAND_BUFFER_H_
+#define IREE_HAL_LOCAL_TASK_COMMAND_BUFFER_H_
+
+#include "iree/base/api.h"
+#include "iree/hal/api.h"
+#include "iree/hal/local/arena.h"
+#include "iree/hal/local/task_queue_state.h"
+#include "iree/task/scope.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif // __cplusplus
+
+iree_status_t iree_hal_task_command_buffer_create(
+ iree_hal_device_t* device, iree_task_scope_t* scope,
+ iree_hal_command_buffer_mode_t mode,
+ iree_hal_command_category_t command_categories,
+ iree_arena_block_pool_t* block_pool,
+ iree_hal_command_buffer_t** out_command_buffer);
+
+// Issues a recorded command buffer using the serial |queue_state|.
+// |queue_state| is used to track the synchronization scope of the queue from
+// prior commands such as signaled events and will be mutated as events are
+// reset or new events are signaled.
+//
+// |retire_task| will be scheduled once all commands issued from the command
+// buffer retire and can be used as a fence point.
+//
+// Any new tasks that are allocated as part of the issue operation (such as
+// barrier tasks to handle event synchronization) will be acquired from |arena|.
+// The lifetime of |arena| must be at least that of |retire_task| ensuring that
+// all of the allocated commands issued have completed and their memory in the
+// arena can be recycled.
+//
+// |pending_submission| will receive the ready list of commands and must be
+// submitted to the executor (or discarded on failure) by the caller.
+iree_status_t iree_hal_task_command_buffer_issue(
+ iree_hal_command_buffer_t* command_buffer,
+ iree_hal_task_queue_state_t* queue_state, iree_task_t* retire_task,
+ iree_arena_allocator_t* arena, iree_task_submission_t* pending_submission);
+
+#ifdef __cplusplus
+} // extern "C"
+#endif // __cplusplus
+
+#endif // IREE_HAL_LOCAL_TASK_COMMAND_BUFFER_H_
diff --git a/iree/hal/local/task_device.c b/iree/hal/local/task_device.c
new file mode 100644
index 0000000..ec2071f
--- /dev/null
+++ b/iree/hal/local/task_device.c
@@ -0,0 +1,351 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "iree/hal/local/task_device.h"
+
+#include "iree/base/tracing.h"
+#include "iree/hal/local/arena.h"
+#include "iree/hal/local/event_pool.h"
+#include "iree/hal/local/local_descriptor_set.h"
+#include "iree/hal/local/local_descriptor_set_layout.h"
+#include "iree/hal/local/local_executable.h"
+#include "iree/hal/local/local_executable_cache.h"
+#include "iree/hal/local/local_executable_layout.h"
+#include "iree/hal/local/task_command_buffer.h"
+#include "iree/hal/local/task_event.h"
+#include "iree/hal/local/task_queue.h"
+#include "iree/hal/local/task_semaphore.h"
+
+#define IREE_HAL_LOCAL_TASK_EVENT_POOL_CAPACITY 32
+
+typedef struct {
+ iree_hal_resource_t resource;
+ iree_string_view_t identifier;
+
+ // Block pool used for small allocations like tasks and submissions.
+ iree_arena_block_pool_t small_block_pool;
+
+ // Block pool used for command buffers with a larger block size (as command
+ // buffers can contain inlined data uploads).
+ iree_arena_block_pool_t large_block_pool;
+
+ // iree_event_t pool for semaphore wait operations.
+ iree_hal_local_event_pool_t* event_pool;
+
+ iree_task_executor_t* executor;
+
+ iree_host_size_t loader_count;
+ iree_hal_executable_loader_t** loaders;
+
+ iree_allocator_t host_allocator;
+ iree_hal_allocator_t* device_allocator;
+
+ iree_host_size_t queue_count;
+ iree_hal_task_queue_t queues[];
+} iree_hal_task_device_t;
+
+static const iree_hal_device_vtable_t iree_hal_task_device_vtable;
+
+static iree_hal_task_device_t* iree_hal_task_device_cast(
+ iree_hal_device_t* base_value) {
+ IREE_HAL_ASSERT_TYPE(base_value, &iree_hal_task_device_vtable);
+ return (iree_hal_task_device_t*)base_value;
+}
+
+void iree_hal_task_device_params_initialize(
+ iree_hal_task_device_params_t* out_params) {
+ out_params->arena_block_size = 32 * 1024;
+ out_params->queue_count = 8;
+}
+
+static iree_status_t iree_hal_task_device_check_params(
+ const iree_hal_task_device_params_t* params) {
+ if (params->arena_block_size < 4096) {
+ return iree_make_status(IREE_STATUS_INVALID_ARGUMENT,
+ "arena block size too small (< 4096 bytes)");
+ }
+ if (params->queue_count == 0) {
+ return iree_make_status(IREE_STATUS_INVALID_ARGUMENT,
+ "at least one queue is required");
+ }
+ return iree_ok_status();
+}
+
+static const iree_hal_device_vtable_t iree_hal_task_device_vtable;
+
+iree_status_t iree_hal_task_device_create(
+ iree_string_view_t identifier, const iree_hal_task_device_params_t* params,
+ iree_task_executor_t* executor, iree_host_size_t loader_count,
+ iree_hal_executable_loader_t** loaders, iree_allocator_t host_allocator,
+ iree_hal_device_t** out_device) {
+ IREE_ASSERT_ARGUMENT(params);
+ IREE_ASSERT_ARGUMENT(!loader_count || loaders);
+ IREE_ASSERT_ARGUMENT(out_device);
+ *out_device = NULL;
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ IREE_RETURN_AND_END_ZONE_IF_ERROR(z0,
+ iree_hal_task_device_check_params(params));
+
+ iree_hal_task_device_t* device = NULL;
+ iree_host_size_t total_size =
+ sizeof(*device) + params->queue_count * sizeof(*device->queues) +
+ identifier.size + loader_count * sizeof(*device->loaders);
+ iree_status_t status =
+ iree_allocator_malloc(host_allocator, total_size, (void**)&device);
+ if (iree_status_is_ok(status)) {
+ memset(device, 0, total_size);
+ iree_hal_resource_initialize(&iree_hal_task_device_vtable,
+ &device->resource);
+ iree_string_view_append_to_buffer(
+ identifier, &device->identifier,
+ (char*)device + sizeof(*device) +
+ params->queue_count * sizeof(*device->queues));
+ device->host_allocator = host_allocator;
+ iree_arena_block_pool_initialize(4096, host_allocator,
+ &device->small_block_pool);
+ iree_arena_block_pool_initialize(params->arena_block_size, host_allocator,
+ &device->large_block_pool);
+ device->event_pool = NULL;
+
+ device->executor = executor;
+ iree_task_executor_retain(device->executor);
+
+ device->loader_count = loader_count;
+ device->loaders =
+ (iree_hal_executable_loader_t**)((uint8_t*)device->identifier.data +
+ identifier.size);
+ for (iree_host_size_t i = 0; i < device->loader_count; ++i) {
+ device->loaders[i] = loaders[i];
+ iree_hal_executable_loader_retain(device->loaders[i]);
+ }
+
+ device->queue_count = params->queue_count;
+ for (iree_host_size_t i = 0; i < device->queue_count; ++i) {
+ // TODO(benvanik): add a number to each queue ID.
+ iree_hal_task_queue_initialize(device->identifier, device->executor,
+ &device->small_block_pool,
+ &device->queues[i]);
+ }
+ }
+
+ if (iree_status_is_ok(status)) {
+ status = iree_hal_allocator_create_heap(identifier, host_allocator,
+ &device->device_allocator);
+ }
+
+ if (iree_status_is_ok(status)) {
+ status = iree_hal_local_event_pool_allocate(
+ IREE_HAL_LOCAL_TASK_EVENT_POOL_CAPACITY, host_allocator,
+ &device->event_pool);
+ }
+
+ if (iree_status_is_ok(status)) {
+ *out_device = (iree_hal_device_t*)device;
+ } else {
+ iree_hal_device_release((iree_hal_device_t*)device);
+ }
+ IREE_TRACE_ZONE_END(z0);
+ return status;
+}
+
+static void iree_hal_task_device_destroy(iree_hal_device_t* base_device) {
+ iree_hal_task_device_t* device = (iree_hal_task_device_t*)base_device;
+ iree_allocator_t host_allocator = iree_hal_device_host_allocator(base_device);
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ for (iree_host_size_t i = 0; i < device->queue_count; ++i) {
+ iree_hal_task_queue_deinitialize(&device->queues[i]);
+ }
+ for (iree_host_size_t i = 0; i < device->loader_count; ++i) {
+ iree_hal_executable_loader_release(device->loaders[i]);
+ }
+ iree_task_executor_release(device->executor);
+ iree_hal_local_event_pool_free(device->event_pool);
+ iree_arena_block_pool_deinitialize(&device->large_block_pool);
+ iree_arena_block_pool_deinitialize(&device->small_block_pool);
+ iree_hal_allocator_release(device->device_allocator);
+ iree_allocator_free(host_allocator, device);
+
+ IREE_TRACE_ZONE_END(z0);
+}
+
+static iree_string_view_t iree_hal_task_device_id(
+ iree_hal_device_t* base_device) {
+ iree_hal_task_device_t* device = (iree_hal_task_device_t*)base_device;
+ return device->identifier;
+}
+
+static iree_allocator_t iree_hal_task_device_host_allocator(
+ iree_hal_device_t* base_device) {
+ iree_hal_task_device_t* device = iree_hal_task_device_cast(base_device);
+ return device->host_allocator;
+}
+
+static iree_hal_allocator_t* iree_hal_task_device_allocator(
+ iree_hal_device_t* base_device) {
+ iree_hal_task_device_t* device = iree_hal_task_device_cast(base_device);
+ return device->device_allocator;
+}
+
+static iree_status_t iree_hal_task_device_create_command_buffer(
+ iree_hal_device_t* base_device, iree_hal_command_buffer_mode_t mode,
+ iree_hal_command_category_t command_categories,
+ iree_hal_command_buffer_t** out_command_buffer) {
+ iree_hal_task_device_t* device = (iree_hal_task_device_t*)base_device;
+ // TODO(benvanik): prevent the need for taking a scope here. We need it to
+ // construct the tasks as we record but unfortunately then that means we would
+ // need to know which queue we'd be submitting against ahead of time.
+ return iree_hal_task_command_buffer_create(
+ base_device, &device->queues[0].scope, mode, command_categories,
+ &device->large_block_pool, out_command_buffer);
+}
+
+static iree_status_t iree_hal_task_device_create_descriptor_set(
+ iree_hal_device_t* base_device,
+ iree_hal_descriptor_set_layout_t* set_layout,
+ iree_host_size_t binding_count,
+ const iree_hal_descriptor_set_binding_t* bindings,
+ iree_hal_descriptor_set_t** out_descriptor_set) {
+ return iree_hal_local_descriptor_set_create(set_layout, binding_count,
+ bindings, out_descriptor_set);
+}
+
+static iree_status_t iree_hal_task_device_create_descriptor_set_layout(
+ iree_hal_device_t* base_device,
+ iree_hal_descriptor_set_layout_usage_type_t usage_type,
+ iree_host_size_t binding_count,
+ const iree_hal_descriptor_set_layout_binding_t* bindings,
+ iree_hal_descriptor_set_layout_t** out_descriptor_set_layout) {
+ return iree_hal_local_descriptor_set_layout_create(
+ usage_type, binding_count, bindings,
+ iree_hal_device_host_allocator(base_device), out_descriptor_set_layout);
+}
+
+static iree_status_t iree_hal_task_device_create_event(
+ iree_hal_device_t* base_device, iree_hal_event_t** out_event) {
+ return iree_hal_task_event_create(iree_hal_device_host_allocator(base_device),
+ out_event);
+}
+
+static iree_status_t iree_hal_task_device_create_executable_cache(
+ iree_hal_device_t* base_device, iree_string_view_t identifier,
+ iree_hal_executable_cache_t** out_executable_cache) {
+ iree_hal_task_device_t* device = iree_hal_task_device_cast(base_device);
+ return iree_hal_local_executable_cache_create(
+ identifier, device->loader_count, device->loaders,
+ iree_hal_device_host_allocator(base_device), out_executable_cache);
+}
+
+static iree_status_t iree_hal_task_device_create_executable_layout(
+ iree_hal_device_t* base_device, iree_host_size_t set_layout_count,
+ iree_hal_descriptor_set_layout_t** set_layouts,
+ iree_host_size_t push_constants,
+ iree_hal_executable_layout_t** out_executable_layout) {
+ return iree_hal_local_executable_layout_create(
+ set_layout_count, set_layouts, push_constants,
+ iree_hal_device_host_allocator(base_device), out_executable_layout);
+}
+
+static iree_status_t iree_hal_task_device_create_semaphore(
+ iree_hal_device_t* base_device, uint64_t initial_value,
+ iree_hal_semaphore_t** out_semaphore) {
+ iree_hal_task_device_t* device = iree_hal_task_device_cast(base_device);
+ return iree_hal_task_semaphore_create(device->event_pool, initial_value,
+ device->host_allocator, out_semaphore);
+}
+
+// Returns the queue index to submit work to based on the |queue_affinity|.
+//
+// If we wanted to have dedicated transfer queues we'd fork off based on
+// command_categories. For now all queues are general purpose.
+static iree_host_size_t iree_hal_device_select_queue(
+ iree_hal_task_device_t* device,
+ iree_hal_command_category_t command_categories, uint64_t queue_affinity) {
+ // TODO(benvanik): evaluate if we want to obscure this mapping a bit so that
+ // affinity really means "equivalent affinities map to equivalent queues" and
+ // not a specific queue index.
+ return queue_affinity % device->queue_count;
+}
+
+static iree_status_t iree_hal_task_device_queue_submit(
+ iree_hal_device_t* base_device,
+ iree_hal_command_category_t command_categories, uint64_t queue_affinity,
+ iree_host_size_t batch_count, const iree_hal_submission_batch_t* batches) {
+ iree_hal_task_device_t* device = (iree_hal_task_device_t*)base_device;
+ iree_host_size_t queue_index =
+ iree_hal_device_select_queue(device, command_categories, queue_affinity);
+ return iree_hal_task_queue_submit(&device->queues[queue_index], batch_count,
+ batches);
+}
+
+static iree_status_t iree_hal_task_device_wait_semaphores_with_deadline(
+ iree_hal_device_t* base_device, iree_hal_wait_mode_t wait_mode,
+ const iree_hal_semaphore_list_t* semaphore_list, iree_time_t deadline_ns) {
+ iree_hal_task_device_t* device = iree_hal_task_device_cast(base_device);
+ return iree_hal_task_semaphore_multi_wait(wait_mode, semaphore_list,
+ deadline_ns, device->event_pool,
+ &device->large_block_pool);
+}
+
+static iree_status_t iree_hal_task_device_wait_semaphores_with_timeout(
+ iree_hal_device_t* base_device, iree_hal_wait_mode_t wait_mode,
+ const iree_hal_semaphore_list_t* semaphore_list,
+ iree_duration_t timeout_ns) {
+ return iree_hal_task_device_wait_semaphores_with_deadline(
+ base_device, wait_mode, semaphore_list,
+ iree_relative_timeout_to_deadline_ns(timeout_ns));
+}
+
+static iree_status_t iree_hal_task_device_wait_idle_with_deadline(
+ iree_hal_device_t* base_device, iree_time_t deadline_ns) {
+ iree_hal_task_device_t* device = (iree_hal_task_device_t*)base_device;
+ IREE_TRACE_ZONE_BEGIN(z0);
+ iree_status_t status = iree_ok_status();
+ for (iree_host_size_t i = 0; i < device->queue_count; ++i) {
+ status = iree_hal_task_queue_wait_idle_with_deadline(&device->queues[i],
+ deadline_ns);
+ if (!iree_status_is_ok(status)) break;
+ }
+ IREE_TRACE_ZONE_END(z0);
+ return status;
+}
+
+static iree_status_t iree_hal_task_device_wait_idle_with_timeout(
+ iree_hal_device_t* base_device, iree_duration_t timeout_ns) {
+ return iree_hal_task_device_wait_idle_with_deadline(
+ base_device, iree_relative_timeout_to_deadline_ns(timeout_ns));
+}
+
+static const iree_hal_device_vtable_t iree_hal_task_device_vtable = {
+ .destroy = iree_hal_task_device_destroy,
+ .id = iree_hal_task_device_id,
+ .host_allocator = iree_hal_task_device_host_allocator,
+ .device_allocator = iree_hal_task_device_allocator,
+ .create_command_buffer = iree_hal_task_device_create_command_buffer,
+ .create_descriptor_set = iree_hal_task_device_create_descriptor_set,
+ .create_descriptor_set_layout =
+ iree_hal_task_device_create_descriptor_set_layout,
+ .create_event = iree_hal_task_device_create_event,
+ .create_executable_cache = iree_hal_task_device_create_executable_cache,
+ .create_executable_layout = iree_hal_task_device_create_executable_layout,
+ .create_semaphore = iree_hal_task_device_create_semaphore,
+ .queue_submit = iree_hal_task_device_queue_submit,
+ .wait_semaphores_with_deadline =
+ iree_hal_task_device_wait_semaphores_with_deadline,
+ .wait_semaphores_with_timeout =
+ iree_hal_task_device_wait_semaphores_with_timeout,
+ .wait_idle_with_deadline = iree_hal_task_device_wait_idle_with_deadline,
+ .wait_idle_with_timeout = iree_hal_task_device_wait_idle_with_timeout,
+};
diff --git a/iree/hal/local/task_device.h b/iree/hal/local/task_device.h
new file mode 100644
index 0000000..c1d1edc
--- /dev/null
+++ b/iree/hal/local/task_device.h
@@ -0,0 +1,58 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef IREE_HAL_LOCAL_TASK_DEVICE_H_
+#define IREE_HAL_LOCAL_TASK_DEVICE_H_
+
+#include "iree/base/api.h"
+#include "iree/hal/api.h"
+#include "iree/hal/local/executable_loader.h"
+#include "iree/task/executor.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif // __cplusplus
+
+// Parameters configuring an iree_hal_task_device_t.
+// Must be initialized with iree_hal_task_device_params_initialize prior to use.
+typedef struct {
+ // Number of queues exposed on the device.
+ // Each queue acts as a separate synchronization scope where all work executes
+ // concurrently unless prohibited by semaphores.
+ iree_host_size_t queue_count;
+
+ // Total size of each block in the device shared block pool.
+ // Larger sizes will lower overhead and ensure the heap isn't hit for
+ // transient allocations while also increasing memory consumption.
+ iree_host_size_t arena_block_size;
+} iree_hal_task_device_params_t;
+
+// Initializes |out_params| to default values.
+void iree_hal_task_device_params_initialize(
+ iree_hal_task_device_params_t* out_params);
+
+// Creates a new iree/task/-based local CPU device that uses |executor| for
+// scheduling tasks. |loaders| is the set of executable loaders that are
+// available for loading in the device context.
+iree_status_t iree_hal_task_device_create(
+ iree_string_view_t identifier, const iree_hal_task_device_params_t* params,
+ iree_task_executor_t* executor, iree_host_size_t loader_count,
+ iree_hal_executable_loader_t** loaders, iree_allocator_t host_allocator,
+ iree_hal_device_t** out_device);
+
+#ifdef __cplusplus
+} // extern "C"
+#endif // __cplusplus
+
+#endif // IREE_HAL_LOCAL_TASK_DEVICE_H_
diff --git a/iree/hal/local/task_driver.c b/iree/hal/local/task_driver.c
new file mode 100644
index 0000000..ffbfe1c
--- /dev/null
+++ b/iree/hal/local/task_driver.c
@@ -0,0 +1,127 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "iree/hal/local/task_driver.h"
+
+#include "iree/base/tracing.h"
+
+#define IREE_HAL_TASK_DEVICE_ID_DEFAULT 0
+
+typedef struct {
+ iree_hal_resource_t resource;
+ iree_allocator_t host_allocator;
+
+ iree_string_view_t identifier;
+ iree_hal_task_device_params_t default_params;
+
+ iree_task_executor_t* executor;
+
+ iree_host_size_t loader_count;
+ iree_hal_executable_loader_t* loaders[];
+} iree_hal_task_driver_t;
+
+static const iree_hal_driver_vtable_t iree_hal_task_driver_vtable;
+
+iree_status_t iree_hal_task_driver_create(
+ iree_string_view_t identifier,
+ const iree_hal_task_device_params_t* default_params,
+ iree_task_executor_t* executor, iree_host_size_t loader_count,
+ iree_hal_executable_loader_t** loaders, iree_allocator_t host_allocator,
+ iree_hal_driver_t** out_driver) {
+ IREE_ASSERT_ARGUMENT(default_params);
+ IREE_ASSERT_ARGUMENT(!loader_count || loaders);
+ IREE_ASSERT_ARGUMENT(out_driver);
+ *out_driver = NULL;
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ iree_hal_task_driver_t* driver = NULL;
+ iree_host_size_t total_size = sizeof(*driver) +
+ loader_count * sizeof(*driver->loaders) +
+ identifier.size;
+ iree_status_t status =
+ iree_allocator_malloc(host_allocator, total_size, (void**)&driver);
+ if (iree_status_is_ok(status)) {
+ iree_hal_resource_initialize(&iree_hal_task_driver_vtable,
+ &driver->resource);
+ driver->host_allocator = host_allocator;
+
+ iree_string_view_append_to_buffer(
+ identifier, &driver->identifier,
+ (char*)driver + total_size - identifier.size);
+ memcpy(&driver->default_params, default_params,
+ sizeof(driver->default_params));
+
+ driver->executor = executor;
+ iree_task_executor_retain(driver->executor);
+
+ driver->loader_count = loader_count;
+ for (iree_host_size_t i = 0; i < driver->loader_count; ++i) {
+ driver->loaders[i] = loaders[i];
+ iree_hal_executable_loader_retain(driver->loaders[i]);
+ }
+ }
+
+ if (iree_status_is_ok(status)) {
+ *out_driver = (iree_hal_driver_t*)driver;
+ } else {
+ iree_hal_driver_release((iree_hal_driver_t*)driver);
+ }
+ IREE_TRACE_ZONE_END(z0);
+ return status;
+}
+
+static void iree_hal_task_driver_destroy(iree_hal_driver_t* base_driver) {
+ iree_hal_task_driver_t* driver = (iree_hal_task_driver_t*)base_driver;
+ iree_allocator_t host_allocator = driver->host_allocator;
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ for (iree_host_size_t i = 0; i < driver->loader_count; ++i) {
+ iree_hal_executable_loader_release(driver->loaders[i]);
+ }
+ iree_task_executor_release(driver->executor);
+ iree_allocator_free(host_allocator, driver);
+
+ IREE_TRACE_ZONE_END(z0);
+}
+
+static iree_status_t iree_hal_task_driver_query_available_devices(
+ iree_hal_driver_t* base_driver, iree_allocator_t allocator,
+ iree_hal_device_info_t** out_device_infos,
+ iree_host_size_t* out_device_info_count) {
+ static const iree_hal_device_info_t device_infos[1] = {
+ {
+ .device_id = IREE_HAL_TASK_DEVICE_ID_DEFAULT,
+ .name = iree_string_view_literal("default"),
+ },
+ };
+ *out_device_info_count = IREE_ARRAYSIZE(device_infos);
+ return iree_allocator_clone(
+ allocator, iree_make_const_byte_span(device_infos, sizeof(device_infos)),
+ (void**)out_device_infos);
+}
+
+static iree_status_t iree_hal_task_driver_create_device(
+ iree_hal_driver_t* base_driver, iree_hal_device_id_t device_id,
+ iree_allocator_t allocator, iree_hal_device_t** out_device) {
+ iree_hal_task_driver_t* driver = (iree_hal_task_driver_t*)base_driver;
+ return iree_hal_task_device_create(
+ driver->identifier, &driver->default_params, driver->executor,
+ driver->loader_count, driver->loaders, allocator, out_device);
+}
+
+static const iree_hal_driver_vtable_t iree_hal_task_driver_vtable = {
+ .destroy = iree_hal_task_driver_destroy,
+ .query_available_devices = iree_hal_task_driver_query_available_devices,
+ .create_device = iree_hal_task_driver_create_device,
+};
diff --git a/iree/hal/local/task_driver.h b/iree/hal/local/task_driver.h
new file mode 100644
index 0000000..92117dd
--- /dev/null
+++ b/iree/hal/local/task_driver.h
@@ -0,0 +1,42 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef IREE_HAL_LOCAL_TASK_DRIVER_H_
+#define IREE_HAL_LOCAL_TASK_DRIVER_H_
+
+#include "iree/base/api.h"
+#include "iree/hal/api.h"
+#include "iree/hal/local/executable_loader.h"
+#include "iree/hal/local/task_device.h"
+#include "iree/task/executor.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif // __cplusplus
+
+// Creates a new iree/task/-based local CPU driver that creates devices sharing
+// the same |executor| for scheduling tasks. |loaders| is the set of executable
+// loaders that are available for loading in each device context.
+iree_status_t iree_hal_task_driver_create(
+ iree_string_view_t identifier,
+ const iree_hal_task_device_params_t* default_params,
+ iree_task_executor_t* executor, iree_host_size_t loader_count,
+ iree_hal_executable_loader_t** loaders, iree_allocator_t host_allocator,
+ iree_hal_driver_t** out_driver);
+
+#ifdef __cplusplus
+} // extern "C"
+#endif // __cplusplus
+
+#endif // IREE_HAL_LOCAL_TASK_DRIVER_H_
diff --git a/iree/hal/local/task_event.c b/iree/hal/local/task_event.c
new file mode 100644
index 0000000..89a763d
--- /dev/null
+++ b/iree/hal/local/task_event.c
@@ -0,0 +1,52 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "iree/hal/local/task_event.h"
+
+#include "iree/base/tracing.h"
+
+static const iree_hal_event_vtable_t iree_hal_task_event_vtable;
+
+iree_status_t iree_hal_task_event_create(iree_allocator_t host_allocator,
+ iree_hal_event_t** out_event) {
+ IREE_ASSERT_ARGUMENT(out_event);
+ *out_event = NULL;
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ iree_hal_task_event_t* event = NULL;
+ iree_status_t status =
+ iree_allocator_malloc(host_allocator, sizeof(*event), (void**)&event);
+ if (iree_status_is_ok(status)) {
+ iree_hal_resource_initialize(&iree_hal_task_event_vtable, &event->resource);
+ event->host_allocator = host_allocator;
+ *out_event = (iree_hal_event_t*)event;
+ }
+
+ IREE_TRACE_ZONE_END(z0);
+ return status;
+}
+
+static void iree_hal_task_event_destroy(iree_hal_event_t* base_event) {
+ iree_hal_task_event_t* event = (iree_hal_task_event_t*)base_event;
+ iree_allocator_t host_allocator = event->host_allocator;
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ iree_allocator_free(host_allocator, event);
+
+ IREE_TRACE_ZONE_END(z0);
+}
+
+static const iree_hal_event_vtable_t iree_hal_task_event_vtable = {
+ .destroy = iree_hal_task_event_destroy,
+};
diff --git a/iree/hal/local/task_event.h b/iree/hal/local/task_event.h
new file mode 100644
index 0000000..5489476
--- /dev/null
+++ b/iree/hal/local/task_event.h
@@ -0,0 +1,39 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef IREE_HAL_LOCAL_TASK_EVENT_H_
+#define IREE_HAL_LOCAL_TASK_EVENT_H_
+
+#include "iree/base/api.h"
+#include "iree/hal/api.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif // __cplusplus
+
+typedef struct {
+ iree_hal_resource_t resource;
+ iree_allocator_t host_allocator;
+} iree_hal_task_event_t;
+
+iree_status_t iree_hal_task_event_create(iree_allocator_t host_allocator,
+ iree_hal_event_t** out_event);
+
+iree_hal_task_event_t* iree_hal_task_event_cast(iree_hal_event_t* base_value);
+
+#ifdef __cplusplus
+} // extern "C"
+#endif // __cplusplus
+
+#endif // IREE_HAL_LOCAL_TASK_EVENT_H_
diff --git a/iree/hal/local/task_queue.c b/iree/hal/local/task_queue.c
new file mode 100644
index 0000000..baf6d4b
--- /dev/null
+++ b/iree/hal/local/task_queue.c
@@ -0,0 +1,516 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "iree/hal/local/task_queue.h"
+
+#include "iree/base/tracing.h"
+#include "iree/hal/local/task_command_buffer.h"
+#include "iree/hal/local/task_semaphore.h"
+#include "iree/task/submission.h"
+
+// Each submission is turned into a DAG for execution:
+//
+// +--------------------+ To preserve the sequential issue order an edge is
+// | (previous issue) | added between the previous outstanding issue (if
+// +--------------------+ it exists) such that all issues run in the order
+// | they were submitted to the queue. Note that this
+// v is *only* the issue; the commands issued by two
+// +--------------------+ submissions may still overlap and are only
+// | sequence barrier | guaranteed to begin execution in order.
+// +--------------------+
+// |
+// | +--------------+
+// +-> | +--------------+ Unsatisfied waits are scheduled as wait tasks and
+// . +-| sema waits | block the issuing of commands until all have
+// . +--------------+ been satisfied. If the wait is immediately
+// . | | | | | following a signal from the same queue then it
+// +--------+-+-+-+-+ elided - only cross-queue or external waits
+// | actually go down to system wait handles.
+// v
+// +--------------------+ Command buffers in the batch are issued in-order
+// | command issue | as if all commands had been recorded into the same
+// +--------------------+ command buffer (excluding recording state like
+// | push constants). The dependencies between commands
+// | +--------------+ are determined by the events and barriers recorded
+// +-> | +--------------+ in each command buffer.
+// . +-| commands |
+// . +--------------+
+// . | | | | |
+// +--------+-+-+-+-+
+// |
+// v
+// +--------------------+ After all commands within the batch complete the
+// | semaphore signals | submission is retired and all semaphores are
+// +--------------------+ signaled. Note that this may happen *before* other
+// | earlier submissions complete if there were no
+// ... dependencies between the commands in each batch.
+//
+// Could this be simplified? Probably. Improvements to the task system to allow
+// for efficient multiwaits and better stitching of independent DAGs would help.
+
+//===----------------------------------------------------------------------===//
+// Utilities
+//===----------------------------------------------------------------------===//
+
+// Clones a list of semaphores into an |arena| and initializes |out_target_list|
+// to reference the newly-cloned data.
+static iree_status_t iree_hal_semaphore_list_clone(
+ const iree_hal_semaphore_list_t* source_list, iree_arena_allocator_t* arena,
+ iree_hal_semaphore_list_t* out_target_list) {
+ iree_host_size_t semaphores_size =
+ source_list->count * sizeof(out_target_list->semaphores[0]);
+ iree_host_size_t payload_values_size =
+ source_list->count * sizeof(out_target_list->payload_values[0]);
+ iree_host_size_t total_size = semaphores_size + payload_values_size;
+ uint8_t* buffer = NULL;
+ IREE_RETURN_IF_ERROR(iree_arena_allocate(arena, total_size, (void**)&buffer));
+
+ out_target_list->count = source_list->count;
+ out_target_list->semaphores = (iree_hal_semaphore_t**)buffer;
+ out_target_list->payload_values = (uint64_t*)(buffer + semaphores_size);
+
+ for (iree_host_size_t i = 0; i < source_list->count; ++i) {
+ out_target_list->semaphores[i] = source_list->semaphores[i];
+ iree_hal_semaphore_retain(out_target_list->semaphores[i]);
+ out_target_list->payload_values[i] = source_list->payload_values[i];
+ }
+
+ return iree_ok_status();
+}
+
+static void iree_hal_semaphore_list_release(iree_hal_semaphore_list_t* list) {
+ for (iree_host_size_t i = 0; i < list->count; ++i) {
+ iree_hal_semaphore_release(list->semaphores[i]);
+ }
+}
+
+//===----------------------------------------------------------------------===//
+// iree_hal_task_queue_wait_cmd_t
+//===----------------------------------------------------------------------===//
+
+// Task to fork out and wait on one or more semaphores.
+// This optimizes for same-queue semaphore chaining by ensuring that semaphores
+// used to stitch together subsequent submissions never have to go to the system
+// to wait as the implicit queue ordering ensures that the signals would have
+// happened prior to the sequence command being executed. Cross-queue semaphores
+// will still cause waits if they have not yet been signaled.
+typedef struct {
+ // Call to iree_hal_task_queue_wait_cmd.
+ iree_task_call_t task;
+
+ // Arena used for the submission - additional tasks can be allocated from
+ // this.
+ iree_arena_allocator_t* arena;
+
+ // A list of semaphores to wait on prior to issuing the rest of the
+ // submission.
+ iree_hal_semaphore_list_t wait_semaphores;
+} iree_hal_task_queue_wait_cmd_t;
+
+// Forks out multiple wait tasks prior to issuing the commands.
+static iree_status_t iree_hal_task_queue_wait_cmd(
+ uintptr_t user_context, iree_task_t* task,
+ iree_task_submission_t* pending_submission) {
+ iree_hal_task_queue_wait_cmd_t* cmd = (iree_hal_task_queue_wait_cmd_t*)task;
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ iree_status_t status = iree_ok_status();
+ for (iree_host_size_t i = 0; i < cmd->wait_semaphores.count; ++i) {
+ status = iree_hal_task_semaphore_enqueue_timepoint(
+ cmd->wait_semaphores.semaphores[i],
+ cmd->wait_semaphores.payload_values[i],
+ cmd->task.header.completion_task, cmd->arena, pending_submission);
+ if (IREE_UNLIKELY(!iree_status_is_ok(status))) break;
+ }
+
+ IREE_TRACE_ZONE_END(z0);
+ return status;
+}
+
+// Cleanup for iree_hal_task_queue_wait_cmd_t that releases the retained
+// semaphores.
+static void iree_hal_task_queue_wait_cmd_cleanup(iree_task_t* task,
+ iree_status_t status) {
+ iree_hal_task_queue_wait_cmd_t* cmd = (iree_hal_task_queue_wait_cmd_t*)task;
+ iree_hal_semaphore_list_release(&cmd->wait_semaphores);
+}
+
+// Allocates and initializes a iree_hal_task_queue_wait_cmd_t task.
+static iree_status_t iree_hal_task_queue_wait_cmd_allocate(
+ iree_task_scope_t* scope, const iree_hal_semaphore_list_t* wait_semaphores,
+ iree_arena_allocator_t* arena, iree_hal_task_queue_wait_cmd_t** out_cmd) {
+ iree_hal_task_queue_wait_cmd_t* cmd = NULL;
+ IREE_RETURN_IF_ERROR(iree_arena_allocate(arena, sizeof(*cmd), (void**)&cmd));
+ iree_task_call_initialize(
+ scope, iree_task_make_call_closure(iree_hal_task_queue_wait_cmd, 0),
+ &cmd->task);
+ iree_task_set_cleanup_fn(&cmd->task.header,
+ iree_hal_task_queue_wait_cmd_cleanup);
+ cmd->arena = arena;
+
+ // Clone the wait semaphores from the batch - we retain them and their
+ // payloads.
+ IREE_RETURN_IF_ERROR(iree_hal_semaphore_list_clone(wait_semaphores, arena,
+ &cmd->wait_semaphores));
+
+ *out_cmd = cmd;
+ return iree_ok_status();
+}
+
+//===----------------------------------------------------------------------===//
+// iree_hal_task_queue_issue_cmd_t
+//===----------------------------------------------------------------------===//
+
+// Task to issue all the command buffers in the batch.
+// After this task completes the commands have been issued but have not yet
+// completed and the issued commands may complete in any order.
+typedef struct {
+ // Call to iree_hal_task_queue_issue_cmd.
+ iree_task_call_t task;
+
+ // Arena used for the submission - additional tasks can be allocated from
+ // this.
+ iree_arena_allocator_t* arena;
+
+ // Nasty back reference to the queue so that we can clear the tail_issue_task
+ // if we are the last issue pending.
+ iree_hal_task_queue_t* queue;
+
+ // Command buffers to be issued in the order the appeared in the submission.
+ iree_host_size_t command_buffer_count;
+ iree_hal_command_buffer_t* command_buffers[];
+} iree_hal_task_queue_issue_cmd_t;
+
+// Issues a set of command buffers without waiting for them to complete.
+static iree_status_t iree_hal_task_queue_issue_cmd(
+ uintptr_t user_context, iree_task_t* task,
+ iree_task_submission_t* pending_submission) {
+ iree_hal_task_queue_issue_cmd_t* cmd = (iree_hal_task_queue_issue_cmd_t*)task;
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ iree_status_t status = iree_ok_status();
+
+ // NOTE: it's ok for there to be no command buffers - in that case the
+ // submission was purely for synchronization.
+ if (cmd->command_buffer_count > 0) {
+ for (iree_host_size_t i = 0; i < cmd->command_buffer_count; ++i) {
+ status = iree_hal_task_command_buffer_issue(
+ cmd->command_buffers[i], &cmd->queue->state,
+ cmd->task.header.completion_task, cmd->arena, pending_submission);
+ if (IREE_UNLIKELY(!iree_status_is_ok(status))) break;
+ }
+ }
+
+ IREE_TRACE_ZONE_END(z0);
+ return status;
+}
+
+// Cleanup for iree_hal_task_queue_issue_cmd_t that resets the queue state
+// tracking the last in-flight issue.
+static void iree_hal_task_queue_issue_cmd_cleanup(iree_task_t* task,
+ iree_status_t status) {
+ iree_hal_task_queue_issue_cmd_t* cmd = (iree_hal_task_queue_issue_cmd_t*)task;
+
+ // Reset queue tail issue task if it was us.
+ iree_slim_mutex_lock(&cmd->queue->mutex);
+ if (cmd->queue->tail_issue_task == task) {
+ cmd->queue->tail_issue_task = NULL;
+ }
+ iree_slim_mutex_unlock(&cmd->queue->mutex);
+}
+
+// Allocates and initializes a iree_hal_task_queue_issue_cmd_t task.
+static iree_status_t iree_hal_task_queue_issue_cmd_allocate(
+ iree_task_scope_t* scope, iree_hal_task_queue_t* queue,
+ iree_task_t* retire_task, iree_host_size_t command_buffer_count,
+ iree_hal_command_buffer_t** const command_buffers,
+ iree_arena_allocator_t* arena, iree_hal_task_queue_issue_cmd_t** out_cmd) {
+ iree_hal_task_queue_issue_cmd_t* cmd = NULL;
+ iree_host_size_t total_cmd_size =
+ sizeof(*cmd) + command_buffer_count * sizeof(*cmd->command_buffers);
+ IREE_RETURN_IF_ERROR(
+ iree_arena_allocate(arena, total_cmd_size, (void**)&cmd));
+ iree_task_call_initialize(
+ scope, iree_task_make_call_closure(iree_hal_task_queue_issue_cmd, 0),
+ &cmd->task);
+ iree_task_set_completion_task(&cmd->task.header, retire_task);
+ iree_task_set_cleanup_fn(&cmd->task.header,
+ iree_hal_task_queue_issue_cmd_cleanup);
+ cmd->arena = arena;
+ cmd->queue = queue;
+
+ cmd->command_buffer_count = command_buffer_count;
+ memcpy(cmd->command_buffers, command_buffers,
+ cmd->command_buffer_count * sizeof(*cmd->command_buffers));
+
+ *out_cmd = cmd;
+ return iree_ok_status();
+}
+
+//===----------------------------------------------------------------------===//
+// iree_hal_task_queue_retire_cmd_t
+//===----------------------------------------------------------------------===//
+
+// Task to retire the submission and free the transient memory allocated for
+// it. The task is issued only once all commands from all command buffers in
+// the submission complete. Semaphores will be signaled and dependent
+// submissions may be issued.
+typedef struct {
+ // Call to iree_hal_task_queue_retire_cmd.
+ iree_task_call_t task;
+
+ // Original arena used for all transient allocations required for the
+ // submission. All queue-related commands are allocated from this, **including
+ // this retire command**.
+ iree_arena_allocator_t arena;
+
+ // A list of semaphores to signal upon retiring.
+ iree_hal_semaphore_list_t signal_semaphores;
+} iree_hal_task_queue_retire_cmd_t;
+
+// Retires a submission by signaling semaphores to their desired value and
+// disposing of the temporary arena memory used for the submission.
+static iree_status_t iree_hal_task_queue_retire_cmd(
+ uintptr_t user_context, iree_task_t* task,
+ iree_task_submission_t* pending_submission) {
+ iree_hal_task_queue_retire_cmd_t* cmd =
+ (iree_hal_task_queue_retire_cmd_t*)task;
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ // Signal all semaphores to their new values.
+ // Note that if any signal fails then the whole command will fail and all
+ // semaphores will be signaled to the failure state.
+ iree_status_t status = iree_ok_status();
+ for (iree_host_size_t i = 0; i < cmd->signal_semaphores.count; ++i) {
+ status =
+ iree_hal_semaphore_signal(cmd->signal_semaphores.semaphores[i],
+ cmd->signal_semaphores.payload_values[i]);
+ if (IREE_UNLIKELY(!iree_status_is_ok(status))) break;
+ }
+
+ IREE_TRACE_ZONE_END(z0);
+ return status;
+}
+
+// Cleanup for iree_hal_task_queue_retire_cmd_t that ensures that the arena
+// holding the submission is properly disposed and that semaphores are signaled
+// (or signaled to failure if the command failed).
+static void iree_hal_task_queue_retire_cmd_cleanup(iree_task_t* task,
+ iree_status_t status) {
+ iree_hal_task_queue_retire_cmd_t* cmd =
+ (iree_hal_task_queue_retire_cmd_t*)task;
+
+ // If the command failed then fail all semaphores to ensure future
+ // submissions fail as well (including those on other queues).
+ if (!iree_status_is_ok(status)) {
+ for (iree_host_size_t i = 0; i < cmd->signal_semaphores.count; ++i) {
+ iree_hal_semaphore_fail(cmd->signal_semaphores.semaphores[i],
+ iree_status_clone(status));
+ }
+ }
+
+ // Release all semaphores.
+ iree_hal_semaphore_list_release(&cmd->signal_semaphores);
+
+ // Drop all memory used by the submission (**including cmd**).
+ iree_arena_allocator_t arena = cmd->arena;
+ cmd = NULL;
+ iree_arena_deinitialize(&arena);
+}
+
+// Allocates and initializes a iree_hal_task_queue_retire_cmd_t task.
+// The command will own an arena that can be used for other submission-related
+// allocations.
+static iree_status_t iree_hal_task_queue_retire_cmd_allocate(
+ iree_task_scope_t* scope,
+ const iree_hal_semaphore_list_t* signal_semaphores,
+ iree_arena_block_pool_t* block_pool,
+ iree_hal_task_queue_retire_cmd_t** out_cmd) {
+ // Make an arena we'll use for allocating the command itself.
+ iree_arena_allocator_t arena;
+ iree_arena_initialize(block_pool, &arena);
+
+ // Allocate the command from the arena.
+ iree_hal_task_queue_retire_cmd_t* cmd = NULL;
+ iree_status_t status =
+ iree_arena_allocate(&arena, sizeof(*cmd), (void**)&cmd);
+ if (iree_status_is_ok(status)) {
+ iree_task_call_initialize(
+ scope, iree_task_make_call_closure(iree_hal_task_queue_retire_cmd, 0),
+ &cmd->task);
+ iree_task_set_cleanup_fn(&cmd->task.header,
+ iree_hal_task_queue_retire_cmd_cleanup);
+ }
+
+ // Clone the signal semaphores from the batch - we retain them and their
+ // payloads.
+ if (iree_status_is_ok(status)) {
+ status = iree_hal_semaphore_list_clone(signal_semaphores, &arena,
+ &cmd->signal_semaphores);
+ }
+
+ if (iree_status_is_ok(status)) {
+ // Transfer ownership of the arena to command.
+ memcpy(&cmd->arena, &arena, sizeof(cmd->arena));
+ *out_cmd = cmd;
+ } else {
+ iree_arena_deinitialize(&arena);
+ }
+ return status;
+}
+
+//===----------------------------------------------------------------------===//
+// iree_hal_task_queue_t
+//===----------------------------------------------------------------------===//
+
+void iree_hal_task_queue_initialize(iree_string_view_t identifier,
+ iree_task_executor_t* executor,
+ iree_arena_block_pool_t* block_pool,
+ iree_hal_task_queue_t* out_queue) {
+ IREE_TRACE_ZONE_BEGIN(z0);
+ IREE_TRACE_ZONE_APPEND_TEXT(z0, identifier.data, identifier.size);
+
+ memset(out_queue, 0, sizeof(*out_queue));
+
+ out_queue->executor = executor;
+ iree_task_executor_retain(out_queue->executor);
+ out_queue->block_pool = block_pool;
+
+ iree_task_scope_initialize(identifier, &out_queue->scope);
+
+ iree_slim_mutex_initialize(&out_queue->mutex);
+ iree_hal_task_queue_state_initialize(&out_queue->state);
+ out_queue->tail_issue_task = NULL;
+
+ IREE_TRACE_ZONE_END(z0);
+}
+
+void iree_hal_task_queue_deinitialize(iree_hal_task_queue_t* queue) {
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ iree_status_ignore(
+ iree_task_scope_wait_idle(&queue->scope, IREE_TIME_INFINITE_FUTURE));
+
+ iree_slim_mutex_lock(&queue->mutex);
+ IREE_ASSERT(!queue->tail_issue_task);
+ iree_slim_mutex_unlock(&queue->mutex);
+
+ iree_hal_task_queue_state_deinitialize(&queue->state);
+ iree_slim_mutex_deinitialize(&queue->mutex);
+ iree_task_scope_deinitialize(&queue->scope);
+ iree_task_executor_release(queue->executor);
+
+ IREE_TRACE_ZONE_END(z0);
+}
+
+static iree_status_t iree_hal_task_queue_submit_batch(
+ iree_hal_task_queue_t* queue, const iree_hal_submission_batch_t* batch) {
+ // Task to retire the submission and free the transient memory allocated for
+ // it (including the command itself). We allocate this first so it can get an
+ // arena which we will use to allocate all other commands.
+ iree_hal_task_queue_retire_cmd_t* retire_cmd = NULL;
+ IREE_RETURN_IF_ERROR(iree_hal_task_queue_retire_cmd_allocate(
+ &queue->scope, &batch->signal_semaphores, queue->block_pool,
+ &retire_cmd));
+
+ // NOTE: if we fail from here on we must drop the retire_cmd arena.
+ iree_status_t status = iree_ok_status();
+
+ // Task to fork and wait for unsatisfied semaphore dependencies.
+ // This is optional and only required if we have previous submissions still
+ // in-flight - if the queue is empty then we can directly schedule the waits.
+ iree_hal_task_queue_wait_cmd_t* wait_cmd = NULL;
+ if (batch->wait_semaphores.count > 0) {
+ status = iree_hal_task_queue_wait_cmd_allocate(
+ &queue->scope, &batch->wait_semaphores, &retire_cmd->arena, &wait_cmd);
+ }
+
+ // Task to issue all the command buffers in the batch.
+ // After this task completes the commands have been issued but have not yet
+ // completed and the issued commands may complete in any order.
+ iree_hal_task_queue_issue_cmd_t* issue_cmd = NULL;
+ if (iree_status_is_ok(status)) {
+ status = iree_hal_task_queue_issue_cmd_allocate(
+ &queue->scope, queue, &retire_cmd->task.header,
+ batch->command_buffer_count, batch->command_buffers, &retire_cmd->arena,
+ &issue_cmd);
+ }
+
+ // Last chance for failure - from here on we are submitting.
+ if (IREE_UNLIKELY(!iree_status_is_ok(status))) {
+ iree_arena_deinitialize(&retire_cmd->arena);
+ return status;
+ }
+
+ iree_task_submission_t submission;
+ iree_task_submission_initialize(&submission);
+
+ // Sequencing: wait on semaphores or go directly into the executor queue.
+ if (wait_cmd != NULL) {
+ // Ensure that we only issue command buffers after all waits have completed.
+ iree_task_set_completion_task(&wait_cmd->task.header,
+ &issue_cmd->task.header);
+ iree_task_submission_enqueue(&submission, &wait_cmd->task.header);
+ } else {
+ // No waits needed; directly enqueue.
+ iree_task_submission_enqueue(&submission, &issue_cmd->task.header);
+ }
+
+ iree_slim_mutex_lock(&queue->mutex);
+
+ // If there is an in-flight issue pending then we need to chain onto that
+ // so that we ensure FIFO submission order is preserved. Note that we are only
+ // waiting for the issue to complete and *not* all of the commands that are
+ // issued.
+ if (queue->tail_issue_task != NULL) {
+ iree_task_set_completion_task(queue->tail_issue_task,
+ &issue_cmd->task.header);
+ }
+ queue->tail_issue_task = &issue_cmd->task.header;
+
+ iree_slim_mutex_unlock(&queue->mutex);
+
+ // Submit the tasks immediately. The executor may queue them up until we
+ // force the flush after all batches have been processed.
+ iree_task_executor_submit(queue->executor, &submission);
+ return iree_ok_status();
+}
+
+iree_status_t iree_hal_task_queue_submit(
+ iree_hal_task_queue_t* queue, iree_host_size_t batch_count,
+ const iree_hal_submission_batch_t* batches) {
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ // For now we process each batch independently. To elide additional semaphore
+ // work and prevent unneeded coordinator scheduling logic we could instead
+ // build the whole DAG prior to submitting.
+ for (iree_host_size_t i = 0; i < batch_count; ++i) {
+ const iree_hal_submission_batch_t* batch = &batches[i];
+ IREE_RETURN_AND_END_ZONE_IF_ERROR(
+ z0, iree_hal_task_queue_submit_batch(queue, batch));
+ }
+
+ iree_task_executor_flush(queue->executor);
+
+ IREE_TRACE_ZONE_END(z0);
+ return iree_ok_status();
+}
+
+iree_status_t iree_hal_task_queue_wait_idle_with_deadline(
+ iree_hal_task_queue_t* queue, iree_time_t deadline_ns) {
+ IREE_TRACE_ZONE_BEGIN(z0);
+ iree_status_t status = iree_task_scope_wait_idle(&queue->scope, deadline_ns);
+ IREE_TRACE_ZONE_END(z0);
+ return status;
+}
diff --git a/iree/hal/local/task_queue.h b/iree/hal/local/task_queue.h
new file mode 100644
index 0000000..5dfc449
--- /dev/null
+++ b/iree/hal/local/task_queue.h
@@ -0,0 +1,78 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef IREE_HAL_LOCAL_TASK_QUEUE_H_
+#define IREE_HAL_LOCAL_TASK_QUEUE_H_
+
+#include "iree/base/api.h"
+#include "iree/base/synchronization.h"
+#include "iree/hal/api.h"
+#include "iree/hal/local/arena.h"
+#include "iree/hal/local/task_queue_state.h"
+#include "iree/task/executor.h"
+#include "iree/task/scope.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif // __cplusplus
+
+typedef struct {
+ // Shared executor that the queue submits tasks to.
+ iree_task_executor_t* executor;
+
+ // Shared block pool for allocating submission transients (tasks/events/etc).
+ iree_arena_block_pool_t* block_pool;
+
+ // Scope used for all tasks in the queue.
+ // This allows for easy waits on all outstanding queue tasks as well as
+ // differentiation of tasks within the executor.
+ iree_task_scope_t scope;
+
+ // Guards queue state. Submissions and waits may come from any user thread and
+ // we do a bit of bookkeeping during command buffer issue that will come from
+ // an executor thread.
+ iree_slim_mutex_t mutex;
+
+ // State tracking used during command buffer issue.
+ // The intra-queue synchronization (barriers/events) carries across command
+ // buffers and this is used to rendezvous the tasks in each set.
+ iree_hal_task_queue_state_t state;
+
+ // The last active iree_hal_task_queue_issue_cmd_t submitted to the queue.
+ // If this is NULL then there are no issues pending - though there may still
+ // be active work that was previously issued. This is used to chain together
+ // issues in FIFO order such that all submissions *issue* in order but not
+ // *execute* in order.
+ iree_task_t* tail_issue_task;
+} iree_hal_task_queue_t;
+
+void iree_hal_task_queue_initialize(iree_string_view_t identifier,
+ iree_task_executor_t* executor,
+ iree_arena_block_pool_t* block_pool,
+ iree_hal_task_queue_t* out_queue);
+
+void iree_hal_task_queue_deinitialize(iree_hal_task_queue_t* queue);
+
+iree_status_t iree_hal_task_queue_submit(
+ iree_hal_task_queue_t* queue, iree_host_size_t batch_count,
+ const iree_hal_submission_batch_t* batches);
+
+iree_status_t iree_hal_task_queue_wait_idle_with_deadline(
+ iree_hal_task_queue_t* queue, iree_time_t deadline_ns);
+
+#ifdef __cplusplus
+} // extern "C"
+#endif // __cplusplus
+
+#endif // IREE_HAL_LOCAL_TASK_QUEUE_H_
diff --git a/iree/hal/local/task_queue_state.c b/iree/hal/local/task_queue_state.c
new file mode 100644
index 0000000..7e91241
--- /dev/null
+++ b/iree/hal/local/task_queue_state.c
@@ -0,0 +1,25 @@
+// Copyright 2021 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "iree/hal/local/task_queue_state.h"
+
+#include "iree/base/tracing.h"
+
+void iree_hal_task_queue_state_initialize(
+ iree_hal_task_queue_state_t* out_queue_state) {
+ memset(out_queue_state, 0, sizeof(*out_queue_state));
+}
+
+void iree_hal_task_queue_state_deinitialize(
+ iree_hal_task_queue_state_t* queue_state) {}
diff --git a/iree/hal/local/task_queue_state.h b/iree/hal/local/task_queue_state.h
new file mode 100644
index 0000000..9b3c04b
--- /dev/null
+++ b/iree/hal/local/task_queue_state.h
@@ -0,0 +1,49 @@
+// Copyright 2021 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef IREE_HAL_LOCAL_TASK_QUEUE_STATE_H_
+#define IREE_HAL_LOCAL_TASK_QUEUE_STATE_H_
+
+#include "iree/base/api.h"
+#include "iree/base/atomics.h"
+#include "iree/hal/api.h"
+#include "iree/task/scope.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif // __cplusplus
+
+// State tracking for an individual queue.
+//
+// Thread-compatible: only intended to be used by a queue with the submission
+// lock held.
+typedef struct {
+ // TODO(#4518): track event state.
+ int reserved;
+} iree_hal_task_queue_state_t;
+
+// Initializes queue state with the given |identifier| used to annotate tasks
+// submitted to the queue.
+void iree_hal_task_queue_state_initialize(
+ iree_hal_task_queue_state_t* out_queue_state);
+
+// Deinitializes queue state and cleans up any tracking intermediates.
+void iree_hal_task_queue_state_deinitialize(
+ iree_hal_task_queue_state_t* queue_state);
+
+#ifdef __cplusplus
+} // extern "C"
+#endif // __cplusplus
+
+#endif // IREE_HAL_LOCAL_TASK_QUEUE_STATE_H_
diff --git a/iree/hal/local/task_semaphore.c b/iree/hal/local/task_semaphore.c
new file mode 100644
index 0000000..9e30b9c
--- /dev/null
+++ b/iree/hal/local/task_semaphore.c
@@ -0,0 +1,510 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "iree/hal/local/task_semaphore.h"
+
+#include <inttypes.h>
+
+#include "iree/base/synchronization.h"
+#include "iree/base/tracing.h"
+#include "iree/base/wait_handle.h"
+
+// Sentinel used the semaphore has failed and an error status is set.
+#define IREE_HAL_TASK_SEMAPHORE_FAILURE_VALUE UINT64_MAX
+
+//===----------------------------------------------------------------------===//
+// iree_hal_task_timepoint_t
+//===----------------------------------------------------------------------===//
+
+// Represents a point in the timeline that someone is waiting to be reached.
+// When the semaphore is signaled to at least the specified value then the
+// given event will be signaled and the timepoint discarded.
+//
+// Instances are owned and retained by the caller that requested them - usually
+// in the arena associated with the submission, but could be on the stack of a
+// synchronously waiting thread.
+typedef struct iree_hal_task_timepoint_s {
+ struct iree_hal_task_timepoint_s* next;
+ struct iree_hal_task_timepoint_s* prev;
+ uint64_t payload_value;
+ iree_event_t event;
+} iree_hal_task_timepoint_t;
+
+// A doubly-linked FIFO list of timepoints.
+// The order of the timepoints does *not* match increasing payload values but
+// instead the order they were added to the list.
+//
+// Note that the timepoints are not owned by the list - this just nicely
+// stitches together timepoints for the semaphore.
+typedef struct {
+ iree_hal_task_timepoint_t* head;
+ iree_hal_task_timepoint_t* tail;
+} iree_hal_task_timepoint_list_t;
+
+static void iree_hal_task_timepoint_list_initialize(
+ iree_hal_task_timepoint_list_t* out_list) {
+ memset(out_list, 0, sizeof(*out_list));
+}
+
+// Moves |source_list| into |out_target_list|.
+// |source_list| will be reset and the prior contents of |out_target_list| will
+// be discarded.
+static void iree_hal_task_timepoint_list_move(
+ iree_hal_task_timepoint_list_t* source_list,
+ iree_hal_task_timepoint_list_t* out_target_list) {
+ memcpy(out_target_list, source_list, sizeof(*out_target_list));
+ memset(source_list, 0, sizeof(*source_list));
+}
+
+// Appends a timepoint to the end of the timepoint list.
+static void iree_hal_task_timepoint_list_append(
+ iree_hal_task_timepoint_list_t* list,
+ iree_hal_task_timepoint_t* timepoint) {
+ timepoint->next = NULL;
+ timepoint->prev = list->tail;
+ if (list->tail != NULL) {
+ list->tail->next = timepoint;
+ list->tail = timepoint;
+ } else {
+ list->head = timepoint;
+ list->tail = timepoint;
+ }
+}
+
+// Erases a timepoint from the list.
+static void iree_hal_task_timepoint_list_erase(
+ iree_hal_task_timepoint_list_t* list,
+ iree_hal_task_timepoint_t* timepoint) {
+ if (timepoint->prev != NULL) timepoint->prev->next = timepoint->next;
+ if (timepoint == list->head) list->head = timepoint->next;
+ if (timepoint == list->tail) list->tail = timepoint->prev;
+ timepoint->prev = NULL;
+ timepoint->next = NULL;
+}
+
+// Scans the |pending_list| for all timepoints that are satisfied by the
+// timeline having reached |payload_value|. Each satisfied timepoint will be
+// moved to |out_ready_list|.
+static void iree_hal_task_timepoint_list_take_ready(
+ iree_hal_task_timepoint_list_t* pending_list, uint64_t payload_value,
+ iree_hal_task_timepoint_list_t* out_ready_list) {
+ iree_hal_task_timepoint_list_initialize(out_ready_list);
+ iree_hal_task_timepoint_t* next = pending_list->head;
+ while (next != NULL) {
+ iree_hal_task_timepoint_t* timepoint = next;
+ next = timepoint->next;
+ bool is_satisfied = timepoint->payload_value <= payload_value;
+ if (!is_satisfied) continue;
+
+ // Remove from pending list.
+ iree_hal_task_timepoint_list_erase(pending_list, timepoint);
+
+ // Add to ready list.
+ iree_hal_task_timepoint_list_append(out_ready_list, timepoint);
+ }
+}
+
+// Notifies all of the timepoints in the |ready_list| that their condition has
+// been satisfied. |ready_list| will be reset as ownership of the events is
+// held by the originator.
+static void iree_hal_task_timepoint_list_notify_ready(
+ iree_hal_task_timepoint_list_t* ready_list) {
+ iree_hal_task_timepoint_t* next = ready_list->head;
+ while (next != NULL) {
+ iree_hal_task_timepoint_t* timepoint = next;
+ next = timepoint->next;
+ timepoint->next = NULL;
+ timepoint->prev = NULL;
+ iree_event_set(&timepoint->event);
+ }
+ iree_hal_task_timepoint_list_initialize(ready_list);
+}
+
+//===----------------------------------------------------------------------===//
+// iree_hal_task_semaphore_t
+//===----------------------------------------------------------------------===//
+
+typedef struct {
+ iree_hal_resource_t resource;
+ iree_allocator_t host_allocator;
+ iree_hal_local_event_pool_t* event_pool;
+
+ // Guards all mutable fields. 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_TASK_SEMAPHORE_FAILURE_VALUE to
+ // indicate that the semaphore has been signaled for failure and
+ // |failure_status| contains the error.
+ uint64_t current_value;
+
+ // OK or the status passed to iree_hal_semaphore_fail. Owned by the semaphore.
+ iree_status_t failure_status;
+
+ // In-process notification signaled when the semaphore value changes. This is
+ // used exclusively for wait-ones to avoid going to the kernel for a full wait
+ // handle operation.
+ iree_notification_t notification;
+
+ // A list of all reserved timepoints waiting for the semaphore to reach a
+ // certain payload value.
+ iree_hal_task_timepoint_list_t timepoint_list;
+} iree_hal_task_semaphore_t;
+
+static const iree_hal_semaphore_vtable_t iree_hal_task_semaphore_vtable;
+
+static iree_hal_task_semaphore_t* iree_hal_task_semaphore_cast(
+ iree_hal_semaphore_t* base_semaphore) {
+ return (iree_hal_task_semaphore_t*)base_semaphore;
+}
+
+iree_status_t iree_hal_task_semaphore_create(
+ iree_hal_local_event_pool_t* event_pool, uint64_t initial_value,
+ iree_allocator_t host_allocator, iree_hal_semaphore_t** out_semaphore) {
+ IREE_ASSERT_ARGUMENT(event_pool);
+ IREE_ASSERT_ARGUMENT(out_semaphore);
+ *out_semaphore = NULL;
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ iree_hal_task_semaphore_t* semaphore = NULL;
+ iree_status_t status = iree_allocator_malloc(
+ host_allocator, sizeof(*semaphore), (void**)&semaphore);
+ if (iree_status_is_ok(status)) {
+ iree_hal_resource_initialize(&iree_hal_task_semaphore_vtable,
+ &semaphore->resource);
+ semaphore->host_allocator = host_allocator;
+ semaphore->event_pool = event_pool;
+
+ iree_slim_mutex_initialize(&semaphore->mutex);
+ semaphore->current_value = initial_value;
+ semaphore->failure_status = iree_ok_status();
+ iree_notification_initialize(&semaphore->notification);
+ iree_hal_task_timepoint_list_initialize(&semaphore->timepoint_list);
+
+ *out_semaphore = (iree_hal_semaphore_t*)semaphore;
+ }
+
+ IREE_TRACE_ZONE_END(z0);
+ return status;
+}
+
+static void iree_hal_task_semaphore_destroy(
+ iree_hal_semaphore_t* base_semaphore) {
+ iree_hal_task_semaphore_t* semaphore =
+ iree_hal_task_semaphore_cast(base_semaphore);
+ iree_allocator_t host_allocator = semaphore->host_allocator;
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ iree_status_free(semaphore->failure_status);
+ iree_notification_deinitialize(&semaphore->notification);
+ iree_allocator_free(host_allocator, semaphore);
+
+ IREE_TRACE_ZONE_END(z0);
+}
+
+static iree_status_t iree_hal_task_semaphore_query(
+ iree_hal_semaphore_t* base_semaphore, uint64_t* out_value) {
+ iree_hal_task_semaphore_t* semaphore =
+ iree_hal_task_semaphore_cast(base_semaphore);
+
+ iree_slim_mutex_lock(&semaphore->mutex);
+
+ *out_value = semaphore->current_value;
+
+ iree_status_t status = iree_ok_status();
+ if (*out_value >= IREE_HAL_TASK_SEMAPHORE_FAILURE_VALUE) {
+ status = iree_status_clone(semaphore->failure_status);
+ }
+
+ iree_slim_mutex_unlock(&semaphore->mutex);
+
+ return status;
+}
+
+static iree_status_t iree_hal_task_semaphore_signal(
+ iree_hal_semaphore_t* base_semaphore, uint64_t new_value) {
+ iree_hal_task_semaphore_t* semaphore =
+ iree_hal_task_semaphore_cast(base_semaphore);
+
+ iree_slim_mutex_lock(&semaphore->mutex);
+
+ if (new_value <= semaphore->current_value) {
+ uint64_t current_value = semaphore->current_value;
+ iree_slim_mutex_unlock(&semaphore->mutex);
+ 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;
+
+ // Scan for all timepoints that are now satisfied and move them to our local
+ // ready list. This way we can notify them without needing to continue holding
+ // the semaphore lock.
+ iree_hal_task_timepoint_list_t ready_list;
+ iree_hal_task_timepoint_list_take_ready(&semaphore->timepoint_list, new_value,
+ &ready_list);
+
+ iree_notification_post(&semaphore->notification, IREE_ALL_WAITERS);
+ iree_slim_mutex_unlock(&semaphore->mutex);
+
+ // Notify all waiters - note that this must happen outside the lock.
+ iree_hal_task_timepoint_list_notify_ready(&ready_list);
+
+ return iree_ok_status();
+}
+
+static void iree_hal_task_semaphore_fail(iree_hal_semaphore_t* base_semaphore,
+ iree_status_t status) {
+ iree_hal_task_semaphore_t* semaphore =
+ iree_hal_task_semaphore_cast(base_semaphore);
+
+ 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);
+ return;
+ }
+
+ // Signal to our failure sentinel value.
+ semaphore->current_value = IREE_HAL_TASK_SEMAPHORE_FAILURE_VALUE;
+ semaphore->failure_status = status;
+
+ // Take the whole timepoint list as we'll be signaling all of them. Since
+ // we hold the lock no other timepoints can be created while we are cleaning
+ // up.
+ iree_hal_task_timepoint_list_t ready_list;
+ iree_hal_task_timepoint_list_move(&semaphore->timepoint_list, &ready_list);
+
+ iree_notification_post(&semaphore->notification, IREE_ALL_WAITERS);
+ iree_slim_mutex_unlock(&semaphore->mutex);
+
+ // Notify all waiters - note that this must happen outside the lock.
+ iree_hal_task_timepoint_list_notify_ready(&ready_list);
+}
+
+// Acquires a timepoint waiting for the given value.
+// |out_timepoint| is owned by the caller and must be kept live until the
+// timepoint has been reached (or it is cancelled by the caller).
+static iree_status_t iree_hal_task_semaphore_acquire_timepoint(
+ iree_hal_task_semaphore_t* semaphore, uint64_t minimum_value,
+ iree_hal_task_timepoint_t* out_timepoint) {
+ memset(out_timepoint, 0, sizeof(*out_timepoint));
+ out_timepoint->payload_value = minimum_value;
+ IREE_RETURN_IF_ERROR(iree_hal_local_event_pool_acquire(
+ semaphore->event_pool, 1, &out_timepoint->event));
+ iree_hal_task_timepoint_list_append(&semaphore->timepoint_list,
+ out_timepoint);
+ return iree_ok_status();
+}
+
+typedef struct {
+ iree_task_wait_t task;
+ iree_hal_task_semaphore_t* semaphore;
+ iree_hal_task_timepoint_t timepoint;
+} iree_hal_task_semaphore_wait_cmd_t;
+
+// Cleans up a wait task by returning the event used to the pool and - if the
+// task failed - ensuring we scrub it from the timepoint list.
+static void iree_hal_task_semaphore_wait_cmd_cleanup(iree_task_t* task,
+ iree_status_t status) {
+ iree_hal_task_semaphore_wait_cmd_t* cmd =
+ (iree_hal_task_semaphore_wait_cmd_t*)task;
+ iree_hal_local_event_pool_release(cmd->semaphore->event_pool, 1,
+ &cmd->timepoint.event);
+ if (IREE_UNLIKELY(!iree_status_is_ok(status))) {
+ // Abort the timepoint. Note that this is not designed to be fast as
+ // semaphore failure is an exceptional case.
+ iree_slim_mutex_lock(&cmd->semaphore->mutex);
+ iree_hal_task_timepoint_list_erase(&cmd->semaphore->timepoint_list,
+ &cmd->timepoint);
+ iree_slim_mutex_unlock(&cmd->semaphore->mutex);
+ }
+}
+
+iree_status_t iree_hal_task_semaphore_enqueue_timepoint(
+ iree_hal_semaphore_t* base_semaphore, uint64_t minimum_value,
+ iree_task_t* issue_task, iree_arena_allocator_t* arena,
+ iree_task_submission_t* submission) {
+ iree_hal_task_semaphore_t* semaphore =
+ iree_hal_task_semaphore_cast(base_semaphore);
+
+ iree_slim_mutex_lock(&semaphore->mutex);
+
+ iree_status_t status = iree_ok_status();
+ if (semaphore->current_value >= minimum_value) {
+ // Fast path: already satisfied.
+ } else {
+ // Slow path: acquire a system wait handle and perform a full wait.
+ iree_hal_task_semaphore_wait_cmd_t* cmd = NULL;
+ status = iree_arena_allocate(arena, sizeof(*cmd), (void**)&cmd);
+ if (iree_status_is_ok(status)) {
+ status = iree_hal_task_semaphore_acquire_timepoint(
+ semaphore, minimum_value, &cmd->timepoint);
+ }
+ if (iree_status_is_ok(status)) {
+ iree_task_wait_initialize(issue_task->scope, cmd->timepoint.event,
+ &cmd->task);
+ iree_task_set_cleanup_fn(&cmd->task.header,
+ iree_hal_task_semaphore_wait_cmd_cleanup);
+ iree_task_set_completion_task(&cmd->task.header, issue_task);
+ cmd->semaphore = semaphore;
+ iree_task_submission_enqueue(submission, &cmd->task.header);
+ }
+ }
+
+ iree_slim_mutex_unlock(&semaphore->mutex);
+ return status;
+}
+
+static iree_status_t iree_hal_task_semaphore_wait_with_deadline(
+ iree_hal_semaphore_t* base_semaphore, uint64_t value,
+ iree_time_t deadline_ns) {
+ iree_hal_task_semaphore_t* semaphore =
+ iree_hal_task_semaphore_cast(base_semaphore);
+
+ iree_slim_mutex_lock(&semaphore->mutex);
+
+ if (semaphore->current_value >= value) {
+ // Fast path: already satisfied.
+ iree_slim_mutex_unlock(&semaphore->mutex);
+ return iree_ok_status();
+ } else if (deadline_ns == IREE_TIME_INFINITE_PAST) {
+ // Not satisfied but a poll, so can avoid the expensive wait handle work.
+ iree_slim_mutex_unlock(&semaphore->mutex);
+ return iree_status_from_code(IREE_STATUS_DEADLINE_EXCEEDED);
+ }
+
+ // Slow path: acquire a timepoint while we hold the lock.
+ iree_hal_task_timepoint_t timepoint;
+ iree_status_t status =
+ iree_hal_task_semaphore_acquire_timepoint(semaphore, value, &timepoint);
+
+ iree_slim_mutex_unlock(&semaphore->mutex);
+ if (IREE_UNLIKELY(!iree_status_is_ok(status))) 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.event, deadline_ns);
+ if (!iree_status_is_ok(status)) {
+ iree_slim_mutex_lock(&semaphore->mutex);
+ iree_hal_task_timepoint_list_erase(&semaphore->timepoint_list, &timepoint);
+ iree_slim_mutex_unlock(&semaphore->mutex);
+ }
+ iree_hal_local_event_pool_release(semaphore->event_pool, 1, &timepoint.event);
+ return status;
+}
+
+static iree_status_t iree_hal_task_semaphore_wait_with_timeout(
+ iree_hal_semaphore_t* base_semaphore, uint64_t value,
+ iree_duration_t timeout_ns) {
+ return iree_hal_task_semaphore_wait_with_deadline(
+ base_semaphore, value, iree_relative_timeout_to_deadline_ns(timeout_ns));
+}
+
+iree_status_t iree_hal_task_semaphore_multi_wait(
+ iree_hal_wait_mode_t wait_mode,
+ const iree_hal_semaphore_list_t* semaphore_list, iree_time_t deadline_ns,
+ iree_hal_local_event_pool_t* event_pool,
+ iree_arena_block_pool_t* block_pool) {
+ IREE_ASSERT_ARGUMENT(semaphore_list);
+ if (semaphore_list->count == 0) {
+ return iree_ok_status();
+ } else if (semaphore_list->count == 1) {
+ // Fast-path for a single semaphore.
+ return iree_hal_semaphore_wait_with_deadline(
+ semaphore_list->semaphores[0], semaphore_list->payload_values[0],
+ deadline_ns);
+ }
+
+ IREE_TRACE_ZONE_BEGIN(z0);
+
+ // Avoid heap allocations by using the device block pool for the wait set.
+ iree_arena_allocator_t arena;
+ iree_arena_initialize(block_pool, &arena);
+ iree_wait_set_t* wait_set = NULL;
+ iree_status_t status = iree_wait_set_allocate(
+ semaphore_list->count, iree_arena_allocator(&arena), &wait_set);
+
+ // Acquire a wait handle for each semaphore timepoint we are to wait on.
+ // TODO(benvanik): flip this API around so we can batch request events from
+ // the event pool. We should be acquiring all required time points in one
+ // call.
+ iree_host_size_t timepoint_count = 0;
+ iree_hal_task_timepoint_t* timepoints = NULL;
+ iree_host_size_t total_timepoint_size =
+ semaphore_list->count * sizeof(timepoints[0]);
+ status =
+ iree_arena_allocate(&arena, total_timepoint_size, (void**)&timepoints);
+ if (iree_status_is_ok(status)) {
+ memset(timepoints, 0, total_timepoint_size);
+ for (iree_host_size_t i = 0; i < semaphore_list->count; ++i) {
+ iree_hal_task_semaphore_t* semaphore =
+ iree_hal_task_semaphore_cast(semaphore_list->semaphores[i]);
+ iree_slim_mutex_lock(&semaphore->mutex);
+ if (semaphore->current_value >= semaphore_list->payload_values[i]) {
+ // Fast path: already satisfied.
+ } else {
+ // Slow path: get a native wait handle for the timepoint.
+ iree_hal_task_timepoint_t* timepoint = &timepoints[timepoint_count++];
+ status = iree_hal_task_semaphore_acquire_timepoint(
+ semaphore, semaphore_list->payload_values[i], timepoint);
+ if (iree_status_is_ok(status)) {
+ status = iree_wait_set_insert(wait_set, timepoint->event);
+ }
+ }
+ iree_slim_mutex_unlock(&semaphore->mutex);
+ if (!iree_status_is_ok(status)) break;
+ }
+ }
+
+ // Perform the wait.
+ if (iree_status_is_ok(status)) {
+ if (wait_mode == IREE_HAL_WAIT_MODE_ANY) {
+ status = iree_wait_any(wait_set, deadline_ns, /*out_wake_handle=*/NULL);
+ } else {
+ status = iree_wait_all(wait_set, deadline_ns);
+ }
+ }
+
+ if (timepoints != NULL) {
+ // TODO(benvanik): if we flip the API to multi-acquire events from the pool
+ // above then we can multi-release here too.
+ for (iree_host_size_t i = 0; i < timepoint_count; ++i) {
+ iree_hal_local_event_pool_release(event_pool, 1, &timepoints[i].event);
+ }
+ }
+ iree_wait_set_free(wait_set);
+ iree_arena_deinitialize(&arena);
+
+ IREE_TRACE_ZONE_END(z0);
+ return status;
+}
+
+static const iree_hal_semaphore_vtable_t iree_hal_task_semaphore_vtable = {
+ .destroy = iree_hal_task_semaphore_destroy,
+ .query = iree_hal_task_semaphore_query,
+ .signal = iree_hal_task_semaphore_signal,
+ .fail = iree_hal_task_semaphore_fail,
+ .wait_with_deadline = iree_hal_task_semaphore_wait_with_deadline,
+ .wait_with_timeout = iree_hal_task_semaphore_wait_with_timeout,
+};
diff --git a/iree/hal/local/task_semaphore.h b/iree/hal/local/task_semaphore.h
new file mode 100644
index 0000000..88eafb9
--- /dev/null
+++ b/iree/hal/local/task_semaphore.h
@@ -0,0 +1,58 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// https://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef IREE_HAL_LOCAL_TASK_SEMAPHORE_H_
+#define IREE_HAL_LOCAL_TASK_SEMAPHORE_H_
+
+#include "iree/base/api.h"
+#include "iree/hal/api.h"
+#include "iree/hal/local/arena.h"
+#include "iree/hal/local/event_pool.h"
+#include "iree/task/submission.h"
+#include "iree/task/task.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif // __cplusplus
+
+// Creates a semaphore that integrates with the task system to allow for
+// pipelined wait and signal operations.
+iree_status_t iree_hal_task_semaphore_create(
+ iree_hal_local_event_pool_t* event_pool, uint64_t initial_value,
+ iree_allocator_t host_allocator, iree_hal_semaphore_t** out_semaphore);
+
+// Reserves a new timepoint in the timeline for the given minimum payload value.
+// |issue_task| will wait until the timeline semaphore is signaled to at least
+// |minimum_value| before proceeding, with a possible wait task generated and
+// appended to the |submission|. Allocations for any intermediates will be made
+// from |arena| whose lifetime must be tied to the submission.
+iree_status_t iree_hal_task_semaphore_enqueue_timepoint(
+ iree_hal_semaphore_t* semaphore, uint64_t minimum_value,
+ iree_task_t* issue_task, iree_arena_allocator_t* arena,
+ iree_task_submission_t* submission);
+
+// Performs a multi-wait on one or more semaphores.
+// Returns IREE_STATUS_DEADLINE_EXCEEDED if the wait does not complete before
+// |deadline_ns| elapses.
+iree_status_t iree_hal_task_semaphore_multi_wait(
+ iree_hal_wait_mode_t wait_mode,
+ const iree_hal_semaphore_list_t* semaphore_list, iree_time_t deadline_ns,
+ iree_hal_local_event_pool_t* event_pool,
+ iree_arena_block_pool_t* block_pool);
+
+#ifdef __cplusplus
+} // extern "C"
+#endif // __cplusplus
+
+#endif // IREE_HAL_LOCAL_TASK_SEMAPHORE_H_
diff --git a/iree/hal/semaphore.h b/iree/hal/semaphore.h
index 06795a8..a010503 100644
--- a/iree/hal/semaphore.h
+++ b/iree/hal/semaphore.h
@@ -97,7 +97,8 @@
// Signals the |semaphore| with a failure. The |status| will be returned from
// iree_hal_semaphore_query and iree_hal_semaphore_signal for the lifetime
-// of the semaphore.
+// of the semaphore. Ownership of the status transfers to the semaphore and
+// callers must clone it if they wish to retain it.
IREE_API_EXPORT void IREE_API_CALL
iree_hal_semaphore_fail(iree_hal_semaphore_t* semaphore, iree_status_t status);
diff --git a/iree/task/executor.c b/iree/task/executor.c
index 3bbe474..a4d6aa6 100644
--- a/iree/task/executor.c
+++ b/iree/task/executor.c
@@ -530,75 +530,87 @@
}
IREE_TRACE_ZONE_BEGIN(z0);
- // Check for incoming submissions and move their posted tasks into our
- // local lists. Any of the tasks here are ready to execute immediately and
- // ones we should be able to distribute to workers without delay. The
- // waiting tasks are to the best of the caller's knowledge not ready yet.
- //
- // Note that we only do this once per coordination; that's so we don't
- // starve if submissions come in faster than we can schedule them.
- // Coordination will run again when workers become idle and will pick up
- // any changes then.
- //
- // As we schedule tasks we may spawn new ones (like a dispatch -> many
- // dispatch slices) and we keep track of those here. By doing a pass through
- // all ready tasks and only then merging in the new submission we get
- // breadth-first traversal of task graphs even if they originate from
- // various places and have no relation - hopefully leading to better average
- // latency.
- iree_task_submission_t pending_submission;
- iree_task_submission_initialize_from_lifo_slist(
- &executor->incoming_ready_slist, &pending_submission);
- iree_task_list_append_from_fifo_slist(&pending_submission.waiting_list,
- &executor->incoming_waiting_slist);
+ // We may be adding tasks/waiting/etc on each pass through coordination - to
+ // ensure we completely drain the incoming queues and satisfied waits we loop
+ // until there's nothing left to coordinate.
+ bool schedule_dirty = true;
+ do {
+ // Check for incoming submissions and move their posted tasks into our
+ // local lists. Any of the tasks here are ready to execute immediately and
+ // ones we should be able to distribute to workers without delay. The
+ // waiting tasks are to the best of the caller's knowledge not ready yet.
+ //
+ // Note that we only do this once per coordination; that's so we don't
+ // starve if submissions come in faster than we can schedule them.
+ // Coordination will run again when workers become idle and will pick up
+ // any changes then.
+ //
+ // As we schedule tasks we may spawn new ones (like a dispatch -> many
+ // dispatch slices) and we keep track of those here. By doing a pass through
+ // all ready tasks and only then merging in the new submission we get
+ // breadth-first traversal of task graphs even if they originate from
+ // various places and have no relation - hopefully leading to better average
+ // latency.
+ iree_task_submission_t pending_submission;
+ iree_task_submission_initialize_from_lifo_slist(
+ &executor->incoming_ready_slist, &pending_submission);
+ iree_task_list_append_from_fifo_slist(&pending_submission.waiting_list,
+ &executor->incoming_waiting_slist);
- // Scratch coordinator submission batch used during scheduling to batch up
- // all tasks that will be posted to each worker. We could stash this on the
- // executor but given that which thread is playing the role of the coordinator
- // is random it's better to ensure that these bytes never incur a cache miss
- // by making them live here in the stack of the chosen thread.
- iree_task_post_batch_t* post_batch =
- iree_alloca(sizeof(iree_task_post_batch_t) +
- executor->worker_count * sizeof(iree_task_list_t));
- iree_task_post_batch_initialize(executor, current_worker, post_batch);
+ // Scratch coordinator submission batch used during scheduling to batch up
+ // all tasks that will be posted to each worker. We could stash this on the
+ // executor but given that which thread is playing the role of the
+ // coordinator is random it's better to ensure that these bytes never incur
+ // a cache miss by making them live here in the stack of the chosen thread.
+ iree_task_post_batch_t* post_batch =
+ iree_alloca(sizeof(iree_task_post_batch_t) +
+ executor->worker_count * sizeof(iree_task_list_t));
+ iree_task_post_batch_initialize(executor, current_worker, post_batch);
- // Poll the waiting tasks to see if any have resolved. This dramatically
- // cuts latency in cases where the wait handle completes prior to us
- // entering the real wait. When we have semaphores sequencing back-to-back
- // work this ensures that we pack in future dispatch work earlier vs.
- // waiting for a full thread hop.
- //
- // If any waits have resolved then they'll be moved to the ready list here
- // and then get processed FIFO with the tasks that were ready in the
- // request.
- iree_task_executor_poll_waiting_tasks(executor, &pending_submission);
+ // Poll the waiting tasks to see if any have resolved. This dramatically
+ // cuts latency in cases where the wait handle completes prior to us
+ // entering the real wait. When we have semaphores sequencing back-to-back
+ // work this ensures that we pack in future dispatch work earlier vs.
+ // waiting for a full thread hop.
+ //
+ // If any waits have resolved then they'll be moved to the ready list here
+ // and then get processed FIFO with the tasks that were ready in the
+ // request.
+ iree_task_executor_poll_waiting_tasks(executor, &pending_submission);
- // Schedule all ready tasks in this batch. Some may complete inline (such
- // as ready barriers with all their dependencies resolved) while others may
- // be scheduled on workers via the post batch.
- iree_task_executor_schedule_ready_tasks(executor, &pending_submission,
- post_batch);
+ // Schedule all ready tasks in this batch. Some may complete inline (such
+ // as ready barriers with all their dependencies resolved) while others may
+ // be scheduled on workers via the post batch.
+ iree_task_executor_schedule_ready_tasks(executor, &pending_submission,
+ post_batch);
- // Merge any newly waiting tasks into the global wait list.
- iree_task_executor_merge_wait_list(executor,
- &pending_submission.waiting_list);
+ // Merge any newly waiting tasks into the global wait list.
+ iree_task_executor_merge_wait_list(executor,
+ &pending_submission.waiting_list);
- // Post all new work to workers; they may wake and begin executing
- // immediately. Returns whether this worker has new tasks for it to work on.
- bool did_post = iree_task_post_batch_submit(post_batch);
- if (!did_post && speculative) {
- // No work was found; wait on one or more of our wait handles.
- // This will block the calling thread but that's fine as they were going
- // to wait anyway and were just speculatively seeing if there was work first
- // by requesting coordination. If work completes here we'll catch it on
- // the poll next loop around.
- iree_task_executor_wait_any_task(executor, current_worker,
- &pending_submission);
- }
+ // Post all new work to workers; they may wake and begin executing
+ // immediately. Returns whether this worker has new tasks for it to work on.
+ bool did_post = iree_task_post_batch_submit(post_batch);
+ if (!did_post && speculative) {
+ // No work was found; wait on one or more of our wait handles.
+ // This will block the calling thread but that's fine as they were going
+ // to wait anyway and were just speculatively seeing if there was work
+ // first by requesting coordination. If work completes here we'll catch it
+ // on the poll next loop around.
+ iree_task_executor_wait_any_task(executor, current_worker,
+ &pending_submission);
+ }
- // Merge any new work into the submission list for future coordinators to
- // deal with - we don't want the possibility of starvation by looping on this.
- iree_task_executor_merge_submission(executor, &pending_submission);
+ // Merge any new work into the submission list for future coordinators to
+ // deal with - we don't want the possibility of starvation by looping on
+ // this.
+ if (!iree_task_submission_is_empty(&pending_submission)) {
+ iree_task_executor_merge_submission(executor, &pending_submission);
+ schedule_dirty = true;
+ } else {
+ schedule_dirty = false;
+ }
+ } while (schedule_dirty);
iree_slim_mutex_unlock(&executor->coordinator_mutex);
IREE_TRACE_ZONE_END(z0);
diff --git a/iree/task/list.c b/iree/task/list.c
index 6e70000..bba9a21 100644
--- a/iree/task/list.c
+++ b/iree/task/list.c
@@ -107,8 +107,9 @@
void iree_task_list_erase(iree_task_list_t* list, iree_task_t* prev_task,
iree_task_t* task) {
if (task == list->head) {
- // Removing head.
+ // Removing head (which may _also_ be the tail).
list->head = task->next_task;
+ if (list->tail == task) list->tail = task->next_task;
} else if (task == list->tail) {
// Removing tail.
list->tail = prev_task;
diff --git a/iree/task/list_test.cc b/iree/task/list_test.cc
index 19b472f..3933082 100644
--- a/iree/task/list_test.cc
+++ b/iree/task/list_test.cc
@@ -175,6 +175,8 @@
iree_task_list_erase(&list, NULL, task1);
EXPECT_TRUE(iree_task_list_is_empty(&list));
+ EXPECT_EQ(NULL, iree_task_list_front(&list));
+ EXPECT_EQ(NULL, iree_task_list_back(&list));
}
TEST(TaskListTest, PrependEmpty) {