Propagating errors from task system calls/dispatches.
Waits still need work that will come with follow-on refactorings.
Failures of calls and dispatch workgroups will now make their way to the
parent scope and properly discard all subsequent tasks. The scope
retains the first failure encountered so that it can be returned back to
user layers (propagated to HAL semaphores, the event loop, etc).

Progress on #4026.
diff --git a/iree/base/assert.h b/iree/base/assert.h
index 450bf9d..930baab 100644
--- a/iree/base/assert.h
+++ b/iree/base/assert.h
@@ -61,6 +61,8 @@
 #define IREE_ASSERT_TRUE(expr, ...) IREE_ASSERT(!!(expr), __VA_ARGS__)
 #define IREE_ASSERT_FALSE(expr, ...) IREE_ASSERT(!(expr), __VA_ARGS__)
 
+#define IREE_ASSERT_UNREACHABLE(...) IREE_ASSERT(false, __VA_ARGS__)
+
 #define IREE_ASSERT_EQ(x, y, ...) _IREE_ASSERT_CMP(x, ==, y, __VA_ARGS__)
 #define IREE_ASSERT_NE(x, y, ...) _IREE_ASSERT_CMP(x, !=, y, __VA_ARGS__)
 #define IREE_ASSERT_LE(x, y, ...) _IREE_ASSERT_CMP(x, <=, y, __VA_ARGS__)
diff --git a/iree/hal/local/task_queue.c b/iree/hal/local/task_queue.c
index 8baa950..23fcb43 100644
--- a/iree/hal/local/task_queue.c
+++ b/iree/hal/local/task_queue.c
@@ -135,8 +135,8 @@
 
 // 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) {
+static void iree_hal_task_queue_wait_cmd_cleanup(
+    iree_task_t* task, iree_status_code_t status_code) {
   iree_hal_task_queue_wait_cmd_t* cmd = (iree_hal_task_queue_wait_cmd_t*)task;
   iree_hal_semaphore_list_release(&cmd->wait_semaphores);
 }
@@ -219,8 +219,8 @@
 
 // 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) {
+static void iree_hal_task_queue_issue_cmd_cleanup(
+    iree_task_t* task, iree_status_code_t status_code) {
   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.
@@ -307,17 +307,17 @@
 // 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) {
+static void iree_hal_task_queue_retire_cmd_cleanup(
+    iree_task_t* task, iree_status_code_t status_code) {
   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)) {
+  if (IREE_UNLIKELY(status_code != IREE_STATUS_OK)) {
     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));
+                              iree_status_from_code(status_code));
     }
   }
 
diff --git a/iree/hal/local/task_semaphore.c b/iree/hal/local/task_semaphore.c
index 96d7ed5..59c0327 100644
--- a/iree/hal/local/task_semaphore.c
+++ b/iree/hal/local/task_semaphore.c
@@ -321,12 +321,12 @@
 
 // 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) {
+static void iree_hal_task_semaphore_wait_cmd_cleanup(
+    iree_task_t* task, iree_status_code_t status_code) {
   iree_hal_task_semaphore_wait_cmd_t* cmd =
       (iree_hal_task_semaphore_wait_cmd_t*)task;
   iree_event_pool_release(cmd->semaphore->event_pool, 1, &cmd->timepoint.event);
-  if (IREE_UNLIKELY(!iree_status_is_ok(status))) {
+  if (IREE_UNLIKELY(status_code != IREE_STATUS_OK)) {
     // 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);
diff --git a/iree/task/executor_test.cc b/iree/task/executor_test.cc
index b5dbd66..fac31e9 100644
--- a/iree/task/executor_test.cc
+++ b/iree/task/executor_test.cc
@@ -124,10 +124,10 @@
                                 [](void* user_context, iree_task_t* task,
                                    iree_task_submission_t* pending_submission) {
                                   IREE_TRACE_SCOPE0("call1");
-                                  EXPECT_EQ(1, user_context);
+                                  EXPECT_EQ((void*)1, user_context);
                                   return iree_ok_status();
                                 },
-                                1),
+                                (void*)1),
                             &call1);
 
 #if 1
diff --git a/iree/task/scope.c b/iree/task/scope.c
index 37aaf5e..245fe90 100644
--- a/iree/task/scope.c
+++ b/iree/task/scope.c
@@ -107,7 +107,6 @@
 
 void iree_task_scope_fail(iree_task_scope_t* scope, iree_task_t* task,
                           iree_status_t status) {
-  // TODO(benvanik): logging/tracing based on task.
   iree_task_scope_try_set_status(scope, status);
 }
 
diff --git a/iree/task/task.c b/iree/task/task.c
index 123307b..01b9196 100644
--- a/iree/task/task.c
+++ b/iree/task/task.c
@@ -53,12 +53,33 @@
   return true;
 }
 
-static void iree_task_cleanup(iree_task_t* task, iree_status_t status) {
+static void iree_task_try_set_status(iree_atomic_intptr_t* permanent_status,
+                                     iree_status_t new_status) {
+  if (IREE_UNLIKELY(iree_status_is_ok(new_status))) return;
+
+  IREE_TRACE_ZONE_BEGIN(z0);
+  IREE_TRACE_ZONE_APPEND_TEXT(z0, "failed: ");
+  IREE_TRACE_ZONE_APPEND_TEXT(
+      z0, iree_status_code_string(iree_status_code(new_status)));
+
+  iree_status_t old_status = iree_ok_status();
+  if (!iree_atomic_compare_exchange_strong_intptr(
+          permanent_status, (intptr_t*)&old_status, (intptr_t)new_status,
+          iree_memory_order_seq_cst, iree_memory_order_seq_cst)) {
+    // Previous status was not OK; drop our new status.
+    IREE_IGNORE_ERROR(new_status);
+  }
+
+  IREE_TRACE_ZONE_END(z0);
+}
+
+static void iree_task_cleanup(iree_task_t* task,
+                              iree_status_code_t status_code) {
   // Call the (optional) cleanup function.
   // NOTE: this may free the memory of the task itself!
   iree_task_pool_t* pool = task->pool;
   if (task->cleanup_fn) {
-    task->cleanup_fn(task, status);
+    task->cleanup_fn(task, status_code);
   }
 
   // Return the task to the pool it was allocated from.
@@ -96,8 +117,6 @@
       break;
     }
     case IREE_TASK_TYPE_FENCE: {
-      iree_task_scope_fail(task->scope, task,
-                           iree_status_from_code(IREE_STATUS_ABORTED));
       iree_task_scope_end(task->scope);
       break;
     }
@@ -107,29 +126,58 @@
       break;
   }
 
