Implement full canonical Burst in NN util code

Bug: 180492058
Bug: 177267324
Test: mma
Test: presubmit
Change-Id: I5018f6cf2dbaf705f74f4f46318142c64433e19d
Merged-In: I5018f6cf2dbaf705f74f4f46318142c64433e19d
(cherry picked from commit acff4063b63c04cbb28af87eab61e9a1fa70980a)
diff --git a/neuralnetworks/1.2/utils/src/Conversions.cpp b/neuralnetworks/1.2/utils/src/Conversions.cpp
index 86a417a..2c45583 100644
--- a/neuralnetworks/1.2/utils/src/Conversions.cpp
+++ b/neuralnetworks/1.2/utils/src/Conversions.cpp
@@ -331,6 +331,10 @@
     return validatedConvert(timing);
 }
 
+GeneralResult<SharedMemory> convert(const hardware::hidl_memory& memory) {
+    return validatedConvert(memory);
+}
+
 GeneralResult<std::vector<Extension>> convert(const hidl_vec<hal::V1_2::Extension>& extensions) {
     return validatedConvert(extensions);
 }
diff --git a/neuralnetworks/1.2/utils/src/ExecutionBurstController.cpp b/neuralnetworks/1.2/utils/src/ExecutionBurstController.cpp
index 2265861..eedf591 100644
--- a/neuralnetworks/1.2/utils/src/ExecutionBurstController.cpp
+++ b/neuralnetworks/1.2/utils/src/ExecutionBurstController.cpp
@@ -17,283 +17,321 @@
 #define LOG_TAG "ExecutionBurstController"
 
 #include "ExecutionBurstController.h"
+#include "ExecutionBurstUtils.h"
 
 #include <android-base/logging.h>
+#include <android-base/thread_annotations.h>
+#include <nnapi/IBurst.h>
+#include <nnapi/IPreparedModel.h>
+#include <nnapi/Result.h>
+#include <nnapi/TypeUtils.h>
+#include <nnapi/Types.h>
+#include <nnapi/Validation.h>
+#include <nnapi/hal/1.0/Conversions.h>
+#include <nnapi/hal/HandleError.h>
+#include <nnapi/hal/ProtectCallback.h>
+#include <nnapi/hal/TransferValue.h>
 
 #include <algorithm>
 #include <cstring>
 #include <limits>
 #include <memory>
 #include <string>
+#include <thread>
 #include <tuple>
 #include <utility>
 #include <vector>
 
-#include "ExecutionBurstUtils.h"
-#include "HalInterfaces.h"
+#include "Callbacks.h"
+#include "Conversions.h"
 #include "Tracing.h"
 #include "Utils.h"
 
