Skip to content

Commit

Permalink
rocr: Allocate AQL queue on device memory
Browse files Browse the repository at this point in the history
- Use HSA_ALLOCATE_QUEUE_DEV_MEM=1 to create AQL queue in device
memory.
- Before writing AQL packet header to the queue use an SFENCE to ensure
that there is no reodering of the writes over PCIE

Change-Id: I5eacdc35108c4a1e245c75ae349b7495451aa60d
  • Loading branch information
saleelk committed Sep 5, 2024
1 parent fe8d8c1 commit 3baaa6e
Show file tree
Hide file tree
Showing 10 changed files with 95 additions and 40 deletions.
5 changes: 5 additions & 0 deletions runtime/hsa-runtime/core/driver/kfd/amd_kfd_driver.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -117,6 +117,11 @@ KfdDriver::AllocateMemory(const core::MemoryRegion &mem_region,
? 1
: kmt_alloc_flags.ui32.GTTAccess);

kmt_alloc_flags.ui32.Uncached =
(alloc_flags & core::MemoryRegion::AllocateUncached
? 1
: kmt_alloc_flags.ui32.Uncached);

if (m_region.IsLocalMemory()) {
// Allocate physically contiguous memory. AllocateKfdMemory function call
// will fail if this flag is not supported in KFD.
Expand Down
25 changes: 16 additions & 9 deletions runtime/hsa-runtime/core/inc/amd_gpu_agent.h
Original file line number Diff line number Diff line change
Expand Up @@ -2,24 +2,24 @@
//
// The University of Illinois/NCSA
// Open Source License (NCSA)
//
//
// Copyright (c) 2014-2020, Advanced Micro Devices, Inc. All rights reserved.
//
//
// Developed by:
//
//
// AMD Research and AMD HSA Software Development
//
//
// Advanced Micro Devices, Inc.
//
//
// www.amd.com
//
//
// Permission is hereby granted, free of charge, to any person obtaining a copy
// of this software and associated documentation files (the "Software"), to
// deal with the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
//
// - Redistributions of source code must retain the above copyright notice,
// this list of conditions and the following disclaimers.
// - Redistributions in binary form must reproduce the above copyright
Expand All @@ -29,7 +29,7 @@
// nor the names of its contributors may be used to endorse or promote
// products derived from this Software without specific prior written
// permission.
//
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL
Expand Down Expand Up @@ -348,7 +348,7 @@ class GpuAgent : public GpuAgentInt {
}

core::Agent* GetNearestCpuAgent(void) const;

void RegisterGangPeer(core::Agent& gang_peer, unsigned int bandwidth_factor) override;

void RegisterRecSdmaEngIdMaskPeer(core::Agent& gang_peer, uint32_t rec_sdma_eng_id_mask) override;
Expand Down Expand Up @@ -417,6 +417,9 @@ class GpuAgent : public GpuAgentInt {
if (t0_.GPUClockCounter == t1_.GPUClockCounter) SyncClocks();
}

// @brief Override from AMD::GpuAgentInt.
__forceinline bool is_xgmi_cpu_gpu() const { return xgmi_cpu_gpu_; }

const size_t MAX_SCRATCH_APERTURE_PER_XCC = (1ULL << 32);
size_t MaxScratchDevice() const { return properties_.NumXcc * MAX_SCRATCH_APERTURE_PER_XCC; }

Expand Down Expand Up @@ -624,6 +627,7 @@ class GpuAgent : public GpuAgentInt {

// @brief HDP flush registers
hsa_amd_hdp_flush_t HDP_flush_ = {nullptr, nullptr};

private:
// @brief Query the driver to get the region list owned by this agent.
void InitRegionList();
Expand Down Expand Up @@ -782,6 +786,9 @@ class GpuAgent : public GpuAgentInt {
std::map<uint64_t, uint32_t> rec_sdma_eng_id_peers_info_;

bool uses_rec_sdma_eng_id_mask_;

// @bried XGMI CPU<->GPU
bool xgmi_cpu_gpu_;
};

} // namespace amd
Expand Down
21 changes: 11 additions & 10 deletions runtime/hsa-runtime/core/inc/memory_region.h
Original file line number Diff line number Diff line change
Expand Up @@ -2,24 +2,24 @@
//
// The University of Illinois/NCSA
// Open Source License (NCSA)
//
// Copyright (c) 2014-2020, Advanced Micro Devices, Inc. All rights reserved.
//
//
// Copyright (c) 2014-2024, Advanced Micro Devices, Inc. All rights reserved.
//
// Developed by:
//
//
// AMD Research and AMD HSA Software Development
//
//
// Advanced Micro Devices, Inc.
//
//
// www.amd.com
//
//
// Permission is hereby granted, free of charge, to any person obtaining a copy
// of this software and associated documentation files (the "Software"), to
// deal with the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
//
// - Redistributions of source code must retain the above copyright notice,
// this list of conditions and the following disclaimers.
// - Redistributions in binary form must reproduce the above copyright
Expand All @@ -29,7 +29,7 @@
// nor the names of its contributors may be used to endorse or promote
// products derived from this Software without specific prior written
// permission.
//
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL
Expand Down Expand Up @@ -104,7 +104,8 @@ class MemoryRegion : public Checked<0x9C961F19EE175BB3> {
// Note: The node_id needs to be the node_id of the device even though this is allocating
// system memory
AllocateGTTAccess = (1 << 9),
AllocateContiguous = (1 << 10), // Physically contiguous memory
AllocateContiguous = (1 << 10), // Physically contiguous memory
AllocateUncached = (1 << 11), // Uncached memory
};

typedef uint32_t AllocateFlags;
Expand Down
8 changes: 8 additions & 0 deletions runtime/hsa-runtime/core/inc/queue.h
Original file line number Diff line number Diff line change
Expand Up @@ -182,11 +182,13 @@ class Queue : public Checked<0xFA3906A679F9DB49>, private LocalQueue {
Queue(int mem_flags = 0) : LocalQueue(mem_flags), amd_queue_(queue()->amd_queue) {
queue()->core_queue = this;
public_handle_ = Convert(this);
pcie_write_ordering_ = false;
}

Queue(int agent_node_id, int mem_flags) : LocalQueue(agent_node_id, mem_flags), amd_queue_(queue()->amd_queue) {
queue()->core_queue = this;
public_handle_ = Convert(this);
pcie_write_ordering_ = false;
}

virtual ~Queue() {}
Expand Down Expand Up @@ -385,6 +387,10 @@ class Queue : public Checked<0xFA3906A679F9DB49>, private LocalQueue {

bool IsType(rtti_t id) { return _IsA(id); }

bool needsPcieOrdering() const { return pcie_write_ordering_; }

void setPcieOrdering(bool val) { pcie_write_ordering_ = val; }

protected:
static void set_public_handle(Queue* ptr, hsa_queue_t* handle) {
ptr->do_set_public_handle(handle);
Expand All @@ -405,6 +411,8 @@ class Queue : public Checked<0xFA3906A679F9DB49>, private LocalQueue {
// HSA Queue ID - used to bind a unique ID
static std::atomic<uint64_t> hsa_queue_counter_;

bool pcie_write_ordering_;

DISALLOW_COPY_AND_ASSIGN(Queue);
};
} // namespace core
Expand Down
28 changes: 22 additions & 6 deletions runtime/hsa-runtime/core/runtime/amd_aql_queue.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@
// The University of Illinois/NCSA
// Open Source License (NCSA)
//
// Copyright (c) 2014-2020, Advanced Micro Devices, Inc. All rights reserved.
// Copyright (c) 2014-2024, Advanced Micro Devices, Inc. All rights reserved.
//
// Developed by:
//
Expand Down Expand Up @@ -344,6 +344,7 @@ AqlQueue::AqlQueue(GpuAgent* agent, size_t req_size_pkts, HSAuint32 node_id, Scr
if (!core::Runtime::runtime_singleton_->flag().cu_mask_skip_init()) SetCUMasking(0, nullptr);

active_ = true;
setPcieOrdering(agent->is_xgmi_cpu_gpu());

PM4IBGuard.Dismiss();
RingGuard.Dismiss();
Expand Down Expand Up @@ -727,10 +728,15 @@ void AqlQueue::AllocRegisteredRingBuffer(uint32_t queue_size_pkts) {
ring_buf_alloc_bytes_ = queue_size_pkts * sizeof(core::AqlPacket);
assert(IsMultipleOf(ring_buf_alloc_bytes_, 4096) && "Ring buffer sizes must be 4KiB aligned.");

ring_buf_ = agent_->system_allocator()(
ring_buf_alloc_bytes_, 0x1000,
core::MemoryRegion::AllocateExecutable |
(queue_full_workaround_ ? core::MemoryRegion::AllocateDoubleMap : 0));
if (core::Runtime::runtime_singleton_->flag().dev_mem_queue()) {
ring_buf_ = agent_->finegrain_allocator()(ring_buf_alloc_bytes_,
core::MemoryRegion::AllocateUncached);
} else {
ring_buf_ = agent_->system_allocator()(
ring_buf_alloc_bytes_, 0x1000,
core::MemoryRegion::AllocateExecutable |
(queue_full_workaround_ ? core::MemoryRegion::AllocateDoubleMap : 0));
}

assert(ring_buf_ != NULL && "AQL queue memory allocation failure");

Expand All @@ -751,7 +757,13 @@ void AqlQueue::FreeRegisteredRingBuffer() {
(void*)(uintptr_t(ring_buf_) + (ring_buf_alloc_bytes_ / 2)));
#endif
} else {
agent_->system_deallocator()(ring_buf_);
if (ring_buf_) {
if (core::Runtime::runtime_singleton_->flag().dev_mem_queue()) {
agent_->finegrain_deallocator()(ring_buf_);
} else {
agent_->system_deallocator()(ring_buf_);
}
}
}

ring_buf_ = NULL;
Expand Down Expand Up @@ -1542,6 +1554,10 @@ void AqlQueue::ExecutePM4(uint32_t* cmd_data, size_t cmd_size_b, hsa_fence_scope
// Overwrite the AQL invalid header (first dword) last.
// This prevents the slot from being read until it's fully written.
memcpy(&queue_slot[1], &slot_data[1], slot_size_b - sizeof(uint32_t));
if (core::Runtime::runtime_singleton_->flag().dev_mem_queue() && !agent_->is_xgmi_cpu_gpu()) {
// Ensure the packet body is written as header may get reordered when writing over PCIE
_mm_sfence();
}
atomic::Store(&queue_slot[0], slot_data[0], std::memory_order_release);

// Submit the packet slot.
Expand Down
4 changes: 4 additions & 0 deletions runtime/hsa-runtime/core/runtime/amd_blit_kernel.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -1256,6 +1256,10 @@ void BlitKernel::PopulateQueue(uint64_t index, uint64_t code_handle, void* args,
std::atomic_thread_fence(std::memory_order_acquire);
queue_buffer[index & queue_bitmask_] = packet;
std::atomic_thread_fence(std::memory_order_release);
if (core::Runtime::runtime_singleton_->flag().dev_mem_queue() && !queue_->needsPcieOrdering()) {
// Ensure the packet body is written as header may get reordered when writing over PCIE
_mm_sfence();
}
queue_buffer[index & queue_bitmask_].header = kDispatchPacketHeader;

LogPrint(HSA_AMD_LOG_FLAG_BLIT_KERNEL_PKTS,
Expand Down
10 changes: 8 additions & 2 deletions runtime/hsa-runtime/core/runtime/amd_gpu_agent.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -118,7 +118,8 @@ GpuAgent::GpuAgent(HSAuint32 node, const HsaNodeProperties& node_props, bool xna
scratch_cache_(
[this](void* base, size_t size, bool large) { ReleaseScratch(base, size, large); }),
trap_handler_tma_region_(NULL),
pcs_hosttrap_data_() {
pcs_hosttrap_data_(),
xgmi_cpu_gpu_(false) {
const bool is_apu_node = (properties_.NumCPUCores > 0);
profile_ = (is_apu_node) ? HSA_PROFILE_FULL : HSA_PROFILE_BASE;

Expand Down Expand Up @@ -219,6 +220,11 @@ GpuAgent::GpuAgent(HSAuint32 node, const HsaNodeProperties& node_props, bool xna
wallclock_frequency_ = uint64_t(info.gpu_counter_freq) * 1000ull;
#endif

auto& firstCpu = core::Runtime::runtime_singleton_->cpu_agents()[0];
auto linkInfo = core::Runtime::runtime_singleton_->GetLinkInfo(firstCpu->node_id(),
node_id());
xgmi_cpu_gpu_ = (linkInfo.info.link_type == HSA_AMD_LINK_INFO_TYPE_XGMI);

// Populate region list.
InitRegionList();

Expand Down Expand Up @@ -574,7 +580,7 @@ void GpuAgent::ReserveScratch()
{
size_t reserved_sz = core::Runtime::runtime_singleton_->flag().scratch_single_limit();
if (reserved_sz > MaxScratchDevice()) {
fprintf(stdout, "User specified scratch limit exceeds device limits (requested:%lu max:%lu)!\n",
fprintf(stdout, "User specified scratch limit exceeds device limits (requested:%lu max:%lu)!\n",
reserved_sz, MaxScratchDevice());
reserved_sz = MaxScratchDevice();
}
Expand Down
14 changes: 3 additions & 11 deletions runtime/hsa-runtime/core/runtime/amd_topology.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -380,17 +380,6 @@ void BuildTopology() {
}
const_cast<Flag&>(core::Runtime::runtime_singleton_->flag()).parse_masks(maxGpu, maxCu);

// Temporary work-around, disable SDMA ganging on non-APUs in non-SPX modes
// Check xGMI APU status
bool isXgmiApu = false;
auto& firstCpu = core::Runtime::runtime_singleton_->cpu_agents()[0];
for (auto& peer_gpu : core::Runtime::runtime_singleton_->gpu_agents()) {
auto linfo = core::Runtime::runtime_singleton_->GetLinkInfo(firstCpu->node_id(),
peer_gpu->node_id());
isXgmiApu = linfo.info.link_type == HSA_AMD_LINK_INFO_TYPE_XGMI;
if (isXgmiApu) break;
}

// Register destination agents that can SDMA gang copy for source agents
for (auto& src_gpu : core::Runtime::runtime_singleton_->gpu_agents()) {
uint32_t src_id = src_gpu->node_id();
Expand All @@ -406,6 +395,9 @@ void BuildTopology() {
// Weigth of 15 - Direct GPU link in single partition mode
// Weight of 41 - Inter-socket GPU link in multi-partition mode
if (linfo.info.link_type == HSA_AMD_LINK_INFO_TYPE_XGMI) {
// Temporary work-around, disable SDMA ganging on non-APUs in non-SPX modes
// Check xGMI APU status
const bool isXgmiApu = reinterpret_cast<AMD::GpuAgent*>(src_gpu)->is_xgmi_cpu_gpu();
if (linfo.info.numa_distance == 13 || linfo.info.numa_distance == 41)
gang_factor = isXgmiApu ? 2 : 1;
else if (linfo.info.numa_distance == 15 && linfo.info.min_bandwidth)
Expand Down
13 changes: 12 additions & 1 deletion runtime/hsa-runtime/core/runtime/intercept_queue.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -258,6 +258,10 @@ uint64_t InterceptQueue::Submit(const AqlPacket* packets, uint64_t count) {
// Submit barrier which will wake async queue processing.
ring[barrier & mask].packet.body = {};
ring[barrier & mask].barrier_and.completion_signal = Signal::Convert(async_doorbell_);
if (Runtime::runtime_singleton_->flag().dev_mem_queue() && !needsPcieOrdering()) {
// Ensure the packet body is written as header may get reordered when writing over PCIE
_mm_sfence();
}
atomic::Store(&ring[barrier & mask].barrier_and.header, kBarrierHeader,
std::memory_order_release);
// Update the wrapped queue's doorbell so it knows there is a new packet in the queue.
Expand Down Expand Up @@ -301,6 +305,10 @@ uint64_t InterceptQueue::Submit(const AqlPacket* packets, uint64_t count) {
++packets_index;
}
if (write_index != 0) {
if (Runtime::runtime_singleton_->flag().dev_mem_queue() && !needsPcieOrdering()) {
// Ensure the packet body is written as header may get reordered when writing over PCIE
_mm_sfence();
}
atomic::Store(&ring[write & mask].packet.header, packets[first_written_packet_index].packet.header,
std::memory_order_release);
HSA::hsa_signal_store_screlease(wrapped->amd_queue_.hsa_queue.doorbell_signal,
Expand Down Expand Up @@ -366,7 +374,10 @@ void InterceptQueue::StoreRelaxed(hsa_signal_value_t value) {
Cursor.pkt_index = i;
auto& handler = interceptors[Cursor.interceptor_index];
handler.first(&ring[i & mask], 1, i, handler.second, PacketWriter);

if (Runtime::runtime_singleton_->flag().dev_mem_queue() && !needsPcieOrdering()) {
// Ensure the packet body is written as header may get reordered when writing over PCIE
_mm_sfence();
}
// Invalidate consumed packet.
atomic::Store(&ring[i & mask].packet.header, kInvalidHeader, std::memory_order_release);

Expand Down
7 changes: 6 additions & 1 deletion runtime/hsa-runtime/core/util/flag.h
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@
// The University of Illinois/NCSA
// Open Source License (NCSA)
//
// Copyright (c) 2014-2021, Advanced Micro Devices, Inc. All rights reserved.
// Copyright (c) 2014-2024, Advanced Micro Devices, Inc. All rights reserved.
//
// Developed by:
//
Expand Down Expand Up @@ -247,6 +247,9 @@ class Flag {
// Will either rename to HSA_OVERRIDE_CPU_AFFINITY later or remove completely.
var = os::GetEnvVar("HSA_OVERRIDE_CPU_AFFINITY_DEBUG");
override_cpu_affinity_ = (var == "0") ? false : true;

var = os::GetEnvVar("HSA_ALLOCATE_QUEUE_DEV_MEM");
dev_mem_queue_ = (var == "1") ? true : false;
}

void parse_masks(uint32_t maxGpu, uint32_t maxCU) {
Expand Down Expand Up @@ -357,6 +360,7 @@ class Flag {

size_t pc_sampling_max_device_buffer_size() const { return pc_sampling_max_device_buffer_size_; }

bool dev_mem_queue() const { return dev_mem_queue_; }
private:
bool check_flat_scratch_;
bool enable_vm_fault_message_;
Expand Down Expand Up @@ -385,6 +389,7 @@ class Flag {
bool image_print_srd_;
bool enable_mwaitx_;
bool enable_ipc_mode_legacy_;
bool dev_mem_queue_;

SDMA_OVERRIDE enable_sdma_;
SDMA_OVERRIDE enable_peer_sdma_;
Expand Down

0 comments on commit 3baaa6e

Please sign in to comment.