-  iree_task_cleanup(task, iree_status_from_code(IREE_STATUS_ABORTED));
+  iree_task_cleanup(task, IREE_STATUS_ABORTED);
   // NOTE: task is invalidated here and cannot be used!
 
   IREE_TRACE_ZONE_END(z0);
 }
 
 static void iree_task_retire(iree_task_t* task,
-                             iree_task_submission_t* pending_submission) {
+                             iree_task_submission_t* pending_submission,
+                             iree_status_t status) {
   IREE_ASSERT_EQ(0, iree_atomic_load_int32(&task->pending_dependency_count,
                                            iree_memory_order_acquire));
 
   // Decrement the pending count on the completion task, if any.
   iree_task_t* completion_task = task->completion_task;
   task->completion_task = NULL;
-  if (completion_task &&
+  bool completion_task_ready =
+      completion_task &&
       iree_atomic_fetch_sub_int32(&completion_task->pending_dependency_count, 1,
-                                  iree_memory_order_acq_rel) == 1) {
-    // The completion task has retired and can now be made ready.
-    iree_task_submission_enqueue(pending_submission, completion_task);
+                                  iree_memory_order_acq_rel) == 1;
+
+  if (iree_status_is_ok(status)) {
+    // Task completed successfully.
+    iree_task_cleanup(task, IREE_STATUS_OK);
+    if (completion_task_ready) {
+      // This was the last pending dependency and the completion task is ready
+      // to run.
+      iree_task_submission_enqueue(pending_submission, completion_task);
+    }
+  } else {
+    // Task failed.
+    iree_task_scope_fail(task->scope, task, status);
+    status = iree_ok_status();  // consumed by the fail
+    iree_task_cleanup(task, IREE_STATUS_ABORTED);
+    if (completion_task_ready) {
+      // This was the last pending dependency and we know that we can safely
+      // abort the completion task by discarding.
+      iree_task_list_t discard_worklist;
+      iree_task_list_initialize(&discard_worklist);
+      iree_task_discard(completion_task, &discard_worklist);
+      iree_task_list_discard(&discard_worklist);
+    } else if (completion_task) {
+      // One or more pending dependencies are not yet satisfied and the
+      // completion task must stay alive. We can mark it as aborted, though,
+      // so that it knows not to execute when it is ready to run.
+      // TODO(benvanik): make this atomic? we only ever add bits and it's safe
+      // for it to run if we got this far.
+      completion_task->flags |= IREE_TASK_FLAG_ABORTED;
+    }
   }
 
-  iree_task_cleanup(task, iree_ok_status());
   // NOTE: task is invalidated here and cannot be used!
+  task = NULL;
 }
 
 //==============================================================================
@@ -143,36 +191,69 @@
 
 void iree_task_nop_retire(iree_task_nop_t* task,
                           iree_task_submission_t* pending_submission) {
-  iree_task_retire(&task->header, pending_submission);
+  iree_task_retire(&task->header, pending_submission, iree_ok_status());
 }
 
 //==============================================================================
 // IREE_TASK_TYPE_CALL
 //==============================================================================
 
+// Returns an XXBBGGRR color (red in the lowest bits).
+// Must not be 0 (tracy will ignore).
+static uint32_t iree_math_ptr_to_xrgb(const void* ptr) {
+  // This is just a simple hack to give us a unique(ish) per-pointer color.
+  // It's only to make it easier to distinguish which tiles are from the same
+  // dispatch.
+  uint64_t ptr64 = (uintptr_t)ptr;
+  return (uint32_t)ptr64 ^ (uint32_t)(ptr64 >> 32);
+}
+
 void iree_task_call_initialize(iree_task_scope_t* scope,
                                iree_task_call_closure_t closure,
                                iree_task_call_t* out_task) {
   iree_task_initialize(IREE_TASK_TYPE_CALL, scope, &out_task->header);
   out_task->closure = closure;
+  iree_atomic_store_intptr(&out_task->status, 0, iree_memory_order_release);
 }
 
