Commit 0599a628 authored by Chris Sullivan's avatar Chris Sullivan Committed by Robert Kimball

Preallocate intermediate buffers (#1231)

* Utilize GPUMemoryManager/Allocator for preallocation of intermediate tensor buffer memory.

* Formatting.

* Merge with master required rework of memory due to CFE pass. Moved function memory pool allocation to pass as a result.

* Formatting.

* Added pass source files.

* Updated tests to account for new assert check. All GPUAllocators should be deconstructed before allocation is made in GPUMemoryManager.

* GPUAllocator::close() can be used to close the allocator prior to destruction

* Removed open allocators. Replaced check with inspection of pass::MemoryManager node list.

* Formatting.

* Rename m_memory_buffers -> m_tensor_memory_buffers. Use full path to static alignment variable.

* FunctionMemoryReservation -> TensorMemoryReservation. Only return true in pass if reservation is made (bug fix).

* Moved static compilation mutex.

* Update external function with new pass name.

* GPU_ExternalFunction: Add s_memory_pool_alignment, remove optimize_and_assemble method.
parent 45b50d06
......@@ -38,6 +38,7 @@ set(SRC
gpu_tensor_view.cpp
gpu_util.cpp
type_info.cpp
pass/tensor_memory_reservation.cpp
)
if (NGRAPH_GPU_ENABLE)
......
......@@ -20,6 +20,7 @@
#include <cuda_runtime.h>
#include <cudnn.h>
#include <fstream>
#include <mutex>
#include <string>
#include <tuple>
......@@ -103,11 +104,13 @@
#include "ngraph/runtime/gpu/gpu_external_function.hpp"
#include "ngraph/runtime/gpu/gpu_kernel_emitters.hpp"
#include "ngraph/runtime/gpu/gpu_runtime_context.hpp"
#include "ngraph/runtime/gpu/pass/tensor_memory_reservation.hpp"
using namespace std;
using namespace ngraph;
static const string s_output_dir = "gpu_codegen";
static std::mutex s_compilation;
class StaticInitializers
{
......@@ -236,6 +239,8 @@ static const runtime::gpu::OpMap dispatcher{
{TI(ngraph::op::And), &runtime::gpu::GPU_Emitter::emit_elementwise<ngraph::op::And>},
{TI(ngraph::op::Or), &runtime::gpu::GPU_Emitter::emit_elementwise<ngraph::op::Or>}};
const size_t runtime::gpu::GPU_ExternalFunction::GPU_ExternalFunction::s_memory_pool_alignment = 64;
runtime::gpu::GPU_ExternalFunction::GPU_ExternalFunction(
const shared_ptr<ngraph::Function>& function,
std::shared_ptr<GPU_Backend::BackendContext>& shared_context,
......@@ -246,6 +251,7 @@ runtime::gpu::GPU_ExternalFunction::GPU_ExternalFunction(
, m_is_compiled(false)
, m_release_function(release_function)
, m_temporaries_used(false)
, m_tensor_memory_buffers(new std::unordered_map<std::string, size_t>)
, m_shared_context(shared_context)
{
}
......@@ -372,7 +378,7 @@ void runtime::gpu::GPU_ExternalFunction::emit_constant_declarations()
{
shared_ptr<descriptor::TensorView> tv = node->get_outputs()[0].get_tensor_view();
// get an allocator for transient per kernel gpu memory
GPUAllocator allocator =
runtime::gpu::GPUAllocator allocator =
m_shared_context->m_primitive_emitter->get_memory_allocator();
size_t idx = allocator.reserve_argspace(
c->get_data_ptr(),
......@@ -445,11 +451,10 @@ void runtime::gpu::GPU_ExternalFunction::emit_temp_mem_pool_allocation(
}
if (m_temporaries_used)
{
size_t temp_pool_size = current_function->get_temporary_pool_size();
m_writer << "// Allocate the memory pool\n";
// TODO memory pool malloc.
m_writer << "void* pool_base_ptr = ngraph::runtime::gpu::create_gpu_buffer("
<< temp_pool_size << ");\n";
m_writer << "void* pool_base_ptr = ngraph::runtime::gpu::invoke_memory_primitive(ctx, "
<< m_tensor_memory_buffers->at(current_function->get_name()) << ");\n";
// Add temporaries to the variable name map
for (shared_ptr<Node> node : m_function_ordered_ops.at(current_function))
......@@ -465,14 +470,6 @@ void runtime::gpu::GPU_ExternalFunction::emit_temp_mem_pool_allocation(
}
}
void runtime::gpu::GPU_ExternalFunction::emit_temp_mem_pool_release()
{
if (m_temporaries_used)
{
m_writer << "ngraph::runtime::gpu::free_gpu_buffer(pool_base_ptr);\n";
}
}
void runtime::gpu::GPU_ExternalFunction::emit_functions()
{
for (shared_ptr<Function> current_function : m_pass_manager.get_state().get_functions())
......@@ -614,7 +611,6 @@ void runtime::gpu::GPU_ExternalFunction::emit_functions()
emit_debug_function_exit(node.get());
}
}
emit_temp_mem_pool_release();
}
m_writer.block_end(); // End generated function
}
......@@ -635,28 +631,41 @@ void runtime::gpu::GPU_ExternalFunction::compile()
{
return;
}
std::unique_lock<std::mutex> lock(s_compilation);
m_function_name = m_function->get_name();
string dump_filename = file_util::path_join(s_output_dir, m_function_name + "_ops.txt");
// For now, just make everyone row-major.
m_pass_manager.register_pass<pass::ResultCopyElimination>();
m_pass_manager.register_pass<pass::AssignLayout<descriptor::layout::DenseTensorViewLayout>>();
string common_function_string;
auto allocator = std::make_shared<runtime::gpu::GPUAllocator>(
m_shared_context->m_primitive_emitter->get_memory_allocator());
m_pass_manager.register_pass<ngraph::pass::ResultCopyElimination>();
m_pass_manager
.register_pass<ngraph::pass::AssignLayout<descriptor::layout::DenseTensorViewLayout>>();
m_pass_manager.register_pass<ngraph::pass::Liveness>();
m_pass_manager.register_pass<ngraph::pass::MemoryLayout>(s_memory_pool_alignment);
m_pass_manager.register_pass<runtime::gpu::pass::TensorMemoryReservation>(
allocator, m_tensor_memory_buffers);
std::string common_function_string;
auto femitter = bind(&ngraph::runtime::gpu::GPU_ExternalFunction::emit_op_as_function,
this,
placeholders::_1,
placeholders::_2);
m_pass_manager.register_pass<ngraph::pass::CommonFunctionCollection>(
femitter, m_node_function_map, common_function_string);
m_pass_manager.register_pass<pass::Liveness>();
m_pass_manager.register_pass<pass::MemoryLayout>(64);
m_pass_manager.register_pass<pass::DumpSorted>(dump_filename);
string dump_filename = file_util::path_join(s_output_dir, m_function_name + "_ops.txt");
m_pass_manager.register_pass<ngraph::pass::DumpSorted>(dump_filename);
m_pass_manager.run_passes(m_function);
for (shared_ptr<Function> current_function : m_pass_manager.get_state().get_functions())
{
m_function_ordered_ops.insert({current_function, current_function->get_ordered_ops()});
m_function_ordered_ops.emplace(current_function, current_function->get_ordered_ops());
}
emit_header();
......@@ -667,6 +676,7 @@ void runtime::gpu::GPU_ExternalFunction::compile()
emit_functions();
// allocate device buffers for primitive arguments and workspace
allocator->close();
m_shared_context->m_primitive_emitter->allocate_primitive_memory();
string code = m_writer.get_code();
......
......@@ -74,13 +74,14 @@ namespace ngraph
return m_shared_context->m_primitive_emitter;
}
static const size_t s_memory_pool_alignment;
protected:
void compile();
EntryPoint m_compiled_function;
private:
void collect_unique_functions();
void emit_header();
void emit_timer_functions();
void emit_constant_declarations();
......@@ -89,14 +90,13 @@ namespace ngraph
void emit_debug_function_entry(Node* node);
void emit_debug_function_exit(Node* node);
void emit_temp_mem_pool_allocation(std::shared_ptr<Function> current_function);
void emit_temp_mem_pool_release();
void release_function() { m_function = nullptr; }
void store_emitted_functions(const std::string& code);
std::string emit_op_as_function(const Node& node, const std::string& function_name);
std::string strip_comments(const std::string& s) const;
codegen::CodeWriter m_writer;
pass::Manager m_pass_manager;
ngraph::pass::Manager m_pass_manager;
std::unique_ptr<codegen::Compiler> m_compiler;
std::unique_ptr<codegen::ExecutionEngine> m_execution_engine;
......@@ -117,6 +117,7 @@ namespace ngraph
std::string m_function_name;
std::string m_pch_header_source;
std::shared_ptr<std::unordered_map<std::string, size_t>> m_tensor_memory_buffers;
std::shared_ptr<GPU_Backend::BackendContext> m_shared_context;
};
}
......
......@@ -27,7 +27,7 @@ constexpr const uint32_t initial_buffer_size = 10 * 1024 * 1024;
runtime::gpu::GPUMemoryManager::GPUMemoryManager(GPUPrimitiveEmitter* emitter)
: m_buffer_offset(0)
, m_buffered_mem(initial_buffer_size)
, m_workspace_manager(alignment)
, m_workspace_manager(runtime::gpu::GPUMemoryManager::alignment)
, m_argspace_mem(1, {nullptr, 0})
, m_workspace_mem(1, {nullptr, 0})
, m_primitive_emitter(emitter)
......@@ -62,9 +62,16 @@ runtime::gpu::GPUMemoryManager::~GPUMemoryManager()
void runtime::gpu::GPUMemoryManager::allocate()
{
if (m_workspace_manager.get_node_list().size() != 1)
{
throw std::runtime_error(
"Attempt to allocate memory while reservations are inprogress. Ensure all "
"GPUAllocators are closed before allocating.");
}
if (m_buffer_offset)
{
m_buffer_offset = pass::MemoryManager::align(m_buffer_offset, alignment);
m_buffer_offset = ngraph::pass::MemoryManager::align(
m_buffer_offset, runtime::gpu::GPUMemoryManager::alignment);
// the back most node is always empty, fill it here
m_argspace_mem.back().ptr = runtime::gpu::create_gpu_buffer(m_buffer_offset);
m_argspace_mem.back().size = m_buffer_offset;
......@@ -82,8 +89,6 @@ void runtime::gpu::GPUMemoryManager::allocate()
m_workspace_mem.back().ptr = runtime::gpu::create_gpu_buffer(workspace_size);
m_workspace_mem.back().size = workspace_size;
m_workspace_mem.push_back({nullptr, 0});
// construct a new manager if the current one was used
m_workspace_manager = pass::MemoryManager(alignment);
}
}
......@@ -104,6 +109,11 @@ size_t runtime::gpu::GPUMemoryManager::queue_for_transfer(const void* data, size
return offset;
}
runtime::gpu::GPUAllocator::GPUAllocator(GPUMemoryManager* mgr)
: m_manager(mgr)
{
}
runtime::gpu::GPUAllocator::GPUAllocator(const GPUAllocator& g)
{
m_manager = g.m_manager;
......@@ -113,7 +123,7 @@ runtime::gpu::GPUAllocator::GPUAllocator(const GPUAllocator& g)
size_t runtime::gpu::GPUAllocator::reserve_argspace(const void* data, size_t size)
{
// add parameter data to host buffer that will be transfered to device
size = pass::MemoryManager::align(size, runtime::gpu::GPUMemoryManager::alignment);
size = ngraph::pass::MemoryManager::align(size, runtime::gpu::GPUMemoryManager::alignment);
size_t offset = m_manager->queue_for_transfer(data, size);
auto local = std::prev(m_manager->m_argspace_mem.end());
// return a lambda that will yield the gpu memory address. this
......@@ -154,7 +164,7 @@ size_t runtime::gpu::GPUAllocator::reserve_workspace(size_t size, bool zero_init
return m_manager->m_primitive_emitter->insert(mem_primitive);
}
runtime::gpu::GPUAllocator::~GPUAllocator()
void runtime::gpu::GPUAllocator::close()
{
while (!m_active.empty())
{
......@@ -162,3 +172,7 @@ runtime::gpu::GPUAllocator::~GPUAllocator()
m_active.pop();
}
}
runtime::gpu::GPUAllocator::~GPUAllocator()
{
this->close();
}
......@@ -36,10 +36,7 @@ namespace ngraph
{
public:
GPUAllocator() = delete;
GPUAllocator(GPUMemoryManager* mgr)
: m_manager(mgr)
{
}
GPUAllocator(GPUMemoryManager* mgr);
GPUAllocator(const GPUAllocator& g);
~GPUAllocator();
......@@ -52,6 +49,8 @@ namespace ngraph
size_t reserve_argspace(const void* data, size_t size);
size_t reserve_workspace(size_t size, bool zero_initialize = true);
void close();
private:
GPUMemoryManager* m_manager;
std::stack<size_t> m_active;
......@@ -74,7 +73,7 @@ namespace ngraph
size_t m_buffer_offset;
std::vector<uint8_t> m_buffered_mem;
pass::MemoryManager m_workspace_manager;
ngraph::pass::MemoryManager m_workspace_manager;
static constexpr const uint16_t alignment = 8;
struct allocation
......
/*******************************************************************************
* Copyright 2017-2018 Intel Corporation
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*******************************************************************************/
#include <memory>
#include "ngraph/function.hpp"
#include "ngraph/node.hpp"
#include "ngraph/pass/manager_state.hpp"
#include "ngraph/graph_util.hpp"
#include "ngraph/runtime/gpu/gpu_memory_manager.hpp"
#include "ngraph/runtime/gpu/pass/tensor_memory_reservation.hpp"
using namespace ngraph;
bool ngraph::runtime::gpu::pass::TensorMemoryReservation::run_on_function(
std::shared_ptr<Function> f)
{
auto allocator = m_allocator.lock();
auto buffers = m_memory_buffers.lock();
if (allocator && buffers)
{
size_t mem_pool_size = f->get_temporary_pool_size();
if (mem_pool_size)
{
size_t pool_idx = allocator->reserve_workspace(mem_pool_size, false);
buffers->insert({f->get_name(), pool_idx});
return true;
}
}
return false;
}
/*******************************************************************************
* Copyright 2017-2018 Intel Corporation
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*******************************************************************************/
#pragma once
#include <memory>
#include "ngraph/pass/pass.hpp"
namespace ngraph
{
namespace runtime
{
namespace gpu
{
class GPUAllocator;
namespace pass
{
class TensorMemoryReservation;
}
}
}
}
class ngraph::runtime::gpu::pass::TensorMemoryReservation : public ngraph::pass::FunctionPass
{
public:
TensorMemoryReservation(std::weak_ptr<ngraph::runtime::gpu::GPUAllocator> allocator,
std::weak_ptr<std::unordered_map<std::string, size_t>> buffers)
: ngraph::pass::FunctionPass()
, m_allocator(allocator)
, m_memory_buffers(buffers)
{
}
virtual bool run_on_function(std::shared_ptr<ngraph::Function> f);
private:
std::weak_ptr<ngraph::runtime::gpu::GPUAllocator> m_allocator;
std::weak_ptr<std::unordered_map<std::string, size_t>> m_memory_buffers;
};
......@@ -43,8 +43,11 @@ TEST(gpu_test, memory_manager_unallocated)
TEST(gpu_test, memory_manager_allocated)
{
runtime::gpu::GPUPrimitiveEmitter emitter;
auto allocator = emitter.get_memory_allocator();
size_t idx = allocator.reserve_workspace(10);
size_t idx;
{
auto allocator = emitter.get_memory_allocator();
idx = allocator.reserve_workspace(10);
}
emitter.allocate_primitive_memory();
runtime::gpu::memory_primitive& mem_primitive = emitter.get_memory_primitives()[idx];
EXPECT_NO_THROW(mem_primitive());
......@@ -52,10 +55,13 @@ TEST(gpu_test, memory_manager_allocated)
TEST(gpu_test, memory_manager_extract_arguments)
{
runtime::gpu::GPUPrimitiveEmitter emitter;
auto allocator = emitter.get_memory_allocator();
std::vector<float> fp32_args = {2112.0f, 2112.0f};
size_t idx = allocator.reserve_argspace(fp32_args.data(), fp32_args.size() * sizeof(float));
runtime::gpu::GPUPrimitiveEmitter emitter;
size_t idx;
{
auto allocator = emitter.get_memory_allocator();
idx = allocator.reserve_argspace(fp32_args.data(), fp32_args.size() * sizeof(float));
}
emitter.allocate_primitive_memory();
runtime::gpu::memory_primitive& mem_primitive = emitter.get_memory_primitives()[idx];
std::vector<float> host(2, 0);
......@@ -65,10 +71,12 @@ TEST(gpu_test, memory_manager_extract_arguments)
TEST(gpu_test, memory_manager_argspace_size)
{
runtime::gpu::GPUPrimitiveEmitter emitter;
auto allocator = emitter.get_memory_allocator();
std::vector<float> fp32_args = {2112.0f, 2112.0f};
allocator.reserve_argspace(fp32_args.data(), fp32_args.size() * sizeof(float));
runtime::gpu::GPUPrimitiveEmitter emitter;
{
auto allocator = emitter.get_memory_allocator();
allocator.reserve_argspace(fp32_args.data(), fp32_args.size() * sizeof(float));
}
emitter.allocate_primitive_memory();
EXPECT_EQ(emitter.sizeof_device_allocation(), fp32_args.size() * sizeof(float));
}
......
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment