diff --git a/source/loader/CMakeLists.txt b/source/loader/CMakeLists.txt index 474fa6c79b..c884607500 100644 --- a/source/loader/CMakeLists.txt +++ b/source/loader/CMakeLists.txt @@ -106,18 +106,31 @@ if(UR_ENABLE_SANITIZER) target_sources(ur_loader PRIVATE ${CMAKE_CURRENT_SOURCE_DIR}/../ur/ur.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/asan_allocator.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/asan_allocator.hpp ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/asan_interceptor.cpp ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/asan_interceptor.hpp + ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/asan_libdevice.hpp + ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/asan_quarantine.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/asan_quarantine.hpp + ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/asan_report.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/asan_report.hpp + ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/asan_shadow_setup.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/asan_shadow_setup.hpp ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/common.hpp - ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/device_sanitizer_report.hpp + ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/stacktrace.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/stacktrace.hpp + ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/ur_sanddi.cpp ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/ur_sanitizer_layer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/ur_sanitizer_layer.hpp - ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/ur_sanddi.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/ur_sanitizer_utils.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/ur_sanitizer_utils.hpp ) target_sources(ur_loader PRIVATE - ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/linux/san_utils.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/linux/backtrace.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/layers/sanitizer/linux/sanitizer_utils.cpp ) target_include_directories(ur_loader PRIVATE diff --git a/source/loader/layers/sanitizer/asan_allocator.cpp b/source/loader/layers/sanitizer/asan_allocator.cpp new file mode 100644 index 0000000000..1ffa477ca1 --- /dev/null +++ b/source/loader/layers/sanitizer/asan_allocator.cpp @@ -0,0 +1,25 @@ +/* + * + * Copyright (C) 2024 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 + * + * @file asan_allocator.cpp + * + */ + +#include "asan_allocator.hpp" +#include "ur_sanitizer_layer.hpp" + +namespace ur_sanitizer_layer { + +void AllocInfo::print() { + context.logger.info( + "AllocInfo(Alloc=[{}-{}), User=[{}-{}), AllocSize={}, Type={})", + (void *)AllocBegin, (void *)(AllocBegin + AllocSize), (void *)UserBegin, + (void *)(UserEnd), AllocSize, ToString(Type)); +} + +} // namespace ur_sanitizer_layer diff --git a/source/loader/layers/sanitizer/asan_allocator.hpp b/source/loader/layers/sanitizer/asan_allocator.hpp new file mode 100644 index 0000000000..88dfd2d074 --- /dev/null +++ b/source/loader/layers/sanitizer/asan_allocator.hpp @@ -0,0 +1,70 @@ +/* + * + * Copyright (C) 2024 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 + * + * @file asan_allocator.hpp + * + */ + +#pragma once + +#include "common.hpp" +#include "stacktrace.hpp" + +#include +#include + +namespace ur_sanitizer_layer { + +enum class AllocType : uint32_t { + UNKNOWN, + DEVICE_USM, + SHARED_USM, + HOST_USM, + MEM_BUFFER, + DEVICE_GLOBAL +}; + +struct AllocInfo { + uptr AllocBegin = 0; + uptr UserBegin = 0; + uptr UserEnd = 0; + size_t AllocSize = 0; + + AllocType Type = AllocType::UNKNOWN; + bool IsReleased = false; + + ur_context_handle_t Context = nullptr; + ur_device_handle_t Device = nullptr; + + StackTrace AllocStack; + StackTrace ReleaseStack; + + void print(); +}; + +using AllocationMap = std::map>; +using AllocationIterator = AllocationMap::iterator; + +inline const char *ToString(AllocType Type) { + switch (Type) { + case AllocType::DEVICE_USM: + return "Device USM"; + case AllocType::HOST_USM: + return "Host USM"; + case AllocType::SHARED_USM: + return "Shared USM"; + case AllocType::MEM_BUFFER: + return "Memory Buffer"; + case AllocType::DEVICE_GLOBAL: + return "Device Global"; + default: + return "Unknown Type"; + } +} + +} // namespace ur_sanitizer_layer diff --git a/source/loader/layers/sanitizer/asan_interceptor.cpp b/source/loader/layers/sanitizer/asan_interceptor.cpp index 6aef92c220..c55d752410 100644 --- a/source/loader/layers/sanitizer/asan_interceptor.cpp +++ b/source/loader/layers/sanitizer/asan_interceptor.cpp @@ -12,36 +12,18 @@ */ #include "asan_interceptor.hpp" -#include "ur_sanitizer_layer.hpp" +#include "asan_quarantine.hpp" +#include "asan_report.hpp" +#include "asan_shadow_setup.hpp" +#include "stacktrace.hpp" +#include "ur_sanitizer_utils.hpp" namespace ur_sanitizer_layer { namespace { -// These magic values are written to shadow for better error -// reporting. -constexpr int kUsmDeviceRedzoneMagic = (char)0x81; -constexpr int kUsmHostRedzoneMagic = (char)0x82; -constexpr int kUsmSharedRedzoneMagic = (char)0x83; -constexpr int kMemBufferRedzoneMagic = (char)0x84; -constexpr int kDeviceGlobalRedZoneMagic = (char)0x85; - -constexpr auto kSPIR_AsanShadowMemoryGlobalStart = - "__AsanShadowMemoryGlobalStart"; -constexpr auto kSPIR_AsanShadowMemoryGlobalEnd = "__AsanShadowMemoryGlobalEnd"; -constexpr auto kSPIR_AsanShadowMemoryLocalStart = - "__AsanShadowMemoryLocalStart"; -constexpr auto kSPIR_AsanShadowMemoryLocalEnd = "__AsanShadowMemoryLocalEnd"; - -constexpr auto kSPIR_DeviceType = "__DeviceType"; - constexpr auto kSPIR_DeviceSanitizerReportMem = "__DeviceSanitizerReportMem"; -constexpr auto kSPIR_AsanDeviceGlobalCount = "__AsanDeviceGlobalCount"; -constexpr auto kSPIR_AsanDeviceGlobalMetadata = "__AsanDeviceGlobalMetadata"; - -DeviceSanitizerReport SPIR_DeviceSanitizerReportMem; - uptr MemToShadow_CPU(uptr USM_SHADOW_BASE, uptr UPtr) { return USM_SHADOW_BASE + (UPtr >> 3); } @@ -55,82 +37,151 @@ uptr MemToShadow_PVC(uptr USM_SHADOW_BASE, uptr UPtr) { } } -ur_context_handle_t getContext(ur_queue_handle_t Queue) { - ur_context_handle_t Context; - [[maybe_unused]] auto Result = context.urDdiTable.Queue.pfnGetInfo( - Queue, UR_QUEUE_INFO_CONTEXT, sizeof(ur_context_handle_t), &Context, - nullptr); - assert(Result == UR_RESULT_SUCCESS); - return Context; +ur_result_t urEnqueueUSMSet(ur_queue_handle_t Queue, void *Ptr, char Value, + size_t Size, uint32_t NumEvents = 0, + const ur_event_handle_t *EventWaitList = nullptr, + ur_event_handle_t *OutEvent = nullptr) { + if (Size == 0) { + return UR_RESULT_SUCCESS; + } + return context.urDdiTable.Enqueue.pfnUSMFill( + Queue, Ptr, 1, &Value, Size, NumEvents, EventWaitList, OutEvent); } -ur_device_handle_t getDevice(ur_queue_handle_t Queue) { - ur_device_handle_t Device; - [[maybe_unused]] auto Result = context.urDdiTable.Queue.pfnGetInfo( - Queue, UR_QUEUE_INFO_DEVICE, sizeof(ur_device_handle_t), &Device, - nullptr); - assert(Result == UR_RESULT_SUCCESS); - return Device; -} +ur_result_t enqueueMemSetShadow(ur_context_handle_t Context, + std::shared_ptr &DeviceInfo, + ur_queue_handle_t Queue, uptr Ptr, uptr Size, + u8 Value) { + if (Size == 0) { + return UR_RESULT_SUCCESS; + } + if (DeviceInfo->Type == DeviceType::CPU) { + uptr ShadowBegin = MemToShadow_CPU(DeviceInfo->ShadowOffset, Ptr); + uptr ShadowEnd = + MemToShadow_CPU(DeviceInfo->ShadowOffset, Ptr + Size - 1); -ur_program_handle_t getProgram(ur_kernel_handle_t Kernel) { - ur_program_handle_t Program; - [[maybe_unused]] auto Result = context.urDdiTable.Kernel.pfnGetInfo( - Kernel, UR_KERNEL_INFO_PROGRAM, sizeof(ur_program_handle_t), &Program, - nullptr); - assert(Result == UR_RESULT_SUCCESS); - return Program; -} + // Poison shadow memory outside of asan runtime is not allowed, so we + // need to avoid memset's call from being intercepted. + static auto MemSet = + (void *(*)(void *, int, size_t))GetMemFunctionPointer("memset"); + if (!MemSet) { + return UR_RESULT_ERROR_UNKNOWN; + } + context.logger.debug("enqueueMemSetShadow(addr={}, count={}, value={})", + (void *)ShadowBegin, ShadowEnd - ShadowBegin + 1, + (void *)(size_t)Value); + MemSet((void *)ShadowBegin, Value, ShadowEnd - ShadowBegin + 1); + } else if (DeviceInfo->Type == DeviceType::GPU_PVC) { + uptr ShadowBegin = MemToShadow_PVC(DeviceInfo->ShadowOffset, Ptr); + uptr ShadowEnd = + MemToShadow_PVC(DeviceInfo->ShadowOffset, Ptr + Size - 1); + assert(ShadowBegin <= ShadowEnd); + { + static const size_t PageSize = + GetVirtualMemGranularity(Context, DeviceInfo->Handle); -void getProgramDevices(ur_program_handle_t Program, - std::vector &Devices) { - size_t PropSize; - [[maybe_unused]] ur_result_t Result = context.urDdiTable.Program.pfnGetInfo( - Program, UR_PROGRAM_INFO_DEVICES, 0, nullptr, &PropSize); - assert(Result == UR_RESULT_SUCCESS); + ur_physical_mem_properties_t Desc{ + UR_STRUCTURE_TYPE_PHYSICAL_MEM_PROPERTIES, nullptr, 0}; + static ur_physical_mem_handle_t PhysicalMem{}; - Devices.resize(PropSize / sizeof(ur_device_handle_t)); - Result = context.urDdiTable.Program.pfnGetInfo( - Program, UR_PROGRAM_INFO_DEVICES, PropSize, Devices.data(), nullptr); - assert(Result == UR_RESULT_SUCCESS); -} + // Make sure [Ptr, Ptr + Size] is mapped to physical memory + for (auto MappedPtr = RoundDownTo(ShadowBegin, PageSize); + MappedPtr <= ShadowEnd; MappedPtr += PageSize) { + if (!PhysicalMem) { + auto URes = context.urDdiTable.PhysicalMem.pfnCreate( + Context, DeviceInfo->Handle, PageSize, &Desc, + &PhysicalMem); + if (URes != UR_RESULT_SUCCESS) { + context.logger.error("urPhysicalMemCreate(): {}", URes); + return URes; + } + } -size_t getLocalMemorySize(ur_device_handle_t Device) { - size_t LocalMemorySize; - [[maybe_unused]] auto Result = context.urDdiTable.Device.pfnGetInfo( - Device, UR_DEVICE_INFO_LOCAL_MEM_SIZE, sizeof(LocalMemorySize), - &LocalMemorySize, nullptr); - assert(Result == UR_RESULT_SUCCESS); - return LocalMemorySize; -} + context.logger.debug("urVirtualMemMap: {} ~ {}", + (void *)MappedPtr, + (void *)(MappedPtr + PageSize - 1)); + + // FIXME: No flag to check the failed reason is VA is already mapped + auto URes = context.urDdiTable.VirtualMem.pfnMap( + Context, (void *)MappedPtr, PageSize, PhysicalMem, 0, + UR_VIRTUAL_MEM_ACCESS_FLAG_READ_WRITE); + if (URes != UR_RESULT_SUCCESS) { + context.logger.debug("urVirtualMemMap(): {}", URes); + } -std::string getKernelName(ur_kernel_handle_t Kernel) { - size_t KernelNameSize = 0; - [[maybe_unused]] auto Res = context.urDdiTable.Kernel.pfnGetInfo( - Kernel, UR_KERNEL_INFO_FUNCTION_NAME, 0, nullptr, &KernelNameSize); - assert(Res == UR_RESULT_SUCCESS); + // Initialize to zero + if (URes == UR_RESULT_SUCCESS) { + // Reset PhysicalMem to null since it's been mapped + PhysicalMem = nullptr; - std::vector KernelNameBuf(KernelNameSize); - Res = context.urDdiTable.Kernel.pfnGetInfo( - Kernel, UR_KERNEL_INFO_FUNCTION_NAME, KernelNameSize, - KernelNameBuf.data(), nullptr); - assert(Res == UR_RESULT_SUCCESS); + auto URes = + urEnqueueUSMSet(Queue, (void *)MappedPtr, 0, PageSize); + if (URes != UR_RESULT_SUCCESS) { + context.logger.error("urEnqueueUSMFill(): {}", URes); + return URes; + } + } + } + } - return std::string(KernelNameBuf.data(), KernelNameSize - 1); + auto URes = urEnqueueUSMSet(Queue, (void *)ShadowBegin, Value, + ShadowEnd - ShadowBegin + 1); + context.logger.debug( + "enqueueMemSetShadow (addr={}, count={}, value={}): {}", + (void *)ShadowBegin, ShadowEnd - ShadowBegin + 1, + (void *)(size_t)Value, URes); + if (URes != UR_RESULT_SUCCESS) { + context.logger.error("urEnqueueUSMFill(): {}", URes); + return URes; + } + } else { + context.logger.error("Unsupport device type"); + return UR_RESULT_ERROR_INVALID_ARGUMENT; + } + return UR_RESULT_SUCCESS; } } // namespace -SanitizerInterceptor::SanitizerInterceptor() - : m_IsInASanContext(IsInASanContext()), - m_ShadowMemInited(m_IsInASanContext) {} +SanitizerInterceptor::SanitizerInterceptor() { + auto Options = getenv_to_map("UR_LAYER_ASAN_OPTIONS"); + if (!Options.has_value()) { + return; + } -SanitizerInterceptor::~SanitizerInterceptor() { - if (!m_IsInASanContext && m_ShadowMemInited && !DestroyShadowMem()) { - context.logger.error("Failed to destroy shadow memory"); + auto KV = Options->find("debug"); + if (KV != Options->end()) { + auto Value = KV->second.front(); + cl_Debug = Value == "1" || Value == "true" ? 1 : 0; + } + + KV = Options->find("quarantine_size_mb"); + if (KV != Options->end()) { + auto Value = KV->second.front(); + try { + cl_MaxQuarantineSizeMB = std::stoul(Value); + } catch (...) { + die("[ERROR]: \"cl_MaxQuarantineSizeMB\" should be an " + "integer"); + } + } + if (cl_MaxQuarantineSizeMB) { + m_Quarantine = + std::make_unique(cl_MaxQuarantineSizeMB * 1024 * 1024); + } + + KV = Options->find("detect_locals"); + if (KV != Options->end()) { + auto Value = KV->second.front(); + cl_DetectLocals = Value == "1" || Value == "true" ? true : false; } } +SanitizerInterceptor::~SanitizerInterceptor() { + DestroyShadowMemoryOnCPU(); + DestroyShadowMemoryOnPVC(); +} + /// The memory chunk allocated from the underlying allocator looks like this: /// L L L L L L U U U U U U R R /// L -- left redzone words (0 or more bytes) @@ -141,26 +192,34 @@ SanitizerInterceptor::~SanitizerInterceptor() { ur_result_t SanitizerInterceptor::allocateMemory( ur_context_handle_t Context, ur_device_handle_t Device, const ur_usm_desc_t *Properties, ur_usm_pool_handle_t Pool, size_t Size, - void **ResultPtr, AllocType Type) { - auto Alignment = Properties->align; - assert(Alignment == 0 || IsPowerOfTwo(Alignment)); + AllocType Type, void **ResultPtr) { auto ContextInfo = getContextInfo(Context); - std::shared_ptr DeviceInfo; - if (Device) { - DeviceInfo = ContextInfo->getDeviceInfo(Device); + std::shared_ptr DeviceInfo = + Device ? getDeviceInfo(Device) : nullptr; + + /// Modified from llvm/compiler-rt/lib/asan/asan_allocator.cpp + uint32_t Alignment = Properties ? Properties->align : 0; + // Alignment must be zero or a power-of-two + if (0 != (Alignment & (Alignment - 1))) { + return UR_RESULT_ERROR_INVALID_ARGUMENT; } + const uint32_t MinAlignment = ASAN_SHADOW_GRANULARITY; if (Alignment == 0) { - Alignment = - DeviceInfo ? DeviceInfo->Alignment : ASAN_SHADOW_GRANULARITY; + Alignment = DeviceInfo ? DeviceInfo->Alignment : MinAlignment; + } + if (Alignment < MinAlignment) { + Alignment = MinAlignment; } - // Copy from LLVM compiler-rt/lib/asan uptr RZLog = ComputeRZLog(Size); uptr RZSize = RZLog2Size(RZLog); uptr RoundedSize = RoundUpTo(Size, Alignment); uptr NeededSize = RoundedSize + RZSize * 2; + if (Alignment > MinAlignment) { + NeededSize += Alignment; + } void *Allocated = nullptr; @@ -178,7 +237,6 @@ ur_result_t SanitizerInterceptor::allocateMemory( return UR_RESULT_ERROR_INVALID_ARGUMENT; } - // Copy from LLVM compiler-rt/lib/asan uptr AllocBegin = reinterpret_cast(Allocated); [[maybe_unused]] uptr AllocEnd = AllocBegin + NeededSize; uptr UserBegin = AllocBegin + RZSize; @@ -190,32 +248,32 @@ ur_result_t SanitizerInterceptor::allocateMemory( *ResultPtr = reinterpret_cast(UserBegin); - auto AI = std::make_shared( - AllocInfo{AllocBegin, UserBegin, UserEnd, NeededSize, Type}); + auto AI = std::make_shared(AllocInfo{AllocBegin, + UserBegin, + UserEnd, + NeededSize, + Type, + false, + Context, + Device, + GetCurrentBacktrace(), + {}}); + + AI->print(); // For updating shadow memory - if (DeviceInfo) { // device/shared USM - std::scoped_lock Guard(DeviceInfo->Mutex); - DeviceInfo->AllocInfos.emplace_back(AI); - } else { // host USM's AllocInfo needs to insert into all devices - for (auto &pair : ContextInfo->DeviceMap) { - auto DeviceInfo = pair.second; - std::scoped_lock Guard(DeviceInfo->Mutex); - DeviceInfo->AllocInfos.emplace_back(AI); - } + if (Device) { // Device/Shared USM + ContextInfo->insertAllocInfo({Device}, AI); + } else { // Host USM + ContextInfo->insertAllocInfo(ContextInfo->DeviceList, AI); } // For memory release { - std::scoped_lock Guard(ContextInfo->Mutex); - ContextInfo->AllocatedUSMMap[AllocBegin] = std::move(AI); + std::scoped_lock Guard(m_AllocationMapMutex); + m_AllocationMap.emplace(AI->AllocBegin, std::move(AI)); } - context.logger.info( - "AllocInfos(AllocBegin={}, User={}-{}, NeededSize={}, Type={})", - (void *)AllocBegin, (void *)UserBegin, (void *)UserEnd, NeededSize, - ToString(Type)); - return UR_RESULT_SUCCESS; } @@ -223,59 +281,98 @@ ur_result_t SanitizerInterceptor::releaseMemory(ur_context_handle_t Context, void *Ptr) { auto ContextInfo = getContextInfo(Context); - std::shared_lock Guard(ContextInfo->Mutex); - auto Addr = reinterpret_cast(Ptr); - // Find the last element is not greater than key - auto AllocInfoIt = ContextInfo->AllocatedUSMMap.upper_bound((uptr)Addr); - if (AllocInfoIt == ContextInfo->AllocatedUSMMap.begin()) { - context.logger.error( - "Can't find release pointer({}) in AllocatedAddressesMap", Ptr); + auto AllocInfoItOp = findAllocInfoByAddress(Addr); + + if (!AllocInfoItOp) { + // "Addr" might be a host pointer + ReportBadFree(Addr, GetCurrentBacktrace(), nullptr); return UR_RESULT_ERROR_INVALID_ARGUMENT; } - --AllocInfoIt; + + auto AllocInfoIt = *AllocInfoItOp; auto &AllocInfo = AllocInfoIt->second; - context.logger.debug("USMAllocInfo(AllocBegin={}, UserBegin={})", - AllocInfo->AllocBegin, AllocInfo->UserBegin); + if (AllocInfo->Context != Context) { + if (AllocInfo->UserBegin == Addr) { + ReportBadContext(Addr, GetCurrentBacktrace(), AllocInfo); + } else { + // "Addr" might be a host pointer + ReportBadFree(Addr, GetCurrentBacktrace(), nullptr); + } + return UR_RESULT_ERROR_INVALID_ARGUMENT; + } if (Addr != AllocInfo->UserBegin) { - context.logger.error("Releasing pointer({}) is not match to {}", Ptr, - AllocInfo->UserBegin); + ReportBadFree(Addr, GetCurrentBacktrace(), AllocInfo); return UR_RESULT_ERROR_INVALID_ARGUMENT; } - // TODO: Update shadow memory - return context.urDdiTable.USM.pfnFree(Context, - (void *)AllocInfo->AllocBegin); + if (AllocInfo->IsReleased) { + ReportDoubleFree(Addr, GetCurrentBacktrace(), AllocInfo); + return UR_RESULT_ERROR_INVALID_ARGUMENT; + } + + AllocInfo->IsReleased = true; + AllocInfo->ReleaseStack = GetCurrentBacktrace(); + + if (AllocInfo->Type == AllocType::HOST_USM) { + ContextInfo->insertAllocInfo(ContextInfo->DeviceList, AllocInfo); + } else { + ContextInfo->insertAllocInfo({AllocInfo->Device}, AllocInfo); + } + + // If quarantine is disabled, USM is freed immediately + if (!m_Quarantine) { + context.logger.debug("Free: {}", (void *)AllocInfo->AllocBegin); + std::scoped_lock Guard(m_AllocationMapMutex); + m_AllocationMap.erase(AllocInfoIt); + return context.urDdiTable.USM.pfnFree(Context, + (void *)(AllocInfo->AllocBegin)); + } + + auto ReleaseList = m_Quarantine->put(AllocInfo->Device, AllocInfoIt); + if (ReleaseList.size()) { + std::scoped_lock Guard(m_AllocationMapMutex); + for (auto &It : ReleaseList) { + context.logger.info("Quarantine Free: {}", + (void *)It->second->AllocBegin); + m_AllocationMap.erase(It); + UR_CALL(context.urDdiTable.USM.pfnFree( + Context, (void *)(It->second->AllocBegin))); + } + } + + return UR_RESULT_SUCCESS; } ur_result_t SanitizerInterceptor::preLaunchKernel(ur_kernel_handle_t Kernel, ur_queue_handle_t Queue, - ur_event_handle_t &Event, - LaunchInfo &LaunchInfo, - uint32_t numWorkgroup) { - UR_CALL(prepareLaunch(Queue, Kernel, LaunchInfo, numWorkgroup)); + LaunchInfo &LaunchInfo) { + auto Context = GetContext(Queue); + auto Device = GetDevice(Queue); + auto ContextInfo = getContextInfo(Context); + auto DeviceInfo = getDeviceInfo(Device); - UR_CALL(updateShadowMemory(Queue)); + ManagedQueue InternalQueue(Context, Device); + if (!InternalQueue) { + context.logger.error("Failed to create internal queue"); + return UR_RESULT_ERROR_INVALID_QUEUE; + } - // Return LastEvent in QueueInfo - auto Context = getContext(Queue); - auto ContextInfo = getContextInfo(Context); - auto QueueInfo = ContextInfo->getQueueInfo(Queue); + UR_CALL( + prepareLaunch(Context, DeviceInfo, InternalQueue, Kernel, LaunchInfo)); - std::scoped_lock Guard(QueueInfo->Mutex); - Event = QueueInfo->LastEvent; - QueueInfo->LastEvent = nullptr; + UR_CALL(updateShadowMemory(ContextInfo, DeviceInfo, InternalQueue)); return UR_RESULT_SUCCESS; } -void SanitizerInterceptor::postLaunchKernel(ur_kernel_handle_t Kernel, - ur_queue_handle_t Queue, - ur_event_handle_t &Event, - LaunchInfo &LaunchInfo) { - auto Program = getProgram(Kernel); +ur_result_t SanitizerInterceptor::postLaunchKernel(ur_kernel_handle_t Kernel, + ur_queue_handle_t Queue, + ur_event_handle_t &Event, + LaunchInfo &LaunchInfo) { + auto Program = GetProgram(Kernel); ur_event_handle_t ReadEvent{}; // If kernel has defined SPIR_DeviceSanitizerReportMem, then we try to read it @@ -289,200 +386,33 @@ void SanitizerInterceptor::postLaunchKernel(ur_kernel_handle_t Kernel, if (Result == UR_RESULT_SUCCESS) { Event = ReadEvent; - auto AH = &LaunchInfo.SPIR_DeviceSanitizerReportMem; - if (!AH->Flag) { - return; + const auto &AH = LaunchInfo.SPIR_DeviceSanitizerReportMem; + if (!AH.Flag) { + return UR_RESULT_SUCCESS; } - - const char *File = AH->File[0] ? AH->File : ""; - const char *Func = AH->Func[0] ? AH->Func : ""; - auto KernelName = getKernelName(Kernel); - - // Try to demangle the kernel name - KernelName = DemangleName(KernelName); - - context.logger.always("\n====ERROR: DeviceSanitizer: {} on {}", - ToString(AH->ErrorType), - ToString(AH->MemoryType)); - context.logger.always( - "{} of size {} at kernel <{}> LID({}, {}, {}) GID({}, " - "{}, {})", - AH->IsWrite ? "WRITE" : "READ", AH->AccessSize, KernelName.c_str(), - AH->LID0, AH->LID1, AH->LID2, AH->GID0, AH->GID1, AH->GID2); - context.logger.always(" #0 {} {}:{}", Func, File, AH->Line); - if (!AH->IsRecover) { - exit(1); + if (AH.ErrorType == DeviceSanitizerErrorType::USE_AFTER_FREE) { + ReportUseAfterFree(AH, Kernel, GetContext(Queue)); + } else if (AH.ErrorType == DeviceSanitizerErrorType::OUT_OF_BOUNDS) { + ReportOutOfBoundsError(AH, Kernel); + } else { + ReportGenericError(AH); } } -} - -ur_result_t SanitizerInterceptor::allocShadowMemory( - ur_context_handle_t Context, std::shared_ptr &DeviceInfo) { - if (DeviceInfo->Type == DeviceType::CPU) { - if (!m_IsInASanContext) { - static std::once_flag OnceFlag; - bool Result = true; - std::call_once(OnceFlag, [&]() { - Result = m_ShadowMemInited = SetupShadowMem(); - }); - - if (!Result) { - context.logger.error("Failed to allocate shadow memory"); - return UR_RESULT_ERROR_OUT_OF_RESOURCES; - } - } - DeviceInfo->ShadowOffset = LOW_SHADOW_BEGIN; - DeviceInfo->ShadowOffsetEnd = HIGH_SHADOW_END; - } else if (DeviceInfo->Type == DeviceType::GPU_PVC) { - /// SHADOW MEMORY MAPPING (PVC, with CPU 47bit) - /// Host/Shared USM : 0x0 ~ 0x0fff_ffff_ffff - /// ? : 0x1000_0000_0000 ~ 0x1fff_ffff_ffff - /// Device USM : 0x2000_0000_0000 ~ 0x3fff_ffff_ffff - constexpr size_t SHADOW_SIZE = 1ULL << 46; - // FIXME: Currently, Level-Zero doesn't create independent VAs for each contexts, - // which will cause out-of-resource error when users use multiple contexts - static uptr ShadowOffset, ShadowOffsetEnd; - - if (!ShadowOffset) { - // TODO: Protect Bad Zone - auto Result = context.urDdiTable.VirtualMem.pfnReserve( - Context, nullptr, SHADOW_SIZE, (void **)&ShadowOffset); - if (Result != UR_RESULT_SUCCESS) { - context.logger.error( - "Failed to allocate shadow memory on PVC: {}", Result); - return Result; - } - ShadowOffsetEnd = ShadowOffset + SHADOW_SIZE; - } - - DeviceInfo->ShadowOffset = ShadowOffset; - DeviceInfo->ShadowOffsetEnd = ShadowOffsetEnd; - } else { - context.logger.error("Unsupport device type"); - return UR_RESULT_ERROR_INVALID_ARGUMENT; - } - context.logger.info("ShadowMemory(Global): {} - {}", - (void *)DeviceInfo->ShadowOffset, - (void *)DeviceInfo->ShadowOffsetEnd); - return UR_RESULT_SUCCESS; + return Result; } -ur_result_t SanitizerInterceptor::enqueueMemSetShadow( - ur_context_handle_t Context, ur_device_handle_t Device, - ur_queue_handle_t Queue, uptr Ptr, uptr Size, u8 Value, - ur_event_handle_t DepEvent, ur_event_handle_t *OutEvent) { - - auto ContextInfo = getContextInfo(Context); - auto DeviceInfo = ContextInfo->getDeviceInfo(Device); - - if (DeviceInfo->Type == DeviceType::CPU) { - uptr ShadowBegin = MemToShadow_CPU(DeviceInfo->ShadowOffset, Ptr); - uptr ShadowEnd = - MemToShadow_CPU(DeviceInfo->ShadowOffset, Ptr + Size - 1); - - // Poison shadow memory outside of asan runtime is not allowed, so we - // need to avoid memset's call from being intercepted. - static auto MemSet = - (void *(*)(void *, int, size_t))GetMemFunctionPointer("memset"); - if (!MemSet) { - return UR_RESULT_ERROR_UNKNOWN; - } - - MemSet((void *)ShadowBegin, Value, ShadowEnd - ShadowBegin + 1); - context.logger.debug( - "enqueueMemSetShadow (addr={}, count={}, value={})", - (void *)ShadowBegin, ShadowEnd - ShadowBegin + 1, - (void *)(size_t)Value); - } else if (DeviceInfo->Type == DeviceType::GPU_PVC) { - uptr ShadowBegin = MemToShadow_PVC(DeviceInfo->ShadowOffset, Ptr); - uptr ShadowEnd = - MemToShadow_PVC(DeviceInfo->ShadowOffset, Ptr + Size - 1); - - uint32_t NumEventsInWaitList = DepEvent ? 1 : 0; - const ur_event_handle_t *EventsWaitList = - DepEvent ? &DepEvent : nullptr; - ur_event_handle_t InternalEvent{}; - ur_event_handle_t *Event = OutEvent ? OutEvent : &InternalEvent; - - { - static const size_t PageSize = [Context, Device]() { - size_t Size; - [[maybe_unused]] auto Result = - context.urDdiTable.VirtualMem.pfnGranularityGetInfo( - Context, Device, - UR_VIRTUAL_MEM_GRANULARITY_INFO_RECOMMENDED, - sizeof(Size), &Size, nullptr); - assert(Result == UR_RESULT_SUCCESS); - context.logger.info("PVC PageSize: {}", Size); - return Size; - }(); - - ur_physical_mem_properties_t Desc{ - UR_STRUCTURE_TYPE_PHYSICAL_MEM_PROPERTIES, nullptr, 0}; - static ur_physical_mem_handle_t PhysicalMem{}; - - // Make sure [Ptr, Ptr + Size] is mapped to physical memory - for (auto MappedPtr = RoundDownTo(ShadowBegin, PageSize); - MappedPtr <= ShadowEnd; MappedPtr += PageSize) { - if (!PhysicalMem) { - auto URes = context.urDdiTable.PhysicalMem.pfnCreate( - Context, Device, PageSize, &Desc, &PhysicalMem); - if (URes != UR_RESULT_SUCCESS) { - context.logger.error("urPhysicalMemCreate(): {}", URes); - return URes; - } - } - - context.logger.debug("urVirtualMemMap: {} ~ {}", - (void *)MappedPtr, - (void *)(MappedPtr + PageSize - 1)); - - // FIXME: No flag to check the failed reason is VA is already mapped - auto URes = context.urDdiTable.VirtualMem.pfnMap( - Context, (void *)MappedPtr, PageSize, PhysicalMem, 0, - UR_VIRTUAL_MEM_ACCESS_FLAG_READ_WRITE); - if (URes != UR_RESULT_SUCCESS) { - context.logger.debug("urVirtualMemMap(): {}", URes); - } - - // Initialize to zero - if (URes == UR_RESULT_SUCCESS) { - // Reset PhysicalMem to null since it's been mapped - PhysicalMem = nullptr; - - const char Pattern[] = {0}; - - auto URes = context.urDdiTable.Enqueue.pfnUSMFill( - Queue, (void *)MappedPtr, 1, Pattern, PageSize, - NumEventsInWaitList, EventsWaitList, Event); - if (URes != UR_RESULT_SUCCESS) { - context.logger.error("urEnqueueUSMFill(): {}", URes); - return URes; - } - - NumEventsInWaitList = 1; - EventsWaitList = Event; - } - } - } - - const char Pattern[] = {(char)Value}; - auto URes = context.urDdiTable.Enqueue.pfnUSMFill( - Queue, (void *)ShadowBegin, 1, Pattern, ShadowEnd - ShadowBegin + 1, - NumEventsInWaitList, EventsWaitList, Event); - context.logger.debug( - "enqueueMemSetShadow (addr={}, count={}, value={}): {}", - (void *)ShadowBegin, ShadowEnd - ShadowBegin + 1, - (void *)(size_t)Value, URes); - if (URes != UR_RESULT_SUCCESS) { - context.logger.error("urEnqueueUSMFill(): {}", URes); - return URes; - } +ur_result_t DeviceInfo::allocShadowMemory(ur_context_handle_t Context) { + if (Type == DeviceType::CPU) { + UR_CALL(SetupShadowMemoryOnCPU(ShadowOffset, ShadowOffsetEnd)); + } else if (Type == DeviceType::GPU_PVC) { + UR_CALL(SetupShadowMemoryOnPVC(Context, ShadowOffset, ShadowOffsetEnd)); } else { context.logger.error("Unsupport device type"); return UR_RESULT_ERROR_INVALID_ARGUMENT; } + context.logger.info("ShadowMemory(Global): {} - {}", (void *)ShadowOffset, + (void *)ShadowOffsetEnd); return UR_RESULT_SUCCESS; } @@ -494,28 +424,49 @@ ur_result_t SanitizerInterceptor::enqueueMemSetShadow( /// /// ref: https://github.com/google/sanitizers/wiki/AddressSanitizerAlgorithm#mapping ur_result_t SanitizerInterceptor::enqueueAllocInfo( - ur_context_handle_t Context, ur_device_handle_t Device, - ur_queue_handle_t Queue, std::shared_ptr &AllocInfo, - ur_event_handle_t &LastEvent) { + ur_context_handle_t Context, std::shared_ptr &DeviceInfo, + ur_queue_handle_t Queue, std::shared_ptr &AI) { + if (AI->IsReleased) { + int ShadowByte; + switch (AI->Type) { + case AllocType::HOST_USM: + ShadowByte = kUsmHostDeallocatedMagic; + break; + case AllocType::DEVICE_USM: + ShadowByte = kUsmDeviceDeallocatedMagic; + break; + case AllocType::SHARED_USM: + ShadowByte = kUsmSharedDeallocatedMagic; + break; + case AllocType::MEM_BUFFER: + ShadowByte = kMemBufferDeallocatedMagic; + break; + default: + ShadowByte = 0xff; + assert(false && "Unknow AllocInfo Type"); + } + UR_CALL(enqueueMemSetShadow(Context, DeviceInfo, Queue, AI->AllocBegin, + AI->AllocSize, ShadowByte)); + return UR_RESULT_SUCCESS; + } + // Init zero - UR_CALL(enqueueMemSetShadow(Context, Device, Queue, AllocInfo->AllocBegin, - AllocInfo->AllocSize, 0, LastEvent, - &LastEvent)); + UR_CALL(enqueueMemSetShadow(Context, DeviceInfo, Queue, AI->AllocBegin, + AI->AllocSize, 0)); - uptr TailBegin = RoundUpTo(AllocInfo->UserEnd, ASAN_SHADOW_GRANULARITY); - uptr TailEnd = AllocInfo->AllocBegin + AllocInfo->AllocSize; + uptr TailBegin = RoundUpTo(AI->UserEnd, ASAN_SHADOW_GRANULARITY); + uptr TailEnd = AI->AllocBegin + AI->AllocSize; // User tail - if (TailBegin != AllocInfo->UserEnd) { - auto Value = AllocInfo->UserEnd - - RoundDownTo(AllocInfo->UserEnd, ASAN_SHADOW_GRANULARITY); - UR_CALL(enqueueMemSetShadow(Context, Device, Queue, AllocInfo->UserEnd, - 1, static_cast(Value), LastEvent, - &LastEvent)); + if (TailBegin != AI->UserEnd) { + auto Value = + AI->UserEnd - RoundDownTo(AI->UserEnd, ASAN_SHADOW_GRANULARITY); + UR_CALL(enqueueMemSetShadow(Context, DeviceInfo, Queue, AI->UserEnd, 1, + static_cast(Value))); } int ShadowByte; - switch (AllocInfo->Type) { + switch (AI->Type) { case AllocType::HOST_USM: ShadowByte = kUsmHostRedzoneMagic; break; @@ -529,7 +480,7 @@ ur_result_t SanitizerInterceptor::enqueueAllocInfo( ShadowByte = kMemBufferRedzoneMagic; break; case AllocType::DEVICE_GLOBAL: - ShadowByte = kDeviceGlobalRedZoneMagic; + ShadowByte = kDeviceGlobalRedzoneMagic; break; default: ShadowByte = 0xff; @@ -537,41 +488,26 @@ ur_result_t SanitizerInterceptor::enqueueAllocInfo( } // Left red zone - UR_CALL(enqueueMemSetShadow(Context, Device, Queue, AllocInfo->AllocBegin, - AllocInfo->UserBegin - AllocInfo->AllocBegin, - ShadowByte, LastEvent, &LastEvent)); + UR_CALL(enqueueMemSetShadow(Context, DeviceInfo, Queue, AI->AllocBegin, + AI->UserBegin - AI->AllocBegin, ShadowByte)); // Right red zone - UR_CALL(enqueueMemSetShadow(Context, Device, Queue, TailBegin, - TailEnd - TailBegin, ShadowByte, LastEvent, - &LastEvent)); + UR_CALL(enqueueMemSetShadow(Context, DeviceInfo, Queue, TailBegin, + TailEnd - TailBegin, ShadowByte)); return UR_RESULT_SUCCESS; } -ur_result_t SanitizerInterceptor::updateShadowMemory(ur_queue_handle_t Queue) { - auto Context = getContext(Queue); - auto Device = getDevice(Queue); - assert(Device != nullptr); - - auto ContextInfo = getContextInfo(Context); - - auto DeviceInfo = ContextInfo->getDeviceInfo(Device); - auto QueueInfo = ContextInfo->getQueueInfo(Queue); +ur_result_t SanitizerInterceptor::updateShadowMemory( + std::shared_ptr &ContextInfo, + std::shared_ptr &DeviceInfo, ur_queue_handle_t Queue) { + auto &AllocInfos = ContextInfo->AllocInfosMap[DeviceInfo->Handle]; + std::scoped_lock Guard(AllocInfos.Mutex); - std::unique_lock DeviceGuard(DeviceInfo->Mutex, - std::defer_lock); - std::scoped_lock, ur_mutex> Guard( - DeviceGuard, QueueInfo->Mutex); - - ur_event_handle_t LastEvent = QueueInfo->LastEvent; - - for (auto &AllocInfo : DeviceInfo->AllocInfos) { - UR_CALL(enqueueAllocInfo(Context, Device, Queue, AllocInfo, LastEvent)); + for (auto &AI : AllocInfos.List) { + UR_CALL(enqueueAllocInfo(ContextInfo->Handle, DeviceInfo, Queue, AI)); } - DeviceInfo->AllocInfos.clear(); - - QueueInfo->LastEvent = LastEvent; + AllocInfos.List.clear(); return UR_RESULT_SUCCESS; } @@ -579,30 +515,21 @@ ur_result_t SanitizerInterceptor::updateShadowMemory(ur_queue_handle_t Queue) { ur_result_t SanitizerInterceptor::registerDeviceGlobals(ur_context_handle_t Context, ur_program_handle_t Program) { - std::vector Devices; - getProgramDevices(Program, Devices); + std::vector Devices = GetProgramDevices(Program); + + auto ContextInfo = getContextInfo(Context); for (auto Device : Devices) { - ur_queue_handle_t Queue; - ur_result_t Result = context.urDdiTable.Queue.pfnCreate( - Context, Device, nullptr, &Queue); - if (Result != UR_RESULT_SUCCESS) { - context.logger.error("Failed to create command queue: {}", Result); - return Result; - } + ManagedQueue Queue(Context, Device); uint64_t NumOfDeviceGlobal; - Result = context.urDdiTable.Enqueue.pfnDeviceGlobalVariableRead( + auto Result = context.urDdiTable.Enqueue.pfnDeviceGlobalVariableRead( Queue, Program, kSPIR_AsanDeviceGlobalCount, true, sizeof(NumOfDeviceGlobal), 0, &NumOfDeviceGlobal, 0, nullptr, nullptr); - if (Result == UR_RESULT_ERROR_INVALID_ARGUMENT) { + if (Result != UR_RESULT_SUCCESS) { context.logger.info("No device globals"); continue; - } else if (Result != UR_RESULT_SUCCESS) { - context.logger.error("Device Global[{}] Read Failed: {}", - kSPIR_AsanDeviceGlobalCount, Result); - return Result; } std::vector GVInfos(NumOfDeviceGlobal); @@ -616,28 +543,41 @@ SanitizerInterceptor::registerDeviceGlobals(ur_context_handle_t Context, return Result; } - auto ContextInfo = getContextInfo(Context); - auto DeviceInfo = ContextInfo->getDeviceInfo(Device); + auto DeviceInfo = getDeviceInfo(Device); for (size_t i = 0; i < NumOfDeviceGlobal; i++) { - auto AI = std::make_shared(AllocInfo{ - GVInfos[i].Addr, GVInfos[i].Addr, - GVInfos[i].Addr + GVInfos[i].Size, GVInfos[i].SizeWithRedZone, - AllocType::DEVICE_GLOBAL}); - - std::scoped_lock Guard(DeviceInfo->Mutex); - DeviceInfo->AllocInfos.emplace_back(AI); + auto AI = std::make_shared( + AllocInfo{GVInfos[i].Addr, + GVInfos[i].Addr, + GVInfos[i].Addr + GVInfos[i].Size, + GVInfos[i].SizeWithRedZone, + AllocType::DEVICE_GLOBAL, + false, + Context, + Device, + GetCurrentBacktrace(), + {}}); + + ContextInfo->insertAllocInfo({Device}, AI); } } return UR_RESULT_SUCCESS; } -ur_result_t SanitizerInterceptor::insertContext(ur_context_handle_t Context) { - auto ContextInfo = std::make_shared(); - +ur_result_t +SanitizerInterceptor::insertContext(ur_context_handle_t Context, + std::shared_ptr &CI) { std::scoped_lock Guard(m_ContextMapMutex); - assert(m_ContextMap.find(Context) == m_ContextMap.end()); - m_ContextMap.emplace(Context, std::move(ContextInfo)); + + if (m_ContextMap.find(Context) != m_ContextMap.end()) { + CI = m_ContextMap.at(Context); + return UR_RESULT_SUCCESS; + } + + CI = std::make_shared(Context); + + // Don't move CI, since it's a return value as well + m_ContextMap.emplace(Context, CI); return UR_RESULT_SUCCESS; } @@ -650,172 +590,174 @@ ur_result_t SanitizerInterceptor::eraseContext(ur_context_handle_t Context) { return UR_RESULT_SUCCESS; } -ur_result_t SanitizerInterceptor::insertDevice(ur_context_handle_t Context, - ur_device_handle_t Device) { - auto DeviceInfo = std::make_shared(); +ur_result_t +SanitizerInterceptor::insertDevice(ur_device_handle_t Device, + std::shared_ptr &DI) { + std::scoped_lock Guard(m_DeviceMapMutex); + + if (m_DeviceMap.find(Device) != m_DeviceMap.end()) { + DI = m_DeviceMap.at(Device); + return UR_RESULT_SUCCESS; + } + + DI = std::make_shared(Device); // Query device type - ur_device_type_t DeviceType; - UR_CALL(context.urDdiTable.Device.pfnGetInfo( - Device, UR_DEVICE_INFO_TYPE, sizeof(DeviceType), &DeviceType, nullptr)); - switch (DeviceType) { - case UR_DEVICE_TYPE_CPU: - DeviceInfo->Type = DeviceType::CPU; - break; - case UR_DEVICE_TYPE_GPU: - DeviceInfo->Type = DeviceType::GPU_PVC; - break; - default: - DeviceInfo->Type = DeviceType::UNKNOWN; + DI->Type = GetDeviceType(Device); + if (DI->Type == DeviceType::UNKNOWN) { + return UR_RESULT_ERROR_UNSUPPORTED_FEATURE; } // Query alignment UR_CALL(context.urDdiTable.Device.pfnGetInfo( - Device, UR_DEVICE_INFO_MEM_BASE_ADDR_ALIGN, - sizeof(DeviceInfo->Alignment), &DeviceInfo->Alignment, nullptr)); + Device, UR_DEVICE_INFO_MEM_BASE_ADDR_ALIGN, sizeof(DI->Alignment), + &DI->Alignment, nullptr)); - // Allocate shadow memory - UR_CALL(allocShadowMemory(Context, DeviceInfo)); - - auto ContextInfo = getContextInfo(Context); - std::scoped_lock Guard(ContextInfo->Mutex); - ContextInfo->DeviceMap.emplace(Device, std::move(DeviceInfo)); + // Don't move DI, since it's a return value as well + m_DeviceMap.emplace(Device, DI); return UR_RESULT_SUCCESS; } -ur_result_t SanitizerInterceptor::insertQueue(ur_context_handle_t Context, - ur_queue_handle_t Queue) { - auto QueueInfo = std::make_shared(); - QueueInfo->LastEvent = nullptr; - - auto ContextInfo = getContextInfo(Context); - std::scoped_lock Guard(ContextInfo->Mutex); - ContextInfo->QueueMap.emplace(Queue, std::move(QueueInfo)); - - return UR_RESULT_SUCCESS; -} - -ur_result_t SanitizerInterceptor::eraseQueue(ur_context_handle_t Context, - ur_queue_handle_t Queue) { - auto ContextInfo = getContextInfo(Context); - std::scoped_lock Guard(ContextInfo->Mutex); - assert(ContextInfo->QueueMap.find(Queue) != ContextInfo->QueueMap.end()); - ContextInfo->QueueMap.erase(Queue); +ur_result_t SanitizerInterceptor::eraseDevice(ur_device_handle_t Device) { + std::scoped_lock Guard(m_DeviceMapMutex); + assert(m_DeviceMap.find(Device) != m_DeviceMap.end()); + m_DeviceMap.erase(Device); + // TODO: Remove devices in each context return UR_RESULT_SUCCESS; } -ur_result_t SanitizerInterceptor::prepareLaunch(ur_queue_handle_t Queue, - ur_kernel_handle_t Kernel, - LaunchInfo &LaunchInfo, - uint32_t numWorkgroup) { - auto Context = getContext(Queue); - auto Device = getDevice(Queue); - auto Program = getProgram(Kernel); - - LaunchInfo.Context = Context; - - auto ContextInfo = getContextInfo(Context); - auto DeviceInfo = ContextInfo->getDeviceInfo(Device); - auto QueueInfo = ContextInfo->getQueueInfo(Queue); - - std::scoped_lock Guard(QueueInfo->Mutex); - ur_event_handle_t LastEvent = QueueInfo->LastEvent; +ur_result_t SanitizerInterceptor::prepareLaunch( + ur_context_handle_t Context, std::shared_ptr &DeviceInfo, + ur_queue_handle_t Queue, ur_kernel_handle_t Kernel, + LaunchInfo &LaunchInfo) { + auto Program = GetProgram(Kernel); do { - // Set global variable to program - auto EnqueueWriteGlobal = [&](const char *Name, const void *Value) { - ur_event_handle_t NewEvent{}; - uint32_t NumEvents = LastEvent ? 1 : 0; - const ur_event_handle_t *EventsList = - LastEvent ? &LastEvent : nullptr; + // Write global variable to program + auto EnqueueWriteGlobal = [Queue, Program](const char *Name, + const void *Value, + size_t Size) { auto Result = context.urDdiTable.Enqueue.pfnDeviceGlobalVariableWrite( - Queue, Program, Name, false, sizeof(uptr), 0, Value, - NumEvents, EventsList, &NewEvent); + Queue, Program, Name, false, Size, 0, Value, 0, nullptr, + nullptr); if (Result != UR_RESULT_SUCCESS) { - context.logger.warning("Device Global[{}] Write Failed: {}", - Name, Result); + context.logger.warning( + "Failed to write device global \"{}\": {}", Name, Result); return false; } - LastEvent = NewEvent; return true; }; + // Write debug + EnqueueWriteGlobal(kSPIR_AsanDebug, &cl_Debug, sizeof(cl_Debug)); + // Write shadow memory offset for global memory EnqueueWriteGlobal(kSPIR_AsanShadowMemoryGlobalStart, - &DeviceInfo->ShadowOffset); + &DeviceInfo->ShadowOffset, + sizeof(DeviceInfo->ShadowOffset)); EnqueueWriteGlobal(kSPIR_AsanShadowMemoryGlobalEnd, - &DeviceInfo->ShadowOffsetEnd); + &DeviceInfo->ShadowOffsetEnd, + sizeof(DeviceInfo->ShadowOffsetEnd)); // Write device type - EnqueueWriteGlobal(kSPIR_DeviceType, &DeviceInfo->Type); + EnqueueWriteGlobal(kSPIR_DeviceType, &DeviceInfo->Type, + sizeof(DeviceInfo->Type)); if (DeviceInfo->Type == DeviceType::CPU) { break; } - // Write shadow memory offset for local memory - auto LocalMemorySize = getLocalMemorySize(Device); - auto LocalShadowMemorySize = - (numWorkgroup * LocalMemorySize) >> ASAN_SHADOW_SCALE; - - context.logger.info("LocalInfo(WorkGroup={}, LocalMemorySize={}, " - "LocalShadowMemorySize={})", - numWorkgroup, LocalMemorySize, - LocalShadowMemorySize); - - ur_usm_desc_t Desc{UR_STRUCTURE_TYPE_USM_HOST_DESC, nullptr, 0, 0}; - auto Result = context.urDdiTable.USM.pfnDeviceAlloc( - Context, Device, &Desc, nullptr, LocalShadowMemorySize, - (void **)&LaunchInfo.LocalShadowOffset); - if (Result != UR_RESULT_SUCCESS) { - context.logger.error( - "Failed to allocate shadow memory for local memory: {}", - numWorkgroup, Result); - context.logger.error("Maybe the number of workgroup too large"); - return Result; + if (LaunchInfo.LocalWorkSize.empty()) { + LaunchInfo.LocalWorkSize.reserve(3); + // FIXME: This is W/A until urKernelSuggestGroupSize is added + LaunchInfo.LocalWorkSize[0] = 1; + LaunchInfo.LocalWorkSize[1] = 1; + LaunchInfo.LocalWorkSize[2] = 1; } - LaunchInfo.LocalShadowOffsetEnd = - LaunchInfo.LocalShadowOffset + LocalShadowMemorySize - 1; - EnqueueWriteGlobal(kSPIR_AsanShadowMemoryLocalStart, - &LaunchInfo.LocalShadowOffset); - EnqueueWriteGlobal(kSPIR_AsanShadowMemoryLocalEnd, - &LaunchInfo.LocalShadowOffsetEnd); + const size_t *LocalWorkSize = LaunchInfo.LocalWorkSize.data(); + uint32_t NumWG = 1; + for (uint32_t Dim = 0; Dim < LaunchInfo.WorkDim; ++Dim) { + NumWG *= (LaunchInfo.GlobalWorkSize[Dim] + LocalWorkSize[Dim] - 1) / + LocalWorkSize[Dim]; + } - { - ur_event_handle_t NewEvent{}; - uint32_t NumEvents = LastEvent ? 1 : 0; - const ur_event_handle_t *EventsList = - LastEvent ? &LastEvent : nullptr; - const char Pattern[] = {0}; - - auto URes = context.urDdiTable.Enqueue.pfnUSMFill( - Queue, (void *)LaunchInfo.LocalShadowOffset, 1, Pattern, - LocalShadowMemorySize, NumEvents, EventsList, &NewEvent); + auto EnqueueAllocateDevice = [Context, &DeviceInfo, Queue, + NumWG](size_t Size, uptr &Ptr) { + auto URes = context.urDdiTable.USM.pfnDeviceAlloc( + Context, DeviceInfo->Handle, nullptr, nullptr, Size, + (void **)&Ptr); if (URes != UR_RESULT_SUCCESS) { - context.logger.error("urEnqueueUSMFill(): {}", URes); + context.logger.error( + "Failed to allocate shadow memory for local memory: {}", + URes); + context.logger.error( + "Maybe the number of workgroup ({}) too large", NumWG); return URes; } - LastEvent = NewEvent; - } + // Initialize shadow memory of local memory + URes = urEnqueueUSMSet(Queue, (void *)Ptr, 0, Size); + if (URes == UR_RESULT_ERROR_OUT_OF_DEVICE_MEMORY) { + context.logger.error( + "Failed to allocate shadow memory for local memory: {}", + URes); + context.logger.error( + "Maybe the number of workgroup ({}) too large", NumWG); + return URes; + } + return URes; + }; - context.logger.info("ShadowMemory(Local, {} - {})", - (void *)LaunchInfo.LocalShadowOffset, - (void *)LaunchInfo.LocalShadowOffsetEnd); + // Write shadow memory offset for local memory + if (cl_DetectLocals) { + // CPU needn't this + if (DeviceInfo->Type == DeviceType::GPU_PVC) { + size_t LocalMemorySize = GetLocalMemorySize(DeviceInfo->Handle); + size_t LocalShadowMemorySize = + (NumWG * LocalMemorySize) >> ASAN_SHADOW_SCALE; + + context.logger.debug( + "LocalMemoryInfo(WorkGroup={}, LocalMemorySize={}, " + "LocalShadowMemorySize={})", + NumWG, LocalMemorySize, LocalShadowMemorySize); + + UR_CALL(EnqueueAllocateDevice(LocalShadowMemorySize, + LaunchInfo.LocalShadowOffset)); + + LaunchInfo.LocalShadowOffsetEnd = + LaunchInfo.LocalShadowOffset + LocalShadowMemorySize - 1; + + context.logger.info("ShadowMemory(Local, {} - {})", + (void *)LaunchInfo.LocalShadowOffset, + (void *)LaunchInfo.LocalShadowOffsetEnd); + } + } } while (false); - QueueInfo->LastEvent = LastEvent; return UR_RESULT_SUCCESS; } +std::optional +SanitizerInterceptor::findAllocInfoByAddress(uptr Address) { + std::shared_lock Guard(m_AllocationMapMutex); + auto It = m_AllocationMap.upper_bound(Address); + if (It == m_AllocationMap.begin()) { + return std::optional{}; + } + return --It; +} + LaunchInfo::~LaunchInfo() { + [[maybe_unused]] ur_result_t Result; if (LocalShadowOffset) { - [[maybe_unused]] auto Result = + Result = context.urDdiTable.USM.pfnFree(Context, (void *)LocalShadowOffset); assert(Result == UR_RESULT_SUCCESS); } + Result = context.urDdiTable.Context.pfnRelease(Context); + assert(Result == UR_RESULT_SUCCESS); } } // namespace ur_sanitizer_layer diff --git a/source/loader/layers/sanitizer/asan_interceptor.hpp b/source/loader/layers/sanitizer/asan_interceptor.hpp index c13d1a3b61..a691bee7b7 100644 --- a/source/loader/layers/sanitizer/asan_interceptor.hpp +++ b/source/loader/layers/sanitizer/asan_interceptor.hpp @@ -12,92 +12,125 @@ #pragma once +#include "asan_allocator.hpp" +#include "asan_libdevice.hpp" #include "common.hpp" -#include "device_sanitizer_report.hpp" +#include "ur_sanitizer_layer.hpp" -#include #include +#include +#include #include #include namespace ur_sanitizer_layer { -enum class AllocType : uint32_t { - DEVICE_USM, - SHARED_USM, - HOST_USM, - MEM_BUFFER, - DEVICE_GLOBAL -}; +class Quarantine; -struct AllocInfo { - uptr AllocBegin; - uptr UserBegin; - uptr UserEnd; - size_t AllocSize; - AllocType Type; +struct AllocInfoList { + std::vector> List; + ur_shared_mutex Mutex; }; -enum class DeviceType { UNKNOWN, CPU, GPU_PVC, GPU_DG2 }; - struct DeviceInfo { - DeviceType Type; - size_t Alignment; - uptr ShadowOffset; - uptr ShadowOffsetEnd; + ur_device_handle_t Handle; - // Lock InitPool & AllocInfos - ur_shared_mutex Mutex; - std::vector> AllocInfos; + DeviceType Type = DeviceType::UNKNOWN; + size_t Alignment = 0; + uptr ShadowOffset = 0; + uptr ShadowOffsetEnd = 0; + + ur_mutex Mutex; + std::queue> Quarantine; + size_t QuarantineSize = 0; + + explicit DeviceInfo(ur_device_handle_t Device) : Handle(Device) { + [[maybe_unused]] auto Result = + context.urDdiTable.Device.pfnRetain(Device); + assert(Result == UR_RESULT_SUCCESS); + } + + ~DeviceInfo() { + [[maybe_unused]] auto Result = + context.urDdiTable.Device.pfnRelease(Handle); + assert(Result == UR_RESULT_SUCCESS); + } + + ur_result_t allocShadowMemory(ur_context_handle_t Context); }; struct QueueInfo { - ur_mutex Mutex; + ur_queue_handle_t Handle; + + ur_shared_mutex Mutex; ur_event_handle_t LastEvent; + + explicit QueueInfo(ur_queue_handle_t Queue) + : Handle(Queue), LastEvent(nullptr) { + [[maybe_unused]] auto Result = + context.urDdiTable.Queue.pfnRetain(Queue); + assert(Result == UR_RESULT_SUCCESS); + } + + ~QueueInfo() { + [[maybe_unused]] auto Result = + context.urDdiTable.Queue.pfnRelease(Handle); + assert(Result == UR_RESULT_SUCCESS); + } }; struct ContextInfo { + ur_context_handle_t Handle; - std::shared_ptr getDeviceInfo(ur_device_handle_t Device) { - std::shared_lock Guard(Mutex); - assert(DeviceMap.find(Device) != DeviceMap.end()); - return DeviceMap[Device]; - } + std::vector DeviceList; + std::unordered_map AllocInfosMap; - std::shared_ptr getQueueInfo(ur_queue_handle_t Queue) { - std::shared_lock Guard(Mutex); - assert(QueueMap.find(Queue) != QueueMap.end()); - return QueueMap[Queue]; + explicit ContextInfo(ur_context_handle_t Context) : Handle(Context) { + [[maybe_unused]] auto Result = + context.urDdiTable.Context.pfnRetain(Context); + assert(Result == UR_RESULT_SUCCESS); } - std::shared_ptr getUSMAllocInfo(uptr Address) { - std::shared_lock Guard(Mutex); - assert(AllocatedUSMMap.find(Address) != AllocatedUSMMap.end()); - return AllocatedUSMMap[Address]; + ~ContextInfo() { + [[maybe_unused]] auto Result = + context.urDdiTable.Context.pfnRelease(Handle); + assert(Result == UR_RESULT_SUCCESS); } - ur_shared_mutex Mutex; - std::unordered_map> - DeviceMap; - std::unordered_map> QueueMap; - - /// key: USMAllocInfo.AllocBegin - /// value: USMAllocInfo - /// Use AllocBegin as key can help to detect underflow pointer - std::map> AllocatedUSMMap; + void insertAllocInfo(const std::vector &Devices, + std::shared_ptr &AI) { + for (auto Device : Devices) { + auto &AllocInfos = AllocInfosMap[Device]; + std::scoped_lock Guard(AllocInfos.Mutex); + AllocInfos.List.emplace_back(AI); + } + } }; struct LaunchInfo { - uptr LocalShadowOffset; - uptr LocalShadowOffsetEnd; - ur_context_handle_t Context; - + uptr LocalShadowOffset = 0; + uptr LocalShadowOffsetEnd = 0; DeviceSanitizerReport SPIR_DeviceSanitizerReportMem; - size_t LocalWorkSize[3]; - - LaunchInfo() - : LocalShadowOffset(0), LocalShadowOffsetEnd(0), Context(nullptr) {} + ur_context_handle_t Context = nullptr; + const size_t *GlobalWorkSize = nullptr; + const size_t *GlobalWorkOffset = nullptr; + std::vector LocalWorkSize; + uint32_t WorkDim = 0; + + LaunchInfo(ur_context_handle_t Context, const size_t *GlobalWorkSize, + const size_t *LocalWorkSize, const size_t *GlobalWorkOffset, + uint32_t WorkDim) + : Context(Context), GlobalWorkSize(GlobalWorkSize), + GlobalWorkOffset(GlobalWorkOffset), WorkDim(WorkDim) { + [[maybe_unused]] auto Result = + context.urDdiTable.Context.pfnRetain(Context); + assert(Result == UR_RESULT_SUCCESS); + if (LocalWorkSize) { + this->LocalWorkSize = + std::vector(LocalWorkSize, LocalWorkSize + WorkDim); + } + } ~LaunchInfo(); }; @@ -109,7 +142,7 @@ struct DeviceGlobalInfo { class SanitizerInterceptor { public: - SanitizerInterceptor(); + explicit SanitizerInterceptor(); ~SanitizerInterceptor(); @@ -117,7 +150,7 @@ class SanitizerInterceptor { ur_device_handle_t Device, const ur_usm_desc_t *Properties, ur_usm_pool_handle_t Pool, size_t Size, - void **ResultPtr, AllocType Type); + AllocType Type, void **ResultPtr); ur_result_t releaseMemory(ur_context_handle_t Context, void *Ptr); ur_result_t registerDeviceGlobals(ur_context_handle_t Context, @@ -125,48 +158,52 @@ class SanitizerInterceptor { ur_result_t preLaunchKernel(ur_kernel_handle_t Kernel, ur_queue_handle_t Queue, - ur_event_handle_t &Event, - LaunchInfo &LaunchInfo, uint32_t numWorkgroup); - void postLaunchKernel(ur_kernel_handle_t Kernel, ur_queue_handle_t Queue, - ur_event_handle_t &Event, LaunchInfo &LaunchInfo); + LaunchInfo &LaunchInfo); + + ur_result_t postLaunchKernel(ur_kernel_handle_t Kernel, + ur_queue_handle_t Queue, + ur_event_handle_t &Event, + LaunchInfo &LaunchInfo); - ur_result_t insertContext(ur_context_handle_t Context); + ur_result_t insertContext(ur_context_handle_t Context, + std::shared_ptr &CI); ur_result_t eraseContext(ur_context_handle_t Context); - ur_result_t insertDevice(ur_context_handle_t Context, - ur_device_handle_t Device); + ur_result_t insertDevice(ur_device_handle_t Device, + std::shared_ptr &CI); + ur_result_t eraseDevice(ur_device_handle_t Device); - ur_result_t insertQueue(ur_context_handle_t Context, - ur_queue_handle_t Queue); - ur_result_t eraseQueue(ur_context_handle_t Context, - ur_queue_handle_t Queue); + std::optional findAllocInfoByAddress(uptr Address); + + std::shared_ptr getContextInfo(ur_context_handle_t Context) { + std::shared_lock Guard(m_ContextMapMutex); + assert(m_ContextMap.find(Context) != m_ContextMap.end()); + return m_ContextMap[Context]; + } private: - ur_result_t updateShadowMemory(ur_queue_handle_t Queue); + ur_result_t updateShadowMemory(std::shared_ptr &ContextInfo, + std::shared_ptr &DeviceInfo, + ur_queue_handle_t Queue); ur_result_t enqueueAllocInfo(ur_context_handle_t Context, - ur_device_handle_t Device, + std::shared_ptr &DeviceInfo, ur_queue_handle_t Queue, - std::shared_ptr &AI, - ur_event_handle_t &LastEvent); + std::shared_ptr &AI); /// Initialize Global Variables & Kernel Name at first Launch - ur_result_t prepareLaunch(ur_queue_handle_t Queue, - ur_kernel_handle_t Kernel, LaunchInfo &LaunchInfo, - uint32_t numWorkgroup); + ur_result_t prepareLaunch(ur_context_handle_t Context, + std::shared_ptr &DeviceInfo, + ur_queue_handle_t Queue, + ur_kernel_handle_t Kernel, + LaunchInfo &LaunchInfo); ur_result_t allocShadowMemory(ur_context_handle_t Context, std::shared_ptr &DeviceInfo); - ur_result_t enqueueMemSetShadow(ur_context_handle_t Context, - ur_device_handle_t Device, - ur_queue_handle_t Queue, uptr Addr, - uptr Size, u8 Value, - ur_event_handle_t DepEvent, - ur_event_handle_t *OutEvent); - std::shared_ptr getContextInfo(ur_context_handle_t Context) { - std::shared_lock Guard(m_ContextMapMutex); - assert(m_ContextMap.find(Context) != m_ContextMap.end()); - return m_ContextMap[Context]; + std::shared_ptr getDeviceInfo(ur_device_handle_t Device) { + std::shared_lock Guard(m_DeviceMapMutex); + assert(m_DeviceMap.find(Device) != m_DeviceMap.end()); + return m_DeviceMap[Device]; } private: @@ -174,25 +211,20 @@ class SanitizerInterceptor { m_ContextMap; ur_shared_mutex m_ContextMapMutex; - bool m_IsInASanContext; - bool m_ShadowMemInited; -}; + std::unordered_map> + m_DeviceMap; + ur_shared_mutex m_DeviceMapMutex; -inline const char *ToString(AllocType Type) { - switch (Type) { - case AllocType::DEVICE_USM: - return "Device USM"; - case AllocType::HOST_USM: - return "Host USM"; - case AllocType::SHARED_USM: - return "Shared USM"; - case AllocType::MEM_BUFFER: - return "Memory Buffer"; - case AllocType::DEVICE_GLOBAL: - return "Device Global"; - default: - return "Unknown Type"; - } -} + /// Assumption: all USM chunks are allocated in one VA + AllocationMap m_AllocationMap; + ur_shared_mutex m_AllocationMapMutex; + + // We use "uint64_t" here because EnqueueWriteGlobal will fail when it's "uint32_t" + uint64_t cl_Debug = 0; + uint32_t cl_MaxQuarantineSizeMB = 0; + bool cl_DetectLocals = true; + + std::unique_ptr m_Quarantine; +}; } // namespace ur_sanitizer_layer diff --git a/source/loader/layers/sanitizer/device_sanitizer_report.hpp b/source/loader/layers/sanitizer/asan_libdevice.hpp similarity index 54% rename from source/loader/layers/sanitizer/device_sanitizer_report.hpp rename to source/loader/layers/sanitizer/asan_libdevice.hpp index 374ffc8d62..46ddee4423 100644 --- a/source/loader/layers/sanitizer/device_sanitizer_report.hpp +++ b/source/loader/layers/sanitizer/asan_libdevice.hpp @@ -22,6 +22,8 @@ enum class DeviceSanitizerErrorType : int32_t { MISALIGNED, USE_AFTER_FREE, OUT_OF_SHADOW_BOUNDS, + UNKNOWN_DEVICE, + NULL_POINTER, }; enum class DeviceSanitizerMemoryType : int32_t { @@ -38,8 +40,8 @@ enum class DeviceSanitizerMemoryType : int32_t { struct DeviceSanitizerReport { int Flag = 0; - char File[256 + 1] = ""; - char Func[256 + 1] = ""; + char File[256 + 1] = {}; + char Func[256 + 1] = {}; int32_t Line = 0; @@ -51,14 +53,49 @@ struct DeviceSanitizerReport { uint64_t LID1 = 0; uint64_t LID2 = 0; + uintptr_t Address = 0; bool IsWrite = false; uint32_t AccessSize = 0; - DeviceSanitizerMemoryType MemoryType; - DeviceSanitizerErrorType ErrorType; + DeviceSanitizerMemoryType MemoryType = DeviceSanitizerMemoryType::UNKNOWN; + DeviceSanitizerErrorType ErrorType = DeviceSanitizerErrorType::UNKNOWN; bool IsRecover = false; }; +constexpr unsigned ASAN_SHADOW_SCALE = 3; +constexpr unsigned ASAN_SHADOW_GRANULARITY = 1ULL << ASAN_SHADOW_SCALE; + +// These magic values are written to shadow for better error +// reporting. +constexpr int kUsmDeviceRedzoneMagic = (char)0x81; +constexpr int kUsmHostRedzoneMagic = (char)0x82; +constexpr int kUsmSharedRedzoneMagic = (char)0x83; +constexpr int kMemBufferRedzoneMagic = (char)0x84; +constexpr int kDeviceGlobalRedzoneMagic = (char)0x85; +constexpr int kNullPointerRedzoneMagic = (char)0x86; + +constexpr int kUsmDeviceDeallocatedMagic = (char)0x91; +constexpr int kUsmHostDeallocatedMagic = (char)0x92; +constexpr int kUsmSharedDeallocatedMagic = (char)0x93; +constexpr int kMemBufferDeallocatedMagic = (char)0x93; + +constexpr int kSharedLocalRedzoneMagic = (char)0xa1; + +// Same with host ASan stack +const int kPrivateLeftRedzoneMagic = (char)0xf1; +const int kPrivateMidRedzoneMagic = (char)0xf2; +const int kPrivateRightRedzoneMagic = (char)0xf3; + +constexpr auto kSPIR_AsanShadowMemoryGlobalStart = + "__AsanShadowMemoryGlobalStart"; +constexpr auto kSPIR_AsanShadowMemoryGlobalEnd = "__AsanShadowMemoryGlobalEnd"; + +constexpr auto kSPIR_DeviceType = "__DeviceType"; +constexpr auto kSPIR_AsanDebug = "__AsanDebug"; + +constexpr auto kSPIR_AsanDeviceGlobalCount = "__AsanDeviceGlobalCount"; +constexpr auto kSPIR_AsanDeviceGlobalMetadata = "__AsanDeviceGlobalMetadata"; + inline const char *ToString(DeviceSanitizerMemoryType MemoryType) { switch (MemoryType) { case DeviceSanitizerMemoryType::USM_DEVICE: @@ -90,6 +127,10 @@ inline const char *ToString(DeviceSanitizerErrorType ErrorType) { return "use-after-free"; case DeviceSanitizerErrorType::OUT_OF_SHADOW_BOUNDS: return "out-of-shadow-bounds-access"; + case DeviceSanitizerErrorType::UNKNOWN_DEVICE: + return "unknown-device"; + case DeviceSanitizerErrorType::NULL_POINTER: + return "null-pointer-access"; default: return "unknown-error"; } diff --git a/source/loader/layers/sanitizer/asan_quarantine.cpp b/source/loader/layers/sanitizer/asan_quarantine.cpp new file mode 100644 index 0000000000..9826aeb62a --- /dev/null +++ b/source/loader/layers/sanitizer/asan_quarantine.cpp @@ -0,0 +1,36 @@ +/* + * + * Copyright (C) 2024 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 + * + * @file asan_quarantine.cpp + * + */ + +#include "asan_quarantine.hpp" + +namespace ur_sanitizer_layer { + +std::vector Quarantine::put(ur_device_handle_t Device, + AllocationIterator &It) { + auto &AI = It->second; + auto AllocSize = AI->AllocSize; + auto &Cache = getCache(Device); + + std::vector DequeueList; + std::scoped_lock Guard(Cache.Mutex); + while (Cache.size() + AllocSize > m_MaxQuarantineSize) { + auto ElementOp = Cache.dequeue(); + if (!ElementOp) { + break; + } + DequeueList.emplace_back(*ElementOp); + } + Cache.enqueue(It); + return DequeueList; +} + +} // namespace ur_sanitizer_layer diff --git a/source/loader/layers/sanitizer/asan_quarantine.hpp b/source/loader/layers/sanitizer/asan_quarantine.hpp new file mode 100644 index 0000000000..6dc15d382a --- /dev/null +++ b/source/loader/layers/sanitizer/asan_quarantine.hpp @@ -0,0 +1,74 @@ +/* + * + * Copyright (C) 2024 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 + * + * @file asan_quarantine.hpp + * + */ + +#pragma once + +#include "asan_allocator.hpp" + +#include +#include +#include +#include + +namespace ur_sanitizer_layer { + +class QuarantineCache { + public: + using Element = AllocationIterator; + using List = std::queue; + + // The following methods are not thread safe, use this lock + ur_mutex Mutex; + + // Total memory used, including internal accounting. + uptr size() const { return m_Size; } + + void enqueue(Element &It) { + m_List.push(It); + m_Size += It->second->AllocSize; + } + + std::optional dequeue() { + if (m_List.empty()) { + return std::optional{}; + } + auto It = m_List.front(); + m_List.pop(); + m_Size -= It->second->AllocSize; + return It; + } + + private: + List m_List; + std::atomic_uintptr_t m_Size = 0; +}; + +class Quarantine { + public: + explicit Quarantine(size_t MaxQuarantineSize) + : m_MaxQuarantineSize(MaxQuarantineSize) {} + + std::vector put(ur_device_handle_t Device, + AllocationIterator &Ptr); + + private: + QuarantineCache &getCache(ur_device_handle_t Device) { + std::scoped_lock Guard(m_Mutex); + return m_Map[Device]; + } + + std::unordered_map m_Map; + ur_mutex m_Mutex; + size_t m_MaxQuarantineSize; +}; + +} // namespace ur_sanitizer_layer diff --git a/source/loader/layers/sanitizer/asan_report.cpp b/source/loader/layers/sanitizer/asan_report.cpp new file mode 100644 index 0000000000..56cbbb4b21 --- /dev/null +++ b/source/loader/layers/sanitizer/asan_report.cpp @@ -0,0 +1,160 @@ +/* + * + * Copyright (C) 2024 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 + * + * @file asan_report.cpp + * + */ + +#include "asan_report.hpp" + +#include "asan_allocator.hpp" +#include "asan_interceptor.hpp" +#include "asan_libdevice.hpp" +#include "ur_sanitizer_layer.hpp" +#include "ur_sanitizer_utils.hpp" + +namespace ur_sanitizer_layer { + +void ReportBadFree(uptr Addr, const StackTrace &stack, + const std::shared_ptr &AI) { + context.logger.always( + "\n====ERROR: DeviceSanitizer: bad-free on address {}", (void *)Addr); + stack.print(); + + if (!AI) { + context.logger.always("{} may be allocated on Host Memory", + (void *)Addr); + exit(1); + } + + assert(!AI->IsReleased && "Chunk must be not released"); + + context.logger.always("{} is located inside of {} region [{}, {})", + (void *)Addr, ToString(AI->Type), + (void *)AI->UserBegin, (void *)AI->UserEnd); + context.logger.always("allocated here:"); + AI->AllocStack.print(); + + exit(1); +} + +void ReportBadContext(uptr Addr, const StackTrace &stack, + const std::shared_ptr &AI) { + context.logger.always( + "\n====ERROR: DeviceSanitizer: bad-context on address {}", + (void *)Addr); + stack.print(); + + context.logger.always("{} is located inside of {} region [{}, {})", + (void *)Addr, ToString(AI->Type), + (void *)AI->UserBegin, (void *)AI->UserEnd); + context.logger.always("allocated here:"); + AI->AllocStack.print(); + + if (AI->IsReleased) { + context.logger.always("freed here:"); + AI->ReleaseStack.print(); + } + + exit(1); +} + +void ReportDoubleFree(uptr Addr, const StackTrace &Stack, + const std::shared_ptr &AI) { + context.logger.always( + "\n====ERROR: DeviceSanitizer: double-free on address {}", + (void *)Addr); + Stack.print(); + + context.logger.always("{} is located inside of {} region [{}, {})", + (void *)Addr, ToString(AI->Type), + (void *)AI->UserBegin, (void *)AI->UserEnd); + context.logger.always("freed here:"); + AI->ReleaseStack.print(); + context.logger.always("previously allocated here:"); + AI->AllocStack.print(); + exit(1); +} + +void ReportGenericError(const DeviceSanitizerReport &Report) { + context.logger.always("\n====ERROR: DeviceSanitizer: {}", + ToString(Report.ErrorType)); + exit(1); +} + +void ReportOutOfBoundsError(const DeviceSanitizerReport &Report, + ur_kernel_handle_t Kernel) { + const char *File = Report.File[0] ? Report.File : ""; + const char *Func = Report.Func[0] ? Report.Func : ""; + auto KernelName = GetKernelName(Kernel); + + // Try to demangle the kernel name + KernelName = DemangleName(KernelName); + + context.logger.always("\n====ERROR: DeviceSanitizer: {} on {}", + ToString(Report.ErrorType), + ToString(Report.MemoryType)); + context.logger.always( + "{} of size {} at kernel <{}> LID({}, {}, {}) GID({}, " + "{}, {})", + Report.IsWrite ? "WRITE" : "READ", Report.AccessSize, + KernelName.c_str(), Report.LID0, Report.LID1, Report.LID2, Report.GID0, + Report.GID1, Report.GID2); + context.logger.always(" #0 {} {}:{}", Func, File, Report.Line); + + exit(1); +} + +void ReportUseAfterFree(const DeviceSanitizerReport &Report, + ur_kernel_handle_t Kernel, + ur_context_handle_t Context) { + const char *File = Report.File[0] ? Report.File : ""; + const char *Func = Report.Func[0] ? Report.Func : ""; + auto KernelName = GetKernelName(Kernel); + + // Try to demangle the kernel name + KernelName = DemangleName(KernelName); + + context.logger.always("\n====ERROR: DeviceSanitizer: {} on address {}", + ToString(Report.ErrorType), (void *)Report.Address); + context.logger.always( + "{} of size {} at kernel <{}> LID({}, {}, {}) GID({}, " + "{}, {})", + Report.IsWrite ? "WRITE" : "READ", Report.AccessSize, + KernelName.c_str(), Report.LID0, Report.LID1, Report.LID2, Report.GID0, + Report.GID1, Report.GID2); + context.logger.always(" #0 {} {}:{}", Func, File, Report.Line); + context.logger.always(""); + + auto AllocInfoItOp = + context.interceptor->findAllocInfoByAddress(Report.Address); + if (!AllocInfoItOp) { + context.logger.always("Failed to find which chunck {} is allocated", + (void *)Report.Address); + } else { + auto &AllocInfo = (*AllocInfoItOp)->second; + if (AllocInfo->Context != Context) { + context.logger.always("Failed to find which chunck {} is allocated", + (void *)Report.Address); + } + assert(AllocInfo->IsReleased); + + context.logger.always("{} is located inside of {} region [{}, {})", + (void *)Report.Address, ToString(AllocInfo->Type), + (void *)AllocInfo->UserBegin, + (void *)AllocInfo->UserEnd); + context.logger.always("allocated here:"); + AllocInfo->AllocStack.print(); + context.logger.always("released here:"); + AllocInfo->ReleaseStack.print(); + } + + exit(1); +} + +} // namespace ur_sanitizer_layer diff --git a/source/loader/layers/sanitizer/asan_report.hpp b/source/loader/layers/sanitizer/asan_report.hpp new file mode 100644 index 0000000000..d107b271c0 --- /dev/null +++ b/source/loader/layers/sanitizer/asan_report.hpp @@ -0,0 +1,42 @@ +/* + * + * Copyright (C) 2024 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 + * + * @file asan_report.hpp + * + */ + +#pragma once + +#include "common.hpp" + +#include + +namespace ur_sanitizer_layer { + +struct DeviceSanitizerReport; +struct AllocInfo; +struct StackTrace; + +void ReportBadFree(uptr Addr, const StackTrace &stack, + const std::shared_ptr &AllocInfo); + +void ReportBadContext(uptr Addr, const StackTrace &stack, + const std::shared_ptr &AllocInfos); + +void ReportDoubleFree(uptr Addr, const StackTrace &Stack, + const std::shared_ptr &AllocInfo); + +void ReportGenericError(const DeviceSanitizerReport &Report); + +void ReportOutOfBoundsError(const DeviceSanitizerReport &Report, + ur_kernel_handle_t Kernel); + +void ReportUseAfterFree(const DeviceSanitizerReport &Report, + ur_kernel_handle_t Kernel, ur_context_handle_t Context); + +} // namespace ur_sanitizer_layer diff --git a/source/loader/layers/sanitizer/asan_shadow_setup.cpp b/source/loader/layers/sanitizer/asan_shadow_setup.cpp new file mode 100644 index 0000000000..3ff58b6bed --- /dev/null +++ b/source/loader/layers/sanitizer/asan_shadow_setup.cpp @@ -0,0 +1,137 @@ +/* + * + * Copyright (C) 2024 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 + * + * @file asan_shadow_setup.cpp + * + */ + +#include "asan_shadow_setup.hpp" +#include "ur_sanitizer_layer.hpp" + +namespace ur_sanitizer_layer { + +namespace cpu { + +// Based on "compiler-rt/lib/asan/asan_mapping.h" +// Typical shadow mapping on Linux/x86_64 with SHADOW_OFFSET == 0x00007fff8000: +constexpr uptr LOW_SHADOW_BEGIN = 0x00007fff8000ULL; +constexpr uptr LOW_SHADOW_END = 0x00008fff6fffULL; +constexpr uptr SHADOW_GAP_BEGIN = 0x00008fff7000ULL; +constexpr uptr SHADOW_GAP_END = 0x02008fff6fffULL; +constexpr uptr HIGH_SHADOW_BEGIN = 0x02008fff7000ULL; +constexpr uptr HIGH_SHADOW_END = 0x10007fff7fffULL; +constexpr uptr LOW_SHADOW_SIZE = LOW_SHADOW_END - LOW_SHADOW_BEGIN; +constexpr uptr SHADOW_GAP_SIZE = SHADOW_GAP_END - SHADOW_GAP_BEGIN; +constexpr uptr HIGH_SHADOW_SIZE = HIGH_SHADOW_END - HIGH_SHADOW_BEGIN; + +bool IsShadowMemInited; + +ur_result_t SetupShadowMemory(uptr &ShadowBegin, uptr &ShadowEnd) { + static ur_result_t Result = []() { + if (!MmapFixedNoReserve(LOW_SHADOW_BEGIN, LOW_SHADOW_SIZE)) { + return UR_RESULT_ERROR_OUT_OF_HOST_MEMORY; + } + if (!MmapFixedNoReserve(HIGH_SHADOW_BEGIN, HIGH_SHADOW_SIZE)) { + return UR_RESULT_ERROR_OUT_OF_HOST_MEMORY; + } + if (!MmapFixedNoAccess(SHADOW_GAP_BEGIN, SHADOW_GAP_SIZE)) { + return UR_RESULT_ERROR_OUT_OF_HOST_MEMORY; + } + IsShadowMemInited = true; + return UR_RESULT_SUCCESS; + }(); + ShadowBegin = LOW_SHADOW_BEGIN; + ShadowEnd = HIGH_SHADOW_END; + return Result; +} + +ur_result_t DestroyShadowMemory() { + static ur_result_t Result = []() { + if (!IsShadowMemInited) { + return UR_RESULT_SUCCESS; + } + if (!Munmap(LOW_SHADOW_BEGIN, LOW_SHADOW_SIZE)) { + return UR_RESULT_ERROR_UNKNOWN; + } + if (!Munmap(HIGH_SHADOW_BEGIN, HIGH_SHADOW_SIZE)) { + return UR_RESULT_ERROR_UNKNOWN; + } + if (!Munmap(SHADOW_GAP_BEGIN, SHADOW_GAP_SIZE)) { + return UR_RESULT_ERROR_UNKNOWN; + } + return UR_RESULT_SUCCESS; + }(); + return Result; +} + +} // namespace cpu + +namespace pvc { + +/// SHADOW MEMORY MAPPING (PVC, with CPU 47bit) +/// Host/Shared USM : 0x0 ~ 0x0fff_ffff_ffff +/// ? : 0x1000_0000_0000 ~ 0x1fff_ffff_ffff +/// Device USM : 0x2000_0000_0000 ~ 0x3fff_ffff_ffff +constexpr size_t SHADOW_SIZE = 1ULL << 46; + +uptr LOW_SHADOW_BEGIN; +uptr HIGH_SHADOW_END; + +ur_context_handle_t ShadowContext; + +ur_result_t SetupShadowMemory(ur_context_handle_t Context, uptr &ShadowBegin, + uptr &ShadowEnd) { + // Currently, Level-Zero doesn't create independent VAs for each contexts, if we reserve + // shadow memory for each contexts, this will cause out-of-resource error when user uses + // multiple contexts. Therefore, we just create one shadow memory here. + static ur_result_t Result = [&Context]() { + // TODO: Protect Bad Zone + auto Result = context.urDdiTable.VirtualMem.pfnReserve( + Context, nullptr, SHADOW_SIZE, (void **)&LOW_SHADOW_BEGIN); + if (Result == UR_RESULT_SUCCESS) { + HIGH_SHADOW_END = LOW_SHADOW_BEGIN + SHADOW_SIZE; + // Retain the context which reserves shadow memory + ShadowContext = Context; + context.urDdiTable.Context.pfnRetain(Context); + } + return Result; + }(); + ShadowBegin = LOW_SHADOW_BEGIN; + ShadowEnd = HIGH_SHADOW_END; + return Result; +} + +ur_result_t DestroyShadowMemory() { + static ur_result_t Result = []() { + if (!ShadowContext) { + return UR_RESULT_SUCCESS; + } + auto Result = context.urDdiTable.VirtualMem.pfnFree( + ShadowContext, (const void *)LOW_SHADOW_BEGIN, SHADOW_SIZE); + context.urDdiTable.Context.pfnRelease(ShadowContext); + return Result; + }(); + return Result; +} + +} // namespace pvc + +ur_result_t SetupShadowMemoryOnCPU(uptr &ShadowBegin, uptr &ShadowEnd) { + return cpu::SetupShadowMemory(ShadowBegin, ShadowEnd); +} + +ur_result_t DestroyShadowMemoryOnCPU() { return cpu::DestroyShadowMemory(); } + +ur_result_t SetupShadowMemoryOnPVC(ur_context_handle_t Context, + uptr &ShadowBegin, uptr &ShadowEnd) { + return pvc::SetupShadowMemory(Context, ShadowBegin, ShadowEnd); +} + +ur_result_t DestroyShadowMemoryOnPVC() { return pvc::DestroyShadowMemory(); } + +} // namespace ur_sanitizer_layer diff --git a/source/loader/layers/sanitizer/asan_shadow_setup.hpp b/source/loader/layers/sanitizer/asan_shadow_setup.hpp new file mode 100644 index 0000000000..4b188d8831 --- /dev/null +++ b/source/loader/layers/sanitizer/asan_shadow_setup.hpp @@ -0,0 +1,26 @@ +/* + * + * Copyright (C) 2024 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 + * + * @file asan_shadow_setup.hpp + * + */ + +#pragma once + +#include "common.hpp" + +namespace ur_sanitizer_layer { + +ur_result_t SetupShadowMemoryOnCPU(uptr &ShadowBegin, uptr &ShadowEnd); +ur_result_t DestroyShadowMemoryOnCPU(); + +ur_result_t SetupShadowMemoryOnPVC(ur_context_handle_t Context, + uptr &ShadowBegin, uptr &ShadowEnd); +ur_result_t DestroyShadowMemoryOnPVC(); + +} // namespace ur_sanitizer_layer diff --git a/source/loader/layers/sanitizer/common.hpp b/source/loader/layers/sanitizer/common.hpp index f0c7152d8b..d5612100aa 100644 --- a/source/loader/layers/sanitizer/common.hpp +++ b/source/loader/layers/sanitizer/common.hpp @@ -28,21 +28,6 @@ using uptr = uintptr_t; using u8 = unsigned char; using u32 = unsigned int; -constexpr unsigned ASAN_SHADOW_SCALE = 3; -constexpr unsigned ASAN_SHADOW_GRANULARITY = 1ULL << ASAN_SHADOW_SCALE; - -// Based on "compiler-rt/lib/asan/asan_mapping.h" -// Typical shadow mapping on Linux/x86_64 with SHADOW_OFFSET == 0x00007fff8000: -constexpr uptr LOW_SHADOW_BEGIN = 0x00007fff8000ULL; -constexpr uptr LOW_SHADOW_END = 0x00008fff6fffULL; -constexpr uptr SHADOW_GAP_BEGIN = 0x00008fff7000ULL; -constexpr uptr SHADOW_GAP_END = 0x02008fff6fffULL; -constexpr uptr HIGH_SHADOW_BEGIN = 0x02008fff7000ULL; -constexpr uptr HIGH_SHADOW_END = 0x10007fff7fffULL; -constexpr uptr LOW_SHADOW_SIZE = LOW_SHADOW_END - LOW_SHADOW_BEGIN; -constexpr uptr SHADOW_GAP_SIZE = SHADOW_GAP_END - SHADOW_GAP_BEGIN; -constexpr uptr HIGH_SHADOW_SIZE = HIGH_SHADOW_END - HIGH_SHADOW_BEGIN; - inline constexpr bool IsPowerOfTwo(uptr x) { return (x & (x - 1)) == 0 && x != 0; } @@ -94,17 +79,22 @@ inline constexpr uptr ComputeRZLog(uptr user_requested_size) { return Result; \ } -#ifndef NDEBUG -#define UR_ASSERT_EQ(Call, Result) assert(Call == Result) -#else -#define UR_ASSERT_EQ(Call, Result) (void)Call -#endif +using BacktraceInfo = std::string; -bool IsInASanContext(); +struct SourceInfo { + std::string file; + std::string function; + int line; + int column; +}; -bool SetupShadowMem(); +enum class DeviceType : uint64_t { UNKNOWN = 0, CPU, GPU_PVC, GPU_DG2 }; + +bool IsInASanContext(); -bool DestroyShadowMem(); +bool MmapFixedNoReserve(uptr Addr, uptr Size); +bool MmapFixedNoAccess(uptr Addr, uptr Size); +bool Munmap(uptr Addr, uptr Size); void *GetMemFunctionPointer(const char *); diff --git a/source/loader/layers/sanitizer/linux/backtrace.cpp b/source/loader/layers/sanitizer/linux/backtrace.cpp new file mode 100644 index 0000000000..87c822d036 --- /dev/null +++ b/source/loader/layers/sanitizer/linux/backtrace.cpp @@ -0,0 +1,36 @@ +/* + * + * Copyright (C) 2024 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 + * + */ +#include "stacktrace.hpp" + +#include +#include + +namespace ur_sanitizer_layer { + +StackTrace GetCurrentBacktrace() { + void *Frames[MAX_BACKTRACE_FRAMES]; + int FrameCount = backtrace(Frames, MAX_BACKTRACE_FRAMES); + char **Symbols = backtrace_symbols(Frames, FrameCount); + + if (Symbols == nullptr) { + return StackTrace(); + } + + StackTrace Stack; + for (int i = 0; i < FrameCount; i++) { + BacktraceInfo addr_info(Symbols[i]); + Stack.stack.emplace_back(std::move(addr_info)); + } + free(Symbols); + + return Stack; +} + +} // namespace ur_sanitizer_layer diff --git a/source/loader/layers/sanitizer/linux/san_utils.cpp b/source/loader/layers/sanitizer/linux/sanitizer_utils.cpp similarity index 69% rename from source/loader/layers/sanitizer/linux/san_utils.cpp rename to source/loader/layers/sanitizer/linux/sanitizer_utils.cpp index 06d6b8d997..63718207ba 100644 --- a/source/loader/layers/sanitizer/linux/san_utils.cpp +++ b/source/loader/layers/sanitizer/linux/sanitizer_utils.cpp @@ -7,7 +7,7 @@ * See LICENSE.TXT * SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception * - * @file san_utils.cpp + * @file sanitizer_utils.cpp * */ @@ -27,7 +27,7 @@ namespace ur_sanitizer_layer { bool IsInASanContext() { return (void *)__asan_init != nullptr; } -static bool ReserveShadowMem(uptr Addr, uptr Size) { +bool MmapFixedNoReserve(uptr Addr, uptr Size) { Size = RoundUpTo(Size, EXEC_PAGESIZE); Addr = RoundDownTo(Addr, EXEC_PAGESIZE); void *P = @@ -36,42 +36,14 @@ static bool ReserveShadowMem(uptr Addr, uptr Size) { return Addr == (uptr)P; } -static bool ProtectShadowGap(uptr Addr, uptr Size) { +bool MmapFixedNoAccess(uptr Addr, uptr Size) { void *P = mmap((void *)Addr, Size, PROT_NONE, MAP_PRIVATE | MAP_FIXED | MAP_NORESERVE | MAP_ANONYMOUS, -1, 0); return Addr == (uptr)P; } -bool SetupShadowMem() { - if (!ReserveShadowMem(LOW_SHADOW_BEGIN, LOW_SHADOW_SIZE)) { - return false; - } - - if (!ReserveShadowMem(HIGH_SHADOW_BEGIN, HIGH_SHADOW_SIZE)) { - return false; - } - - if (!ProtectShadowGap(SHADOW_GAP_BEGIN, SHADOW_GAP_SIZE)) { - return false; - } - return true; -} - -bool DestroyShadowMem() { - if (munmap((void *)LOW_SHADOW_BEGIN, LOW_SHADOW_SIZE) == -1) { - return false; - } - - if (munmap((void *)HIGH_SHADOW_BEGIN, HIGH_SHADOW_SIZE) == -1) { - return false; - } - - if (munmap((void *)SHADOW_GAP_BEGIN, SHADOW_GAP_SIZE) == -1) { - return false; - } - return true; -} +bool Munmap(uptr Addr, uptr Size) { return munmap((void *)Addr, Size) == 0; } void *GetMemFunctionPointer(const char *FuncName) { void *handle = dlopen(LIBC_SO, RTLD_LAZY | RTLD_NOLOAD); diff --git a/source/loader/layers/sanitizer/stacktrace.cpp b/source/loader/layers/sanitizer/stacktrace.cpp new file mode 100644 index 0000000000..6994ce58ad --- /dev/null +++ b/source/loader/layers/sanitizer/stacktrace.cpp @@ -0,0 +1,46 @@ +/* + * + * Copyright (C) 2024 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 + * + * @file stacktrace.cpp + * + */ + +#include "stacktrace.hpp" +#include "ur_sanitizer_layer.hpp" + +namespace ur_sanitizer_layer { + +namespace { + +bool Contains(const std::string &s, const char *p) { + return s.find(p) != std::string::npos; +} + +} // namespace + +void StackTrace::print() const { + if (!stack.size()) { + context.logger.always(" failed to acquire backtrace"); + } + + unsigned index = 0; + + for (auto &BI : stack) { + // Skip runtime modules + if (Contains(BI, "libsycl.so") || + Contains(BI, "libpi_unified_runtime.so") || + Contains(BI, "libur_loader.so")) { + continue; + } + context.logger.always(" #{} {}", index, BI); + ++index; + } + context.logger.always(""); +} + +} // namespace ur_sanitizer_layer diff --git a/source/loader/layers/sanitizer/stacktrace.hpp b/source/loader/layers/sanitizer/stacktrace.hpp new file mode 100644 index 0000000000..31b661cc7d --- /dev/null +++ b/source/loader/layers/sanitizer/stacktrace.hpp @@ -0,0 +1,31 @@ +/* + * + * Copyright (C) 2024 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 + * + * @file stacktrace.hpp + * + */ + +#pragma once + +#include "common.hpp" + +#include + +namespace ur_sanitizer_layer { + +constexpr size_t MAX_BACKTRACE_FRAMES = 64; + +struct StackTrace { + std::vector stack; + + void print() const; +}; + +StackTrace GetCurrentBacktrace(); + +} // namespace ur_sanitizer_layer diff --git a/source/loader/layers/sanitizer/ur_sanddi.cpp b/source/loader/layers/sanitizer/ur_sanddi.cpp index f4e4f3ef01..e07329ed5a 100644 --- a/source/loader/layers/sanitizer/ur_sanddi.cpp +++ b/source/loader/layers/sanitizer/ur_sanddi.cpp @@ -12,9 +12,31 @@ #include "asan_interceptor.hpp" #include "ur_sanitizer_layer.hpp" +#include "ur_sanitizer_utils.hpp" namespace ur_sanitizer_layer { +namespace { + +ur_result_t setupContext(ur_context_handle_t Context, uint32_t numDevices, + const ur_device_handle_t *phDevices) { + std::shared_ptr CI; + UR_CALL(context.interceptor->insertContext(Context, CI)); + for (uint32_t i = 0; i < numDevices; ++i) { + auto hDevice = phDevices[i]; + std::shared_ptr DI; + UR_CALL(context.interceptor->insertDevice(hDevice, DI)); + if (!DI->ShadowOffset) { + UR_CALL(DI->allocShadowMemory(Context)); + } + CI->DeviceList.emplace_back(hDevice); + CI->AllocInfosMap[hDevice]; + } + return UR_RESULT_SUCCESS; +} + +} // namespace + /////////////////////////////////////////////////////////////////////////////// /// @brief Intercept function for urUSMHostAlloc __urdlllocal ur_result_t UR_APICALL urUSMHostAlloc( @@ -36,7 +58,7 @@ __urdlllocal ur_result_t UR_APICALL urUSMHostAlloc( context.logger.debug("==== urUSMHostAlloc"); return context.interceptor->allocateMemory( - hContext, nullptr, pUSMDesc, pool, size, ppMem, AllocType::HOST_USM); + hContext, nullptr, pUSMDesc, pool, size, AllocType::HOST_USM, ppMem); } /////////////////////////////////////////////////////////////////////////////// @@ -61,7 +83,7 @@ __urdlllocal ur_result_t UR_APICALL urUSMDeviceAlloc( context.logger.debug("==== urUSMDeviceAlloc"); return context.interceptor->allocateMemory( - hContext, hDevice, pUSMDesc, pool, size, ppMem, AllocType::DEVICE_USM); + hContext, hDevice, pUSMDesc, pool, size, AllocType::DEVICE_USM, ppMem); } /////////////////////////////////////////////////////////////////////////////// @@ -86,7 +108,7 @@ __urdlllocal ur_result_t UR_APICALL urUSMSharedAlloc( context.logger.debug("==== urUSMSharedAlloc"); return context.interceptor->allocateMemory( - hContext, hDevice, pUSMDesc, pool, size, ppMem, AllocType::SHARED_USM); + hContext, hDevice, pUSMDesc, pool, size, AllocType::SHARED_USM, ppMem); } /////////////////////////////////////////////////////////////////////////////// @@ -107,73 +129,48 @@ __urdlllocal ur_result_t UR_APICALL urUSMFree( } /////////////////////////////////////////////////////////////////////////////// -/// @brief Intercept function for urQueueCreate -__urdlllocal ur_result_t UR_APICALL urQueueCreate( +/// @brief Intercept function for urProgramBuild +__urdlllocal ur_result_t UR_APICALL urProgramBuild( ur_context_handle_t hContext, ///< [in] handle of the context object - ur_device_handle_t hDevice, ///< [in] handle of the device object - const ur_queue_properties_t - *pProperties, ///< [in][optional] pointer to queue creation properties. - ur_queue_handle_t - *phQueue ///< [out] pointer to handle of queue object created -) { - auto pfnCreate = context.urDdiTable.Queue.pfnCreate; - - if (nullptr == pfnCreate) { - return UR_RESULT_ERROR_UNSUPPORTED_FEATURE; - } - - context.logger.debug("==== urQueueCreate"); - - ur_result_t result = pfnCreate(hContext, hDevice, pProperties, phQueue); - if (result == UR_RESULT_SUCCESS) { - result = context.interceptor->insertQueue(hContext, *phQueue); - } - - return result; -} - -/////////////////////////////////////////////////////////////////////////////// -/// @brief Intercept function for urQueueRelease -__urdlllocal ur_result_t UR_APICALL urQueueRelease( - ur_queue_handle_t hQueue ///< [in] handle of the queue object to release + ur_program_handle_t hProgram, ///< [in] handle of the program object + const char *pOptions ///< [in] string of build options ) { - auto pfnRelease = context.urDdiTable.Queue.pfnRelease; + auto pfnProgramBuild = context.urDdiTable.Program.pfnBuild; - if (nullptr == pfnRelease) { + if (nullptr == pfnProgramBuild) { return UR_RESULT_ERROR_UNSUPPORTED_FEATURE; } - context.logger.debug("==== urQueueRelease"); + context.logger.debug("==== urProgramBuild"); - ur_context_handle_t hContext; - UR_CALL(context.urDdiTable.Queue.pfnGetInfo(hQueue, UR_QUEUE_INFO_CONTEXT, - sizeof(ur_context_handle_t), - &hContext, nullptr)); - UR_CALL(context.interceptor->eraseQueue(hContext, hQueue)); + UR_CALL(pfnProgramBuild(hContext, hProgram, pOptions)); - ur_result_t result = pfnRelease(hQueue); + UR_CALL(context.interceptor->registerDeviceGlobals(hContext, hProgram)); - return result; + return UR_RESULT_SUCCESS; } /////////////////////////////////////////////////////////////////////////////// -/// @brief Intercept function for urProgramBuild -__urdlllocal ur_result_t UR_APICALL urProgramBuild( - ur_context_handle_t hContext, ///< [in] handle of the context object - ur_program_handle_t hProgram, ///< [in] handle of the program object - const char *pOptions ///< [in] string of build options +/// @brief Intercept function for urProgramBuildExp +__urdlllocal ur_result_t UR_APICALL urProgramBuildExp( + ur_program_handle_t hProgram, ///< [in] Handle of the program to build. + uint32_t numDevices, ///< [in] number of devices + ur_device_handle_t * + phDevices, ///< [in][range(0, numDevices)] pointer to array of device handles + const char * + pOptions ///< [in][optional] pointer to build options null-terminated string. ) { - auto pfnProgramBuild = context.urDdiTable.Program.pfnBuild; + auto pfnBuildExp = context.urDdiTable.ProgramExp.pfnBuildExp; - if (nullptr == pfnProgramBuild) { + if (nullptr == pfnBuildExp) { return UR_RESULT_ERROR_UNSUPPORTED_FEATURE; } - context.logger.debug("==== urProgramBuild"); + context.logger.debug("==== urProgramBuildExp"); - UR_CALL(pfnProgramBuild(hContext, hProgram, pOptions)); - - UR_CALL(context.interceptor->registerDeviceGlobals(hContext, hProgram)); + UR_CALL(pfnBuildExp(hProgram, numDevices, phDevices, pOptions)); + UR_CALL(context.interceptor->registerDeviceGlobals(GetContext(hProgram), + hProgram)); return UR_RESULT_SUCCESS; } @@ -217,43 +214,19 @@ __urdlllocal ur_result_t UR_APICALL urEnqueueKernelLaunch( context.logger.debug("==== urEnqueueKernelLaunch"); - LaunchInfo LaunchInfo; - const size_t *pUserLocalWorkSize = pLocalWorkSize; - if (!pUserLocalWorkSize) { - pUserLocalWorkSize = LaunchInfo.LocalWorkSize; - // FIXME: This is W/A until urKernelSuggestGroupSize is added - LaunchInfo.LocalWorkSize[0] = 1; - LaunchInfo.LocalWorkSize[1] = 1; - LaunchInfo.LocalWorkSize[2] = 1; - } - - uint32_t numWork = 1; - for (uint32_t dim = 0; dim < workDim; ++dim) { - numWork *= (pGlobalWorkSize[dim] + pUserLocalWorkSize[dim] - 1) / - pUserLocalWorkSize[dim]; - } - - std::vector hEvents; - for (uint32_t i = 0; i < numEventsInWaitList; ++i) { - hEvents.push_back(phEventWaitList[i]); - } + LaunchInfo LaunchInfo(GetContext(hQueue), pGlobalWorkSize, pLocalWorkSize, + pGlobalWorkOffset, workDim); - // preLaunchKernel must append to num_events_in_wait_list, not prepend - ur_event_handle_t hPreEvent{}; - UR_CALL(context.interceptor->preLaunchKernel(hKernel, hQueue, hPreEvent, - LaunchInfo, numWork)); - if (hPreEvent) { - hEvents.push_back(hPreEvent); - } + UR_CALL(context.interceptor->preLaunchKernel(hKernel, hQueue, LaunchInfo)); ur_event_handle_t hEvent{}; ur_result_t result = pfnKernelLaunch( hQueue, hKernel, workDim, pGlobalWorkOffset, pGlobalWorkSize, - pLocalWorkSize, hEvents.size(), hEvents.data(), &hEvent); + pLocalWorkSize, numEventsInWaitList, phEventWaitList, &hEvent); if (result == UR_RESULT_SUCCESS) { - context.interceptor->postLaunchKernel(hKernel, hQueue, hEvent, - LaunchInfo); + UR_CALL(context.interceptor->postLaunchKernel(hKernel, hQueue, hEvent, + LaunchInfo)); } if (phEvent) { @@ -286,17 +259,7 @@ __urdlllocal ur_result_t UR_APICALL urContextCreate( pfnCreate(numDevices, phDevices, pProperties, phContext); if (result == UR_RESULT_SUCCESS) { - auto Context = *phContext; - result = context.interceptor->insertContext(Context); - if (result != UR_RESULT_SUCCESS) { - return result; - } - for (uint32_t i = 0; i < numDevices; ++i) { - result = context.interceptor->insertDevice(Context, phDevices[i]); - if (result != UR_RESULT_SUCCESS) { - return result; - } - } + UR_CALL(setupContext(*phContext, numDevices, phDevices)); } return result; @@ -328,17 +291,7 @@ __urdlllocal ur_result_t UR_APICALL urContextCreateWithNativeHandle( hNativeContext, numDevices, phDevices, pProperties, phContext); if (result == UR_RESULT_SUCCESS) { - auto Context = *phContext; - result = context.interceptor->insertContext(Context); - if (result != UR_RESULT_SUCCESS) { - return result; - } - for (uint32_t i = 0; i < numDevices; ++i) { - result = context.interceptor->insertDevice(Context, phDevices[i]); - if (result != UR_RESULT_SUCCESS) { - return result; - } - } + UR_CALL(setupContext(*phContext, numDevices, phDevices)); } return result; @@ -426,16 +379,16 @@ __urdlllocal ur_result_t UR_APICALL urGetProgramProcAddrTable( return UR_RESULT_SUCCESS; } /////////////////////////////////////////////////////////////////////////////// -/// @brief Exported function for filling application's Enqueue table +/// @brief Exported function for filling application's ProgramExp table /// with current process' addresses /// /// @returns /// - ::UR_RESULT_SUCCESS /// - ::UR_RESULT_ERROR_INVALID_NULL_POINTER /// - ::UR_RESULT_ERROR_UNSUPPORTED_VERSION -__urdlllocal ur_result_t UR_APICALL urGetEnqueueProcAddrTable( +__urdlllocal ur_result_t UR_APICALL urGetProgramExpProcAddrTable( ur_api_version_t version, ///< [in] API version requested - ur_enqueue_dditable_t + ur_program_exp_dditable_t *pDdiTable ///< [in,out] pointer to table of DDI function pointers ) { if (nullptr == pDdiTable) { @@ -451,21 +404,21 @@ __urdlllocal ur_result_t UR_APICALL urGetEnqueueProcAddrTable( ur_result_t result = UR_RESULT_SUCCESS; - pDdiTable->pfnKernelLaunch = ur_sanitizer_layer::urEnqueueKernelLaunch; + pDdiTable->pfnBuildExp = ur_sanitizer_layer::urProgramBuildExp; return result; } /////////////////////////////////////////////////////////////////////////////// -/// @brief Exported function for filling application's Queue table +/// @brief Exported function for filling application's Enqueue table /// with current process' addresses /// /// @returns /// - ::UR_RESULT_SUCCESS /// - ::UR_RESULT_ERROR_INVALID_NULL_POINTER /// - ::UR_RESULT_ERROR_UNSUPPORTED_VERSION -__urdlllocal ur_result_t UR_APICALL urGetQueueProcAddrTable( +__urdlllocal ur_result_t UR_APICALL urGetEnqueueProcAddrTable( ur_api_version_t version, ///< [in] API version requested - ur_queue_dditable_t + ur_enqueue_dditable_t *pDdiTable ///< [in,out] pointer to table of DDI function pointers ) { if (nullptr == pDdiTable) { @@ -481,8 +434,7 @@ __urdlllocal ur_result_t UR_APICALL urGetQueueProcAddrTable( ur_result_t result = UR_RESULT_SUCCESS; - pDdiTable->pfnCreate = ur_sanitizer_layer::urQueueCreate; - pDdiTable->pfnRelease = ur_sanitizer_layer::urQueueRelease; + pDdiTable->pfnKernelLaunch = ur_sanitizer_layer::urEnqueueKernelLaunch; return result; } @@ -562,13 +514,13 @@ ur_result_t context_t::init(ur_dditable_t *dditable, } if (UR_RESULT_SUCCESS == result) { - result = ur_sanitizer_layer::urGetEnqueueProcAddrTable( - UR_API_VERSION_CURRENT, &dditable->Enqueue); + result = ur_sanitizer_layer::urGetProgramExpProcAddrTable( + UR_API_VERSION_CURRENT, &dditable->ProgramExp); } if (UR_RESULT_SUCCESS == result) { - result = ur_sanitizer_layer::urGetQueueProcAddrTable( - UR_API_VERSION_CURRENT, &dditable->Queue); + result = ur_sanitizer_layer::urGetEnqueueProcAddrTable( + UR_API_VERSION_CURRENT, &dditable->Enqueue); } if (UR_RESULT_SUCCESS == result) { @@ -578,4 +530,5 @@ ur_result_t context_t::init(ur_dditable_t *dditable, return result; } + } // namespace ur_sanitizer_layer diff --git a/source/loader/layers/sanitizer/ur_sanitizer_utils.cpp b/source/loader/layers/sanitizer/ur_sanitizer_utils.cpp new file mode 100644 index 0000000000..394aa09d3f --- /dev/null +++ b/source/loader/layers/sanitizer/ur_sanitizer_utils.cpp @@ -0,0 +1,171 @@ +/* + * + * Copyright (C) 2024 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 + * + * @file ur_sanitizer_utils.cpp + * + */ + +#include "ur_sanitizer_utils.hpp" +#include "ur_sanitizer_layer.hpp" + +namespace ur_sanitizer_layer { + +ManagedQueue::ManagedQueue(ur_context_handle_t Context, + ur_device_handle_t Device) { + [[maybe_unused]] auto Result = + context.urDdiTable.Queue.pfnCreate(Context, Device, nullptr, &Handle); + assert(Result == UR_RESULT_SUCCESS); + context.logger.debug(">>> ManagedQueue {}", (void *)Handle); +} + +ManagedQueue::~ManagedQueue() { + context.logger.debug("<<< ~ManagedQueue {}", (void *)Handle); + + [[maybe_unused]] ur_result_t Result; + Result = context.urDdiTable.Queue.pfnFinish(Handle); + if (Result != UR_RESULT_SUCCESS) { + context.logger.error("Failed to finish ManagedQueue: {}", Result); + } + assert(Result == UR_RESULT_SUCCESS); + Result = context.urDdiTable.Queue.pfnRelease(Handle); + assert(Result == UR_RESULT_SUCCESS); +} + +ur_context_handle_t GetContext(ur_queue_handle_t Queue) { + ur_context_handle_t Context{}; + [[maybe_unused]] auto Result = context.urDdiTable.Queue.pfnGetInfo( + Queue, UR_QUEUE_INFO_CONTEXT, sizeof(ur_context_handle_t), &Context, + nullptr); + assert(Result == UR_RESULT_SUCCESS && "getContext() failed"); + return Context; +} + +ur_context_handle_t GetContext(ur_program_handle_t Program) { + ur_context_handle_t Context{}; + [[maybe_unused]] auto Result = context.urDdiTable.Program.pfnGetInfo( + Program, UR_PROGRAM_INFO_CONTEXT, sizeof(ur_context_handle_t), &Context, + nullptr); + assert(Result == UR_RESULT_SUCCESS && "getContext() failed"); + return Context; +} + +ur_context_handle_t GetContext(ur_kernel_handle_t Kernel) { + ur_context_handle_t Context{}; + [[maybe_unused]] auto Result = context.urDdiTable.Kernel.pfnGetInfo( + Kernel, UR_KERNEL_INFO_CONTEXT, sizeof(ur_context_handle_t), &Context, + nullptr); + assert(Result == UR_RESULT_SUCCESS && "getContext() failed"); + return Context; +} + +ur_device_handle_t GetDevice(ur_queue_handle_t Queue) { + ur_device_handle_t Device{}; + [[maybe_unused]] auto Result = context.urDdiTable.Queue.pfnGetInfo( + Queue, UR_QUEUE_INFO_DEVICE, sizeof(ur_device_handle_t), &Device, + nullptr); + assert(Result == UR_RESULT_SUCCESS && "getDevice() failed"); + return Device; +} + +ur_program_handle_t GetProgram(ur_kernel_handle_t Kernel) { + ur_program_handle_t Program{}; + [[maybe_unused]] auto Result = context.urDdiTable.Kernel.pfnGetInfo( + Kernel, UR_KERNEL_INFO_PROGRAM, sizeof(ur_program_handle_t), &Program, + nullptr); + assert(Result == UR_RESULT_SUCCESS && "getProgram() failed"); + return Program; +} + +size_t GetLocalMemorySize(ur_device_handle_t Device) { + size_t LocalMemorySize{}; + [[maybe_unused]] auto Result = context.urDdiTable.Device.pfnGetInfo( + Device, UR_DEVICE_INFO_LOCAL_MEM_SIZE, sizeof(LocalMemorySize), + &LocalMemorySize, nullptr); + assert(Result == UR_RESULT_SUCCESS && "getLocalMemorySize() failed"); + return LocalMemorySize; +} + +std::string GetKernelName(ur_kernel_handle_t Kernel) { + size_t KernelNameSize = 0; + [[maybe_unused]] auto Result = context.urDdiTable.Kernel.pfnGetInfo( + Kernel, UR_KERNEL_INFO_FUNCTION_NAME, 0, nullptr, &KernelNameSize); + assert(Result == UR_RESULT_SUCCESS && "getKernelName() failed"); + + std::vector KernelNameBuf(KernelNameSize); + Result = context.urDdiTable.Kernel.pfnGetInfo( + Kernel, UR_KERNEL_INFO_FUNCTION_NAME, KernelNameSize, + KernelNameBuf.data(), nullptr); + assert(Result == UR_RESULT_SUCCESS && "getKernelName() failed"); + + return std::string(KernelNameBuf.data(), KernelNameSize - 1); +} + +ur_device_handle_t GetUSMAllocDevice(ur_context_handle_t Context, + const void *MemPtr) { + ur_device_handle_t Device{}; + // if urGetMemAllocInfo failed, return nullptr + context.urDdiTable.USM.pfnGetMemAllocInfo(Context, MemPtr, + UR_USM_ALLOC_INFO_DEVICE, + sizeof(Device), &Device, nullptr); + return Device; +} + +DeviceType GetDeviceType(ur_device_handle_t Device) { + ur_device_type_t DeviceType = UR_DEVICE_TYPE_DEFAULT; + [[maybe_unused]] auto Result = context.urDdiTable.Device.pfnGetInfo( + Device, UR_DEVICE_INFO_TYPE, sizeof(DeviceType), &DeviceType, nullptr); + assert(Result == UR_RESULT_SUCCESS && "getDeviceType() failed"); + switch (DeviceType) { + case UR_DEVICE_TYPE_CPU: + case UR_DEVICE_TYPE_FPGA: + // TODO: Check fpga is fpga emulator + return DeviceType::CPU; + case UR_DEVICE_TYPE_GPU: { + // TODO: Check device name + return DeviceType::GPU_PVC; + } + default: + return DeviceType::UNKNOWN; + } +} + +std::vector GetProgramDevices(ur_program_handle_t Program) { + size_t PropSize; + [[maybe_unused]] ur_result_t Result = context.urDdiTable.Program.pfnGetInfo( + Program, UR_PROGRAM_INFO_DEVICES, 0, nullptr, &PropSize); + assert(Result == UR_RESULT_SUCCESS); + + std::vector Devices; + Devices.resize(PropSize / sizeof(ur_device_handle_t)); + Result = context.urDdiTable.Program.pfnGetInfo( + Program, UR_PROGRAM_INFO_DEVICES, PropSize, Devices.data(), nullptr); + assert(Result == UR_RESULT_SUCCESS); + + return Devices; +} + +size_t GetKernelNumArgs(ur_kernel_handle_t Kernel) { + size_t NumArgs = 0; + [[maybe_unused]] auto Res = context.urDdiTable.Kernel.pfnGetInfo( + Kernel, UR_KERNEL_INFO_NUM_ARGS, sizeof(NumArgs), &NumArgs, nullptr); + assert(Res == UR_RESULT_SUCCESS); + return NumArgs; +} + +size_t GetVirtualMemGranularity(ur_context_handle_t Context, + ur_device_handle_t Device) { + size_t Size; + [[maybe_unused]] auto Result = + context.urDdiTable.VirtualMem.pfnGranularityGetInfo( + Context, Device, UR_VIRTUAL_MEM_GRANULARITY_INFO_RECOMMENDED, + sizeof(Size), &Size, nullptr); + assert(Result == UR_RESULT_SUCCESS); + return Size; +} + +} // namespace ur_sanitizer_layer diff --git a/source/loader/layers/sanitizer/ur_sanitizer_utils.hpp b/source/loader/layers/sanitizer/ur_sanitizer_utils.hpp new file mode 100644 index 0000000000..5d82376856 --- /dev/null +++ b/source/loader/layers/sanitizer/ur_sanitizer_utils.hpp @@ -0,0 +1,48 @@ +/* + * + * Copyright (C) 2024 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 + * + * @file ur_sanitizer_utils.hpp + * + */ + +#pragma once + +#include "common.hpp" + +namespace ur_sanitizer_layer { + +struct ManagedQueue { + ManagedQueue(ur_context_handle_t Context, ur_device_handle_t Device); + ~ManagedQueue(); + + // Disable copy semantics + ManagedQueue(const ManagedQueue &) = delete; + ManagedQueue &operator=(const ManagedQueue &) = delete; + + operator ur_queue_handle_t() { return Handle; } + + private: + ur_queue_handle_t Handle = nullptr; +}; + +ur_context_handle_t GetContext(ur_queue_handle_t Queue); +ur_context_handle_t GetContext(ur_program_handle_t Program); +ur_context_handle_t GetContext(ur_kernel_handle_t Kernel); +ur_device_handle_t GetDevice(ur_queue_handle_t Queue); +DeviceType GetDeviceType(ur_device_handle_t Device); +std::string GetKernelName(ur_kernel_handle_t Kernel); +size_t GetLocalMemorySize(ur_device_handle_t Device); +ur_program_handle_t GetProgram(ur_kernel_handle_t Kernel); +std::vector GetProgramDevices(ur_program_handle_t Program); +ur_device_handle_t GetUSMAllocDevice(ur_context_handle_t Context, + const void *MemPtr); +size_t GetKernelNumArgs(ur_kernel_handle_t Kernel); +size_t GetVirtualMemGranularity(ur_context_handle_t Context, + ur_device_handle_t Device); + +} // namespace ur_sanitizer_layer