diff --git a/source/adapters/native_cpu/common.hpp b/source/adapters/native_cpu/common.hpp index 2b4aabfbad..af0d11c5af 100644 --- a/source/adapters/native_cpu/common.hpp +++ b/source/adapters/native_cpu/common.hpp @@ -12,6 +12,7 @@ #include "logger/ur_logger.hpp" #include "ur/ur.hpp" +#include constexpr size_t MaxMessageSize = 256; @@ -70,3 +71,31 @@ template inline void decrementOrDelete(T *refC) { if (refC->decrementReferenceCount() == 0) delete refC; } + +inline uint64_t get_timestamp() { + return std::chrono::duration_cast( + std::chrono::high_resolution_clock::now().time_since_epoch()) + .count(); +} + +namespace native_cpu { + +inline void *aligned_malloc(size_t alignment, size_t size) { + void *ptr = nullptr; +#ifdef _MSC_VER + ptr = _aligned_malloc(size, alignment); +#else + ptr = std::aligned_alloc(alignment, size); +#endif + return ptr; +} + +inline void aligned_free(void *ptr) { +#ifdef _MSC_VER + _aligned_free(ptr); +#else + free(ptr); +#endif +} + +} // namespace native_cpu diff --git a/source/adapters/native_cpu/context.hpp b/source/adapters/native_cpu/context.hpp index c59ab4eafb..b9d2d22dd1 100644 --- a/source/adapters/native_cpu/context.hpp +++ b/source/adapters/native_cpu/context.hpp @@ -64,17 +64,10 @@ static size_t get_padding(uint32_t alignment) { // allocation so that the pointer returned to the user // always satisfies (ptr % align) == 0. static inline void *malloc_impl(uint32_t alignment, size_t size) { - void *ptr = nullptr; assert(alignment >= alignof(usm_alloc_info) && "memory not aligned to usm_alloc_info"); -#ifdef _MSC_VER - ptr = _aligned_malloc(alloc_header_size + get_padding(alignment) + size, - alignment); - -#else - ptr = std::aligned_alloc(alignment, - alloc_header_size + get_padding(alignment) + size); -#endif + void *ptr = native_cpu::aligned_malloc( + alignment, alloc_header_size + get_padding(alignment) + size); return ptr; } @@ -100,11 +93,8 @@ struct ur_context_handle_t_ : RefCounted { const native_cpu::usm_alloc_info &info = native_cpu::get_alloc_info(ptr); UR_ASSERT(info.type != UR_USM_TYPE_UNKNOWN, UR_RESULT_ERROR_INVALID_MEM_OBJECT); -#ifdef _MSC_VER - _aligned_free(info.base_alloc_ptr); -#else - free(info.base_alloc_ptr); -#endif + + native_cpu::aligned_free(info.base_alloc_ptr); allocations.erase(ptr); return UR_RESULT_SUCCESS; } diff --git a/source/adapters/native_cpu/device.cpp b/source/adapters/native_cpu/device.cpp index 0061fae907..e2d31c120a 100644 --- a/source/adapters/native_cpu/device.cpp +++ b/source/adapters/native_cpu/device.cpp @@ -10,6 +10,7 @@ #include +#include "common.hpp" #include "platform.hpp" #if defined(_MSC_VER) || defined(__MINGW32__) || defined(__MINGW64__) @@ -244,7 +245,6 @@ UR_APIEXPORT ur_result_t UR_APICALL urDeviceGetInfo(ur_device_handle_t hDevice, return ReturnValue(uint32_t{4}); case UR_DEVICE_INFO_NATIVE_VECTOR_WIDTH_HALF: return ReturnValue(uint32_t{16}); - // Imported from level_zero case UR_DEVICE_INFO_USM_HOST_SUPPORT: case UR_DEVICE_INFO_USM_DEVICE_SUPPORT: case UR_DEVICE_INFO_USM_SINGLE_SHARED_SUPPORT: @@ -464,19 +464,12 @@ UR_APIEXPORT ur_result_t UR_APICALL urDeviceCreateWithNativeHandle( UR_APIEXPORT ur_result_t UR_APICALL urDeviceGetGlobalTimestamps( ur_device_handle_t hDevice, uint64_t *pDeviceTimestamp, uint64_t *pHostTimestamp) { - std::ignore = hDevice; // todo + std::ignore = hDevice; if (pHostTimestamp) { - using namespace std::chrono; - *pHostTimestamp = - duration_cast(steady_clock::now().time_since_epoch()) - .count(); + *pHostTimestamp = get_timestamp(); } if (pDeviceTimestamp) { - // todo: calculate elapsed time properly - using namespace std::chrono; - *pDeviceTimestamp = - duration_cast(steady_clock::now().time_since_epoch()) - .count(); + *pDeviceTimestamp = get_timestamp(); } return UR_RESULT_SUCCESS; } diff --git a/source/adapters/native_cpu/enqueue.cpp b/source/adapters/native_cpu/enqueue.cpp index 33d8c35c36..7e03b323cc 100644 --- a/source/adapters/native_cpu/enqueue.cpp +++ b/source/adapters/native_cpu/enqueue.cpp @@ -13,6 +13,7 @@ #include "ur_api.h" #include "common.hpp" +#include "event.hpp" #include "kernel.hpp" #include "memory.hpp" #include "queue.hpp" @@ -67,10 +68,8 @@ UR_APIEXPORT ur_result_t UR_APICALL urEnqueueKernelLaunch( const size_t *pGlobalWorkOffset, const size_t *pGlobalWorkSize, const size_t *pLocalWorkSize, uint32_t numEventsInWaitList, const ur_event_handle_t *phEventWaitList, ur_event_handle_t *phEvent) { - std::ignore = numEventsInWaitList; - std::ignore = phEventWaitList; - std::ignore = phEvent; + urEventWait(numEventsInWaitList, phEventWaitList); UR_ASSERT(hQueue, UR_RESULT_ERROR_INVALID_NULL_HANDLE); UR_ASSERT(hKernel, UR_RESULT_ERROR_INVALID_NULL_HANDLE); UR_ASSERT(pGlobalWorkOffset, UR_RESULT_ERROR_INVALID_NULL_POINTER); @@ -103,10 +102,9 @@ UR_APIEXPORT ur_result_t UR_APICALL urEnqueueKernelLaunch( } // TODO: add proper error checking - // TODO: add proper event dep management native_cpu::NDRDescT ndr(workDim, pGlobalWorkOffset, pGlobalWorkSize, pLocalWorkSize); - auto &tp = hQueue->device->tp; + auto &tp = hQueue->getDevice()->tp; const size_t numParallelThreads = tp.num_threads(); hKernel->updateMemPool(numParallelThreads); std::vector> futures; @@ -118,6 +116,9 @@ UR_APIEXPORT ur_result_t UR_APICALL urEnqueueKernelLaunch( ndr.GlobalSize[2], ndr.LocalSize[0], ndr.LocalSize[1], ndr.LocalSize[2], ndr.GlobalOffset[0], ndr.GlobalOffset[1], ndr.GlobalOffset[2]); + auto event = new ur_event_handle_t_(hQueue, UR_COMMAND_KERNEL_LAUNCH); + event->tick_start(); + #ifndef NATIVECPU_USE_OCK hKernel->handleLocalArgs(1, 0); for (unsigned g2 = 0; g2 < numWG2; g2++) { @@ -127,7 +128,7 @@ UR_APIEXPORT ur_result_t UR_APICALL urEnqueueKernelLaunch( for (unsigned local1 = 0; local1 < ndr.LocalSize[1]; local1++) { for (unsigned local0 = 0; local0 < ndr.LocalSize[0]; local0++) { state.update(g0, g1, g2, local0, local1, local2); - hKernel->_subhandler(hKernel->_args.data(), &state); + hKernel->_subhandler(hKernel->getArgs().data(), &state); } } } @@ -158,13 +159,12 @@ UR_APIEXPORT ur_result_t UR_APICALL urEnqueueKernelLaunch( for (unsigned g2 = 0; g2 < numWG2; g2++) { for (unsigned g1 = 0; g1 < numWG1; g1++) { for (unsigned g0 = 0; g0 < new_num_work_groups_0; g0 += 1) { - futures.emplace_back( - tp.schedule_task([&ndr = std::as_const(ndr), itemsPerThread, - hKernel, g0, g1, g2](size_t) { + futures.emplace_back(tp.schedule_task( + [ndr, itemsPerThread, kernel = *hKernel, g0, g1, g2](size_t) { native_cpu::state resized_state = getResizedState(ndr, itemsPerThread); resized_state.update(g0, g1, g2); - hKernel->_subhandler(hKernel->_args.data(), &resized_state); + kernel._subhandler(kernel.getArgs().data(), &resized_state); })); } // Peel the remaining work items. Since the local size is 1, we iterate @@ -172,7 +172,7 @@ UR_APIEXPORT ur_result_t UR_APICALL urEnqueueKernelLaunch( for (unsigned g0 = new_num_work_groups_0 * itemsPerThread; g0 < numWG0; g0++) { state.update(g0, g1, g2); - hKernel->_subhandler(hKernel->_args.data(), &state); + hKernel->_subhandler(hKernel->getArgs().data(), &state); } } } @@ -190,7 +190,7 @@ UR_APIEXPORT ur_result_t UR_APICALL urEnqueueKernelLaunch( for (unsigned g0 = 0; g0 < numWG0; g0++) { kernel.handleLocalArgs(numParallelThreads, threadId); state.update(g0, g1, g2); - kernel._subhandler(kernel._args.data(), &state); + kernel._subhandler(kernel.getArgs().data(), &state); } })); } @@ -207,7 +207,7 @@ UR_APIEXPORT ur_result_t UR_APICALL urEnqueueKernelLaunch( size_t threadId, ur_kernel_handle_t_ kernel) mutable { kernel.handleLocalArgs(numParallelThreads, threadId); state.update(g0, g1, g2); - kernel._subhandler(kernel._args.data(), &state); + kernel._subhandler(kernel.getArgs().data(), &state); }); } } @@ -216,11 +216,12 @@ UR_APIEXPORT ur_result_t UR_APICALL urEnqueueKernelLaunch( auto groupsPerThread = numGroups / numParallelThreads; auto remainder = numGroups % numParallelThreads; for (unsigned thread = 0; thread < numParallelThreads; thread++) { - futures.emplace_back(tp.schedule_task( - [&groups, thread, groupsPerThread, hKernel](size_t threadId) { + futures.emplace_back( + tp.schedule_task([groups, thread, groupsPerThread, + kernel = *hKernel](size_t threadId) { for (unsigned i = 0; i < groupsPerThread; i++) { auto index = thread * groupsPerThread + i; - groups[index](threadId, *hKernel); + groups[index](threadId, kernel); } })); } @@ -228,25 +229,32 @@ UR_APIEXPORT ur_result_t UR_APICALL urEnqueueKernelLaunch( // schedule the remaining tasks if (remainder) { futures.emplace_back( - tp.schedule_task([&groups, remainder, + tp.schedule_task([groups, remainder, scheduled = numParallelThreads * groupsPerThread, - hKernel](size_t threadId) { + kernel = *hKernel](size_t threadId) { for (unsigned i = 0; i < remainder; i++) { auto index = scheduled + i; - groups[index](threadId, *hKernel); + groups[index](threadId, kernel); } })); } } } - for (auto &f : futures) - f.get(); #endif // NATIVECPU_USE_OCK - // TODO: we should avoid calling clear here by avoiding using push_back - // in setKernelArgs. - hKernel->_args.clear(); - hKernel->_localArgInfo.clear(); + event->set_futures(futures); + + *phEvent = event; + event->set_callback([hKernel, event]() { + event->tick_end(); + // TODO: avoid calling clear() here. + hKernel->_localArgInfo.clear(); + }); + + if (hQueue->isInOrder()) { + urEventWait(1, phEvent); + } + return UR_RESULT_SUCCESS; } @@ -274,15 +282,23 @@ UR_APIEXPORT ur_result_t UR_APICALL urEnqueueEventsWaitWithBarrier( template static inline ur_result_t enqueueMemBufferReadWriteRect_impl( - ur_queue_handle_t, ur_mem_handle_t Buff, bool, + ur_queue_handle_t hQueue, ur_mem_handle_t Buff, bool, ur_rect_offset_t BufferOffset, ur_rect_offset_t HostOffset, ur_rect_region_t region, size_t BufferRowPitch, size_t BufferSlicePitch, size_t HostRowPitch, size_t HostSlicePitch, typename std::conditional::type DstMem, - uint32_t, const ur_event_handle_t *, ur_event_handle_t *) { - // TODO: events, blocking, check other constraints, performance optimizations + uint32_t NumEventsInWaitList, const ur_event_handle_t *phEventWaitList, + ur_event_handle_t *phEvent) { + ur_event_handle_t event; + if constexpr (IsRead) + event = new ur_event_handle_t_(hQueue, UR_COMMAND_MEM_BUFFER_READ_RECT); + else + event = new ur_event_handle_t_(hQueue, UR_COMMAND_MEM_BUFFER_WRITE_RECT); + event->tick_start(); + // TODO: blocking, check other constraints, performance optimizations // More sharing with level_zero where possible + urEventWait(NumEventsInWaitList, phEventWaitList); if (BufferRowPitch == 0) BufferRowPitch = region.width; if (BufferSlicePitch == 0) @@ -306,21 +322,26 @@ static inline ur_result_t enqueueMemBufferReadWriteRect_impl( else buff_mem = ur_cast(DstMem)[host_origin]; } + + event->tick_end(); + *phEvent = event; return UR_RESULT_SUCCESS; } static inline ur_result_t doCopy_impl(ur_queue_handle_t hQueue, void *DstPtr, const void *SrcPtr, size_t Size, uint32_t numEventsInWaitList, - const ur_event_handle_t *EventWaitList, - ur_event_handle_t *Event) { - // todo: non-blocking, events, UR integration - std::ignore = EventWaitList; - std::ignore = Event; - std::ignore = hQueue; - std::ignore = numEventsInWaitList; + const ur_event_handle_t *phEventWaitList, + ur_event_handle_t *phEvent, + ur_command_t command_type) { + ur_event_handle_t event = new ur_event_handle_t_(hQueue, command_type); + event->tick_start(); + urEventWait(numEventsInWaitList, phEventWaitList); if (SrcPtr != DstPtr && Size) memmove(DstPtr, SrcPtr, Size); + event->tick_end(); + if (phEvent) + *phEvent = event; return UR_RESULT_SUCCESS; } @@ -331,8 +352,9 @@ UR_APIEXPORT ur_result_t UR_APICALL urEnqueueMemBufferRead( std::ignore = blockingRead; void *FromPtr = /*Src*/ hBuffer->_mem + offset; - return doCopy_impl(hQueue, pDst, FromPtr, size, numEventsInWaitList, - phEventWaitList, phEvent); + auto res = doCopy_impl(hQueue, pDst, FromPtr, size, numEventsInWaitList, + phEventWaitList, phEvent, UR_COMMAND_MEM_BUFFER_READ); + return res; } UR_APIEXPORT ur_result_t UR_APICALL urEnqueueMemBufferWrite( @@ -342,8 +364,9 @@ UR_APIEXPORT ur_result_t UR_APICALL urEnqueueMemBufferWrite( std::ignore = blockingWrite; void *ToPtr = hBuffer->_mem + offset; - return doCopy_impl(hQueue, ToPtr, pSrc, size, numEventsInWaitList, - phEventWaitList, phEvent); + auto res = doCopy_impl(hQueue, ToPtr, pSrc, size, numEventsInWaitList, + phEventWaitList, phEvent, UR_COMMAND_MEM_BUFFER_WRITE); + return res; } UR_APIEXPORT ur_result_t UR_APICALL urEnqueueMemBufferReadRect( @@ -377,10 +400,11 @@ UR_APIEXPORT ur_result_t UR_APICALL urEnqueueMemBufferCopy( ur_mem_handle_t hBufferDst, size_t srcOffset, size_t dstOffset, size_t size, uint32_t numEventsInWaitList, const ur_event_handle_t *phEventWaitList, ur_event_handle_t *phEvent) { + urEventWait(numEventsInWaitList, phEventWaitList); const void *SrcPtr = hBufferSrc->_mem + srcOffset; void *DstPtr = hBufferDst->_mem + dstOffset; return doCopy_impl(hQueue, DstPtr, SrcPtr, size, numEventsInWaitList, - phEventWaitList, phEvent); + phEventWaitList, phEvent, UR_COMMAND_MEM_BUFFER_COPY); } UR_APIEXPORT ur_result_t UR_APICALL urEnqueueMemBufferCopyRect( @@ -484,15 +508,17 @@ UR_APIEXPORT ur_result_t UR_APICALL urEnqueueMemBufferMap( ur_map_flags_t mapFlags, size_t offset, size_t size, uint32_t numEventsInWaitList, const ur_event_handle_t *phEventWaitList, ur_event_handle_t *phEvent, void **ppRetMap) { - std::ignore = hQueue; std::ignore = blockingMap; std::ignore = mapFlags; std::ignore = size; - std::ignore = numEventsInWaitList; - std::ignore = phEventWaitList; - std::ignore = phEvent; + urEventWait(numEventsInWaitList, phEventWaitList); + ur_event_handle_t event = + new ur_event_handle_t_(hQueue, UR_COMMAND_MEM_BUFFER_MAP); + event->tick_start(); *ppRetMap = hBuffer->_mem + offset; + event->tick_end(); + *phEvent = event; return UR_RESULT_SUCCESS; } @@ -501,12 +527,10 @@ UR_APIEXPORT ur_result_t UR_APICALL urEnqueueMemUnmap( ur_queue_handle_t hQueue, ur_mem_handle_t hMem, void *pMappedPtr, uint32_t numEventsInWaitList, const ur_event_handle_t *phEventWaitList, ur_event_handle_t *phEvent) { - std::ignore = hQueue; std::ignore = hMem; std::ignore = pMappedPtr; - std::ignore = numEventsInWaitList; - std::ignore = phEventWaitList; - std::ignore = phEvent; + urEventWait(numEventsInWaitList, phEventWaitList); + *phEvent = new ur_event_handle_t_(hQueue, UR_COMMAND_MEM_UNMAP); return UR_RESULT_SUCCESS; } @@ -515,10 +539,10 @@ UR_APIEXPORT ur_result_t UR_APICALL urEnqueueUSMFill( ur_queue_handle_t hQueue, void *ptr, size_t patternSize, const void *pPattern, size_t size, uint32_t numEventsInWaitList, const ur_event_handle_t *phEventWaitList, ur_event_handle_t *phEvent) { - std::ignore = hQueue; - std::ignore = numEventsInWaitList; - std::ignore = phEventWaitList; - std::ignore = phEvent; + urEventWait(numEventsInWaitList, phEventWaitList); + ur_event_handle_t event = + new ur_event_handle_t_(hQueue, UR_COMMAND_MEM_BUFFER_MAP); + event->tick_start(); UR_ASSERT(ptr, UR_RESULT_ERROR_INVALID_NULL_POINTER); UR_ASSERT(pPattern, UR_RESULT_ERROR_INVALID_NULL_POINTER); @@ -564,6 +588,10 @@ UR_APIEXPORT ur_result_t UR_APICALL urEnqueueUSMFill( } } } + + event->tick_end(); + *phEvent = event; + return UR_RESULT_SUCCESS; } @@ -571,17 +599,19 @@ UR_APIEXPORT ur_result_t UR_APICALL urEnqueueUSMMemcpy( ur_queue_handle_t hQueue, bool blocking, void *pDst, const void *pSrc, size_t size, uint32_t numEventsInWaitList, const ur_event_handle_t *phEventWaitList, ur_event_handle_t *phEvent) { - std::ignore = hQueue; std::ignore = blocking; - std::ignore = numEventsInWaitList; - std::ignore = phEventWaitList; - std::ignore = phEvent; + urEventWait(numEventsInWaitList, phEventWaitList); + ur_event_handle_t event = + new ur_event_handle_t_(hQueue, UR_COMMAND_MEM_BUFFER_MAP); + event->tick_start(); UR_ASSERT(hQueue, UR_RESULT_ERROR_INVALID_QUEUE); UR_ASSERT(pDst, UR_RESULT_ERROR_INVALID_NULL_POINTER); UR_ASSERT(pSrc, UR_RESULT_ERROR_INVALID_NULL_POINTER); memcpy(pDst, pSrc, size); + event->tick_end(); + *phEvent = event; return UR_RESULT_SUCCESS; } diff --git a/source/adapters/native_cpu/event.cpp b/source/adapters/native_cpu/event.cpp index 9049e3c1b6..37eaf1f6d1 100644 --- a/source/adapters/native_cpu/event.cpp +++ b/source/adapters/native_cpu/event.cpp @@ -11,50 +11,70 @@ #include "ur_api.h" #include "common.hpp" +#include "event.hpp" +#include "queue.hpp" +#include +#include UR_APIEXPORT ur_result_t UR_APICALL urEventGetInfo(ur_event_handle_t hEvent, ur_event_info_t propName, size_t propSize, void *pPropValue, size_t *pPropSizeRet) { - std::ignore = hEvent; - std::ignore = propName; - std::ignore = propSize; - std::ignore = pPropValue; - std::ignore = pPropSizeRet; - - DIE_NO_IMPLEMENTATION; + UrReturnHelper ReturnValue(propSize, pPropValue, pPropSizeRet); + switch (propName) { + case UR_EVENT_INFO_COMMAND_QUEUE: + return ReturnValue(hEvent->getQueue()); + case UR_EVENT_INFO_COMMAND_TYPE: + return ReturnValue(hEvent->getCommandType()); + case UR_EVENT_INFO_REFERENCE_COUNT: + return ReturnValue(hEvent->getReferenceCount()); + case UR_EVENT_INFO_COMMAND_EXECUTION_STATUS: + return ReturnValue(hEvent->getExecutionStatus()); + case UR_EVENT_INFO_CONTEXT: + return ReturnValue(hEvent->getContext()); + default: + break; + } + + return UR_RESULT_ERROR_INVALID_ENUMERATION; } UR_APIEXPORT ur_result_t UR_APICALL urEventGetProfilingInfo( ur_event_handle_t hEvent, ur_profiling_info_t propName, size_t propSize, void *pPropValue, size_t *pPropSizeRet) { - std::ignore = hEvent; - std::ignore = propName; - std::ignore = propSize; - std::ignore = pPropValue; - std::ignore = pPropSizeRet; - - DIE_NO_IMPLEMENTATION; + UrReturnHelper ReturnValue(propSize, pPropValue, pPropSizeRet); + switch (propName) { + case UR_PROFILING_INFO_COMMAND_START: + return ReturnValue(hEvent->get_start_timestamp()); + case UR_PROFILING_INFO_COMMAND_END: + return ReturnValue(hEvent->get_end_timestamp()); + case UR_PROFILING_INFO_COMMAND_QUEUED: + case UR_PROFILING_INFO_COMMAND_SUBMIT: + case UR_PROFILING_INFO_COMMAND_COMPLETE: + default: + break; + } + + return UR_RESULT_ERROR_INVALID_ENUMERATION; } UR_APIEXPORT ur_result_t UR_APICALL urEventWait(uint32_t numEvents, const ur_event_handle_t *phEventWaitList) { - std::ignore = numEvents; - std::ignore = phEventWaitList; - // TODO: currently we do everything synchronously so this is a no-op + for (uint32_t i = 0; i < numEvents; i++) { + phEventWaitList[i]->wait(); + } return UR_RESULT_SUCCESS; } UR_APIEXPORT ur_result_t UR_APICALL urEventRetain(ur_event_handle_t hEvent) { - std::ignore = hEvent; - - DIE_NO_IMPLEMENTATION; + hEvent->incrementReferenceCount(); + return UR_RESULT_SUCCESS; } UR_APIEXPORT ur_result_t UR_APICALL urEventRelease(ur_event_handle_t hEvent) { - std::ignore = hEvent; - DIE_NO_IMPLEMENTATION; + decrementOrDelete(hEvent); + return UR_RESULT_SUCCESS; } UR_APIEXPORT ur_result_t UR_APICALL urEventGetNativeHandle( @@ -99,3 +119,47 @@ UR_APIEXPORT ur_result_t UR_APICALL urEnqueueTimestampRecordingExp( DIE_NO_IMPLEMENTATION; } + +ur_event_handle_t_::ur_event_handle_t_(ur_queue_handle_t queue, + ur_command_t command_type) + : queue(queue), context(queue->getContext()), command_type(command_type), + done(false) { + this->queue->addEvent(this); +} + +ur_event_handle_t_::~ur_event_handle_t_() { + if (!done) { + wait(); + } +} + +void ur_event_handle_t_::wait() { + std::unique_lock lock(mutex); + if (done) { + return; + } + for (auto &f : futures) { + f.wait(); + } + queue->removeEvent(this); + done = true; + // The callback may need to acquire the lock, so we unlock it here + lock.unlock(); + + if (callback) + callback(); +} + +void ur_event_handle_t_::tick_start() { + if (!queue->isProfiling()) + return; + std::lock_guard lock(mutex); + timestamp_start = get_timestamp(); +} + +void ur_event_handle_t_::tick_end() { + if (!queue->isProfiling()) + return; + std::lock_guard lock(mutex); + timestamp_end = get_timestamp(); +} diff --git a/source/adapters/native_cpu/event.hpp b/source/adapters/native_cpu/event.hpp new file mode 100644 index 0000000000..60176a33a6 --- /dev/null +++ b/source/adapters/native_cpu/event.hpp @@ -0,0 +1,66 @@ +//===----------- event.hpp - Native CPU Adapter ---------------------------===// +// +// Copyright (C) 2023 Intel Corporation +// +// Part of the Unified-Runtime Project, under the Apache License v2.0 with LLVM +// Exceptions. See LICENSE.TXT +// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception +// +//===----------------------------------------------------------------------===// +#pragma once +#include "common.hpp" +#include "ur_api.h" +#include +#include +#include +#include + +struct ur_event_handle_t_ : RefCounted { + + ur_event_handle_t_(ur_queue_handle_t queue, ur_command_t command_type); + + ~ur_event_handle_t_(); + + void set_callback(const std::function &cb) { callback = cb; } + + void wait(); + + uint32_t getExecutionStatus() { + // TODO: add support for UR_EVENT_STATUS_RUNNING + std::lock_guard lock(mutex); + if (done) { + return UR_EVENT_STATUS_COMPLETE; + } + return UR_EVENT_STATUS_SUBMITTED; + } + + ur_queue_handle_t getQueue() const { return queue; } + + ur_context_handle_t getContext() const { return context; } + + ur_command_t getCommandType() const { return command_type; } + + void set_futures(std::vector> &fs) { + std::lock_guard lock(mutex); + futures = std::move(fs); + } + + void tick_start(); + + void tick_end(); + + uint64_t get_start_timestamp() const { return timestamp_start; } + + uint64_t get_end_timestamp() const { return timestamp_end; } + +private: + ur_queue_handle_t queue; + ur_context_handle_t context; + ur_command_t command_type; + bool done; + std::mutex mutex; + std::vector> futures; + std::function callback; + uint64_t timestamp_start = 0; + uint64_t timestamp_end = 0; +}; diff --git a/source/adapters/native_cpu/kernel.cpp b/source/adapters/native_cpu/kernel.cpp index 9363f92b7b..e9c7b929fc 100644 --- a/source/adapters/native_cpu/kernel.cpp +++ b/source/adapters/native_cpu/kernel.cpp @@ -59,18 +59,14 @@ UR_APIEXPORT ur_result_t UR_APICALL urKernelSetArgValue( ur_kernel_handle_t hKernel, uint32_t argIndex, size_t argSize, const ur_kernel_arg_value_properties_t *pProperties, const void *pArgValue) { - // Todo: error checking - // Todo: I think that the opencl spec (and therefore the pi spec mandates that - // arg is copied (this is why it is defined as const void*, I guess we should - // do it - // TODO: can args arrive out of order? + // TODO: error checking std::ignore = argIndex; std::ignore = pProperties; UR_ASSERT(hKernel, UR_RESULT_ERROR_INVALID_NULL_HANDLE); UR_ASSERT(argSize, UR_RESULT_ERROR_INVALID_KERNEL_ARGUMENT_SIZE); - hKernel->_args.emplace_back(const_cast(pArgValue)); + hKernel->addArg(pArgValue, argIndex, argSize); return UR_RESULT_SUCCESS; } @@ -81,7 +77,7 @@ UR_APIEXPORT ur_result_t UR_APICALL urKernelSetArgLocal( std::ignore = pProperties; // emplace a placeholder kernel arg, gets replaced with a pointer to the // memory pool before enqueueing the kernel. - hKernel->_args.emplace_back(nullptr); + hKernel->addPtrArg(nullptr, argIndex); hKernel->_localArgInfo.emplace_back(argIndex, argSize); return UR_RESULT_SUCCESS; } @@ -221,14 +217,13 @@ UR_APIEXPORT ur_result_t UR_APICALL urKernelSetArgPointer(ur_kernel_handle_t hKernel, uint32_t argIndex, const ur_kernel_arg_pointer_properties_t *pProperties, const void *pArgValue) { - // TODO: out_of_order args? std::ignore = argIndex; std::ignore = pProperties; UR_ASSERT(hKernel, UR_RESULT_ERROR_INVALID_NULL_HANDLE); UR_ASSERT(pArgValue, UR_RESULT_ERROR_INVALID_NULL_POINTER); - hKernel->_args.push_back(const_cast(pArgValue)); + hKernel->addPtrArg(const_cast(pArgValue), argIndex); return UR_RESULT_SUCCESS; } @@ -262,7 +257,6 @@ UR_APIEXPORT ur_result_t UR_APICALL urKernelSetArgMemObj(ur_kernel_handle_t hKernel, uint32_t argIndex, const ur_kernel_arg_mem_obj_properties_t *pProperties, ur_mem_handle_t hArgValue) { - // TODO: out_of_order args? std::ignore = argIndex; std::ignore = pProperties; @@ -271,11 +265,11 @@ urKernelSetArgMemObj(ur_kernel_handle_t hKernel, uint32_t argIndex, // Taken from ur/adapters/cuda/kernel.cpp // zero-sized buffers are expected to be null. if (hArgValue == nullptr) { - hKernel->_args.emplace_back(nullptr); + hKernel->addPtrArg(nullptr, argIndex); return UR_RESULT_SUCCESS; } - hKernel->_args.emplace_back(hArgValue->_mem); + hKernel->addPtrArg(hArgValue->_mem, argIndex); return UR_RESULT_SUCCESS; } diff --git a/source/adapters/native_cpu/kernel.hpp b/source/adapters/native_cpu/kernel.hpp index 084a0ee695..e2df672d05 100644 --- a/source/adapters/native_cpu/kernel.hpp +++ b/source/adapters/native_cpu/kernel.hpp @@ -11,22 +11,11 @@ #include "common.hpp" #include "nativecpu_state.hpp" #include "program.hpp" -#include +#include #include #include -namespace native_cpu { - -struct NativeCPUArgDesc { - void *MPtr; - - NativeCPUArgDesc(void *Ptr) : MPtr(Ptr){}; -}; - -} // namespace native_cpu - -using nativecpu_kernel_t = void(const native_cpu::NativeCPUArgDesc *, - native_cpu::state *); +using nativecpu_kernel_t = void(void *const *, native_cpu::state *); using nativecpu_ptr_t = nativecpu_kernel_t *; using nativecpu_task_t = std::function; @@ -44,9 +33,9 @@ struct ur_kernel_handle_t_ : RefCounted { : hProgram(hProgram), _name{name}, _subhandler{std::move(subhandler)} {} ur_kernel_handle_t_(const ur_kernel_handle_t_ &other) - : hProgram(other.hProgram), _name(other._name), - _subhandler(other._subhandler), _args(other._args), - _localArgInfo(other._localArgInfo), _localMemPool(other._localMemPool), + : Args(other.Args), hProgram(other.hProgram), _name(other._name), + _subhandler(other._subhandler), _localArgInfo(other._localArgInfo), + _localMemPool(other._localMemPool), _localMemPoolSize(other._localMemPoolSize), ReqdWGSize(other.ReqdWGSize) { incrementReferenceCount(); @@ -55,8 +44,10 @@ struct ur_kernel_handle_t_ : RefCounted { ~ur_kernel_handle_t_() { if (decrementReferenceCount() == 0) { free(_localMemPool); + Args.deallocate(); } } + ur_kernel_handle_t_(ur_program_handle_t hProgram, const char *name, nativecpu_task_t subhandler, std::optional ReqdWGSize, @@ -66,10 +57,67 @@ struct ur_kernel_handle_t_ : RefCounted { ReqdWGSize(ReqdWGSize), MaxWGSize(MaxWGSize), MaxLinearWGSize(MaxLinearWGSize) {} + struct arguments { + using args_index_t = std::vector; + args_index_t Indices; + std::vector ParamSizes; + std::vector OwnsMem; + static constexpr size_t MaxAlign = 16 * sizeof(double); + + /// Add an argument to the kernel. + /// If the argument existed before, it is replaced. + /// Otherwise, it is added. + /// Gaps are filled with empty arguments. + /// Implicit offset argument is kept at the back of the indices collection. + void addArg(size_t Index, size_t Size, const void *Arg) { + if (Index + 1 > Indices.size()) { + Indices.resize(Index + 1); + OwnsMem.resize(Index + 1); + ParamSizes.resize(Index + 1); + + // Update the stored value for the argument + Indices[Index] = native_cpu::aligned_malloc(MaxAlign, Size); + OwnsMem[Index] = true; + ParamSizes[Index] = Size; + } else { + if (ParamSizes[Index] != Size) { + Indices[Index] = realloc(Indices[Index], Size); + ParamSizes[Index] = Size; + } + } + std::memcpy(Indices[Index], Arg, Size); + } + + void addPtrArg(size_t Index, void *Arg) { + if (Index + 1 > Indices.size()) { + Indices.resize(Index + 1); + OwnsMem.resize(Index + 1); + ParamSizes.resize(Index + 1); + + OwnsMem[Index] = false; + ParamSizes[Index] = sizeof(uint8_t *); + } + Indices[Index] = Arg; + } + + // This is called by the destructor of ur_kernel_handle_t_, since + // ur_kernel_handle_t_ implements reference counting and we want + // to deallocate only when the reference count is 0. + void deallocate() { + assert(OwnsMem.size() == Indices.size() && "Size mismatch"); + for (size_t Index = 0; Index < Indices.size(); Index++) { + if (OwnsMem[Index]) + native_cpu::aligned_free(Indices[Index]); + } + } + + const args_index_t &getIndices() const noexcept { return Indices; } + + } Args; + ur_program_handle_t hProgram; std::string _name; nativecpu_task_t _subhandler; - std::vector _args; std::vector _localArgInfo; std::optional getReqdWGSize() const { @@ -99,13 +147,21 @@ struct ur_kernel_handle_t_ : RefCounted { // For each local argument we have size*numthreads size_t offset = 0; for (auto &entry : _localArgInfo) { - _args[entry.argIndex].MPtr = + Args.Indices[entry.argIndex] = _localMemPool + offset + (entry.argSize * threadId); // update offset in the memory pool offset += entry.argSize * numParallelThread; } } + const std::vector &getArgs() const { return Args.getIndices(); } + + void addArg(const void *Ptr, size_t Index, size_t Size) { + Args.addArg(Index, Size, Ptr); + } + + void addPtrArg(void *Ptr, size_t Index) { Args.addPtrArg(Index, Ptr); } + private: char *_localMemPool = nullptr; size_t _localMemPoolSize = 0; diff --git a/source/adapters/native_cpu/queue.cpp b/source/adapters/native_cpu/queue.cpp index 7ee1fdf04c..e2dda24236 100644 --- a/source/adapters/native_cpu/queue.cpp +++ b/source/adapters/native_cpu/queue.cpp @@ -31,11 +31,9 @@ UR_APIEXPORT ur_result_t UR_APICALL urQueueGetInfo(ur_queue_handle_t hQueue, UR_APIEXPORT ur_result_t UR_APICALL urQueueCreate( ur_context_handle_t hContext, ur_device_handle_t hDevice, const ur_queue_properties_t *pProperties, ur_queue_handle_t *phQueue) { - std::ignore = hContext; - std::ignore = hDevice; - std::ignore = pProperties; + // TODO: UR_QUEUE_FLAG_PROFILING_ENABLE and other props - auto Queue = new ur_queue_handle_t_(hDevice); + auto Queue = new ur_queue_handle_t_(hDevice, hContext, pProperties); *phQueue = Queue; return UR_RESULT_SUCCESS; @@ -78,8 +76,7 @@ UR_APIEXPORT ur_result_t UR_APICALL urQueueCreateWithNativeHandle( } UR_APIEXPORT ur_result_t UR_APICALL urQueueFinish(ur_queue_handle_t hQueue) { - std::ignore = hQueue; - // TODO: is this fine as no-op? + hQueue->finish(); return UR_RESULT_SUCCESS; } diff --git a/source/adapters/native_cpu/queue.hpp b/source/adapters/native_cpu/queue.hpp index 8c34af6327..05ff78d822 100644 --- a/source/adapters/native_cpu/queue.hpp +++ b/source/adapters/native_cpu/queue.hpp @@ -9,10 +9,48 @@ //===----------------------------------------------------------------------===// #pragma once #include "common.hpp" -#include "device.hpp" +#include "event.hpp" +#include "ur_api.h" +#include struct ur_queue_handle_t_ : RefCounted { - ur_device_handle_t_ *const device; + ur_queue_handle_t_(ur_device_handle_t device, ur_context_handle_t context, + const ur_queue_properties_t *pProps) + : device(device), context(context), + inOrder(pProps ? !(pProps->flags & + UR_QUEUE_FLAG_OUT_OF_ORDER_EXEC_MODE_ENABLE) + : true), + profilingEnabled(pProps ? pProps->flags & UR_QUEUE_FLAG_PROFILING_ENABLE + : false) {} - ur_queue_handle_t_(ur_device_handle_t_ *device) : device(device) {} + ur_device_handle_t getDevice() const { return device; } + + ur_context_handle_t getContext() const { return context; } + + void addEvent(ur_event_handle_t event) { events.insert(event); } + + void removeEvent(ur_event_handle_t event) { events.erase(event); } + + void finish() { + while (!events.empty()) { + auto ev = *events.begin(); + // ur_event_handle_t_::wait removes itself from the events set in the + // queue + ev->wait(); + } + events.clear(); + } + + ~ur_queue_handle_t_() { finish(); } + + bool isInOrder() const { return inOrder; } + + bool isProfiling() const { return profilingEnabled; } + +private: + ur_device_handle_t device; + ur_context_handle_t context; + std::set events; + const bool inOrder; + const bool profilingEnabled; }; diff --git a/test/conformance/event/event_adapter_native_cpu.match b/test/conformance/event/event_adapter_native_cpu.match index 17066b6d52..2989926af4 100644 --- a/test/conformance/event/event_adapter_native_cpu.match +++ b/test/conformance/event/event_adapter_native_cpu.match @@ -1,33 +1,12 @@ {{NONDETERMINISTIC}} -urEventGetInfoTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UR_EVENT_INFO_COMMAND_QUEUE -urEventGetInfoTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UR_EVENT_INFO_CONTEXT -urEventGetInfoTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UR_EVENT_INFO_COMMAND_TYPE -urEventGetInfoTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UR_EVENT_INFO_COMMAND_EXECUTION_STATUS -urEventGetInfoTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UR_EVENT_INFO_REFERENCE_COUNT -urEventGetInfoNegativeTest.InvalidNullHandle/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} -urEventGetInfoNegativeTest.InvalidEnumeration/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} -urEventGetInfoNegativeTest.InvalidSizePropSize/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} -urEventGetInfoNegativeTest.InvalidSizePropSizeSmall/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} -urEventGetInfoNegativeTest.InvalidNullPointerPropValue/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} -urEventGetInfoNegativeTest.InvalidNullPointerPropSizeRet/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} urEventGetProfilingInfoTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UR_PROFILING_INFO_COMMAND_QUEUED urEventGetProfilingInfoTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UR_PROFILING_INFO_COMMAND_SUBMIT -urEventGetProfilingInfoTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UR_PROFILING_INFO_COMMAND_START -urEventGetProfilingInfoTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UR_PROFILING_INFO_COMMAND_END urEventGetProfilingInfoTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UR_PROFILING_INFO_COMMAND_COMPLETE urEventGetProfilingInfoWithTimingComparisonTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} urEventGetProfilingInfoNegativeTest.InvalidNullHandle/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} -urEventGetProfilingInfoNegativeTest.InvalidEnumeration/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} urEventGetProfilingInfoNegativeTest.InvalidValue/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} urEventWaitTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} -urEventRetainTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} -urEventReleaseTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} -urEventGetNativeHandleTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} -urEventGetNativeHandleTest.InvalidNullPointerNativeEvent/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} -urEventCreateWithNativeHandleTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} urEventSetCallbackTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} urEventSetCallbackTest.ValidateParameters/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} urEventSetCallbackTest.AllStates/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} urEventSetCallbackTest.EventAlreadyCompleted/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} -urEventSetCallbackNegativeTest.InvalidNullPointerCallback/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} -urEventSetCallbackNegativeTest.InvalidEnumeration/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} diff --git a/test/conformance/queue/queue_adapter_native_cpu.match b/test/conformance/queue/queue_adapter_native_cpu.match index 32ea573390..5d39450e12 100644 --- a/test/conformance/queue/queue_adapter_native_cpu.match +++ b/test/conformance/queue/queue_adapter_native_cpu.match @@ -23,7 +23,6 @@ urQueueCreateWithParamTest.MatchingDeviceHandles/SYCL_NATIVE_CPU___SYCL_Native_C urQueueCreateWithParamTest.MatchingDeviceHandles/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UR_QUEUE_FLAG_SUBMISSION_IMMEDIATE urQueueCreateWithParamTest.MatchingDeviceHandles/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UR_QUEUE_FLAG_USE_DEFAULT_STREAM urQueueCreateWithParamTest.MatchingDeviceHandles/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UR_QUEUE_FLAG_SYNC_WITH_DEFAULT_STREAM -urQueueFinishTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} urQueueFlushTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} urQueueGetInfoTestWithInfoParam.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UR_QUEUE_INFO_CONTEXT urQueueGetInfoTestWithInfoParam.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UR_QUEUE_INFO_DEVICE diff --git a/test/conformance/usm/usm_adapter_native_cpu.match b/test/conformance/usm/usm_adapter_native_cpu.match index 84d214c97f..5bf8aaed90 100644 --- a/test/conformance/usm/usm_adapter_native_cpu.match +++ b/test/conformance/usm/usm_adapter_native_cpu.match @@ -2,7 +2,6 @@ urUSMDeviceAllocTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled urUSMDeviceAllocTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled urUSMDeviceAllocTest.SuccessWithDescriptors/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled -urUSMDeviceAllocTest.SuccessWithDescriptors/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled urUSMDeviceAllocTest.InvalidNullHandleContext/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled urUSMDeviceAllocTest.InvalidNullHandleDevice/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled urUSMDeviceAllocTest.InvalidNullPtrResult/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled @@ -24,21 +23,6 @@ urUSMDeviceAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_N urUSMDeviceAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled_64_8 urUSMDeviceAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled_64_512 urUSMDeviceAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled_64_2048 -urUSMDeviceAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_4_8 -urUSMDeviceAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_4_512 -urUSMDeviceAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_4_2048 -urUSMDeviceAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_8_8 -urUSMDeviceAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_8_512 -urUSMDeviceAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_8_2048 -urUSMDeviceAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_16_8 -urUSMDeviceAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_16_512 -urUSMDeviceAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_16_2048 -urUSMDeviceAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_32_8 -urUSMDeviceAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_32_512 -urUSMDeviceAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_32_2048 -urUSMDeviceAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_64_8 -urUSMDeviceAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_64_512 -urUSMDeviceAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_64_2048 urUSMFreeTest.SuccessDeviceAlloc/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} urUSMFreeTest.SuccessHostAlloc/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} urUSMFreeTest.SuccessSharedAlloc/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} @@ -54,7 +38,6 @@ urUSMGetMemAllocInfoNegativeTest.InvalidValuePropSize/SYCL_NATIVE_CPU___SYCL_Nat urUSMHostAllocTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled urUSMHostAllocTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled urUSMHostAllocTest.SuccessWithDescriptors/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled -urUSMHostAllocTest.SuccessWithDescriptors/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled urUSMHostAllocTest.InvalidNullHandleContext/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled urUSMHostAllocTest.InvalidNullPtrMem/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled urUSMHostAllocTest.InvalidUSMSize/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled @@ -75,21 +58,6 @@ urUSMHostAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Nat urUSMHostAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled_64_8 urUSMHostAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled_64_512 urUSMHostAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled_64_2048 -urUSMHostAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_4_8 -urUSMHostAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_4_512 -urUSMHostAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_4_2048 -urUSMHostAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_8_8 -urUSMHostAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_8_512 -urUSMHostAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_8_2048 -urUSMHostAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_16_8 -urUSMHostAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_16_512 -urUSMHostAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_16_2048 -urUSMHostAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_32_8 -urUSMHostAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_32_512 -urUSMHostAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_32_2048 -urUSMHostAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_64_8 -urUSMHostAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_64_512 -urUSMHostAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_64_2048 urUSMPoolCreateTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} urUSMPoolCreateTest.SuccessWithFlag/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} urUSMPoolGetInfoTestWithInfoParam.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UR_USM_POOL_INFO_CONTEXT @@ -105,11 +73,8 @@ urUSMPoolDestroyTest.InvalidNullHandleContext/SYCL_NATIVE_CPU___SYCL_Native_CPU_ urUSMPoolRetainTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} urUSMPoolRetainTest.InvalidNullHandlePool/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}} urUSMSharedAllocTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled -urUSMSharedAllocTest.Success/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled urUSMSharedAllocTest.SuccessWithDescriptors/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled -urUSMSharedAllocTest.SuccessWithDescriptors/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled urUSMSharedAllocTest.SuccessWithMultipleAdvices/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled -urUSMSharedAllocTest.SuccessWithMultipleAdvices/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled urUSMSharedAllocTest.InvalidNullHandleContext/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled urUSMSharedAllocTest.InvalidNullHandleDevice/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled urUSMSharedAllocTest.InvalidNullPtrMem/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled @@ -131,18 +96,3 @@ urUSMSharedAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_N urUSMSharedAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled_64_8 urUSMSharedAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled_64_512 urUSMSharedAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolEnabled_64_2048 -urUSMSharedAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_4_8 -urUSMSharedAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_4_512 -urUSMSharedAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_4_2048 -urUSMSharedAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_8_8 -urUSMSharedAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_8_512 -urUSMSharedAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_8_2048 -urUSMSharedAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_16_8 -urUSMSharedAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_16_512 -urUSMSharedAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_16_2048 -urUSMSharedAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_32_8 -urUSMSharedAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_32_512 -urUSMSharedAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_32_2048 -urUSMSharedAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_64_8 -urUSMSharedAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_64_512 -urUSMSharedAllocAlignmentTest.SuccessAlignedAllocations/SYCL_NATIVE_CPU___SYCL_Native_CPU__{{.*}}__UsePoolDisabled_64_2048