feature: add host functions workers

* add common host function worker interface
* add worker as a single thread per csr with 3 modes
* add logic for waiting on internal tag, check gpu hang
* if tag is in pending state, read callback data, run callback
and signal completion
* threads will exit the work loop once stop request
is called in finish
* add multi thread unit tests

Related-To: NEO-14577
Signed-off-by: Kamil Kopryk <kamil.kopryk@intel.com>
This commit is contained in:
Kamil Kopryk
2025-10-31 10:59:39 +00:00
committed by Compute-Runtime-Automation
parent ae05e0551e
commit f84a5fbee9
24 changed files with 901 additions and 6 deletions

View File

@@ -30,8 +30,17 @@ set(NEO_CORE_COMMAND_STREAM
${CMAKE_CURRENT_SOURCE_DIR}/definitions${BRANCH_DIR_SUFFIX}stream_properties.inl
${CMAKE_CURRENT_SOURCE_DIR}/device_command_stream.h
${CMAKE_CURRENT_SOURCE_DIR}/host_function.h
${CMAKE_CURRENT_SOURCE_DIR}/host_function.cpp
${CMAKE_CURRENT_SOURCE_DIR}/host_function.inl
${CMAKE_CURRENT_SOURCE_DIR}/host_function_enablers.inl
${CMAKE_CURRENT_SOURCE_DIR}/host_function_worker_cv.h
${CMAKE_CURRENT_SOURCE_DIR}/host_function_worker_cv.cpp
${CMAKE_CURRENT_SOURCE_DIR}/host_function_worker_interface.h
${CMAKE_CURRENT_SOURCE_DIR}/host_function_worker_interface.cpp
${CMAKE_CURRENT_SOURCE_DIR}/host_function_worker_counting_semaphore.cpp
${CMAKE_CURRENT_SOURCE_DIR}/host_function_worker_counting_semaphore.h
${CMAKE_CURRENT_SOURCE_DIR}/host_function_worker_atomic.h
${CMAKE_CURRENT_SOURCE_DIR}/host_function_worker_atomic.cpp
${CMAKE_CURRENT_SOURCE_DIR}/linear_stream.cpp
${CMAKE_CURRENT_SOURCE_DIR}/linear_stream.h
${CMAKE_CURRENT_SOURCE_DIR}/preemption.cpp

View File

@@ -9,6 +9,7 @@
#include "shared/source/command_container/implicit_scaling.h"
#include "shared/source/command_stream/aub_subcapture_status.h"
#include "shared/source/command_stream/host_function_worker_interface.h"
#include "shared/source/command_stream/scratch_space_controller.h"
#include "shared/source/command_stream/submission_status.h"
#include "shared/source/command_stream/submissions_aggregator.h"
@@ -97,6 +98,7 @@ CommandStreamReceiver::CommandStreamReceiver(ExecutionEnvironment &executionEnvi
auto &compilerProductHelper = rootDeviceEnvironment.getHelper<CompilerProductHelper>();
this->heaplessModeEnabled = compilerProductHelper.isHeaplessModeEnabled(hwInfo);
this->heaplessStateInitEnabled = compilerProductHelper.isHeaplessStateInitEnabled(heaplessModeEnabled);
this->hostFunctionWorkerMode = debugManager.flags.HostFunctionWorkMode.get();
}
CommandStreamReceiver::~CommandStreamReceiver() {
@@ -235,6 +237,25 @@ WaitStatus CommandStreamReceiver::waitForTaskCountAndCleanTemporaryAllocationLis
return waitForTaskCountAndCleanAllocationList(requiredTaskCount, TEMPORARY_ALLOCATION);
}
void CommandStreamReceiver::createHostFunctionWorker() {
if (this->hostFunctionWorker != nullptr) {
return;
}
this->hostFunctionWorker = HostFunctionFactory::createHostFunctionWorker(this->hostFunctionWorkerMode,
this->isAubMode(),
this->downloadAllocationImpl,
this->getHostFunctionDataAllocation(),
&this->getHostFunctionData());
this->hostFunctionWorker->start();
}
IHostFunctionWorker *CommandStreamReceiver::getHostFunctionWorker() {
return this->hostFunctionWorker;
}
void CommandStreamReceiver::ensureCommandBufferAllocation(LinearStream &commandStream, size_t minimumRequiredSize, size_t additionalAllocationSize) {
if (commandStream.getAvailableSpace() >= minimumRequiredSize) {
return;
@@ -419,6 +440,10 @@ void CommandStreamReceiver::cleanupResources() {
tagsMultiAllocation = nullptr;
}
if (hostFunctionWorker) {
cleanupHostFunctionWorker();
}
if (hostFunctionDataMultiAllocation) {
hostFunctionDataAllocation = nullptr;
@@ -464,6 +489,12 @@ void CommandStreamReceiver::cleanupResources() {
ownedPrivateAllocations.clear();
}
void CommandStreamReceiver ::cleanupHostFunctionWorker() {
hostFunctionWorker->finish();
delete hostFunctionWorker;
hostFunctionWorker = nullptr;
}
WaitStatus CommandStreamReceiver::waitForCompletionWithTimeout(const WaitParams &params, TaskCountType taskCountToWait) {
bool printWaitForCompletion = debugManager.flags.LogWaitingForCompletion.get();
if (printWaitForCompletion) {
@@ -697,6 +728,10 @@ void *CommandStreamReceiver::getIndirectHeapCurrentPtr(IndirectHeapType heapType
return nullptr;
}
void CommandStreamReceiver::signalHostFunctionWorker() {
hostFunctionWorker->submit();
}
void CommandStreamReceiver::ensureHostFunctionDataInitialization() {
if (!this->hostFunctionInitialized.load(std::memory_order_acquire)) {
initializeHostFunctionData();
@@ -717,6 +752,9 @@ void CommandStreamReceiver::initializeHostFunctionData() {
this->hostFunctionData.entry = reinterpret_cast<decltype(HostFunctionData::entry)>(ptrOffset(hostFunctionBuffer, HostFunctionHelper::entryOffset));
this->hostFunctionData.userData = reinterpret_cast<decltype(HostFunctionData::userData)>(ptrOffset(hostFunctionBuffer, HostFunctionHelper::userDataOffset));
this->hostFunctionData.internalTag = reinterpret_cast<decltype(HostFunctionData::internalTag)>(ptrOffset(hostFunctionBuffer, HostFunctionHelper::internalTagOffset));
createHostFunctionWorker();
this->hostFunctionInitialized.store(true, std::memory_order_release);
}
@@ -968,12 +1006,14 @@ bool CommandStreamReceiver::createPreemptionAllocation() {
std::unique_lock<CommandStreamReceiver::MutexType> CommandStreamReceiver::obtainUniqueOwnership() {
return std::unique_lock<CommandStreamReceiver::MutexType>(this->ownershipMutex);
}
std::unique_lock<CommandStreamReceiver::MutexType> CommandStreamReceiver::tryObtainUniqueOwnership() {
return std::unique_lock<CommandStreamReceiver::MutexType>(this->ownershipMutex, std::try_to_lock);
}
std::unique_lock<CommandStreamReceiver::MutexType> CommandStreamReceiver::obtainHostPtrSurfaceCreationLock() {
return std::unique_lock<CommandStreamReceiver::MutexType>(this->hostPtrSurfaceCreationMutex);
}
AllocationsList &CommandStreamReceiver::getTemporaryAllocations() { return internalAllocationStorage->getTemporaryAllocations(); }
AllocationsList &CommandStreamReceiver::getAllocationsForReuse() { return internalAllocationStorage->getAllocationsForReuse(); }
AllocationsList &CommandStreamReceiver::getDeferredAllocations() { return internalAllocationStorage->getDeferredAllocations(); }

View File

@@ -67,6 +67,7 @@ class KmdNotifyHelper;
class GfxCoreHelper;
class ProductHelper;
class ReleaseHelper;
class IHostFunctionWorker;
enum class WaitStatus;
struct AubSubCaptureStatus;
class SharedPoolAllocation;
@@ -148,6 +149,8 @@ class CommandStreamReceiver : NEO::NonCopyableAndNonMovableClass {
MOCKABLE_VIRTUAL WaitStatus waitForTaskCount(TaskCountType requiredTaskCount);
WaitStatus waitForTaskCountAndCleanAllocationList(TaskCountType requiredTaskCount, uint32_t allocationUsage);
MOCKABLE_VIRTUAL WaitStatus waitForTaskCountAndCleanTemporaryAllocationList(TaskCountType requiredTaskCount);
MOCKABLE_VIRTUAL void createHostFunctionWorker();
IHostFunctionWorker *getHostFunctionWorker();
LinearStream &getCS(size_t minRequiredSize = 1024u);
OSInterface *getOSInterface() const;
@@ -270,7 +273,6 @@ class CommandStreamReceiver : NEO::NonCopyableAndNonMovableClass {
MOCKABLE_VIRTUAL bool createPerDssBackedBuffer(Device &device);
[[nodiscard]] MOCKABLE_VIRTUAL std::unique_lock<MutexType> obtainUniqueOwnership();
[[nodiscard]] MOCKABLE_VIRTUAL std::unique_lock<MutexType> tryObtainUniqueOwnership();
bool peekTimestampPacketWriteEnabled() const { return timestampPacketWriteEnabled; }
bool isLatestTaskCountFlushed() {
@@ -567,6 +569,7 @@ class CommandStreamReceiver : NEO::NonCopyableAndNonMovableClass {
bool isLatestFlushIsTaskCountUpdateOnly() const { return latestFlushIsTaskCountUpdateOnly; }
MOCKABLE_VIRTUAL uint32_t getContextGroupId() const;
MOCKABLE_VIRTUAL void signalHostFunctionWorker();
void ensureHostFunctionDataInitialization();
HostFunctionData &getHostFunctionData();
@@ -584,6 +587,7 @@ class CommandStreamReceiver : NEO::NonCopyableAndNonMovableClass {
TaskCountType taskLevel, DispatchFlags &dispatchFlags, Device &device) = 0;
void cleanupResources();
void cleanupHostFunctionWorker();
void printDeviceIndex();
void checkForNewResources(TaskCountType submittedTaskCount, TaskCountType allocationTaskCount, GraphicsAllocation &gfxAllocation);
bool checkImplicitFlushForGpuIdle();
@@ -610,6 +614,7 @@ class CommandStreamReceiver : NEO::NonCopyableAndNonMovableClass {
std::unique_ptr<TagAllocatorBase> timestampPacketAllocator;
std::unique_ptr<Thread> userPauseConfirmation;
std::unique_ptr<IndirectHeap> globalStatelessHeap;
IHostFunctionWorker *hostFunctionWorker = nullptr;
ResidencyContainer residencyAllocations;
PrivateAllocsToReuseContainer ownedPrivateAllocations;
@@ -664,7 +669,6 @@ class CommandStreamReceiver : NEO::NonCopyableAndNonMovableClass {
std::atomic<TaskCountType> taskCount{0};
std::atomic<uint32_t> numClients = 0u;
DispatchMode dispatchMode = DispatchMode::immediateDispatch;
SamplerCacheFlushState samplerCacheFlushRequired = SamplerCacheFlushState::samplerCacheFlushNotRequired;
PreemptionMode lastPreemptionMode = PreemptionMode::Initial;
@@ -673,7 +677,7 @@ class CommandStreamReceiver : NEO::NonCopyableAndNonMovableClass {
uint32_t lastSentL3Config = 0;
uint32_t latestSentStatelessMocsConfig;
uint64_t lastSentSliceCount;
int32_t hostFunctionWorkerMode = -1;
uint32_t requiredScratchSlot0Size = 0;
uint32_t requiredScratchSlot1Size = 0;
uint32_t lastAdditionalKernelExecInfo;

View File

@@ -0,0 +1,37 @@
/*
* Copyright (C) 2025 Intel Corporation
*
* SPDX-License-Identifier: MIT
*
*/
#include "shared/source/command_stream/host_function.h"
#include "shared/source/command_stream/command_stream_receiver.h"
#include "shared/source/command_stream/host_function_worker_atomic.h"
#include "shared/source/command_stream/host_function_worker_counting_semaphore.h"
#include "shared/source/command_stream/host_function_worker_cv.h"
#include "shared/source/command_stream/host_function_worker_interface.h"
namespace NEO::HostFunctionFactory {
IHostFunctionWorker *createHostFunctionWorker(int32_t hostFunctionWorkerMode,
bool isAubMode,
const std::function<void(GraphicsAllocation &)> &downloadAllocationImpl,
GraphicsAllocation *allocation,
HostFunctionData *data) {
bool skipHostFunctionExecution = isAubMode;
switch (hostFunctionWorkerMode) {
default:
case 0:
return new HostFunctionWorkerCountingSemaphore(skipHostFunctionExecution, downloadAllocationImpl, allocation, data);
case 1:
return new HostFunctionWorkerCV(skipHostFunctionExecution, downloadAllocationImpl, allocation, data);
case 2:
return new HostFunctionWorkerAtomic(skipHostFunctionExecution, downloadAllocationImpl, allocation, data);
}
}
} // namespace NEO::HostFunctionFactory

View File

@@ -9,10 +9,14 @@
#include <cstddef>
#include <cstdint>
#include <functional>
namespace NEO {
class LinearStream;
class CommandStreamReceiver;
class IHostFunctionWorker;
class GraphicsAllocation;
struct HostFunctionData {
volatile uint64_t *entry = nullptr;
@@ -47,4 +51,13 @@ struct HostFunctionHelper {
static void programWaitForHostFunctionCompletion(LinearStream *commandStream, void *cmdBuffer, const HostFunctionData &hostFunctionData);
};
namespace HostFunctionFactory {
IHostFunctionWorker *createHostFunctionWorker(int32_t hostFunctionWorkerMode,
bool isAubMode,
const std::function<void(GraphicsAllocation &)> &downloadAllocationImpl,
GraphicsAllocation *allocation,
HostFunctionData *data);
}
} // namespace NEO

View File

@@ -0,0 +1,68 @@
/*
* Copyright (C) 2025 Intel Corporation
*
* SPDX-License-Identifier: MIT
*
*/
#include "shared/source/command_stream/host_function_worker_atomic.h"
#include "shared/source/command_stream/host_function.h"
namespace NEO {
HostFunctionWorkerAtomic::HostFunctionWorkerAtomic(bool skipHostFunctionExecution,
const std::function<void(GraphicsAllocation &)> &downloadAllocationImpl,
GraphicsAllocation *allocation,
HostFunctionData *data)
: IHostFunctionWorker(skipHostFunctionExecution, downloadAllocationImpl, allocation, data) {
}
HostFunctionWorkerAtomic::~HostFunctionWorkerAtomic() = default;
void HostFunctionWorkerAtomic::start() {
std::lock_guard<std::mutex> lg{workerMutex};
if (!worker) {
worker = std::make_unique<std::jthread>([this](std::stop_token st) {
this->workerLoop(st);
});
}
}
void HostFunctionWorkerAtomic::finish() {
std::lock_guard<std::mutex> lg{workerMutex};
if (worker) {
worker->request_stop();
pending.fetch_add(1u);
pending.notify_one();
worker.reset(nullptr);
}
}
void HostFunctionWorkerAtomic::submit() noexcept {
pending.fetch_add(1, std::memory_order_release);
pending.notify_one();
}
void HostFunctionWorkerAtomic::workerLoop(std::stop_token st) noexcept {
while (true) {
while (pending.load(std::memory_order_acquire) == 0) {
pending.wait(0, std::memory_order_acquire);
}
if (st.stop_requested()) {
return;
}
pending.fetch_sub(1, std::memory_order_acq_rel);
bool sucess = this->runHostFunction(st);
if (!sucess) [[unlikely]] {
return;
}
}
}
} // namespace NEO

View File

@@ -0,0 +1,38 @@
/*
* Copyright (C) 2025 Intel Corporation
*
* SPDX-License-Identifier: MIT
*
*/
#pragma once
#include "shared/source/command_stream/host_function_worker_interface.h"
#include <atomic>
#include <memory>
#include <thread>
namespace NEO {
class HostFunctionWorkerAtomic : public IHostFunctionWorker {
public:
HostFunctionWorkerAtomic(bool skipHostFunctionExecution,
const std::function<void(GraphicsAllocation &)> &downloadAllocationImpl,
GraphicsAllocation *allocation,
HostFunctionData *data);
~HostFunctionWorkerAtomic() override;
void start() override;
void finish() override;
void submit() noexcept override;
private:
void workerLoop(std::stop_token st) noexcept;
std::atomic<uint32_t> pending{0};
};
static_assert(NonCopyableAndNonMovable<HostFunctionWorkerAtomic>);
} // namespace NEO

View File

@@ -0,0 +1,59 @@
/*
* Copyright (C) 2025 Intel Corporation
*
* SPDX-License-Identifier: MIT
*
*/
#include "shared/source/command_stream/host_function_worker_counting_semaphore.h"
namespace NEO {
HostFunctionWorkerCountingSemaphore::HostFunctionWorkerCountingSemaphore(bool skipHostFunctionExecution, const std::function<void(GraphicsAllocation &)> &downloadAllocationImpl, GraphicsAllocation *allocation, HostFunctionData *data)
: IHostFunctionWorker(skipHostFunctionExecution, downloadAllocationImpl, allocation, data) {
}
HostFunctionWorkerCountingSemaphore::~HostFunctionWorkerCountingSemaphore() = default;
void HostFunctionWorkerCountingSemaphore::start() {
std::lock_guard<std::mutex> lg{workerMutex};
if (!worker) {
worker = std::make_unique<std::jthread>([this](std::stop_token st) {
this->workerLoop(st);
});
}
}
void HostFunctionWorkerCountingSemaphore::finish() {
std::lock_guard<std::mutex> lg{workerMutex};
if (worker) {
worker->request_stop();
semaphore.release();
worker.reset(nullptr);
}
}
void HostFunctionWorkerCountingSemaphore::submit() noexcept {
semaphore.release();
}
void HostFunctionWorkerCountingSemaphore::workerLoop(std::stop_token st) noexcept {
while (true) {
semaphore.acquire();
if (st.stop_requested()) [[unlikely]] {
return;
}
bool success = runHostFunction(st);
if (!success) [[unlikely]] {
return;
}
}
}
} // namespace NEO

View File

@@ -0,0 +1,38 @@
/*
* Copyright (C) 2025 Intel Corporation
*
* SPDX-License-Identifier: MIT
*
*/
#pragma once
#include "shared/source/command_stream/host_function_worker_interface.h"
#include <mutex>
#include <semaphore>
#include <thread>
namespace NEO {
class HostFunctionWorkerCountingSemaphore : public IHostFunctionWorker {
public:
HostFunctionWorkerCountingSemaphore(bool skipHostFunctionExecution,
const std::function<void(GraphicsAllocation &)> &downloadAllocationImpl,
GraphicsAllocation *allocation,
HostFunctionData *data);
~HostFunctionWorkerCountingSemaphore() override;
void start() override;
void finish() override;
void submit() noexcept override;
private:
void workerLoop(std::stop_token st) noexcept;
std::counting_semaphore<> semaphore{0};
};
static_assert(NonCopyableAndNonMovable<HostFunctionWorkerCountingSemaphore>);
} // namespace NEO

View File

@@ -0,0 +1,75 @@
/*
* Copyright (C) 2025 Intel Corporation
*
* SPDX-License-Identifier: MIT
*
*/
#include "shared/source/command_stream/host_function_worker_cv.h"
#include "shared/source/command_stream/command_stream_receiver.h"
#include "shared/source/command_stream/host_function.h"
#include "shared/source/utilities/wait_util.h"
namespace NEO {
HostFunctionWorkerCV::HostFunctionWorkerCV(bool skipHostFunctionExecution,
const std::function<void(GraphicsAllocation &)> &downloadAllocationImpl,
GraphicsAllocation *allocation,
HostFunctionData *data)
: IHostFunctionWorker(skipHostFunctionExecution, downloadAllocationImpl, allocation, data) {
}
HostFunctionWorkerCV::~HostFunctionWorkerCV() = default;
void HostFunctionWorkerCV::start() {
std::lock_guard<std::mutex> lg{workerMutex};
if (!worker) {
worker = std::make_unique<std::jthread>([this](std::stop_token st) {
this->workerLoop(st);
});
}
}
void HostFunctionWorkerCV::finish() {
std::lock_guard<std::mutex> lg{workerMutex};
if (worker) {
worker->request_stop();
cv.notify_one();
worker.reset(nullptr);
}
}
void HostFunctionWorkerCV::submit() noexcept {
{
std::lock_guard<std::mutex> lock{pendingAccessMutex};
++pending;
}
cv.notify_one();
}
void HostFunctionWorkerCV::workerLoop(std::stop_token st) noexcept {
std::unique_lock<std::mutex> lock{pendingAccessMutex, std::defer_lock};
while (true) {
lock.lock();
cv.wait(lock, [&]() {
return pending > 0 || st.stop_requested();
});
if (st.stop_requested()) [[unlikely]] {
return;
}
--pending;
lock.unlock();
bool sucess = this->runHostFunction(st);
if (!sucess) [[unlikely]] {
return;
}
}
}
} // namespace NEO

View File

@@ -0,0 +1,39 @@
/*
* Copyright (C) 2025 Intel Corporation
*
* SPDX-License-Identifier: MIT
*
*/
#pragma once
#include "shared/source/command_stream/host_function_worker_interface.h"
#include <condition_variable>
#include <mutex>
#include <thread>
namespace NEO {
class HostFunctionWorkerCV : public IHostFunctionWorker {
public:
HostFunctionWorkerCV(bool skipHostFunctionExecution,
const std::function<void(GraphicsAllocation &)> &downloadAllocationImpl,
GraphicsAllocation *allocation,
HostFunctionData *data);
~HostFunctionWorkerCV() override;
void start() override;
void finish() override;
void submit() noexcept override;
private:
void workerLoop(std::stop_token st) noexcept;
std::mutex pendingAccessMutex;
std::condition_variable cv;
uint32_t pending{0};
};
static_assert(NonCopyableAndNonMovable<HostFunctionWorkerCV>);
} // namespace NEO

View File

@@ -0,0 +1,68 @@
/*
* Copyright (C) 2025 Intel Corporation
*
* SPDX-License-Identifier: MIT
*
*/
#include "shared/source/command_stream/host_function_worker_interface.h"
#include "shared/source/command_stream/host_function.h"
#include "shared/source/utilities/wait_util.h"
#include <chrono>
#include <type_traits>
namespace NEO {
IHostFunctionWorker::IHostFunctionWorker(bool skipHostFunctionExecution,
const std::function<void(GraphicsAllocation &)> &downloadAllocationImpl,
GraphicsAllocation *allocation,
HostFunctionData *data)
: downloadAllocationImpl(downloadAllocationImpl),
allocation(allocation),
data(data),
skipHostFunctionExecution(skipHostFunctionExecution) {
}
IHostFunctionWorker::~IHostFunctionWorker() = default;
bool IHostFunctionWorker::runHostFunction(std::stop_token st) noexcept {
using tagStatusT = std::underlying_type_t<HostFunctionTagStatus>;
const auto start = std::chrono::steady_clock::now();
std::chrono::microseconds waitTime{0};
if (!this->skipHostFunctionExecution) {
while (true) {
if (this->downloadAllocationImpl) [[unlikely]] {
this->downloadAllocationImpl(*this->allocation);
}
const volatile uint32_t *hostFuntionTagAddress = this->data->internalTag;
waitTime = std::chrono::duration_cast<std::chrono::microseconds>(std::chrono::steady_clock::now() - start);
bool pendingJobFound = WaitUtils::waitFunctionWithPredicate<const tagStatusT>(hostFuntionTagAddress,
static_cast<tagStatusT>(HostFunctionTagStatus::pending),
std::equal_to<tagStatusT>(),
waitTime.count());
if (pendingJobFound) {
break;
}
if (st.stop_requested()) {
return false;
}
}
using CallbackT = void (*)(void *);
CallbackT callback = reinterpret_cast<CallbackT>(*this->data->entry);
void *callbackData = reinterpret_cast<void *>(*this->data->userData);
callback(callbackData);
}
*this->data->internalTag = static_cast<tagStatusT>(HostFunctionTagStatus::completed);
return true;
}
} // namespace NEO

View File

@@ -0,0 +1,49 @@
/*
* Copyright (C) 2025 Intel Corporation
*
* SPDX-License-Identifier: MIT
*
*/
#pragma once
#include "shared/source/helpers/non_copyable_or_moveable.h"
#include <functional>
#include <memory>
#include <mutex>
#include <stop_token>
#include <thread>
namespace NEO {
class GraphicsAllocation;
struct HostFunctionData;
class IHostFunctionWorker : public NonCopyableAndNonMovableClass {
public:
IHostFunctionWorker(bool skipHostFunctionExecution,
const std::function<void(GraphicsAllocation &)> &downloadAllocationImpl,
GraphicsAllocation *allocation,
HostFunctionData *data);
virtual ~IHostFunctionWorker() = 0;
virtual void start() = 0;
virtual void finish() = 0;
virtual void submit() noexcept = 0;
protected:
MOCKABLE_VIRTUAL bool runHostFunction(std::stop_token st) noexcept;
std::unique_ptr<std::jthread> worker;
std::mutex workerMutex;
private:
std::function<void(GraphicsAllocation &)> downloadAllocationImpl;
GraphicsAllocation *allocation = nullptr;
HostFunctionData *data = nullptr;
bool skipHostFunctionExecution = false;
};
static_assert(NonCopyableAndNonMovable<IHostFunctionWorker>);
} // namespace NEO

View File

@@ -320,6 +320,7 @@ DECLARE_DEBUG_VARIABLE(int32_t, OverrideCopyOffloadMode, -1, "-1: default, 0: di
DECLARE_DEBUG_VARIABLE(int32_t, UseSingleListForTemporaryAllocations, -1, "-1: default, 0: disabled, 0: enabled. If enabled, use single list, instead of per CSR for tracking temporary allocations")
DECLARE_DEBUG_VARIABLE(int32_t, OverrideMaxMemAllocSizeMb, -1, "-1: default, >=0 override reported max mem alloc size in MB")
DECLARE_DEBUG_VARIABLE(int32_t, DetectIncorrectPointersOnSetArgCalls, -1, "-1: default do not detect, 0: do not detect, 1: detect incorrect pointers and return error")
DECLARE_DEBUG_VARIABLE(int32_t, HostFunctionWorkMode, -1, "-1: default - counting semaphore based, 0: counting semaphore based, 1: condition variable base, 2: atomics based")
DECLARE_DEBUG_VARIABLE(int32_t, ForceDisableGraphPatchPreamble, -1, "-1: default, 0: enable patch preamble, 1: disable graph patch preamble. If disabled, do not patch preamble graph internal command lists")
/*LOGGING FLAGS*/