diff --git a/tt_metal/detail/tt_metal.hpp b/tt_metal/detail/tt_metal.hpp index d362bee71a2..d6168102a5e 100644 --- a/tt_metal/detail/tt_metal.hpp +++ b/tt_metal/detail/tt_metal.hpp @@ -276,7 +276,7 @@ inline namespace v0 { void SetLazyCommandQueueMode(bool lazy); - void AllocateBuffer(Buffer* buffer, bool bottom_up); + DeviceAddr AllocateBuffer(const Buffer* buffer, bool bottom_up); void DeallocateBuffer(Buffer *buffer); } // namespace detail diff --git a/tt_metal/graph/graph_tracking.cpp b/tt_metal/graph/graph_tracking.cpp index 04a9ae4240a..17a72ddd5ee 100644 --- a/tt_metal/graph/graph_tracking.cpp +++ b/tt_metal/graph/graph_tracking.cpp @@ -27,7 +27,7 @@ bool GraphTracker::add_hook(const std::shared_ptr& new_hook) { return true; } -void GraphTracker::track_allocate(Buffer* buffer, bool bottom_up) { +void GraphTracker::track_allocate(const Buffer* buffer, bool bottom_up) { if (processors.empty()) { return; } @@ -73,7 +73,7 @@ void GraphTracker::track_program(Program* program) { } } -bool GraphTracker::hook_allocate(Buffer* buffer, bool bottom_up) { +bool GraphTracker::hook_allocate(const Buffer* buffer, bool bottom_up) { if (hook == nullptr) return false; diff --git a/tt_metal/graph/graph_tracking.hpp b/tt_metal/graph/graph_tracking.hpp index a39cf3d4a56..54ee8eef41d 100644 --- a/tt_metal/graph/graph_tracking.hpp +++ b/tt_metal/graph/graph_tracking.hpp @@ -28,7 +28,7 @@ inline namespace v0 { IGraphProcessor() = default; - virtual void track_allocate(tt::tt_metal::Buffer* buffer, bool bottom_up) {}; + virtual void track_allocate(const tt::tt_metal::Buffer* buffer, bool bottom_up) {}; virtual void track_deallocate(tt::tt_metal::Buffer* buffer) {}; @@ -54,7 +54,7 @@ inline namespace v0 { class IGraphHooks { public: IGraphHooks() = default; - virtual bool hook_allocate(tt::tt_metal::Buffer* buffer, bool bottom_up) = 0; + virtual bool hook_allocate(const tt::tt_metal::Buffer* buffer, bool bottom_up) = 0; virtual bool hook_deallocate(tt::tt_metal::Buffer* buffer) = 0; @@ -77,7 +77,7 @@ inline namespace v0 { bool add_hook(const std::shared_ptr& hook); - void track_allocate(Buffer* buffer, bool bottom_up); + void track_allocate(const Buffer* buffer, bool bottom_up); void track_deallocate(Buffer* buffer); @@ -118,7 +118,7 @@ inline namespace v0 { } } - bool hook_allocate(Buffer* buffer, bool bottom_up); + bool hook_allocate(const Buffer* buffer, bool bottom_up); bool hook_deallocate(Buffer* buffer); diff --git a/tt_metal/impl/buffers/buffer.cpp b/tt_metal/impl/buffers/buffer.cpp index 299130bd2a3..647420cf872 100644 --- a/tt_metal/impl/buffers/buffer.cpp +++ b/tt_metal/impl/buffers/buffer.cpp @@ -116,50 +116,6 @@ inline std::tuple>, std::vector Buffer::create( - Device *device, - DeviceAddr size, - DeviceAddr page_size, - const BufferType buffer_type, - const TensorMemoryLayout buffer_layout, - const std::optional& shard_parameters, - const std::optional bottom_up, - bool allocate) { - auto bufferPtr = new Buffer(device, size, page_size, buffer_type, buffer_layout, shard_parameters, bottom_up); - auto buffer = std::shared_ptr(bufferPtr, deallocateAndDelete); - buffer->weak_self = buffer; - if (allocate) { - buffer->allocate(); - } - return buffer; -} - -Buffer::Buffer( - Device *device, - DeviceAddr size, - DeviceAddr page_size, - const BufferType buffer_type, - const TensorMemoryLayout buffer_layout, - const std::optional& shard_parameters, - const std::optional bottom_up) : - device_(device), - size_(size), - page_size_(page_size), - buffer_type_(buffer_type), - buffer_layout_(buffer_layout), - shard_parameters_(shard_parameters), - bottom_up_(bottom_up), - buffer_page_mapping_(nullptr) { - TT_FATAL(this->device_ != nullptr and this->device_->allocator_ != nullptr, "Device and allocator need to not be null."); - - if (size == 0) { - allocation_status_.store(AllocationStatus::ALLOCATED, std::memory_order::relaxed); - return; - } - - validate_buffer_size_and_page_size(size, page_size, buffer_type, buffer_layout, shard_parameters); -} - BufferPageMapping generate_buffer_page_mapping(const Buffer& buffer) { BufferPageMapping buffer_page_mapping; @@ -226,71 +182,129 @@ BufferPageMapping generate_buffer_page_mapping(const Buffer& buffer) { return buffer_page_mapping; } -void Buffer::allocate() { - { - std::unique_lock lock(allocation_mutex_); - TT_FATAL(allocation_status_.load(std::memory_order::relaxed) == AllocationStatus::NOT_ALLOCATED, "Can't allocate buffer after it was already allocated"); - allocation_status_.store(AllocationStatus::ALLOCATION_REQUESTED, std::memory_order::relaxed); +Buffer::Buffer( + Device *device, + DeviceAddr size, + DeviceAddr page_size, + const BufferType buffer_type, + const TensorMemoryLayout buffer_layout, + const std::optional& shard_parameters, + const std::optional bottom_up) : + device_(device), + size_(size), + page_size_(page_size), + buffer_type_(buffer_type), + buffer_layout_(buffer_layout), + shard_parameters_(shard_parameters), + bottom_up_(bottom_up), + buffer_page_mapping_(nullptr) { + TT_FATAL(this->device_ != nullptr && this->device_->allocator_ != nullptr, "Device and allocator need to not be null."); + + if (size != 0) { + validate_buffer_size_and_page_size(size, page_size, buffer_type, buffer_layout, shard_parameters); } +} - device_->push_work([self = weak_self.lock()] { - std::unique_lock lock(self->allocation_mutex_); - if (self->allocation_status_.load(std::memory_order::relaxed) != AllocationStatus::ALLOCATION_REQUESTED) { - // The allocation was interrupted by a deallocation +std::shared_ptr Buffer::create( + Device *device, + DeviceAddr size, + DeviceAddr page_size, + const BufferType buffer_type, + const TensorMemoryLayout buffer_layout, + const std::optional& shard_parameters, + const std::optional bottom_up) { + auto* bufferPtr = new Buffer(device, size, page_size, buffer_type, buffer_layout, shard_parameters, bottom_up); + auto buffer = std::shared_ptr(bufferPtr, deleter); + buffer->weak_self = buffer; + + if (buffer->size_ == 0) { + buffer->allocation_status_ = AllocationStatus::ALLOCATED; + return buffer; + } + + // Faster path for single-threaded mode + if (buffer->device_->can_use_passthrough_scheduling()) { + buffer->allocate_impl(); + buffer->allocation_status_ = AllocationStatus::ALLOCATED; + return buffer; + } + + buffer->device_->push_work([buffer] { + auto expected_status = AllocationStatus::ALLOCATION_REQUESTED; + if (!buffer->allocation_status_.compare_exchange_strong(expected_status, AllocationStatus::ALLOCATING)) { + // Buffer was already deallocated before we got here return; } - bool bottom_up = self->bottom_up_.value_or(self->is_dram()); - detail::AllocateBuffer(self.get(), bottom_up); - detail::BUFFER_MAP.insert({self->device_->id(), self->address_}, self.get()); + buffer->allocate_impl(); - self->allocation_status_.store(AllocationStatus::ALLOCATED, std::memory_order::relaxed); - lock.unlock(); - self->allocation_cv_.notify_all(); + // We need compare exchange here to handle the case of deallocation being requested before we finished allocating + expected_status = AllocationStatus::ALLOCATING; + if (buffer->allocation_status_.compare_exchange_strong(expected_status, AllocationStatus::ALLOCATED)) { + buffer->allocation_status_.notify_all(); + } }); + + return buffer; } -void Buffer::deallocate() { - if (size_ == 0) { - // 0-size buffer, no need to deallocate - return; - } +void Buffer::allocate_impl() { + bool bottom_up = bottom_up_.value_or(is_dram()); + address_ = detail::AllocateBuffer(this, bottom_up); + detail::BUFFER_MAP.insert({-device_->id(), address_}, this); +} - { - std::unique_lock lock(allocation_mutex_); - auto status = allocation_status_.load(std::memory_order::relaxed); - if (status != AllocationStatus::ALLOCATED && status != AllocationStatus::ALLOCATION_REQUESTED) { - // Buffer isn't allocated, nothing to be done - return; +bool Buffer::prepare_deallocation(std::atomic& status) { + while (true) { + auto current_status = status.load(); + switch (current_status) { + case AllocationStatus::ALLOCATION_REQUESTED: + // Allocation was requested but not started, canceling allocation, nothing else to be done + if (status.compare_exchange_weak(current_status, AllocationStatus::DEALLOCATED)) { + status.notify_all(); + return false; + } + break; + case AllocationStatus::ALLOCATING: + case AllocationStatus::ALLOCATED: + // Allocation already started, will have to deallocate + if (status.compare_exchange_weak(current_status, AllocationStatus::DEALLOCATION_REQUESTED)) { + status.notify_all(); + return true; + } + break; + case AllocationStatus::DEALLOCATION_REQUESTED: + case AllocationStatus::DEALLOCATED: + // Deallocation was already started, nothing to be done + return false; } - // Overwriting either ALLOCATED or ALLOCATION_REQUESTED with DEALLOCATION_REQUESTED - allocation_status_.store(AllocationStatus::DEALLOCATION_REQUESTED, std::memory_order::relaxed); + } +} + +void Buffer::deallocate() { + if (!prepare_deallocation(allocation_status_)) { + return; } device_->push_work([self = weak_self.lock()] { - // Because the status is DEALLOCATION_REQUESTED, it won't be changed by anyone else, no need to lock a mutex - if (!self->device_->initialized_) { - return; + if (self->device_->initialized_ && self->size_ != 0) { + detail::BUFFER_MAP.erase({self->device_->id(), self->address_}); + detail::DeallocateBuffer(self.get()); } - detail::BUFFER_MAP.erase({self->device()->id(), self->address()}); - detail::DeallocateBuffer(self.get()); - self->allocation_status_.store(AllocationStatus::DEALLOCATED, std::memory_order::relaxed); + self->allocation_status_ = AllocationStatus::DEALLOCATED; }); } -void Buffer::deallocateAndDelete(Buffer* buffer) { - // This is the last reference to the buffer, no need to lock or update AllocationStatus +void Buffer::deleter(Buffer* buffer) { + // There is no concurrent allocations/deallocations happening, so no extra checks are required + if (buffer->allocation_status_ == AllocationStatus::DEALLOCATED) { + return; + } + buffer->device_->push_work([buffer] { - // Buffer will be deleted at the end of this block std::unique_ptr unique_buffer = std::unique_ptr(buffer); - auto status = buffer->allocation_status_.load(std::memory_order::relaxed); - if (status == AllocationStatus::NOT_ALLOCATED || status == AllocationStatus::ALLOCATION_REQUESTED || status == AllocationStatus::DEALLOCATED) { - // Buffer isn't allocated, nothing to be done - return; - } - if (!buffer->device_->initialized_ || buffer->size_ == 0) { return; } @@ -301,30 +315,29 @@ void Buffer::deallocateAndDelete(Buffer* buffer) { } bool Buffer::is_allocated() const { - auto allocation_status = allocation_status_.load(std::memory_order::relaxed); + auto allocation_status = allocation_status_.load(); + if (device_->can_use_passthrough_scheduling()) { return allocation_status == AllocationStatus::ALLOCATED; } - // For calls from different threads we consider buffer to be allocated even if it's just ALLOCATION_REQUESTED, + + // For calls from different threads we consider buffer to be allocated even if it's just ALLOCATION_REQUESTED or ALLOCATING, // because once the caller will try to access it, the buffer will already be fully allocated - return allocation_status == AllocationStatus::ALLOCATED || allocation_status == AllocationStatus::ALLOCATION_REQUESTED; + return allocation_status == AllocationStatus::ALLOCATION_REQUESTED + || allocation_status == AllocationStatus::ALLOCATING + || allocation_status == AllocationStatus::ALLOCATED; } uint32_t Buffer::address() const { if (device_->can_use_passthrough_scheduling()) { - // No locking required, because address can only be modified from the same thread return address_; } - std::unique_lock lock(allocation_mutex_); - allocation_cv_.wait(lock, [this] { return this->allocation_status_.load(std::memory_order::relaxed) != AllocationStatus::ALLOCATION_REQUESTED; }); - return address_; -} + // Waiting for the buffer to be allocated if the allocation is pending + allocation_status_.wait(AllocationStatus::ALLOCATION_REQUESTED); + allocation_status_.wait(AllocationStatus::ALLOCATING); -void Buffer::set_address(uint64_t addr) { - TT_FATAL(device_->can_use_passthrough_scheduling() , "Buffer::set_address must be called in device worker thread"); - TT_FATAL(allocation_status_.load(std::memory_order::relaxed) == AllocationStatus::ALLOCATION_REQUESTED, "Buffer address can only be set during allocation"); - address_ = addr; + return address_; } DeviceAddr Buffer::page_size() const { diff --git a/tt_metal/impl/buffers/buffer.hpp b/tt_metal/impl/buffers/buffer.hpp index b612d03dd6a..30c6dec9a02 100644 --- a/tt_metal/impl/buffers/buffer.hpp +++ b/tt_metal/impl/buffers/buffer.hpp @@ -103,7 +103,6 @@ struct BufferConfig { DeviceAddr page_size; // Size of unit being interleaved. For non-interleaved buffers: size == page_size BufferType buffer_type; TensorMemoryLayout buffer_layout = TensorMemoryLayout::INTERLEAVED; - bool allocate = true; }; typedef BufferConfig InterleavedBufferConfig; @@ -117,7 +116,6 @@ struct ShardedBufferConfig { BufferType buffer_type = BufferType::L1; TensorMemoryLayout buffer_layout = TensorMemoryLayout::HEIGHT_SHARDED; ShardSpecBuffer shard_parameters; - bool allocate = true; }; } // namespace v0 @@ -149,8 +147,7 @@ class Buffer final { BufferType buffer_type, TensorMemoryLayout buffer_layout = TensorMemoryLayout::INTERLEAVED, const std::optional& shard_parameter = std::nullopt, - std::optional bottom_up = std::nullopt, - bool allocate = true); + std::optional bottom_up = std::nullopt); Buffer(const Buffer &other) = delete; Buffer &operator=(const Buffer &other) = delete; @@ -163,7 +160,6 @@ class Buffer final { // Returns address of buffer in the first bank uint32_t address() const; - void set_address(uint64_t addr); DeviceAddr page_size() const; void set_page_size(DeviceAddr page_size); @@ -217,46 +213,35 @@ class Buffer final { const std::optional& shard_parameter, std::optional bottom_up); - void allocate(); - void deallocate(); - static void deallocateAndDelete(Buffer* buffer); - - friend void DeallocateBuffer(Buffer &buffer); - - DeviceAddr translate_page_address(uint64_t offset, uint32_t bank_id) const; - enum class AllocationStatus : uint8_t { - // The buffer is created in NOT_ALLOCATED state (except for 0-size buffers, which are initially ALLOCATED). - // The buffer can transition from NOT_ALLOCATED to ALLOCATION_REQUESTED only once in its lifetime. - NOT_ALLOCATED, - // The task is scheduled on the queue to allocate the buffer. - // When the task succeeds, the buffer transitions into ALLOCATED state. - // The scheduled allocation can be interrupted by a deallocation, which would transition the buffer to DEALLOCATION_REQUESTED, - // and then to DEALLOCATED states. ALLOCATION_REQUESTED, - // The buffer is completely allocated and the address is available. - // The buffer can transition from ALLOCATED only to DEALLOCATION_REQUESTED. + ALLOCATING, ALLOCATED, - // The task is scheduled to deallocate the buffer. - // When the task succeeds, the buffer transitions into DEALLOCATED state. DEALLOCATION_REQUESTED, - // The buffer is completely deallocated. - // This is the final state, no transitions from this state are possible. DEALLOCATED, }; + void allocate_impl(); + // Deallocate is allowed to be called multiple times on the same buffer + void deallocate(); + static void deleter(Buffer* buffer); + + static bool prepare_deallocation(std::atomic& status); + + friend void DeallocateBuffer(Buffer &buffer); + + DeviceAddr translate_page_address(uint64_t offset, uint32_t bank_id) const; + Device * const device_; const DeviceAddr size_; // Size in bytes const BufferType buffer_type_; const TensorMemoryLayout buffer_layout_; const std::optional bottom_up_; - std::atomic allocation_status_ = AllocationStatus::NOT_ALLOCATED; - mutable std::condition_variable allocation_cv_; - mutable std::mutex allocation_mutex_; + std::atomic allocation_status_ = AllocationStatus::ALLOCATION_REQUESTED; + std::atomic address_ = 0; // These members must be only accessed on the device worker thread - DeviceAddr address_ = 0; // Address of buffer DeviceAddr page_size_; // Size of unit being interleaved. For non-interleaved buffers: size == page_size std::optional shard_parameters_; std::shared_ptr buffer_page_mapping_; diff --git a/tt_metal/impl/dispatch/command_queue.cpp b/tt_metal/impl/dispatch/command_queue.cpp index 010ad9e4359..a5b355a9fd4 100644 --- a/tt_metal/impl/dispatch/command_queue.cpp +++ b/tt_metal/impl/dispatch/command_queue.cpp @@ -2789,62 +2789,6 @@ void EnqueueGetBufferAddr(CommandQueue& cq, uint32_t* dst_buf_addr, const Buffer .type = EnqueueCommandType::GET_BUF_ADDR, .blocking = blocking, .shadow_buffer = buffer, .dst = dst_buf_addr}); } -void EnqueueAllocateBufferImpl(AllocBufferMetadata alloc_md) { - Buffer* buffer = alloc_md.buffer; - uint32_t allocated_addr; - if (is_sharded(buffer->buffer_layout())) { - allocated_addr = allocator::allocate_buffer( - *(buffer->device()->allocator_), - buffer->shard_spec().size() * buffer->num_cores() * buffer->page_size(), - buffer->page_size(), - buffer->buffer_type(), - alloc_md.bottom_up, - buffer->num_cores()); - } else { - allocated_addr = allocator::allocate_buffer( - *(buffer->device()->allocator_), - buffer->size(), - buffer->page_size(), - buffer->buffer_type(), - alloc_md.bottom_up, - std::nullopt); - } - TT_ASSERT(allocated_addr <= std::numeric_limits::max()); - buffer->set_address(static_cast(allocated_addr)); -} - -void EnqueueAllocateBuffer(CommandQueue& cq, Buffer* buffer, bool bottom_up, bool blocking) { - auto alloc_md = AllocBufferMetadata{ - .buffer = buffer, - .allocator = *(buffer->device()->allocator_), - .bottom_up = bottom_up, - }; - cq.run_command(CommandInterface{ - .type = EnqueueCommandType::ALLOCATE_BUFFER, - .blocking = blocking, - .alloc_md = alloc_md, - }); -} - -void EnqueueDeallocateBufferImpl(AllocBufferMetadata alloc_md) { - allocator::deallocate_buffer(alloc_md.allocator, alloc_md.device_address, alloc_md.buffer_type); -} - -void EnqueueDeallocateBuffer( - CommandQueue& cq, Allocator& allocator, uint32_t device_address, BufferType buffer_type, bool blocking) { - // Need to explictly pass in relevant buffer attributes here, since the Buffer* ptr can be deallocated a this point - auto alloc_md = AllocBufferMetadata{ - .allocator = allocator, - .buffer_type = buffer_type, - .device_address = device_address, - }; - cq.run_command(CommandInterface{ - .type = EnqueueCommandType::DEALLOCATE_BUFFER, - .blocking = blocking, - .alloc_md = alloc_md, - }); -} - inline namespace v0 { void EnqueueReadBuffer( @@ -3216,14 +3160,6 @@ void CommandQueue::run_command_impl(const CommandInterface& command) { TT_ASSERT(command.blocking.has_value(), "Must specify blocking value!"); EnqueueWriteBufferImpl(*this, command.buffer.value(), command.src.value(), command.blocking.value()); break; - case EnqueueCommandType::ALLOCATE_BUFFER: - TT_ASSERT(command.alloc_md.has_value(), "Must provide buffer allocation metdata!"); - EnqueueAllocateBufferImpl(command.alloc_md.value()); - break; - case EnqueueCommandType::DEALLOCATE_BUFFER: - TT_ASSERT(command.alloc_md.has_value(), "Must provide buffer allocation metdata!"); - EnqueueDeallocateBufferImpl(command.alloc_md.value()); - break; case EnqueueCommandType::GET_BUF_ADDR: TT_ASSERT(command.dst.has_value(), "Must provide a dst address!"); TT_ASSERT(command.shadow_buffer.has_value(), "Must provide a shadow buffer!"); diff --git a/tt_metal/impl/dispatch/command_queue.hpp b/tt_metal/impl/dispatch/command_queue.hpp index 821a806ff9c..d8b4f72fb14 100644 --- a/tt_metal/impl/dispatch/command_queue.hpp +++ b/tt_metal/impl/dispatch/command_queue.hpp @@ -36,8 +36,6 @@ using RuntimeArgs = std::vector>; enum class EnqueueCommandType { ENQUEUE_READ_BUFFER, ENQUEUE_WRITE_BUFFER, - ALLOCATE_BUFFER, - DEALLOCATE_BUFFER, GET_BUF_ADDR, ADD_BUFFER_TO_PROGRAM, SET_RUNTIME_ARGS, @@ -577,8 +575,6 @@ class HWCommandQueue { std::variant, std::shared_ptr> buffer, HostDataType src, bool blocking); - friend void EnqueueAllocateBufferImpl(AllocBufferMetadata alloc_md); - friend void EnqueueDeallocateBufferImpl(AllocBufferMetadata alloc_md); friend void EnqueueGetBufferAddrImpl(void* dst_buf_addr, const Buffer* buffer); friend void EnqueueRecordEventImpl(CommandQueue& cq, const std::shared_ptr& event); friend void EnqueueWaitForEventImpl(CommandQueue& cq, const std::shared_ptr& event); @@ -696,9 +692,6 @@ class CommandQueue { // Primitives used to place host only operations on the SW Command Queue. // These are used in functions exposed through tt_metal.hpp or host_api.hpp -void EnqueueAllocateBuffer(CommandQueue& cq, Buffer* buffer, bool bottom_up, bool blocking); -void EnqueueDeallocateBuffer( - CommandQueue& cq, Allocator& allocator, uint32_t device_address, BufferType buffer_type, bool blocking); void EnqueueGetBufferAddr(CommandQueue& cq, uint32_t* dst_buf_addr, const Buffer* buffer, bool blocking); void EnqueueSetRuntimeArgs( CommandQueue& cq, diff --git a/tt_metal/tt_metal.cpp b/tt_metal/tt_metal.cpp index 14b405a1846..eb5ef5465c1 100644 --- a/tt_metal/tt_metal.cpp +++ b/tt_metal/tt_metal.cpp @@ -851,13 +851,35 @@ void CompileProgram(Device *device, Program &program, bool fd_bootloader_mode) { program.compile(device, fd_bootloader_mode); } -void AllocateBuffer(Buffer *buffer, bool bottom_up) { +DeviceAddr AllocateBuffer(const Buffer *buffer, bool bottom_up) { if(GraphTracker::instance().hook_allocate(buffer, bottom_up)) { GraphTracker::instance().track_allocate(buffer, bottom_up); - return; + return 0; + } + + uint32_t allocated_addr; + if (is_sharded(buffer->buffer_layout())) { + allocated_addr = allocator::allocate_buffer( + *(buffer->device()->allocator_), + buffer->shard_spec().size() * buffer->num_cores() * buffer->page_size(), + buffer->page_size(), + buffer->buffer_type(), + bottom_up, + buffer->num_cores()); + } else { + allocated_addr = allocator::allocate_buffer( + *(buffer->device()->allocator_), + buffer->size(), + buffer->page_size(), + buffer->buffer_type(), + bottom_up, + std::nullopt); } - EnqueueAllocateBuffer(buffer->device()->command_queue(), buffer, bottom_up, false); + TT_ASSERT(allocated_addr <= std::numeric_limits::max()); + GraphTracker::instance().track_allocate(buffer, bottom_up); + + return allocated_addr; } void DeallocateBuffer(Buffer *buffer) { @@ -865,12 +887,8 @@ void DeallocateBuffer(Buffer *buffer) { if(GraphTracker::instance().hook_deallocate(buffer)) { return; } - EnqueueDeallocateBuffer( - buffer->device()->command_queue(), - *(buffer->device()->allocator_), - buffer->address(), - buffer->buffer_type(), - false); + + allocator::deallocate_buffer(*buffer->device()->allocator_, buffer->address(), buffer->buffer_type()); } } // namespace detail @@ -1086,7 +1104,7 @@ uint32_t CreateSemaphore( std::shared_ptr CreateBuffer(const InterleavedBufferConfig &config) { return Buffer::create( - config.device, config.size, config.page_size, config.buffer_type, config.buffer_layout, std::nullopt, std::nullopt, config.allocate); + config.device, config.size, config.page_size, config.buffer_type, config.buffer_layout, std::nullopt, std::nullopt); } std::shared_ptr CreateBuffer(const ShardedBufferConfig &config) { @@ -1097,8 +1115,7 @@ std::shared_ptr CreateBuffer(const ShardedBufferConfig &config) { config.buffer_type, config.buffer_layout, config.shard_parameters, - std::nullopt, - config.allocate); + std::nullopt); } void DeallocateBuffer(Buffer &buffer) { buffer.deallocate(); } diff --git a/ttnn/cpp/ttnn/graph/graph_processor.cpp b/ttnn/cpp/ttnn/graph/graph_processor.cpp index 76a2668a9dc..bebeadebd9d 100644 --- a/ttnn/cpp/ttnn/graph/graph_processor.cpp +++ b/ttnn/cpp/ttnn/graph/graph_processor.cpp @@ -90,7 +90,7 @@ GraphProcessor::GraphProcessor(RunMode mode) : run_mode(mode) { end_function_any_map[typeid(std::reference_wrapper)] = [ptr = this] (const std::any& val) mutable {ptr->end_function_process_tensor(val);}; } -void GraphProcessor::track_allocate(tt::tt_metal::Buffer* buffer, bool bottom_up) { +void GraphProcessor::track_allocate(const tt::tt_metal::Buffer* buffer, bool bottom_up) { const std::lock_guard lock(mutex); auto buf_id = add_buffer(buffer); @@ -299,7 +299,7 @@ int GraphProcessor::add_tensor(const Tensor& t) { return tensor_counter; } -int GraphProcessor::add_buffer(tt::tt_metal::Buffer* buffer) { +int GraphProcessor::add_buffer(const tt::tt_metal::Buffer* buffer) { auto buffer_alloc_id = reinterpret_cast(buffer); auto counter = id_to_counter.count(buffer_alloc_id) > 0 ? id_to_counter[buffer_alloc_id] : graph.size(); if (id_to_counter.count(buffer_alloc_id) == 0) { @@ -478,7 +478,7 @@ nlohmann::json GraphProcessor::end_graph_capture() { return res; } -bool ProcessorHooks::hook_allocate(tt::tt_metal::Buffer* buffer, bool bottom_up) { +bool ProcessorHooks::hook_allocate(const tt::tt_metal::Buffer* buffer, bool bottom_up) { return do_block; } diff --git a/ttnn/cpp/ttnn/graph/graph_processor.hpp b/ttnn/cpp/ttnn/graph/graph_processor.hpp index fa614e5d63a..4f7d6f1b6e7 100644 --- a/ttnn/cpp/ttnn/graph/graph_processor.hpp +++ b/ttnn/cpp/ttnn/graph/graph_processor.hpp @@ -22,7 +22,7 @@ namespace ttnn::graph { public: ProcessorHooks() = default; - bool hook_allocate(tt::tt_metal::Buffer* buffer, bool bottom_up) override; + bool hook_allocate(const tt::tt_metal::Buffer* buffer, bool bottom_up) override; bool hook_deallocate(tt::tt_metal::Buffer* buffer) override; @@ -40,7 +40,7 @@ namespace ttnn::graph { GraphProcessor(tt::tt_metal::IGraphProcessor::RunMode mode); ~GraphProcessor() override; - void track_allocate(tt::tt_metal::Buffer* buffer, bool bottom_up) override; + void track_allocate(const tt::tt_metal::Buffer* buffer, bool bottom_up) override; void track_deallocate(tt::tt_metal::Buffer* buffer) override; @@ -80,7 +80,7 @@ namespace ttnn::graph { std::unordered_map end_function_any_map; int add_tensor(const Tensor& t); - int add_buffer(tt::tt_metal::Buffer* buffer); + int add_buffer(const tt::tt_metal::Buffer* buffer); void begin_function_process_ref_vector(const std::any& any_val); void begin_function_process_ref_vector_optional(const std::any& any_val);