-iree_status_t iree_task_call_execute(
-    iree_task_call_t* task, iree_task_submission_t* pending_submission) {
+void iree_task_call_execute(iree_task_call_t* task,
+                            iree_task_submission_t* pending_submission) {
   IREE_TRACE_ZONE_BEGIN(z0);
+  IREE_TRACE_ZONE_SET_COLOR(z0,
+                            iree_math_ptr_to_xrgb(task->closure.user_context));
 
-  // Execute the user callback.
-  // Note that this may enqueue more nested tasks, including tasks that prevent
-  // this task from retiring.
-  iree_status_t status = task->closure.fn(task->closure.user_context,
-                                          &task->header, pending_submission);
+  if (IREE_LIKELY(
+          !iree_any_bit_set(task->header.flags, IREE_TASK_FLAG_ABORTED))) {
+    // Execute the user callback.
+    // Note that this may enqueue more nested tasks, including tasks that
+    // prevent this task from retiring.
+    iree_status_t status = task->closure.fn(task->closure.user_context,
+                                            &task->header, pending_submission);
+    if (!iree_status_is_ok(status)) {
+      // Stash the failure status on the task.
+      // If there's still pending dependencies we won't be able to discard
+      // immediately and need to keep the status around until they all complete.
+      iree_task_try_set_status(&task->status, status);
+      status = iree_ok_status();  // consumed by try_set_status
+
+      // TODO(benvanik): discard pending_submission? As we may have pending work
+      // from multiple scopes it's dangerous to discard all. We could filter
+      // based on scope, though, and if we did that we (probably) wouldn't need
+      // to handle the permanent status on the task and could discard
+      // immediately.
+    }
+  }
+
+  // Check to see if there are no pending dependencies before retiring; the
+  // dependency count can go up if new nested tasks were enqueued.
   if (iree_atomic_load_int32(&task->header.pending_dependency_count,
                              iree_memory_order_acquire) == 0) {
-    iree_task_retire(&task->header, pending_submission);
+    iree_status_t status = (iree_status_t)iree_atomic_exchange_intptr(
+        &task->status, 0, iree_memory_order_seq_cst);
+    iree_task_retire(&task->header, pending_submission, status);
   }
 
   IREE_TRACE_ZONE_END(z0);
-  return status;
 }
 
 //==============================================================================
@@ -227,7 +308,7 @@
     }
   }
 
-  iree_task_retire(&task->header, pending_submission);
+  iree_task_retire(&task->header, pending_submission, iree_ok_status());
   IREE_TRACE_ZONE_END(z0);
 }
 
@@ -247,7 +328,7 @@
 
   iree_task_scope_end(task->header.scope);
 
-  iree_task_retire(&task->header, pending_submission);
+  iree_task_retire(&task->header, pending_submission, iree_ok_status());
   IREE_TRACE_ZONE_END(z0);
 }
 
@@ -272,7 +353,7 @@
                            iree_task_submission_t* pending_submission) {
   IREE_TRACE_ZONE_BEGIN(z0);
   // TODO(benvanik): allow deinit'ing the wait handle (if transient).
-  iree_task_retire(&task->header, pending_submission);
+  iree_task_retire(&task->header, pending_submission, iree_ok_status());
   IREE_TRACE_ZONE_END(z0);
 }
 
@@ -282,16 +363,6 @@
 
 // Returns an XXBBGGRR color (red in the lowest bits).
 // Must not be 0 (tracy will ignore).
-static uint32_t iree_math_ptr_to_xrgb(const uintptr_t ptr) {
-  // This is just a simple hack to give us a unique(ish) per-pointer color.
-  // It's only to make it easier to distinguish which tiles are from the same
-  // dispatch.
-  uint64_t ptr64 = ptr;
-  return (uint32_t)ptr64 ^ (uint32_t)(ptr64 >> 32);
-}
-
-// Returns an XXBBGGRR color (red in the lowest bits).
-// Must not be 0 (tracy will ignore).
 static uint32_t iree_task_tile_to_color(
     const iree_task_tile_context_t* tile_context);
 
@@ -376,6 +447,7 @@
   memcpy(out_task->workgroup_size, workgroup_size,
          sizeof(out_task->workgroup_size));
   out_task->local_memory_size = 0;
+  iree_atomic_store_intptr(&out_task->status, 0, iree_memory_order_release);
   memset(&out_task->statistics, 0, sizeof(out_task->statistics));
 }
 
@@ -611,7 +683,19 @@
       &dispatch_task->statistics,
       &dispatch_task->header.scope->dispatch_statistics);
 
-  iree_task_retire(&dispatch_task->header, pending_submission);
+  // Consume the status of the dispatch that may have been set from a workgroup
+  // and notify the scope. We need to do this here so that each slice/shard
+  // retires before we discard any subsequent tasks: otherwise a failure of
+  // one shard would discard the shared dispatch task (and potentially
+  // everything) while other shards were still running. We also want to avoid
+  // fine-grained synchronization across slices/shards that would occur by each
+  // checking to see if any other has hit an error; failure in a dispatch should
+  // be so exceedingly rare that allowing some shards to complete after one
+  // encounters an error is not a problem.
+  iree_status_t status = (iree_status_t)iree_atomic_exchange_intptr(
+      &dispatch_task->status, 0, iree_memory_order_seq_cst);
+
+  iree_task_retire(&dispatch_task->header, pending_submission, status);
   IREE_TRACE_ZONE_END(z0);
 }
 
@@ -628,6 +712,7 @@
                        dispatch_task->header.scope, &out_task->header);
   iree_task_set_completion_task(&out_task->header, &dispatch_task->header);
   out_task->closure = dispatch_task->closure;
+  out_task->dispatch_status = &dispatch_task->status;
 
   memcpy(out_task->workgroup_base, workgroup_base,
          sizeof(out_task->workgroup_base));
@@ -666,7 +751,7 @@
   return slice_task;
 }
 
-iree_status_t iree_task_dispatch_slice_execute(
+void iree_task_dispatch_slice_execute(
     iree_task_dispatch_slice_t* task, iree_byte_span_t local_memory,
     iree_task_submission_t* pending_submission) {
   IREE_TRACE_ZONE_BEGIN(z0);
@@ -689,14 +774,19 @@
   // This ensures that how much memory is used by some executions does not
   // inadvertently leak over into other executions.
   if (IREE_UNLIKELY(task->local_memory_size > local_memory.data_length)) {
-    return iree_make_status(IREE_STATUS_RESOURCE_EXHAUSTED,
-                            "dispatch requires %ub of local memory but only "
-                            "%zub is available per-worker",
-                            task->local_memory_size, local_memory.data_length);
+    iree_task_retire(
+        &task->header, pending_submission,
+        iree_make_status(IREE_STATUS_RESOURCE_EXHAUSTED,
+                         "dispatch requires %ub of local memory but only "
+                         "%zub is available per-worker",
+                         task->local_memory_size, local_memory.data_length));
+    IREE_TRACE_ZONE_END(z0);
+    return;
   }
   tile_context.local_memory =
       iree_make_byte_span(local_memory.data, task->local_memory_size);
 
+  iree_status_t status = iree_ok_status();
   const uint32_t base_x = task->workgroup_base[0];
   const uint32_t base_y = task->workgroup_base[1];
   const uint32_t base_z = task->workgroup_base[2];
@@ -721,19 +811,22 @@
         IREE_TRACE_ZONE_APPEND_VALUE(z_tile, z);
         // IREE_TRACE_ZONE_APPEND_VALUE(z_tile, (uint64_t)task->closure.fn);
 
-        iree_status_t status = task->closure.fn(
-            task->closure.user_context, &tile_context, pending_submission);
+        status = task->closure.fn(task->closure.user_context, &tile_context,
+                                  pending_submission);
 
         IREE_TRACE_ZONE_END(z_tile);
-        if (IREE_UNLIKELY(!iree_status_is_ok(status))) {
-          // NOTE: we don't bother to update statistics here on failure as the
-          // partial results won't really help much.
-          IREE_TRACE_ZONE_END(z0);
-          return status;
-        }
+
+        // If any tile fails we bail early from the loop. This doesn't match
+        // what an accelerator would do but saves some unneeded work.
+        // Note that other slices may have completed execution, be executing
+        // concurrently with this one, or still be pending - this does not
+        // have any influence on them and they may continue to execute even
+        // after we bail from here.
+        if (!iree_status_is_ok(status)) goto abort_slice;
       }
     }
   }
+abort_slice:
 
   // Push aggregate statistics up to the dispatch.
   if (task->dispatch_statistics) {
@@ -741,9 +834,13 @@
                                         task->dispatch_statistics);
   }
 
-  iree_task_retire(&task->header, pending_submission);
+  // Propagate failures to the dispatch task.
+  if (!iree_status_is_ok(status)) {
+    iree_task_try_set_status(task->dispatch_status, status);
+  }
+
+  iree_task_retire(&task->header, pending_submission, iree_ok_status());
   IREE_TRACE_ZONE_END(z0);
-  return iree_ok_status();
 }
 
 //==============================================================================
@@ -777,7 +874,7 @@
   return shard_task;
 }
 
-iree_status_t iree_task_dispatch_shard_execute(
+void iree_task_dispatch_shard_execute(
     iree_task_dispatch_shard_t* task, iree_byte_span_t local_memory,
     iree_task_submission_t* pending_submission) {
   IREE_TRACE_ZONE_BEGIN(z0);
@@ -801,11 +898,15 @@
   // inadvertently leak over into other executions.
   if (IREE_UNLIKELY(dispatch_task->local_memory_size >
                     local_memory.data_length)) {
-    return iree_make_status(IREE_STATUS_RESOURCE_EXHAUSTED,
-                            "dispatch requires %ub of local memory but only "
-                            "%zub is available per-worker",
-                            dispatch_task->local_memory_size,
-                            local_memory.data_length);
+    iree_task_retire(
+        &task->header, pending_submission,
+        iree_make_status(IREE_STATUS_RESOURCE_EXHAUSTED,
+                         "dispatch requires %ub of local memory but only "
+                         "%zub is available per-worker",
+                         dispatch_task->local_memory_size,
+                         local_memory.data_length));
+    IREE_TRACE_ZONE_END(z0);
+    return;
   }
   tile_context.local_memory =
       iree_make_byte_span(local_memory.data, dispatch_task->local_memory_size);
@@ -818,6 +919,7 @@
   tile_context.statistics = &shard_statistics;
 
   // Loop over all tiles until they are all processed.
+  iree_status_t status = iree_ok_status();
   const uint32_t tile_count = shared_state->tile_count;
   const uint32_t tiles_per_reservation = shared_state->tiles_per_reservation;
   uint32_t tile_base = iree_atomic_fetch_add_int32(&shared_state->tile_index,
@@ -848,29 +950,35 @@
       IREE_TRACE_ZONE_APPEND_VALUE(z_tile, tile_context.workgroup_xyz[2]);
       // IREE_TRACE_ZONE_APPEND_VALUE(z_tile, (uint64_t)task->closure.fn);
 
-      iree_status_t status =
-          dispatch_task->closure.fn(dispatch_task->closure.user_context,
-                                    &tile_context, pending_submission);
+      status = dispatch_task->closure.fn(dispatch_task->closure.user_context,
+                                         &tile_context, pending_submission);
 
       IREE_TRACE_ZONE_END(z_tile);
-      if (IREE_UNLIKELY(!iree_status_is_ok(status))) {
-        // NOTE: we don't bother to update statistics here on failure as the
-        // partial results won't really help much.
-        IREE_TRACE_ZONE_END(z0);
-        return status;
-      }
+
+      // If any tile fails we bail early from the loop. This doesn't match
+      // what an accelerator would do but saves some unneeded work.
+      // Note that other slices may have completed execution, be executing
+      // concurrently with this one, or still be pending - this does not
+      // have any influence on them and they may continue to execute even
+      // after we bail from here.
+      if (!iree_status_is_ok(status)) goto abort_shard;
     }
 
     tile_base = iree_atomic_fetch_add_int32(&shared_state->tile_index,
                                             tiles_per_reservation,
                                             iree_memory_order_relaxed);
   }
+abort_shard:
 
   // Push aggregate statistics up to the dispatch.
   iree_task_dispatch_statistics_merge(&shard_statistics,
                                       &dispatch_task->statistics);
 
-  iree_task_retire(&task->header, pending_submission);
+  // Propagate failures to the dispatch task.
+  if (!iree_status_is_ok(status)) {
+    iree_task_try_set_status(&dispatch_task->status, status);
+  }
+
+  iree_task_retire(&task->header, pending_submission, iree_ok_status());
   IREE_TRACE_ZONE_END(z0);
-  return iree_ok_status();
 }