-namespace android::nn {
+namespace android::hardware::neuralnetworks::V1_2::utils {
 namespace {
 
-class BurstContextDeathHandler : public hardware::hidl_death_recipient {
-  public:
-    using Callback = std::function<void()>;
-
-    BurstContextDeathHandler(const Callback& onDeathCallback) : mOnDeathCallback(onDeathCallback) {
-        CHECK(onDeathCallback != nullptr);
+nn::GeneralResult<sp<IBurstContext>> executionBurstResultCallback(
+        V1_0::ErrorStatus status, const sp<IBurstContext>& burstContext) {
+    HANDLE_HAL_STATUS(status) << "IPreparedModel::configureExecutionBurst failed with status "
+                              << toString(status);
+    if (burstContext == nullptr) {
+        return NN_ERROR(nn::ErrorStatus::GENERAL_FAILURE)
+               << "IPreparedModel::configureExecutionBurst returned nullptr for burst";
     }
-
-    void serviceDied(uint64_t /*cookie*/, const wp<hidl::base::V1_0::IBase>& /*who*/) override {
-        LOG(ERROR) << "BurstContextDeathHandler::serviceDied -- service unexpectedly died!";
-        mOnDeathCallback();
-    }
-
-  private:
-    const Callback mOnDeathCallback;
-};
-
-}  // anonymous namespace
-
-hardware::Return<void> ExecutionBurstController::ExecutionBurstCallback::getMemories(
-        const hardware::hidl_vec<int32_t>& slots, getMemories_cb cb) {
-    std::lock_guard<std::mutex> guard(mMutex);
-
-    // get all memories
-    hardware::hidl_vec<hardware::hidl_memory> memories(slots.size());
-    std::transform(slots.begin(), slots.end(), memories.begin(), [this](int32_t slot) {
-        return slot < mMemoryCache.size() ? mMemoryCache[slot] : hardware::hidl_memory{};
-    });
-
-    // ensure all memories are valid
-    if (!std::all_of(memories.begin(), memories.end(),
-                     [](const hardware::hidl_memory& memory) { return memory.valid(); })) {
-        cb(V1_0::ErrorStatus::INVALID_ARGUMENT, {});
-        return hardware::Void();
-    }
-
-    // return successful
-    cb(V1_0::ErrorStatus::NONE, std::move(memories));
-    return hardware::Void();
+    return burstContext;
 }
 
-std::vector<int32_t> ExecutionBurstController::ExecutionBurstCallback::getSlots(
-        const hardware::hidl_vec<hardware::hidl_memory>& memories,
-        const std::vector<intptr_t>& keys) {
-    std::lock_guard<std::mutex> guard(mMutex);
-
-    // retrieve (or bind) all slots corresponding to memories
-    std::vector<int32_t> slots;
-    slots.reserve(memories.size());
-    for (size_t i = 0; i < memories.size(); ++i) {
-        slots.push_back(getSlotLocked(memories[i], keys[i]));
+nn::GeneralResult<hidl_vec<hidl_memory>> getMemoriesHelper(
+        const hidl_vec<int32_t>& slots,
+        const std::shared_ptr<ExecutionBurstController::MemoryCache>& memoryCache) {
+    hidl_vec<hidl_memory> memories(slots.size());
+    for (size_t i = 0; i < slots.size(); ++i) {
+        const int32_t slot = slots[i];
+        const auto memory = NN_TRY(memoryCache->getMemory(slot));
+        memories[i] = NN_TRY(V1_0::utils::unvalidatedConvert(memory));
+        if (!memories[i].valid()) {
+            return NN_ERROR() << "memory at slot " << slot << " is invalid";
+        }
     }
-    return slots;
+    return memories;
 }
 
-std::pair<bool, int32_t> ExecutionBurstController::ExecutionBurstCallback::freeMemory(
-        intptr_t key) {
-    std::lock_guard<std::mutex> guard(mMutex);
+}  // namespace
 
-    auto iter = mMemoryIdToSlot.find(key);
-    if (iter == mMemoryIdToSlot.end()) {
-        return {false, 0};
-    }
-    const int32_t slot = iter->second;
-    mMemoryIdToSlot.erase(key);
-    mMemoryCache[slot] = {};
-    mFreeSlots.push(slot);
-    return {true, slot};
+// MemoryCache methods
+
+ExecutionBurstController::MemoryCache::MemoryCache() {
+    constexpr size_t kPreallocatedCount = 1024;
+    std::vector<int32_t> freeSlotsSpace;
+    freeSlotsSpace.reserve(kPreallocatedCount);
+    mFreeSlots = std::stack<int32_t, std::vector<int32_t>>(std::move(freeSlotsSpace));
+    mMemoryCache.reserve(kPreallocatedCount);
+    mCacheCleaner.reserve(kPreallocatedCount);
 }
 
-int32_t ExecutionBurstController::ExecutionBurstCallback::getSlotLocked(
-        const hardware::hidl_memory& memory, intptr_t key) {
-    auto iter = mMemoryIdToSlot.find(key);
-    if (iter == mMemoryIdToSlot.end()) {
-        const int32_t slot = allocateSlotLocked();
-        mMemoryIdToSlot[key] = slot;
-        mMemoryCache[slot] = memory;
-        return slot;
-    } else {
+void ExecutionBurstController::MemoryCache::setBurstContext(sp<IBurstContext> burstContext) {
+    std::lock_guard guard(mMutex);
+    mBurstContext = std::move(burstContext);
+}
+
+std::pair<int32_t, ExecutionBurstController::MemoryCache::SharedCleanup>
+ExecutionBurstController::MemoryCache::cacheMemory(const nn::SharedMemory& memory) {
+    std::unique_lock lock(mMutex);
+    base::ScopedLockAssertion lockAssert(mMutex);
+
+    // Use existing cache entry if (1) the Memory object is in the cache and (2) the cache entry is
+    // not currently being freed.
+    auto iter = mMemoryIdToSlot.find(memory);
+    while (iter != mMemoryIdToSlot.end()) {
         const int32_t slot = iter->second;
-        return slot;
+        if (auto cleaner = mCacheCleaner.at(slot).lock()) {
+            return std::make_pair(slot, std::move(cleaner));
+        }
+
+        // If the code reaches this point, the Memory object was in the cache, but is currently
+        // being destroyed. This code waits until the cache entry has been freed, then loops to
+        // ensure the cache entry has been freed or has been made present by another thread.
+        mCond.wait(lock);
+        iter = mMemoryIdToSlot.find(memory);
     }
+
+    // Allocate a new cache entry.
+    const int32_t slot = allocateSlotLocked();
+    mMemoryIdToSlot[memory] = slot;
+    mMemoryCache[slot] = memory;
+
+    // Create reference-counted self-cleaning cache object.
+    auto self = weak_from_this();
+    Task cleanup = [memory, memoryCache = std::move(self)] {
+        if (const auto lock = memoryCache.lock()) {
+            lock->freeMemory(memory);
+        }
+    };
+    auto cleaner = std::make_shared<const Cleanup>(std::move(cleanup));
+    mCacheCleaner[slot] = cleaner;
+
+    return std::make_pair(slot, std::move(cleaner));
 }
 
-int32_t ExecutionBurstController::ExecutionBurstCallback::allocateSlotLocked() {
+nn::GeneralResult<nn::SharedMemory> ExecutionBurstController::MemoryCache::getMemory(int32_t slot) {
+    std::lock_guard guard(mMutex);
+    if (slot < 0 || static_cast<size_t>(slot) >= mMemoryCache.size()) {
+        return NN_ERROR() << "Invalid slot: " << slot << " vs " << mMemoryCache.size();
+    }
+    return mMemoryCache[slot];
+}
+
+void ExecutionBurstController::MemoryCache::freeMemory(const nn::SharedMemory& memory) {
+    {
+        std::lock_guard guard(mMutex);
+        const int32_t slot = mMemoryIdToSlot.at(memory);
+        if (mBurstContext) {
+            mBurstContext->freeMemory(slot);
+        }
+        mMemoryIdToSlot.erase(memory);
+        mMemoryCache[slot] = {};
+        mCacheCleaner[slot].reset();
+        mFreeSlots.push(slot);
+    }
+    mCond.notify_all();
+}
+
+int32_t ExecutionBurstController::MemoryCache::allocateSlotLocked() {
     constexpr size_t kMaxNumberOfSlots = std::numeric_limits<int32_t>::max();
 
-    // if there is a free slot, use it
-    if (mFreeSlots.size() > 0) {
+    // If there is a free slot, use it.
+    if (!mFreeSlots.empty()) {
         const int32_t slot = mFreeSlots.top();
         mFreeSlots.pop();
         return slot;
     }
 
-    // otherwise use a slot for the first time
-    CHECK(mMemoryCache.size() < kMaxNumberOfSlots) << "Exceeded maximum number of slots!";
+    // Use a slot for the first time.
+    CHECK_LT(mMemoryCache.size(), kMaxNumberOfSlots) << "Exceeded maximum number of slots!";
     const int32_t slot = static_cast<int32_t>(mMemoryCache.size());
     mMemoryCache.emplace_back();
+    mCacheCleaner.emplace_back();
 
     return slot;
 }
 
-std::unique_ptr<ExecutionBurstController> ExecutionBurstController::create(
-        const sp<V1_2::IPreparedModel>& preparedModel,
+// ExecutionBurstCallback methods
+
+ExecutionBurstController::ExecutionBurstCallback::ExecutionBurstCallback(
+        const std::shared_ptr<MemoryCache>& memoryCache)
+    : kMemoryCache(memoryCache) {
+    CHECK(memoryCache != nullptr);
+}
+
+Return<void> ExecutionBurstController::ExecutionBurstCallback::getMemories(
+        const hidl_vec<int32_t>& slots, getMemories_cb cb) {
+    const auto memoryCache = kMemoryCache.lock();
+    if (memoryCache == nullptr) {
+        LOG(ERROR) << "ExecutionBurstController::ExecutionBurstCallback::getMemories called after "
+                      "the MemoryCache has been freed";
+        cb(V1_0::ErrorStatus::GENERAL_FAILURE, {});
+        return Void();
+    }
+
+    const auto maybeMemories = getMemoriesHelper(slots, memoryCache);
+    if (!maybeMemories.has_value()) {
+        const auto& [message, code] = maybeMemories.error();
+        LOG(ERROR) << "ExecutionBurstController::ExecutionBurstCallback::getMemories failed with "
+                   << code << ": " << message;
+        cb(V1_0::ErrorStatus::INVALID_ARGUMENT, {});
+        return Void();
+    }
+
+    cb(V1_0::ErrorStatus::NONE, maybeMemories.value());
+    return Void();
+}
+
+// ExecutionBurstController methods
+
+nn::GeneralResult<std::shared_ptr<const ExecutionBurstController>> ExecutionBurstController::create(
+        const sp<V1_2::IPreparedModel>& preparedModel, FallbackFunction fallback,
         std::chrono::microseconds pollingTimeWindow) {
     // check inputs
     if (preparedModel == nullptr) {
-        LOG(ERROR) << "ExecutionBurstController::create passed a nullptr";
-        return nullptr;
+        return NN_ERROR() << "ExecutionBurstController::create passed a nullptr";
     }
 
-    // create callback object
-    sp<ExecutionBurstCallback> callback = new ExecutionBurstCallback();
-
     // create FMQ objects
-    auto [requestChannelSenderTemp, requestChannelDescriptor] =
-            RequestChannelSender::create(kExecutionBurstChannelLength);
-    auto [resultChannelReceiverTemp, resultChannelDescriptor] =
-            ResultChannelReceiver::create(kExecutionBurstChannelLength, pollingTimeWindow);
-    std::shared_ptr<RequestChannelSender> requestChannelSender =
-            std::move(requestChannelSenderTemp);
-    std::shared_ptr<ResultChannelReceiver> resultChannelReceiver =
-            std::move(resultChannelReceiverTemp);
+    auto [requestChannelSender, requestChannelDescriptor] =
+            NN_TRY(RequestChannelSender::create(kExecutionBurstChannelLength));
+    auto [resultChannelReceiver, resultChannelDescriptor] =
+            NN_TRY(ResultChannelReceiver::create(kExecutionBurstChannelLength, pollingTimeWindow));
 
     // check FMQ objects
-    if (!requestChannelSender || !resultChannelReceiver || !requestChannelDescriptor ||
-        !resultChannelDescriptor) {
-        LOG(ERROR) << "ExecutionBurstController::create failed to create FastMessageQueue";
-        return nullptr;
-    }
+    CHECK(requestChannelSender != nullptr);
+    CHECK(requestChannelDescriptor != nullptr);
+    CHECK(resultChannelReceiver != nullptr);
+    CHECK(resultChannelDescriptor != nullptr);
+
+    // create memory cache
+    auto memoryCache = std::make_shared<MemoryCache>();
+
+    // create callback object
+    auto burstCallback = sp<ExecutionBurstCallback>::make(memoryCache);
+    auto cb = hal::utils::CallbackValue(executionBurstResultCallback);
 
     // configure burst
-    V1_0::ErrorStatus errorStatus;
-    sp<IBurstContext> burstContext;
-    const hardware::Return<void> ret = preparedModel->configureExecutionBurst(
-            callback, *requestChannelDescriptor, *resultChannelDescriptor,
-            [&errorStatus, &burstContext](V1_0::ErrorStatus status,
-                                          const sp<IBurstContext>& context) {
-                errorStatus = status;
-                burstContext = context;
-            });
+    const Return<void> ret = preparedModel->configureExecutionBurst(
+            burstCallback, *requestChannelDescriptor, *resultChannelDescriptor, cb);
+    HANDLE_TRANSPORT_FAILURE(ret);
 
-    // check burst
-    if (!ret.isOk()) {
-        LOG(ERROR) << "IPreparedModel::configureExecutionBurst failed with description "
-                   << ret.description();
-        return nullptr;
-    }
-    if (errorStatus != V1_0::ErrorStatus::NONE) {
-        LOG(ERROR) << "IPreparedModel::configureExecutionBurst failed with status "
-                   << toString(errorStatus);
-        return nullptr;
-    }
-    if (burstContext == nullptr) {
-        LOG(ERROR) << "IPreparedModel::configureExecutionBurst returned nullptr for burst";
-        return nullptr;
-    }
+    auto burstContext = NN_TRY(cb.take());
+    memoryCache->setBurstContext(burstContext);
 
     // create death handler object
-    BurstContextDeathHandler::Callback onDeathCallback = [requestChannelSender,
-                                                          resultChannelReceiver] {
-        requestChannelSender->invalidate();
-        resultChannelReceiver->invalidate();
-    };
-    const sp<BurstContextDeathHandler> deathHandler = new BurstContextDeathHandler(onDeathCallback);
-
-    // linkToDeath registers a callback that will be invoked on service death to
-    // proactively handle service crashes. If the linkToDeath call fails,
-    // asynchronous calls are susceptible to hangs if the service crashes before
-    // providing the response.
-    const hardware::Return<bool> deathHandlerRet = burstContext->linkToDeath(deathHandler, 0);
-    if (!deathHandlerRet.isOk() || deathHandlerRet != true) {
-        LOG(ERROR) << "ExecutionBurstController::create -- Failed to register a death recipient "
-                      "for the IBurstContext object.";
-        return nullptr;
-    }
+    auto deathHandler = NN_TRY(neuralnetworks::utils::DeathHandler::create(burstContext));
+    deathHandler.protectCallbackForLifetimeOfDeathHandler(requestChannelSender.get());
+    deathHandler.protectCallbackForLifetimeOfDeathHandler(resultChannelReceiver.get());
 
     // make and return controller
-    return std::make_unique<ExecutionBurstController>(requestChannelSender, resultChannelReceiver,
-                                                      burstContext, callback, deathHandler);
+    return std::make_shared<const ExecutionBurstController>(
+            PrivateConstructorTag{}, std::move(fallback), std::move(requestChannelSender),
+            std::move(resultChannelReceiver), std::move(burstCallback), std::move(burstContext),
+            std::move(memoryCache), std::move(deathHandler));
 }
 
 ExecutionBurstController::ExecutionBurstController(
-        const std::shared_ptr<RequestChannelSender>& requestChannelSender,
-        const std::shared_ptr<ResultChannelReceiver>& resultChannelReceiver,
-        const sp<IBurstContext>& burstContext, const sp<ExecutionBurstCallback>& callback,
-        const sp<hardware::hidl_death_recipient>& deathHandler)
-    : mRequestChannelSender(requestChannelSender),
-      mResultChannelReceiver(resultChannelReceiver),
-      mBurstContext(burstContext),
-      mMemoryCache(callback),
-      mDeathHandler(deathHandler) {}
+        PrivateConstructorTag /*tag*/, FallbackFunction fallback,
+        std::unique_ptr<RequestChannelSender> requestChannelSender,
+        std::unique_ptr<ResultChannelReceiver> resultChannelReceiver,
+        sp<ExecutionBurstCallback> callback, sp<IBurstContext> burstContext,
+        std::shared_ptr<MemoryCache> memoryCache, neuralnetworks::utils::DeathHandler deathHandler)
+    : kFallback(std::move(fallback)),
+      mRequestChannelSender(std::move(requestChannelSender)),
+      mResultChannelReceiver(std::move(resultChannelReceiver)),
+      mBurstCallback(std::move(callback)),
+      mBurstContext(std::move(burstContext)),
+      mMemoryCache(std::move(memoryCache)),
+      kDeathHandler(std::move(deathHandler)) {}
 
-ExecutionBurstController::~ExecutionBurstController() {
-    // It is safe to ignore any errors resulting from this unlinkToDeath call
-    // because the ExecutionBurstController object is already being destroyed
-    // and its underlying IBurstContext object is no longer being used by the NN
-    // runtime.
-    if (mDeathHandler) {
-        mBurstContext->unlinkToDeath(mDeathHandler).isOk();
+ExecutionBurstController::OptionalCacheHold ExecutionBurstController::cacheMemory(
+        const nn::SharedMemory& memory) const {
+    auto [slot, hold] = mMemoryCache->cacheMemory(memory);
+    return hold;
+}
+
+nn::ExecutionResult<std::pair<std::vector<nn::OutputShape>, nn::Timing>>
+ExecutionBurstController::execute(const nn::Request& request, nn::MeasureTiming measure) const {
+    // This is the first point when we know an execution is occurring, so begin to collect
+    // systraces. Note that the first point we can begin collecting systraces in
+    // ExecutionBurstServer is when the RequestChannelReceiver realizes there is data in the FMQ, so
+    // ExecutionBurstServer collects systraces at different points in the code.
+    NNTRACE_FULL(NNTRACE_LAYER_IPC, NNTRACE_PHASE_EXECUTION, "ExecutionBurstController::execute");
+
+    // if the request is valid but of a higher version than what's supported in burst execution,
+    // fall back to another execution path
+    if (const auto version = NN_TRY(hal::utils::makeExecutionFailure(nn::validate(request)));
+        version > nn::Version::ANDROID_Q) {
+        // fallback to another execution path if the packet could not be sent
+        if (kFallback) {
+            return kFallback(request, measure);
+        }
+        return NN_ERROR() << "Request object has features not supported by IBurst::execute";
     }
-}
 
-static std::tuple<int, std::vector<V1_2::OutputShape>, V1_2::Timing, bool> getExecutionResult(
-        V1_0::ErrorStatus status, std::vector<V1_2::OutputShape> outputShapes, V1_2::Timing timing,
-        bool fallback) {
-    auto [n, checkedOutputShapes, checkedTiming] =
-            getExecutionResult(convertToV1_3(status), std::move(outputShapes), timing);
-    return {n, convertToV1_2(checkedOutputShapes), convertToV1_2(checkedTiming), fallback};
-}
+    // clear pools field of request, as they will be provided via slots
+    const auto requestWithoutPools =
+            nn::Request{.inputs = request.inputs, .outputs = request.outputs, .pools = {}};
+    auto hidlRequest = NN_TRY(
+            hal::utils::makeExecutionFailure(V1_0::utils::unvalidatedConvert(requestWithoutPools)));
+    const auto hidlMeasure = NN_TRY(hal::utils::makeExecutionFailure(convert(measure)));
 
-std::tuple<int, std::vector<V1_2::OutputShape>, V1_2::Timing, bool>
-ExecutionBurstController::compute(const V1_0::Request& request, V1_2::MeasureTiming measure,
-                                  const std::vector<intptr_t>& memoryIds) {
-    // This is the first point when we know an execution is occurring, so begin
-    // to collect systraces. Note that the first point we can begin collecting
-    // systraces in ExecutionBurstServer is when the RequestChannelReceiver
-    // realizes there is data in the FMQ, so ExecutionBurstServer collects
-    // systraces at different points in the code.
-    NNTRACE_FULL(NNTRACE_LAYER_IPC, NNTRACE_PHASE_EXECUTION, "ExecutionBurstController::compute");
+    // Ensure that at most one execution is in flight at any given time.
+    const bool alreadyInFlight = mExecutionInFlight.test_and_set();
+    if (alreadyInFlight) {
+        return NN_ERROR() << "IBurst already has an execution in flight";
+    }
+    const auto guard = base::make_scope_guard([this] { mExecutionInFlight.clear(); });
 
-    std::lock_guard<std::mutex> guard(mMutex);
+    std::vector<int32_t> slots;
+    std::vector<OptionalCacheHold> holds;
+    slots.reserve(request.pools.size());
+    holds.reserve(request.pools.size());
+    for (const auto& memoryPool : request.pools) {
+        auto [slot, hold] = mMemoryCache->cacheMemory(std::get<nn::SharedMemory>(memoryPool));
+        slots.push_back(slot);
+        holds.push_back(std::move(hold));
+    }
 
     // send request packet
-    const std::vector<int32_t> slots = mMemoryCache->getSlots(request.pools, memoryIds);
-    const bool success = mRequestChannelSender->send(request, measure, slots);
-    if (!success) {
-        LOG(ERROR) << "Error sending FMQ packet";
-        // only use fallback execution path if the packet could not be sent
-        return getExecutionResult(V1_0::ErrorStatus::GENERAL_FAILURE, {}, kNoTiming12,
-                                  /*fallback=*/true);
+    const auto sendStatus = mRequestChannelSender->send(hidlRequest, hidlMeasure, slots);
+    if (!sendStatus.ok()) {
+        // fallback to another execution path if the packet could not be sent
+        if (kFallback) {
+            return kFallback(request, measure);
+        }
+        return NN_ERROR() << "Error sending FMQ packet: " << sendStatus.error();
     }
 
     // get result packet
-    const auto result = mResultChannelReceiver->getBlocking();
-    if (!result) {
-        LOG(ERROR) << "Error retrieving FMQ packet";
-        // only use fallback execution path if the packet could not be sent
-        return getExecutionResult(V1_0::ErrorStatus::GENERAL_FAILURE, {}, kNoTiming12,
-                                  /*fallback=*/false);
-    }
-
-    // unpack results and return (only use fallback execution path if the
-    // packet could not be sent)
-    auto [status, outputShapes, timing] = std::move(*result);
-    return getExecutionResult(status, std::move(outputShapes), timing, /*fallback=*/false);
+    const auto [status, outputShapes, timing] =
+            NN_TRY(hal::utils::makeExecutionFailure(mResultChannelReceiver->getBlocking()));
+    return executionCallback(status, outputShapes, timing);
 }
 
-void ExecutionBurstController::freeMemory(intptr_t key) {
-    std::lock_guard<std::mutex> guard(mMutex);
-
-    bool valid;
-    int32_t slot;
-    std::tie(valid, slot) = mMemoryCache->freeMemory(key);
-    if (valid) {
-        mBurstContext->freeMemory(slot).isOk();
-    }
-}
-
-}  // namespace android::nn
+}  // namespace android::hardware::neuralnetworks::V1_2::utils
diff --git a/neuralnetworks/1.2/utils/src/ExecutionBurstServer.cpp b/neuralnetworks/1.2/utils/src/ExecutionBurstServer.cpp
index 022548d..50af881 100644
--- a/neuralnetworks/1.2/utils/src/ExecutionBurstServer.cpp
+++ b/neuralnetworks/1.2/utils/src/ExecutionBurstServer.cpp
@@ -17,8 +17,19 @@
 #define LOG_TAG "ExecutionBurstServer"
 
 #include "ExecutionBurstServer.h"
+#include "Conversions.h"
+#include "ExecutionBurstUtils.h"
 
 #include <android-base/logging.h>
+#include <nnapi/IBurst.h>
+#include <nnapi/Result.h>
+#include <nnapi/TypeUtils.h>
+#include <nnapi/Types.h>
+#include <nnapi/Validation.h>
+#include <nnapi/hal/1.0/Conversions.h>
+#include <nnapi/hal/HandleError.h>
+#include <nnapi/hal/ProtectCallback.h>
+#include <nnapi/hal/TransferValue.h>
 
 #include <algorithm>
 #include <cstring>
@@ -29,134 +40,146 @@
 #include <utility>
 #include <vector>
 
-#include "ExecutionBurstUtils.h"
-#include "HalInterfaces.h"
 #include "Tracing.h"
 
-namespace android::nn {
+namespace android::hardware::neuralnetworks::V1_2::utils {
 namespace {
 
-// DefaultBurstExecutorWithCache adapts an IPreparedModel so that it can be
-// used as an IBurstExecutorWithCache. Specifically, the cache simply stores the
-// hidl_memory object, and the execution forwards calls to the provided
-// IPreparedModel's "executeSynchronously" method. With this class, hidl_memory
-// must be mapped and unmapped for each execution.
-class DefaultBurstExecutorWithCache : public ExecutionBurstServer::IBurstExecutorWithCache {
-  public:
-    DefaultBurstExecutorWithCache(V1_2::IPreparedModel* preparedModel)
-        : mpPreparedModel(preparedModel) {}
+using neuralnetworks::utils::makeExecutionFailure;
 
-    bool isCacheEntryPresent(int32_t slot) const override {
-        const auto it = mMemoryCache.find(slot);
-        return (it != mMemoryCache.end()) && it->second.valid();
+constexpr V1_2::Timing kNoTiming = {std::numeric_limits<uint64_t>::max(),
+                                    std::numeric_limits<uint64_t>::max()};
+
+nn::GeneralResult<std::vector<nn::SharedMemory>> getMemoriesCallback(
+        V1_0::ErrorStatus status, const hidl_vec<hidl_memory>& memories) {
+    HANDLE_HAL_STATUS(status) << "getting burst memories failed with " << toString(status);
+    std::vector<nn::SharedMemory> canonicalMemories;
+    canonicalMemories.reserve(memories.size());
+    for (const auto& memory : memories) {
+        canonicalMemories.push_back(NN_TRY(nn::convert(memory)));
     }
-
-    void addCacheEntry(const hardware::hidl_memory& memory, int32_t slot) override {
-        mMemoryCache[slot] = memory;
-    }
-
-    void removeCacheEntry(int32_t slot) override { mMemoryCache.erase(slot); }
-
-    std::tuple<V1_0::ErrorStatus, hardware::hidl_vec<V1_2::OutputShape>, V1_2::Timing> execute(
-            const V1_0::Request& request, const std::vector<int32_t>& slots,
-            V1_2::MeasureTiming measure) override {
-        // convert slots to pools
-        hardware::hidl_vec<hardware::hidl_memory> pools(slots.size());
-        std::transform(slots.begin(), slots.end(), pools.begin(),
-                       [this](int32_t slot) { return mMemoryCache[slot]; });
-
-        // create full request
-        V1_0::Request fullRequest = request;
-        fullRequest.pools = std::move(pools);
-
-        // setup execution
-        V1_0::ErrorStatus returnedStatus = V1_0::ErrorStatus::GENERAL_FAILURE;
-        hardware::hidl_vec<V1_2::OutputShape> returnedOutputShapes;
-        V1_2::Timing returnedTiming;
-        auto cb = [&returnedStatus, &returnedOutputShapes, &returnedTiming](
-                          V1_0::ErrorStatus status,
-                          const hardware::hidl_vec<V1_2::OutputShape>& outputShapes,
-                          const V1_2::Timing& timing) {
-            returnedStatus = status;
-            returnedOutputShapes = outputShapes;
-            returnedTiming = timing;
-        };
-
-        // execute
-        const hardware::Return<void> ret =
-                mpPreparedModel->executeSynchronously(fullRequest, measure, cb);
-        if (!ret.isOk() || returnedStatus != V1_0::ErrorStatus::NONE) {
-            LOG(ERROR) << "IPreparedModelAdapter::execute -- Error executing";
-            return {returnedStatus, std::move(returnedOutputShapes), kNoTiming};
-        }
-
-        return std::make_tuple(returnedStatus, std::move(returnedOutputShapes), returnedTiming);
-    }
-
-  private:
-    V1_2::IPreparedModel* const mpPreparedModel;
-    std::map<int32_t, hardware::hidl_memory> mMemoryCache;
-};
+    return canonicalMemories;
+}
 
 }  // anonymous namespace
 
+ExecutionBurstServer::MemoryCache::MemoryCache(nn::SharedBurst burstExecutor,
+                                               sp<IBurstCallback> burstCallback)
+    : kBurstExecutor(std::move(burstExecutor)), kBurstCallback(std::move(burstCallback)) {
+    CHECK(kBurstExecutor != nullptr);
+    CHECK(kBurstCallback != nullptr);
+}
+
+nn::GeneralResult<std::vector<std::pair<nn::SharedMemory, nn::IBurst::OptionalCacheHold>>>
+ExecutionBurstServer::MemoryCache::getCacheEntries(const std::vector<int32_t>& slots) {
+    std::lock_guard guard(mMutex);
+    NN_TRY(ensureCacheEntriesArePresentLocked(slots));
+
+    std::vector<std::pair<nn::SharedMemory, nn::IBurst::OptionalCacheHold>> results;
+    results.reserve(slots.size());
+    for (int32_t slot : slots) {
+        results.push_back(NN_TRY(getCacheEntryLocked(slot)));
+    }
+
+    return results;
+}
+
+nn::GeneralResult<void> ExecutionBurstServer::MemoryCache::ensureCacheEntriesArePresentLocked(
+        const std::vector<int32_t>& slots) {
+    const auto slotIsKnown = [this](int32_t slot)
+                                     REQUIRES(mMutex) { return mCache.count(slot) > 0; };
+
+    // find unique unknown slots
+    std::vector<int32_t> unknownSlots = slots;
+    std::sort(unknownSlots.begin(), unknownSlots.end());
+    auto unknownSlotsEnd = std::unique(unknownSlots.begin(), unknownSlots.end());
+    unknownSlotsEnd = std::remove_if(unknownSlots.begin(), unknownSlotsEnd, slotIsKnown);
+    unknownSlots.erase(unknownSlotsEnd, unknownSlots.end());
+
+    // quick-exit if all slots are known
+    if (unknownSlots.empty()) {
+        return {};
+    }
+
+    auto cb = neuralnetworks::utils::CallbackValue(getMemoriesCallback);
+
+    const auto ret = kBurstCallback->getMemories(unknownSlots, cb);
+    HANDLE_TRANSPORT_FAILURE(ret);
+
+    auto returnedMemories = NN_TRY(cb.take());
+
+    if (returnedMemories.size() != unknownSlots.size()) {
+        return NN_ERROR()
+               << "ExecutionBurstServer::MemoryCache::ensureCacheEntriesArePresentLocked: Error "
+                  "retrieving memories -- count mismatch between requested memories ("
+               << unknownSlots.size() << ") and returned memories (" << returnedMemories.size()
+               << ")";
+    }
+
+    // add memories to unknown slots
+    for (size_t i = 0; i < unknownSlots.size(); ++i) {
+        addCacheEntryLocked(unknownSlots[i], std::move(returnedMemories[i]));
+    }
+
+    return {};
+}
+
+nn::GeneralResult<std::pair<nn::SharedMemory, nn::IBurst::OptionalCacheHold>>
+ExecutionBurstServer::MemoryCache::getCacheEntryLocked(int32_t slot) {
+    if (const auto iter = mCache.find(slot); iter != mCache.end()) {
+        return iter->second;
+    }
+    return NN_ERROR()
+           << "ExecutionBurstServer::MemoryCache::getCacheEntryLocked failed because slot " << slot
+           << " is not present in the cache";
+}
+
+void ExecutionBurstServer::MemoryCache::addCacheEntryLocked(int32_t slot, nn::SharedMemory memory) {
+    auto hold = kBurstExecutor->cacheMemory(memory);
+    mCache.emplace(slot, std::make_pair(std::move(memory), std::move(hold)));
+}
+
+void ExecutionBurstServer::MemoryCache::removeCacheEntry(int32_t slot) {
+    std::lock_guard guard(mMutex);
+    mCache.erase(slot);
+}
+
 // ExecutionBurstServer methods
 
-sp<ExecutionBurstServer> ExecutionBurstServer::create(
+nn::GeneralResult<sp<ExecutionBurstServer>> ExecutionBurstServer::create(
         const sp<IBurstCallback>& callback, const MQDescriptorSync<FmqRequestDatum>& requestChannel,
-        const MQDescriptorSync<FmqResultDatum>& resultChannel,
-        std::shared_ptr<IBurstExecutorWithCache> executorWithCache,
+        const MQDescriptorSync<FmqResultDatum>& resultChannel, nn::SharedBurst burstExecutor,
         std::chrono::microseconds pollingTimeWindow) {
     // check inputs
-    if (callback == nullptr || executorWithCache == nullptr) {
-        LOG(ERROR) << "ExecutionBurstServer::create passed a nullptr";
-        return nullptr;
+    if (callback == nullptr || burstExecutor == nullptr) {
+        return NN_ERROR() << "ExecutionBurstServer::create passed a nullptr";
     }
 
     // create FMQ objects
-    std::unique_ptr<RequestChannelReceiver> requestChannelReceiver =
-            RequestChannelReceiver::create(requestChannel, pollingTimeWindow);
-    std::unique_ptr<ResultChannelSender> resultChannelSender =
-            ResultChannelSender::create(resultChannel);
+    auto requestChannelReceiver =
+            NN_TRY(RequestChannelReceiver::create(requestChannel, pollingTimeWindow));
+    auto resultChannelSender = NN_TRY(ResultChannelSender::create(resultChannel));
 
     // check FMQ objects
-    if (!requestChannelReceiver || !resultChannelSender) {
-        LOG(ERROR) << "ExecutionBurstServer::create failed to create FastMessageQueue";
-        return nullptr;
-    }
+    CHECK(requestChannelReceiver != nullptr);
+    CHECK(resultChannelSender != nullptr);
 
     // make and return context
-    return new ExecutionBurstServer(callback, std::move(requestChannelReceiver),
-                                    std::move(resultChannelSender), std::move(executorWithCache));
+    return sp<ExecutionBurstServer>::make(PrivateConstructorTag{}, callback,
+                                          std::move(requestChannelReceiver),
+                                          std::move(resultChannelSender), std::move(burstExecutor));
 }
 
-sp<ExecutionBurstServer> ExecutionBurstServer::create(
-        const sp<IBurstCallback>& callback, const MQDescriptorSync<FmqRequestDatum>& requestChannel,
-        const MQDescriptorSync<FmqResultDatum>& resultChannel, V1_2::IPreparedModel* preparedModel,
-        std::chrono::microseconds pollingTimeWindow) {
-    // check relevant input
-    if (preparedModel == nullptr) {
-        LOG(ERROR) << "ExecutionBurstServer::create passed a nullptr";
-        return nullptr;
-    }
-
-    // adapt IPreparedModel to have caching
-    const std::shared_ptr<DefaultBurstExecutorWithCache> preparedModelAdapter =
-            std::make_shared<DefaultBurstExecutorWithCache>(preparedModel);
-
-    // make and return context
-    return ExecutionBurstServer::create(callback, requestChannel, resultChannel,
-                                        preparedModelAdapter, pollingTimeWindow);
-}
-
-ExecutionBurstServer::ExecutionBurstServer(
-        const sp<IBurstCallback>& callback, std::unique_ptr<RequestChannelReceiver> requestChannel,
-        std::unique_ptr<ResultChannelSender> resultChannel,
-        std::shared_ptr<IBurstExecutorWithCache> executorWithCache)
+ExecutionBurstServer::ExecutionBurstServer(PrivateConstructorTag /*tag*/,
+                                           const sp<IBurstCallback>& callback,
+                                           std::unique_ptr<RequestChannelReceiver> requestChannel,
+                                           std::unique_ptr<ResultChannelSender> resultChannel,
+                                           nn::SharedBurst burstExecutor)
     : mCallback(callback),
       mRequestChannelReceiver(std::move(requestChannel)),
       mResultChannelSender(std::move(resultChannel)),
-      mExecutorWithCache(std::move(executorWithCache)) {
+      mBurstExecutor(std::move(burstExecutor)),
+      mMemoryCache(mBurstExecutor, mCallback) {
     // TODO: highly document the threading behavior of this class
     mWorker = std::thread([this] { task(); });
 }
@@ -170,51 +193,9 @@
     mWorker.join();
 }
 
-hardware::Return<void> ExecutionBurstServer::freeMemory(int32_t slot) {
-    std::lock_guard<std::mutex> hold(mMutex);
-    mExecutorWithCache->removeCacheEntry(slot);
-    return hardware::Void();
-}
-
-void ExecutionBurstServer::ensureCacheEntriesArePresentLocked(const std::vector<int32_t>& slots) {
-    const auto slotIsKnown = [this](int32_t slot) {
-        return mExecutorWithCache->isCacheEntryPresent(slot);
-    };
-
-    // find unique unknown slots
-    std::vector<int32_t> unknownSlots = slots;
-    auto unknownSlotsEnd = unknownSlots.end();
-    std::sort(unknownSlots.begin(), unknownSlotsEnd);
-    unknownSlotsEnd = std::unique(unknownSlots.begin(), unknownSlotsEnd);
-    unknownSlotsEnd = std::remove_if(unknownSlots.begin(), unknownSlotsEnd, slotIsKnown);
-    unknownSlots.erase(unknownSlotsEnd, unknownSlots.end());
-
-    // quick-exit if all slots are known
-    if (unknownSlots.empty()) {
-        return;
-    }
-
-    V1_0::ErrorStatus errorStatus = V1_0::ErrorStatus::GENERAL_FAILURE;
-    std::vector<hardware::hidl_memory> returnedMemories;
-    auto cb = [&errorStatus, &returnedMemories](
-                      V1_0::ErrorStatus status,
-                      const hardware::hidl_vec<hardware::hidl_memory>& memories) {
-        errorStatus = status;
-        returnedMemories = memories;
-    };
-
-    const hardware::Return<void> ret = mCallback->getMemories(unknownSlots, cb);
-
-    if (!ret.isOk() || errorStatus != V1_0::ErrorStatus::NONE ||
-        returnedMemories.size() != unknownSlots.size()) {
-        LOG(ERROR) << "Error retrieving memories";
-        return;
-    }
-
-    // add memories to unknown slots
-    for (size_t i = 0; i < unknownSlots.size(); ++i) {
-        mExecutorWithCache->addCacheEntry(returnedMemories[i], unknownSlots[i]);
-    }
+Return<void> ExecutionBurstServer::freeMemory(int32_t slot) {
+    mMemoryCache.removeCacheEntry(slot);
+    return Void();
 }
 
 void ExecutionBurstServer::task() {
@@ -223,38 +204,65 @@
         // receive request
         auto arguments = mRequestChannelReceiver->getBlocking();
 
-        // if the request packet was not properly received, return a generic
-        // error and skip the execution
+        // if the request packet was not properly received, return a generic error and skip the
+        // execution
         //
-        // if the burst is being torn down, skip the execution so the "task"
-        // function can end
-        if (!arguments) {
+        // if the burst is being torn down, skip the execution so the "task" function can end
+        if (!arguments.has_value()) {
             if (!mTeardown) {
                 mResultChannelSender->send(V1_0::ErrorStatus::GENERAL_FAILURE, {}, kNoTiming);
             }
             continue;
         }
 
-        // otherwise begin tracing execution
-        NNTRACE_FULL(NNTRACE_LAYER_IPC, NNTRACE_PHASE_EXECUTION,
-                     "ExecutionBurstServer getting memory, executing, and returning results");
+        // unpack the arguments; types are Request, std::vector<int32_t>, and MeasureTiming,
+        // respectively
+        const auto [requestWithoutPools, slotsOfPools, measure] = std::move(arguments).value();
 
-        // unpack the arguments; types are Request, std::vector<int32_t>, and
-        // MeasureTiming, respectively
-        const auto [requestWithoutPools, slotsOfPools, measure] = std::move(*arguments);
-
-        // ensure executor with cache has required memory
-        std::lock_guard<std::mutex> hold(mMutex);
-        ensureCacheEntriesArePresentLocked(slotsOfPools);
-
-        // perform computation; types are ErrorStatus, hidl_vec<OutputShape>,
-        // and Timing, respectively
-        const auto [errorStatus, outputShapes, returnedTiming] =
-                mExecutorWithCache->execute(requestWithoutPools, slotsOfPools, measure);
+        auto result = execute(requestWithoutPools, slotsOfPools, measure);
 
         // return result
-        mResultChannelSender->send(errorStatus, outputShapes, returnedTiming);
+        if (result.has_value()) {
+            const auto& [outputShapes, timing] = result.value();
+            mResultChannelSender->send(V1_0::ErrorStatus::NONE, outputShapes, timing);
+        } else {
+            const auto& [message, code, outputShapes] = result.error();
+            LOG(ERROR) << "IBurst::execute failed with " << code << ": " << message;
+            mResultChannelSender->send(convert(code).value(), convert(outputShapes).value(),
+                                       kNoTiming);
+        }
     }
 }
 
-}  // namespace android::nn
+nn::ExecutionResult<std::pair<hidl_vec<OutputShape>, Timing>> ExecutionBurstServer::execute(
+        const V1_0::Request& requestWithoutPools, const std::vector<int32_t>& slotsOfPools,
+        MeasureTiming measure) {
+    NNTRACE_FULL(NNTRACE_LAYER_IPC, NNTRACE_PHASE_EXECUTION,
+                 "ExecutionBurstServer getting memory, executing, and returning results");
+
+    // ensure executor with cache has required memory
+    const auto cacheEntries =
+            NN_TRY(makeExecutionFailure(mMemoryCache.getCacheEntries(slotsOfPools)));
+
+    // convert request, populating its pools
+    // This code performs an unvalidated convert because the request object without its pools is
+    // invalid because it is incomplete. Instead, the validation is performed after the memory pools
+    // have been added to the request.
+    auto canonicalRequest =
+            NN_TRY(makeExecutionFailure(nn::unvalidatedConvert(requestWithoutPools)));
+    CHECK(canonicalRequest.pools.empty());
+    std::transform(cacheEntries.begin(), cacheEntries.end(),
+                   std::back_inserter(canonicalRequest.pools),
+                   [](const auto& cacheEntry) { return cacheEntry.first; });
+    NN_TRY(makeExecutionFailure(validate(canonicalRequest)));
+
+    nn::MeasureTiming canonicalMeasure = NN_TRY(makeExecutionFailure(nn::convert(measure)));
+
+    const auto [outputShapes, timing] =
+            NN_TRY(mBurstExecutor->execute(canonicalRequest, canonicalMeasure));
+
+    return std::make_pair(NN_TRY(makeExecutionFailure(convert(outputShapes))),
+                          NN_TRY(makeExecutionFailure(convert(timing))));
+}
+
+}  // namespace android::hardware::neuralnetworks::V1_2::utils
diff --git a/neuralnetworks/1.2/utils/src/ExecutionBurstUtils.cpp b/neuralnetworks/1.2/utils/src/ExecutionBurstUtils.cpp
index f0275f9..ca3a52c 100644
--- a/neuralnetworks/1.2/utils/src/ExecutionBurstUtils.cpp
+++ b/neuralnetworks/1.2/utils/src/ExecutionBurstUtils.cpp
@@ -19,11 +19,15 @@
 #include "ExecutionBurstUtils.h"
 
 #include <android-base/logging.h>
+#include <android-base/properties.h>
 #include <android/hardware/neuralnetworks/1.0/types.h>
 #include <android/hardware/neuralnetworks/1.1/types.h>
 #include <android/hardware/neuralnetworks/1.2/types.h>
 #include <fmq/MessageQueue.h>
 #include <hidl/MQDescriptor.h>
+#include <nnapi/Result.h>
+#include <nnapi/Types.h>
+#include <nnapi/hal/ProtectCallback.h>
 
 #include <atomic>
 #include <chrono>
@@ -39,84 +43,97 @@
 constexpr V1_2::Timing kNoTiming = {std::numeric_limits<uint64_t>::max(),
                                     std::numeric_limits<uint64_t>::max()};
 
+std::chrono::microseconds getPollingTimeWindow(const std::string& property) {
+    constexpr int32_t kDefaultPollingTimeWindow = 0;
+#ifdef NN_DEBUGGABLE
+    constexpr int32_t kMinPollingTimeWindow = 0;
+    const int32_t selectedPollingTimeWindow =
+            base::GetIntProperty(property, kDefaultPollingTimeWindow, kMinPollingTimeWindow);
+    return std::chrono::microseconds(selectedPollingTimeWindow);
+#else
+    (void)property;
+    return std::chrono::microseconds(kDefaultPollingTimeWindow);
+#endif  // NN_DEBUGGABLE
+}
+
+}  // namespace
+
+std::chrono::microseconds getBurstControllerPollingTimeWindow() {
+    return getPollingTimeWindow("debug.nn.burst-controller-polling-window");
+}
+
+std::chrono::microseconds getBurstServerPollingTimeWindow() {
+    return getPollingTimeWindow("debug.nn.burst-server-polling-window");
 }
 
 // serialize a request into a packet
 std::vector<FmqRequestDatum> serialize(const V1_0::Request& request, V1_2::MeasureTiming measure,
                                        const std::vector<int32_t>& slots) {
     // count how many elements need to be sent for a request
-    size_t count = 2 + request.inputs.size() + request.outputs.size() + request.pools.size();
+    size_t count = 2 + request.inputs.size() + request.outputs.size() + slots.size();
     for (const auto& input : request.inputs) {
         count += input.dimensions.size();
     }
     for (const auto& output : request.outputs) {
         count += output.dimensions.size();
     }
+    CHECK_LE(count, std::numeric_limits<uint32_t>::max());
 
     // create buffer to temporarily store elements
     std::vector<FmqRequestDatum> data;
     data.reserve(count);
 
     // package packetInfo
-    {
-        FmqRequestDatum datum;
-        datum.packetInformation(
-                {/*.packetSize=*/static_cast<uint32_t>(count),
-                 /*.numberOfInputOperands=*/static_cast<uint32_t>(request.inputs.size()),
-                 /*.numberOfOutputOperands=*/static_cast<uint32_t>(request.outputs.size()),
-                 /*.numberOfPools=*/static_cast<uint32_t>(request.pools.size())});
-        data.push_back(datum);
-    }
+    data.emplace_back();
+    data.back().packetInformation(
+            {.packetSize = static_cast<uint32_t>(count),
+             .numberOfInputOperands = static_cast<uint32_t>(request.inputs.size()),
+             .numberOfOutputOperands = static_cast<uint32_t>(request.outputs.size()),
+             .numberOfPools = static_cast<uint32_t>(slots.size())});
 
     // package input data
     for (const auto& input : request.inputs) {
         // package operand information
-        FmqRequestDatum datum;
-        datum.inputOperandInformation(
-                {/*.hasNoValue=*/input.hasNoValue,
-                 /*.location=*/input.location,
-                 /*.numberOfDimensions=*/static_cast<uint32_t>(input.dimensions.size())});
-        data.push_back(datum);
+        data.emplace_back();
+        data.back().inputOperandInformation(
+                {.hasNoValue = input.hasNoValue,
+                 .location = input.location,
+                 .numberOfDimensions = static_cast<uint32_t>(input.dimensions.size())});
 
         // package operand dimensions
         for (uint32_t dimension : input.dimensions) {
-            FmqRequestDatum datum;
-            datum.inputOperandDimensionValue(dimension);
-            data.push_back(datum);
+            data.emplace_back();
+            data.back().inputOperandDimensionValue(dimension);
         }
     }
 
     // package output data
     for (const auto& output : request.outputs) {
         // package operand information
-        FmqRequestDatum datum;
-        datum.outputOperandInformation(
-                {/*.hasNoValue=*/output.hasNoValue,
-                 /*.location=*/output.location,
-                 /*.numberOfDimensions=*/static_cast<uint32_t>(output.dimensions.size())});
-        data.push_back(datum);
+        data.emplace_back();
+        data.back().outputOperandInformation(
+                {.hasNoValue = output.hasNoValue,
+                 .location = output.location,
+                 .numberOfDimensions = static_cast<uint32_t>(output.dimensions.size())});
 
         // package operand dimensions
         for (uint32_t dimension : output.dimensions) {
-            FmqRequestDatum datum;
-            datum.outputOperandDimensionValue(dimension);
-            data.push_back(datum);
+            data.emplace_back();
+            data.back().outputOperandDimensionValue(dimension);
         }
     }
 
     // package pool identifier
     for (int32_t slot : slots) {
-        FmqRequestDatum datum;
-        datum.poolIdentifier(slot);
-        data.push_back(datum);
+        data.emplace_back();
+        data.back().poolIdentifier(slot);
     }
 
     // package measureTiming
-    {
-        FmqRequestDatum datum;
-        datum.measureTiming(measure);
-        data.push_back(datum);
-    }
+    data.emplace_back();
+    data.back().measureTiming(measure);
+
+    CHECK_EQ(data.size(), count);
 
     // return packet
     return data;
@@ -137,46 +154,38 @@
     data.reserve(count);
 
     // package packetInfo
-    {
-        FmqResultDatum datum;
-        datum.packetInformation({/*.packetSize=*/static_cast<uint32_t>(count),
-                                 /*.errorStatus=*/errorStatus,
-                                 /*.numberOfOperands=*/static_cast<uint32_t>(outputShapes.size())});
-        data.push_back(datum);
-    }
+    data.emplace_back();
+    data.back().packetInformation({.packetSize = static_cast<uint32_t>(count),
+                                   .errorStatus = errorStatus,
+                                   .numberOfOperands = static_cast<uint32_t>(outputShapes.size())});
 
     // package output shape data
     for (const auto& operand : outputShapes) {
         // package operand information
-        FmqResultDatum::OperandInformation info{};
-        info.isSufficient = operand.isSufficient;
-        info.numberOfDimensions = static_cast<uint32_t>(operand.dimensions.size());
-
-        FmqResultDatum datum;
-        datum.operandInformation(info);
-        data.push_back(datum);
+        data.emplace_back();
+        data.back().operandInformation(
+                {.isSufficient = operand.isSufficient,
+                 .numberOfDimensions = static_cast<uint32_t>(operand.dimensions.size())});
 
         // package operand dimensions
         for (uint32_t dimension : operand.dimensions) {
-            FmqResultDatum datum;
-            datum.operandDimensionValue(dimension);
-            data.push_back(datum);
+            data.emplace_back();
+            data.back().operandDimensionValue(dimension);
         }
     }
 
     // package executionTiming
-    {
-        FmqResultDatum datum;
-        datum.executionTiming(timing);
-        data.push_back(datum);
-    }
+    data.emplace_back();
+    data.back().executionTiming(timing);
+
+    CHECK_EQ(data.size(), count);
 
     // return result
     return data;
 }
 
 // deserialize request
-std::optional<std::tuple<V1_0::Request, std::vector<int32_t>, V1_2::MeasureTiming>> deserialize(
+nn::Result<std::tuple<V1_0::Request, std::vector<int32_t>, V1_2::MeasureTiming>> deserialize(
         const std::vector<FmqRequestDatum>& data) {
     using discriminator = FmqRequestDatum::hidl_discriminator;
 
@@ -184,8 +193,7 @@
 
     // validate packet information
     if (data.size() == 0 || data[index].getDiscriminator() != discriminator::packetInformation) {
-        LOG(ERROR) << "FMQ Request packet ill-formed";
-        return std::nullopt;
+        return NN_ERROR() << "FMQ Request packet ill-formed";
     }
 
     // unpackage packet information
@@ -198,8 +206,7 @@
 
     // verify packet size
     if (data.size() != packetSize) {
-        LOG(ERROR) << "FMQ Request packet ill-formed";
-        return std::nullopt;
+        return NN_ERROR() << "FMQ Request packet ill-formed";
     }
 
     // unpackage input operands
@@ -208,8 +215,7 @@
     for (size_t operand = 0; operand < numberOfInputOperands; ++operand) {
         // validate input operand information
         if (data[index].getDiscriminator() != discriminator::inputOperandInformation) {
-            LOG(ERROR) << "FMQ Request packet ill-formed";
-            return std::nullopt;
+            return NN_ERROR() << "FMQ Request packet ill-formed";
         }
 
         // unpackage operand information
@@ -226,8 +232,7 @@
         for (size_t i = 0; i < numberOfDimensions; ++i) {
             // validate dimension
             if (data[index].getDiscriminator() != discriminator::inputOperandDimensionValue) {
-                LOG(ERROR) << "FMQ Request packet ill-formed";
-                return std::nullopt;
+                return NN_ERROR() << "FMQ Request packet ill-formed";
             }
 
             // unpackage dimension
@@ -240,7 +245,7 @@
 
         // store result
         inputs.push_back(
-                {/*.hasNoValue=*/hasNoValue, /*.location=*/location, /*.dimensions=*/dimensions});
+                {.hasNoValue = hasNoValue, .location = location, .dimensions = dimensions});
     }
 
     // unpackage output operands
@@ -249,8 +254,7 @@
     for (size_t operand = 0; operand < numberOfOutputOperands; ++operand) {
         // validate output operand information
         if (data[index].getDiscriminator() != discriminator::outputOperandInformation) {
-            LOG(ERROR) << "FMQ Request packet ill-formed";
-            return std::nullopt;
+            return NN_ERROR() << "FMQ Request packet ill-formed";
         }
 
         // unpackage operand information
@@ -267,8 +271,7 @@
         for (size_t i = 0; i < numberOfDimensions; ++i) {
             // validate dimension
             if (data[index].getDiscriminator() != discriminator::outputOperandDimensionValue) {
-                LOG(ERROR) << "FMQ Request packet ill-formed";
-                return std::nullopt;
+                return NN_ERROR() << "FMQ Request packet ill-formed";
             }
 
             // unpackage dimension
@@ -281,7 +284,7 @@
 
         // store result
         outputs.push_back(
-                {/*.hasNoValue=*/hasNoValue, /*.location=*/location, /*.dimensions=*/dimensions});
+                {.hasNoValue = hasNoValue, .location = location, .dimensions = dimensions});
     }
 
     // unpackage pools
@@ -290,8 +293,7 @@
     for (size_t pool = 0; pool < numberOfPools; ++pool) {
         // validate input operand information
         if (data[index].getDiscriminator() != discriminator::poolIdentifier) {
-            LOG(ERROR) << "FMQ Request packet ill-formed";
-            return std::nullopt;
+            return NN_ERROR() << "FMQ Request packet ill-formed";
         }
 
         // unpackage operand information
@@ -304,8 +306,7 @@
 
     // validate measureTiming
     if (data[index].getDiscriminator() != discriminator::measureTiming) {
-        LOG(ERROR) << "FMQ Request packet ill-formed";
-        return std::nullopt;
+        return NN_ERROR() << "FMQ Request packet ill-formed";
     }
 
     // unpackage measureTiming
@@ -314,27 +315,23 @@
 
     // validate packet information
     if (index != packetSize) {
-        LOG(ERROR) << "FMQ Result packet ill-formed";
-        return std::nullopt;
+        return NN_ERROR() << "FMQ Result packet ill-formed";
     }
 
     // return request
-    V1_0::Request request = {/*.inputs=*/inputs, /*.outputs=*/outputs, /*.pools=*/{}};
+    V1_0::Request request = {.inputs = inputs, .outputs = outputs, .pools = {}};
     return std::make_tuple(std::move(request), std::move(slots), measure);
 }
 
 // deserialize a packet into the result
-std::optional<std::tuple<V1_0::ErrorStatus, std::vector<V1_2::OutputShape>, V1_2::Timing>>
-deserialize(const std::vector<FmqResultDatum>& data) {
+nn::Result<std::tuple<V1_0::ErrorStatus, std::vector<V1_2::OutputShape>, V1_2::Timing>> deserialize(
+        const std::vector<FmqResultDatum>& data) {
     using discriminator = FmqResultDatum::hidl_discriminator;
-
-    std::vector<V1_2::OutputShape> outputShapes;
     size_t index = 0;
 
     // validate packet information
     if (data.size() == 0 || data[index].getDiscriminator() != discriminator::packetInformation) {
-        LOG(ERROR) << "FMQ Result packet ill-formed";
-        return std::nullopt;
+        return NN_ERROR() << "FMQ Result packet ill-formed";
     }
 
     // unpackage packet information
@@ -346,16 +343,16 @@
 
     // verify packet size
     if (data.size() != packetSize) {
-        LOG(ERROR) << "FMQ Result packet ill-formed";
-        return std::nullopt;
+        return NN_ERROR() << "FMQ Result packet ill-formed";
     }
 
     // unpackage operands
+    std::vector<V1_2::OutputShape> outputShapes;
+    outputShapes.reserve(numberOfOperands);
     for (size_t operand = 0; operand < numberOfOperands; ++operand) {
         // validate operand information
         if (data[index].getDiscriminator() != discriminator::operandInformation) {
-            LOG(ERROR) << "FMQ Result packet ill-formed";
-            return std::nullopt;
+            return NN_ERROR() << "FMQ Result packet ill-formed";
         }
 
         // unpackage operand information
@@ -370,8 +367,7 @@
         for (size_t i = 0; i < numberOfDimensions; ++i) {
             // validate dimension
             if (data[index].getDiscriminator() != discriminator::operandDimensionValue) {
-                LOG(ERROR) << "FMQ Result packet ill-formed";
-                return std::nullopt;
+                return NN_ERROR() << "FMQ Result packet ill-formed";
             }
 
             // unpackage dimension
@@ -383,13 +379,12 @@
         }
 
         // store result
-        outputShapes.push_back({/*.dimensions=*/dimensions, /*.isSufficient=*/isSufficient});
+        outputShapes.push_back({.dimensions = dimensions, .isSufficient = isSufficient});
     }
 
     // validate execution timing
     if (data[index].getDiscriminator() != discriminator::executionTiming) {
-        LOG(ERROR) << "FMQ Result packet ill-formed";
-        return std::nullopt;
+        return NN_ERROR() << "FMQ Result packet ill-formed";
     }
 
     // unpackage execution timing
@@ -398,123 +393,113 @@
 
     // validate packet information
     if (index != packetSize) {
-        LOG(ERROR) << "FMQ Result packet ill-formed";
-        return std::nullopt;
+        return NN_ERROR() << "FMQ Result packet ill-formed";
     }
 
     // return result
     return std::make_tuple(errorStatus, std::move(outputShapes), timing);
 }
 
-V1_0::ErrorStatus legacyConvertResultCodeToErrorStatus(int resultCode) {
-    return convertToV1_0(convertResultCodeToErrorStatus(resultCode));
-}
-
 // RequestChannelSender methods
 
-std::pair<std::unique_ptr<RequestChannelSender>, const FmqRequestDescriptor*>
+nn::GeneralResult<
+        std::pair<std::unique_ptr<RequestChannelSender>, const MQDescriptorSync<FmqRequestDatum>*>>
 RequestChannelSender::create(size_t channelLength) {
-    std::unique_ptr<FmqRequestChannel> fmqRequestChannel =
-            std::make_unique<FmqRequestChannel>(channelLength, /*confEventFlag=*/true);
-    if (!fmqRequestChannel->isValid()) {
-        LOG(ERROR) << "Unable to create RequestChannelSender";
-        return {nullptr, nullptr};
+    auto requestChannelSender =
+            std::make_unique<RequestChannelSender>(PrivateConstructorTag{}, channelLength);
+    if (!requestChannelSender->mFmqRequestChannel.isValid()) {
+        return NN_ERROR() << "Unable to create RequestChannelSender";
     }
 
-    const FmqRequestDescriptor* descriptor = fmqRequestChannel->getDesc();
-    return std::make_pair(std::make_unique<RequestChannelSender>(std::move(fmqRequestChannel)),
-                          descriptor);
+    const MQDescriptorSync<FmqRequestDatum>* descriptor =
+            requestChannelSender->mFmqRequestChannel.getDesc();
+    return std::make_pair(std::move(requestChannelSender), descriptor);
 }
 
-RequestChannelSender::RequestChannelSender(std::unique_ptr<FmqRequestChannel> fmqRequestChannel)
-    : mFmqRequestChannel(std::move(fmqRequestChannel)) {}
+RequestChannelSender::RequestChannelSender(PrivateConstructorTag /*tag*/, size_t channelLength)
+    : mFmqRequestChannel(channelLength, /*configureEventFlagWord=*/true) {}
 
-bool RequestChannelSender::send(const V1_0::Request& request, V1_2::MeasureTiming measure,
-                                const std::vector<int32_t>& slots) {
+nn::Result<void> RequestChannelSender::send(const V1_0::Request& request,
+                                            V1_2::MeasureTiming measure,
+                                            const std::vector<int32_t>& slots) {
     const std::vector<FmqRequestDatum> serialized = serialize(request, measure, slots);
     return sendPacket(serialized);
 }
 
-bool RequestChannelSender::sendPacket(const std::vector<FmqRequestDatum>& packet) {
+nn::Result<void> RequestChannelSender::sendPacket(const std::vector<FmqRequestDatum>& packet) {
     if (!mValid) {
-        return false;
+        return NN_ERROR() << "FMQ object is invalid";
     }
 
-    if (packet.size() > mFmqRequestChannel->availableToWrite()) {
-        LOG(ERROR)
-                << "RequestChannelSender::sendPacket -- packet size exceeds size available in FMQ";
-        return false;
+    if (packet.size() > mFmqRequestChannel.availableToWrite()) {
+        return NN_ERROR()
+               << "RequestChannelSender::sendPacket -- packet size exceeds size available in FMQ";
     }
 
-    // Always send the packet with "blocking" because this signals the futex and
-    // unblocks the consumer if it is waiting on the futex.
-    return mFmqRequestChannel->writeBlocking(packet.data(), packet.size());
+    // Always send the packet with "blocking" because this signals the futex and unblocks the
+    // consumer if it is waiting on the futex.
+    const bool success = mFmqRequestChannel.writeBlocking(packet.data(), packet.size());
+    if (!success) {
+        return NN_ERROR()
+               << "RequestChannelSender::sendPacket -- FMQ's writeBlocking returned an error";
+    }
+
+    return {};
 }
 
-void RequestChannelSender::invalidate() {
+void RequestChannelSender::notifyAsDeadObject() {
     mValid = false;
 }
 
 // RequestChannelReceiver methods
 
-std::unique_ptr<RequestChannelReceiver> RequestChannelReceiver::create(
-        const FmqRequestDescriptor& requestChannel, std::chrono::microseconds pollingTimeWindow) {
-    std::unique_ptr<FmqRequestChannel> fmqRequestChannel =
-            std::make_unique<FmqRequestChannel>(requestChannel);
+nn::GeneralResult<std::unique_ptr<RequestChannelReceiver>> RequestChannelReceiver::create(
+        const MQDescriptorSync<FmqRequestDatum>& requestChannel,
+        std::chrono::microseconds pollingTimeWindow) {
+    auto requestChannelReceiver = std::make_unique<RequestChannelReceiver>(
+            PrivateConstructorTag{}, requestChannel, pollingTimeWindow);
 
-    if (!fmqRequestChannel->isValid()) {
-        LOG(ERROR) << "Unable to create RequestChannelReceiver";
-        return nullptr;
+    if (!requestChannelReceiver->mFmqRequestChannel.isValid()) {
+        return NN_ERROR() << "Unable to create RequestChannelReceiver";
     }
-    if (fmqRequestChannel->getEventFlagWord() == nullptr) {
-        LOG(ERROR)
-                << "RequestChannelReceiver::create was passed an MQDescriptor without an EventFlag";
-        return nullptr;
+    if (requestChannelReceiver->mFmqRequestChannel.getEventFlagWord() == nullptr) {
+        return NN_ERROR()
+               << "RequestChannelReceiver::create was passed an MQDescriptor without an EventFlag";
     }
 
-    return std::make_unique<RequestChannelReceiver>(std::move(fmqRequestChannel),
-                                                    pollingTimeWindow);
+    return requestChannelReceiver;
 }
 
-RequestChannelReceiver::RequestChannelReceiver(std::unique_ptr<FmqRequestChannel> fmqRequestChannel,
-                                               std::chrono::microseconds pollingTimeWindow)
-    : mFmqRequestChannel(std::move(fmqRequestChannel)), kPollingTimeWindow(pollingTimeWindow) {}
+RequestChannelReceiver::RequestChannelReceiver(
+        PrivateConstructorTag /*tag*/, const MQDescriptorSync<FmqRequestDatum>& requestChannel,
+        std::chrono::microseconds pollingTimeWindow)
+    : mFmqRequestChannel(requestChannel), kPollingTimeWindow(pollingTimeWindow) {}
 
-std::optional<std::tuple<V1_0::Request, std::vector<int32_t>, V1_2::MeasureTiming>>
+nn::Result<std::tuple<V1_0::Request, std::vector<int32_t>, V1_2::MeasureTiming>>
 RequestChannelReceiver::getBlocking() {
-    const auto packet = getPacketBlocking();
-    if (!packet) {
-        return std::nullopt;
-    }
-
-    return deserialize(*packet);
+    const auto packet = NN_TRY(getPacketBlocking());
+    return deserialize(packet);
 }
 
 void RequestChannelReceiver::invalidate() {
     mTeardown = true;
 
     // force unblock
-    // ExecutionBurstServer is by default waiting on a request packet. If the
-    // client process destroys its burst object, the server may still be waiting
-    // on the futex. This force unblock wakes up any thread waiting on the
-    // futex.
-    // TODO: look for a different/better way to signal/notify the futex to wake
-    // up any thread waiting on it
-    FmqRequestDatum datum;
-    datum.packetInformation({/*.packetSize=*/0, /*.numberOfInputOperands=*/0,
-                             /*.numberOfOutputOperands=*/0, /*.numberOfPools=*/0});
-    mFmqRequestChannel->writeBlocking(&datum, 1);
+    // ExecutionBurstServer is by default waiting on a request packet. If the client process
+    // destroys its burst object, the server may still be waiting on the futex. This force unblock
+    // wakes up any thread waiting on the futex.
+    const auto data = serialize(V1_0::Request{}, V1_2::MeasureTiming::NO, {});
+    mFmqRequestChannel.writeBlocking(data.data(), data.size());
 }
 
-std::optional<std::vector<FmqRequestDatum>> RequestChannelReceiver::getPacketBlocking() {
+nn::Result<std::vector<FmqRequestDatum>> RequestChannelReceiver::getPacketBlocking() {
     if (mTeardown) {
-        return std::nullopt;
+        return NN_ERROR() << "FMQ object is being torn down";
     }
 
-    // First spend time polling if results are available in FMQ instead of
-    // waiting on the futex. Polling is more responsive (yielding lower
-    // latencies), but can take up more power, so only poll for a limited period
-    // of time.
+    // First spend time polling if results are available in FMQ instead of waiting on the futex.
+    // Polling is more responsive (yielding lower latencies), but can take up more power, so only
+    // poll for a limited period of time.
 
     auto& getCurrentTime = std::chrono::high_resolution_clock::now;
     const auto timeToStopPolling = getCurrentTime() + kPollingTimeWindow;
@@ -522,173 +507,144 @@
     while (getCurrentTime() < timeToStopPolling) {
         // if class is being torn down, immediately return
         if (mTeardown.load(std::memory_order_relaxed)) {
-            return std::nullopt;
+            return NN_ERROR() << "FMQ object is being torn down";
         }
 
-        // Check if data is available. If it is, immediately retrieve it and
-        // return.
-        const size_t available = mFmqRequestChannel->availableToRead();
+        // Check if data is available. If it is, immediately retrieve it and return.
+        const size_t available = mFmqRequestChannel.availableToRead();
         if (available > 0) {
-            // This is the first point when we know an execution is occurring,
-            // so begin to collect systraces. Note that a similar systrace does
-            // not exist at the corresponding point in
-            // ResultChannelReceiver::getPacketBlocking because the execution is
-            // already in flight.
-            NNTRACE_FULL(NNTRACE_LAYER_IPC, NNTRACE_PHASE_EXECUTION,
-                         "ExecutionBurstServer getting packet");
             std::vector<FmqRequestDatum> packet(available);
-            const bool success = mFmqRequestChannel->read(packet.data(), available);
+            const bool success = mFmqRequestChannel.readBlocking(packet.data(), available);
             if (!success) {
-                LOG(ERROR) << "Error receiving packet";
-                return std::nullopt;
+                return NN_ERROR() << "Error receiving packet";
             }
-            return std::make_optional(std::move(packet));
+            return packet;
         }
     }
 
-    // If we get to this point, we either stopped polling because it was taking
-    // too long or polling was not allowed. Instead, perform a blocking call
-    // which uses a futex to save power.
+    // If we get to this point, we either stopped polling because it was taking too long or polling
+    // was not allowed. Instead, perform a blocking call which uses a futex to save power.
 
     // wait for request packet and read first element of request packet
     FmqRequestDatum datum;
-    bool success = mFmqRequestChannel->readBlocking(&datum, 1);
-
-    // This is the first point when we know an execution is occurring, so begin
-    // to collect systraces. Note that a similar systrace does not exist at the
-    // corresponding point in ResultChannelReceiver::getPacketBlocking because
-    // the execution is already in flight.
-    NNTRACE_FULL(NNTRACE_LAYER_IPC, NNTRACE_PHASE_EXECUTION, "ExecutionBurstServer getting packet");
+    bool success = mFmqRequestChannel.readBlocking(&datum, 1);
 
     // retrieve remaining elements
-    // NOTE: all of the data is already available at this point, so there's no
-    // need to do a blocking wait to wait for more data. This is known because
-    // in FMQ, all writes are published (made available) atomically. Currently,
-    // the producer always publishes the entire packet in one function call, so
-    // if the first element of the packet is available, the remaining elements
-    // are also available.
-    const size_t count = mFmqRequestChannel->availableToRead();
+    // NOTE: all of the data is already available at this point, so there's no need to do a blocking
+    // wait to wait for more data. This is known because in FMQ, all writes are published (made
+    // available) atomically. Currently, the producer always publishes the entire packet in one
+    // function call, so if the first element of the packet is available, the remaining elements are
+    // also available.
+    const size_t count = mFmqRequestChannel.availableToRead();
     std::vector<FmqRequestDatum> packet(count + 1);
     std::memcpy(&packet.front(), &datum, sizeof(datum));
-    success &= mFmqRequestChannel->read(packet.data() + 1, count);
+    success &= mFmqRequestChannel.read(packet.data() + 1, count);
 
     // terminate loop
     if (mTeardown) {
-        return std::nullopt;
+        return NN_ERROR() << "FMQ object is being torn down";
     }
 
     // ensure packet was successfully received
     if (!success) {
-        LOG(ERROR) << "Error receiving packet";
-        return std::nullopt;
+        return NN_ERROR() << "Error receiving packet";
     }
 
-    return std::make_optional(std::move(packet));
+    return packet;
 }
 
 // ResultChannelSender methods
 
-std::unique_ptr<ResultChannelSender> ResultChannelSender::create(
-        const FmqResultDescriptor& resultChannel) {
-    std::unique_ptr<FmqResultChannel> fmqResultChannel =
-            std::make_unique<FmqResultChannel>(resultChannel);
+nn::GeneralResult<std::unique_ptr<ResultChannelSender>> ResultChannelSender::create(
+        const MQDescriptorSync<FmqResultDatum>& resultChannel) {
+    auto resultChannelSender =
+            std::make_unique<ResultChannelSender>(PrivateConstructorTag{}, resultChannel);
 
-    if (!fmqResultChannel->isValid()) {
-        LOG(ERROR) << "Unable to create RequestChannelSender";
-        return nullptr;
+    if (!resultChannelSender->mFmqResultChannel.isValid()) {
+        return NN_ERROR() << "Unable to create RequestChannelSender";
     }
-    if (fmqResultChannel->getEventFlagWord() == nullptr) {
-        LOG(ERROR) << "ResultChannelSender::create was passed an MQDescriptor without an EventFlag";
-        return nullptr;
+    if (resultChannelSender->mFmqResultChannel.getEventFlagWord() == nullptr) {
+        return NN_ERROR()
+               << "ResultChannelSender::create was passed an MQDescriptor without an EventFlag";
     }
 
-    return std::make_unique<ResultChannelSender>(std::move(fmqResultChannel));
+    return resultChannelSender;
 }
 
-ResultChannelSender::ResultChannelSender(std::unique_ptr<FmqResultChannel> fmqResultChannel)
-    : mFmqResultChannel(std::move(fmqResultChannel)) {}
+ResultChannelSender::ResultChannelSender(PrivateConstructorTag /*tag*/,
+                                         const MQDescriptorSync<FmqResultDatum>& resultChannel)
+    : mFmqResultChannel(resultChannel) {}
 
-bool ResultChannelSender::send(V1_0::ErrorStatus errorStatus,
+void ResultChannelSender::send(V1_0::ErrorStatus errorStatus,
                                const std::vector<V1_2::OutputShape>& outputShapes,
                                V1_2::Timing timing) {
     const std::vector<FmqResultDatum> serialized = serialize(errorStatus, outputShapes, timing);
-    return sendPacket(serialized);
+    sendPacket(serialized);
 }
 
-bool ResultChannelSender::sendPacket(const std::vector<FmqResultDatum>& packet) {
-    if (packet.size() > mFmqResultChannel->availableToWrite()) {
+void ResultChannelSender::sendPacket(const std::vector<FmqResultDatum>& packet) {
+    if (packet.size() > mFmqResultChannel.availableToWrite()) {
         LOG(ERROR)
                 << "ResultChannelSender::sendPacket -- packet size exceeds size available in FMQ";
         const std::vector<FmqResultDatum> errorPacket =
                 serialize(V1_0::ErrorStatus::GENERAL_FAILURE, {}, kNoTiming);
 
-        // Always send the packet with "blocking" because this signals the futex
-        // and unblocks the consumer if it is waiting on the futex.
-        return mFmqResultChannel->writeBlocking(errorPacket.data(), errorPacket.size());
+        // Always send the packet with "blocking" because this signals the futex and unblocks the
+        // consumer if it is waiting on the futex.
+        mFmqResultChannel.writeBlocking(errorPacket.data(), errorPacket.size());
+    } else {
+        // Always send the packet with "blocking" because this signals the futex and unblocks the
+        // consumer if it is waiting on the futex.
+        mFmqResultChannel.writeBlocking(packet.data(), packet.size());
     }
-
-    // Always send the packet with "blocking" because this signals the futex and
-    // unblocks the consumer if it is waiting on the futex.
-    return mFmqResultChannel->writeBlocking(packet.data(), packet.size());
 }
 
 // ResultChannelReceiver methods
 
-std::pair<std::unique_ptr<ResultChannelReceiver>, const FmqResultDescriptor*>
+nn::GeneralResult<
+        std::pair<std::unique_ptr<ResultChannelReceiver>, const MQDescriptorSync<FmqResultDatum>*>>
 ResultChannelReceiver::create(size_t channelLength, std::chrono::microseconds pollingTimeWindow) {
-    std::unique_ptr<FmqResultChannel> fmqResultChannel =
-            std::make_unique<FmqResultChannel>(channelLength, /*confEventFlag=*/true);
-    if (!fmqResultChannel->isValid()) {
-        LOG(ERROR) << "Unable to create ResultChannelReceiver";
-        return {nullptr, nullptr};
+    auto resultChannelReceiver = std::make_unique<ResultChannelReceiver>(
+            PrivateConstructorTag{}, channelLength, pollingTimeWindow);
+    if (!resultChannelReceiver->mFmqResultChannel.isValid()) {
+        return NN_ERROR() << "Unable to create ResultChannelReceiver";
     }
 
-    const FmqResultDescriptor* descriptor = fmqResultChannel->getDesc();
-    return std::make_pair(
-            std::make_unique<ResultChannelReceiver>(std::move(fmqResultChannel), pollingTimeWindow),
-            descriptor);
+    const MQDescriptorSync<FmqResultDatum>* descriptor =
+            resultChannelReceiver->mFmqResultChannel.getDesc();
+    return std::make_pair(std::move(resultChannelReceiver), descriptor);
 }
 
-ResultChannelReceiver::ResultChannelReceiver(std::unique_ptr<FmqResultChannel> fmqResultChannel,
+ResultChannelReceiver::ResultChannelReceiver(PrivateConstructorTag /*tag*/, size_t channelLength,
                                              std::chrono::microseconds pollingTimeWindow)
-    : mFmqResultChannel(std::move(fmqResultChannel)), kPollingTimeWindow(pollingTimeWindow) {}
+    : mFmqResultChannel(channelLength, /*configureEventFlagWord=*/true),
+      kPollingTimeWindow(pollingTimeWindow) {}
 
-std::optional<std::tuple<V1_0::ErrorStatus, std::vector<V1_2::OutputShape>, V1_2::Timing>>
+nn::Result<std::tuple<V1_0::ErrorStatus, std::vector<V1_2::OutputShape>, V1_2::Timing>>
 ResultChannelReceiver::getBlocking() {
-    const auto packet = getPacketBlocking();
-    if (!packet) {
-        return std::nullopt;
-    }
-
-    return deserialize(*packet);
+    const auto packet = NN_TRY(getPacketBlocking());
+    return deserialize(packet);
 }
 
-void ResultChannelReceiver::invalidate() {
+void ResultChannelReceiver::notifyAsDeadObject() {
     mValid = false;
 
     // force unblock
-    // ExecutionBurstController waits on a result packet after sending a
-    // request. If the driver containing ExecutionBurstServer crashes, the
-    // controller may be waiting on the futex. This force unblock wakes up any
-    // thread waiting on the futex.
-    // TODO: look for a different/better way to signal/notify the futex to
-    // wake up any thread waiting on it
-    FmqResultDatum datum;
-    datum.packetInformation({/*.packetSize=*/0,
-                             /*.errorStatus=*/V1_0::ErrorStatus::GENERAL_FAILURE,
-                             /*.numberOfOperands=*/0});
-    mFmqResultChannel->writeBlocking(&datum, 1);
+    // ExecutionBurstController waits on a result packet after sending a request. If the driver
+    // containing ExecutionBurstServer crashes, the controller may be waiting on the futex. This
+    // force unblock wakes up any thread waiting on the futex.
+    const auto data = serialize(V1_0::ErrorStatus::GENERAL_FAILURE, {}, kNoTiming);
+    mFmqResultChannel.writeBlocking(data.data(), data.size());
 }
 
-std::optional<std::vector<FmqResultDatum>> ResultChannelReceiver::getPacketBlocking() {
+nn::Result<std::vector<FmqResultDatum>> ResultChannelReceiver::getPacketBlocking() {
     if (!mValid) {
-        return std::nullopt;
+        return NN_ERROR() << "FMQ object is invalid";
     }
 
-    // First spend time polling if results are available in FMQ instead of
-    // waiting on the futex. Polling is more responsive (yielding lower
-    // latencies), but can take up more power, so only poll for a limited period
-    // of time.
+    // First spend time polling if results are available in FMQ instead of waiting on the futex.
+    // Polling is more responsive (yielding lower latencies), but can take up more power, so only
+    // poll for a limited period of time.
 
     auto& getCurrentTime = std::chrono::high_resolution_clock::now;
     const auto timeToStopPolling = getCurrentTime() + kPollingTimeWindow;
@@ -696,54 +652,49 @@
     while (getCurrentTime() < timeToStopPolling) {
         // if class is being torn down, immediately return
         if (!mValid.load(std::memory_order_relaxed)) {
-            return std::nullopt;
+            return NN_ERROR() << "FMQ object is invalid";
         }
 
-        // Check if data is available. If it is, immediately retrieve it and
-        // return.
-        const size_t available = mFmqResultChannel->availableToRead();
+        // Check if data is available. If it is, immediately retrieve it and return.
+        const size_t available = mFmqResultChannel.availableToRead();
         if (available > 0) {
             std::vector<FmqResultDatum> packet(available);
-            const bool success = mFmqResultChannel->read(packet.data(), available);
+            const bool success = mFmqResultChannel.readBlocking(packet.data(), available);
             if (!success) {
-                LOG(ERROR) << "Error receiving packet";
-                return std::nullopt;
+                return NN_ERROR() << "Error receiving packet";
             }
-            return std::make_optional(std::move(packet));
+            return packet;
         }
     }
 
-    // If we get to this point, we either stopped polling because it was taking
-    // too long or polling was not allowed. Instead, perform a blocking call
-    // which uses a futex to save power.
+    // If we get to this point, we either stopped polling because it was taking too long or polling
+    // was not allowed. Instead, perform a blocking call which uses a futex to save power.
 
     // wait for result packet and read first element of result packet
     FmqResultDatum datum;
-    bool success = mFmqResultChannel->readBlocking(&datum, 1);
+    bool success = mFmqResultChannel.readBlocking(&datum, 1);
 
     // retrieve remaining elements
-    // NOTE: all of the data is already available at this point, so there's no
-    // need to do a blocking wait to wait for more data. This is known because
-    // in FMQ, all writes are published (made available) atomically. Currently,
-    // the producer always publishes the entire packet in one function call, so
-    // if the first element of the packet is available, the remaining elements
-    // are also available.
-    const size_t count = mFmqResultChannel->availableToRead();
+    // NOTE: all of the data is already available at this point, so there's no need to do a blocking
+    // wait to wait for more data. This is known because in FMQ, all writes are published (made
+    // available) atomically. Currently, the producer always publishes the entire packet in one
+    // function call, so if the first element of the packet is available, the remaining elements are
+    // also available.
+    const size_t count = mFmqResultChannel.availableToRead();
     std::vector<FmqResultDatum> packet(count + 1);
     std::memcpy(&packet.front(), &datum, sizeof(datum));
-    success &= mFmqResultChannel->read(packet.data() + 1, count);
+    success &= mFmqResultChannel.read(packet.data() + 1, count);
 
     if (!mValid) {
-        return std::nullopt;
+        return NN_ERROR() << "FMQ object is invalid";
     }
 
     // ensure packet was successfully received
     if (!success) {
-        LOG(ERROR) << "Error receiving packet";
-        return std::nullopt;
+        return NN_ERROR() << "Error receiving packet";
     }
 
-    return std::make_optional(std::move(packet));
+    return packet;
 }
 
 }  // namespace android::hardware::neuralnetworks::V1_2::utils
diff --git a/neuralnetworks/1.2/utils/src/PreparedModel.cpp b/neuralnetworks/1.2/utils/src/PreparedModel.cpp
index 6841c5e..71a4ea8 100644
--- a/neuralnetworks/1.2/utils/src/PreparedModel.cpp
+++ b/neuralnetworks/1.2/utils/src/PreparedModel.cpp
@@ -18,6 +18,8 @@
 
 #include "Callbacks.h"
 #include "Conversions.h"
+#include "ExecutionBurstController.h"
+#include "ExecutionBurstUtils.h"
 #include "Utils.h"
 
 #include <android/hardware/neuralnetworks/1.0/types.h>
@@ -27,12 +29,12 @@
 #include <nnapi/IPreparedModel.h>
 #include <nnapi/Result.h>
 #include <nnapi/Types.h>
-#include <nnapi/hal/1.0/Burst.h>
 #include <nnapi/hal/1.0/Conversions.h>
 #include <nnapi/hal/CommonUtils.h>
 #include <nnapi/hal/HandleError.h>
 #include <nnapi/hal/ProtectCallback.h>
 
+#include <chrono>
 #include <memory>
 #include <tuple>
 #include <utility>
@@ -119,7 +121,14 @@
 }
 
 nn::GeneralResult<nn::SharedBurst> PreparedModel::configureExecutionBurst() const {
-    return V1_0::utils::Burst::create(shared_from_this());
+    auto self = shared_from_this();
+    auto fallback = [preparedModel = std::move(self)](const nn::Request& request,
+                                                      nn::MeasureTiming measure)
+            -> nn::ExecutionResult<std::pair<std::vector<nn::OutputShape>, nn::Timing>> {
+        return preparedModel->execute(request, measure, {}, {});
+    };
+    const auto pollingTimeWindow = getBurstControllerPollingTimeWindow();
+    return ExecutionBurstController::create(kPreparedModel, std::move(fallback), pollingTimeWindow);
 }
 
 std::any PreparedModel::getUnderlyingResource() const {