[GPU] Apply multi-threads for async compilation context (#15683)
* [GPU] Apply multi-threads for async compilation context (#15683) - Use CPUStreamExecutor in compilation context - Use single compilation context, impl_cache and kernels_cache for multple streams - Move compilation context to cldnn::program - Move impl_cache to cldnn::program - Create thread-safe impl_cache - Create thread independent compilation function in kernels_cache - Use kernels_cache in program and remove it from network * [GPU] Fix segfault issue: ocl_engine and ocl_device are released during remained compilation context task are running (#15683) - compilation context has own CPUStreamExecutor * [GPU] Follow-up codereview (#15683) - LruCacheThreadSafe inherit LruCache - FuncRemoveItem has std::pair<Key,Value> as input - Change prepare_tools to init_program * [GPU] Create primitive_impl::build_kernels (#15683) * [GPU] Fix unit test build error (#15683) * [GPU] Remove redundant code (#15683) - Remove try catch for debug - Call compilation_context.cancel() in destructor of network * [GPU] combine two atomic counter in kernels_cache (#15683) * [GPU] Follow-up code review (#15683) * [GPU] Fix nullptr exception in unit test (#15683) * [GPU] Follow-up code review (#15683) - Modify mutex lock in compilation context * [GPU] Fix windows build issue (#15683)
This commit is contained in:
parent
2d960fc6c5
commit
c1c8d6320e
@ -231,16 +231,8 @@ public:
|
||||
/// Returns memory state @p variable_id of stateful network
|
||||
VariableState& get_variable_memory(const std::string &variable_id);
|
||||
|
||||
/// Return kernels_cache
|
||||
kernels_cache& get_kernels_cache() const { return *_kernels_cache; }
|
||||
|
||||
/// Return implentations_cache
|
||||
ImplementationsCache& get_implementations_cache() const { return *_impls_cache; }
|
||||
|
||||
/// Return in_mem_kernels_cache
|
||||
KernelsCache& get_in_mem_kernels_cache() const { return *_in_mem_kernels_cache; }
|
||||
|
||||
ICompilationContext& get_compilation_context() const { return *_compilation_context; }
|
||||
std::mutex& get_impl_cache_mutex() const { return _in_mem_cache_mutex; }
|
||||
|
||||
const ExecutionConfig& get_config() const { return _config; }
|
||||
@ -272,7 +264,6 @@ private:
|
||||
output_chains_map _output_chains;
|
||||
|
||||
mutable std::mutex _in_mem_cache_mutex;
|
||||
std::unique_ptr<ICompilationContext> _compilation_context;
|
||||
|
||||
void build_exec_order();
|
||||
void allocate_primitive_instance(program_node const& node);
|
||||
@ -284,11 +275,8 @@ private:
|
||||
void add_default_output_chains();
|
||||
output_chains_map::iterator add_output_chain(std::shared_ptr<primitive_inst>& p_inst);
|
||||
|
||||
std::unique_ptr<kernels_cache> _kernels_cache;
|
||||
// Move from cldnn::program to cldnn::network for multi-threads issue.
|
||||
std::unique_ptr<ImplementationsCache> _impls_cache;
|
||||
std::unique_ptr<KernelsCache> _in_mem_kernels_cache;
|
||||
const size_t _impls_cache_capacity = 10000;
|
||||
const size_t _in_mem_kernels_cache_capacity = 10000;
|
||||
};
|
||||
} // namespace cldnn
|
||||
|
@ -26,6 +26,7 @@ class pass_manager;
|
||||
class base_pass;
|
||||
class program_wrapper;
|
||||
class kernels_cache;
|
||||
class ICompilationContext;
|
||||
|
||||
|
||||
struct program {
|
||||
@ -252,6 +253,10 @@ public:
|
||||
void query_local_block_io_supported();
|
||||
void calc_nodes_hash();
|
||||
|
||||
ImplementationsCache& get_implementations_cache() const { return *_impls_cache; }
|
||||
ICompilationContext& get_compilation_context() const { return *_compilation_context; }
|
||||
void cancel_compilation_context();
|
||||
|
||||
private:
|
||||
uint32_t prog_id = 0;
|
||||
engine& _engine;
|
||||
@ -266,6 +271,9 @@ private:
|
||||
std::unique_ptr<pass_manager> pm;
|
||||
bool is_body_program;
|
||||
int8_t is_subgroup_local_block_io_supported;
|
||||
std::unique_ptr<ImplementationsCache> _impls_cache;
|
||||
const size_t _impls_cache_capacity = 10000;
|
||||
std::unique_ptr<ICompilationContext> _compilation_context;
|
||||
|
||||
std::map<primitive_id, std::shared_ptr<program_node>> nodes_map;
|
||||
std::list<primitive_id> optimized_out;
|
||||
@ -305,7 +313,9 @@ private:
|
||||
void cleanup();
|
||||
void transfer_memory_to_device();
|
||||
|
||||
InferenceEngine::CPUStreamsExecutor::Config make_task_executor_config(const ExecutionConfig& config, std::string tags = "") const;
|
||||
std::shared_ptr<InferenceEngine::CPUStreamsExecutor> make_task_executor(const ExecutionConfig& config) const;
|
||||
|
||||
/*
|
||||
** Analysis functions
|
||||
*/
|
||||
@ -343,6 +353,8 @@ private:
|
||||
// old_node - node which will be replaced
|
||||
// new_node - node which will replace the old one
|
||||
void replace(program_node& old_node, program_node& new_node);
|
||||
|
||||
void init_program();
|
||||
};
|
||||
|
||||
} // namespace cldnn
|
||||
|
@ -8,6 +8,8 @@
|
||||
#include <unordered_map>
|
||||
#include <functional>
|
||||
#include <iostream>
|
||||
#include <thread>
|
||||
#include <mutex>
|
||||
|
||||
#include "kernel.hpp"
|
||||
|
||||
@ -30,15 +32,15 @@ public:
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Get the least recently used element object in the cache
|
||||
* @brief Get the least recently used element with key and value pair in the cache
|
||||
*
|
||||
* @return Value
|
||||
*/
|
||||
Value get_lru_element() const {
|
||||
std::pair<Key, Value> get_lru_element() const {
|
||||
if (_lru_data_list.size()) {
|
||||
return _lru_data_list.back().second;
|
||||
return _lru_data_list.back();
|
||||
} else {
|
||||
return Value();
|
||||
return std::make_pair(Key(), Value());
|
||||
}
|
||||
}
|
||||
|
||||
@ -164,6 +166,46 @@ private:
|
||||
}
|
||||
};
|
||||
|
||||
using ImplementationsCache = cldnn::LruCache<size_t, std::shared_ptr<primitive_impl>>;
|
||||
using KernelsCache = cldnn::LruCache<size_t, cldnn::kernel::ptr>;
|
||||
|
||||
template<typename Key, typename Value>
|
||||
class LruCacheThreadSafe : LruCache<Key, Value> {
|
||||
public:
|
||||
using parent = LruCache<Key, Value>;
|
||||
using FuncRemoveItem = std::function<void(std::pair<Key, Value>&)>;
|
||||
|
||||
explicit LruCacheThreadSafe(size_t caps) : parent(caps) { }
|
||||
|
||||
bool add(const Key& key, const Value& value) {
|
||||
std::lock_guard<std::mutex> lock(_mutex);
|
||||
auto popped_item = parent::get_lru_element();
|
||||
auto ret = parent::add(key, value);
|
||||
if (ret && _remove_popped_item) {
|
||||
_remove_popped_item(popped_item);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
bool has(const Key& key) const {
|
||||
std::lock_guard<std::mutex> lock(_mutex);
|
||||
return parent::has(key);
|
||||
}
|
||||
|
||||
Value get(const Key& key) {
|
||||
std::lock_guard<std::mutex> lock(_mutex);
|
||||
return parent::get(key);
|
||||
}
|
||||
|
||||
void set_remove_item_callback(FuncRemoveItem callback) {
|
||||
_remove_popped_item = callback;
|
||||
}
|
||||
|
||||
private:
|
||||
FuncRemoveItem _remove_popped_item;
|
||||
mutable std::mutex _mutex;
|
||||
};
|
||||
|
||||
|
||||
using ImplementationsCache = cldnn::LruCacheThreadSafe<size_t, std::shared_ptr<primitive_impl>>;
|
||||
|
||||
} // namespace cldnn
|
||||
|
@ -1,91 +1,76 @@
|
||||
// Copyright (C) 2022 Intel Corporation
|
||||
// Copyright (C) 2022-2023 Intel Corporation
|
||||
// SPDX-License-Identifier: Apache-2.0
|
||||
//
|
||||
|
||||
|
||||
#include "compilation_context.hpp"
|
||||
#include "threading/ie_thread_safe_containers.hpp"
|
||||
#include "kernel_selector/kernel_base.h"
|
||||
#include <mutex>
|
||||
#include <atomic>
|
||||
#include <unordered_set>
|
||||
#include "intel_gpu/runtime/utils.hpp"
|
||||
|
||||
namespace cldnn {
|
||||
class CompilationTaskQueue {
|
||||
using CompilationTaskData = std::pair<size_t, ICompilationContext::Task>;
|
||||
|
||||
public:
|
||||
void push_task(size_t task_key, ICompilationContext::Task&& task) {
|
||||
std::lock_guard<std::mutex> lock(_mutex);
|
||||
if (_queue_keymap.find(task_key) == _queue_keymap.end()) {
|
||||
auto insert_it = _queue.insert(_queue.end(), {task_key, task});
|
||||
_queue_keymap.insert({task_key, insert_it});
|
||||
}
|
||||
}
|
||||
|
||||
bool pop_front_task(size_t& task_key, ICompilationContext::Task& task) {
|
||||
std::lock_guard<std::mutex> lock(_mutex);
|
||||
if (!_queue.empty()) {
|
||||
auto front = _queue.front();
|
||||
task = front.second;
|
||||
task_key = front.first;
|
||||
_queue.pop_front();
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
void erase_task_key(size_t removed_key) {
|
||||
std::lock_guard<std::mutex> lock(_mutex);
|
||||
if (_queue_keymap.find(removed_key) != _queue_keymap.end()) {
|
||||
_queue_keymap.erase(removed_key);
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
std::deque<CompilationTaskData> _queue;
|
||||
std::unordered_map<size_t, std::deque<CompilationTaskData>::iterator> _queue_keymap;
|
||||
std::mutex _mutex;
|
||||
};
|
||||
|
||||
class CompilationContext : public ICompilationContext {
|
||||
public:
|
||||
CompilationContext(cldnn::engine& engine, const ExecutionConfig& config, size_t program_id) {
|
||||
_kernels_cache = cldnn::make_unique<kernels_cache>(engine, config, program_id, nullptr, kernel_selector::KernelBase::get_db().get_batch_header_str());
|
||||
_worker = std::thread([this](){
|
||||
while (!_stop_compilation) {
|
||||
CompilationContext::Task task;
|
||||
size_t task_key;
|
||||
bool success = _queue.pop_front_task(task_key, task);
|
||||
if (success) {
|
||||
task(*_kernels_cache);
|
||||
_queue.erase_task_key(task_key);
|
||||
} else {
|
||||
std::chrono::milliseconds ms{1};
|
||||
std::this_thread::sleep_for(ms);
|
||||
}
|
||||
}
|
||||
});
|
||||
CompilationContext(InferenceEngine::CPUStreamsExecutor::Config task_executor_config) : _task_executor_config(task_executor_config) {
|
||||
_task_executor_config._streams = 4;
|
||||
_task_executor = std::make_shared<InferenceEngine::CPUStreamsExecutor>(_task_executor_config);
|
||||
}
|
||||
|
||||
void push_task(size_t key, ICompilationContext::Task&& task) override {
|
||||
_queue.push_task(key, std::move(task));
|
||||
void push_task(size_t key, Task&& task) override {
|
||||
if (_stop_compilation)
|
||||
return;
|
||||
|
||||
std::lock_guard<std::mutex> lock(_mutex);
|
||||
if (_task_keys.find(key) == _task_keys.end()) {
|
||||
_task_keys.insert(key);
|
||||
if (_task_executor != nullptr)
|
||||
_task_executor->run(task);
|
||||
}
|
||||
}
|
||||
|
||||
void remove_keys(std::vector<size_t>&& keys) override {
|
||||
std::lock_guard<std::mutex> lock(_mutex);
|
||||
if (!_task_keys.empty()) {
|
||||
for (auto key : keys) {
|
||||
if (_task_keys.find(key) != _task_keys.end()) {
|
||||
_task_keys.erase(key);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
~CompilationContext() noexcept {
|
||||
cancel();
|
||||
}
|
||||
|
||||
bool is_stopped() override {
|
||||
return _stop_compilation;
|
||||
}
|
||||
|
||||
void cancel() noexcept override {
|
||||
if (_stop_compilation)
|
||||
return;
|
||||
|
||||
_stop_compilation = true;
|
||||
if (_worker.joinable())
|
||||
_worker.join();
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(_mutex);
|
||||
if (_task_executor != nullptr)
|
||||
_task_executor.reset();
|
||||
_task_keys.clear();
|
||||
}
|
||||
}
|
||||
|
||||
~CompilationContext() noexcept { cancel(); }
|
||||
|
||||
private:
|
||||
std::unique_ptr<kernels_cache> _kernels_cache;
|
||||
std::thread _worker;
|
||||
InferenceEngine::CPUStreamsExecutor::Config _task_executor_config;
|
||||
InferenceEngine::CPUStreamsExecutor::Ptr _task_executor;
|
||||
std::mutex _mutex;
|
||||
std::unordered_set<size_t> _task_keys;
|
||||
std::atomic_bool _stop_compilation{false};
|
||||
|
||||
CompilationTaskQueue _queue;
|
||||
};
|
||||
|
||||
std::unique_ptr<ICompilationContext> ICompilationContext::create(cldnn::engine& engine, const ExecutionConfig& config, size_t program_id) {
|
||||
return cldnn::make_unique<CompilationContext>(engine, config, program_id);
|
||||
std::unique_ptr<ICompilationContext> ICompilationContext::create(InferenceEngine::CPUStreamsExecutor::Config task_executor_config) {
|
||||
return cldnn::make_unique<CompilationContext>(task_executor_config);
|
||||
}
|
||||
|
||||
} // namespace cldnn
|
||||
|
@ -276,6 +276,19 @@ protected:
|
||||
(std::accumulate(gws.begin(), gws.end(), 1, std::multiplies<size_t>()) == 0);
|
||||
}
|
||||
}
|
||||
|
||||
void set_kernels(std::map<const std::string, kernel::ptr>& kernels) {
|
||||
if (is_cpu())
|
||||
return;
|
||||
|
||||
_kernel_ids.clear();
|
||||
_kernels.clear();
|
||||
_kernels.reserve(kernels.size());
|
||||
for (auto& k : kernels) {
|
||||
_kernel_ids.push_back(k.first);
|
||||
_kernels.emplace_back(std::move(k.second));
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace ocl
|
||||
|
@ -1,10 +1,10 @@
|
||||
// Copyright (C) 2022 Intel Corporation
|
||||
// Copyright (C) 2022-2023 Intel Corporation
|
||||
// SPDX-License-Identifier: Apache-2.0
|
||||
//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "kernels_cache.hpp"
|
||||
#include <threading/ie_cpu_streams_executor.hpp>
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
|
||||
@ -12,12 +12,14 @@ namespace cldnn {
|
||||
|
||||
class ICompilationContext {
|
||||
public:
|
||||
using Task = std::function<void(kernels_cache&)>;
|
||||
using Task = std::function<void()>;
|
||||
virtual void push_task(size_t key, Task&& task) = 0;
|
||||
virtual void cancel() noexcept = 0;
|
||||
virtual void remove_keys(std::vector<size_t>&& keys) = 0;
|
||||
virtual ~ICompilationContext() = default;
|
||||
virtual bool is_stopped() = 0;
|
||||
virtual void cancel() = 0;
|
||||
|
||||
static std::unique_ptr<ICompilationContext> create(cldnn::engine& engine, const ExecutionConfig& config, size_t program_id);
|
||||
static std::unique_ptr<ICompilationContext> create(InferenceEngine::CPUStreamsExecutor::Config task_executor_config);
|
||||
};
|
||||
|
||||
} // namespace cldnn
|
||||
|
@ -63,7 +63,6 @@ struct primitive_impl {
|
||||
}
|
||||
virtual std::vector<std::shared_ptr<cldnn::kernel_string>> get_kernels_source() { return {}; }
|
||||
virtual void reset_kernels_source() {}
|
||||
virtual void set_kernels(std::vector<kernel::ptr>) {}
|
||||
virtual std::vector<kernel::ptr> get_kernels() const { return {}; }
|
||||
virtual void set_kernel_ids(std::vector<kernel_id> kernel_ids) {}
|
||||
virtual void save(cldnn::BinaryOutputBuffer& ob) const {}
|
||||
@ -80,6 +79,8 @@ struct primitive_impl {
|
||||
OPENVINO_ASSERT(false, "[GPU] update_dispatch_data is not implemented for dynamic implemenation ", _kernel_name);
|
||||
}
|
||||
|
||||
virtual void set_kernels(std::map<const std::string, kernel::ptr>& kernels) {}
|
||||
|
||||
protected:
|
||||
std::string _kernel_name;
|
||||
bool _is_dynamic = false;
|
||||
|
@ -331,14 +331,7 @@ network::network(program::ptr program, const ExecutionConfig& config, stream::pt
|
||||
|
||||
if (is_dynamic()) {
|
||||
GPU_DEBUG_DEFINE_MEM_LOGGER("dynamic_network_initialization");
|
||||
_kernels_cache = std::unique_ptr<kernels_cache>(new kernels_cache(program->get_engine(),
|
||||
program->get_config(),
|
||||
program->get_id(),
|
||||
program->get_task_executor(),
|
||||
kernel_selector::KernelBase::get_db().get_batch_header_str()));
|
||||
_impls_cache = std::unique_ptr<ImplementationsCache>(new ImplementationsCache(_impls_cache_capacity));
|
||||
_in_mem_kernels_cache = std::unique_ptr<KernelsCache>(new KernelsCache(_in_mem_kernels_cache_capacity));
|
||||
_compilation_context = ICompilationContext::create(program->get_engine(), program->get_config(), program->get_id());
|
||||
}
|
||||
}
|
||||
|
||||
@ -471,8 +464,8 @@ network::network(cldnn::BinaryInputBuffer& ib, const ExecutionConfig& config, st
|
||||
}
|
||||
|
||||
network::~network() {
|
||||
if (_compilation_context)
|
||||
_compilation_context->cancel();
|
||||
if (_program != nullptr)
|
||||
_program->cancel_compilation_context();
|
||||
_memory_pool->clear_pool_for_network(net_id);
|
||||
GPU_DEBUG_GET_INSTANCE(debug_config);
|
||||
GPU_DEBUG_IF(!debug_config->dump_profiling_data.empty()) {
|
||||
|
@ -321,10 +321,9 @@ bool primitive_inst::update_impl() {
|
||||
// Update param if fake_alignment is available
|
||||
auto updated_params = _node->type()->get_fake_aligned_params(*_impl_params);
|
||||
auto impl_key = get_impl_key(updated_params);
|
||||
auto& cache = get_network().get_implementations_cache();
|
||||
auto& cache = get_network().get_program()->get_implementations_cache();
|
||||
bool has_cached_impl = false;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(get_network().get_impl_cache_mutex());
|
||||
has_cached_impl = cache.has(impl_key);
|
||||
if (has_cached_impl) {
|
||||
_impl = cache.get(impl_key)->clone();
|
||||
@ -337,11 +336,13 @@ bool primitive_inst::update_impl() {
|
||||
}
|
||||
if (!has_cached_impl) {
|
||||
if (_dynamic_impl) {
|
||||
auto& compilation_context = get_network().get_compilation_context();
|
||||
compilation_context.push_task(impl_key, [this, updated_params, impl_key](kernels_cache& kc) {
|
||||
auto& cache = get_network().get_implementations_cache();
|
||||
auto& compilation_context = get_network().get_program()->get_compilation_context();
|
||||
compilation_context.push_task(impl_key, [this, &compilation_context, updated_params, impl_key]() {
|
||||
if (compilation_context.is_stopped())
|
||||
return;
|
||||
auto _program = get_network().get_program();
|
||||
auto& cache = _program->get_implementations_cache();
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(get_network().get_impl_cache_mutex());
|
||||
// Check existense in the cache one more time as several iterations of model execution could happens and multiple compilation
|
||||
// tasks created for same shapes
|
||||
if (cache.has(impl_key))
|
||||
@ -349,13 +350,8 @@ bool primitive_inst::update_impl() {
|
||||
}
|
||||
|
||||
auto impl = _node->type()->choose_impl(*_node, updated_params);
|
||||
auto kernel_ids = kc.add_kernels_source(impl->get_kernels_source());
|
||||
impl->set_kernel_ids(kernel_ids);
|
||||
kc.compile();
|
||||
impl->init_kernels(kc);
|
||||
kc.reset();
|
||||
|
||||
std::lock_guard<std::mutex> lock(get_network().get_impl_cache_mutex());
|
||||
auto kernels = _program->get_kernels_cache().compile(impl->get_kernels_source());
|
||||
impl->set_kernels(kernels);
|
||||
cache.add(impl_key, impl->clone());
|
||||
});
|
||||
_impl = _dynamic_impl->clone();
|
||||
@ -364,13 +360,9 @@ bool primitive_inst::update_impl() {
|
||||
update_shape_info(*_impl_params);
|
||||
} else {
|
||||
_impl = _node->type()->choose_impl(*_node, updated_params);
|
||||
auto& kernels_cache = get_network().get_kernels_cache();
|
||||
auto kernel_ids = kernels_cache.add_kernels_source(_impl->get_kernels_source());
|
||||
_impl->set_kernel_ids(kernel_ids);
|
||||
kernels_cache.compile();
|
||||
_impl->init_kernels(kernels_cache);
|
||||
kernels_cache.reset();
|
||||
std::lock_guard<std::mutex> lock(get_network().get_impl_cache_mutex());
|
||||
auto& kernels_cache = get_network().get_program()->get_kernels_cache();
|
||||
auto kernels = kernels_cache.compile(_impl->get_kernels_source());
|
||||
_impl->set_kernels(kernels);
|
||||
cache.add(impl_key, _impl->clone());
|
||||
|
||||
auto new_impl_str = _impl != nullptr ? _impl->get_kernel_name() : "nullptr";
|
||||
@ -707,12 +699,11 @@ event::ptr primitive_inst::update_weights() {
|
||||
} else {
|
||||
GPU_DEBUG_TRACE_DETAIL << id() << ": reorder weights from " << original_layout.to_short_string()
|
||||
<< " to " << expected_layout.to_short_string() << std::endl;
|
||||
auto& kernels_cache = get_network().get_kernels_cache();
|
||||
auto kernel_id = kernels_cache.set_kernel_source(weights_params.clKernel->code.kernelString, false);
|
||||
kernels_cache.compile();
|
||||
kernel = kernels_cache.get_kernel(kernel_id);
|
||||
auto& kernels_cache = get_network().get_program()->get_kernels_cache();
|
||||
auto kernels = kernels_cache.compile({weights_params.clKernel->code.kernelString});
|
||||
OPENVINO_ASSERT(kernels.size() == 1, "The output of kernel compile has issue");
|
||||
kernel = kernels.begin()->second;
|
||||
cache.add(kernel_key, kernel);
|
||||
kernels_cache.reset();
|
||||
}
|
||||
|
||||
auto& stream = get_network().get_stream();
|
||||
|
@ -18,6 +18,7 @@
|
||||
#include "program_dump_graph.h"
|
||||
#include "sliding_window_utils.hpp"
|
||||
#include "program_helpers.h"
|
||||
#include "compilation_context.hpp"
|
||||
|
||||
#include "matrix_nms_inst.h"
|
||||
#include "roi_pooling_inst.h"
|
||||
@ -111,16 +112,9 @@ program::program(engine& engine_ref,
|
||||
is_subgroup_local_block_io_supported(-1) {
|
||||
_config.apply_user_properties(_engine.get_device_info());
|
||||
init_primitives();
|
||||
set_options();
|
||||
query_local_block_io_supported();
|
||||
_task_executor = make_task_executor(_config);
|
||||
|
||||
GPU_DEBUG_INFO << "Program config\n" << config.to_string();
|
||||
|
||||
pm = std::unique_ptr<pass_manager>(new pass_manager(*this));
|
||||
init_program();
|
||||
prepare_nodes(topology);
|
||||
_kernels_cache = std::unique_ptr<kernels_cache>(new kernels_cache(_engine, _config, prog_id, _task_executor,
|
||||
kernel_selector::KernelBase::get_db().get_batch_header_str()));
|
||||
program_node::reset_unique_id();
|
||||
|
||||
if (no_optimizations) {
|
||||
@ -144,14 +138,7 @@ program::program(engine& engine_ref,
|
||||
is_subgroup_local_block_io_supported(-1) {
|
||||
_config.apply_user_properties(_engine.get_device_info());
|
||||
init_primitives();
|
||||
set_options();
|
||||
query_local_block_io_supported();
|
||||
|
||||
_task_executor = make_task_executor(_config);
|
||||
|
||||
_kernels_cache = std::unique_ptr<kernels_cache>(new kernels_cache(_engine, _config, prog_id, _task_executor,
|
||||
kernel_selector::KernelBase::get_db().get_batch_header_str()));
|
||||
pm = std::unique_ptr<pass_manager>(new pass_manager(*this));
|
||||
init_program();
|
||||
prepare_nodes(nodes);
|
||||
build_program(is_internal);
|
||||
calc_nodes_hash();
|
||||
@ -165,10 +152,32 @@ program::program(engine& engine)
|
||||
is_subgroup_local_block_io_supported(-1) {
|
||||
_config.apply_user_properties(_engine.get_device_info());
|
||||
}
|
||||
|
||||
program::~program() {
|
||||
query_local_block_io_supported();
|
||||
}
|
||||
|
||||
void program::init_program() {
|
||||
set_options();
|
||||
query_local_block_io_supported();
|
||||
|
||||
pm = std::unique_ptr<pass_manager>(new pass_manager(*this));
|
||||
|
||||
_task_executor = make_task_executor(_config);
|
||||
_kernels_cache = std::unique_ptr<kernels_cache>(new kernels_cache(_engine, _config, prog_id, _task_executor,
|
||||
kernel_selector::KernelBase::get_db().get_batch_header_str()));
|
||||
|
||||
_compilation_context = ICompilationContext::create(make_task_executor_config(_config,
|
||||
"Task executor config for CompilationContext in GPU plugin"));
|
||||
|
||||
_impls_cache = cldnn::make_unique<ImplementationsCache>(_impls_cache_capacity);
|
||||
// Remove items of compilation context's internal queue when some impl is popped in kernels_cache
|
||||
// compilation context's queue check duplication of inserted task
|
||||
_impls_cache->set_remove_item_callback([this](std::pair<size_t, std::shared_ptr<cldnn::primitive_impl>>& item) {
|
||||
get_compilation_context().remove_keys({item.first});
|
||||
});
|
||||
}
|
||||
|
||||
void program::init_primitives() {
|
||||
static bool is_initialized = false;
|
||||
if (!is_initialized) {
|
||||
@ -202,8 +211,8 @@ static void adjust_num_cores(InferenceEngine::CPUStreamsExecutor::Config& config
|
||||
config._streams = std::min(config._streams, num_cores);
|
||||
}
|
||||
|
||||
std::shared_ptr<InferenceEngine::CPUStreamsExecutor> program::make_task_executor(const ExecutionConfig& config) const {
|
||||
InferenceEngine::CPUStreamsExecutor::Config task_executor_config("CPU Tasks executor for GPU plugin", 1);
|
||||
InferenceEngine::CPUStreamsExecutor::Config program::make_task_executor_config(const ExecutionConfig& config, std::string tags) const {
|
||||
InferenceEngine::CPUStreamsExecutor::Config task_executor_config(tags, 1);
|
||||
task_executor_config._streams = config.get_property(ov::compilation_num_threads);
|
||||
auto priority = config.get_property(ov::intel_gpu::hint::host_task_priority);
|
||||
switch (priority) {
|
||||
@ -215,6 +224,11 @@ std::shared_ptr<InferenceEngine::CPUStreamsExecutor> program::make_task_executor
|
||||
|
||||
adjust_num_cores(task_executor_config);
|
||||
|
||||
return task_executor_config;
|
||||
}
|
||||
|
||||
std::shared_ptr<InferenceEngine::CPUStreamsExecutor> program::make_task_executor(const ExecutionConfig& config) const {
|
||||
InferenceEngine::CPUStreamsExecutor::Config task_executor_config = make_task_executor_config(config, "CPU Tasks executor for GPU plugin");
|
||||
return std::make_shared<InferenceEngine::CPUStreamsExecutor>(task_executor_config);
|
||||
}
|
||||
|
||||
@ -1717,3 +1731,8 @@ std::pair<int64_t, int64_t> program::get_estimated_device_mem_usage() {
|
||||
void program::remove_kernel(kernel_id id) {
|
||||
_kernels_cache->remove_kernel(id);
|
||||
}
|
||||
|
||||
void program::cancel_compilation_context() {
|
||||
if (_compilation_context != nullptr)
|
||||
_compilation_context->cancel();
|
||||
}
|
||||
|
@ -54,7 +54,7 @@ std::string reorder_options(const std::string& org_options) {
|
||||
} // namespace
|
||||
|
||||
namespace cldnn {
|
||||
|
||||
std::atomic<size_t> kernels_cache::_kernel_idx{0};
|
||||
std::mutex kernels_cache::_mutex;
|
||||
|
||||
std::string kernels_cache::get_cache_path() const {
|
||||
@ -189,7 +189,7 @@ static std::vector<unsigned char> getProgramBinaries(cl::Program program) {
|
||||
}
|
||||
|
||||
// TODO: This build_batch method should be backend specific
|
||||
void kernels_cache::build_batch(const engine& build_engine, const batch_program& batch) {
|
||||
void kernels_cache::build_batch(const engine& build_engine, const batch_program& batch, std::map<const std::string, kernel::ptr>& compiled_kernels) {
|
||||
OV_ITT_SCOPED_TASK(ov::intel_gpu::itt::domains::intel_gpu_plugin, "KernelsCache::build_batch");
|
||||
|
||||
auto& cl_build_engine = dynamic_cast<const ocl::ocl_engine&>(build_engine);
|
||||
@ -286,7 +286,7 @@ void kernels_cache::build_batch(const engine& build_engine, const batch_program&
|
||||
cl_context context = cl_build_engine.get_cl_context().get();
|
||||
kernel::ptr kernel = kernels_factory::create(_engine, context, kern, entry_point);
|
||||
const auto& kmap = std::make_pair(k_id->second, kernel);
|
||||
_kernels.insert(kmap);
|
||||
compiled_kernels.insert(kmap);
|
||||
} else {
|
||||
throw std::runtime_error("Could not find entry point");
|
||||
}
|
||||
@ -391,7 +391,7 @@ void kernels_cache::build_all() {
|
||||
auto& batch = batches[idx];
|
||||
tasks.push_back([this, &_build_engine, &batch, &exception] {
|
||||
try {
|
||||
build_batch(_build_engine, batch);
|
||||
build_batch(_build_engine, batch, _kernels);
|
||||
} catch(...) {
|
||||
exception = std::current_exception();
|
||||
}
|
||||
@ -405,7 +405,7 @@ void kernels_cache::build_all() {
|
||||
}
|
||||
} else {
|
||||
for (size_t idx = 0; idx < batches.size(); idx++) {
|
||||
build_batch(_build_engine, batches[idx]);
|
||||
build_batch(_build_engine, batches[idx], _kernels);
|
||||
}
|
||||
}
|
||||
|
||||
@ -436,10 +436,7 @@ std::vector<kernel_id> kernels_cache::add_kernels_source(std::vector<std::shared
|
||||
for (size_t i = 0; i < kernel_sources.size(); ++i) {
|
||||
std::lock_guard<std::mutex> lock(_mutex);
|
||||
auto kernel_string = kernel_sources[i];
|
||||
// we need unique id in order to avoid conflict across topologies.
|
||||
const auto kernel_num = _kernels.size() + (_kernel_idx++);
|
||||
kernel_id id = kernel_string->entry_point + "_" + std::to_string(kernel_num);
|
||||
|
||||
kernel_id id = gen_kernel_id(kernel_string->entry_point);
|
||||
auto res = _kernels_code.emplace(kernel_string, id, dump_custom_program);
|
||||
|
||||
assert(_kernels.find(id) == _kernels.end());
|
||||
@ -457,37 +454,10 @@ void kernels_cache::add_kernels(const std::vector<std::string>& kernel_ids, cons
|
||||
for (size_t i = 0; i < kernel_ids.size(); i++) {
|
||||
const auto& kmap = std::make_pair(kernel_ids[i], kernels[i]);
|
||||
_kernels.insert(kmap);
|
||||
_kernel_idx++;
|
||||
}
|
||||
}
|
||||
|
||||
void kernels_cache::compile() {
|
||||
OV_ITT_SCOPED_TASK(ov::intel_gpu::itt::domains::intel_gpu_plugin, "KernelsCache::BuildAll");
|
||||
|
||||
std::unique_ptr<ocl::ocl_engine> _build_engine = nullptr;
|
||||
if (_engine.type() == engine_types::ocl) {
|
||||
_build_engine = std::unique_ptr<ocl::ocl_engine>(new ocl::ocl_engine(_engine.get_device(), runtime_types::ocl));
|
||||
}
|
||||
|
||||
// create batches
|
||||
std::vector<batch_program> batches;
|
||||
get_program_source(_kernels_code, &batches);
|
||||
|
||||
// build batches
|
||||
for (size_t idx = 0; idx < batches.size(); idx++) {
|
||||
build_batch(*_build_engine, batches[idx]);
|
||||
}
|
||||
|
||||
_kernels_code.clear();
|
||||
_pending_compilation = false;
|
||||
#if defined(__unix__) && !defined(__ANDROID__)
|
||||
// NOTE: In linux, without malloc_trim, an amount of the memory used by compilation is not being returned to system thought they are freed.
|
||||
// (It is at least 500 MB when we perform parallel compilation)
|
||||
// It is observed that freeing the memory manually with malloc_trim saves significant amount of the memory.
|
||||
// Also, this is not happening in Windows.
|
||||
// So, added malloc_trim for linux build until we figure out a better solution.
|
||||
malloc_trim(0);
|
||||
#endif
|
||||
}
|
||||
void kernels_cache::save(BinaryOutputBuffer& ob) const {
|
||||
OPENVINO_ASSERT(_engine.type() == engine_types::ocl, "[GPU] Not supported engine type");
|
||||
|
||||
@ -570,6 +540,7 @@ void kernels_cache::load(BinaryInputBuffer& ib) {
|
||||
cl_context cl_context = build_engine->get_cl_context().get();
|
||||
kernel::ptr kernel = kernels_factory::create(_engine, cl_context, cl_kernel, entry_point);
|
||||
_kernels.insert({k_id->second, kernel});
|
||||
_kernel_idx++;
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -582,4 +553,41 @@ void kernels_cache::load(BinaryInputBuffer& ib) {
|
||||
}
|
||||
}
|
||||
|
||||
std::map<const std::string, kernel::ptr> kernels_cache::compile(std::vector<std::shared_ptr<kernel_string>> kernel_sources,
|
||||
bool dump_custom_program) {
|
||||
OV_ITT_SCOPED_TASK(ov::intel_gpu::itt::domains::intel_gpu_plugin, "KernelsCache::Compile_ThreadSafe");
|
||||
kernels_code t_kernels_code;
|
||||
|
||||
// Get kernels code from kernel sources
|
||||
for (size_t idx = 0; idx < kernel_sources.size(); ++idx) {
|
||||
auto kernel_string = kernel_sources[idx];
|
||||
kernel_id id = gen_kernel_id(kernel_string->entry_point);
|
||||
t_kernels_code.emplace(kernel_string, id, dump_custom_program);
|
||||
}
|
||||
|
||||
ocl::ocl_engine& _build_engine = downcast<ocl::ocl_engine>(_engine);
|
||||
|
||||
// Create batches
|
||||
std::vector<batch_program> batches;
|
||||
get_program_source(t_kernels_code, &batches);
|
||||
|
||||
std::map<const std::string, kernel::ptr> output_kernels;
|
||||
// Build batches
|
||||
for (size_t idx = 0; idx < batches.size(); ++idx) {
|
||||
build_batch(_build_engine, batches[idx], output_kernels);
|
||||
}
|
||||
|
||||
t_kernels_code.clear();
|
||||
#if defined(__unix__) && !defined(__ANDROID__)
|
||||
// NOTE: In linux, without malloc_trim, an amount of the memory used by compilation is not being returned to system thought they are freed.
|
||||
// (It is at least 500 MB when we perform parallel compilation)
|
||||
// It is observed that freeing the memory manually with malloc_trim saves significant amount of the memory.
|
||||
// Also, this is not happening in Windows.
|
||||
// So, added malloc_trim for linux build until we figure out a better solution.
|
||||
malloc_trim(0);
|
||||
#endif
|
||||
|
||||
return output_kernels;
|
||||
}
|
||||
|
||||
} // namespace cldnn
|
||||
|
@ -22,6 +22,7 @@
|
||||
#include "ocl/ocl_engine.hpp"
|
||||
|
||||
namespace cldnn {
|
||||
|
||||
class kernels_cache {
|
||||
public:
|
||||
using source_code = std::vector<std::string>;
|
||||
@ -81,18 +82,23 @@ private:
|
||||
ExecutionConfig _config;
|
||||
uint32_t _prog_id = 0;
|
||||
kernels_code _kernels_code;
|
||||
size_t _kernel_idx = 0;
|
||||
static std::atomic<size_t> _kernel_idx;
|
||||
std::atomic<bool> _pending_compilation{false};
|
||||
std::map<const std::string, kernel::ptr> _kernels;
|
||||
std::vector<std::string> batch_header_str;
|
||||
|
||||
void get_program_source(const kernels_code& kernels_source_code, std::vector<batch_program>*) const;
|
||||
void build_batch(const engine& build_engine, const batch_program& batch);
|
||||
void build_batch(const engine& build_engine, const batch_program& batch, std::map<const std::string, kernel::ptr>& compiled_kernels);
|
||||
|
||||
std::string get_cache_path() const;
|
||||
bool is_cache_enabled() const;
|
||||
size_t get_max_kernels_per_batch() const;
|
||||
|
||||
inline std::string gen_kernel_id(std::string entry_point) {
|
||||
// we need unique id in order to avoid conflict across topologies.
|
||||
return entry_point + "_" + std::to_string((_kernel_idx++));
|
||||
}
|
||||
|
||||
public:
|
||||
explicit kernels_cache(engine& engine,
|
||||
const ExecutionConfig& config,
|
||||
@ -116,9 +122,9 @@ public:
|
||||
}
|
||||
std::vector<kernel_id> add_kernels_source(std::vector<std::shared_ptr<kernel_string>> kernel_sources, bool dump_custom_program = false);
|
||||
void add_kernels(const std::vector<std::string>& kernel_ids, const std::vector<kernel::ptr>& kernels);
|
||||
void compile();
|
||||
void save(BinaryOutputBuffer& ob) const;
|
||||
void load(BinaryInputBuffer& ib);
|
||||
std::map<const std::string, kernel::ptr> compile(std::vector<std::shared_ptr<kernel_string>> kernel_sources, bool dump_custom_program = false);
|
||||
};
|
||||
|
||||
} // namespace cldnn
|
||||
|
@ -23,7 +23,7 @@ TEST(lru_cache, basic_data_type)
|
||||
input_values.push_back(std::make_pair(i, i + 10));
|
||||
}
|
||||
|
||||
ASSERT_EQ(ca.get_lru_element(), int());
|
||||
ASSERT_EQ(ca.get_lru_element().second, int());
|
||||
|
||||
std::vector<bool> expected_hitted = {false, false, false, false, true, true, false};
|
||||
for (size_t i = 0; i < input_values.size(); i++) {
|
||||
@ -95,7 +95,7 @@ TEST(lru_cache, custom_data_type) {
|
||||
|
||||
std::vector<bool> expected_hitted = {false, false, false, false, true, true, true, false};
|
||||
|
||||
ASSERT_EQ(ca.get_lru_element(), std::shared_ptr<lru_cache_test_data>());
|
||||
ASSERT_EQ(ca.get_lru_element().second, std::shared_ptr<lru_cache_test_data>());
|
||||
for (size_t i = 0; i < inputs.size(); i++) {
|
||||
auto& in = inputs[i];
|
||||
std::shared_ptr<lru_cache_test_data> p_data;
|
||||
|
Loading…
Reference in New Issue
Block a user