diff --git a/iree/task/task.h b/iree/task/task.h
index b2593b6..a4a9626 100644
--- a/iree/task/task.h
+++ b/iree/task/task.h
@@ -112,16 +112,28 @@
   // behavior but without an additional task as dispatches are still required
   // to store information for slices.
   IREE_TASK_FLAG_DISPATCH_RETIRE = 1u << 3,
+
+  // An error occurred at or before the task and it has been aborted.
+  // Aborted tasks may continue to execute if they're already in-flight but must
+  // not begin execution after the flag has been set.
+  //
+  // The actual error that occurred is routed to the parent task scope as it
+  // happens and may be available for querying before all tasks have been
+  // cleaned up.
+  IREE_TASK_FLAG_ABORTED = 1u << 4,
 };
 typedef uint16_t iree_task_flags_t;
 
 typedef struct iree_task_t iree_task_t;
 
 // A function called to cleanup tasks.
-// The provided |status| is unowned and must be cloned if used beyond the scope
-// of the cleanup function (such as when stored for later usage).
-typedef void(IREE_API_PTR* iree_task_cleanup_fn_t)(iree_task_t* task,
-                                                   iree_status_t status);
+// Each task has its associated cleanup function called exactly once.
+// The provided |status_code| indicates the execution status of the task prior
+// to cleanup and will usually be IREE_STATUS_OK indicating the task was
+// successfully issued or IREE_STATUS_ABORTED if the task was discard prior to
+// issuing.
+typedef void(IREE_API_PTR* iree_task_cleanup_fn_t)(
+    iree_task_t* task, iree_status_code_t status_code);
 
 // A task within the task system that runs on an executor.
 // Tasks have an iree_task_type_t that defines which parameters are valid and
@@ -185,7 +197,9 @@
                           iree_task_t* out_task);
 
 // Sets the optional function called when the task completes (whether successful
-// or not).
+// or not). The cleanup function will receive a status indicating whether the
+// cleanup is from expected execution as the task retires (IREE_STATUS_OK)
+// or because it was aborted (IREE_STATUS_ABORTED).
 void iree_task_set_cleanup_fn(iree_task_t* task,
                               iree_task_cleanup_fn_t cleanup_fn);
 
@@ -268,6 +282,16 @@
 
   // Function closure to call when the task is executed.
   iree_task_call_closure_t closure;
+
+  // Resulting status from the call available once all nested tasks have
+  // completed (or would have completed). It's possible for a call to nest
+  // additional work under it and then return a failure; to ensure we don't
+  // discard the root call while the nested tasks are still executing we set the
+  // status here and wait for the nested tasks to complete. We'll try not to
+  // issue work that was enqueued while the call was executing but it's possible
+  // for work to come from other angles and we need to err on the side of
+  // safety.
+  iree_atomic_intptr_t status;
 } iree_task_call_t;
 
 void iree_task_call_initialize(iree_task_scope_t* scope,
@@ -535,6 +559,13 @@
   // dispatch closure.
   uint32_t local_memory_size;
 
+  // Resulting status from the dispatch available once all workgroups have
+  // completed (or would have completed). If multiple shards/slices processing
+  // the workgroups hit an error the first will be taken and the result ignored.
+  // A dispatch with a non-ok status will mark the parent task scope as failing
+  // when it retires.
+  iree_atomic_intptr_t status;
+
   // Statistics storage used for aggregating counters across all slices.
   iree_task_dispatch_statistics_t statistics;
 
@@ -590,6 +621,9 @@
   // This removes the need for touching the dispatch struct when beginning a
   // tile which would likely be a cache miss as we fan out to other cores.
 
+  // Status of the dispatch aggregating failues from all slices.
+  iree_atomic_intptr_t* dispatch_status;
+
   // Function closure to call per tile (same as the closure in the dispatch).
   iree_task_dispatch_closure_t closure;
 
diff --git a/iree/task/task_impl.h b/iree/task/task_impl.h
index f938d87..817f245 100644
--- a/iree/task/task_impl.h
+++ b/iree/task/task_impl.h
@@ -34,9 +34,10 @@
 // Executes and retires a user call.
 // May block the caller for an indeterminate amount of time and should only be
 // called from threads owned by or donated to the executor.
-// Returns the status of the user call.
-iree_status_t iree_task_call_execute(
-    iree_task_call_t* task, iree_task_submission_t* pending_submission);
+//
+// Errors are propagated to the parent scope.
+void iree_task_call_execute(iree_task_call_t* task,
+                            iree_task_submission_t* pending_submission);
 
 //==============================================================================
 // IREE_TASK_TYPE_BARRIER
@@ -128,7 +129,7 @@
 //
 // Returns ok if all tiles were successfully executed and otherwise returns
 // an unspecified status (probably the first non-ok status hit).
-iree_status_t iree_task_dispatch_slice_execute(
+void iree_task_dispatch_slice_execute(
     iree_task_dispatch_slice_t* task, iree_byte_span_t local_memory,
     iree_task_submission_t* pending_submission);
 
@@ -150,10 +151,9 @@
 // |local_memory| is a block of memory exclusively available to the shard
 // during execution. Contents are undefined both before and after execution.
 //
-// Returns ok if all tiles processed in the shard successfully executed and
-// otherwise returns an unspecified status (probably the first non-ok status
-// hit).
-iree_status_t iree_task_dispatch_shard_execute(
+// Errors are propagated to the parent scope and the dispatch will fail once
+// all shards have completed.
+void iree_task_dispatch_shard_execute(
     iree_task_dispatch_shard_t* task, iree_byte_span_t local_memory,
     iree_task_submission_t* pending_submission);
 
diff --git a/iree/task/task_test_call.cc b/iree/task/task_test_call.cc
index ac076bc..014c3ca 100644
--- a/iree/task/task_test_call.cc
+++ b/iree/task/task_test_call.cc
@@ -17,8 +17,13 @@
 
 namespace {
 
+using iree::Status;
+using iree::StatusCode;
+using iree::testing::status::StatusIs;
+
 class TaskCallTest : public TaskTest {};
 
+// Tests issuing a single call and waiting for it to complete.
 TEST_F(TaskCallTest, Issue) {
   struct TestCtx {
     int did_call = 0;
@@ -40,6 +45,123 @@
                             &task);
   IREE_ASSERT_OK(SubmitTasksAndWaitIdle(&task.header, &task.header));
   EXPECT_EQ(1, ctx.did_call);
+  IREE_EXPECT_OK(iree_task_scope_consume_status(&scope_));
+}
+
+// Tests issuing a single call that returns a failure.
+// The failure should be propagated back on the task scope.
+TEST_F(TaskCallTest, IssueFailure) {
+  struct TestCtx {
+    int did_call = 0;
+  };
+  TestCtx ctx;
+
+  // Call successfully issues but fails with some user error.
+  iree_task_call_t task;
+  iree_task_call_initialize(&scope_,
+                            iree_task_make_call_closure(
+                                [](void* user_context, iree_task_t* task,
+                                   iree_task_submission_t* pending_submission) {
+                                  auto* ctx = (TestCtx*)user_context;
+                                  EXPECT_TRUE(NULL != ctx);
+                                  EXPECT_EQ(0, ctx->did_call);
+                                  ++ctx->did_call;
+                                  return iree_make_status(
+                                      IREE_STATUS_UNAUTHENTICATED, "whoops!");
+                                },
+                                (void*)&ctx),
+                            &task);
+
+  // The task should still be cleaned up, even if it fails.
+  static int did_cleanup = 0;
+  did_cleanup = 0;
+  iree_task_set_cleanup_fn(
+      &task.header, +[](iree_task_t* task, iree_status_code_t status_code) {
+        EXPECT_EQ(status_code, IREE_STATUS_ABORTED);
+        ++did_cleanup;
+      });
+
+  IREE_ASSERT_OK(SubmitTasksAndWaitIdle(&task.header, &task.header));
+
+  // Expect both the call to have been made and the task cleaned up.
+  // The scope has the failure status.
+  EXPECT_EQ(1, ctx.did_call);
+  EXPECT_EQ(1, did_cleanup);
+  EXPECT_THAT(Status(iree_task_scope_consume_status(&scope_)),
+              StatusIs(StatusCode::kUnauthenticated));
+}
+
+// Tests issuing chained calls where the first fails.
+// The failure should be propagated back on the task scope and the chained call
+// should be aborted.
+TEST_F(TaskCallTest, IssueFailureChained) {
+  struct TestCtx {
+    int did_call_a = 0;
+    int did_call_b = 0;
+  };
+  TestCtx ctx;
+
+  // First call that will fail.
+  iree_task_call_t task_a;
+  iree_task_call_initialize(&scope_,
+                            iree_task_make_call_closure(
+                                [](void* user_context, iree_task_t* task,
+                                   iree_task_submission_t* pending_submission) {
+                                  auto* ctx = (TestCtx*)user_context;
+                                  EXPECT_TRUE(NULL != ctx);
+                                  EXPECT_EQ(0, ctx->did_call_a);
+                                  ++ctx->did_call_a;
+                                  // Force a failure.
+                                  return iree_make_status(
+                                      IREE_STATUS_UNAUTHENTICATED, "whoops!");
+                                },
+                                (void*)&ctx),
+                            &task_a);
+  static int did_cleanup_a = 0;
+  did_cleanup_a = 0;
+  iree_task_set_cleanup_fn(
+      &task_a.header, +[](iree_task_t* task, iree_status_code_t status_code) {
+        // Expect that the cleanup gets a signal indicating the task failed.
+        EXPECT_EQ(status_code, IREE_STATUS_ABORTED);
+        ++did_cleanup_a;
+      });
+
+  // Second call that will be aborted after the first fails.
+  iree_task_call_t task_b;
+  iree_task_call_initialize(&scope_,
+                            iree_task_make_call_closure(
+                                [](void* user_context, iree_task_t* task,
+                                   iree_task_submission_t* pending_submission) {
+                                  // This should never get called!
+                                  auto* ctx = (TestCtx*)user_context;
+                                  EXPECT_TRUE(NULL != ctx);
+                                  EXPECT_EQ(0, ctx->did_call_b);
+                                  ++ctx->did_call_b;
+                                  return iree_ok_status();
+                                },
+                                (void*)&ctx),
+                            &task_b);
+  static int did_cleanup_b = 0;
+  did_cleanup_b = 0;
+  iree_task_set_cleanup_fn(
+      &task_b.header, +[](iree_task_t* task, iree_status_code_t status_code) {
+        // Expect that the cleanup gets a signal indicating the task failed.
+        EXPECT_EQ(status_code, IREE_STATUS_ABORTED);
+        ++did_cleanup_b;
+      });
+
+  // A -> B
+  iree_task_set_completion_task(&task_a.header, &task_b.header);
+
+  IREE_ASSERT_OK(SubmitTasksAndWaitIdle(&task_a.header, &task_b.header));
+
+  // Expect that A was called but B was not, and both were cleaned up.
+  EXPECT_EQ(1, ctx.did_call_a);
+  EXPECT_EQ(1, did_cleanup_a);
+  EXPECT_EQ(0, ctx.did_call_b);
+  EXPECT_EQ(1, did_cleanup_b);
+  EXPECT_THAT(Status(iree_task_scope_consume_status(&scope_)),
+              StatusIs(StatusCode::kUnauthenticated));
 }
 
 // Issues task_a which then issues a nested task_b and waits for it to complete
@@ -100,6 +222,91 @@
   IREE_ASSERT_OK(SubmitTasksAndWaitIdle(&task_a.header, &task_a.header));
   EXPECT_EQ(2, ctx.did_call_a);
   EXPECT_EQ(1, ctx.did_call_b);
+  IREE_EXPECT_OK(iree_task_scope_consume_status(&scope_));
+}
+
+// Issues task_a which then issues a nested task_b and task_c; task_b fails and
+// it's expected that task_c completes before failing task_a.
+// Sibling tasks don't abort each other and as such we are guaranteed that C
+// will run: A -> [B fail, C ok] -> A fail
+TEST_F(TaskCallTest, IssueNestedFailure) {
+  struct TestCtx {
+    std::atomic<int> did_call_a = {0};
+    std::atomic<int> did_call_b = {0};
+    std::atomic<int> did_call_c = {0};
+    std::atomic<bool> has_issued = {false};
+    iree_task_call_t task_b;
+    iree_task_call_t task_c;
+  };
+  TestCtx ctx;
+
+  // task_a will get called only once due to the error: the pre-nesting call
+  // will schedule task_b/task_c and then the expected call after the tasks
+  // complete will not be made as task_b fails.
+  iree_task_call_t task_a;
+  iree_task_call_initialize(
+      &scope_,
+      iree_task_make_call_closure(
+          [](void* user_context, iree_task_t* task,
+             iree_task_submission_t* pending_submission) {
+            auto* ctx = (TestCtx*)user_context;
+            EXPECT_TRUE(NULL != ctx);
+
+            if (!ctx->has_issued) {
+              ctx->has_issued = true;
+              EXPECT_EQ(0, ctx->did_call_a);
+              ++ctx->did_call_a;
+
+              // task_b: (fails)
+              iree_task_call_initialize(
+                  task->scope,
+                  iree_task_make_call_closure(
+                      [](void* user_context, iree_task_t* task,
+                         iree_task_submission_t* pending_submission) {
+                        auto* ctx = (TestCtx*)user_context;
+                        EXPECT_TRUE(NULL != ctx);
+                        EXPECT_EQ(0, ctx->did_call_b);
+                        ++ctx->did_call_b;
+                        return iree_make_status(IREE_STATUS_DATA_LOSS, "uh oh");
+                      },
+                      user_context),
+                  &ctx->task_b);
+              iree_task_set_completion_task(&ctx->task_b.header, task);
+              iree_task_submission_enqueue(pending_submission,
+                                           &ctx->task_b.header);
+
+              // task_c: (ok)
+              iree_task_call_initialize(
+                  task->scope,
+                  iree_task_make_call_closure(
+                      [](void* user_context, iree_task_t* task,
+                         iree_task_submission_t* pending_submission) {
+                        auto* ctx = (TestCtx*)user_context;
+                        EXPECT_TRUE(NULL != ctx);
+                        EXPECT_EQ(0, ctx->did_call_c);
+                        ++ctx->did_call_c;
+                        return iree_ok_status();
+                      },
+                      user_context),
+                  &ctx->task_c);
+              iree_task_set_completion_task(&ctx->task_c.header, task);
+              iree_task_submission_enqueue(pending_submission,
+                                           &ctx->task_c.header);
+            } else {
+              EXPECT_EQ(1, ctx->did_call_a);
+              ++ctx->did_call_a;
+            }
+
+            return iree_ok_status();
+          },
+          (void*)&ctx),
+      &task_a);
+  IREE_ASSERT_OK(SubmitTasksAndWaitIdle(&task_a.header, &task_a.header));
+  EXPECT_EQ(1, ctx.did_call_a);
+  EXPECT_EQ(1, ctx.did_call_b);
+  EXPECT_EQ(1, ctx.did_call_c);
+  EXPECT_THAT(Status(iree_task_scope_consume_status(&scope_)),
+              StatusIs(StatusCode::kDataLoss));
 }
 
 }  // namespace
diff --git a/iree/task/task_test_dispatch.cc b/iree/task/task_test_dispatch.cc
index cd515f6..74e51c0 100644
--- a/iree/task/task_test_dispatch.cc
+++ b/iree/task/task_test_dispatch.cc
@@ -17,6 +17,10 @@
 
 namespace {
 
+using iree::Status;
+using iree::StatusCode;
+using iree::testing::status::StatusIs;
+
 class GridCoverage {
  public:
   explicit GridCoverage(const uint32_t workgroup_count[3])
@@ -165,4 +169,63 @@
   EXPECT_TRUE(coverage.Verify());
 }
 
+TEST_F(TaskDispatchTest, IssueFailure) {
+  const uint32_t kWorkgroupSize[3] = {1, 1, 1};
+  const uint32_t kWorkgroupCount[3] = {64, 1, 1};
+
+  auto tile = [](void* user_context,
+                 const iree_task_tile_context_t* tile_context,
+                 iree_task_submission_t* pending_submission) -> iree_status_t {
+    return tile_context->workgroup_xyz[0] == 32
+               ? iree_make_status(IREE_STATUS_DATA_LOSS, "whoops!")
+               : iree_ok_status();
+  };
+
+  iree_task_dispatch_t task;
+  iree_task_dispatch_initialize(&scope_,
+                                iree_task_make_dispatch_closure(tile, NULL),
+                                kWorkgroupSize, kWorkgroupCount, &task);
+  IREE_ASSERT_OK(SubmitTasksAndWaitIdle(&task.header, &task.header));
+  EXPECT_THAT(Status(iree_task_scope_consume_status(&scope_)),
+              StatusIs(StatusCode::kDataLoss));
+}
+
+TEST_F(TaskDispatchTest, IssueFailureChained) {
+  const uint32_t kWorkgroupSize[3] = {1, 1, 1};
+  const uint32_t kWorkgroupCount[3] = {64, 1, 1};
+
+  auto tile = [](void* user_context,
+                 const iree_task_tile_context_t* tile_context,
+                 iree_task_submission_t* pending_submission) -> iree_status_t {
+    return tile_context->workgroup_xyz[0] == 32
+               ? iree_make_status(IREE_STATUS_DATA_LOSS, "whoops!")
+               : iree_ok_status();
+  };
+
+  iree_task_dispatch_t dispatch_task;
+  iree_task_dispatch_initialize(
+      &scope_, iree_task_make_dispatch_closure(tile, NULL), kWorkgroupSize,
+      kWorkgroupCount, &dispatch_task);
+
+  int did_call = 0;
+  iree_task_call_t call_task;
+  iree_task_call_initialize(&scope_,
+                            iree_task_make_call_closure(
+                                [](void* user_context, iree_task_t* task,
+                                   iree_task_submission_t* pending_submission) {
+                                  int* did_call_ptr = (int*)user_context;
+                                  ++(*did_call_ptr);
+                                  return iree_ok_status();
+                                },
+                                &did_call),
+                            &call_task);
+  iree_task_set_completion_task(&dispatch_task.header, &call_task.header);
+
+  IREE_ASSERT_OK(
+      SubmitTasksAndWaitIdle(&dispatch_task.header, &call_task.header));
+  EXPECT_EQ(0, did_call);
+  EXPECT_THAT(Status(iree_task_scope_consume_status(&scope_)),
+              StatusIs(StatusCode::kDataLoss));
+}
+
 }  // namespace
diff --git a/iree/task/worker.c b/iree/task/worker.c
index 3f22467..ca5b2d9 100644
--- a/iree/task/worker.c
+++ b/iree/task/worker.c
@@ -167,7 +167,7 @@
 // Executes a task on a worker.
 // Only task types that are scheduled to workers are handled; all others must be
 // handled by the coordinator during scheduling.
-static iree_status_t iree_task_worker_execute(
+static void iree_task_worker_execute(
     iree_task_worker_t* worker, iree_task_t* task,
     iree_task_submission_t* pending_submission) {
   // Execute the task and resolve the task and gather any tasks that are now
@@ -180,31 +180,28 @@
   // TODO(benvanik): handle partial tasks and re-queuing.
   switch (task->type) {
     case IREE_TASK_TYPE_CALL: {
-      IREE_RETURN_IF_ERROR(
-          iree_task_call_execute((iree_task_call_t*)task, pending_submission));
+      iree_task_call_execute((iree_task_call_t*)task, pending_submission);
       break;
     }
     case IREE_TASK_TYPE_DISPATCH_SLICE: {
-      IREE_RETURN_IF_ERROR(iree_task_dispatch_slice_execute(
-          (iree_task_dispatch_slice_t*)task, worker->local_memory,
-          pending_submission));
+      iree_task_dispatch_slice_execute((iree_task_dispatch_slice_t*)task,
+                                       worker->local_memory,
+                                       pending_submission);
       break;
     }
     case IREE_TASK_TYPE_DISPATCH_SHARD: {
-      IREE_RETURN_IF_ERROR(iree_task_dispatch_shard_execute(
-          (iree_task_dispatch_shard_t*)task, worker->local_memory,
-          pending_submission));
+      iree_task_dispatch_shard_execute((iree_task_dispatch_shard_t*)task,
+                                       worker->local_memory,
+                                       pending_submission);
       break;
     }
     default:
-      return iree_make_status(IREE_STATUS_INVALID_ARGUMENT,
-                              "incorrect task type for worker execution");
+      IREE_ASSERT_UNREACHABLE("incorrect task type for worker execution");
+      break;
   }
 
-  // NOTE: task is invalidated here!
+  // NOTE: task is invalidated above and must not be used!
   task = NULL;
-
-  return iree_ok_status();
 }
 
 // Pumps the worker thread once, processing a single task.
@@ -252,22 +249,7 @@
 
   // Execute the task (may call out to arbitrary user code and may submit more
   // tasks for execution).
-  iree_status_t status =
-      iree_task_worker_execute(worker, task, pending_submission);
-
-  // TODO(#4026): propagate failure to task scope.
-  // We currently drop the error on the floor here; that's because the error
-  // should have already been propagated to the scope and everyone should be
-  // checking that before running things anyway.
-  //
-  // Since we can host work from multiple scopes and want to ensure an error
-  // in one doesn't bring down the whole system we pretend we executed
-  // something here by falling through.
-  if (!iree_status_is_ok(status)) {
-    iree_status_fprint(stderr, status);
-  }
-  IREE_ASSERT_TRUE(iree_status_is_ok(status));
-  iree_status_ignore(status);
+  iree_task_worker_execute(worker, task, pending_submission);
 
   IREE_TRACE_ZONE_END(z0);
   return true;  // try again