Browse Source

modify ge hybrid

tags/v0.7.0-beta
zhangzhenghai 4 years ago
parent
commit
6b3508d9ee
61 changed files with 1370 additions and 3685 deletions
  1. +5
    -21
      src/ge/hybrid/common/npu_memory_allocator.cc
  2. +1
    -20
      src/ge/hybrid/common/npu_memory_allocator.h
  3. +2
    -2
      src/ge/hybrid/common/tensor_value.cc
  4. +1
    -3
      src/ge/hybrid/common/tensor_value.h
  5. +30
    -1
      src/ge/hybrid/executor/hybrid_execution_context.cc
  6. +17
    -9
      src/ge/hybrid/executor/hybrid_execution_context.h
  7. +32
    -75
      src/ge/hybrid/executor/hybrid_model_async_executor.cc
  8. +3
    -5
      src/ge/hybrid/executor/hybrid_model_async_executor.h
  9. +85
    -18
      src/ge/hybrid/executor/hybrid_model_executor.cc
  10. +13
    -4
      src/ge/hybrid/executor/hybrid_model_executor.h
  11. +2
    -1
      src/ge/hybrid/executor/hybrid_profiler.cc
  12. +8
    -33
      src/ge/hybrid/executor/node_done_manager.cc
  13. +2
    -3
      src/ge/hybrid/executor/node_done_manager.h
  14. +3
    -123
      src/ge/hybrid/executor/node_state.cc
  15. +16
    -61
      src/ge/hybrid/executor/node_state.h
  16. +1
    -0
      src/ge/hybrid/executor/rt_callback_manager.cc
  17. +0
    -112
      src/ge/hybrid/executor/subgraph_context.cc
  18. +0
    -61
      src/ge/hybrid/executor/subgraph_context.h
  19. +0
    -373
      src/ge/hybrid/executor/subgraph_executor.cc
  20. +0
    -101
      src/ge/hybrid/executor/subgraph_executor.h
  21. +76
    -118
      src/ge/hybrid/executor/worker/execution_engine.cc
  22. +15
    -6
      src/ge/hybrid/executor/worker/execution_engine.h
  23. +177
    -48
      src/ge/hybrid/executor/worker/shape_inference_engine.cc
  24. +58
    -8
      src/ge/hybrid/executor/worker/shape_inference_engine.h
  25. +160
    -10
      src/ge/hybrid/executor/worker/task_compile_engine.cc
  26. +32
    -1
      src/ge/hybrid/executor/worker/task_compile_engine.h
  27. +1
    -12
      src/ge/hybrid/hybrid_davinci_model.cc
  28. +0
    -2
      src/ge/hybrid/hybrid_davinci_model.h
  29. +0
    -2
      src/ge/hybrid/hybrid_davinci_model_stub.cc
  30. +0
    -62
      src/ge/hybrid/model/graph_item.cc
  31. +0
    -64
      src/ge/hybrid/model/graph_item.h
  32. +28
    -33
      src/ge/hybrid/model/hybrid_model.cc
  33. +35
    -13
      src/ge/hybrid/model/hybrid_model.h
  34. +154
    -422
      src/ge/hybrid/model/hybrid_model_builder.cc
  35. +8
    -14
      src/ge/hybrid/model/hybrid_model_builder.h
  36. +3
    -40
      src/ge/hybrid/model/node_item.cc
  37. +3
    -14
      src/ge/hybrid/model/node_item.h
  38. +202
    -89
      src/ge/hybrid/node_executor/aicore/aicore_node_executor.cc
  39. +11
    -5
      src/ge/hybrid/node_executor/aicore/aicore_node_executor.h
  40. +5
    -291
      src/ge/hybrid/node_executor/aicore/aicore_op_task.cc
  41. +4
    -46
      src/ge/hybrid/node_executor/aicore/aicore_op_task.h
  42. +54
    -56
      src/ge/hybrid/node_executor/aicore/aicore_task_builder.cc
  43. +11
    -10
      src/ge/hybrid/node_executor/aicore/aicore_task_builder.h
  44. +7
    -4
      src/ge/hybrid/node_executor/aicore/aicore_task_compiler.cc
  45. +1
    -0
      src/ge/hybrid/node_executor/aicpu/aicpu_ext_info.cc
  46. +1
    -0
      src/ge/hybrid/node_executor/aicpu/aicpu_ext_info.h
  47. +27
    -52
      src/ge/hybrid/node_executor/aicpu/aicpu_node_executor.cc
  48. +2
    -2
      src/ge/hybrid/node_executor/aicpu/aicpu_node_executor.h
  49. +4
    -5
      src/ge/hybrid/node_executor/compiledsubgraph/known_node_executor.cc
  50. +0
    -318
      src/ge/hybrid/node_executor/controlop/control_op_executor.cc
  51. +0
    -100
      src/ge/hybrid/node_executor/controlop/control_op_executor.h
  52. +0
    -207
      src/ge/hybrid/node_executor/hccl/hccl_node_executor.cc
  53. +0
    -59
      src/ge/hybrid/node_executor/hccl/hccl_node_executor.h
  54. +4
    -27
      src/ge/hybrid/node_executor/hostcpu/ge_local_node_executor.cc
  55. +1
    -12
      src/ge/hybrid/node_executor/hostcpu/ge_local_node_executor.h
  56. +17
    -103
      src/ge/hybrid/node_executor/node_executor.cc
  57. +9
    -121
      src/ge/hybrid/node_executor/node_executor.h
  58. +0
    -81
      src/ge/hybrid/node_executor/partitioned_call/partitioned_call_node_executor.cc
  59. +0
    -54
      src/ge/hybrid/node_executor/partitioned_call/partitioned_call_node_executor.h
  60. +30
    -114
      src/ge/hybrid/node_executor/task_context.cc
  61. +9
    -34
      src/ge/hybrid/node_executor/task_context.h

+ 5
- 21
src/ge/hybrid/common/npu_memory_allocator.cc View File

@@ -25,11 +25,6 @@ namespace hybrid {
std::map<uint32_t, std::unique_ptr<NpuMemoryAllocator>> NpuMemoryAllocator::allocators_;
std::mutex NpuMemoryAllocator::mu_;

AllocationAttr::AllocationAttr(int padding, void *try_reuse_addr)
: padding_(padding), try_reuse_addr_(try_reuse_addr) {}
AllocationAttr::AllocationAttr(int padding) : AllocationAttr(padding, nullptr) {}
AllocationAttr::AllocationAttr(void *try_reuse_addr) : AllocationAttr(0, try_reuse_addr) {}

NpuMemoryAllocator *NpuMemoryAllocator::GetAllocator() {
int32_t device_id = 0;
if (rtGetDevice(&device_id) != RT_ERROR_NONE) {
@@ -43,26 +38,15 @@ NpuMemoryAllocator *NpuMemoryAllocator::GetAllocator() {

NpuMemoryAllocator::NpuMemoryAllocator(uint32_t device_id) : device_id_(device_id) {}

void *NpuMemoryAllocator::Allocate(std::size_t size, AllocationAttr *attr) {
void *try_reuse_addr = nullptr;
size_t allocate_size = size;
if (attr != nullptr) {
try_reuse_addr = attr->try_reuse_addr_;
if (attr->padding_ != 0) {
// padding up to multiple of attr->padding, and add extra attr->padding_
allocate_size = (size + 2 * attr->padding_ - 1) / attr->padding_ * attr->padding_;
GELOGD("Padding size %ld by %d. final size = %zu.", size, attr->padding_, allocate_size);
}
}

void *buffer = MemManager::CachingInstance(RT_MEMORY_HBM)
.Malloc(allocate_size, reinterpret_cast<uint8_t *>(try_reuse_addr), device_id_);
void *NpuMemoryAllocator::Allocate(std::size_t size, void *try_reuse_addr) {
void *buffer =
MemManager::CachingInstance(RT_MEMORY_HBM).Malloc(size, reinterpret_cast<uint8_t *>(try_reuse_addr), device_id_);
if (buffer == nullptr) {
GELOGE(MEMALLOC_FAILED, "Failed to malloc memory, device_id = %u, size = %zu", device_id_, allocate_size);
GELOGE(MEMALLOC_FAILED, "Failed to malloc memory, device_id = %u, size = %zu", device_id_, size);
return nullptr;
}

GELOGI("Allocating buffer of size %zu successfully. device_id = %u, address = %p", allocate_size, device_id_, buffer);
GELOGI("Allocating buffer of size %u successfully. device_id = %u, address = %p", size, device_id_, buffer);
return buffer;
}



+ 1
- 20
src/ge/hybrid/common/npu_memory_allocator.h View File

@@ -26,35 +26,16 @@

namespace ge {
namespace hybrid {
class AllocationAttr {
public:
explicit AllocationAttr(int padding);
explicit AllocationAttr(void *try_reuse_addr);
AllocationAttr(int padding, void *try_reuse_addr);
~AllocationAttr() = default;

private:
friend class NpuMemoryAllocator;
int padding_ = 0;
void *try_reuse_addr_ = nullptr;
};

class NpuMemoryAllocator {
public:
~NpuMemoryAllocator() = default;
static NpuMemoryAllocator *GetAllocator(uint32_t device_id);
static NpuMemoryAllocator *GetAllocator();
static void DestroyAllocator();
static AllocationAttr *AttrWithDefaultPadding() {
static AllocationAttr attr(kDefaultPadding, nullptr);
return &attr;
}

void *Allocate(std::size_t size, AllocationAttr *attr = nullptr);
void *Allocate(std::size_t size, void *try_reuse_addr = nullptr);
void Deallocate(void *data);

static constexpr int kDefaultPadding = 32;

private:
explicit NpuMemoryAllocator(uint32_t device_id);
uint32_t device_id_;


+ 2
- 2
src/ge/hybrid/common/tensor_value.cc View File

@@ -24,7 +24,7 @@ namespace hybrid {
TensorBuffer::TensorBuffer(NpuMemoryAllocator *allocator, void *buffer, size_t size)
: allocator_(allocator), buffer_(buffer), size_(size) {}

std::unique_ptr<TensorBuffer> TensorBuffer::Create(NpuMemoryAllocator *allocator, size_t size, AllocationAttr *attr) {
std::unique_ptr<TensorBuffer> TensorBuffer::Create(NpuMemoryAllocator *allocator, size_t size) {
void *buffer = nullptr;
if (size == 0) {
GELOGD("size is 0");
@@ -36,7 +36,7 @@ std::unique_ptr<TensorBuffer> TensorBuffer::Create(NpuMemoryAllocator *allocator
return nullptr;
}

buffer = allocator->Allocate(size, attr);
buffer = allocator->Allocate(size);
if (buffer == nullptr) {
GELOGE(MEMALLOC_FAILED, "Failed to allocate memory. size = %zu", size);
return nullptr;


+ 1
- 3
src/ge/hybrid/common/tensor_value.h View File

@@ -24,12 +24,10 @@
namespace ge {
namespace hybrid {
class NpuMemoryAllocator;
class AllocationAttr;

class TensorBuffer {
public:
static std::unique_ptr<TensorBuffer> Create(NpuMemoryAllocator *allocator, size_t size,
AllocationAttr *attr = nullptr);
static std::unique_ptr<TensorBuffer> Create(NpuMemoryAllocator *allocator, size_t size);

static std::unique_ptr<TensorBuffer> Create(void *buffer, size_t size);



+ 30
- 1
src/ge/hybrid/executor/hybrid_execution_context.cc View File

@@ -17,5 +17,34 @@
#include "hybrid_execution_context.h"

namespace ge {
namespace hybrid {} // namespace hybrid
namespace hybrid {
NodeStatePtr GraphExecutionContext::GetOrCreateNodeState(const NodePtr &node) {
auto &node_state = node_states[node];
if (node_state == nullptr) {
const NodeItem *node_item = model->GetNodeItem(node);
if (node_item == nullptr) {
return nullptr;
}
node_state.reset(new (std::nothrow) NodeState(*node_item));
}

return node_state;
}

void GraphExecutionContext::OnError(Status error_code) {
GELOGE(error_code, "Error occurred while executing model");
{
std::lock_guard<std::mutex> lk(mu_);
this->status = error_code;
}

compile_queue.Stop();
execution_queue.Stop();
}

Status GraphExecutionContext::GetStatus() {
std::lock_guard<std::mutex> lk(mu_);
return status;
}
} // namespace hybrid
} // namespace ge

+ 17
- 9
src/ge/hybrid/executor/hybrid_execution_context.h View File

@@ -20,7 +20,6 @@
#include <atomic>
#include <unordered_map>
#include "common/blocking_queue.h"
#include "framework/common/debug/ge_log.h"
#include "hybrid/common/npu_memory_allocator.h"
#include "hybrid/common/tensor_value.h"
#include "hybrid/executor/hybrid_profiler.h"
@@ -34,26 +33,34 @@ namespace hybrid {
struct GraphExecutionContext {
uint64_t session_id = 0;
const HybridModel *model = nullptr;
NodeDoneManager cv_manager;
BlockingQueue<NodeStatePtr> compile_queue;
BlockingQueue<NodeStatePtr> execution_queue;
std::vector<TensorValue> all_inputs;
std::vector<TensorValue> all_outputs;
std::unordered_map<NodePtr, NodeStatePtr> node_states;
rtStream_t stream = nullptr;
rtContext_t rt_context = nullptr;
rtContext_t rt_gen_context = nullptr;
std::unique_ptr<CallbackManager> callback_manager;
NpuMemoryAllocator *allocator = nullptr;
mutable std::unique_ptr<HybridProfiler> profiler;
bool trace_enabled = false;
long profiling_level = 0;
int profiling_level = 0;
bool dump_enabled = false;
long iteration = 0;
Status status = SUCCESS;
std::mutex mu_;

NodeStatePtr GetOrCreateNodeState(const NodePtr &node);
void OnError(Status status);
Status GetStatus();
};

#define RECORD_PROFILING_EVENT(context, evt_type, fmt, category, node_name, ...) \
#define RECORD_PROFILING_EVENT(context, event_type, fmt, category, node_name, ...) \
do { \
if ((context)->profiler != nullptr) { \
if (node_name != nullptr) { \
context->profiler->RecordEvent(evt_type, "tid:%lu [%s] [%s] " fmt, GetTid(), node_name, category, \
##__VA_ARGS__); \
context->profiler->RecordEvent(event_type, "[%s] [%s] " fmt, node_name, category, ##__VA_ARGS__); \
} else { \
context->profiler->RecordEvent(evt_type, "tid:%lu [%s] " fmt, GetTid(), category, ##__VA_ARGS__); \
context->profiler->RecordEvent(event_type, "[%s] " fmt, category, ##__VA_ARGS__); \
} \
} \
} while (0)
@@ -72,6 +79,7 @@ struct GraphExecutionContext {

#define RECORD_CALLBACK_EVENT(context, name, fmt, ...) \
RECORD_PROFILING_EVENT((context), HybridProfiler::CALLBACK, fmt, "Callback", name, ##__VA_ARGS__)

} // namespace hybrid
} // namespace ge
#endif // GE_HYBRID_EXECUTOR_HYBRID_EXECUTION_CONTEXT_H_

+ 32
- 75
src/ge/hybrid/executor/hybrid_model_async_executor.cc View File

@@ -77,18 +77,19 @@ Status HybridModelAsyncExecutor::Init() {
GE_CHECK_NOTNULL(data_inputer_);
GE_CHK_RT_RET(rtStreamCreate(&stream_, RT_STREAM_PRIORITY_DEFAULT));

executor_ = std::unique_ptr<HybridModelExecutor>(new (std::nothrow) HybridModelExecutor(model_, device_id_, stream_));
GE_CHECK_NOTNULL(executor_);
GE_CHK_STATUS_RET(executor_->Init(), "Failed to init hybrid engine");
engine_ = std::unique_ptr<HybridModelExecutor>(new (std::nothrow) HybridModelExecutor(model_, device_id_, stream_));
GE_CHECK_NOTNULL(engine_);
GE_CHK_STATUS_RET(engine_->Init(), "Failed to init hybrid engine");

GE_CHK_STATUS_RET(InitInputTensors(), "Failed to init input tensors");
return SUCCESS;
}

Status HybridModelAsyncExecutor::PreRun(InputData &current_data) {
GE_CHK_STATUS_RET(SyncVarData(), "Failed to sync var data");
RECORD_MODEL_EXECUTION_EVENT(executor_->GetContext(), "[SyncVarData] End");
RECORD_MODEL_EXECUTION_EVENT(engine_->GetContext(), "[SyncVarData] End");
GE_CHK_STATUS_RET(CopyInputData(current_data), "Failed to copy input data to model");
RECORD_MODEL_EXECUTION_EVENT(executor_->GetContext(), "[CopyInputData] End");
RECORD_MODEL_EXECUTION_EVENT(engine_->GetContext(), "[CopyInputData] End");
return SUCCESS;
}

@@ -118,21 +119,21 @@ Status HybridModelAsyncExecutor::RunInternal() {
args.inputs[it.first] = it.second;
}

RECORD_MODEL_EXECUTION_EVENT(executor_->GetContext(), "[RunInternal] [iteration = %d] Start", iterator_count_);
RECORD_MODEL_EXECUTION_EVENT(engine_->GetContext(), "[RunInternal] [iteration = %d] Start", iterator_count_);
ret = PreRun(current_data);
GE_CHK_BOOL_TRUE_EXEC_WITH_LOG(
ret != SUCCESS, (void)HandleResult(ret, current_data.index, args, data_wrapper->GetOutput());
ret != SUCCESS, (void)HandleResult(ret, current_data.index, args.outputs, data_wrapper->GetOutput());
CsaInteract::GetInstance().StoreInternalErrorCode(ret, ERROR_MODULE_FMK, JOBSUBSTATE_GRAPH_EXEC);
continue, "PreRun failed."); // [No need to check value]

ret = executor_->Execute(args);
ret = HandleResult(ret, current_data.index, args, data_wrapper->GetOutput());
ret = engine_->Execute(args);
ret = HandleResult(ret, current_data.index, args.outputs, data_wrapper->GetOutput());
if (ret != SUCCESS) {
CsaInteract::GetInstance().StoreInternalErrorCode(ret, ERROR_MODULE_RUNTIME, JOBSUBSTATE_GRAPH_EXEC);
continue;
}

RECORD_MODEL_EXECUTION_EVENT(executor_->GetContext(), "[RunInternal] [iteration = %d] End", iterator_count_);
RECORD_MODEL_EXECUTION_EVENT(engine_->GetContext(), "[RunInternal] [iteration = %d] End", iterator_count_);
iterator_count_++;
GELOGI("run iterator count is %lu", iterator_count_);
}
@@ -142,8 +143,8 @@ Status HybridModelAsyncExecutor::RunInternal() {
return SUCCESS;
}

Status HybridModelAsyncExecutor::HandleResult(Status exec_ret, uint32_t data_id, HybridModelExecutor::ExecuteArgs &args,
OutputData *output_data) {
Status HybridModelAsyncExecutor::HandleResult(Status exec_ret, uint32_t data_id,
const std::vector<TensorValue> &output_tensors, OutputData *output_data) {
GELOGD("Start to handle result. model id = %u, data index = %u, execution ret = %u", model_id_, data_id, exec_ret);
std::vector<ge::OutputTensorInfo> output_tensor_info_list;
if (exec_ret == END_OF_SEQUENCE) {
@@ -157,7 +158,7 @@ Status HybridModelAsyncExecutor::HandleResult(Status exec_ret, uint32_t data_id,
}

GE_CHECK_NOTNULL(output_data);
auto ret = CopyOutputs(args, output_data, output_tensor_info_list);
auto ret = CopyOutputs(output_tensors, output_data, output_tensor_info_list);
if (ret != SUCCESS) {
OnComputeDone(data_id, INTERNAL_ERROR, output_tensor_info_list);
return INTERNAL_ERROR;
@@ -214,8 +215,9 @@ Status HybridModelAsyncExecutor::CopyInputData(const InputData &current_data) {
Status HybridModelAsyncExecutor::InitInputTensors() {
auto allocator = NpuMemoryAllocator::GetAllocator(device_id_);
GE_CHECK_NOTNULL(allocator);
int input_index = 0;
for (const auto &input_node : model_->GetRootGraphItem()->GetInputNodes()) {
for (const auto &it : model_->input_nodes_) {
auto input_index = it.first;
auto input_node = it.second;
GELOGD("Init input[%u], node = %s", input_index, input_node->NodeName().c_str());
auto output_desc = input_node->op_desc->GetOutputDescPtr(kDataOutputIndex);
GE_CHECK_NOTNULL(output_desc);
@@ -233,7 +235,6 @@ Status HybridModelAsyncExecutor::InitInputTensors() {
TensorValue tensor(shared_ptr<TensorBuffer>(buffer.release()));
tensor.SetName("Input_" + input_node->NodeName());
input_tensors_.emplace(input_index, tensor);
input_index += 1;
}

return SUCCESS;
@@ -249,33 +250,35 @@ Status HybridModelAsyncExecutor::OnComputeDone(uint32_t data_index, uint32_t res
return result_code;
}

Status HybridModelAsyncExecutor::CopyOutputs(HybridModelExecutor::ExecuteArgs &args, OutputData *output_data,
Status HybridModelAsyncExecutor::CopyOutputs(const std::vector<TensorValue> &output_tensors, OutputData *output_data,
std::vector<ge::OutputTensorInfo> &outputs) {
// copy output data from op to designated position
std::vector<ConstGeTensorDescPtr> &output_tensor_desc_list = args.output_desc;
std::vector<TensorValue> &output_tensors = args.outputs;
if (output_tensor_desc_list.size() != output_tensors.size()) {
NodeItem *net_output_node = model_->net_output_node_;
GE_CHECK_NOTNULL(net_output_node);
auto all_input_desc = net_output_node->op_desc->GetAllInputsDescPtr();

if (all_input_desc.size() != output_tensors.size()) {
GELOGE(INTERNAL_ERROR, "Output sizes mismatch. From op_desc = %zu, and from output tensors = %zu",
output_tensor_desc_list.size(), output_tensors.size());
all_input_desc.size(), output_tensors.size());
return INTERNAL_ERROR;
}

GELOGD("Number of outputs = %zu", output_tensor_desc_list.size());
GELOGD("Number of outputs = %zu", all_input_desc.size());
for (size_t i = 0; i < output_tensors.size(); ++i) {
GELOGD("Start to process output[%zu]", i);
auto &output_tensor = output_tensors[i];
auto &tensor_desc = output_tensor_desc_list.at(i);
auto &tensor_desc = all_input_desc.at(i);
GE_CHECK_NOTNULL(tensor_desc);
int64_t output_size = -1;
GE_CHK_GRAPH_STATUS_RET(TensorUtils::CalcTensorMemSize(tensor_desc->GetShape(), tensor_desc->GetFormat(),
GE_CHK_GRAPH_STATUS_RET(TensorUtils::CalcTensorMemSize(tensor_desc->MutableShape(), tensor_desc->GetFormat(),
tensor_desc->GetDataType(), output_size),
"Failed to calc tensor size for output[%zu]. shape = [%s], type = %s, format = %s", i,
tensor_desc->GetShape().ToString().c_str(),
tensor_desc->MutableShape().ToString().c_str(),
TypeUtils::DataTypeToSerialString(tensor_desc->GetDataType()).c_str(),
TypeUtils::FormatToSerialString(tensor_desc->GetFormat()).c_str());

GELOGD("Got tensor size for output[%zu] successfully. shape = [%s], type = %s, format = %s, size = %ld", i,
tensor_desc->GetShape().ToString().c_str(),
tensor_desc->MutableShape().ToString().c_str(),
TypeUtils::DataTypeToSerialString(tensor_desc->GetDataType()).c_str(),
TypeUtils::FormatToSerialString(tensor_desc->GetFormat()).c_str(), output_size);

@@ -283,7 +286,7 @@ Status HybridModelAsyncExecutor::CopyOutputs(HybridModelExecutor::ExecuteArgs &a
GE_CHECK_LE(output_size, UINT32_MAX);
if (output_tensor.GetSize() < static_cast<size_t>(output_size)) {
GELOGE(INTERNAL_ERROR, "output[%zu] tensor size(%zu) is not enough for output shape [%s]", i,
output_tensor.GetSize(), tensor_desc->GetShape().ToString().c_str());
output_tensor.GetSize(), tensor_desc->MutableShape().ToString().c_str());
return INTERNAL_ERROR;
}

@@ -299,7 +302,7 @@ Status HybridModelAsyncExecutor::CopyOutputs(HybridModelExecutor::ExecuteArgs &a
output.data = std::move(data_buf);
output_data->blobs.emplace_back(data_buf.get(), static_cast<uint32_t>(output_size), false);
} else {
GELOGW("Output[%zu] is empty. shape = [%s]", i, tensor_desc->GetShape().ToString().c_str());
GELOGW("Output[%zu] is empty. shape = [%s]", i, tensor_desc->MutableShape().ToString().c_str());
output.data = nullptr;
output_data->blobs.emplace_back(nullptr, 0U, false);
}
@@ -307,53 +310,7 @@ Status HybridModelAsyncExecutor::CopyOutputs(HybridModelExecutor::ExecuteArgs &a
outputs.emplace_back(std::move(output));
GELOGD("Output[%zu] added, type = %s, shape = [%s], size = %ld", i,
TypeUtils::DataTypeToSerialString(tensor_desc->GetDataType()).c_str(),
tensor_desc->GetShape().ToString().c_str(), output_size);
}

return SUCCESS;
}

Status HybridModelAsyncExecutor::Execute(const vector<GeTensor> &inputs, vector<GeTensor> &outputs) {
GELOGD("Start to execute model.");
// prepare inputs
InputData input_data;
for (auto &tensor : inputs) {
DataBuffer buffer;
buffer.data = const_cast<uint8_t *>(tensor.GetData().GetData());
buffer.length = tensor.GetData().size();
input_data.blobs.emplace_back(buffer);
}
GE_CHK_STATUS_RET(CopyInputData(input_data), "Failed to copy input data to model");
GELOGD("Done copying input data successfully.");

HybridModelExecutor::ExecuteArgs args;
args.inputs.resize(input_tensors_.size());
args.input_desc.resize(input_tensors_.size());
for (auto &it : input_tensors_) {
args.inputs[it.first] = it.second;
args.input_desc[it.first] = MakeShared<GeTensorDesc>(inputs[it.first].GetTensorDesc());
}

GE_CHK_STATUS_RET(executor_->Execute(args), "Failed to execute model.");

std::vector<ge::OutputTensorInfo> output_tensor_info_list;
OutputData output_data;
GE_CHK_STATUS_RET(CopyOutputs(args, &output_data, output_tensor_info_list), "Failed to copy outputs.");
GELOGD("Done copying output data successfully. output count = %zu", output_tensor_info_list.size());

int out_index = 0;
outputs.resize(output_tensor_info_list.size());
for (auto &out_tensor_info : output_tensor_info_list) {
auto &ge_tensor = outputs[out_index];
if (out_tensor_info.length > 0) {
GE_CHK_GRAPH_STATUS_RET(ge_tensor.SetData(out_tensor_info.data.get(), out_tensor_info.length),
"Failed to set output[%d].", out_index);
}

ge_tensor.MutableTensorDesc() = *args.output_desc[out_index];
GELOGD("Set output[%d], tensor size = %ld, shape = [%s]", out_index, out_tensor_info.length,
ge_tensor.MutableTensorDesc().MutableShape().ToString().c_str());
++out_index;
tensor_desc->MutableShape().ToString().c_str(), output_size);
}

return SUCCESS;


+ 3
- 5
src/ge/hybrid/executor/hybrid_model_async_executor.h View File

@@ -35,8 +35,6 @@ class HybridModelAsyncExecutor {

Status Init();

Status Execute(const vector<GeTensor> &inputs, vector<GeTensor> &outputs);

Status Start(const std::shared_ptr<ModelListener> &listener);

void SetDeviceId(uint32_t device_id);
@@ -54,10 +52,10 @@ class HybridModelAsyncExecutor {

Status SyncVarData();

Status HandleResult(Status exec_ret, uint32_t data_id, HybridModelExecutor::ExecuteArgs &args,
Status HandleResult(Status exec_ret, uint32_t data_id, const std::vector<TensorValue> &output_tensors,
OutputData *output_data);

Status CopyOutputs(HybridModelExecutor::ExecuteArgs &args, OutputData *output_data,
Status CopyOutputs(const std::vector<TensorValue> &output_tensors, OutputData *output_data,
std::vector<ge::OutputTensorInfo> &outputs);

Status OnComputeDone(uint32_t data_index, uint32_t result_code, std::vector<ge::OutputTensorInfo> &outputs);
@@ -72,7 +70,7 @@ class HybridModelAsyncExecutor {
uint32_t model_id_ = 0U;
std::atomic_bool run_flag_;
std::unique_ptr<DataInputer> data_inputer_;
std::unique_ptr<HybridModelExecutor> executor_;
std::unique_ptr<HybridModelExecutor> engine_;
std::future<Status> future_;
uint64_t iterator_count_ = 0;



+ 85
- 18
src/ge/hybrid/executor/hybrid_model_executor.cc View File

@@ -26,17 +26,17 @@ HybridModelExecutor::HybridModelExecutor(HybridModel *model, uint32_t device_id,
Status HybridModelExecutor::Init() {
GELOGD("Start to init HybridGraphEngine.");
GE_CHK_STATUS_RET_NOLOG(InitExecutionContext());
infer_shape_engine_.reset(new (std::nothrow) ShapeInferenceEngine(&context_));
compile_engine_.reset(new (std::nothrow) TaskCompileEngine(&context_));
execute_engine_.reset(new (std::nothrow) ExecutionEngine(&context_, context_.callback_manager.get()));
GE_CHK_STATUS_RET_NOLOG(compile_engine_->Init());
GELOGD("HybridGraphEngine initialized successfully.");
return SUCCESS;
}

Status HybridModelExecutor::Execute(HybridModelExecutor::ExecuteArgs &args) {
GELOGD("Start to execute model.");
auto root_graph_item = model_->GetRootGraphItem();
GE_CHECK_NOTNULL(root_graph_item);

SubgraphExecutor executor(model_->GetRootGraphItem(), &context_);
auto ret = ExecuteGraphInternal(executor, args);
auto ret = ExecuteGraphInternal(args);
Cleanup();
RECORD_MODEL_EXECUTION_EVENT(&context_, "[Cleanup] End");
GE_CHK_STATUS_RET(ret, "Failed to execute model");
@@ -46,22 +46,24 @@ Status HybridModelExecutor::Execute(HybridModelExecutor::ExecuteArgs &args) {
context_.profiler->Reset();
}

context_.iteration += 1;
return SUCCESS;
}

Status HybridModelExecutor::ExecuteGraphInternal(SubgraphExecutor &executor, HybridModelExecutor::ExecuteArgs &args) {
Status HybridModelExecutor::ExecuteGraphInternal(HybridModelExecutor::ExecuteArgs &args) {
RECORD_MODEL_EXECUTION_EVENT(&context_, "[InitContext] Start");
GE_CHK_STATUS_RET_NOLOG(ResetExecutionContext(context_));
RECORD_MODEL_EXECUTION_EVENT(&context_, "[InitContext] End");

GE_CHK_STATUS_RET(executor.ExecuteAsync(args.inputs, args.input_desc), "Failed to execute partitioned call.");
RECORD_MODEL_EXECUTION_EVENT(&context_, "[ExecuteAsync] End");

GE_CHK_STATUS_RET(executor.Synchronize(), "Failed to sync root graph.");
GE_CHK_STATUS_RET_NOLOG(InitInputsAndOutputs(args, context_));
RECORD_MODEL_EXECUTION_EVENT(&context_, "[InitInputsAndOutputs] End");
GE_CHK_STATUS_RET_NOLOG(compile_engine_->Start(pool_));
RECORD_MODEL_EXECUTION_EVENT(&context_, "[CompileProcess] Started");
GE_CHK_STATUS_RET_NOLOG(infer_shape_engine_->Start(pool_));
RECORD_MODEL_EXECUTION_EVENT(&context_, "[InferShapeProcess] Started");
GE_CHK_STATUS_RET(execute_engine_->Start(), "Run execution engine failed.");
RECORD_MODEL_EXECUTION_EVENT(&context_, "[ExecutionProcess] End");
GE_CHK_STATUS_RET_NOLOG(Synchronize());
RECORD_MODEL_EXECUTION_EVENT(&context_, "[Synchronize] End");

GE_CHK_STATUS_RET(executor.GetOutputs(args.outputs, args.output_desc), "Failed to get outputs");
GE_CHK_STATUS_RET_NOLOG(GetOutput(args));
RECORD_MODEL_EXECUTION_EVENT(&context_, "[GetOutput] End");
return SUCCESS;
}
@@ -69,16 +71,18 @@ Status HybridModelExecutor::ExecuteGraphInternal(SubgraphExecutor &executor, Hyb
Status HybridModelExecutor::Cleanup() {
GELOGD("Start to cleanup.");
context_.callback_manager->Destroy();
context_.cv_manager.Reset();
context_.node_states.clear();
context_.all_inputs.clear();
context_.all_outputs.clear();
context_.compile_queue.Clear();
context_.execution_queue.Clear();
RuntimeInferenceContext::DestroyContext(to_string(context_.session_id));
GELOGD("Cleanup successfully.");
return SUCCESS;
}

Status HybridModelExecutor::InitExecutionContext() {
GE_CHK_RT_RET(rtCtxGetCurrent(&context_.rt_context));
GE_CHK_RT_RET(rtCtxCreate(&context_.rt_gen_context, RT_CTX_GEN_MODE, 0));
GE_CHK_RT_RET(rtCtxSetCurrent(context_.rt_context));

context_.stream = stream_;
context_.model = model_;
context_.session_id = ::ge::GetContext().SessionId();
@@ -90,15 +94,78 @@ Status HybridModelExecutor::InitExecutionContext() {
if (IsLogEnable(GE_MODULE_NAME, DLOG_DEBUG)) {
context_.trace_enabled = true;
}

return SUCCESS;
}

Status HybridModelExecutor::ResetExecutionContext(GraphExecutionContext &context) {
auto &model = *context.model;
context.all_inputs.resize(model.TotalInputs());
context.all_outputs.resize(model.TotalOutputs());
context.compile_queue.Restart();
context.execution_queue.Restart();
GE_CHK_STATUS_RET_NOLOG(context.callback_manager->Init());

for (auto const_node : model.GetConstNodes()) {
auto weight_tensor = model.GetWeight(const_node);
GE_CHECK_NOTNULL(weight_tensor);
for (auto &dst_aid_and_nid : const_node->outputs[0]) {
auto *dst_node_item = dst_aid_and_nid.second;
auto input_offset = dst_node_item->input_start + dst_aid_and_nid.first;
context.all_inputs[input_offset] = *weight_tensor;
}
}

string ctx_id = std::to_string(context.session_id);
RuntimeInferenceContext::DestroyContext(ctx_id);
GE_CHK_GRAPH_STATUS_RET(RuntimeInferenceContext::CreateContext(ctx_id), "Failed to Destroy RuntimeInferenceContext");
return SUCCESS;
}

Status HybridModelExecutor::InitInputsAndOutputs(HybridModelExecutor::ExecuteArgs &args,
GraphExecutionContext &context) {
for (const auto &it : model_->GetInputNodes()) {
uint32_t input_index = it.first;
if (input_index >= args.inputs.size()) {
GELOGE(PARAM_INVALID, "Not enough inputs. NumInputs = %zu, but input index = %u", args.inputs.size(),
input_index);
return PARAM_INVALID;
}

auto node_item = it.second;
auto &input_tensor = args.inputs[input_index];
GELOGD("Set input tensor[%u] to inputs with index = %d, addr = %p, size = %zu", input_index, node_item->input_start,
input_tensor.GetData(), input_tensor.GetSize());
context.all_inputs[node_item->input_start] = input_tensor;
}

for (size_t i = 0; i < model_->GetOutputOffsets().size(); ++i) {
auto offset = model_->GetOutputOffsets()[i];
if (i < args.outputs.size() && args.outputs[i].GetData() != nullptr) {
GELOGD("Use user allocated output memory. output index = %zu, output offset = %d", i, offset);
context.all_outputs[offset] = args.outputs[i];
}
}

return SUCCESS;
}

Status HybridModelExecutor::Synchronize() {
GE_CHK_RT_RET(rtStreamSynchronize(stream_));
return SUCCESS;
}

Status HybridModelExecutor::GetOutput(HybridModelExecutor::ExecuteArgs &args) {
auto &net_output_input_offsets = model_->GetNetOutputInputOffsets();
auto num_outputs = net_output_input_offsets.size();
args.outputs.resize(num_outputs);
for (size_t i = 0; i < num_outputs; ++i) {
auto offset = net_output_input_offsets[i];
GELOGI("Get output[%zu] from offset %d", i, offset);
args.outputs[i] = context_.all_inputs[offset];
}

return SUCCESS;
}
} // namespace hybrid
} // namespace ge

+ 13
- 4
src/ge/hybrid/executor/hybrid_model_executor.h View File

@@ -20,7 +20,9 @@
#include "graph/load/new_model_manager/data_inputer.h"
#include "hybrid/executor/hybrid_execution_context.h"
#include "hybrid/executor/rt_callback_manager.h"
#include "hybrid/executor/subgraph_executor.h"
#include "hybrid/executor/worker/execution_engine.h"
#include "hybrid/executor/worker/shape_inference_engine.h"
#include "hybrid/executor/worker/task_compile_engine.h"

namespace ge {
namespace hybrid {
@@ -28,9 +30,7 @@ class HybridModelExecutor {
public:
struct ExecuteArgs {
std::vector<TensorValue> inputs;
std::vector<ConstGeTensorDescPtr> input_desc;
std::vector<TensorValue> outputs;
std::vector<ConstGeTensorDescPtr> output_desc;
};

HybridModelExecutor(HybridModel *model, uint32_t device_id, rtStream_t stream);
@@ -44,15 +44,24 @@ class HybridModelExecutor {
Status Execute(ExecuteArgs &args);

private:
Status ExecuteGraphInternal(SubgraphExecutor &executor, ExecuteArgs &args);
Status ExecuteGraphInternal(ExecuteArgs &args);
Status Cleanup();
Status InitExecutionContext();
static Status ResetExecutionContext(GraphExecutionContext &context);

Status InitInputsAndOutputs(ExecuteArgs &args, GraphExecutionContext &context);
Status GetOutput(ExecuteArgs &args);

Status Synchronize();

ThreadPool pool_;
HybridModel *model_;
uint32_t device_id_;
rtStream_t stream_;
GraphExecutionContext context_;
std::unique_ptr<ShapeInferenceEngine> infer_shape_engine_;
std::unique_ptr<TaskCompileEngine> compile_engine_;
std::unique_ptr<ExecutionEngine> execute_engine_;
};
} // namespace hybrid
} // namespace ge


+ 2
- 1
src/ge/hybrid/executor/hybrid_profiler.cc View File

@@ -59,10 +59,11 @@ void HybridProfiler::Dump(std::ostream &output_stream) {

auto first_evt = events_[0];
auto start = first_evt.timestamp;
output_stream << "Start " << first_evt.desc << std::endl;
std::vector<decltype(start)> prev_timestamps;
prev_timestamps.resize(kMaxEventTypes, start);

for (int i = 0; i < counter_; ++i) {
for (int i = 1; i < counter_; ++i) {
auto &evt = events_[i];
auto elapsed = std::chrono::duration_cast<std::chrono::microseconds>(evt.timestamp - start).count();
auto &prev_ts = prev_timestamps[evt.event_type];


+ 8
- 33
src/ge/hybrid/executor/node_done_manager.cc View File

@@ -15,49 +15,35 @@
*/

#include "hybrid/executor/node_done_manager.h"
#include <chrono>
#include "framework/common/debug/ge_log.h"

namespace ge {
namespace hybrid {
namespace {
constexpr int kDefaultWaitTimeoutInSec = 10;
}
bool NodeDoneManager::Cond::Await() {
std::unique_lock<std::mutex> lk(cond_mu_);
if (!cv_.wait_for(lk, std::chrono::seconds(kDefaultWaitTimeoutInSec),
[&]() { return is_released_ || is_cancelled_; })) {
GELOGE(INTERNAL_ERROR, "Wait timed out.");
return false;
}

std::unique_lock<std::mutex> lk(mu_);
cv_.wait(lk, [&]() { return is_released_ || is_cancelled_; });
return is_released_;
}

void NodeDoneManager::Cond::Release() {
std::unique_lock<std::mutex> lk(cond_mu_);
std::unique_lock<std::mutex> lk(mu_);
is_released_ = true;
cv_.notify_all();
}

void NodeDoneManager::Cond::Cancel() {
std::unique_lock<std::mutex> lk(cond_mu_);
std::unique_lock<std::mutex> lk(mu_);
is_cancelled_ = true;
cv_.notify_all();
}

bool NodeDoneManager::Cond::IsRelease() {
std::unique_lock<std::mutex> lk(cond_mu_);
std::unique_lock<std::mutex> lk(mu_);
return is_released_;
}

NodeDoneManager::Cond *NodeDoneManager::GetSubject(const NodePtr &node) {
std::lock_guard<std::mutex> lk(mu_);
if (destroyed_) {
GELOGD("Already destroyed.");
return nullptr;
}

auto it = subjects_.find(node);
if (it == subjects_.end()) {
return &subjects_[node];
@@ -66,10 +52,8 @@ NodeDoneManager::Cond *NodeDoneManager::GetSubject(const NodePtr &node) {
return &it->second;
}

void NodeDoneManager::Destroy() {
GELOGD("Start to reset NodeDoneManager.");
void NodeDoneManager::Reset() {
std::lock_guard<std::mutex> lk(mu_);
GELOGD("Cond size = %zu.", subjects_.size());
for (auto &sub : subjects_) {
if (!sub.second.IsRelease()) {
sub.second.Cancel();
@@ -78,24 +62,15 @@ void NodeDoneManager::Destroy() {
}

subjects_.clear();
destroyed_ = true;
GELOGD("Done resetting NodeDoneManager successfully.");
}

void NodeDoneManager::NodeDone(const NodePtr &node) {
auto sub = GetSubject(node);
if (sub != nullptr) {
sub->Release();
GELOGD("[%s] Node released.", node->GetName().c_str());
}
GetSubject(node)->Release();
GELOGD("[%s] Node released.", node->GetName().c_str());
}

bool NodeDoneManager::Await(const NodePtr &node) {
auto sub = GetSubject(node);
if (sub == nullptr) {
return false;
}

GELOGD("[%s] Await start. is_released = %s", node->GetName().c_str(), sub->IsRelease() ? "true" : "false");
bool ret = sub->Await();
GELOGD("[%s] Await ended. is_released = %s", node->GetName().c_str(), sub->IsRelease() ? "true" : "false");


+ 2
- 3
src/ge/hybrid/executor/node_done_manager.h View File

@@ -31,7 +31,7 @@ class NodeDoneManager {

bool Await(const NodePtr &node);

void Destroy();
void Reset();

private:
class Cond {
@@ -42,7 +42,7 @@ class NodeDoneManager {
bool Await();

private:
std::mutex cond_mu_;
std::mutex mu_;
std::condition_variable cv_;
bool is_released_ = false;
bool is_cancelled_ = false;
@@ -51,7 +51,6 @@ class NodeDoneManager {
Cond *GetSubject(const NodePtr &node);
std::mutex mu_;
std::unordered_map<NodePtr, Cond> subjects_;
bool destroyed_ = false;
};
} // namespace hybrid
} // namespace ge


+ 3
- 123
src/ge/hybrid/executor/node_state.cc View File

@@ -15,133 +15,13 @@
*/

#include "hybrid/executor/node_state.h"
#include <chrono>
#include "framework/common/debug/log.h"
#include "graph/compute_graph.h"
#include "hybrid_execution_context.h"
#include "subgraph_context.h"

namespace ge {
namespace hybrid {
namespace {
constexpr auto kMaxWaitTimeInSec = 10;
}
ShapeInferenceState::ShapeInferenceState(const NodeItem &node_item) : node_item(node_item) {
this->num_pending_shapes_ = node_item.num_inputs - node_item.num_static_input_shapes;
GELOGD("[%s] ShapeInferenceState created, pending shape count = %d", node_item.NodeName().c_str(),
this->num_pending_shapes_);
}

void ShapeInferenceState::UpdateInputShape(uint32_t idx, const GeShape &ori_shape, const GeShape &shape) {
if (node_item.is_input_shape_static[idx]) {
GELOGD("[%s] Trying to update static shape, idx = %u. old shape = [%s], new shape = [%s]",
node_item.NodeName().c_str(), idx, node_item.op_desc->MutableInputDesc(idx)->GetShape().ToString().c_str(),
shape.ToString().c_str());
return;
}

GELOGD("[%s] Update input shape [%u] with Shape: [%s] and OriginalShape: [%s]", node_item.NodeName().c_str(), idx,
shape.ToString().c_str(), ori_shape.ToString().c_str());

std::lock_guard<std::mutex> lk(mu_);
node_item.op_desc->MutableInputDesc(idx)->SetShape(shape);
node_item.op_desc->MutableInputDesc(idx)->SetOriginShape(ori_shape);
if (--num_pending_shapes_ == 0) {
ready_cv_.notify_all();
}
}

void ShapeInferenceState::UpdateInputShapeFuture(uint32_t idx, ShapeFuture &&future) {
if (node_item.is_input_shape_static[idx]) {
GELOGD("[%s] Trying to update constant shape, idx = %u", node_item.NodeName().c_str(), idx);
return;
}

GELOGD("[%s] Update input shape [%u] with ShapeFuture.", node_item.NodeName().c_str(), idx);
std::lock_guard<std::mutex> lk(mu_);
shape_futures.emplace_back(idx, std::move(future));
if (--num_pending_shapes_ == 0) {
ready_cv_.notify_all();
}
}

Status ShapeInferenceState::AwaitShapesReady(const GraphExecutionContext &context) {
std::unique_lock<std::mutex> lk(mu_);
if (num_pending_shapes_ > 0) {
GELOGD("[%s] Await pending shape or shape future start.", node_item.NodeName().c_str());
if (!ready_cv_.wait_for(lk, std::chrono::seconds(kMaxWaitTimeInSec), [&]() { return num_pending_shapes_ == 0; })) {
GELOGE(INTERNAL_ERROR, "[%s] Wait for shape timeout.", node_item.NodeName().c_str());
return INTERNAL_ERROR;
}
GELOGD("[%s] Await pending shape or shape future end.", node_item.NodeName().c_str());
}

for (auto &p : shape_futures) {
auto idx = p.first;
auto &future = p.second;
GeShape shape;
GeShape ori_shape;
RECORD_SHAPE_INFERENCE_EVENT(&context, node_item.NodeName().c_str(), "[AwaitShape] [idx = %u] Start", idx);
GE_CHK_STATUS_RET(future.Get(ori_shape, shape), "[%s] Get shape failed. index = %u", node_item.NodeName().c_str(),
idx);
RECORD_SHAPE_INFERENCE_EVENT(&context, node_item.NodeName().c_str(), "[AwaitShape] [idx = %u] End", idx);

GELOGD("[%s] Update input shape [%u] with shape: [%s] and ori_shape: [%s]", node_item.NodeName().c_str(), idx,
shape.ToString().c_str(), ori_shape.ToString().c_str());
node_item.op_desc->MutableInputDesc(idx)->SetShape(std::move(shape));
node_item.op_desc->MutableInputDesc(idx)->SetOriginShape(ori_shape);
}

return SUCCESS;
}

ShapeFuture::ShapeFuture(NodePtr src_node, uint32_t src_index, SubgraphContext *subgraph_context)
: src_node_(std::move(src_node)), src_index_(src_index), subgraph_context_(subgraph_context) {}

NodeState::NodeState(const NodeItem &node_item, SubgraphContext *subgraph_context)
: node_item_(&node_item), shape_inference_state_(node_item), subgraph_context_(subgraph_context) {
this->op_desc_ = node_item.node->GetOpDesc();
}

Status NodeState::AwaitInputTensors(GraphExecutionContext &context) const {
for (auto &src_node : node_item_->dependents_for_execution) {
GELOGI("[%s] Start to wait for data dependent node: [%s]", node_item_->NodeName().c_str(),
src_node->GetName().c_str());
RECORD_EXECUTION_EVENT(&context, node_item_->NodeName().c_str(), "[AwaitNodeDone] [%s] Start",
src_node->GetName().c_str());
if (!subgraph_context_->Await(src_node)) {
GELOGE(INTERNAL_ERROR, "[%s] Await node [%s] failed.", GetName().c_str(), src_node->GetName().c_str());
return INTERNAL_ERROR;
}

RECORD_EXECUTION_EVENT(&context, node_item_->NodeName().c_str(), "[AwaitNodeDone] [%s] End",
src_node->GetName().c_str());
GELOGI("[%s] Done waiting node.", src_node->GetName().c_str());
}

return SUCCESS;
}

Status NodeState::WaitForPrepareDone() {
if (prepare_future_.valid()) {
GELOGD("[%s] Start to wait for prepare future.", GetName().c_str());
GE_CHK_STATUS_RET(prepare_future_.get(), "[%s] PreRun failed.", GetName().c_str());
}

return SUCCESS;
}

Status ShapeFuture::Get(GeShape &ori_shape, GeShape &shape) {
GELOGI("Start to wait node: %s for getting shape", src_node_->GetName().c_str());
if (!subgraph_context_->Await(src_node_)) {
GELOGE(INTERNAL_ERROR, "cancelled");
return INTERNAL_ERROR;
}

shape = src_node_->GetOpDesc()->MutableOutputDesc(src_index_)->MutableShape();
ori_shape = src_node_->GetOpDesc()->MutableOutputDesc(src_index_)->GetOriginShape();
GELOGI("Get shape from %s:%u. shape = [%s]", src_node_->GetName().c_str(), src_index_, shape.ToString().c_str());
return SUCCESS;
NodeState::NodeState(const NodeItem &node_item) {
this->node_item = &node_item;
this->op_desc = node_item.node->GetOpDesc();
}
} // namespace hybrid
} // namespace ge

+ 16
- 61
src/ge/hybrid/executor/node_state.h View File

@@ -17,83 +17,38 @@
#ifndef GE_HYBRID_EXECUTOR_NODE_STATE_H_
#define GE_HYBRID_EXECUTOR_NODE_STATE_H_

#include <condition_variable>
#include <future>
#include <mutex>
#include "external/ge/ge_api_error_codes.h"
#include "hybrid/model/node_item.h"
#include "node_done_manager.h"

namespace ge {
namespace hybrid {
class NodeTask;
class GraphExecutionContext;
class SubgraphContext;

class ShapeFuture {
public:
ShapeFuture(NodePtr src_node, uint32_t src_index, SubgraphContext *subgraph_context);
~ShapeFuture() = default;
Status Get(GeShape &ori_shape, GeShape &shape);

private:
NodePtr src_node_;
uint32_t src_index_;
SubgraphContext *subgraph_context_;
};

struct ShapeInferenceState {
explicit ShapeInferenceState(const NodeItem &node_item);

void UpdateInputShape(uint32_t idx, const GeShape &ori_shape, const GeShape &shape);

void UpdateInputShapeFuture(uint32_t idx, ShapeFuture &&future);

Status AwaitShapesReady(const GraphExecutionContext &context);

const NodeItem &node_item;

private:
std::vector<std::pair<uint32_t, ShapeFuture>> shape_futures;
int num_pending_shapes_ = 0;
std::condition_variable ready_cv_;
std::mutex mu_;
};
class NodeTask;

// saving sth. dynamic during execution
struct NodeState {
// 存放一些会变化的信息...
class NodeState {
public:
NodeState(const NodeItem &node_item, SubgraphContext *subgraph_context);
NodeState() = default;
explicit NodeState(const NodeItem &node_item);
~NodeState() = default;

OpDesc *GetOpDesc() const { return op_desc_.get(); }

inline const NodeItem *GetNodeItem() const { return node_item_; }

inline const string &GetName() const { return node_item_->NodeName(); }

inline const string &GetType() const { return node_item_->NodeType(); }
inline int NodeId() const { return node_item->node_id; }

ShapeInferenceState &GetShapeInferenceState() { return shape_inference_state_; }
inline Node *GetNode() const { return node_item->node.get(); }

const shared_ptr<NodeTask> &GetKernelTask() const { return kernel_task_; }
OpDesc *GetOpDesc() const { return op_desc.get(); }

void SetKernelTask(const shared_ptr<NodeTask> &kernel_task) { kernel_task_ = kernel_task; }
inline const NodeItem *GetNodeItem() const { return node_item; }

Status WaitForPrepareDone();
inline const string &GetName() const { return node_item->NodeName(); }

void SetPrepareFuture(std::future<Status> &&prepare_future) { this->prepare_future_ = std::move(prepare_future); }
inline const string &GetType() const { return node_item->NodeType(); }

Status AwaitInputTensors(GraphExecutionContext &context) const;
// private:
const NodeItem *node_item = nullptr;
std::shared_ptr<NodeTask> kernel_task = nullptr;

private:
const NodeItem *node_item_ = nullptr;
std::shared_ptr<NodeTask> kernel_task_ = nullptr;
std::future<Status> prepare_future_;
OpDescPtr op_desc_;
ShapeInferenceState shape_inference_state_;
SubgraphContext *subgraph_context_;
std::mutex mu_;
bool is_compiled = false;
OpDescPtr op_desc;
};

using NodeStatePtr = std::shared_ptr<NodeState>;


+ 1
- 0
src/ge/hybrid/executor/rt_callback_manager.cc View File

@@ -42,6 +42,7 @@ Status CallbackManager::Init() {
rtContext_t ctx = nullptr;
GE_CHK_RT_RET(rtCtxGetCurrent(&ctx));
ret_future_ = std::async([&](rtContext_t context) -> Status { return CallbackProcess(context); }, ctx);

if (!ret_future_.valid()) {
GELOGE(INTERNAL_ERROR, "Failed to init callback manager.");
return INTERNAL_ERROR;


+ 0
- 112
src/ge/hybrid/executor/subgraph_context.cc View File

@@ -1,112 +0,0 @@
/**
* Copyright 2019-2020 Huawei Technologies Co., Ltd
*
* 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 "subgraph_context.h"

#include "common/debug/log.h"

namespace ge {
namespace hybrid {
SubgraphContext::SubgraphContext(const GraphItem *graph_item) : graph_item_(graph_item) {}

Status SubgraphContext::Init() {
GE_CHECK_NOTNULL(graph_item_);
GELOGD("[%s] Start to init subgraph context. total inputs = %d, total outputs = %d", graph_item_->GetName().c_str(),
graph_item_->TotalInputs(), graph_item_->TotalOutputs());
all_inputs_.resize(graph_item_->TotalInputs());
all_outputs_.resize(graph_item_->TotalOutputs());

return SUCCESS;
}

NodeStatePtr SubgraphContext::GetOrCreateNodeState(const NodeItem *node_item) {
std::lock_guard<std::mutex> lk(mu_);
auto &node_state = node_states_[node_item];
if (node_state == nullptr) {
node_state.reset(new (std::nothrow) NodeState(*node_item, this));
}

return node_state;
}

Status SubgraphContext::SetInput(int index, const TensorValue &tensor) {
if (static_cast<size_t>(index) >= all_inputs_.size()) {
GELOGE(INTERNAL_ERROR, "output index output range. all input num = %zu, input index = %d", all_inputs_.size(),
index);
return INTERNAL_ERROR;
}

all_inputs_[index] = tensor;
return SUCCESS;
}

Status SubgraphContext::SetInput(const NodeItem &node_item, int input_index, const TensorValue &tensor) {
auto index = node_item.input_start + input_index;
return SetInput(index, tensor);
}

Status SubgraphContext::SetOutput(const NodeItem &node_item, int output_index, const TensorValue &tensor) {
auto index = node_item.output_start + output_index;
if (output_index >= node_item.num_outputs || static_cast<size_t>(index) >= all_outputs_.size()) {
GELOGE(INTERNAL_ERROR, "output index output range. all output num = %zu, node_item = %s, output index = %d",
all_outputs_.size(), node_item.DebugString().c_str(), output_index);
return INTERNAL_ERROR;
}

all_outputs_[index] = tensor;
return SUCCESS;
}

Status SubgraphContext::GetInput(int index, TensorValue &tensor) {
GE_CHECK_GE(all_inputs_.size(), index + 1U);
tensor = all_inputs_[index];
return SUCCESS;
}

Status SubgraphContext::GetOutputs(std::vector<TensorValue> &outputs) {
if (graph_item_->IsDynamic()) {
GELOGD("[%s] graph is dynamic, get outputs from net output input tensors", graph_item_->GetName().c_str());
// get from net output inputs
auto output_node = graph_item_->GetOutputNode();
GE_CHECK_NOTNULL(output_node);
for (int i = 0; i < output_node->num_inputs; ++i) {
TensorValue tensor;
GE_CHK_STATUS_RET_NOLOG(GetInput(output_node->input_start + i, tensor));
GELOGD("[%s] Adding output tensor by input index [%d], tensor = %s", graph_item_->GetName().c_str(),
output_node->input_start + i, tensor.DebugString().c_str());
outputs.emplace_back(std::move(tensor));
}
} else {
GELOGD("[%s] graph is non-dynamic, get outputs from subgraph outputs", graph_item_->GetName().c_str());
for (auto &tensor : all_outputs_) {
GELOGD("[%s] Adding output tensor: %s", graph_item_->GetName().c_str(), tensor.DebugString().c_str());
outputs.emplace_back(tensor);
}
}

return SUCCESS;
}

bool SubgraphContext::Await(const NodePtr &node) { return node_done_manager_.Await(node); }

void SubgraphContext::OnError(Status error) {
GELOGE(error, "[%s] Error occurred while executing graph.", graph_item_->GetName().c_str());
node_done_manager_.Destroy();
}

void SubgraphContext::NodeDone(const NodePtr &node) { node_done_manager_.NodeDone(node); }
} // namespace hybrid
} // namespace ge

+ 0
- 61
src/ge/hybrid/executor/subgraph_context.h View File

@@ -1,61 +0,0 @@
/**
* Copyright 2019-2020 Huawei Technologies Co., Ltd
*
* 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.
*/

#ifndef GE_HYBRID_EXECUTOR_ITERATION_CONTEXT_H_
#define GE_HYBRID_EXECUTOR_ITERATION_CONTEXT_H_

#include <vector>

#include "hybrid/common/tensor_value.h"
#include "hybrid/executor/node_state.h"
#include "hybrid/executor/node_done_manager.h"
#include "hybrid/model/graph_item.h"
#include "hybrid/model/node_item.h"

namespace ge {
namespace hybrid {
class SubgraphContext {
public:
explicit SubgraphContext(const GraphItem *graph_item);
~SubgraphContext() = default;

Status Init();
NodeStatePtr GetOrCreateNodeState(const NodeItem *node_item);

void OnError(Status error);

Status SetInput(const NodeItem &node_item, int input_index, const TensorValue &tensor);
Status SetOutput(const NodeItem &node_item, int output_index, const TensorValue &tensor);
Status SetInput(int index, const TensorValue &tensor);
Status GetInput(int index, TensorValue &tensor);
Status GetOutputs(std::vector<TensorValue> &outputs);

bool Await(const NodePtr &node);
void NodeDone(const NodePtr &node);

private:
friend class TaskContext;
const GraphItem *graph_item_;
std::mutex mu_;
std::vector<TensorValue> all_inputs_;
std::vector<TensorValue> all_outputs_;
NodeDoneManager node_done_manager_;
std::unordered_map<const NodeItem *, NodeStatePtr> node_states_;
};
} // namespace hybrid
} // namespace ge

#endif // GE_HYBRID_EXECUTOR_ITERATION_CONTEXT_H_

+ 0
- 373
src/ge/hybrid/executor/subgraph_executor.cc View File

@@ -1,373 +0,0 @@
/**
* Copyright 2019-2020 Huawei Technologies Co., Ltd
*
* 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 "hybrid/executor/subgraph_executor.h"
#include "hybrid/executor/worker/task_compile_engine.h"
#include "hybrid/executor/worker/execution_engine.h"
#include "hybrid/node_executor/node_executor.h"

namespace ge {
namespace hybrid {
namespace {
constexpr int kDefaultThreadNum = 4;
constexpr int kDataInputIndex = 0;
} // namespace

SubgraphExecutor::SubgraphExecutor(const GraphItem *graph_item, GraphExecutionContext *context, bool force_infer_shape)
: graph_item_(graph_item),
context_(context),
force_infer_shape_(force_infer_shape),
pre_run_pool_(kDefaultThreadNum) {}

SubgraphExecutor::~SubgraphExecutor() { GELOGD("[%s] SubgraphExecutor destroyed.", graph_item_->GetName().c_str()); }

Status SubgraphExecutor::Init(const std::vector<TensorValue> &inputs,
const std::vector<ConstGeTensorDescPtr> &input_desc) {
subgraph_context_.reset(new (std::nothrow) SubgraphContext(graph_item_));
GE_CHECK_NOTNULL(subgraph_context_);
GE_CHK_STATUS_RET(subgraph_context_->Init(), "[%s] Failed to init subgraph context.", graph_item_->GetName().c_str());

shape_inference_engine_.reset(new (std::nothrow) ShapeInferenceEngine(context_, subgraph_context_.get()));
GE_CHECK_NOTNULL(shape_inference_engine_);

if (graph_item_->IsDynamic()) {
GE_CHK_STATUS_RET(InitInputsForUnknownShape(inputs, input_desc), "[%s] Failed to set inputs.",
graph_item_->GetName().c_str());
} else {
GE_CHK_STATUS_RET(InitInputsForKnownShape(inputs),
"[%s] Failed to init subgraph executor for known shape subgraph.",
graph_item_->GetName().c_str());
}

return SUCCESS;
}

Status SubgraphExecutor::InitInputsForUnknownShape(const std::vector<TensorValue> &inputs,
const std::vector<ConstGeTensorDescPtr> &input_desc) {
// Number of inputs of parent node should be greater or equal than that of subgraph
auto input_nodes = graph_item_->GetInputNodes();
if (inputs.size() < input_nodes.size()) {
GELOGE(INTERNAL_ERROR, "[%s] Number of inputs [%zu] is not sufficient for subgraph which needs [%zu] inputs.",
graph_item_->GetName().c_str(), inputs.size(), input_nodes.size());
return INTERNAL_ERROR;
}

for (size_t i = 0; i < input_nodes.size(); ++i) {
auto &input_node = input_nodes[i];
if (input_node == nullptr) {
GELOGD("[%s] Input[%zu] is not needed by subgraph, skip it.", graph_item_->GetName().c_str(), i);
continue;
}

auto &input_tensor = inputs[i];
GELOGD("[%s] Set input tensor[%zu] to inputs with index = %d, tensor = %s", graph_item_->GetName().c_str(), i,
input_node->input_start, input_tensor.DebugString().c_str());

GE_CHK_STATUS_RET(subgraph_context_->SetInput(*input_node, kDataInputIndex, input_tensor),
"[%s] Failed to set input tensor[%zu]", graph_item_->GetName().c_str(), i);

if (force_infer_shape_ || input_node->is_dynamic) {
GELOGD("[%s] Start to update input[%zu] for subgraph data node.", graph_item_->GetName().c_str(), i);
GE_CHECK_LE(i + 1, input_desc.size());
const auto &tensor_desc = input_desc[i];
auto node_state = subgraph_context_->GetOrCreateNodeState(input_node);
GE_CHECK_NOTNULL(node_state);
node_state->GetShapeInferenceState().UpdateInputShape(0, tensor_desc->GetOriginShape(), tensor_desc->GetShape());
}
}

GELOGD("[%s] Done setting inputs.", graph_item_->GetName().c_str());
return SUCCESS;
}

Status SubgraphExecutor::InitInputsForKnownShape(const std::vector<TensorValue> &inputs) {
auto &input_index_mapping = graph_item_->GetInputIndexMapping();
for (size_t i = 0; i < input_index_mapping.size(); ++i) {
auto &parent_input_index = input_index_mapping[i];
if (static_cast<size_t>(parent_input_index) >= inputs.size()) {
GELOGE(INTERNAL_ERROR,
"[%s] Number of inputs [%zu] is not sufficient for subgraph which needs at lease [%d] inputs",
graph_item_->GetName().c_str(), inputs.size(), parent_input_index + 1);

return INTERNAL_ERROR;
}

auto &input_tensor = inputs[parent_input_index];
subgraph_context_->SetInput(i, input_tensor);
GELOGD("[%s] Set input tensor[%zu] with inputs with index = %d, tensor = %s", graph_item_->GetName().c_str(), i,
parent_input_index, input_tensor.DebugString().c_str());
}

return SUCCESS;
}

Status SubgraphExecutor::ExecuteAsync(const std::vector<TensorValue> &inputs,
const std::vector<ConstGeTensorDescPtr> &input_desc) {
GELOGD("[%s] is dynamic = %s", graph_item_->GetName().c_str(), graph_item_->IsDynamic() ? "true" : "false");
GE_CHK_STATUS_RET(Init(inputs, input_desc), "[%s] Failed to init executor.", graph_item_->GetName().c_str());

if (!graph_item_->IsDynamic()) {
return ExecuteAsyncForKnownShape(inputs);
}

GE_CHK_STATUS_RET(ScheduleTasks(), "[%s] Failed to execute tasks.", graph_item_->GetName().c_str());
GELOGD("[%s] Done executing subgraph successfully.", graph_item_->GetName().c_str());
return SUCCESS;
}

Status SubgraphExecutor::ExecuteAsyncForKnownShape(const std::vector<TensorValue> &inputs) {
GELOGD("[%s] subgraph is not dynamic.", graph_item_->GetName().c_str());
if (graph_item_->GetAllNodes().size() != 1) {
GELOGE(INTERNAL_ERROR, "[%s] Invalid known shape subgraph. node size = %zu", graph_item_->GetName().c_str(),
graph_item_->GetAllNodes().size());
return INTERNAL_ERROR;
}

auto node_item = graph_item_->GetAllNodes()[0];
GE_CHECK_NOTNULL(node_item);
auto node_state = subgraph_context_->GetOrCreateNodeState(node_item);
GE_CHECK_NOTNULL(node_state);
node_state->SetKernelTask(node_item->kernel_task);

known_shape_task_context_ = TaskContext::Create(*node_item, context_, subgraph_context_.get());
GE_CHECK_NOTNULL(known_shape_task_context_);

GE_CHK_STATUS_RET(ExecutionEngine::ExecuteAsync(*node_state, known_shape_task_context_, *context_),
"[%s] Failed to execute node [%s] for known subgraph.", graph_item_->GetName().c_str(),
known_shape_task_context_->GetNodeName());

GELOGD("[%s] Done execute non-dynamic subgraph successfully.", graph_item_->GetName().c_str());
return SUCCESS;
}

Status SubgraphExecutor::ExecuteAsync(TaskContext &task_context) {
std::vector<TensorValue> inputs;
std::vector<ConstGeTensorDescPtr> input_desc;
for (int i = 0; i < task_context.NumInputs(); ++i) {
auto tensor = task_context.GetInput(i);
GE_CHECK_NOTNULL(tensor);
inputs.emplace_back(*tensor);
input_desc.emplace_back(task_context.GetInputDesc(i));
}

GE_CHK_STATUS_RET(ExecuteAsync(inputs, input_desc), "[%s] Failed to execute subgraph.",
graph_item_->GetName().c_str());

GE_CHK_STATUS_RET(SetOutputsToParentNode(task_context), "[%s] Failed to set output shapes to parent node.",
graph_item_->GetName().c_str());
return SUCCESS;
}

Status SubgraphExecutor::PrepareNodes() {
GELOGD("[%s] Start to prepare nodes. force infer shape = %s.", graph_item_->GetName().c_str(),
force_infer_shape_ ? "true" : "false");
auto &all_nodes = graph_item_->GetAllNodes();
for (size_t i = 0; i < all_nodes.size(); ++i) {
auto &node_item = *all_nodes[i];
// for while op
if (force_infer_shape_ && !node_item.is_dynamic) {
GELOGD("[%s] Force infer shape is set, updating node to dynamic.", node_item.NodeName().c_str());
auto &mutable_node_item = const_cast<NodeItem &>(node_item);
mutable_node_item.SetToDynamic();
}

GELOGD("[%s] Start to prepare node [%s].", graph_item_->GetName().c_str(), node_item.NodeName().c_str());
auto node_state = subgraph_context_->GetOrCreateNodeState(&node_item);
GE_CHECK_NOTNULL(node_state);
auto p_node_state = node_state.get();

if (node_item.node_type == NETOUTPUT) {
// Wait for all inputs become valid
// after PrepareNodes returned. all output tensors and shapes are valid
GE_CHK_STATUS_RET_NOLOG(p_node_state->GetShapeInferenceState().AwaitShapesReady(*context_));
GE_CHK_STATUS_RET_NOLOG(p_node_state->AwaitInputTensors(*context_));
continue;
}

// only do shape inference and compilation for nodes with dynamic shapes.
if (node_item.is_dynamic) {
auto prepare_future = pre_run_pool_.commit([this, p_node_state]() -> Status {
GE_CHK_STATUS_RET_NOLOG(InferShape(shape_inference_engine_.get(), *p_node_state));
return PrepareForExecution(context_, *p_node_state);
});

p_node_state->SetPrepareFuture(std::move(prepare_future));
} else {
GELOGD("[%s] Skipping shape inference and compilation for node with static shape.", node_item.NodeName().c_str());
if (node_item.kernel_task == nullptr) {
GELOGW("[%s] Node of static shape got no task.", node_item.NodeName().c_str());
GE_CHK_STATUS_RET(TaskCompileEngine::Compile(*p_node_state, context_), "[%s] Failed to create task.",
p_node_state->GetName().c_str());
} else {
node_state->SetKernelTask(node_item.kernel_task);
}
}

if (!ready_queue_.Push(p_node_state)) {
GELOGE(INTERNAL_ERROR, "[%s] Error occurs while launching tasks. quit from preparing nodes.",
graph_item_->GetName().c_str());
return INTERNAL_ERROR;
}

GELOGD("[%s] Push node [%s] to queue.", graph_item_->GetName().c_str(), node_item.NodeName().c_str());
}

return SUCCESS;
}

Status SubgraphExecutor::InferShape(ShapeInferenceEngine *shape_inference_engine, NodeState &node_state) {
const auto &node_item = *node_state.GetNodeItem();
GE_CHK_STATUS_RET(shape_inference_engine->InferShape(node_state), "[%s] Failed to InferShape.",
node_state.GetName().c_str());
GE_CHK_STATUS_RET(shape_inference_engine->PropagateOutputShapes(node_item), "[%s] Failed to PropagateOutputShapes.",
node_state.GetName().c_str());
return SUCCESS;
}

Status SubgraphExecutor::PrepareForExecution(GraphExecutionContext *ctx, NodeState &node_state) {
auto &node_item = *node_state.GetNodeItem();
if (node_item.kernel_task == nullptr) {
GE_CHK_STATUS_RET(TaskCompileEngine::Compile(node_state, ctx), "Failed to create task for node[%s]",
node_state.GetName().c_str());
} else {
node_state.SetKernelTask(node_item.kernel_task);
}

GELOGD("[%s] Start to invoke CalcOpRunningParam.", node_item.NodeName().c_str());
RECORD_COMPILE_EVENT(ctx, node_item.NodeName().c_str(), "[CalcOpRunningParam] Start");
GE_CHK_STATUS_RET(NodeExecutorManager::GetInstance().CalcOpRunningParam(*node_item.node),
"[%s] Failed to invoke CalcOpRunningParam.", node_item.NodeName().c_str());
RECORD_COMPILE_EVENT(ctx, node_item.NodeName().c_str(), "[CalcOpRunningParam] End");
GELOGD("[%s] Done invoking CalcOpRunningParam successfully.", node_item.NodeName().c_str());
return SUCCESS;
}

Status SubgraphExecutor::LaunchTasks() {
while (true) {
NodeState *node_state = nullptr;
if (!ready_queue_.Pop(node_state)) {
GELOGE(INTERNAL_ERROR, "[%s] Failed to pop node.", graph_item_->GetName().c_str());
return INTERNAL_ERROR;
}

if (node_state == nullptr) {
GELOGD("[%s] Got EOF from queue.", graph_item_->GetName().c_str());
return SUCCESS;
}

GE_CHK_STATUS_RET_NOLOG(node_state->WaitForPrepareDone());

GELOGD("[%s] Start to execute.", node_state->GetName().c_str());
auto task_context = TaskContext::Create(*node_state->GetNodeItem(), context_, subgraph_context_.get());
GE_CHECK_NOTNULL(task_context);
task_context->SetForceInferShape(force_infer_shape_);
auto shared_task_context = std::shared_ptr<TaskContext>(task_context.release());
GE_CHK_STATUS_RET(ExecutionEngine::ExecuteAsync(*node_state, shared_task_context, *context_),
"[%s] Execute node failed.", node_state->GetName().c_str());

GELOGD("[%s] Done executing node successfully.", node_state->GetName().c_str());
}
}

Status SubgraphExecutor::ScheduleTasks() {
GELOGD("[%s] Start to schedule prepare workers.", graph_item_->GetName().c_str());
auto prepare_future = std::async([&]() -> Status {
auto ret = PrepareNodes();
ready_queue_.Push(nullptr);
return ret;
});

GELOGD("[%s] Start to execute subgraph.", graph_item_->GetName().c_str());
auto ret = LaunchTasks();
if (ret != SUCCESS) {
GELOGE(ret, "[%s] Failed to execute subgraph.", graph_item_->GetName().c_str());
subgraph_context_->OnError(ret);
ready_queue_.Stop();
prepare_future.wait();
return ret;
}

GE_CHK_STATUS_RET(prepare_future.get(), "[%s] Error occurred in task preparation.", graph_item_->GetName().c_str());

GELOGD("[%s] Done launching all tasks successfully.", graph_item_->GetName().c_str());
return SUCCESS;
}

Status SubgraphExecutor::GetOutputs(vector<TensorValue> &outputs) { return subgraph_context_->GetOutputs(outputs); }

Status SubgraphExecutor::GetOutputs(vector<TensorValue> &outputs, std::vector<ConstGeTensorDescPtr> &output_desc) {
GE_CHK_STATUS_RET(GetOutputs(outputs), "[%s] Failed to get output tensors.", graph_item_->GetName().c_str());

// copy output data from op to designated position
std::vector<GeTensorDescPtr> output_tensor_desc_list;
GE_CHK_STATUS_RET(graph_item_->GetOutputDescList(output_desc), "[%s] Failed to get output tensor desc.",
graph_item_->GetName().c_str());
return SUCCESS;
}

Status SubgraphExecutor::Synchronize() {
GELOGD("[%s] Synchronize start.", graph_item_->GetName().c_str());
GE_CHK_RT_RET(rtStreamSynchronize(context_->stream));
GELOGD("[%s] Done synchronizing successfully.", graph_item_->GetName().c_str());
return SUCCESS;
}

Status SubgraphExecutor::SetOutputsToParentNode(TaskContext &task_context) {
// get output tensors and tensor desc list
std::vector<TensorValue> outputs;
std::vector<ConstGeTensorDescPtr> output_desc_list;
GE_CHK_STATUS_RET(subgraph_context_->GetOutputs(outputs), "[%s] Failed to get output tensors.",
graph_item_->GetName().c_str());
GE_CHK_STATUS_RET(graph_item_->GetOutputDescList(output_desc_list), "[%s] Failed to get output tensor desc.",
graph_item_->GetName().c_str());

if (outputs.size() != output_desc_list.size()) {
GELOGE(INTERNAL_ERROR, "[%s] num output tensors = %zu, num output tensor desc = %zu",
graph_item_->GetName().c_str(), outputs.size(), output_desc_list.size());
return INTERNAL_ERROR;
}

// mapping to parent task context
for (size_t i = 0; i < outputs.size(); ++i) {
int parent_output_index = graph_item_->GetParentOutputIndex(i);
GE_CHECK_GE(parent_output_index, 0);
// update tensor
GELOGD("[%s] Updating output[%zu] to parent output[%d]", graph_item_->GetName().c_str(), i, parent_output_index);

GELOGD("[%s] Updating output tensor, index = %d, tensor = %s", graph_item_->GetName().c_str(), parent_output_index,
outputs[i].DebugString().c_str());
task_context.SetOutput(parent_output_index, outputs[i]);

// updating shapes. dynamic format/dtype is not supported.
// It should be noted that even the subgraph is of known shape, it is also necessary to update parent output desc,
// for instance, IfOp may have two known-shaped subgraphs of different output shapes
const auto &output_desc = output_desc_list[i];
auto parent_output_desc = task_context.MutableOutputDesc(parent_output_index);
GE_CHECK_NOTNULL(parent_output_desc);
GELOGD("[%s] Updating output shape[%d] from [%s] to [%s]", graph_item_->GetName().c_str(), parent_output_index,
parent_output_desc->MutableShape().ToString().c_str(), output_desc->GetShape().ToString().c_str());
parent_output_desc->SetShape(output_desc->GetShape());

GELOGD("[%s] Updating output original shape[%d] from [%s] to [%s]", graph_item_->GetName().c_str(),
parent_output_index, parent_output_desc->GetOriginShape().ToString().c_str(),
output_desc->GetOriginShape().ToString().c_str());
parent_output_desc->SetOriginShape(output_desc->GetOriginShape());
}

return SUCCESS;
}
} // namespace hybrid
} // namespace ge

+ 0
- 101
src/ge/hybrid/executor/subgraph_executor.h View File

@@ -1,101 +0,0 @@
/**
* Copyright 2019-2020 Huawei Technologies Co., Ltd
*
* 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.
*/

#ifndef GE_HYBRID_EXECUTOR_EXECUTOR_SUBGRAPH_EXECUTOR_H_
#define GE_HYBRID_EXECUTOR_EXECUTOR_SUBGRAPH_EXECUTOR_H_

#include <vector>

#include "common/blocking_queue.h"
#include "common/thread_pool.h"
#include "hybrid/executor/subgraph_context.h"
#include "hybrid/executor/node_state.h"
#include "hybrid/executor/hybrid_execution_context.h"
#include "hybrid/executor/worker/shape_inference_engine.h"
#include "hybrid/model/graph_item.h"
#include "hybrid/node_executor/task_context.h"

namespace ge {
namespace hybrid {
// Executor for executing a subgraph
class SubgraphExecutor {
public:
SubgraphExecutor(const GraphItem *graph_item, GraphExecutionContext *context, bool force_infer_shape = false);
~SubgraphExecutor();

/**
* Execute subgraph async, output tensor address(not data) and output tensor descriptions are
* valid after this method returned
* @param inputs input tensors
* @param input_desc input tensor descriptions
* @return SUCCESS on success, error code otherwise
*/
Status ExecuteAsync(const std::vector<TensorValue> &inputs, const std::vector<ConstGeTensorDescPtr> &input_desc);

/**
* Execute subgraph async, output tensor address(not data) and output tensor descriptions are
* valid after this method returned
* @param task_context instance of TaskContext
* @return SUCCESS on success, error code otherwise
*/
Status ExecuteAsync(TaskContext &task_context);

/**
* Synchronize all tasks in the subgraph. output tensor data are valid after this method returned
* @return SUCCESS on success, error code otherwise
*/
Status Synchronize();

/**
* Get output tensors
* @param outputs output tensors
* @return SUCCESS on success, error code otherwise
*/
Status GetOutputs(std::vector<TensorValue> &outputs);

/**
* Get output tensors and output tensor descriptions
* @param outputs output tensors
* @param output_desc output tensor descriptions
* @return SUCCESS on success, error code otherwise
*/
Status GetOutputs(std::vector<TensorValue> &outputs, std::vector<ConstGeTensorDescPtr> &output_desc);

private:
static Status PrepareForExecution(GraphExecutionContext *ctx, NodeState &node_state);
static Status InferShape(ShapeInferenceEngine *shape_inference_engine, NodeState &node_state);
Status Init(const std::vector<TensorValue> &inputs, const std::vector<ConstGeTensorDescPtr> &input_desc);
Status InitInputsForUnknownShape(const std::vector<TensorValue> &inputs,
const std::vector<ConstGeTensorDescPtr> &input_desc);
Status InitInputsForKnownShape(const std::vector<TensorValue> &inputs);
Status ExecuteAsyncForKnownShape(const std::vector<TensorValue> &inputs);
Status ScheduleTasks();
Status PrepareNodes();
Status LaunchTasks();
Status SetOutputsToParentNode(TaskContext &task_context);

const GraphItem *graph_item_;
GraphExecutionContext *context_;
std::unique_ptr<SubgraphContext> subgraph_context_;
bool force_infer_shape_;
ThreadPool pre_run_pool_;
BlockingQueue<NodeState *> ready_queue_;
std::unique_ptr<ShapeInferenceEngine> shape_inference_engine_;
std::shared_ptr<TaskContext> known_shape_task_context_;
};
} // namespace hybrid
} // namespace ge
#endif // GE_HYBRID_EXECUTOR_EXECUTOR_SUBGRAPH_EXECUTOR_H_

+ 76
- 118
src/ge/hybrid/executor/worker/execution_engine.cc View File

@@ -15,6 +15,7 @@
*/

#include "hybrid/executor/worker/execution_engine.h"
#include <sstream>
#include "graph/runtime_inference_context.h"
#include "graph/utils/tensor_utils.h"
#include "graph/utils/tensor_adapter.h"
@@ -22,38 +23,9 @@

namespace ge {
namespace hybrid {
namespace {
constexpr int64_t kMaxPadding = 63;

Status LogInputs(const NodeItem &node_item, const TaskContext &task_context) {
for (auto i = 0; i < task_context.NumInputs(); ++i) {
const auto &input_tensor = task_context.GetInput(i);
GE_CHECK_NOTNULL(input_tensor);
const auto &tensor_desc = node_item.op_desc->MutableInputDesc(i);
GE_CHECK_NOTNULL(tensor_desc);
GELOGD("[%s] Print task args. input[%d] = %s, shape = [%s]", node_item.NodeName().c_str(), i,
input_tensor->DebugString().c_str(), tensor_desc->MutableShape().ToString().c_str());
}

return SUCCESS;
}

Status LogOutputs(const NodeItem &node_item, const TaskContext &task_context) {
for (auto i = 0; i < task_context.NumOutputs(); ++i) {
const auto &output_tensor = task_context.GetOutput(i);
GE_CHECK_NOTNULL(output_tensor);
const auto &tensor_desc = node_item.op_desc->MutableOutputDesc(i);
GE_CHECK_NOTNULL(tensor_desc);
GELOGD("[%s] Print task args. output[%d] = %s, shape = [%s]", node_item.NodeName().c_str(), i,
output_tensor->DebugString().c_str(), tensor_desc->MutableShape().ToString().c_str());
}

return SUCCESS;
}
} // namespace
class NodeDoneCallback {
public:
NodeDoneCallback(GraphExecutionContext *graph_context, std::shared_ptr<TaskContext> task_context);
NodeDoneCallback(GraphExecutionContext *graph_context, std::shared_ptr<TaskContext> &task_context);
~NodeDoneCallback() = default;
Status OnNodeDone();

@@ -63,8 +35,8 @@ class NodeDoneCallback {
std::shared_ptr<TaskContext> context_;
};

NodeDoneCallback::NodeDoneCallback(GraphExecutionContext *graph_context, std::shared_ptr<TaskContext> task_context)
: graph_context_(graph_context), context_(std::move(task_context)) {}
NodeDoneCallback::NodeDoneCallback(GraphExecutionContext *graph_context, std::shared_ptr<TaskContext> &task_context)
: graph_context_(graph_context), context_(task_context) {}

Status NodeDoneCallback::PrepareConstInputs(const NodeItem &node_item) {
for (auto output_idx : node_item.to_const_output_id_list) {
@@ -74,28 +46,17 @@ Status NodeDoneCallback::PrepareConstInputs(const NodeItem &node_item) {
auto output_tensor = context_->GetOutput(output_idx);
GE_CHECK_NOTNULL(output_tensor);

vector<uint8_t> host_buffer(output_tensor->GetSize());
GELOGD("[%s] To cache output[%d] to host, size = %zu", node_item.NodeName().c_str(), output_idx,
output_tensor->GetSize());
GE_CHK_RT_RET(rtMemcpy(host_buffer.data(), host_buffer.size(), output_tensor->GetData(), output_tensor->GetSize(),
RT_MEMCPY_HOST_TO_DEVICE));
Tensor tensor;
tensor.SetData(host_buffer);
auto ge_tensor_desc = node_item.op_desc->MutableOutputDesc(output_idx);
GE_CHECK_NOTNULL(ge_tensor_desc);
tensor.SetTensorDesc(TensorAdapter::GeTensorDesc2TensorDesc(*ge_tensor_desc));

int64_t tensor_size;
GE_CHK_GRAPH_STATUS_RET(TensorUtils::GetTensorSizeInBytes(*ge_tensor_desc, tensor_size),
"Failed to invoke GetTensorSizeInBytes");

if (output_tensor->GetSize() < static_cast<size_t>(tensor_size)) {
GELOGE(INTERNAL_ERROR, "[%s] Tensor size is not enough. output index = %d, required size = %zu, tensor = %s",
node_item.NodeName().c_str(), output_idx, tensor_size, output_tensor->DebugString().c_str());
return INTERNAL_ERROR;
}

vector<uint8_t> host_buffer(tensor_size);
GELOGD("[%s] To cache output[%d] to host, size = %zu", node_item.NodeName().c_str(), output_idx,
output_tensor->GetSize());
GE_CHK_RT_RET(
rtMemcpy(host_buffer.data(), tensor_size, output_tensor->GetData(), tensor_size, RT_MEMCPY_DEVICE_TO_HOST));
tensor.SetData(host_buffer);

string session_id = std::to_string(context_->GetSessionId());
RuntimeInferenceContext *runtime_infer_ctx = nullptr;
GE_CHK_GRAPH_STATUS_RET(RuntimeInferenceContext::GetContext(session_id, &runtime_infer_ctx),
@@ -126,118 +87,115 @@ Status NodeDoneCallback::OnNodeDone() {
GE_CHK_STATUS_RET_NOLOG(PrepareConstInputs(node_item));
// PropagateOutputs for type == DEPEND_COMPUTE
if (node_item.shape_inference_type == DEPEND_COMPUTE) {
if (graph_context_->trace_enabled) {
(void)LogOutputs(node_item, *context_);
}

GE_CHK_STATUS_RET(context_->PropagateOutputs(), "[%s] Failed to propagate outputs failed",
node_item.NodeName().c_str());

RECORD_CALLBACK_EVENT(graph_context_, context_->GetNodeName(), "[PropagateOutputs] End");
}

// release condition variable
// release
if (node_item.has_observer) {
GELOGI("[%s] Notify observer. node_id = %d", node_item.NodeName().c_str(), node_item.node_id);
context_->NodeDone();
graph_context_->cv_manager.NodeDone(node_item.node);
}

RECORD_CALLBACK_EVENT(graph_context_, context_->GetNodeName(), "[Callback] End");
return SUCCESS;
}

Status ExecutionEngine::ExecuteAsync(NodeState &node_state, const std::shared_ptr<TaskContext> &task_context,
GraphExecutionContext &execution_context) {
GELOGI("[%s] Node is ready for execution", task_context->GetNodeName());
RECORD_EXECUTION_EVENT(&execution_context, task_context->GetNodeName(), "Start");
auto cb = std::shared_ptr<NodeDoneCallback>(new (std::nothrow) NodeDoneCallback(&execution_context, task_context));
GE_CHECK_NOTNULL(cb);
auto callback = [&, cb]() {
auto ret = cb->OnNodeDone();
if (ret != SUCCESS) {
task_context->OnError(ret);
ExecutionEngine::ExecutionEngine(GraphExecutionContext *context, CallbackManager *callback_manager)
: context_(context), callback_manager_(callback_manager) {}

Status ExecutionEngine::Start() {
GE_CHK_STATUS_RET_NOLOG(ExecutionProcess());
return SUCCESS;
}

Status ExecutionEngine::ExecutionProcess() {
GELOGI("ExecutorEngine worker started");
auto &ready_queue = context_->execution_queue;
while (true) {
NodeStatePtr node_state = nullptr;
if (!ready_queue.Pop(node_state)) {
GELOGE(FAILED, "Pop task failed");
return FAILED;
}

// EOF
if (node_state == nullptr) {
break;
}
};

GE_CHK_STATUS_RET_NOLOG(DoExecuteAsync(node_state, *task_context, execution_context, callback));
GE_CHK_STATUS_RET_NOLOG(PropagateOutputs(*node_state.GetNodeItem(), *task_context, execution_context));
RECORD_EXECUTION_EVENT(context_, node_state->GetName().c_str(), "Start");
GELOGI("[%s] Node is ready for execution", node_state->GetName().c_str());
auto *node_item = node_state->node_item;
auto task_context = TaskContext::Create(*node_item, context_);
GE_CHECK_NOTNULL(task_context);
auto shared_task_context = shared_ptr<TaskContext>(task_context.release());

auto cb = std::shared_ptr<NodeDoneCallback>(new (std::nothrow) NodeDoneCallback(context_, shared_task_context));
GE_CHECK_NOTNULL(cb);
auto callback = [&, cb]() {
auto ret = cb->OnNodeDone();
if (ret != SUCCESS) {
context_->OnError(ret);
}
};

GE_CHK_STATUS_RET_NOLOG(ExecuteAsync(*node_state, *shared_task_context, callback));
GE_CHK_STATUS_RET_NOLOG(PropagateOutputs(*node_item, *shared_task_context));
}

GELOGI("ExecutorEngine worker ended.");
return SUCCESS;
}

Status ExecutionEngine::DoExecuteAsync(NodeState &node_state, TaskContext &task_context, GraphExecutionContext &context,
const std::function<void()> &callback) {
const auto &task = node_state.GetKernelTask();
Status ExecutionEngine::ExecuteAsync(NodeState &node_state, TaskContext &task_context,
const std::function<void()> &callback) {
const auto &task = node_state.kernel_task;
if (task == nullptr) {
GELOGE(INTERNAL_ERROR, "[%s] NodeTask is null.", node_state.GetName().c_str());
return INTERNAL_ERROR;
}

// Wait for dependent nodes(DEPEND_COMPUTE), so that the input tensors are valid.
RECORD_EXECUTION_EVENT(&context, task_context.GetNodeName(), "[AwaitDependents] Start");
GE_CHK_STATUS_RET(node_state.AwaitInputTensors(context), "[%s] Failed to wait for dependent nodes.",
node_state.GetName().c_str());

const auto &node_item = *node_state.GetNodeItem();
auto executor = node_item.node_executor;
GE_CHECK_NOTNULL(executor);
RECORD_EXECUTION_EVENT(&context, task_context.GetNodeName(), "[PrepareTask] Start");
RECORD_EXECUTION_EVENT(context_, task_context.GetNodeName(), "[PrepareTask] Start");
auto executor = node_state.node_item->node_executor;
GE_CHK_STATUS_RET(executor->PrepareTask(*task, task_context), "[%s] Failed to prepare task",
node_state.GetName().c_str());
RECORD_EXECUTION_EVENT(&context, task_context.GetNodeName(), "[PrepareTask] End");
RECORD_EXECUTION_EVENT(context_, task_context.GetNodeName(), "[PrepareTask] End");
GELOGD("[%s] Done task preparation successfully.", node_state.GetName().c_str());

if (context.trace_enabled) {
LogInputs(node_item, task_context);
if (node_item.shape_inference_type != DEPEND_COMPUTE) {
LogOutputs(node_item, task_context);
if (context_->trace_enabled) {
for (auto i = 0; i < task_context.NumInputs(); ++i) {
const auto &input_tensor = task_context.GetInput(i);
GE_CHECK_NOTNULL(input_tensor);
GELOGD("[%s] Tensor of input[%d] = %s", node_state.GetName().c_str(), i, input_tensor->DebugString().c_str());
}
}

GE_CHK_STATUS_RET(ValidateInputTensors(node_state, task_context), "Failed to validate input tensors.");
RECORD_EXECUTION_EVENT(&context, task_context.GetNodeName(), "[ValidateInputTensors] End");
for (auto i = 0; i < task_context.NumOutputs(); ++i) {
const auto &output_tensor = task_context.GetOutput(i);
GE_CHECK_NOTNULL(output_tensor);
GELOGD("[%s] Tensor of output[%d] = %s", node_state.GetName().c_str(), i, output_tensor->DebugString().c_str());
}
}

RECORD_EXECUTION_EVENT(context_, task_context.GetNodeName(), "[ExecuteTask] Start");
GE_CHK_STATUS_RET(executor->ExecuteTask(*task, task_context, callback), "[%s] Failed to execute task",
node_state.GetName().c_str());
RECORD_EXECUTION_EVENT(&context, task_context.GetNodeName(), "[ExecuteTask] End");
RECORD_EXECUTION_EVENT(context_, task_context.GetNodeName(), "[ExecuteTask] End");

GELOGD("[%s] Done task launch successfully.", node_state.GetName().c_str());
return SUCCESS;
}

Status ExecutionEngine::ValidateInputTensors(const NodeState &node_state, const TaskContext &task_context) {
for (auto i = 0; i < task_context.NumInputs(); ++i) {
const auto &input_tensor = task_context.GetInput(i);
GE_CHECK_NOTNULL(input_tensor);
const auto &tensor_desc = node_state.GetOpDesc()->MutableInputDesc(i);
GE_CHECK_NOTNULL(tensor_desc);
int64_t expected_size;
GE_CHK_GRAPH_STATUS_RET(TensorUtils::GetTensorMemorySizeInBytes(*tensor_desc, expected_size));
GELOGD("[%s] Input[%d] expects [%ld] bytes.", task_context.GetNodeName(), i, expected_size);
auto size_diff = expected_size - static_cast<int64_t>(input_tensor->GetSize());
if (size_diff > 0) {
if (size_diff <= kMaxPadding) {
GELOGW("[%s] Input[%d]: tensor size mismatches. expected: %ld, but given %zu", task_context.GetNodeName(), i,
expected_size, input_tensor->GetSize());
} else {
GELOGE(INTERNAL_ERROR, "[%s] Input[%d]: tensor size mismatches. expected: %ld, but given %zu",
task_context.GetNodeName(), i, expected_size, input_tensor->GetSize());
return INTERNAL_ERROR;
}
}
}

return SUCCESS;
}

Status ExecutionEngine::PropagateOutputs(const NodeItem &node_item, TaskContext &task_context,
GraphExecutionContext &context) {
Status ExecutionEngine::PropagateOutputs(const NodeItem &node_item, TaskContext &task_context) {
if (node_item.shape_inference_type != DEPEND_COMPUTE) {
GE_CHK_STATUS_RET(task_context.PropagateOutputs(), "[%s] Failed to propagate outputs.",
node_item.NodeName().c_str());
RECORD_EXECUTION_EVENT(&context, task_context.GetNodeName(), "[PropagateOutputs] End");
GELOGD("[%s] Done propagating outputs successfully.", node_item.NodeName().c_str());
RECORD_EXECUTION_EVENT(context_, task_context.GetNodeName(), "[PropagateOutputs] End");
}

GELOGD("[%s] Done propagating outputs successfully.", node_item.NodeName().c_str());
return SUCCESS;
}
} // namespace hybrid


+ 15
- 6
src/ge/hybrid/executor/worker/execution_engine.h View File

@@ -17,21 +17,30 @@
#ifndef GE_HYBRID_EXECUTOR_EXECUTOR_EXECUTION_ENGINE_H_
#define GE_HYBRID_EXECUTOR_EXECUTOR_EXECUTION_ENGINE_H_

#include "common/thread_pool.h"
#include "hybrid/common/npu_memory_allocator.h"
#include "hybrid/executor/hybrid_execution_context.h"
#include "hybrid/executor/rt_callback_manager.h"
#include "hybrid/node_executor/task_context.h"

namespace ge {
namespace hybrid {
class ExecutionEngine {
public:
static Status ExecuteAsync(NodeState &node_state, const std::shared_ptr<TaskContext> &task_context,
GraphExecutionContext &execution_context);
explicit ExecutionEngine(GraphExecutionContext *context, CallbackManager *callback_manager);
~ExecutionEngine() = default;

Status Start();

private:
static Status ValidateInputTensors(const NodeState &node_state, const TaskContext &task_context);
static Status PropagateOutputs(const NodeItem &node_item, TaskContext &task_context, GraphExecutionContext &context);
static Status DoExecuteAsync(NodeState &node_state, TaskContext &task_context, GraphExecutionContext &context,
const std::function<void()> &callback);
Status PropagateOutputs(const NodeItem &node_item, TaskContext &task_context);

Status ExecutionProcess();

Status ExecuteAsync(NodeState &node_state, TaskContext &task_context, const std::function<void()> &callback);

GraphExecutionContext *context_;
CallbackManager *callback_manager_;
};
} // namespace hybrid
} // namespace ge


+ 177
- 48
src/ge/hybrid/executor/worker/shape_inference_engine.cc View File

@@ -15,27 +15,117 @@
*/

#include "hybrid/executor/worker/shape_inference_engine.h"

#include "graph/shape_refiner.h"
#include "graph/runtime_inference_context.h"
#include "graph/utils/node_utils.h"
#include "hybrid/node_executor/node_executor.h"

namespace ge {
namespace hybrid {
ShapeInferenceEngine::ShapeInferenceEngine(GraphExecutionContext *execution_context, SubgraphContext *subgraph_context)
: execution_context_(execution_context), subgraph_context_(subgraph_context) {}

Status ShapeInferenceEngine::InferShape(NodeState &node_state) {
// Wait for all input shape become valid
GE_CHK_STATUS_RET_NOLOG(node_state.GetShapeInferenceState().AwaitShapesReady(*execution_context_));
ShapeInferenceEngine::ShapeInferenceEngine(GraphExecutionContext *context) : context_(context) {}

Status ShapeInferenceEngine::Start(ThreadPool &pool) {
GELOGI("RuntimeShapeInferenceEngine start.");
pool.commit([&]() {
auto ret = this->InferShapeProcess();
InferenceDone(ret);
});

return SUCCESS;
}

Status ShapeInferenceEngine::InferShapeProcess() {
GELOGI("RuntimeShapeInferenceEngine worker start.");
const auto &root_nodes = context_->model->RootNodes();
auto &complete_queue = context_->compile_queue;
std::queue<InferenceState *> ready_nodes;
for (auto &node_item : root_nodes) {
auto infer_state = GetOrCreateEntry(*node_item);
GE_CHECK_NOTNULL(infer_state);
ready_nodes.emplace(infer_state);
}

while (!ready_nodes.empty()) {
InferenceState *infer_state = ready_nodes.front();
ready_nodes.pop();
auto node_item = infer_state->node_item;
// even for non-dynamic shape node, it is still necessary to wait for pending shapes if got any.
// which indicates that the parent node is of type 4, in which case the inputs will be valid only
// when computing is done.
GE_CHK_STATUS_RET(infer_state->AwaitShapeFutures(context_), "Await shape failed.");
GELOGI("[%s] Node is ready for shape inference.", node_item.NodeName().c_str());
if (node_item.is_dynamic) {
// may block
RECORD_SHAPE_INFERENCE_EVENT(context_, node_item.NodeName().c_str(), "Start");
GELOGI("[%s] Start to invoke InferShape", node_item.NodeName().c_str());
auto ret = InferShape(*infer_state);
if (ret != SUCCESS) {
return ret;
}

RECORD_SHAPE_INFERENCE_EVENT(context_, node_item.NodeName().c_str(), "[CalcOpRunningParam] Start");
GE_CHK_STATUS_RET(NodeExecutorManager::GetInstance().CalcOpRunningParam(*node_item.node),
"[%s] Failed to invoke CalcOpRunningParam.", node_item.NodeName().c_str());
RECORD_SHAPE_INFERENCE_EVENT(context_, node_item.NodeName().c_str(), "[CalcOpRunningParam] End");
} else {
GELOGD("[%s] Skip static shape node", node_item.NodeName().c_str());
}

if (node_item.node_type != NETOUTPUT) {
GELOGI("[%s] Push to compile queue", node_item.NodeName().c_str());
// may block if full
auto node_state = context_->GetOrCreateNodeState(node_item.node);
complete_queue.Push(node_state);
}

// Propagate
RECORD_SHAPE_INFERENCE_EVENT(context_, node_item.NodeName().c_str(), "[PropagateOutputShapes] Start");
PropagateOutputShapes(*infer_state, ready_nodes);
RECORD_SHAPE_INFERENCE_EVENT(context_, node_item.NodeName().c_str(), "[PropagateOutputShapes] End");
}

return SUCCESS;
}

void ShapeInferenceEngine::InferenceDone(Status status) {
if (status != SUCCESS) {
GELOGE(status, "Error occurred while shape inference");
context_->OnError(status);
} else {
context_->compile_queue.Push(nullptr);
}
inference_states_.clear();
GELOGI("RuntimeShapeInferenceEngine worker END");
}

Status ShapeInferenceEngine::InferShape(InferenceState &entry) {
// input shapes are ready, wait for dependent data if has any
const auto &node_item = entry.node_item;
if (!node_item.dependent_node_list.empty()) {
for (auto &src_node : node_item.dependent_node_list) {
auto *src_node_item = context_->model->GetNodeItem(src_node);
GELOGI("[%s] Start to wait for data dependent node: %s", node_item.NodeName().c_str(),
src_node_item->NodeName().c_str());
RECORD_SHAPE_INFERENCE_EVENT(context_, node_item.NodeName().c_str(), "[AwaitNodeDone] [%s] Start",
src_node->GetName().c_str());
if (!context_->cv_manager.Await(src_node)) {
GELOGE(INTERNAL_ERROR, "[%s] Await node failed.", src_node_item->NodeName().c_str());
return INTERNAL_ERROR;
}

RECORD_SHAPE_INFERENCE_EVENT(context_, node_item.NodeName().c_str(), "[AwaitNodeDone] [%s] End",
src_node->GetName().c_str());
GELOGI("[%s] Done waiting node.", src_node_item->NodeName().c_str());
}
}

auto &node_item = *node_state.GetNodeItem();
// Skip shape inference for node of type DEPEND_COMPUTE
if (node_item.shape_inference_type == DEPEND_COMPUTE) {
GELOGD("[%s] Skipping node with unknown shape type DEPEND_COMPUTE", node_item.NodeName().c_str());
GELOGD("[%s] Skip node with unknown shape type DEPEND_COMPUTE", node_item.NodeName().c_str());
return SUCCESS;
}

// Clear shape range in case shape inference func forgot to do it
if (node_item.shape_inference_type == DEPEND_SHAPE_RANGE) {
// in case InferFunc forgot to reset output shape
for (auto &output_desc : node_item.op_desc->GetAllOutputsDescPtr()) {
@@ -43,16 +133,13 @@ Status ShapeInferenceEngine::InferShape(NodeState &node_state) {
}
}

// Wait for "const input nodes" if node's shape inference function requires any.
GE_CHK_STATUS_RET_NOLOG(AwaitDependentNodes(node_state));

// Do shape inference
// do shape inference
RECORD_SHAPE_INFERENCE_EVENT(context_, node_item.NodeName().c_str(), "[InferShape] Start");
GELOGD("[%s] Start to invoke InferShapeAndType", node_item.NodeName().c_str());
RECORD_SHAPE_INFERENCE_EVENT(execution_context_, node_item.NodeName().c_str(), "[InferShapeAndType] Start");
GE_CHK_STATUS_RET(ShapeRefiner::InferShapeAndType(node_item.node), "Invoke InferShapeAndType failed.");
RECORD_SHAPE_INFERENCE_EVENT(execution_context_, node_item.NodeName().c_str(), "[InferShapeAndType] End");
RECORD_SHAPE_INFERENCE_EVENT(context_, node_item.NodeName().c_str(), "[InferShape] End");

// Check again to make sure shape is valid after shape inference
// Check shape
if (node_item.shape_inference_type != DEPEND_SHAPE_RANGE) {
bool is_unknown_shape = false;
GE_CHK_STATUS_RET(NodeUtils::GetNodeUnknownShapeStatus(*node_item.node, is_unknown_shape),
@@ -62,33 +149,12 @@ Status ShapeInferenceEngine::InferShape(NodeState &node_state) {
node_item.NodeName().c_str());
}

GELOGD("[%s] [HybridTrace] After shape inference. Node = %s", node_item.NodeName().c_str(),
node_item.DebugString().c_str());

GELOGD("[%s] InferShapeAndType finished successfully.", node_item.NodeName().c_str());
return SUCCESS;
}

Status ShapeInferenceEngine::AwaitDependentNodes(NodeState &node_state) {
auto &node_item = *node_state.GetNodeItem();
for (auto &src_node : node_item.dependents_for_shape_inference) {
GELOGI("[%s] Start to wait for data dependent node: %s", node_item.NodeName().c_str(), src_node->GetName().c_str());
RECORD_SHAPE_INFERENCE_EVENT(execution_context_, node_item.NodeName().c_str(), "[AwaitNodeDone] [%s] Start",
src_node->GetName().c_str());
if (!subgraph_context_->Await(src_node)) {
GELOGE(INTERNAL_ERROR, "[%s] Await node failed.", src_node->GetName().c_str());
return INTERNAL_ERROR;
}

RECORD_SHAPE_INFERENCE_EVENT(execution_context_, node_item.NodeName().c_str(), "[AwaitNodeDone] [%s] End",
src_node->GetName().c_str());
GELOGI("[%s] Done waiting node.", src_node->GetName().c_str());
}

return SUCCESS;
}

Status ShapeInferenceEngine::PropagateOutputShapes(const NodeItem &node_item) {
void ShapeInferenceEngine::PropagateOutputShapes(InferenceState &entry, std::queue<InferenceState *> &queue) {
auto &node_item = entry.node_item;
// output shape will not be valid until compute is done.
bool shape_is_future =
node_item.shape_inference_type == DEPEND_SHAPE_RANGE || node_item.shape_inference_type == DEPEND_COMPUTE;
@@ -105,25 +171,88 @@ Status ShapeInferenceEngine::PropagateOutputShapes(const NodeItem &node_item) {
// propagate output to all sub-inputs
for (auto &dst_input_index_and_node : output_nodes) {
auto &dst_node_item = dst_input_index_and_node.second;
auto dst_node_state = subgraph_context_->GetOrCreateNodeState(dst_node_item);
GE_CHECK_NOTNULL(dst_node_state);

auto inference_state = GetOrCreateEntry(*dst_node_item);
GELOGI("[%s] Update dst node [%s], input index = %d", node_item.NodeName().c_str(),
dst_node_item->NodeName().c_str(), dst_input_index_and_node.first);

// in case type 3 and 4, shape will be valid after computing is done
// in case type 3/4, shape will be valid after computing is done
if (shape_is_future) {
ShapeFuture future(node_item.node, i, subgraph_context_);
dst_node_state->GetShapeInferenceState().UpdateInputShapeFuture(dst_input_index_and_node.first,
std::move(future));
ShapeFuture future(node_item.node, i, &context_->cv_manager);
inference_state->UpdateInputShapeFuture(dst_input_index_and_node.first, std::move(future));
} else {
dst_node_state->GetShapeInferenceState().UpdateInputShape(dst_input_index_and_node.first, ori_shape, shape);
inference_state->UpdateInputShape(dst_input_index_and_node.first, ori_shape, shape);
}

if (inference_state->IsInputShapesReady()) {
GELOGI("[%s] Node input shape is ready, add to queue.", inference_state->node_item.NodeName().c_str());
queue.emplace(inference_state);
}
}
}

GELOGD("[%s] Propagating output shapes finished successfully.", node_item.NodeName().c_str());
}

ShapeInferenceEngine::InferenceState *ShapeInferenceEngine::GetOrCreateEntry(const NodeItem &node_item) {
auto &node_state = inference_states_[node_item.node_id];
if (node_state == nullptr) {
node_state.reset(new (std::nothrow) InferenceState(node_item));
}

return node_state.get();
}

ShapeInferenceEngine::InferenceState::InferenceState(const NodeItem &node_item) : node_item(node_item) {
this->num_pending_shapes = node_item.num_inputs;
}

void ShapeInferenceEngine::InferenceState::UpdateInputShape(uint32_t idx, const GeShape &ori_shape,
const GeShape &shape) {
if (node_item.const_input_shapes.count(idx) != 0) {
GELOGD("[%s] Trying to update constant shape, idx = %u. old shape = [%s], new shape = [%s]",
node_item.NodeName().c_str(), idx, node_item.op_desc->MutableInputDesc(idx)->GetShape().ToString().c_str(),
shape.ToString().c_str());
}

GELOGD("[%s] Update input shape [%u] with Shape: [%s] and OriginalShape: [%s]", node_item.NodeName().c_str(), idx,
shape.ToString().c_str(), ori_shape.ToString().c_str());
num_pending_shapes -= 1;
node_item.op_desc->MutableInputDesc(idx)->SetShape(shape);
node_item.op_desc->MutableInputDesc(idx)->SetOriginShape(ori_shape);
}

void ShapeInferenceEngine::InferenceState::UpdateInputShapeFuture(uint32_t idx, ShapeFuture &&future) {
if (node_item.const_input_shapes.count(idx) != 0) {
GELOGE(INTERNAL_ERROR, "[%s] Trying to update constant shape, idx = %u", node_item.NodeName().c_str(), idx);
return;
}

GELOGD("[%s] Update input shape [%u] with ShapeFuture.", node_item.NodeName().c_str(), idx);
num_pending_shapes -= 1;
shape_futures.emplace_back(idx, std::move(future));
}

Status ShapeInferenceEngine::InferenceState::AwaitShapeFutures(GraphExecutionContext *context) {
for (auto &p : shape_futures) {
auto idx = p.first;
auto &future = p.second;
GeShape shape;
GeShape ori_shape;
RECORD_SHAPE_INFERENCE_EVENT(context, node_item.NodeName().c_str(), "[AwaitShape] [idx = %u] Start", idx);
GE_CHK_STATUS_RET(future.Get(ori_shape, shape), "[%s] Get shape failed. index = %u", node_item.NodeName().c_str(),
idx);
RECORD_SHAPE_INFERENCE_EVENT(context, node_item.NodeName().c_str(), "[AwaitShape] [idx = %u] End", idx);

GELOGD("[%s] Update input shape [%u] with shape: [%s] and ori_shape: [%s]", node_item.NodeName().c_str(), idx,
shape.ToString().c_str(), ori_shape.ToString().c_str());
node_item.op_desc->MutableInputDesc(idx)->SetShape(std::move(shape));
node_item.op_desc->MutableInputDesc(idx)->SetOriginShape(ori_shape);
}

return SUCCESS;
}

ShapeInferenceEngine::ShapeFuture::ShapeFuture(NodePtr src_node, uint32_t src_index, NodeDoneManager *node_done_manager)
: src_node_(std::move(src_node)), src_index_(src_index), node_done_manager_(node_done_manager) {}
} // namespace hybrid
} // namespace ge
} // namespace ge

+ 58
- 8
src/ge/hybrid/executor/worker/shape_inference_engine.h View File

@@ -17,25 +17,75 @@
#ifndef GE_HYBRID_EXECUTOR_INFERSHAPE_SHAPE_INFERENCE_ENGINE_H_
#define GE_HYBRID_EXECUTOR_INFERSHAPE_SHAPE_INFERENCE_ENGINE_H_

#include <memory>
#include <thread>
#include <unordered_map>
#include "common/thread_pool.h"
#include "hybrid/executor/hybrid_execution_context.h"
#include "hybrid/executor/subgraph_context.h"

namespace ge {
namespace hybrid {
class ShapeInferenceEngine {
public:
ShapeInferenceEngine(GraphExecutionContext *execution_context, SubgraphContext *subgraph_context);
~ShapeInferenceEngine() = default;
explicit ShapeInferenceEngine(GraphExecutionContext *context);

Status InferShape(NodeState &node_state);
~ShapeInferenceEngine() = default;

Status PropagateOutputShapes(const NodeItem &node_item);
Status Start(ThreadPool &pool);

private:
Status AwaitDependentNodes(NodeState &node_state);
class ShapeFuture {
public:
ShapeFuture(NodePtr src_node, uint32_t src_index, NodeDoneManager *node_done_manager);
~ShapeFuture() = default;
Status Get(GeShape &ori_shape, GeShape &shape) {
GELOGI("Start to wait node: %s for getting shape", src_node_->GetName().c_str());
if (!node_done_manager_->Await(src_node_)) {
GELOGE(INTERNAL_ERROR, "cancelled");
return INTERNAL_ERROR;
}

shape = src_node_->GetOpDesc()->MutableOutputDesc(src_index_)->MutableShape();
ori_shape = src_node_->GetOpDesc()->MutableOutputDesc(src_index_)->GetOriginShape();
GELOGI("Get shape from %s:%u. shape = [%s]", src_node_->GetName().c_str(), src_index_, shape.ToString().c_str());
return SUCCESS;
}

private:
NodePtr src_node_;
uint32_t src_index_;
NodeDoneManager *node_done_manager_;
};

struct InferenceState {
explicit InferenceState(const NodeItem &node_item);
inline bool IsInputShapesReady() const { return num_pending_shapes == 0; }

void UpdateInputShape(uint32_t idx, const GeShape &ori_shape, const GeShape &shape);

Status AwaitShapeFutures(GraphExecutionContext *context);

void UpdateInputShapeFuture(uint32_t idx, ShapeFuture &&future);

const NodeItem &node_item;

private:
std::vector<std::pair<uint32_t, ShapeFuture>> shape_futures;
int num_pending_shapes = 0;
};

InferenceState *GetOrCreateEntry(const NodeItem &node_item);

Status InferShapeProcess();

void InferenceDone(Status status);

Status InferShape(InferenceState &entry);

void PropagateOutputShapes(InferenceState &entry, std::queue<InferenceState *> &queue);

GraphExecutionContext *execution_context_;
SubgraphContext *subgraph_context_;
GraphExecutionContext *context_;
std::unordered_map<int64_t, std::unique_ptr<InferenceState>> inference_states_;
};
} // namespace hybrid
} // namespace ge


+ 160
- 10
src/ge/hybrid/executor/worker/task_compile_engine.cc View File

@@ -16,22 +16,172 @@

#include "hybrid/executor/worker/task_compile_engine.h"
#include "init/gelib.h"
#include "framework/common/debug/log.h"
#include "hybrid/node_executor/node_executor.h"

namespace ge {
namespace hybrid {
Status TaskCompileEngine::Compile(NodeState &node_state, GraphExecutionContext *context) {
const auto &node_item = *node_state.GetNodeItem();
RECORD_COMPILE_EVENT(context, node_item.NodeName().c_str(), "Start");
GE_CHK_RT_RET(rtCtxSetCurrent(context->rt_gen_context));

shared_ptr<NodeTask> kernel_task;
auto ret = node_item.node_executor->CompileTask(*context->model, node_item.node, kernel_task);
RECORD_COMPILE_EVENT(context, node_state.GetName().c_str(), "End");
namespace {
uint32_t kDefaultWorkerCnt = 4;
uint32_t kDefaultDeviceId = 0;
} // namespace
TaskCompileEngine::TaskCompileEngine(GraphExecutionContext *context) : context_(context), pool_(kDefaultWorkerCnt) {}

TaskCompileEngine::~TaskCompileEngine() {
if (rt_context_ != nullptr) {
GELOGD("To destroy compile context: %p.", rt_context_);
GE_CHK_RT(rtCtxDestroy(rt_context_));
}
}

Status TaskCompileEngine::Init() {
GELOGD("Start to init CompileEngine");
rtContext_t current_ctx = nullptr;
GE_CHK_RT(rtCtxGetCurrent(&current_ctx));
GE_CHK_RT_RET(rtCtxCreate(&rt_context_, RT_CTX_GEN_MODE, kDefaultDeviceId));
GELOGD("Context created for compiling. ctx = %p", rt_context_);
GE_CHK_RT_RET(rtCtxSetCurrent(current_ctx));
return SUCCESS;
}

void TaskCompileEngine::Reset() {
complete_queue_.Push(nullptr); // ensure iteration can stop
unique_ptr<ResultQueueEntry> entry;
while (true) {
complete_queue_.Pop(entry);
if (entry == nullptr) {
break;
}

if (entry->future != nullptr) {
entry->future->wait();
}
}

complete_queue_.Clear();
}

Status TaskCompileEngine::Start(ThreadPool &pool) {
pool.commit([&]() { (void)this->CompileProcess(); });

worker_future_ = pool_.commit([&]() -> Status { return this->DistributeCompiledTasks(); });

if (!worker_future_.valid()) {
GELOGE(INTERNAL_ERROR, "Failed to start worker thread");
return INTERNAL_ERROR;
}

return SUCCESS;
}

Status TaskCompileEngine::CompileProcess() {
auto &compile_queue = context_->compile_queue;
while (true) {
NodeStatePtr node_state;
// Stop() will not be invoked, Pop won't failed
(void)compile_queue.Pop(node_state);

// EOF
if (node_state == nullptr) {
GELOGD("Got EOF");
complete_queue_.Push(unique_ptr<ResultQueueEntry>());
break;
}

auto entry = unique_ptr<ResultQueueEntry>(new (std::nothrow) ResultQueueEntry());
GE_CHECK_NOTNULL(entry);
entry->node_state = node_state;

auto node_item = *node_state->node_item;
if (node_item.kernel_task != nullptr) {
GELOGD("use precompiled task. node name = %s", node_item.NodeName().c_str());
node_state->kernel_task = node_item.kernel_task;
complete_queue_.Push(std::move(entry));
continue;
}

auto ret = CompileAsync(*node_state->node_item, *entry);
if (ret == SUCCESS) {
complete_queue_.Push(std::move(entry));
continue;
}

// On Error
worker_future_.wait();
Reset();
return CompileDone(ret);
}

Status ret = worker_future_.get();
Reset();
return CompileDone(ret);
}

Status TaskCompileEngine::CompileDone(Status status) {
if (status != SUCCESS) {
GELOGE(status, "Error occurred while compiling node.");
context_->OnError(status);
} else {
context_->execution_queue.Push(nullptr);
}
GELOGI("CompileEngine worker END. ret = %u", status);
return status;
}

Status TaskCompileEngine::DoCompile(const NodeItem &node_item, NodeState &node_state) {
RECORD_COMPILE_EVENT(context_, node_state.GetName().c_str(), "Start");
GE_CHK_RT_RET(rtCtxSetCurrent(rt_context_));
auto ret = node_item.node_executor->CompileTask(*context_->model, node_item.node, node_state.kernel_task);
RECORD_COMPILE_EVENT(context_, node_state.GetName().c_str(), "End");
GE_CHK_STATUS_RET(ret, "Failed to create task for node: %s", node_item.NodeName().c_str());
node_state.SetKernelTask(kernel_task);
GELOGI("Compiling node %s successfully", node_state.GetName().c_str());
return SUCCESS;
}

Status TaskCompileEngine::CompileAsync(const NodeItem &node_item, ResultQueueEntry &entry) {
auto node_state = entry.node_state;
auto f = pool_.commit([this, node_item, node_state]() -> Status { return DoCompile(node_item, *node_state); });

if (!f.valid()) {
GELOGE(INTERNAL_ERROR, "Failed to commit compile task");
return INTERNAL_ERROR;
}

entry.future = unique_ptr<std::future<Status>>(new (std::nothrow) std::future<Status>(std::move(f)));
GE_CHECK_NOTNULL(entry.future);
return SUCCESS;
}

Status TaskCompileEngine::DistributeCompiledTasks() {
GELOGD("DistributeCompiledTasks start.");
auto &execute_queue = context_->execution_queue;
unique_ptr<ResultQueueEntry> entry;
bool ret = SUCCESS;
while (true) {
if (!complete_queue_.Pop(entry)) {
GELOGE(INTERNAL_ERROR, "Failed to pop item from queue");
ret = INTERNAL_ERROR;
break;
}

// EOF
if (entry == nullptr) {
break;
}

// if has compile future
if (entry->future != nullptr) {
ret = entry->future->get();
if (ret != SUCCESS) {
break;
}
}

execute_queue.Push(entry->node_state);
}

GELOGD("DistributeCompiledTasks out. ret = %u.", ret);
return ret;
}
} // namespace hybrid
} // namespace ge
} // namespace ge

+ 32
- 1
src/ge/hybrid/executor/worker/task_compile_engine.h View File

@@ -17,13 +17,44 @@
#ifndef GE_HYBRID_EXECUTOR_COMPILE_TASK_COMPILE_ENGINE_H_
#define GE_HYBRID_EXECUTOR_COMPILE_TASK_COMPILE_ENGINE_H_

#include <memory>
#include <thread>
#include "common/thread_pool.h"
#include "hybrid/executor/hybrid_execution_context.h"

namespace ge {
namespace hybrid {
class TaskCompileEngine {
public:
static Status Compile(NodeState &node_state, GraphExecutionContext *context);
explicit TaskCompileEngine(GraphExecutionContext *context);

~TaskCompileEngine();

Status Init();

Status Start(ThreadPool &pool);

private:
struct ResultQueueEntry {
NodeStatePtr node_state;
std::unique_ptr<std::future<Status>> future;
};

Status CompileProcess();

Status CompileDone(Status status);

private:
Status DoCompile(const NodeItem &node_item, NodeState &node_state);
Status CompileAsync(const NodeItem &node_item, ResultQueueEntry &entry);
Status DistributeCompiledTasks();
void Reset();

rtContext_t rt_context_ = nullptr;
GraphExecutionContext *context_;
BlockingQueue<unique_ptr<ResultQueueEntry>> complete_queue_;
ThreadPool pool_;
std::future<Status> worker_future_;
};
} // namespace hybrid
} // namespace ge


+ 1
- 12
src/ge/hybrid/hybrid_davinci_model.cc View File

@@ -18,7 +18,6 @@
#include "hybrid_davinci_model.h"
#include "hybrid/model/hybrid_model.h"
#include "hybrid/executor/hybrid_model_async_executor.h"
#include "hybrid/node_executor/node_executor.h"

namespace ge {
namespace hybrid {
@@ -26,19 +25,14 @@ class HybridDavinciModel::Impl {
public:
explicit Impl(GeRootModelPtr ge_model) : model_(std::move(ge_model)), executor_(&model_) {}

~Impl() { NodeExecutorManager::GetInstance().FinalizeExecutors(); }
~Impl() = default;

Status Init() {
GE_CHK_STATUS_RET(NodeExecutorManager::GetInstance().EnsureInitialized(), "Failed to initialize executors");
GE_CHK_STATUS_RET(model_.Init(), "Failed to init model.")
GE_CHK_STATUS_RET(executor_.Init(), "Failed to init model executor.")
return SUCCESS;
}

Status Execute(const vector<GeTensor> &inputs, vector<GeTensor> &outputs) {
return executor_.Execute(inputs, outputs);
}

Status ModelRunStart() { return executor_.Start(listener_); }

Status ModelRunStop() { return executor_.Stop(); }
@@ -82,11 +76,6 @@ Status HybridDavinciModel::Init() {
return impl_->Init();
}

Status HybridDavinciModel::Execute(const vector<GeTensor> &inputs, vector<GeTensor> &outputs) {
GE_CHECK_NOTNULL(impl_);
return impl_->Execute(inputs, outputs);
}

Status HybridDavinciModel::ModelRunStart() {
GE_CHECK_NOTNULL(impl_);
return impl_->ModelRunStart();


+ 0
- 2
src/ge/hybrid/hybrid_davinci_model.h View File

@@ -37,8 +37,6 @@ class HybridDavinciModel {

Status Init();

Status Execute(const vector<GeTensor> &inputs, vector<GeTensor> &outputs);

Status ModelRunStart();

Status ModelRunStop();


+ 0
- 2
src/ge/hybrid/hybrid_davinci_model_stub.cc View File

@@ -26,8 +26,6 @@ std::unique_ptr<HybridDavinciModel> HybridDavinciModel::Create(const GeRootModel

Status HybridDavinciModel::Init() { return UNSUPPORTED; }

Status HybridDavinciModel::Execute(const vector<GeTensor> &inputs, vector<GeTensor> &outputs) { return UNSUPPORTED; }

Status HybridDavinciModel::ModelRunStart() { return UNSUPPORTED; }

Status HybridDavinciModel::ModelRunStop() { return UNSUPPORTED; }


+ 0
- 62
src/ge/hybrid/model/graph_item.cc View File

@@ -1,62 +0,0 @@
/**
* Copyright 2019-2020 Huawei Technologies Co., Ltd
*
* 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 "framework/common/util.h"
#include "graph_item.h"

namespace ge {
namespace hybrid {
namespace {
constexpr int kInvalidIndex = -1;
} // namespace
GraphItem::~GraphItem() { GELOGD("[%s] GraphItem destroyed.", name_.c_str()); }

const vector<NodeItem *> &hybrid::GraphItem::GetAllNodes() const { return node_items_; }

const vector<const NodeItem *> &GraphItem::GetInputNodes() const { return input_nodes_; }

Status GraphItem::GetOutputDescList(vector<ConstGeTensorDescPtr> &output_desc_list) const {
if (is_dynamic_) {
for (auto &node_and_idx : output_edges_) {
const auto &tensor_desc = node_and_idx.first->op_desc->MutableOutputDesc(node_and_idx.second);
GE_CHECK_NOTNULL(tensor_desc);
output_desc_list.emplace_back(tensor_desc);
}
} else {
auto all_output_desc = output_node_->op_desc->GetAllOutputsDescPtr();
for (auto &tensor_desc : output_node_->op_desc->GetAllOutputsDescPtr()) {
output_desc_list.emplace_back(tensor_desc);
}
}

return SUCCESS;
}

bool GraphItem::IsDynamic() const { return is_dynamic_; }

const vector<int> &GraphItem::GetInputIndexMapping() const { return input_index_mapping_; }

int GraphItem::GetParentOutputIndex(size_t index) const {
if (index >= output_index_mapping_.size()) {
return kInvalidIndex;
}

return output_index_mapping_[index];
}

const NodeItem *GraphItem::GetOutputNode() const { return output_node_; }
} // namespace hybrid
} // namespace ge

+ 0
- 64
src/ge/hybrid/model/graph_item.h View File

@@ -1,64 +0,0 @@
/**
* Copyright 2019-2020 Huawei Technologies Co., Ltd
*
* 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.
*/

#ifndef GE_HYBRID_MODEL_SUBGRAPH_ITEM_H_
#define GE_HYBRID_MODEL_SUBGRAPH_ITEM_H_

#include "external/ge/ge_api_error_codes.h"
#include "hybrid/model/node_item.h"

namespace ge {
namespace hybrid {
class GraphItem {
public:
GraphItem() = default;
~GraphItem();
const vector<NodeItem *> &GetAllNodes() const;
const vector<const NodeItem *> &GetInputNodes() const;
Status GetOutputDescList(std::vector<ConstGeTensorDescPtr> &output_desc_list) const;

int TotalInputs() const { return total_inputs_; }

int TotalOutputs() const { return total_outputs_; }

const std::string &GetName() const { return name_; }

void SetName(const string &name) { name_ = name; }

const NodeItem *GetOutputNode() const;

bool IsDynamic() const;
int GetParentOutputIndex(size_t index) const;
const vector<int> &GetInputIndexMapping() const;

private:
friend class HybridModelBuilder;
std::string name_;
std::vector<NodeItem *> node_items_;
std::vector<const NodeItem *> input_nodes_;
const NodeItem *output_node_ = nullptr;
// <src_node, out_index>
std::vector<std::pair<const NodeItem *, int>> output_edges_;
int total_inputs_ = 0;
int total_outputs_ = 0;

bool is_dynamic_ = true;
std::vector<int> input_index_mapping_;
std::vector<int> output_index_mapping_;
};
} // namespace hybrid
} // namespace ge
#endif // GE_HYBRID_MODEL_SUBGRAPH_ITEM_H_

+ 28
- 33
src/ge/hybrid/model/hybrid_model.cc View File

@@ -29,8 +29,6 @@ namespace ge {
namespace hybrid {
HybridModel::HybridModel(GeRootModelPtr ge_model) : ge_root_model_(std::move(ge_model)) {}

HybridModel::~HybridModel() { GELOGD("[%s] HybridModel destroyed.", model_name_.c_str()); }

Status HybridModel::Init() {
GELOGD("Start to init hybrid model.");
GE_CHK_STATUS_RET(HybridModelBuilder(*this).Build(), "Failed to build hybrid model.");
@@ -38,6 +36,22 @@ Status HybridModel::Init() {
return SUCCESS;
}

void HybridModel::Print() const {
for (const auto &node : node_items_) {
GELOGD("%s", node->DebugString().c_str());
}
}

TensorValue *HybridModel::GetWeight(const NodeItem *const_node) const {
auto it = weights_.find(const_node->node_id);
if (it == weights_.end() || it->second == nullptr) {
GELOGE(INTERNAL_ERROR, "[%s] Failed to get weight", const_node->NodeName().c_str());
return nullptr;
}

return it->second.get();
}

TensorValue *HybridModel::GetVariable(const string &name) const {
auto it = variable_tensors_.find(name);
if (it == variable_tensors_.end()) {
@@ -69,26 +83,26 @@ const std::vector<domi::TaskDef> *HybridModel::GetTaskDefs(const NodePtr &node)
}

NodeItem *HybridModel::MutableNodeItem(const NodePtr &node) {
auto it = node_items_.find(node);
if (it == node_items_.end()) {
auto node_id = node->GetOpDesc()->GetId();
if (node_id < 0 || static_cast<size_t>(node_id) > node_items_.size()) {
GELOGE(INTERNAL_ERROR, "index out of range. node_id = %ld, num_nodes = %zu", node_id, node_items_.size());
return nullptr;
}

return it->second.get();
return node_items_[node_id].get();
}

const NodeItem *HybridModel::GetNodeItem(const NodePtr &node) const {
auto it = node_items_.find(node);
if (it == node_items_.end()) {
auto node_id = node->GetOpDesc()->GetId();
if (node_id < 0 || static_cast<size_t>(node_id) > node_items_.size()) {
GELOGE(INTERNAL_ERROR, "Index out of range. node_id = %ld, num_nodes = %zu.", node_id, node_items_.size());
return nullptr;
}

return it->second.get();
return node_items_[node_id].get();
}

GeModelPtr HybridModel::GetGeModel(const NodePtr &node) const {
auto it = known_shape_sub_models_.find(node);
if (it == known_shape_sub_models_.end()) {
auto it = known_shape_sub_graphs_.find(node);
if (it == known_shape_sub_graphs_.end()) {
GELOGE(INTERNAL_ERROR, "[%s] Failed to get GeModel for subgraph node.", node->GetName().c_str());
return nullptr;
}
@@ -96,27 +110,8 @@ GeModelPtr HybridModel::GetGeModel(const NodePtr &node) const {
return it->second;
}

const GraphItem *HybridModel::GetRootGraphItem() const { return root_graph_item_.get(); }

const GraphItem *HybridModel::GetSubgraphItem(const std::string &graph_name) const {
GELOGD("To find subgraph item by name = %s", graph_name.c_str());
auto it = subgraph_items_.find(graph_name);
if (it == subgraph_items_.end()) {
GELOGD("Subgraph item not found by node = %s", graph_name.c_str());
return nullptr;
}

return it->second.get();
}

const GraphItem *HybridModel::GetSubgraphItem(const ComputeGraphPtr &subgraph) const {
if (subgraph == nullptr) {
GELOGE(PARAM_INVALID, "subgraph is nullptr");
return nullptr;
}
const vector<int> &HybridModel::GetNetOutputInputOffsets() const { return net_output_input_offsets_; }

auto subgraph_name = subgraph->GetName();
return GetSubgraphItem(subgraph_name);
}
void HybridModel::SetDeviceId(uint32_t device_id) { device_id_ = device_id; }
} // namespace hybrid
} // namespace ge

+ 35
- 13
src/ge/hybrid/model/hybrid_model.h View File

@@ -26,23 +26,39 @@
#include "graph/node.h"
#include "hybrid/common/tensor_value.h"
#include "hybrid/model/node_item.h"
#include "hybrid/model/graph_item.h"
#include "model/ge_root_model.h"

namespace ge {
namespace hybrid {
class HybridModelAsyncExecutor;
class HybridModel {
public:
explicit HybridModel(GeRootModelPtr ge_model);

~HybridModel();
~HybridModel() = default;

Status Init();

const std::vector<NodeItem *> &RootNodes() const { return root_nodes_; }

const NodeItem *GetNodeItem(const NodePtr &node) const;

size_t NumNodes() const { return node_items_.size(); }

uint64_t GetSessionId() const { return root_runtime_param_.session_id; }

int TotalInputs() const { return total_inputs_; }

const map<uint32_t, NodeItem *> &GetInputNodes() const { return input_nodes_; }

const std::map<uint32_t, std::vector<int>> &GetInputOffsets() const { return input_offsets_; }

const vector<int> &GetNetOutputInputOffsets() const;

const std::vector<int> &GetOutputOffsets() const { return output_offsets_; }

const std::vector<NodeItem *> &GetConstNodes() const { return const_nodes_; }

GeModelPtr GetGeModel(const NodePtr &node) const;

NodeItem *MutableNodeItem(const NodePtr &node);
@@ -51,40 +67,46 @@ class HybridModel {

const uint8_t *GetVarMemBase() const { return var_mem_base_; }

void SetDeviceId(uint32_t device_id) { device_id_ = device_id; }
void SetDeviceId(uint32_t device_id);

void SetModelId(uint32_t model_id) { model_id_ = model_id; }

uint32_t GetModelId() const { return model_id_; }

TensorValue *GetWeight(const NodeItem *const_node) const;

TensorValue *GetVariable(const string &name) const;

NodePtr GetVariableNode(const string &name) const;

const std::vector<domi::TaskDef> *GetTaskDefs(const NodePtr &node) const;

const GraphItem *GetRootGraphItem() const;
int TotalOutputs() const { return total_outputs_; }

const GraphItem *GetSubgraphItem(const std::string &graph_name) const;

const GraphItem *GetSubgraphItem(const ComputeGraphPtr &subgraph) const;
GeRootModelPtr GetGeRootModel() const { return ge_root_model_; }
void Print() const;

private:
friend class HybridModelBuilder;
friend class HybridModelAsyncExecutor;

std::string model_name_;
GeRootModelPtr ge_root_model_;
std::vector<NodeItem *> root_nodes_;
std::map<uint32_t, NodeItem *> input_nodes_;
std::map<uint32_t, std::vector<int>> input_offsets_;
std::vector<int> output_offsets_;
std::vector<int> net_output_input_offsets_;
NodeItem *net_output_node_ = nullptr;
std::vector<std::unique_ptr<NodeItem>> node_items_;
std::vector<NodeItem *> const_nodes_;
std::map<std::string, NodePtr> constant_op_nodes_;
std::map<std::string, NodePtr> variable_nodes_;
std::map<std::string, std::unique_ptr<TensorValue>> variable_tensors_;
std::map<int, std::unique_ptr<TensorValue>> weights_;
std::map<NodePtr, std::vector<domi::TaskDef>> task_defs_;
std::map<NodePtr, GeModelPtr> known_shape_sub_models_;

std::unique_ptr<GraphItem> root_graph_item_;
std::map<std::string, std::unique_ptr<GraphItem>> subgraph_items_;
std::map<NodePtr, std::unique_ptr<NodeItem>> node_items_;
std::map<NodePtr, GeModelPtr> known_shape_sub_graphs_;
int total_inputs_ = 0;
int total_outputs_ = 0;

// runtime fields
uint32_t device_id_ = 0;


+ 154
- 422
src/ge/hybrid/model/hybrid_model_builder.cc View File

@@ -23,6 +23,7 @@
#include "graph/manager/trans_var_data_utils.h"
#include "graph/utils/graph_utils.h"
#include "graph/utils/type_utils.h"
#include "framework/common/debug/log.h"
#include "hybrid/common/npu_memory_allocator.h"
#include "hybrid/node_executor/node_executor.h"

@@ -31,7 +32,6 @@ namespace hybrid {
namespace {
const uint32_t kSubgraphIndex = 0U;
const uint32_t kVarOutputIndex = 0U;
const uint32_t kAlignment = 32;
const int kBytes = 8;

int64_t CalcVarSizeInBytes(const GeTensorDesc &desc) {
@@ -46,9 +46,6 @@ int64_t CalcVarSizeInBytes(const GeTensorDesc &desc) {
for (size_t dim_index = 0; dim_index < dim_num; ++dim_index) {
var_size *= shape.GetDim(dim_index);
}

// padding up to multiple of kAlignment, and add extra kAlignment
var_size = (var_size + kAlignment * 2 - 1) / kAlignment * kAlignment;
return var_size;
}
} // namespace
@@ -59,19 +56,20 @@ HybridModelBuilder::HybridModelBuilder(HybridModel &hybrid_model)

Status HybridModelBuilder::Build() {
GE_CHK_STATUS_RET(ValidateParams(), "Failed to validate GeRootModel");
hybrid_model_.model_name_ = ge_root_model_->GetRootGraph()->GetName();
graph_name_ = ge_root_model_->GetRootGraph()->GetName();
GELOGI("[%s] Start to build hybrid model.", GetGraphName());
GE_CHK_STATUS_RET(InitRuntimeParams(), "[%s] Failed to InitRuntimeParams", GetGraphName());
GE_CHK_STATUS_RET(NodeExecutorManager::GetInstance().EnsureInitialized(), "Failed to initialize executors");
GE_CHK_STATUS_RET(IndexSpecialNodes(), "[%s] Failed to index nodes", GetGraphName());
GE_CHK_STATUS_RET(IndexTaskDefs(), "[%s] Failed to index task defs", GetGraphName());
GE_CHK_STATUS_RET(LoadGraph(), "[%s] Failed to load graph", GetGraphName());
GE_CHK_STATUS_RET(AssignUninitializedConstantOps(), "[%s] Failed to assign uninitialized constants", GetGraphName());
GE_CHK_STATUS_RET(TransAllVarData(), "[%s] Failed to trans all var data", GetGraphName());
GE_CHK_STATUS_RET(CopyVarData(), "[%s] Failed to copy var data", GetGraphName());
GE_CHK_STATUS_RET(InitModelMem(), "[%s] Failed to init memory", GetGraphName());
GE_CHK_STATUS_RET(InitWeights(), "[%s] Failed to init weights", GetGraphName());
GE_CHK_STATUS_RET(InitConstantOps(), "[%s] Failed to init constant op", GetGraphName());
GE_CHK_STATUS_RET(InitVariableTensors(), "[%s] Failed to init variables", GetGraphName());
GE_CHK_STATUS_RET(ResolveRootNodes(), "[%s] Failed to resolve root nodes", GetGraphName());
GE_CHK_STATUS_RET(LoadTasks(), "[%s] Failed to load tasks", GetGraphName());
GELOGI("[%s] Done building hybrid model successfully.", GetGraphName());
return SUCCESS;
@@ -83,17 +81,45 @@ Status HybridModelBuilder::ValidateParams() {
return SUCCESS;
}

Status HybridModelBuilder::BuildNodeItem(const NodePtr &node, NodeItem &node_item) {
Status HybridModelBuilder::ResolveRootNodes() {
for (auto &node : hybrid_model_.node_items_) {
if (node->node->GetInDataNodes().empty()) {
hybrid_model_.root_nodes_.emplace_back(node.get());
GELOGI("[%s] Root node added. node name = %s", GetGraphName(), node->NodeName().c_str());
}
}

if (hybrid_model_.root_nodes_.empty()) {
GELOGE(PARAM_INVALID, "[%s] Root nodes is empty.", GetGraphName());
return PARAM_INVALID;
}

return SUCCESS;
}

Status HybridModelBuilder::BuildNoteItem(const NodePtr &node, NodeItem &node_item) {
GE_CHK_STATUS_RET(NodeUtils::GetNodeUnknownShapeStatus(*node, node_item.is_dynamic),
"[%s] Failed to get shape status.", node->GetName().c_str());

auto op_desc = node->GetOpDesc();
vector<string> dependencies = node->GetOpDesc()->GetOpInferDepends();
GE_CHK_STATUS_RET(ParseDependentInputNodes(node_item, dependencies), "[%s] Failed to parse node dependencies.",
node_item.NodeName().c_str());

auto it = node_ref_inputs_.find(node);
if (it != node_ref_inputs_.end()) {
for (auto &idx_and_node : it->second) {
// var and constant only have one output
node_item.const_input_shapes[idx_and_node.first] =
idx_and_node.second->GetOpDesc()->MutableOutputDesc(kVarOutputIndex);
}
}

node_item.outputs.resize(node_item.num_outputs);
for (int i = 0; i < node_item.num_outputs; ++i) {
auto out_data_anchor = node->GetOutDataAnchor(i);
if (out_data_anchor == nullptr) {
GELOGE(INTERNAL_ERROR, "out anchor[%d] of node %s is nullptr", i, node->GetName().c_str());
GELOGE(INTERNAL_ERROR, "out anchor[%zu] of node %s is nullptr", i, node->GetName().c_str());
return INTERNAL_ERROR;
}

@@ -111,46 +137,27 @@ Status HybridModelBuilder::BuildNodeItem(const NodePtr &node, NodeItem &node_ite
}
}

GE_CHK_STATUS_RET_NOLOG(ResolveRefIo(node_item));
return SUCCESS;
}

Status HybridModelBuilder::ResolveRefIo(NodeItem &node_item) {
bool is_ref = false;
auto &op_desc = *node_item.op_desc;
(void)AttrUtils::GetBool(op_desc, ATTR_NAME_REFERENCE, is_ref);
if (!is_ref) {
return SUCCESS;
}

auto inputs = op_desc.GetAllInputName();
auto outputs = op_desc.GetAllOutputName();
for (auto &output : outputs) {
for (auto &input : inputs) {
if (input.first == output.first) {
auto input_idx = static_cast<int>(input.second);
auto output_idx = static_cast<int>(output.second);
node_item.reuse_inputs[output_idx] = input_idx;
GELOGD("[%s] Output[%d] reuse input[%d]", node_item.NodeName().c_str(), output_idx, input_idx);
}
}
}

return SUCCESS;
}

Status HybridModelBuilder::GetOrCreateNodeItem(const NodePtr &node, NodeItem **node_item) {
auto &node_items = hybrid_model_.node_items_;
auto it = node_items.find(node);
if (it != node_items.end()) {
*node_item = it->second.get();
auto node_id = node->GetOpDesc()->GetId();
if (node_id < 0 || static_cast<size_t>(node_id) > node_items.size()) {
GELOGE(INTERNAL_ERROR, "[%s] Index out of range. node_id = %ld, num_nodes = %zu", node->GetName().c_str(), node_id,
node_items.size());
return INTERNAL_ERROR;
}

auto &node_ptr = node_items[node_id];
if (node_ptr != nullptr) {
*node_item = node_ptr.get();
return SUCCESS;
}

auto new_node = std::unique_ptr<NodeItem>(new (std::nothrow) NodeItem(node));
GE_CHECK_NOTNULL(new_node);
GE_CHECK_NOTNULL(new_node->op_desc);
GE_CHK_STATUS_RET(new_node->Init(), "Failed to init NodeItem [%s] .", node->GetName().c_str());
GE_CHK_STATUS_RET_NOLOG(NodeExecutorManager::GetInstance().GetExecutor(*node, &new_node->node_executor));

// we do not need L2 Buffer
@@ -162,58 +169,18 @@ Status HybridModelBuilder::GetOrCreateNodeItem(const NodePtr &node, NodeItem **n
int32_t unknown_shape_type_val = 0;
(void)AttrUtils::GetInt(new_node->op_desc, ::ge::ATTR_NAME_UNKNOWN_SHAPE_TYPE, unknown_shape_type_val);
new_node->shape_inference_type = static_cast<UnknowShapeOpType>(unknown_shape_type_val);

GE_CHK_STATUS_RET(NodeUtils::GetNodeUnknownShapeStatus(*node, new_node->is_dynamic),
"[%s] Failed to get shape status.", node->GetName().c_str());

if (new_node->is_dynamic && (new_node->IsControlOp() || new_node->NodeType() == PARTITIONEDCALL)) {
new_node->shape_inference_type = DEPEND_COMPUTE;
if (new_node->shape_inference_type == DEPEND_SHAPE_RANGE || new_node->shape_inference_type == DEPEND_COMPUTE) {
new_node->has_observer = true;
}

new_node->node_id = node_index;
new_node->op_desc->SetId(node_index);
node_index += 1;

*node_item = new_node.get();
node_items[node] = std::move(new_node);
node_items[node_id] = std::move(new_node);
return SUCCESS;
}

Status HybridModelBuilder::ParseDependentInputNodes(NodeItem &node_item, const std::vector<string> &dependencies) {
std::set<NodePtr> dependent_input_nodes;
auto &ge_node = node_item.node;

// The input tensors become valid after computation is done for parent nodes of type DEPEND_COMPUTE.
// Wait for these parent nodes before execution.
for (const auto &in_anchor : ge_node->GetAllInDataAnchors()) {
const auto &peer_anchor = in_anchor->GetPeerOutAnchor();
if (peer_anchor == nullptr) {
GELOGD("[%s] Input[%d] do not have peer anchor", node_item.NodeName().c_str(), in_anchor->GetIdx());
continue;
}

auto src_node = peer_anchor->GetOwnerNode();
GE_CHECK_NOTNULL(src_node);

auto src_node_item = MutableNodeItem(src_node);
GE_CHECK_NOTNULL(src_node_item);

if (src_node_item->shape_inference_type == DEPEND_COMPUTE) {
GELOGD("[%s] Add input data dependent node [%s] due to inference type = DEPEND_COMPUTE",
node_item.NodeName().c_str(), src_node_item->NodeName().c_str());

src_node_item->has_observer = true;
node_item.dependents_for_execution.emplace_back(src_node);
}

if (src_node_item->shape_inference_type == DEPEND_SHAPE_RANGE) {
GELOGD("[%s] Add input shape dependent node [%s] due to inference type = DEPEND_SHAPE_RANGE",
node_item.NodeName().c_str(), src_node_item->NodeName().c_str());
src_node_item->has_observer = true;
dependent_input_nodes.emplace(src_node);
}
}

for (const auto &input_name : dependencies) {
int input_index = node_item.op_desc->GetInputIndexByName(input_name);
if (input_index < 0) {
@@ -238,7 +205,7 @@ Status HybridModelBuilder::ParseDependentInputNodes(NodeItem &node_item, const s
}

for (const auto &dep_node : dependent_input_nodes) {
node_item.dependents_for_shape_inference.emplace_back(dep_node);
node_item.dependent_node_list.emplace_back(dep_node);
}

return SUCCESS;
@@ -295,14 +262,9 @@ Status HybridModelBuilder::DoLinkDataAnchors(OutDataAnchorPtr &out_data_anchor,

Status HybridModelBuilder::MergeInputNodes(ComputeGraph &graph) {
const auto &wrapped_node = graph.GetParentNode();
std::set<NodePtr> root_nodes;
for (const auto &node : graph.GetDirectNode()) {
GE_CHECK_NOTNULL(node);
if (node->GetType() != DATA_TYPE) {
if (node->GetInDataNodes().empty()) {
root_nodes.emplace(node);
}

continue;
}

@@ -329,28 +291,12 @@ Status HybridModelBuilder::MergeInputNodes(ComputeGraph &graph) {
for (auto &out_data_anchor : node->GetAllOutDataAnchors()) {
GE_CHECK_NOTNULL(out_data_anchor);
for (auto &peer_in_data_anchor : out_data_anchor->GetPeerInDataAnchors()) {
auto dst_node = peer_in_data_anchor->GetOwnerNode();
root_nodes.emplace(dst_node);
GE_CHK_STATUS_RET_NOLOG(DoUnlinkDataAnchors(out_data_anchor, peer_in_data_anchor));
GE_CHK_STATUS_RET_NOLOG(DoLinkDataAnchors(src_out_anchor, peer_in_data_anchor));
}
}
}

// transfer in control edges to all root nodes
for (auto &root_node : root_nodes) {
auto in_nodes = root_node->GetInAllNodes();
std::set<NodePtr> in_node_set(in_nodes.begin(), in_nodes.end());
for (auto &in_control_node : wrapped_node->GetInControlNodes()) {
if (in_node_set.count(in_control_node) == 0) {
GELOGD("[%s] Restore control edge to [%s]", in_control_node->GetName().c_str(), root_node->GetName().c_str());
GE_CHECK_NOTNULL(in_control_node->GetOutControlAnchor());
(void)in_control_node->GetOutControlAnchor()->LinkTo(root_node->GetInControlAnchor());
}
}
}

wrapped_node->GetInControlAnchor()->UnlinkAll();
return SUCCESS;
}

@@ -361,11 +307,6 @@ Status HybridModelBuilder::MergeNetOutputNode(ComputeGraph &graph) {
const auto &net_output_desc = net_output_node->GetOpDesc();
GE_CHECK_NOTNULL(net_output_desc);

auto all_in_nodes = net_output_node->GetInAllNodes();
auto all_out_nodes = parent_node->GetOutAllNodes();
net_output_node->GetInControlAnchor()->UnlinkAll();
parent_node->GetOutControlAnchor()->UnlinkAll();

for (const auto &in_data_anchor : net_output_node->GetAllInDataAnchors()) {
auto src_out_anchor = in_data_anchor->GetPeerOutAnchor();
GE_CHECK_NOTNULL(src_out_anchor);
@@ -397,25 +338,10 @@ Status HybridModelBuilder::MergeNetOutputNode(ComputeGraph &graph) {
}
}

// transfer out control edges
std::set<NodePtr> in_node_set(all_in_nodes.begin(), all_in_nodes.end());
std::set<NodePtr> out_node_set(all_out_nodes.begin(), all_out_nodes.end());
for (auto &src_node : in_node_set) {
GELOGD("[%s] process in node.", src_node->GetName().c_str());
auto out_nodes = src_node->GetOutAllNodes();
std::set<NodePtr> node_set(out_nodes.begin(), out_nodes.end());
for (auto &dst_node : out_node_set) {
if (node_set.count(dst_node) == 0) {
src_node->GetOutControlAnchor()->LinkTo(dst_node->GetInControlAnchor());
GELOGD("[%s] Restore control edge to [%s]", src_node->GetName().c_str(), dst_node->GetName().c_str());
}
}
}

return SUCCESS;
}

Status HybridModelBuilder::UnfoldSubgraphs(ComputeGraph &root_graph, ComputeGraphPtr &merged_graph) {
Status HybridModelBuilder::MergeSubgraphs(ComputeGraph &root_graph, ComputeGraphPtr &merged_graph) {
merged_graph = MakeShared<ComputeGraph>("MergedGraph");
for (const auto &node : root_graph.GetDirectNode()) {
GE_CHECK_NOTNULL(node);
@@ -445,74 +371,32 @@ Status HybridModelBuilder::UnfoldSubgraphs(ComputeGraph &root_graph, ComputeGrap
}

auto subgraph = NodeUtils::GetSubgraph(*node, kSubgraphIndex);
GE_CHECK_NOTNULL(subgraph);
GE_CHK_GRAPH_STATUS_RET(UnfoldSubgraph(root_graph, *merged_graph, *subgraph), "[%s] Failed to merge subgraph.",
subgraph->GetName().c_str());
}

// invoke before adding subgraphs. in case modify node id in known-shaped subgraphs.
GE_CHK_GRAPH_STATUS_RET(merged_graph->TopologicalSorting(), "Failed to invoke TopologicalSorting on merged graph.");

for (auto &remained_subgraph : root_graph.GetAllSubgraphs()) {
GELOGD("Adding subgraph [%s] to merged-graph.", remained_subgraph->GetName().c_str());
GE_CHK_GRAPH_STATUS_RET(merged_graph->AddSubgraph(remained_subgraph), "Failed to add subgraph [%s]",
remained_subgraph->GetName().c_str());
}

return SUCCESS;
}

Status HybridModelBuilder::UnfoldSubgraph(ComputeGraph &root_graph, ComputeGraph &parent_graph,
ComputeGraph &sub_graph) {
auto parent_node = sub_graph.GetParentNode();
GE_CHECK_NOTNULL(parent_node);

GE_CHK_STATUS_RET(MergeInputNodes(sub_graph), "[%s] Failed to merge data nodes for subgraph",
sub_graph.GetName().c_str());
GE_CHK_STATUS_RET(MergeNetOutputNode(sub_graph), "[%s] Failed to merge net output nodes for subgraph",
sub_graph.GetName().c_str());
GELOGD("[%s] Done merging subgraph inputs and outputs successfully.", sub_graph.GetName().c_str());

for (auto &sub_node : sub_graph.GetDirectNode()) {
auto sub_op_type = sub_node->GetType();
if (sub_op_type == DATA_TYPE || sub_op_type == NETOUTPUT) {
continue;
}

if (sub_op_type == CONSTANT || sub_op_type == VARIABLE) {
GELOGE(INTERNAL_ERROR, "Unexpected node in unknown subgraph. type = %s, node = %s::%s", sub_op_type.c_str(),
sub_graph.GetName().c_str(), sub_node->GetName().c_str());
return INTERNAL_ERROR;
}

if (sub_op_type == PARTITIONEDCALL) {
bool is_unknown_shape = false;
GE_CHK_GRAPH_STATUS_RET(NodeUtils::GetNodeUnknownShapeStatus(*sub_node, is_unknown_shape),
"[%s] Failed to invoke GetNodeUnknownShapeStatus.", sub_node->GetName().c_str());
if (is_unknown_shape) {
auto sub_sub_graph = NodeUtils::GetSubgraph(*sub_node, kSubgraphIndex);
GE_CHECK_NOTNULL(sub_sub_graph);
GE_CHK_STATUS_RET(UnfoldSubgraph(root_graph, parent_graph, *sub_sub_graph), "[%s] Failed to merge subgraph",
sub_sub_graph->GetName().c_str());
GE_CHK_STATUS_RET(MergeInputNodes(*subgraph), "Failed to merge data nodes for subgraph: %s",
subgraph->GetName().c_str());
GE_CHK_STATUS_RET(MergeNetOutputNode(*subgraph), "Failed to merge net output nodes for subgraph: %s",
subgraph->GetName().c_str());
GELOGD("Merging subgraph %s successfully.", subgraph->GetName().c_str());
for (auto &sub_node : subgraph->GetAllNodes()) {
auto sub_op_type = sub_node->GetType();
if (sub_op_type == DATA_TYPE || sub_op_type == NETOUTPUT) {
continue;
}
}

parent_graph.AddNode(sub_node);
GELOGD("[%s::%s] added to parent graph: [%s].", sub_graph.GetName().c_str(), sub_node->GetName().c_str(),
parent_graph.GetName().c_str());
if (sub_op_type == CONSTANT || sub_op_type == CONSTANTOP || sub_op_type == VARIABLE) {
GELOGE(INTERNAL_ERROR, "Unexpected node in unknown subgraph. type = %s, node = %s::%s", sub_op_type.c_str(),
subgraph->GetName().c_str(), sub_node->GetName().c_str());
return INTERNAL_ERROR;
}

merged_graph->AddNode(sub_node);
GELOGD("%s::%s added to merged graph.", subgraph->GetName().c_str(), sub_node->GetName().c_str());
}
}

GELOGD("[%s] Done merging subgraph. remove it from root graph.", sub_graph.GetName().c_str());
root_graph.RemoveSubgraph(sub_graph.GetName());
return SUCCESS;
}

Status HybridModelBuilder::BuildOutputMapping(GraphItem &graph_item, const NodeItem &node_item, bool is_root_graph) {
auto output_size = node_item.op_desc->GetAllInputsSize();
GE_CHECK_LE(output_size, UINT32_MAX);
graph_item.output_edges_.resize(output_size);

Status HybridModelBuilder::ParseNetOutput(const NodeItem &node_item) {
for (auto &in_data_anchor : node_item.node->GetAllInDataAnchors()) {
auto peer_out_anchor = in_data_anchor->GetPeerOutAnchor();
GE_CHECK_NOTNULL(peer_out_anchor);
@@ -524,20 +408,11 @@ Status HybridModelBuilder::BuildOutputMapping(GraphItem &graph_item, const NodeI
auto output_offset = src_node_item->output_start + peer_out_anchor->GetIdx();
GELOGI("Output[%d], node = %s, output_index = %d, output_offset = %d ", in_data_anchor->GetIdx(),
src_node_item->NodeName().c_str(), peer_out_anchor->GetIdx(), output_offset);

graph_item.output_edges_[in_data_anchor->GetIdx()] = {src_node_item, peer_out_anchor->GetIdx()};
hybrid_model_.output_offsets_.emplace_back(output_offset);
}

if (!is_root_graph) {
for (uint32_t i = 0; i < static_cast<uint32_t>(output_size); ++i) {
uint32_t p_index = i;
// Net output of Subgraph of while do not have parent index
if (AttrUtils::GetInt(node_item.op_desc->GetInputDesc(i), ATTR_NAME_PARENT_NODE_INDEX, p_index)) {
GELOGD("[%s] Parent index not set for input[%u].", node_item.NodeName().c_str(), i);
}

graph_item.output_index_mapping_.emplace_back(p_index);
}
for (int i = 0; i < node_item.num_inputs; ++i) {
hybrid_model_.net_output_input_offsets_.emplace_back(node_item.input_start + i);
}

return SUCCESS;
@@ -545,37 +420,82 @@ Status HybridModelBuilder::BuildOutputMapping(GraphItem &graph_item, const NodeI

Status HybridModelBuilder::LoadGraph() {
auto root_graph = ge_root_model_->GetRootGraph();
GE_CHK_STATUS_RET(LoadDynamicSubgraph(*root_graph, true), "Failed to load root graph.");
GELOGD("Done loading root graph successfully.");

for (auto &sub_graph : root_graph->GetAllSubgraphs()) {
GE_CHECK_NOTNULL(sub_graph);
GELOGD("Start to load subgraph [%s]", sub_graph->GetName().c_str());
auto parent_node = sub_graph->GetParentNode();
GE_CHECK_NOTNULL(parent_node);
auto parent_node_item = MutableNodeItem(parent_node);
// parent node is in another known subgraph
if (parent_node_item == nullptr) {
GELOGD("[%s] Subgraph is in another known shaped subgraph, skip it.", sub_graph->GetName().c_str());
continue;
GELOGI("Before merge subgraphs DirectNodesSize = %zu, GetAllNodesSize = %zu", root_graph->GetDirectNodesSize(),
root_graph->GetAllNodesSize());
ComputeGraphPtr merged_graph;
GE_CHK_STATUS_RET_NOLOG(MergeSubgraphs(*root_graph, merged_graph));
GELOGI("After merge subgraphs DirectNodesSize = %zu, GetAllNodesSize = %zu", merged_graph->GetDirectNodesSize(),
merged_graph->GetAllNodesSize());

merged_graph->SetGraphID(runtime_param_.graph_id);
GE_DUMP(merged_graph, "hybrid_merged_graph");
int input_start = 0;
int output_start = 0;
uint32_t data_op_index = 0;
hybrid_model_.node_items_.resize(merged_graph->GetDirectNodesSize());

int64_t node_index = 0;
for (auto &node : merged_graph->GetDirectNode()) {
OpDescPtr op_desc = node->GetOpDesc();
GE_CHECK_NOTNULL(op_desc);
op_desc->SetId(node_index++);
}

for (const auto &node : merged_graph->GetDirectNode()) {
GE_CHECK_NOTNULL(node);
GE_CHECK_NOTNULL(node->GetOpDesc());
const auto &op_type = node->GetType();

NodeItem *node_item = nullptr;
GE_CHK_STATUS_RET_NOLOG(GetOrCreateNodeItem(node, &node_item));
GE_CHK_STATUS_RET_NOLOG(BuildNoteItem(node, *node_item));
GE_CHK_STATUS_RET_NOLOG(UpdateAnchorStatus(node)); // needed by FE generate task

node_item->input_start = input_start;
node_item->output_start = output_start;
input_start += node_item->num_inputs;
output_start += node_item->num_outputs;

if (op_type == DATA_TYPE || op_type == AIPP_DATA_TYPE) {
auto data_index = data_op_index;
if (AttrUtils::GetInt(node->GetOpDesc(), ATTR_NAME_INDEX, data_index)) {
GELOGI("ge_train: get new index %u, old %u", data_index, data_op_index);
}
hybrid_model_.input_nodes_.emplace(data_index, node_item);
data_op_index++;
} else if (op_type == NETOUTPUT) {
hybrid_model_.net_output_node_ = node_item;
GE_CHK_STATUS_RET_NOLOG(ParseNetOutput(*node_item));
} else if (op_type == PARTITIONEDCALL) { // known graph
GE_CHK_STATUS_RET_NOLOG(ParsePartitionedCall(*node_item));
}

if (sub_graph->GetGraphUnknownFlag()) {
GE_CHK_STATUS_RET(LoadDynamicSubgraph(*sub_graph, false), "Failed to load subgraph: [%s]",
sub_graph->GetName().c_str());
} else {
GE_CHK_STATUS_RET(IdentifyVariableOutputs(*parent_node_item), "[%s] Failed to identify ref outputs.",
parent_node_item->NodeName().c_str());
GELOGI("NodeItem created: %s", node_item->DebugString().c_str());
}

// if parent is function control op. need add a virtual partitioned call
if (parent_node_item->IsControlOp()) {
GE_CHK_STATUS_RET(LoadKnownShapedSubgraph(*sub_graph, parent_node_item),
"Failed to load function control op subgraph [%s]", sub_graph->GetName().c_str());
for (auto &it : hybrid_model_.input_nodes_) {
auto input_index = it.first;
auto input_node = it.second;

if (input_node->outputs.empty()) {
GELOGE(INTERNAL_ERROR, "data output anchor is empty");
return INTERNAL_ERROR;
}

for (auto &out : input_node->outputs) {
std::vector<int> offsets;
for (auto &dst_anchor_and_node : out) {
auto dst_node_item = dst_anchor_and_node.second;
offsets.emplace_back(dst_node_item->input_start + dst_anchor_and_node.first);
}

hybrid_model_.input_offsets_.emplace(input_index, std::move(offsets));
}
}

GELOGI("Done loading all subgraphs successfully.");
hybrid_model_.total_inputs_ = input_start;
hybrid_model_.total_outputs_ = output_start;
GELOGI("HybridGraph::LoadGraph OUT");
return SUCCESS;
}

@@ -587,6 +507,7 @@ Status HybridModelBuilder::VarNodeToTensor(const NodePtr &var_node, std::unique_
string var_name = var_node->GetName();
auto tensor_desc = var_node->GetOpDesc()->MutableOutputDesc(0);
uint8_t *var_logic = nullptr;

GE_CHK_STATUS_RET(var_manager_->GetVarAddr(var_name, *tensor_desc, &var_logic),
"Failed to get var addr. var_name = %s, session_id = %ld", var_name.c_str(),
hybrid_model_.GetSessionId());
@@ -638,26 +559,10 @@ Status HybridModelBuilder::HandleDtString(const GeTensor &tensor, void *var_addr
return SUCCESS;
}

Status HybridModelBuilder::AssignUninitializedConstantOps() {
for (auto &it : hybrid_model_.constant_op_nodes_) {
const string &var_name = it.first;
const NodePtr &var_node = it.second;
auto tensor_desc = var_node->GetOpDesc()->MutableOutputDesc(0);
if (!var_manager_->IsVarExist(var_name, *tensor_desc)) {
// allocate constant
GELOGD("[%s] Constant not allocated during graph building. now allocate it.", var_name.c_str());
GE_CHK_STATUS_RET(var_manager_->AssignVarMem(var_name, *tensor_desc, RT_MEMORY_HBM));
GE_CHK_STATUS_RET(var_manager_->SetAllocatedGraphId(var_name, runtime_param_.graph_id));
}
}

return SUCCESS;
}

Status HybridModelBuilder::InitConstantOps() {
for (auto &it : hybrid_model_.constant_op_nodes_) {
const string &var_name = it.first;
const NodePtr &var_node = it.second;
string var_name = it.first;
NodePtr &var_node = it.second;
std::unique_ptr<TensorValue> var_tensor;

GE_CHK_STATUS_RET_NOLOG(VarNodeToTensor(var_node, var_tensor));
@@ -673,7 +578,7 @@ Status HybridModelBuilder::InitConstantOps() {
if (ge_tensor->GetData().size() > 0) {
GE_CHK_STATUS_RET_NOLOG(HandleDtString(*ge_tensor, v_output_addr));

GELOGI("[IMAS]InitConstant memcpy graph_%u type[V] name[%s] output[%d] memaddr[%p] mem_size[%zu] datasize[%zu]",
GELOGI("[IMAS]InitConstant memcpy graph_%u type[V] name[%s] output[%d] memaddr[%p] mem_size[%u] datasize[%zu]",
runtime_param_.graph_id, op_desc->GetName().c_str(), 0, v_output_addr, v_output_size,
ge_tensor->GetData().size());
GE_CHK_RT_RET(rtMemcpy(v_output_addr, v_output_size, ge_tensor->GetData().data(), ge_tensor->GetData().size(),
@@ -709,8 +614,7 @@ Status HybridModelBuilder::InitWeights() {
}

Status HybridModelBuilder::LoadTasks() {
for (auto &it : hybrid_model_.node_items_) {
auto &node_item = it.second;
for (auto &node_item : hybrid_model_.node_items_) {
auto &node_ptr = node_item->node;
if (node_item->node_type == NETOUTPUT) {
continue;
@@ -718,6 +622,7 @@ Status HybridModelBuilder::LoadTasks() {

GELOGD("[%s] Start to build kernel task", node_ptr->GetName().c_str());
auto load_ret = node_item->node_executor->LoadTask(hybrid_model_, node_ptr, node_item->kernel_task);

if (load_ret != UNSUPPORTED && load_ret != SUCCESS) {
GELOGE(load_ret, "[%s] Failed to load task", node_ptr->GetName().c_str());
return load_ret;
@@ -729,23 +634,6 @@ Status HybridModelBuilder::LoadTasks() {
return SUCCESS;
}

Status HybridModelBuilder::LoadGeModel(ComputeGraph &sub_graph, const GeModelPtr &ge_model) {
auto parent_node = sub_graph.GetParentNode();
GE_CHECK_NOTNULL(parent_node);
auto op_type = parent_node->GetType();
if (op_type == IF || op_type == CASE || op_type == WHILE) {
GELOGD("Set ge_model for control op subgraph: [%s], task_size = %d", sub_graph.GetName().c_str(),
ge_model->GetModelTaskDefPtr()->task_size());
subgraph_models_.emplace(sub_graph.GetName(), ge_model);
} else {
GELOGD("Set ge_model for subgraph: [%s], task_size = %d", sub_graph.GetName().c_str(),
ge_model->GetModelTaskDefPtr()->task_size());
hybrid_model_.known_shape_sub_models_.emplace(sub_graph.GetParentNode(), ge_model);
}

return SUCCESS;
}

Status HybridModelBuilder::IndexTaskDefs() {
const auto &root_graph = ge_root_model_->GetRootGraph();
for (auto &it : ge_root_model_->GetSubgraphInstanceNameToModel()) {
@@ -758,9 +646,12 @@ Status HybridModelBuilder::IndexTaskDefs() {
continue;
}

bool is_unknown_shape = sub_graph->GetGraphUnknownFlag();
bool is_unknown_shape = false;
GE_CHK_GRAPH_STATUS_RET(NodeUtils::GetNodeUnknownShapeStatus(*sub_graph->GetParentNode(), is_unknown_shape),
"Failed to invoke GetNodeUnknownShapeStatus.");
if (!is_unknown_shape) {
GE_CHK_STATUS_RET_NOLOG(LoadGeModel(*sub_graph, ge_model));
GELOGD("Set ge_model for subgraph: %s", sub_graph->GetName().c_str());
hybrid_model_.known_shape_sub_graphs_.emplace(sub_graph->GetParentNode(), ge_model);
continue;
}

@@ -785,8 +676,6 @@ Status HybridModelBuilder::IndexTaskDefs() {
op_index = task_def.kernel().context().op_index();
} else if (task_type == RT_MODEL_TASK_KERNEL_EX) {
op_index = task_def.kernel_ex().op_index();
} else if (task_type == RT_MODEL_TASK_HCCL) {
op_index = task_def.kernel_hccl().op_index();
} else {
GELOGD("Skip task type: %d", static_cast<int>(task_type));
continue;
@@ -901,12 +790,12 @@ Status HybridModelBuilder::GetPeerNodeAcrossSubGraphs(const NodePtr &data_node,
for (uint32_t i = 0; i < static_cast<uint32_t>(input_size); ++i) {
uint32_t p_index = 0;
if (!AttrUtils::GetInt(net_output_desc->GetInputDesc(i), ATTR_NAME_PARENT_NODE_INDEX, p_index)) {
GELOGW("SubGraph: %s input tensor %u attr %s not found.", src_graph->GetName().c_str(), i,
GELOGW("SubGraph: %s input tensor %zu attr %s not found.", src_graph->GetName().c_str(), i,
ATTR_NAME_PARENT_NODE_INDEX.c_str());
continue;
}

GELOGD("NetOutput's input[%u], parent_node_index = %u", i, p_index);
GELOGD("NetOutput's input[%zu], parent_node_index = %u", i, p_index);
if (p_index == out_index) {
auto in_anchor = src_net_output_node->GetInDataAnchor(i);
GE_CHECK_NOTNULL(in_anchor);
@@ -941,7 +830,7 @@ Status HybridModelBuilder::InitRuntimeParams() {
ret = ge::AttrUtils::GetInt(first_model, ATTR_MODEL_VAR_SIZE, value);
runtime_param_.var_size = ret ? (uint64_t)value : 0;
runtime_param_.graph_id = ge_root_model_->GetRootGraph()->GetGraphID();
GELOGI("InitRuntimeParams(), session_id:%lu, var_size:%lu. graph_id = %u", runtime_param_.session_id,
GELOGI("InitRuntimeParams(), session_id:%u, var_size:%lu. graph_id = %u", runtime_param_.session_id,
runtime_param_.var_size, runtime_param_.graph_id);

var_manager_ = VarManager::Instance(runtime_param_.session_id);
@@ -949,7 +838,7 @@ Status HybridModelBuilder::InitRuntimeParams() {
return SUCCESS;
}

Status HybridModelBuilder::IdentifyVariableOutputs(NodeItem &node_item) {
Status HybridModelBuilder::ParsePartitionedCall(NodeItem &node_item) {
GELOGD("Start to parse outputs of node: %s", node_item.NodeName().c_str());
auto subgraph = NodeUtils::GetSubgraph(*node_item.node, kSubgraphIndex);
GE_CHECK_NOTNULL(subgraph);
@@ -958,7 +847,6 @@ Status HybridModelBuilder::IdentifyVariableOutputs(NodeItem &node_item) {
auto net_output_desc = net_output_node->GetOpDesc();
GE_CHECK_NOTNULL(net_output_desc);

// constant/variable connected to net output
for (const auto &in_data_anchor : net_output_node->GetAllInDataAnchors()) {
auto src_node = GetPeerNode(in_data_anchor);
GE_CHECK_NOTNULL(src_node);
@@ -976,8 +864,6 @@ Status HybridModelBuilder::IdentifyVariableOutputs(NodeItem &node_item) {
node_item.ref_outputs.emplace(parent_index, src_node);
}

// Data nodes marked with REF_VAR_SRC_VAR_NAME
// Using variable tensor as data's output
for (auto &node : subgraph->GetDirectNode()) {
if (node->GetType() != DATA) {
continue;
@@ -1026,11 +912,6 @@ Status HybridModelBuilder::GetParentNodeOutputIndex(const OpDesc &op_desc, int i
Status HybridModelBuilder::InitModelMem() {
hybrid_model_.var_mem_base_ = var_manager_->GetVarMemoryBase(RT_MEMORY_HBM);
auto total_var_size = hybrid_model_.TotalVarMemSize();
if (total_var_size == 0 && !hybrid_model_.constant_op_nodes_.empty()) {
total_var_size = var_manager_->GetVarMemSize(RT_MEMORY_HBM) > 0 ? var_manager_->GetVarMemMaxSize() : 0;
GELOGD("Model var size = 0. but got uninitialized constant. set var size to %zu.", total_var_size);
}

if (total_var_size > 0 && hybrid_model_.var_mem_base_ == nullptr) {
GE_CHK_STATUS_RET(var_manager_->MallocVarMemory(total_var_size), "Malloc Var Memory Fail.");
hybrid_model_.var_mem_base_ = var_manager_->GetVarMemoryBase(RT_MEMORY_HBM);
@@ -1070,154 +951,5 @@ Status HybridModelBuilder::CopyVarData() {
GELOGI("CopyVarData success.");
return SUCCESS;
}

Status HybridModelBuilder::LoadKnownShapedSubgraph(ComputeGraph &graph, NodeItem *parent_node_item) {
GELOGD("Start to load known shaped subgraph [%s]", graph.GetName().c_str());
auto graph_item = std::unique_ptr<GraphItem>(new (std::nothrow) GraphItem());
GE_CHECK_NOTNULL(graph_item);
graph_item->is_dynamic_ = false;
auto subgraph_name = graph.GetName();
auto wrapper_op_desc = MakeShared<OpDesc>(subgraph_name + "_partitioned_call", PARTITIONEDCALL);
GE_CHECK_NOTNULL(wrapper_op_desc);

for (auto &node : graph.GetDirectNode()) {
GE_CHECK_NOTNULL(node);
auto op_desc = node->GetOpDesc();
GE_CHECK_NOTNULL(op_desc);
const auto &op_type = node->GetType();

if (op_type == DATA) {
int32_t data_index = 0;
if (!AttrUtils::GetInt(node->GetOpDesc(), ATTR_NAME_PARENT_NODE_INDEX, data_index)) {
GELOGE(FAILED, "[%s] Failed to get attr [%s]", node->GetName().c_str(), ATTR_NAME_PARENT_NODE_INDEX.c_str());
return FAILED;
}

(void)wrapper_op_desc->AddInputDesc(op_desc->GetInputDesc(0));
graph_item->input_index_mapping_.emplace_back(data_index);
} else if (op_type == NETOUTPUT) {
int output_index = 0;
for (const auto &output_desc : op_desc->GetAllInputsDescPtr()) {
int32_t data_index = output_index++;
if (!AttrUtils::GetInt(output_desc, ATTR_NAME_PARENT_NODE_INDEX, data_index)) {
GELOGI("[%s] Failed to get attr [%s]", node->GetName().c_str(), ATTR_NAME_PARENT_NODE_INDEX.c_str());
}

GE_CHK_GRAPH_STATUS_RET(wrapper_op_desc->AddOutputDesc(*output_desc),
"[%s] Failed to add output desc. output index = %d", graph.GetName().c_str(),
output_index);

graph_item->output_index_mapping_.emplace_back(data_index);
}
}
}

auto temp_graph = MakeShared<ComputeGraph>("temp");
GE_CHECK_NOTNULL(temp_graph);
auto wrapper_node = temp_graph->AddNode(wrapper_op_desc);
GeModelPtr ge_model = subgraph_models_[subgraph_name];
GE_CHECK_NOTNULL(ge_model);
hybrid_model_.known_shape_sub_models_.emplace(wrapper_node, ge_model);

NodeItem *node_item = nullptr;
GE_CHK_STATUS_RET_NOLOG(GetOrCreateNodeItem(wrapper_node, &node_item));
node_item->input_start = 0;
node_item->output_start = 0;
node_item->outputs.resize(node_item->num_outputs);
graph_item->node_items_.emplace_back(node_item);
graph_item->output_node_ = node_item;
graph_item->total_inputs_ = node_item->num_inputs;
graph_item->total_outputs_ = node_item->num_outputs;

GELOGD("NodeItem create for known shape subgraph [%s], NodeItem = %s", graph.GetName().c_str(),
node_item->DebugString().c_str());

GELOGD("Done parse known shape subgraph successfully. graph = [%s]", graph.GetName().c_str());
graph_item->SetName(graph.GetName());
GELOGD("Done loading known shape subgraph: [%s]", graph_item->GetName().c_str());
hybrid_model_.subgraph_items_.emplace(graph.GetName(), std::move(graph_item));
return SUCCESS;
}

Status HybridModelBuilder::LoadDynamicSubgraph(ComputeGraph &graph, bool is_root_graph) {
GELOGD("Start to load subgraph [%s]", graph.GetName().c_str());
// for known partitioned call, load all nodes
auto graph_item = std::unique_ptr<GraphItem>(new (std::nothrow) GraphItem());
GE_CHECK_NOTNULL(graph_item);

graph_item->is_dynamic_ = true;
graph_item->node_items_.reserve(graph.GetDirectNodesSize());
int input_start = 0;
int output_start = 0;
std::vector<NodeItem *> data_nodes;
for (auto &node : graph.GetDirectNode()) {
GE_CHECK_NOTNULL(node);
GE_CHECK_NOTNULL(node->GetOpDesc());
const auto &op_type = node->GetType();

NodeItem *node_item = nullptr;
GE_CHK_STATUS_RET_NOLOG(GetOrCreateNodeItem(node, &node_item));
GE_CHK_STATUS_RET_NOLOG(BuildNodeItem(node, *node_item));
GE_CHK_STATUS_RET_NOLOG(UpdateAnchorStatus(node)); // needed by FE generate task

node_item->input_start = input_start;
node_item->output_start = output_start;
input_start += node_item->num_inputs;
output_start += node_item->num_outputs;

if (op_type == DATA_TYPE || op_type == AIPP_DATA_TYPE) {
data_nodes.emplace_back(node_item);
} else if (op_type == NETOUTPUT) {
graph_item->output_node_ = node_item;
GE_CHK_STATUS_RET_NOLOG(BuildOutputMapping(*graph_item, *node_item, is_root_graph));
}

graph_item->node_items_.emplace_back(node_item);
GELOGD("NodeItem created: %s", node_item->DebugString().c_str());
}

graph_item->total_inputs_ = input_start;
graph_item->total_outputs_ = output_start;
GE_CHK_STATUS_RET_NOLOG(BuildInputMapping(*graph_item, data_nodes, is_root_graph));
if (is_root_graph) {
graph_item->SetName("Root-Graph");
GELOGD("Done loading dynamic subgraph: [%s]", graph_item->GetName().c_str());
hybrid_model_.root_graph_item_ = std::move(graph_item);
} else {
graph_item->SetName(graph.GetName());
GELOGD("Done loading dynamic subgraph: [%s]", graph_item->GetName().c_str());
hybrid_model_.subgraph_items_.emplace(graph.GetName(), std::move(graph_item));
}

return SUCCESS;
}

Status HybridModelBuilder::BuildInputMapping(GraphItem &graph_item, vector<NodeItem *> &data_nodes,
bool is_root_graph) {
uint32_t data_op_index = 0;
for (auto &node_item : data_nodes) {
auto node = node_item->node;
int data_index = data_op_index;
if (is_root_graph) {
if (AttrUtils::GetInt(node->GetOpDesc(), ATTR_NAME_INDEX, data_index)) {
GELOGI("ge_train: get new index %u, old %u", data_index, data_op_index);
}
data_op_index++;
} else {
if (!AttrUtils::GetInt(node->GetOpDesc(), ATTR_NAME_PARENT_NODE_INDEX, data_index)) {
GELOGE(FAILED, "[%s] Failed to get attr [%s]", node->GetName().c_str(), ATTR_NAME_PARENT_NODE_INDEX.c_str());
return FAILED;
}
}

if (graph_item.input_nodes_.size() <= static_cast<size_t>(data_index)) {
graph_item.input_nodes_.resize(data_index + 1);
}

graph_item.input_nodes_[data_index] = node_item;
}

return SUCCESS;
}
} // namespace hybrid
} // namespace ge

+ 8
- 14
src/ge/hybrid/model/hybrid_model_builder.h View File

@@ -46,20 +46,18 @@ class HybridModelBuilder {
static Status HandleDtString(const GeTensor &tensor, void *var_addr);
static Status MergeInputNodes(ComputeGraph &compute_graph);
static Status MergeNetOutputNode(ComputeGraph &compute_graph);
static Status UnfoldSubgraphs(ComputeGraph &root_graph, ComputeGraphPtr &merged_graph);
static Status UnfoldSubgraph(ComputeGraph &root_graph, ComputeGraph &parent_graph, ComputeGraph &sub_graph);
static Status MergeSubgraphs(ComputeGraph &root_graph, ComputeGraphPtr &merged_graph);
static Status InitWeights();
static Status BuildInputMapping(GraphItem &graph_item, std::vector<NodeItem *> &data_nodes, bool is_root_graph);
static Status ResolveRefIo(NodeItem &node_item);
Status BuildOutputMapping(GraphItem &partitioned_call, const NodeItem &node_item, bool is_root_graph);

Status ValidateParams();
Status LoadGraph();
Status LoadGeModel(ComputeGraph &graph, const GeModelPtr &ge_model);
Status LoadTasks();
Status IdentifyVariableOutputs(NodeItem &node_item);
Status BuildNodeItem(const NodePtr &node, NodeItem &node_item);
Status ParsePartitionedCall(NodeItem &node_item);
Status ParseNetOutput(const NodeItem &node_item);
Status BuildNoteItem(const NodePtr &node, NodeItem &node_item);
Status GetOrCreateNodeItem(const NodePtr &node, NodeItem **node_item);
Status ParseDependentInputNodes(NodeItem &node_item, const std::vector<string> &dependencies);
Status ResolveRootNodes();
Status IndexTaskDefs();
Status IndexSpecialNodes();
Status InitRuntimeParams();
@@ -67,23 +65,19 @@ class HybridModelBuilder {
Status TransAllVarData();
Status CopyVarData();
Status VarNodeToTensor(const NodePtr &var_node, std::unique_ptr<TensorValue> &tensor);
Status AssignUninitializedConstantOps();
Status InitConstantOps();
Status InitVariableTensors();
Status LoadDynamicSubgraph(ComputeGraph &graph, bool is_root_graph);
Status LoadKnownShapedSubgraph(ComputeGraph &graph, NodeItem *parent_node_item);

const char *GetGraphName() const { return hybrid_model_.model_name_.c_str(); }
const char *GetGraphName() const { return graph_name_.c_str(); }

const NodeItem *GetNodeItem(const NodePtr &node) const;
NodeItem *MutableNodeItem(const NodePtr &node);

GeRootModelPtr ge_root_model_;
std::string graph_name_;
std::map<int, std::unique_ptr<TensorValue>> weights_;
std::map<std::string, GeModelPtr> subgraph_models_;
HybridModel &hybrid_model_;
std::map<NodePtr, std::vector<std::pair<int, NodePtr>>> node_ref_inputs_;
int node_index = 0;

RuntimeParam &runtime_param_;
VarManager *var_manager_ = nullptr;


+ 3
- 40
src/ge/hybrid/model/node_item.cc View File

@@ -16,8 +16,6 @@

#include "node_item.h"
#include <sstream>
#include "common/debug/log.h"
#include "hybrid/node_executor/node_executor.h"

namespace ge {
namespace hybrid {
@@ -30,34 +28,12 @@ NodeItem::NodeItem(NodePtr node) : node(std::move(node)) {
this->node_type = this->node->GetType();
}

Status NodeItem::Init() {
for (int i = 0; i < num_inputs; ++i) {
const auto &input_desc = op_desc->MutableInputDesc(i);
GE_CHECK_NOTNULL(input_desc);
if (input_desc->MutableShape().IsUnknownShape()) {
is_input_shape_static.push_back(false);
} else {
num_static_input_shapes++;
is_input_shape_static.push_back(true);
GELOGD("[%s] The shape of input[%d] is static. shape = [%s]", NodeName().c_str(), i,
input_desc->MutableShape().ToString().c_str());
}
}

return SUCCESS;
}

bool NodeItem::IsControlOp() const {
auto op_type = op_desc->GetType();
return op_type == IF || op_type == CASE || op_type == WHILE || op_type == FOR;
}

std::string NodeItem::DebugString() const {
std::stringstream ss;
ss << "Node: ";
ss << "id = " << node_id;
ss << ", name = [" << node->GetName();
ss << "], type = " << node->GetType();
ss << ", name = " << node->GetName();
ss << ", type = " << node->GetType();
ss << ", is_dynamic = " << (is_dynamic ? "True" : "False");
ss << ", unknown_shape_op_type = " << shape_inference_type;
ss << ", input_start = " << input_start;
@@ -65,7 +41,7 @@ std::string NodeItem::DebugString() const {
ss << ", output_start = " << output_start;
ss << ", num_outputs = " << num_outputs;
ss << ", dependent_nodes = [";
for (const auto &dep_node : dependents_for_shape_inference) {
for (const auto &dep_node : dependent_node_list) {
ss << dep_node->GetName() << ", ";
}
ss << "]";
@@ -79,18 +55,5 @@ std::string NodeItem::DebugString() const {

return ss.str();
}

void NodeItem::SetToDynamic() {
num_static_input_shapes = 0;
is_dynamic = true;
for (size_t i = 0; i < is_input_shape_static.size(); ++i) {
is_input_shape_static[i] = false;
}
if (kernel_task != nullptr && !kernel_task->IsSupportDynamicShape()) {
GELOGD("[%s] Dynamic shape is not supported, clear node task.", node_name.c_str());
kernel_task = nullptr;
}
}

} // namespace hybrid
} // namespace ge

+ 3
- 14
src/ge/hybrid/model/node_item.h View File

@@ -18,7 +18,6 @@
#define GE_HYBRID_MODEL_NODE_ITEM_H_

#include <vector>
#include "external/ge/ge_api_error_codes.h"
#include "graph/node.h"
#include "graph/op_desc.h"
#include "framework/common/types.h"
@@ -34,16 +33,10 @@ struct NodeItem {
explicit NodeItem(NodePtr node);
~NodeItem() = default;

Status Init();

const std::string &NodeName() const { return node_name; }

const std::string &NodeType() const { return node_type; }

bool IsControlOp() const;

void SetToDynamic();

std::string DebugString() const;

NodePtr node;
@@ -59,21 +52,17 @@ struct NodeItem {
UnknowShapeOpType shape_inference_type = DEPEND_IN_SHAPE;
std::string node_name;
std::string node_type;
std::vector<ge::NodePtr> dependents_for_shape_inference;
std::vector<ge::NodePtr> dependents_for_execution;
std::vector<ge::NodePtr> dependent_node_list;
std::set<int> to_const_output_id_list;

vector<NodeItem *> inputs;
// src_output_id, dst_anchor_id, dst_node
vector<NodeItem *> inputs;
vector<vector<pair<uint32_t, NodeItem *>>> outputs;

std::shared_ptr<NodeTask> kernel_task;
const NodeExecutor *node_executor = nullptr;
std::map<int, ge::GeTensorDescPtr> const_input_shapes;
std::map<int, ge::NodePtr> ref_outputs;
std::map<int, int> reuse_inputs;

std::vector<bool> is_input_shape_static;
int num_static_input_shapes = 0;
};
} // namespace hybrid
} // namespace ge


+ 202
- 89
src/ge/hybrid/node_executor/aicore/aicore_node_executor.cc View File

@@ -16,8 +16,10 @@

#include "aicore_node_executor.h"
#include "cce/taskdown_common.hpp"
#include "hybrid/executor/hybrid_execution_context.h"
#include "graph/debug/ge_attr_define.h"
#include "hybrid/model/hybrid_model.h"
#include "init/gelib.h"
#include "framework/common/debug/log.h"

namespace ge {
namespace hybrid {
@@ -25,47 +27,16 @@ REGISTER_NODE_EXECUTOR_BUILDER(NodeExecutorManager::ExecutorType::AICORE, AiCore

AiCoreNodeTask::AiCoreNodeTask(std::vector<std::unique_ptr<AiCoreOpTask>> &&tasks) : tasks_(std::move(tasks)) {}

Status AiCoreNodeExecutor::Initialize() {
auto ge_lib = GELib::GetInstance();
GE_CHECK_NOTNULL(ge_lib);
if (!ge_lib->InitFlag()) {
GELOGE(GE_CLI_GE_NOT_INITIALIZED, "Ge_lib is uninitialized, failed.");
return GE_CLI_GE_NOT_INITIALIZED;
}

auto &kernel_manager = ge_lib->OpsKernelManagerObj();
auto aic_ops_store = kernel_manager.GetOpsKernelInfoStore("AIcoreEngine");
GE_CHECK_NOTNULL(aic_ops_store);

compiler_.reset(new (std::nothrow) AiCoreTaskCompiler(aic_ops_store));
GE_CHECK_NOTNULL(compiler_);
return SUCCESS;
}

Status AiCoreNodeExecutor::LoadTask(const HybridModel &model, const NodePtr &node, shared_ptr<NodeTask> &task) const {
GE_CHECK_NOTNULL(node);
GELOGI("AiCoreNodeExecutor(%s) LoadTask Start.", node->GetName().c_str());
GELOGI("AiCoreNodeExecutor[%s] LoadTask Start.", node->GetName().c_str());

auto *task_defs = model.GetTaskDefs(node);
if (task_defs == nullptr || task_defs->empty()) {
bool dynamic_flag = false;
if (!AttrUtils::GetBool(node->GetOpDesc(), "support_dynamicshape", dynamic_flag) || !dynamic_flag) {
GELOGD("Skip create task of node (%s) as 'support_dynamicshape' is false and cann't get task_defs.",
node->GetName().c_str());
return SUCCESS;
} else {
GELOGE(FAILED, "Task_defs is empty for node (%s) which 'support_dynamicshape' is true, failed.",
node->GetName().c_str());
return FAILED;
}
}
Status ret = SUCCESS;
GE_IF_BOOL_EXEC(task_defs != nullptr && !task_defs->empty(), ret = CreateTask(model, *task_defs, node, task));

AiCoreTaskBuilder builder(node->GetOpDesc(), *task_defs);
std::unique_ptr<NodeTask> node_task;
GE_CHK_STATUS_RET(builder.BuildTask(node_task, true), "[%s] Failed to build op tasks.", node->GetName().c_str());
task = std::move(node_task);
GELOGI("AiCoreNodeExecutor(%s) LoadTask End.", node->GetName().c_str());
return SUCCESS;
GELOGI("AiCoreNodeExecutor[%s] LoadTask End, ret[%u].", node->GetName().c_str(), ret);
return ret;
}

Status AiCoreNodeExecutor::GenNodeKey(const NodePtr &node, std::string &node_key) {
@@ -76,19 +47,16 @@ Status AiCoreNodeExecutor::GenNodeKey(const NodePtr &node, std::string &node_key
// make sure unique, (op_id + input_shape) is unique
node_key = std::to_string(op_desc->GetId()) + "/";
node_key.append(std::to_string(op_desc->GetInputsSize()));
auto input_descs = op_desc->GetAllInputsDescPtr();
for (auto &input_desc : input_descs) {
auto input_descs = op_desc->GetAllInputsDesc();
for (auto input_desc : input_descs) {
node_key.push_back('/');
auto &shape = input_desc->MutableShape();
auto num_dims = shape.GetDimNum();
if (num_dims == 0) {
continue;
} // scalar
for (std::size_t i = 0; i < num_dims - 1; i++) {
node_key.append(std::to_string(shape.GetDim(i)));
std::vector<int64_t> dims = input_desc.GetShape().GetDims();
GE_IF_BOOL_EXEC(dims.size() == 0, continue); // scalar
for (std::size_t i = 0; i < dims.size() - 1; i++) {
node_key.append(std::to_string(dims[i]));
node_key.push_back(',');
}
node_key.append(std::to_string(shape.GetDim(num_dims - 1)));
node_key.append(std::to_string(dims[dims.size() - 1]));
}
return SUCCESS;
}
@@ -97,10 +65,8 @@ bool AiCoreNodeTaskRegistry::AddTask(const std::string &node_key, const std::sha
GE_CHECK_NOTNULL(task);
std::lock_guard<std::mutex> lock(mutex_);
auto iter = reg_node_tasks_.find(node_key);
if (iter != reg_node_tasks_.end()) {
GELOGE(FAILED, "AiCoreNodeTaskRegistry(%s) AddTask failed, key already exist.", node_key.c_str());
return false;
}
GE_CHK_BOOL_TRUE_EXEC_WITH_LOG(iter != reg_node_tasks_.end(), return false,
"AiCoreNodeTaskRegistry[%s] AddTask failed, key already exist.", node_key.c_str());
auto ret = reg_node_tasks_.emplace(node_key, task);
return ret.second;
}
@@ -114,84 +80,231 @@ std::shared_ptr<NodeTask> AiCoreNodeTaskRegistry::GetTask(const std::string &nod
Status AiCoreNodeExecutor::CompileTask(const HybridModel &model, const NodePtr &node,
shared_ptr<NodeTask> &task) const {
GE_CHECK_NOTNULL(node);
GELOGI("AiCoreNodeExecutor(%s) CompileTask Start.", node->GetName().c_str());
GELOGI("AiCoreNodeExecutor[%s] CompileTask Start.", node->GetName().c_str());

AiCoreNodeTaskRegistry &registry = AiCoreNodeTaskRegistry::GetInstance();
std::string node_key;
GE_CHK_STATUS_RET(GenNodeKey(node, node_key), "GenNodeKey failed, op name = %s.", node->GetName().c_str());
GE_CHK_STATUS_RET(GenNodeKey(node, node_key), "GenNodeKey failed. op name = %s", node->GetName().c_str());

node_key = std::to_string(model.GetModelId()) + "/" + node_key;
GELOGD("NodeKey for %s = %s", node->GetName().c_str(), node_key.c_str());
task = registry.GetTask(node_key);
if (task != nullptr) {
GELOGI("AiCoreNodeExecutor(%s) CompileTask Skip.", node->GetName().c_str());
return SUCCESS;
}
GE_CHK_TRUE_EXEC_INFO(task != nullptr, return SUCCESS, "AiCoreNodeExecutor[%s] CompileTask Skip.",
node->GetName().c_str());

std::vector<domi::TaskDef> task_defs;
GE_CHK_STATUS_RET(compiler_->CompileOp(node, task_defs), "Compile op(%s) failed.", node->GetName().c_str());
GE_CHK_STATUS_RET_NOLOG(compiler_->CompileOp(node, task_defs));
GELOGD("successfully generated task_defs: %s", node->GetName().c_str());

AiCoreTaskBuilder builder(node->GetOpDesc(), task_defs);
std::unique_ptr<NodeTask> node_task;
GE_CHK_STATUS_RET(builder.BuildTask(node_task, false), "[%s] Failed to build op tasks.", node->GetName().c_str());
task = std::move(node_task);
GE_CHK_STATUS_RET_NOLOG(CreateTask(model, task_defs, node, task));
GELOGD("successfully created node task: %s", node->GetName().c_str());

if (!registry.AddTask(node_key, task)) {
GELOGE(INTERNAL_ERROR, "Add NodeTask failed, op name = %s.", node->GetName().c_str());
return INTERNAL_ERROR;
GE_CHK_BOOL_EXEC(registry.AddTask(node_key, task), return INTERNAL_ERROR, "Add NodeTask failed. op name = %s",
node->GetName().c_str()); // should not happen.
GELOGI("AiCoreNodeExecutor[%s] CompileTask End.", node->GetName().c_str());
return SUCCESS;
}

Status AiCoreNodeExecutor::BuildAiCoreTask(const domi::KernelDef &kernel_def, const OpDescPtr &op_desc,
AiCoreOpTask **task) {
GE_CHECK_NOTNULL(op_desc);
GE_CHECK_NOTNULL(task);

const auto &context = kernel_def.context();
auto kernel_type = static_cast<cce::ccKernelType>(context.kernel_type());
GE_CHK_BOOL_TRUE_EXEC_WITH_LOG(kernel_type != cce::ccKernelType::TE, return UNSUPPORTED,
"Only TBE kernel is supported, but [%s] got %u", op_desc->GetName().c_str(),
context.kernel_type());

auto *aicore_task = new (std::nothrow) AiCoreOpTask();
GE_CHK_BOOL_TRUE_EXEC_WITH_LOG(aicore_task == nullptr, return MEMALLOC_FAILED, "Create AiCore op task failed.");

auto builder = AiCoreTaskBuilder(op_desc, kernel_def);
auto ret = builder.BuildTask(*aicore_task);
GE_IF_BOOL_EXEC(ret != SUCCESS, delete aicore_task; aicore_task = nullptr; return ret);

*task = aicore_task;
return SUCCESS;
}

Status AiCoreNodeExecutor::CreateTask(const HybridModel &model, const std::vector<domi::TaskDef> &task_defs,
const NodePtr &node, std::shared_ptr<NodeTask> &task) {
GE_CHECK_NOTNULL(node);
GELOGD("To CreateTask, task def size = %zu", task_defs.size());
std::vector<std::unique_ptr<AiCoreOpTask>> aicore_op_tasks;
aicore_op_tasks.reserve(task_defs.size());
for (size_t i = 0; i < task_defs.size(); ++i) {
const domi::TaskDef &task_def = task_defs[i];
GELOGD("Op[%s] Task[%d], type = %u, DebugString = %s", node->GetName().c_str(), i, task_def.type(),
task_def.DebugString().c_str());
auto task_type = static_cast<rtModelTaskType_t>(task_def.type());
GE_CHK_BOOL_TRUE_EXEC_WITH_LOG(task_type == RT_MODEL_TASK_KERNEL_EX, return UNSUPPORTED,
"BuildKernelExTask is not supported");
GE_CHK_BOOL_TRUE_EXEC_INFO(task_type != RT_MODEL_TASK_KERNEL, continue, "Skip task type %d",
static_cast<int>(task_type));

const domi::KernelDef &kernel_def = task_def.kernel();
AiCoreOpTask *aicore_op_task = nullptr;
// not use hybrid model now
GE_CHK_STATUS_RET_NOLOG(BuildAiCoreTask(kernel_def, node->GetOpDesc(), &aicore_op_task));
GE_CHK_BOOL_TRUE_EXEC_WITH_LOG(aicore_op_task == nullptr, return FAILED, "BuildAiCoreTask[%s] failed.",
node->GetName().c_str());

aicore_op_tasks.emplace_back(std::unique_ptr<AiCoreOpTask>(aicore_op_task));
}

GELOGI("AiCoreNodeExecutor(%s) CompileTask End.", node->GetName().c_str());
if (!aicore_op_tasks.empty()) {
auto aic_task = std::shared_ptr<NodeTask>(new AiCoreNodeTask(std::move(aicore_op_tasks)));
task = std::move(aic_task);
GELOGD("Generate AiCoreOpTask success");
return SUCCESS;
}

GELOGE(INTERNAL_ERROR, "Failed to build task. node = %s", node->GetName().c_str());
return INTERNAL_ERROR;
}

Status AiCoreNodeExecutor::Initialize() {
std::shared_ptr<GELib> ge_lib = GELib::GetInstance();
GE_CHK_BOOL_TRUE_EXEC_WITH_LOG((ge_lib == nullptr) || !ge_lib->InitFlag(), return GE_CLI_GE_NOT_INITIALIZED,
"Get ge_lib failed.");

auto &kernel_manager = ge_lib->OpsKernelManagerObj();
auto aic_ops_store = kernel_manager.GetOpsKernelInfoStore("AIcoreEngine");
GE_CHK_BOOL_TRUE_EXEC_WITH_LOG(aic_ops_store == nullptr, return GE_CLI_GE_NOT_INITIALIZED,
"Failed to get kernel info store for AIcoreEngine.");

compiler_.reset(new (std::nothrow) AiCoreTaskCompiler(aic_ops_store));
GE_CHECK_NOTNULL(compiler_);
return SUCCESS;
}

Status AiCoreNodeExecutor::Finalize() { return NodeExecutor::Finalize(); }

Status AiCoreNodeTask::ExecuteAsync(TaskContext &context, std::function<void()> done_callback) {
auto op_desc = context.GetNodeItem().op_desc;
GE_CHECK_NOTNULL(op_desc);
GELOGI("[%s] ExecuteAsync Start.", op_desc->GetName().c_str());
for (auto &task : tasks_) {
GE_CHK_STATUS_RET_NOLOG(task->LaunchKernel(context.GetStream()));
GELOGI("AiCoreNodeTask[%s] ExecuteAsync Start.", op_desc->GetName().c_str());
for (size_t i = 0; i < tasks_.size(); i++) {
GE_CHECK_NOTNULL(tasks_[i]);
GE_CHK_STATUS_RET_NOLOG(tasks_[i]->LaunchKernel(context.GetStream()));
}

if (done_callback != nullptr) {
GE_CHK_STATUS_RET_NOLOG(context.RegisterCallback(done_callback));
}

GELOGD("[%s] ExecuteAsync End.", op_desc->GetName().c_str());
GELOGI("AiCoreNodeTask[%s] ExecuteAsync End.", op_desc->GetName().c_str());
return SUCCESS;
}

Status AiCoreNodeTask::UpdateArgs(TaskContext &context) {
Status AiCoreNodeTask::UpdateAtomicArgs(TaskContext &context, std::unique_ptr<AiCoreOpTask> &task) {
GE_CHECK_NOTNULL(task);
auto op_desc = context.GetNodeItem().op_desc;
GE_CHECK_NOTNULL(op_desc);
GELOGI("[%s] AiCoreNodeTask UpdateArgs Start.", op_desc->GetName().c_str());
for (auto &task : tasks_) {
GE_CHK_STATUS_RET_NOLOG(task->UpdateArgs(context));

// refresh atomic output addr
std::vector<int64_t> atomic_output_indexes; // here atomic just clean output
(void)ge::AttrUtils::GetListInt(op_desc, ge::ATOMIC_ATTR_OUTPUT_INDEX, atomic_output_indexes);
GE_RETURN_WITH_LOG_IF_TRUE(atomic_output_indexes.size() > static_cast<size_t>(context.NumOutputs()),
"AtomicAddrClean op's arg_size error.");
auto *arg_off = reinterpret_cast<uint8_t *>(task->args_.get()) + task->offset_;
auto *arg_base = reinterpret_cast<uintptr_t *>(arg_off);
int index = 0;
for (size_t i = 0; i < atomic_output_indexes.size(); ++i) {
const auto output = context.GetOutput(atomic_output_indexes[i]);
GE_CHECK_NOTNULL(output);
arg_base[index++] = reinterpret_cast<uintptr_t>(output->GetData());
}

// refresh atomic workspace addr
auto workspace_sizes = op_desc->GetWorkspaceBytes();
uint64_t ops_workspace_num = static_cast<uint64_t>(workspace_sizes.size());
uint64_t workspace_num = static_cast<uint64_t>(context.NumWorkspaces());
GE_CHK_BOOL_EXEC(ops_workspace_num == workspace_num, return PARAM_INVALID,
"The workspace_num in op_desc %lu is not equal to it %lu in context.", ops_workspace_num,
workspace_num);
GE_IF_BOOL_EXEC(workspace_num == 0, return SUCCESS);

map<string, map<int64_t, int64_t>> workspace_info;
workspace_info = op_desc->TryGetExtAttr(EXT_ATTR_ATOMIC_WORKSPACE_INFO, workspace_info);
if (!workspace_info.empty()) {
bool is_fusion_node = false;
(void)ge::AttrUtils::GetBool(op_desc, ATOMIC_ATTR_IS_FUSION_NODE, is_fusion_node);
GE_CHK_BOOL_TRUE_EXEC_WITH_LOG(is_fusion_node, return PARAM_INVALID,
"Atomic desc[%s] shouldn't be fusion_node in AiCoreNodeTask",
op_desc->GetName().c_str());

for (auto iter = workspace_info.begin(); iter != workspace_info.end(); ++iter) {
GE_CHK_BOOL_TRUE_EXEC_WITH_LOG(op_desc->GetName() != iter->first, return PARAM_INVALID,
"The node name %s and the node name %s in workspace info are inconsistent.",
op_desc->GetName().c_str(), iter->first.c_str());
GE_IF_BOOL_EXEC(iter->second.empty(), continue);

for (auto &info_iter : iter->second) {
auto workspace_index = static_cast<uint64_t>(info_iter.first);

GE_CHK_BOOL_TRUE_EXEC_WITH_LOG(workspace_index >= workspace_num, return PARAM_INVALID,
"The workspace index %lu is more than the size %lu of workspace vector.",
workspace_index, workspace_num);

const auto workspace = context.MutableWorkspace(workspace_index);
arg_base[index++] = reinterpret_cast<uintptr_t>(workspace);
}
}
}
GELOGI("[%s] AiCoreNodeTask UpdateArgs End.", op_desc->GetName().c_str());
return SUCCESS;
}

Status AiCoreNodeTask::UpdateTilingData(TaskContext &context) {
GELOGD("[%s] PrepareWithShape started", context.GetNodeName());
for (auto &task : tasks_) {
GE_CHK_STATUS_RET_NOLOG(task->PrepareWithShape(context));
Status AiCoreNodeTask::UpdateAllArgs(TaskContext &context, std::unique_ptr<AiCoreOpTask> &task) {
GE_CHECK_NOTNULL(task);
auto *arg_off = reinterpret_cast<uint8_t *>(task->args_.get()) + task->offset_;
auto *arg_base = reinterpret_cast<uintptr_t *>(arg_off);
int index = 0;
for (int i = 0; i < context.NumInputs(); ++i) {
const auto input = context.GetInput(i);
GE_CHECK_NOTNULL(input);
arg_base[index++] = reinterpret_cast<uintptr_t>(input->GetData());
}

for (int i = 0; i < context.NumOutputs(); ++i) {
const auto output = context.GetOutput(i);
GE_CHECK_NOTNULL(output);
arg_base[index++] = reinterpret_cast<uintptr_t>(output->GetData());
}

auto op_desc = context.GetNodeItem().op_desc;
GE_CHECK_NOTNULL(op_desc);
auto workspace_sizes = op_desc->GetWorkspaceBytes();
int ops_workspace_num = static_cast<int>(workspace_sizes.size());
int workspace_num = static_cast<int>(context.NumWorkspaces());
GE_CHK_BOOL_EXEC(ops_workspace_num == workspace_num, return PARAM_INVALID,
"The workspace_num in op_desc %lu is not equal to it %lu in context.", ops_workspace_num,
workspace_num);
for (int i = 0; i < workspace_num; ++i) {
const auto workspace = context.MutableWorkspace(i);
arg_base[index++] = reinterpret_cast<uintptr_t>(workspace);
}
GELOGD("[%s] Done PrepareWithShape successfully.", context.GetNodeName());
return SUCCESS;
}

bool AiCoreNodeTask::IsSupportDynamicShape() {
for (size_t i = 0; i < tasks_.size(); ++i) {
if (!tasks_[i]->IsDynamicShapeSupported()) {
GELOGD("[%s] Task does not support dynamic shape.", tasks_[i]->GetName().c_str());
return false;
}
}
Status AiCoreNodeTask::UpdateArgs(TaskContext &context) {
auto op_desc = context.GetNodeItem().op_desc;
GE_CHECK_NOTNULL(op_desc);
GELOGI("AiCoreNodeTask[%s] UpdateArgs Start.", op_desc->GetName().c_str());
GE_IF_BOOL_EXEC(tasks_.size() == 1, return UpdateAllArgs(context, tasks_[0]));

std::vector<int64_t> atomic_output_indexes; // here atomic just clean output
(void)ge::AttrUtils::GetListInt(op_desc, ge::ATOMIC_ATTR_OUTPUT_INDEX, atomic_output_indexes);
GE_CHK_BOOL_TRUE_EXEC_WITH_LOG(atomic_output_indexes.empty(), return FAILED, "ATOMIC_ATTR_OUTPUT_INDEX is empty.");
GE_CHK_BOOL_TRUE_EXEC_WITH_LOG(tasks_.size() != 2, return FAILED, "AtomicAddrClean op task num != 2.");

return true;
GE_CHK_STATUS_RET_NOLOG(UpdateAtomicArgs(context, tasks_[0]));
GE_CHK_STATUS_RET_NOLOG(UpdateAllArgs(context, tasks_[1]));

GELOGI("AiCoreNodeTask[%s] UpdateArgs End.", op_desc->GetName().c_str());
return SUCCESS;
}
} // namespace hybrid
} // namespace ge

+ 11
- 5
src/ge/hybrid/node_executor/aicore/aicore_node_executor.h View File

@@ -25,6 +25,7 @@

namespace ge {
namespace hybrid {

class AiCoreNodeTaskRegistry {
public:
~AiCoreNodeTaskRegistry() = default;
@@ -46,27 +47,32 @@ class AiCoreNodeTaskRegistry {
class AiCoreNodeTask : public NodeTask {
public:
explicit AiCoreNodeTask(std::vector<std::unique_ptr<AiCoreOpTask>> &&tasks);
~AiCoreNodeTask() override = default;
bool IsSupportDynamicShape() override;
Status UpdateTilingData(TaskContext &context) override;

Status UpdateArgs(TaskContext &context) override;
~AiCoreNodeTask() = default;
Status ExecuteAsync(TaskContext &context, std::function<void()> done_callback) override;
Status UpdateArgs(TaskContext &context) override;

private:
static Status UpdateAllArgs(TaskContext &context, std::unique_ptr<AiCoreOpTask> &task);
static Status UpdateAtomicArgs(TaskContext &context, std::unique_ptr<AiCoreOpTask> &task);
std::vector<std::unique_ptr<AiCoreOpTask>> tasks_;
};

class AiCoreNodeExecutor : public NodeExecutor {
public:
Status Initialize() override;
Status Finalize() override;

Status LoadTask(const HybridModel &model, const NodePtr &node, shared_ptr<NodeTask> &task) const override;
Status CompileTask(const HybridModel &model, const NodePtr &node, std::shared_ptr<NodeTask> &task) const override;

private:
static Status CreateTask(const HybridModel &model, const std::vector<domi::TaskDef> &task_defs, const NodePtr &node,
std::shared_ptr<NodeTask> &task);
static Status BuildAiCoreTask(const domi::KernelDef &kernel_def, const OpDescPtr &op_desc, AiCoreOpTask **task);
static Status GenNodeKey(const NodePtr &node, std::string &node_key);
std::unique_ptr<AiCoreTaskCompiler> compiler_;
};

} // namespace hybrid
} // namespace ge
#endif // GE_HYBRID_KERNEL_AICORE_NODE_EXECUTOR_H_

+ 5
- 291
src/ge/hybrid/node_executor/aicore/aicore_op_task.cc View File

@@ -14,305 +14,19 @@
* limitations under the License.
*/

#include "hybrid/node_executor/aicore/aicore_op_task.h"
#include "cce/taskdown_common.hpp"
#include "aicore_op_task.h"
#include "framework/common/debug/log.h"
#include "hybrid/executor/hybrid_execution_context.h"
#include "hybrid/node_executor/aicore/aicore_task_builder.h"

using optiling::OpRunInfo;

namespace ge {
namespace hybrid {
namespace {
constexpr char const *kAttrSupportDynamicShape = "support_dynamicshape";
constexpr char const *kAttrOpParamSize = "op_para_size";
constexpr char const *kAttrAtomicOpParamSize = "atomic_op_para_size";
} // namespace

Status AiCoreOpTask::Init(const OpDesc &op_desc, const domi::TaskDef &task_def) {
GE_CHK_STATUS_RET_NOLOG(InitWithTaskDef(op_desc, task_def));
GE_CHK_STATUS_RET_NOLOG(InitTilingInfo(op_desc));
return SUCCESS;
}

Status AiCoreOpTask::InitWithTaskDef(const OpDesc &op_desc, const domi::TaskDef &task_def) {
GE_CHK_STATUS_RET(ValidateTaskDef(task_def), "[%s] Failed to validate task def: [%s]", op_desc.GetName().c_str(),
task_def.DebugString().c_str());

const domi::KernelDef &kernel_def = task_def.kernel();
const domi::KernelContext &context = kernel_def.context();
stub_name_ = kernel_def.stub_func();
GE_CHK_RT_RET(rtGetFunctionByName(stub_name_.c_str(), &stub_func_));
args_size_ = kernel_def.args_size();
block_dim_ = kernel_def.block_dim();

// malloc args memory
args_.reset(new (std::nothrow) uint8_t[args_size_]);
GE_CHECK_NOTNULL(args_);
errno_t err = memcpy_s(args_.get(), args_size_, kernel_def.args().data(), args_size_);
if (err != EOK) {
GELOGE(INTERNAL_ERROR, "AiCoreTask memcpy args failed.");
return INTERNAL_ERROR;
}

if (context.args_offset().size() < sizeof(uint16_t)) {
GELOGE(INTERNAL_ERROR, "Invalid args_offset, size = %zu.", context.args_offset().size());
return INTERNAL_ERROR;
}

const auto *args_offset_buffer = reinterpret_cast<const uint16_t *>(context.args_offset().data());
uint32_t offset = *args_offset_buffer;
if (offset > args_size_) {
GELOGE(INTERNAL_ERROR, "[%s] Arg offset out of range. offset = %u, arg size = %u", GetName().c_str(), offset,
args_size_);
return INTERNAL_ERROR;
}

arg_base_ = reinterpret_cast<uintptr_t *>(args_.get() + offset);
max_arg_count_ = (args_size_ - offset) / sizeof(void *);
GELOGD("[%s] Done setting kernel args successfully. stub_func = %s, block_dim = %d, arg base = %p, arg size = %u",
op_desc.GetName().c_str(), stub_name_.c_str(), block_dim_, arg_base_, args_size_);

return SUCCESS;
}

Status AiCoreOpTask::ValidateTaskDef(const domi::TaskDef &task_def) {
auto task_type = static_cast<rtModelTaskType_t>(task_def.type());
if (task_type != RT_MODEL_TASK_KERNEL) {
GELOGE(INTERNAL_ERROR, "Invalid task type (%d) in AiCore CreateTask.", static_cast<int>(task_type));
return INTERNAL_ERROR;
}

const domi::KernelDef &kernel_def = task_def.kernel();
const domi::KernelContext &context = kernel_def.context();
auto kernel_type = static_cast<cce::ccKernelType>(context.kernel_type());
if (kernel_type != cce::ccKernelType::TE) {
GELOGE(INTERNAL_ERROR, "Invalid kernel type(%d) in AiCore TaskDef.", static_cast<int>(kernel_type));
return INTERNAL_ERROR;
}

return SUCCESS;
}

Status AiCoreOpTask::PrepareWithShape(TaskContext &context) {
if (tiling_buffer_ != nullptr) {
return UpdateTilingInfo(context);
}

return SUCCESS;
}

Status AiCoreOpTask::UpdateTilingInfo(TaskContext &context) {
auto node = context.GetNodeItem().node;
GE_CHECK_NOTNULL(node);
auto op_desc = node->GetOpDesc();
GE_CHECK_NOTNULL(op_desc);

GELOGD("[%s] Start to update tiling info for task: [%s]", node->GetName().c_str(), stub_name_.c_str());
OpRunInfo tiling_info;
tiling_info.block_dim = -1; // codex: Using uninitialized value

auto execution_context = context.GetExecutionContext();
RECORD_EXECUTION_EVENT(execution_context, context.GetNodeName(), "[CalcTilingInfo] Start");
GE_CHK_STATUS_RET(CalcTilingInfo(node, tiling_info));
RECORD_EXECUTION_EVENT(execution_context, context.GetNodeName(), "[CalcTilingInfo] End");

// update op args by tiling info
block_dim_ = static_cast<uint32_t>(tiling_info.block_dim);
op_desc->SetWorkspaceBytes(tiling_info.workspaces);

tiling_data_ = tiling_info.tiling_data.str();
if (tiling_data_.empty()) {
GELOGE(INTERNAL_ERROR, "[%s] Tiling data is empty.", stub_name_.c_str());
return INTERNAL_ERROR;
}

if (tiling_data_.size() > tiling_buffer_->GetSize()) {
GELOGE(INTERNAL_ERROR, "[%s] Tiling data size now (%zu) shouldn't larger than we alloc before (%zu).",
stub_name_.c_str(), tiling_data_.size(), tiling_buffer_->GetSize());
return INTERNAL_ERROR;
}

RECORD_EXECUTION_EVENT(execution_context, context.GetNodeName(), "[CopyTilingInfo] Start");
GE_CHK_RT_RET(rtMemcpy(tiling_buffer_->GetData(), tiling_buffer_->GetSize(), tiling_data_.c_str(),
tiling_data_.size(), RT_MEMCPY_HOST_TO_DEVICE));
RECORD_EXECUTION_EVENT(execution_context, context.GetNodeName(), "[CopyTilingInfo] End");

GELOGD("[%s] Done updating tiling info for task: [%s]", node->GetName().c_str(), stub_name_.c_str());
return SUCCESS;
}

Status AiCoreOpTask::CalcTilingInfo(const NodePtr &node, OpRunInfo &tiling_info) {
GELOGD("[%s] Start to invoke OpParaCalculate.", node->GetName().c_str());
GE_CHK_STATUS_RET(OpParaCalculate(*node, tiling_info), "Failed calc tiling data of node %s.",
node->GetName().c_str());
GELOGD("[%s] Done invoking OpParaCalculate successfully.", node->GetName().c_str());
return SUCCESS;
}

Status AiCoreOpTask::UpdateArgs(TaskContext &task_context) {
size_t expected_arg_count = task_context.NumInputs() + task_context.NumOutputs() + task_context.NumWorkspaces();
if (tiling_buffer_ != nullptr) {
++expected_arg_count;
}
if (expected_arg_count > max_arg_count_) {
GELOGE(INTERNAL_ERROR, "[%s] Invalid arg memory, max arg count = %u, but expect = %zu", GetName().c_str(),
max_arg_count_, expected_arg_count);
return INTERNAL_ERROR;
}

int index = 0;
for (int i = 0; i < task_context.NumInputs(); ++i) {
const auto input = task_context.GetInput(i);
GE_CHECK_NOTNULL(input);
arg_base_[index++] = reinterpret_cast<uintptr_t>(input->GetData());
}

for (int i = 0; i < task_context.NumOutputs(); ++i) {
const auto output = task_context.GetOutput(i);
GE_CHECK_NOTNULL(output);
arg_base_[index++] = reinterpret_cast<uintptr_t>(output->GetData());
}

int workspace_num = static_cast<int>(task_context.NumWorkspaces());
for (int i = 0; i < workspace_num; ++i) {
const auto workspace = task_context.MutableWorkspace(i);
GE_CHECK_NOTNULL(workspace);
arg_base_[index++] = reinterpret_cast<uintptr_t>(workspace);
}

if (tiling_buffer_ != nullptr) {
arg_base_[index++] = reinterpret_cast<uintptr_t>(tiling_buffer_->GetData());
}

if (task_context.IsTraceEnabled()) {
for (int i = 0; i < index; ++i) {
GELOGD("[%s] Arg[%d] = %lu", stub_name_.c_str(), i, arg_base_[i]);
}
}

return SUCCESS;
}

Status AiCoreOpTask::LaunchKernel(rtStream_t stream) {
GELOGD("AiCoreOpTask LaunchKernel Start (task = %s, block_dim = %u).", stub_name_.c_str(), block_dim_);
GE_CHK_RT_RET(rtKernelLaunch(stub_func_, block_dim_, args_.get(), args_size_, nullptr, stream));
GELOGD("AiCoreOpTask LaunchKernel End (task = %s, block_dim = %u).", stub_name_.c_str(), block_dim_);
return SUCCESS;
}

Status AiCoreOpTask::InitTilingInfo(const OpDesc &op_desc) {
bool dynamic_supported = false;
(void)AttrUtils::GetBool(op_desc, kAttrSupportDynamicShape, dynamic_supported);
if (!dynamic_supported) {
GELOGD("[%s] Dynamic shape is not supported.", op_desc.GetName().c_str());
return SUCCESS;
}

GELOGD("Start alloc tiling data of node %s.", op_desc.GetName().c_str());
int64_t max_size = -1;
(void)AttrUtils::GetInt(op_desc, GetKeyForOpParamSize(), max_size);
GELOGD("Got op param size by key: %s, ret = %ld", GetKeyForOpParamSize().c_str(), max_size);
if (max_size <= 0) {
GELOGE(PARAM_INVALID, "[%s] Invalid op_param_size: %ld.", op_desc.GetName().c_str(), max_size);
return PARAM_INVALID;
}

auto allocator = NpuMemoryAllocator::GetAllocator();
GE_CHECK_NOTNULL(allocator);
tiling_buffer_ = TensorBuffer::Create(allocator, static_cast<size_t>(max_size));
GE_CHECK_NOTNULL(tiling_buffer_);

GELOGD("[%s] Done allocating tiling buffer, size=%ld.", op_desc.GetName().c_str(), max_size);
return SUCCESS;
}

bool AiCoreOpTask::IsDynamicShapeSupported() { return tiling_buffer_ != nullptr; }

const std::string &AiCoreOpTask::GetName() const { return stub_name_; }

std::string AiCoreOpTask::GetKeyForOpParamSize() const { return kAttrOpParamSize; }

Status AtomicAddrCleanOpTask::Init(const OpDesc &op_desc, const domi::TaskDef &task_def) {
GE_CHK_STATUS_RET_NOLOG(AiCoreOpTask::Init(op_desc, task_def));
return InitAtomicAddrCleanIndices(op_desc);
}

Status AtomicAddrCleanOpTask::InitAtomicAddrCleanIndices(const OpDesc &op_desc) {
GELOGD("[%s] Start to setup AtomicAddrClean task.", op_desc.GetName().c_str());
std::vector<int64_t> atomic_output_indices;
(void)ge::AttrUtils::GetListInt(op_desc, ATOMIC_ATTR_OUTPUT_INDEX, atomic_output_indices);
map<string, map<int64_t, int64_t>> workspace_info; // op_name, ws_index, ws_offset
workspace_info = op_desc.TryGetExtAttr(EXT_ATTR_ATOMIC_WORKSPACE_INFO, workspace_info);
if (atomic_output_indices.empty() && workspace_info.empty()) {
GELOGE(INTERNAL_ERROR, "[%s] Neither ATOMIC_ATTR_OUTPUT_INDEX nor EXT_ATTR_ATOMIC_WORKSPACE_INFO is empty.",
op_desc.GetName().c_str());
return INTERNAL_ERROR;
}

for (auto output_index : atomic_output_indices) {
GELOGD("[%s] Adding output index [%ld]", op_desc.GetName().c_str(), output_index);
GE_CHECK_GE(output_index, 0);
GE_CHECK_LE(output_index, INT32_MAX);
atomic_output_indices_.emplace_back(static_cast<int>(output_index));
}

for (auto &iter : workspace_info) {
for (auto &info_iter : iter.second) {
auto workspace_index = info_iter.first;
GELOGD("[%s] Adding workspace index [%ld]", op_desc.GetName().c_str(), workspace_index);
GE_CHECK_GE(workspace_index, 0);
GE_CHECK_LE(workspace_index, INT32_MAX);
atomic_workspace_indices_.emplace_back(static_cast<int>(workspace_index));
}
}

size_t arg_count = atomic_workspace_indices_.size() + atomic_output_indices_.size();
if (tiling_buffer_ != nullptr) {
arg_count += 1;
}

if (arg_count > max_arg_count_) {
GELOGE(INTERNAL_ERROR, "[%s] Invalid arg memory, max arg count = %u, but expect = %zu", GetName().c_str(),
max_arg_count_, arg_count);
return INTERNAL_ERROR;
}
GELOGI("AiCoreOpTask LaunchKernel Start (task = %s, block_dim = %u).", stub_name_.c_str(), block_dim_);

GE_CHK_RT_RET(rtKernelLaunch(stub_func_, block_dim_, args_.get(), args_size_, nullptr, stream));
GELOGI("AiCoreOpTask LaunchKernel End (task = %s, block_dim = %u).", stub_name_.c_str(), block_dim_);
return SUCCESS;
}

std::string AtomicAddrCleanOpTask::GetKeyForOpParamSize() const { return kAttrAtomicOpParamSize; }

Status AtomicAddrCleanOpTask::UpdateArgs(TaskContext &task_context) {
// refresh atomic output addr
int index = 0;
for (auto atomic_output_index : atomic_output_indices_) {
const auto output_tensor = task_context.GetOutput(atomic_output_index);
GE_CHECK_NOTNULL(output_tensor);
arg_base_[index++] = reinterpret_cast<uintptr_t>(output_tensor->GetData());
}

// refresh atomic workspace addr
for (auto atomic_ws_index : atomic_workspace_indices_) {
const auto workspace_tensor = task_context.GetOutput(atomic_ws_index);
GE_CHECK_NOTNULL(workspace_tensor);
arg_base_[index++] = reinterpret_cast<uintptr_t>(workspace_tensor->GetData());
}

if (tiling_buffer_ != nullptr) {
arg_base_[index++] = reinterpret_cast<uintptr_t>(tiling_buffer_->GetData());
} else {
GELOGD("[%s] Not a dynamic op", GetName().c_str());
}

if (task_context.IsTraceEnabled()) {
for (int i = 0; i < index; ++i) {
GELOGD("[%s] Arg[%d] = %lu", GetName().c_str(), i, arg_base_[i]);
}
}

return SUCCESS;
}
} // namespace hybrid
} // namespace ge
} // namespace ge

+ 4
- 46
src/ge/hybrid/node_executor/aicore/aicore_op_task.h View File

@@ -18,69 +18,27 @@
#define GE_HYBRID_KERNEL_AICORE_OP_TASK_H_

#include <memory>
#include <vector>
#include "common/ge_inner_error_codes.h"
#include "runtime/stream.h"
#include "hybrid/common/tensor_value.h"
#include "hybrid/node_executor/task_context.h"
#include "proto/task.pb.h"
#include "register/op_tiling.h"

namespace ge {
namespace hybrid {
class AiCoreOpTask {
public:
AiCoreOpTask() = default;
virtual ~AiCoreOpTask() = default;

virtual Status Init(const OpDesc &op_desc, const domi::TaskDef &task_def);

bool IsDynamicShapeSupported();

// do preparation with shape(without actual io memory)
Status PrepareWithShape(TaskContext &context);

virtual Status UpdateArgs(TaskContext &task_context);

~AiCoreOpTask() = default;
Status LaunchKernel(rtStream_t stream);

const std::string &GetName() const;

protected:
Status UpdateTilingInfo(TaskContext &context);
virtual std::string GetKeyForOpParamSize() const;
virtual Status CalcTilingInfo(const NodePtr &node, optiling::OpRunInfo &tiling_info);

std::unique_ptr<TensorBuffer> tiling_buffer_ = nullptr;
std::string tiling_data_;
uintptr_t *arg_base_ = nullptr;
uint32_t max_arg_count_ = 0;

private:
static Status ValidateTaskDef(const domi::TaskDef &task_def);
Status InitWithTaskDef(const OpDesc &node, const domi::TaskDef &task_def);
Status InitTilingInfo(const OpDesc &op_desc);

friend class AiCoreTaskBuilder;
friend class AiCoreNodeTask;
std::string stub_name_;
void *stub_func_ = nullptr;
std::unique_ptr<uint8_t[]> args_ = nullptr;
uint32_t args_size_ = 0;
uint32_t block_dim_ = 1;
uint16_t offset_ = 0;
};

class AtomicAddrCleanOpTask : public AiCoreOpTask {
public:
Status Init(const OpDesc &op_desc, const domi::TaskDef &task_def) override;
Status UpdateArgs(TaskContext &task_context) override;

protected:
std::string GetKeyForOpParamSize() const override;

private:
Status InitAtomicAddrCleanIndices(const OpDesc &op_desc);
std::vector<int> atomic_output_indices_;
std::vector<int> atomic_workspace_indices_;
};
} // namespace hybrid
} // namespace ge
#endif // GE_HYBRID_KERNEL_AICORE_OP_TASK_H_

+ 54
- 56
src/ge/hybrid/node_executor/aicore/aicore_task_builder.cc View File

@@ -15,78 +15,76 @@
*/

#include "aicore_task_builder.h"
#include "common/debug/log.h"
#include "aicore_node_executor.h"
#include <mutex>
#include "graph/op_desc.h"
#include "cce/taskdown_common.hpp"
#include "framework/common/debug/log.h"
#include "graph/debug/ge_attr_define.h"

namespace ge {
namespace hybrid {
namespace {
const size_t kNumTaskWithAtomicAddrCleanTask = 2;
std::mutex g_reg_mutex;

AiCoreTaskBuilder::AiCoreTaskBuilder(const OpDescPtr &op_desc, const domi::KernelDef &kernel_def)
: op_desc_(op_desc), kernel_def_(kernel_def) {
std::string session_graph_id;
GE_IF_BOOL_EXEC(AttrUtils::GetStr(*op_desc_, ATTR_NAME_SESSION_GRAPH_ID, session_graph_id),
GELOGD("Get original type of session_graph_id."));
// get bin_file_key
stub_name_ = (session_graph_id.empty()) ? op_desc_->GetName() : session_graph_id + "_" + op_desc_->GetName();
}

Status AiCoreTaskBuilder::SetKernelArgs(AiCoreOpTask &task) {
const domi::KernelContext &context = kernel_def_.context();
// get kernel_type
auto kernel_type = static_cast<cce::ccKernelType>(context.kernel_type());
GE_CHK_BOOL_TRUE_EXEC_WITH_LOG(kernel_type != cce::ccKernelType::TE, return UNSUPPORTED,
"Invalid kernel type[%d] in AiCore TaskDef.", static_cast<int>(kernel_type));

task.args_size_ = kernel_def_.args_size();
task.block_dim_ = kernel_def_.block_dim();

// malloc args memory
task.args_.reset(new (std::nothrow) uint8_t[task.args_size_]);
// task.args_ = std::make_unique<uint8_t>(task.args_size_);
GE_CHECK_NOTNULL(task.args_);
errno_t err = memcpy_s(task.args_.get(), task.args_size_, kernel_def_.args().data(), task.args_size_);
GE_CHK_BOOL_TRUE_EXEC_WITH_LOG(err != EOK, return INTERNAL_ERROR, "AiCoreTask memcpy failed.");

const auto *args_offset_tmp = reinterpret_cast<uint16_t *>(const_cast<char *>(context.args_offset().data()));
GE_CHK_BOOL_TRUE_EXEC_WITH_LOG(context.args_offset().size() / sizeof(uint16_t) < 1, return FAILED,
"context.args_offset().size() / sizeof(uint16_t) less than 1");
task.offset_ = *args_offset_tmp;
return SUCCESS;
}

const char *AiCoreKernelRegistry::GetUnique(const string &stub_key) {
std::lock_guard<std::mutex> lock(mutex_);
auto it = unique_stubs_.find(stub_key);
if (it != unique_stubs_.end()) {
return it->c_str();
}
GE_IF_BOOL_EXEC(it != unique_stubs_.end(), return it->c_str());
it = unique_stubs_.insert(unique_stubs_.end(), stub_key);
return it->c_str();
}

AiCoreTaskBuilder::AiCoreTaskBuilder(const OpDescPtr &op_desc, const std::vector<domi::TaskDef> &task_defs)
: op_desc_(op_desc), task_defs_(task_defs) {}
Status AiCoreTaskBuilder::SetStub(AiCoreOpTask &task) {
AiCoreKernelRegistry &registry = AiCoreKernelRegistry::GetInstance();
std::lock_guard<std::mutex> lock(g_reg_mutex);
const char *unique_key = registry.GetUnique(stub_name_);

Status AiCoreTaskBuilder::BuildTask(std::unique_ptr<NodeTask> &node_task, bool ignore_failure_on_atomic) {
GE_CHECK_NOTNULL(op_desc_);
if (task_defs_.size() > kNumTaskWithAtomicAddrCleanTask) {
GELOGE(INTERNAL_ERROR, "[%s] At most 2 task was supported, but got %zu", op_desc_->GetName().c_str(),
task_defs_.size());
return INTERNAL_ERROR;
}

std::vector<std::unique_ptr<AiCoreOpTask>> op_tasks;
if (ExpectAtomicAddrCleanTask()) {
if (task_defs_.size() != kNumTaskWithAtomicAddrCleanTask) {
if (ignore_failure_on_atomic) {
GELOGI("[%s] AtomicAddrClean task was expected, but got %zu task_defs", op_desc_->GetName().c_str(),
task_defs_.size());
return SUCCESS;
} else {
GELOGE(INTERNAL_ERROR, "[%s] AtomicAddrClean task was expected, but got %zu task_defs",
op_desc_->GetName().c_str(), task_defs_.size());
return INTERNAL_ERROR;
}
}

GELOGD("[%s] Build AtomicAddrClean task.", op_desc_->GetName().c_str());
auto atomic_task = std::unique_ptr<AtomicAddrCleanOpTask>(new (std::nothrow) AtomicAddrCleanOpTask());
GE_CHECK_NOTNULL(atomic_task);
GE_CHK_STATUS_RET(atomic_task->Init(*op_desc_, task_defs_.front()), "[%s] Failed to init task for AtomicAddrClean",
op_desc_->GetName().c_str());
op_tasks.emplace_back(std::move(atomic_task));
}

// build aicore task
auto aicore_task = std::unique_ptr<AiCoreOpTask>(new (std::nothrow) AiCoreOpTask());
GE_CHECK_NOTNULL(aicore_task);
GE_CHK_STATUS_RET(aicore_task->Init(*op_desc_, task_defs_.back()), "[%s] Failed to init task for AtomicAddrClean",
op_desc_->GetName().c_str());
op_tasks.emplace_back(std::move(aicore_task));
GE_CHK_RT_RET(rtGetFunctionByName(unique_key, &(task.stub_func_)));
task.stub_name_ = stub_name_;

node_task.reset(new (std::nothrow) AiCoreNodeTask(std::move(op_tasks)));
GE_CHECK_NOTNULL(node_task);
return SUCCESS;
}

bool AiCoreTaskBuilder::ExpectAtomicAddrCleanTask() {
if (op_desc_->HasAttr(ATOMIC_ATTR_OUTPUT_INDEX)) {
GELOGD("[%s] Node has ATOMIC_ATTR_OUTPUT_INDEX", op_desc_->GetName().c_str());
return true;
}
map<string, map<int64_t, int64_t>> workspace_info;
workspace_info = op_desc_->TryGetExtAttr(EXT_ATTR_ATOMIC_WORKSPACE_INFO, workspace_info);

return !workspace_info.empty();
Status AiCoreTaskBuilder::BuildTask(AiCoreOpTask &task) {
GE_CHECK_NOTNULL(op_desc_);
GELOGI("AiCoreTaskBuilder[%s] BuildTask Start.", op_desc_->GetName().c_str());
GE_CHK_STATUS_RET_NOLOG(SetKernelArgs(task));
GE_CHK_STATUS_RET_NOLOG(SetStub(task));
GELOGI("AiCoreTaskBuilder[%s] BuildTask End.", op_desc_->GetName().c_str());
return SUCCESS;
}

} // namespace hybrid
} // namespace ge

+ 11
- 10
src/ge/hybrid/node_executor/aicore/aicore_task_builder.h View File

@@ -17,13 +17,14 @@
#ifndef GE_HYBRID_KERNEL_AICORE_TASK_BUILDER_H_
#define GE_HYBRID_KERNEL_AICORE_TASK_BUILDER_H_

#include <vector>
#include <mutex>
#include <string>
#include <map>
#include <set>
#include "aicore_op_task.h"
#include "framework/common/debug/ge_log.h"
#include "proto/task.pb.h"
#include "graph/utils/attr_utils.h"
#include "graph/op_kernel_bin.h"
#include "proto/task.pb.h"

namespace ge {
namespace hybrid {
@@ -44,16 +45,16 @@ class AiCoreKernelRegistry {

class AiCoreTaskBuilder {
public:
AiCoreTaskBuilder(const OpDescPtr &op_desc, const std::vector<domi::TaskDef> &task_defs);
AiCoreTaskBuilder(const OpDescPtr &op_desc, const domi::KernelDef &kernel_def);
~AiCoreTaskBuilder() = default;

Status BuildTask(std::unique_ptr<NodeTask> &node_task, bool ignore_failure_on_atomic);
Status BuildTask(AiCoreOpTask &task);

private:
bool ExpectAtomicAddrCleanTask();

OpDescPtr op_desc_;
const std::vector<domi::TaskDef> &task_defs_;
Status SetKernelArgs(AiCoreOpTask &task);
Status SetStub(AiCoreOpTask &task);
const OpDescPtr &op_desc_;
const domi::KernelDef &kernel_def_;
std::string stub_name_;
};
} // namespace hybrid
} // namespace ge


+ 7
- 4
src/ge/hybrid/node_executor/aicore/aicore_task_compiler.cc View File

@@ -34,6 +34,7 @@ Status AiCoreTaskCompiler::DoCompileOp(OpsKernelInfoStore &ops_store, const Node
GE_CHECK_NOTNULL(node);
vector<NodePtr> node_vec;
node_vec.emplace_back(node);
std::lock_guard<std::mutex> lk(mu_);
GE_CHK_STATUS_RET(ops_store.CompileOpRun(node_vec), "Failed to execute CompileOp, node = %s",
node->GetName().c_str());
GE_CHK_STATUS_RET(ops_store.CalcOpRunningParam(*node), "Failed to execute CalcOpRunningParam, node = %s",
@@ -43,8 +44,9 @@ Status AiCoreTaskCompiler::DoCompileOp(OpsKernelInfoStore &ops_store, const Node

Status AiCoreTaskCompiler::CompileOp(const NodePtr &node, std::vector<domi::TaskDef> &tasks) const {
GE_CHECK_NOTNULL(node);
GELOGI("AiCoreTaskCompiler(%s) CompileOp Start.", node->GetName().c_str());
GE_CHECK_NOTNULL(aic_kernel_store_);
GELOGI("AiCoreTaskCompiler[%s] CompileOp Start.", node->GetName().c_str());
GE_CHK_BOOL_TRUE_EXEC_WITH_LOG(aic_kernel_store_ == nullptr, return FAILED,
"Failed to get AiCore kernel store, node = %s", node->GetName().c_str());

GE_CHK_STATUS_RET_NOLOG(DoCompileOp(*aic_kernel_store_, node));
GELOGD("successfully compiled op: %s", node->GetName().c_str());
@@ -56,7 +58,7 @@ Status AiCoreTaskCompiler::CompileOp(const NodePtr &node, std::vector<domi::Task
op_desc->SetOutputOffset(output_offsets);
GE_CHK_STATUS_RET_NOLOG(DoGenerateTask(*aic_kernel_store_, *node, tasks));
GELOGD("successfully generated task: %s", node->GetName().c_str());
GELOGI("AiCoreTaskCompiler(%s) CompileOp End.", node->GetName().c_str());
GELOGI("AiCoreTaskCompiler[%s] CompileOp End.", node->GetName().c_str());
return SUCCESS;
}

@@ -89,5 +91,6 @@ Status AiCoreTaskCompiler::DoGenerateTask(OpsKernelInfoStore &store, const Node
GE_CHK_RT(rtModelDestroy(rt_model_));
return ret;
}

} // namespace hybrid
} // namespace ge
} // namespace ge

+ 1
- 0
src/ge/hybrid/node_executor/aicpu/aicpu_ext_info.cc View File

@@ -199,5 +199,6 @@ void AicpuExtInfoHandler::GetShapeAndType(const AicpuShapeAndType *shape_and_typ
data_type = static_cast<DataType>(shape_and_type->type);
shape = std::move(GeShape(dims));
}

} // namespace hybrid
} // namespace ge

+ 1
- 0
src/ge/hybrid/node_executor/aicpu/aicpu_ext_info.h View File

@@ -24,6 +24,7 @@

namespace ge {
namespace hybrid {

using AicpuShapeAndType = aicpu::FWKAdapter::ShapeAndType;
using AicpuExtInfo = aicpu::FWKAdapter::ExtInfo;



+ 27
- 52
src/ge/hybrid/node_executor/aicpu/aicpu_node_executor.cc View File

@@ -40,28 +40,19 @@ Status AicpuNodeTaskBase::AllocTensorBuffer(size_t size, std::unique_ptr<TensorB
}

Status AicpuNodeTaskBase::InitExtInfo(const std::string &kernel_ext_info) {
if (node_item_->is_dynamic) {
// dynamic node must have ext info
GE_CHK_STATUS_RET(aicpu_ext_handle_.Parse(kernel_ext_info),
"Node[%s] parse kernel ext info failed, kernel_ext_info_size=%zu.", node_name_.c_str(),
kernel_ext_info.size());
}

// if no ext info no need copy to device.
if (kernel_ext_info.empty()) {
GELOGI("Node[%s] kernel_ext_info is empty, no need copy to device, is_dynamic=%s.", node_name_.c_str(),
node_item_->is_dynamic ? "true" : "false");
return SUCCESS;
}
GE_CHK_STATUS_RET(aicpu_ext_handle_.Parse(kernel_ext_info),
"Node[%s] parse kernel ext info failed, kernel_ext_info_size=%zu.", node_name_.c_str(),
kernel_ext_info.size());

// copy task args buf
GE_CHK_STATUS_RET(AllocTensorBuffer(kernel_ext_info.size(), ext_info_addr_dev_),
"Node[%s] alloc kernel_ext_info buf failed, size=%zu", node_name_.c_str(), kernel_ext_info.size());

// copy default ext info to device
GE_CHK_RT_RET(rtMemcpy(ext_info_addr_dev_->GetData(), ext_info_addr_dev_->GetSize(), kernel_ext_info.data(),
kernel_ext_info.size(), RT_MEMCPY_HOST_TO_DEVICE));

// if no input and no output(DEPEND_COMPUTE equal no output), copy once, or else copy when update args.
if (node_item_->num_inputs == 0 && ((unknown_type_ == DEPEND_COMPUTE) || (node_item_->num_outputs == 0))) {
GE_CHK_RT_RET(rtMemcpy(ext_info_addr_dev_->GetData(), ext_info_addr_dev_->GetSize(), kernel_ext_info.data(),
kernel_ext_info.size(), RT_MEMCPY_HOST_TO_DEVICE));
}
return SUCCESS;
}

@@ -148,18 +139,16 @@ Status AicpuNodeTaskBase::UpdateExtInfo() {
}

Status AicpuNodeTaskBase::UpdateArgs(TaskContext &context) {
GELOGI("Node[%s] update args begin. is_dynamic=%s, unknown_type=%d", node_name_.c_str(),
node_item_->is_dynamic ? "true" : "false", unknown_type_);
GELOGI("Node[%s] update args begin. unknown_type=%d", node_name_.c_str(), unknown_type_);
if (node_item_->num_inputs == 0 && node_item_->num_outputs == 0) {
GELOGI("Node[%s] has no input and output, no need update args.", node_name_.c_str());
return SUCCESS;
}

GE_CHK_STATUS_RET(UpdateIoAddr(context), "Node[%s] update io addr failed.", node_name_.c_str());
if (node_item_->is_dynamic) {
// dynamic node need update ext info.
GE_CHK_STATUS_RET(UpdateExtInfo(), "Node[%s] update ext info failed.", node_name_.c_str());
}

GE_CHK_STATUS_RET(UpdateExtInfo(), "Node[%s] update ext info failed.", node_name_.c_str());

GELOGI("Node[%s] update args end.", node_name_.c_str());
return SUCCESS;
}
@@ -286,12 +275,9 @@ Status AicpuTfNodeTask::Init(const HybridModel &model) {

fwk_op_kernel.fwkKernelBase.fwk_kernel.workspaceBaseAddr = reinterpret_cast<uintptr_t>(kernel_workspace_->GetData());
fwk_op_kernel.fwkKernelBase.fwk_kernel.inputOutputAddr = reinterpret_cast<uintptr_t>(input_output_addr_->GetData());

if (ext_info_addr_dev_ != nullptr) {
// set ext info addr and ext info num
fwk_op_kernel.fwkKernelBase.fwk_kernel.extInfoAddr = reinterpret_cast<uintptr_t>(ext_info_addr_dev_->GetData());
fwk_op_kernel.fwkKernelBase.fwk_kernel.extInfoLen = ext_info_addr_dev_->GetSize();
}
// set ext info addr and ext info num
fwk_op_kernel.fwkKernelBase.fwk_kernel.extInfoAddr = reinterpret_cast<uintptr_t>(ext_info_addr_dev_->GetData());
fwk_op_kernel.fwkKernelBase.fwk_kernel.extInfoLen = ext_info_addr_dev_->GetSize();

fwk_op_kernel.fwkKernelBase.fwk_kernel.stepIDAddr = GetStepIdAddr(model);

@@ -520,8 +506,7 @@ Status AicpuTfNodeTask::UpdateIoAddr(TaskContext &context) {
io_addrs.emplace_back(reinterpret_cast<uintptr_t>(inputData->GetData()));
}

// known shape or not depend compute
if (!node_item_->is_dynamic || unknown_type_ != DEPEND_COMPUTE) {
if (unknown_type_ != DEPEND_COMPUTE) {
// unknown type 4 do this in call back.
GE_CHK_STATUS_RET_NOLOG(context.AllocateOutputs());
for (auto j = 0; j < node_item_->num_outputs; ++j) {
@@ -563,17 +548,14 @@ Status AicpuTfNodeTask::LaunchTask(TaskContext &context) {
}

Status AicpuTfNodeTask::TaskCallback(TaskContext &context) {
GELOGI("Node[%s] task callback start. is_dynamic=%s, unknown_type=%d.", node_name_.c_str(),
node_item_->is_dynamic ? "true" : "false", unknown_type_);
GELOGI("Node[%s] task callback start. unknown_type=%d.", node_name_.c_str(), unknown_type_);
Status callback_ret = SUCCESS;
if (node_item_->is_dynamic) {
// check need update shape, call update shape.
if (unknown_type_ == DEPEND_SHAPE_RANGE) {
// check result
callback_ret = UpdateOutputShapeFromExtInfo();
} else if (unknown_type_ == DEPEND_COMPUTE) {
callback_ret = UpdateShapeAndDataByResultSummary(context);
}
// check need update shape, call update shape.
if (unknown_type_ == DEPEND_SHAPE_RANGE) {
// check result
callback_ret = UpdateOutputShapeFromExtInfo();
} else if (unknown_type_ == DEPEND_COMPUTE) {
callback_ret = UpdateShapeAndDataByResultSummary(context);
}
GELOGI("Node[%s] task callback end.", node_name_.c_str());
return callback_ret;
@@ -630,13 +612,8 @@ Status AicpuNodeTask::Init(const HybridModel &model) {

GE_CHK_STATUS_RET(InitExtInfo(kernel_ext_info), "Node[%s] init ext info failed.", node_name.c_str());

if (ext_info_addr_dev_ == nullptr) {
aicpu_param_head->extInfoLength = 0;
aicpu_param_head->extInfoAddr = 0;
} else {
aicpu_param_head->extInfoLength = ext_info_addr_dev_->GetSize();
aicpu_param_head->extInfoAddr = reinterpret_cast<uintptr_t>(ext_info_addr_dev_->GetData());
}
aicpu_param_head->extInfoLength = ext_info_addr_dev_->GetSize();
aicpu_param_head->extInfoAddr = reinterpret_cast<uintptr_t>(ext_info_addr_dev_->GetData());

GELOGI("Node[%s] init end.", node_name.c_str());
return SUCCESS;
@@ -687,12 +664,10 @@ Status AicpuNodeTask::LaunchTask(TaskContext &context) {
}

Status AicpuNodeTask::TaskCallback(TaskContext &context) {
GELOGI("Node[%s] task callback start, is_dynamic = %s, unknown_type=%d.", node_name_.c_str(),
node_item_->is_dynamic ? "true" : "false", unknown_type_);
GELOGI("Node[%s] task callback start, unknown_type=%d.", node_name_.c_str(), unknown_type_);
Status callback_ret = SUCCESS;

// check need update shape, call update shape.
if (node_item_->is_dynamic && unknown_type_ == DEPEND_SHAPE_RANGE) {
if (unknown_type_ == DEPEND_SHAPE_RANGE) {
// check result
callback_ret = UpdateOutputShapeFromExtInfo();
} else {


+ 2
- 2
src/ge/hybrid/node_executor/aicpu/aicpu_node_executor.h View File

@@ -24,6 +24,7 @@

namespace ge {
namespace hybrid {

class AicpuNodeTaskBase : public NodeTask {
public:
AicpuNodeTaskBase(const NodeItem *node_item, const domi::TaskDef &task_def)
@@ -69,10 +70,8 @@ class AicpuNodeTaskBase : public NodeTask {

const std::string node_type_;

// valid when node_item_->is_dynamic is true
UnknowShapeOpType unknown_type_ = DEPEND_IN_SHAPE;

// valid when node_item_->is_dynamic is true
AicpuExtInfoHandler aicpu_ext_handle_;

// ext info addr, device mem
@@ -170,6 +169,7 @@ class AiCpuNodeExecutor : public NodeExecutor {

Status PrepareTask(NodeTask &task, TaskContext &context) const override;
};

} // namespace hybrid
} // namespace ge
#endif // GE_HYBRID_KERNEL_AICPU_NODE_EXECUTOR_H_

+ 4
- 5
src/ge/hybrid/node_executor/compiledsubgraph/known_node_executor.cc View File

@@ -26,6 +26,7 @@

namespace ge {
namespace hybrid {

REGISTER_NODE_EXECUTOR_BUILDER(NodeExecutorManager::ExecutorType::COMPILED_SUBGRAPH, KnownNodeExecutor);

Status KnownNodeTask::ExecuteAsync(TaskContext &context, std::function<void()> done_callback) {
@@ -97,11 +98,8 @@ Status KnownNodeTask::Init(TaskContext &context) {
GE_CHK_STATUS_RET(context.AllocateOutputs(), "known node task allocate output failed.");

// init davinicmodel
if (!load_flag_) {
davinci_model_->InitRuntimeParams();
GE_CHK_STATUS_RET(davinci_model_->InitVariableMem(), "init variable mem failed.");
}

davinci_model_->InitRuntimeParams();
GE_CHK_STATUS_RET(davinci_model_->InitVariableMem(), "init variable mem failed.");
// allocate mem base
void *buffer = nullptr;
if (davinci_model_->TotalMemSize() != 0) {
@@ -163,5 +161,6 @@ Status KnownNodeExecutor::ExecuteTask(NodeTask &task, TaskContext &context,
context.GetNodeItem().NodeName().c_str());
return SUCCESS;
}

} // namespace hybrid
} // namespace ge

+ 0
- 318
src/ge/hybrid/node_executor/controlop/control_op_executor.cc View File

@@ -1,318 +0,0 @@
/**
* Copyright 2019-2020 Huawei Technologies Co., Ltd
*
* 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 "control_op_executor.h"
#include "graph/utils/node_utils.h"
#include "hybrid/executor/hybrid_execution_context.h"
#include "hybrid/executor/subgraph_executor.h"

namespace ge {
namespace hybrid {
REGISTER_NODE_EXECUTOR_BUILDER(NodeExecutorManager::ExecutorType::CONTROL_OP, ControlOpNodeExecutor);

Status ControlOpNodeTask::ExecuteSubgraph(const GraphItem *subgraph, TaskContext &task_context,
const std::function<void()> &done_callback) {
GELOGD("[%s] Start to execute subgraph.", subgraph->GetName().c_str());
auto execution_context = const_cast<GraphExecutionContext *>(task_context.GetExecutionContext());
auto executor = MakeShared<SubgraphExecutor>(subgraph, execution_context, task_context.IsForceInferShape());
GE_CHECK_NOTNULL(executor);
GE_CHK_STATUS_RET(executor->ExecuteAsync(task_context), "[%s] Failed to execute partitioned call.",
subgraph->GetName().c_str());

auto callback = [executor, done_callback]() mutable {
if (done_callback != nullptr) {
done_callback();
}
// executor must outlive task context
executor.reset();
};

GE_CHK_STATUS_RET_NOLOG(task_context.RegisterCallback(callback));
GELOGD("[%s] Done executing subgraph successfully.", subgraph->GetName().c_str());
return SUCCESS;
}

Status ControlOpNodeTask::CopyTensorValueToHost(const TensorValue &tensor, int32_t &value) {
GE_CHECK_NOTNULL(tensor.GetData());
GE_CHECK_GE(tensor.GetSize(), sizeof(value));
GE_CHK_RT_RET(rtMemcpy(&value, sizeof(value), tensor.GetData(), sizeof(value), RT_MEMCPY_DEVICE_TO_HOST));
return SUCCESS;
}

Status ControlOpNodeTask::UpdateArgs(TaskContext &context) {
// do nothing
return SUCCESS;
}

Status ControlOpNodeTask::ExecuteAsync(TaskContext &task_context, std::function<void()> done_callback) {
auto ret = DoExecuteAsync(task_context, done_callback);
task_context.SetStatus(ret);

if (done_callback) {
done_callback();
}

return ret;
}

Status IfOpNodeTask::Init(const NodePtr &node, const HybridModel &model) {
GELOGD("[%s] Start to init IfOpNodeTask.", node->GetName().c_str());
auto then_subgraph = NodeUtils::GetSubgraph(*node, kThenBranchIndex);
GE_CHECK_NOTNULL(then_subgraph);
GELOGD("[%s] Adding subgraph [%s] to then-subgraph.", node->GetName().c_str(), then_subgraph->GetName().c_str());
then_ = model.GetSubgraphItem(then_subgraph);
GE_CHECK_NOTNULL(then_);

auto else_subgraph = NodeUtils::GetSubgraph(*node, kElseBranchIndex);
GE_CHECK_NOTNULL(else_subgraph);
GELOGD("[%s] Adding subgraph [%s] to else-subgraph.", node->GetName().c_str(), else_subgraph->GetName().c_str());
else_ = model.GetSubgraphItem(else_subgraph);
GE_CHECK_NOTNULL(else_);

GELOGD("[%s] Done initialization successfully.", node->GetName().c_str());
return SUCCESS;
}

const GraphItem *IfOpNodeTask::SelectBranch(int32_t cond) const { return cond != 0 ? then_ : else_; }

Status IfOpNodeTask::DoExecuteAsync(TaskContext &task_context, const std::function<void()> &done_callback) const {
auto cond_tensor = task_context.GetInput(kIfCondIndex);
GE_CHECK_NOTNULL(cond_tensor);
int32_t cond_val = 0;
GE_CHK_STATUS_RET(CopyTensorValueToHost(*cond_tensor, cond_val), "[%s] Failed to get cond value.",
task_context.GetNodeName());

auto subgraph = SelectBranch(cond_val);
GELOGD("[%s] Taking subgraph [%s] by cond = [%d]", task_context.GetNodeName(), subgraph->GetName().c_str(), cond_val);
GE_CHK_STATUS_RET(ExecuteSubgraph(subgraph, task_context, done_callback),
"[%s] Failed to execute subgraph. cond = %d", task_context.GetNodeName(), cond_val);

GELOGD("[%s] Done executing with cond = %d successfully.", task_context.GetNodeName(), cond_val);
return SUCCESS;
}

Status CaseOpNodeTask::Init(const NodePtr &node, const HybridModel &model) {
size_t num_subgraphs = node->GetOpDesc()->GetSubgraphInstanceNames().size();
GE_CHECK_LE(num_subgraphs, kMaxBranchNum);
GE_CHECK_GE(num_subgraphs, kMinBranchNum);
auto num_branches = static_cast<uint32_t>(num_subgraphs);
GELOGD("[%s] Start to init CaseOpNodeTask with %u branches.", node->GetName().c_str(), num_branches);

for (uint32_t i = 0; i < num_branches; ++i) {
auto sub_graph = NodeUtils::GetSubgraph(*node, i);
GE_CHECK_NOTNULL(sub_graph);
auto graph_item = model.GetSubgraphItem(sub_graph);
GE_CHECK_NOTNULL(graph_item);
GELOGD("[%s] Adding subgraph [%s] to branch %u.", node->GetName().c_str(), sub_graph->GetName().c_str(), i);
subgraphs_.emplace_back(graph_item);
}

GELOGD("[%s] Done initialization successfully.", node->GetName().c_str());
return SUCCESS;
}

const GraphItem *CaseOpNodeTask::SelectBranch(int32_t branch_index) const {
// subgraphs_ is non-empty. checked int Init
if (branch_index < 0 || static_cast<size_t>(branch_index) >= subgraphs_.size()) {
GELOGI("Branch index out of range. index = %d, num_subgraphs = %zu, will taking last branch.", branch_index,
subgraphs_.size());
branch_index = subgraphs_.size() - 1;
}

return subgraphs_[branch_index];
}

Status CaseOpNodeTask::DoExecuteAsync(TaskContext &task_context, const std::function<void()> &done_callback) const {
auto branch_tensor = task_context.GetInput(kCaseBranchIndex);
GE_CHECK_NOTNULL(branch_tensor);
int32_t branch_index = 0;
GE_CHK_STATUS_RET(CopyTensorValueToHost(*branch_tensor, branch_index), "[%s] Failed to get branch index.",
task_context.GetNodeName());

const GraphItem *subgraph = SelectBranch(branch_index);
GELOGI("[%s] Taking subgraph [%s] by branch = [%d]", task_context.GetNodeName(), subgraph->GetName().c_str(),
branch_index);

std::vector<TensorValue> inputs;
std::vector<TensorValue> outputs;
for (int i = 0; i < task_context.NumInputs(); ++i) {
auto input_tensor = task_context.GetInput(i);
GE_CHECK_NOTNULL(input_tensor);
inputs.emplace_back(*input_tensor);
}

GE_CHK_STATUS_RET(ExecuteSubgraph(subgraph, task_context, done_callback), "[%s] Failed to execute else-subgraph.",
task_context.GetNodeName());

GELOGD("[%s] Done executing subgraph[%d] successfully.", task_context.GetNodeName(), branch_index);
return SUCCESS;
}

Status WhileOpNodeTask::Init(const NodePtr &node, const HybridModel &model) {
GELOGD("[%s] Start to init WhileOpNodeTask.", node->GetName().c_str());
auto cond_subgraph = NodeUtils::GetSubgraph(*node, kCondBranchIndex);
GE_CHECK_NOTNULL(cond_subgraph);
GELOGD("[%s] Adding subgraph [%s] to cond-subgraph.", node->GetName().c_str(), cond_subgraph->GetName().c_str());
cond_ = model.GetSubgraphItem(cond_subgraph);
GE_CHECK_NOTNULL(cond_);

auto body_subgraph = NodeUtils::GetSubgraph(*node, kBodyBranchIndex);
GE_CHECK_NOTNULL(body_subgraph);
GELOGD("[%s] Adding subgraph [%s] to body-subgraph.", node->GetName().c_str(), body_subgraph->GetName().c_str());
body_ = model.GetSubgraphItem(body_subgraph);
GE_CHECK_NOTNULL(body_);

GELOGD("[%s] Done initialization successfully.", node->GetName().c_str());
return SUCCESS;
}

Status WhileOpNodeTask::DoExecuteAsync(TaskContext &task_context, const std::function<void()> &done_callback) const {
if (task_context.NumInputs() != task_context.NumOutputs()) {
GELOGE(INTERNAL_ERROR, "[%s] Invalid while args. num_inputs = %d, num_outputs = %d", task_context.GetNodeName(),
task_context.NumInputs(), task_context.NumOutputs());
return INTERNAL_ERROR;
}

// graph build can not set accurate flag unknown_shape_status by now.
// Treating all nodes in while scope as unknown shape.
task_context.SetForceInferShape(true);

int iteration = 0;
while (true) {
bool is_continue = false;
GELOGD("[%s] Start to execute, iteration = %d", task_context.GetNodeName(), iteration);
GE_CHK_STATUS_RET(ExecuteOneLoop(task_context, is_continue), "[%s] Failed to execute iteration %d.",
task_context.GetNodeName(), iteration);

if (!is_continue) {
GELOGD("[%s] Quit from loop. current iteration = %d", task_context.GetNodeName(), iteration);
break;
}

++iteration;
}

return SUCCESS;
}

Status WhileOpNodeTask::ExecuteCond(TaskContext &task_context, bool &is_continue) const {
std::vector<TensorValue> inputs;
std::vector<ConstGeTensorDescPtr> input_desc;
std::vector<ConstGeTensorDescPtr> output_desc;
for (int i = 0; i < task_context.NumInputs(); ++i) {
auto input_tensor = task_context.GetInput(i);
GE_CHECK_NOTNULL(input_tensor);
inputs.emplace_back(*input_tensor);
input_desc.emplace_back(task_context.GetInputDesc(i));
}

auto execution_context = const_cast<GraphExecutionContext *>(task_context.GetExecutionContext());
auto executor = MakeShared<SubgraphExecutor>(cond_, execution_context, task_context.IsForceInferShape());
GE_CHECK_NOTNULL(executor);
GELOGD("[%s] Start to execute cond-subgraph.", task_context.GetNodeName());
GE_CHK_STATUS_RET(executor->ExecuteAsync(inputs, input_desc), "Failed to execute partitioned call.");
GELOGD("[%s] Done executing cond-subgraph successfully.", cond_->GetName().c_str());
GE_CHK_STATUS_RET_NOLOG(task_context.RegisterCallback([executor]() mutable { executor.reset(); }));

// get cond output
GE_CHK_STATUS_RET(executor->Synchronize(), "[%s] Failed to sync cond-subgraph result.", cond_->GetName().c_str());
std::vector<TensorValue> cond_outputs;
GE_CHK_STATUS_RET(executor->GetOutputs(cond_outputs), "[%s] Failed to get cond-output.", cond_->GetName().c_str());
if (cond_outputs.empty()) {
GELOGE(INTERNAL_ERROR, "[%s] Cond output is empty.", task_context.GetNodeName());
return INTERNAL_ERROR;
}

int cond_val = 0;
GE_CHK_STATUS_RET(CopyTensorValueToHost(cond_outputs[0], cond_val), "[%s] Failed to get cond result.",
task_context.GetNodeName());
is_continue = cond_val != 0;
return SUCCESS;
}

Status WhileOpNodeTask::MoveOutputs2Inputs(TaskContext &task_context) {
// set outputs to inputs for next iteration
for (int i = 0; i < task_context.NumInputs(); ++i) {
auto input_tensor = task_context.MutableInput(i);
auto output_tensor = task_context.MutableOutput(i);
GE_CHECK_NOTNULL(input_tensor);
GE_CHECK_NOTNULL(output_tensor);
*input_tensor = *output_tensor;
output_tensor->Destroy();

auto output_tensor_desc = task_context.MutableOutputDesc(i);
GE_CHECK_NOTNULL(output_tensor_desc);
GELOGD("[%s] To update input shape[%d] by output shape. from [%s] to [%s]", task_context.GetNodeName(), i,
task_context.MutableInputDesc(i)->GetShape().ToString().c_str(),
output_tensor_desc->GetShape().ToString().c_str());
*task_context.MutableInputDesc(i) = *output_tensor_desc;
}

return SUCCESS;
}

Status WhileOpNodeTask::ExecuteOneLoop(TaskContext &task_context, bool &is_continue) const {
GE_CHK_STATUS_RET(ExecuteCond(task_context, is_continue), "[%s] Failed to execute cond-subgraph",
task_context.GetNodeName());
if (!is_continue) {
for (int i = 0; i < task_context.NumInputs(); ++i) {
auto input_tensor = task_context.GetInput(i);
GE_CHECK_NOTNULL(input_tensor);
task_context.SetOutput(i, *input_tensor);
}
return SUCCESS;
}

GELOGD("[%s] Start to execute body-subgraph.", task_context.GetNodeName());
GE_CHK_STATUS_RET(ExecuteSubgraph(body_, task_context, nullptr), "[%s] Failed to execute cond-subgraph",
task_context.GetNodeName());
GELOGD("[%s] Done executing body-subgraph successfully.", task_context.GetNodeName());

// set outputs to inputs for next iteration
GE_CHK_STATUS_RET(MoveOutputs2Inputs(task_context), "[%s] Failed to move outputs to inputs",
task_context.GetNodeName());

return SUCCESS;
}

Status ControlOpNodeExecutor::LoadTask(const HybridModel &model, const NodePtr &node,
shared_ptr<NodeTask> &task) const {
auto node_item = model.GetNodeItem(node);
GE_CHECK_NOTNULL(node_item);

unique_ptr<ControlOpNodeTask> node_task;
auto node_type = node->GetType();
if (node_type == IF) {
node_task.reset(new (std::nothrow) IfOpNodeTask());
} else if (node_type == CASE) {
node_task.reset(new (std::nothrow) CaseOpNodeTask());
} else if (node_type == WHILE) {
node_task.reset(new (std::nothrow) WhileOpNodeTask());
} else {
GELOGE(PARAM_INVALID, "[%s] Unsupported type: %s", node->GetName().c_str(), node_type.c_str());
return PARAM_INVALID;
}

GE_CHECK_NOTNULL(node_task);
GE_CHK_STATUS_RET(node_task->Init(node, model), "[%s] Failed to init ControlOpNodeTask.", node->GetName().c_str());

task = std::move(node_task);
return SUCCESS;
}

Status ControlOpNodeExecutor::PrepareTask(NodeTask &task, TaskContext &context) const { return SUCCESS; }
} // namespace hybrid
} // namespace ge

+ 0
- 100
src/ge/hybrid/node_executor/controlop/control_op_executor.h View File

@@ -1,100 +0,0 @@
/**
* Copyright 2019-2020 Huawei Technologies Co., Ltd
*
* 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.
*/

#ifndef GE_HYBRID_CONTROLOP_CONTROL_OP_EXECUTOR_H_
#define GE_HYBRID_CONTROLOP_CONTROL_OP_EXECUTOR_H_

#include <vector>
#include "hybrid/node_executor/node_executor.h"
#include "hybrid/model/graph_item.h"

namespace ge {
namespace hybrid {
class ControlOpNodeTask : public NodeTask {
public:
virtual Status Init(const NodePtr &node, const HybridModel &model) = 0;
Status UpdateArgs(TaskContext &context) override;

Status ExecuteAsync(TaskContext &task_context, std::function<void()> done_callback) override;

protected:
virtual Status DoExecuteAsync(TaskContext &task_context, const std::function<void()> &done_callback) const = 0;
static Status CopyTensorValueToHost(const TensorValue &tensor_value, int32_t &value);
static Status ExecuteSubgraph(const GraphItem *subgraph, TaskContext &task_context,
const std::function<void()> &done_callback);
};

class IfOpNodeTask : public ControlOpNodeTask {
public:
Status Init(const NodePtr &node, const HybridModel &model) override;

protected:
const GraphItem *SelectBranch(int32_t cond) const;
Status DoExecuteAsync(TaskContext &task_context, const std::function<void()> &done_callback) const override;

private:
static constexpr int kIfCondIndex = 0;
static constexpr int kThenBranchIndex = 0;
static constexpr int kElseBranchIndex = 1;

const GraphItem *then_ = nullptr;
const GraphItem *else_ = nullptr;
};

class CaseOpNodeTask : public ControlOpNodeTask {
public:
Status Init(const NodePtr &node, const HybridModel &model) override;

protected:
const GraphItem *SelectBranch(int32_t branch_index) const;
Status DoExecuteAsync(TaskContext &task_context, const std::function<void()> &done_callback) const override;

private:
static constexpr int kCaseBranchIndex = 0;
static constexpr size_t kMaxBranchNum = INT32_MAX;
static constexpr size_t kMinBranchNum = 1;

std::vector<const GraphItem *> subgraphs_;
};

class WhileOpNodeTask : public ControlOpNodeTask {
public:
Status Init(const NodePtr &node, const HybridModel &model) override;

protected:
Status DoExecuteAsync(TaskContext &task_context, const std::function<void()> &done_callback) const override;
Status ExecuteCond(TaskContext &task_context, bool &is_continue) const;

static Status MoveOutputs2Inputs(TaskContext &task_context);

Status ExecuteOneLoop(TaskContext &task_context, bool &is_continue) const;

private:
static constexpr int kCondBranchIndex = 0;
static constexpr int kBodyBranchIndex = 1;

const GraphItem *cond_ = nullptr;
const GraphItem *body_ = nullptr;
};

class ControlOpNodeExecutor : public NodeExecutor {
public:
Status LoadTask(const HybridModel &model, const NodePtr &node, shared_ptr<NodeTask> &task) const override;
Status PrepareTask(NodeTask &task, TaskContext &context) const override;
};
} // namespace hybrid
} // namespace ge
#endif // GE_HYBRID_CONTROLOP_CONTROL_OP_EXECUTOR_H_

+ 0
- 207
src/ge/hybrid/node_executor/hccl/hccl_node_executor.cc View File

@@ -1,207 +0,0 @@
/**
* Copyright 2019-2020 Huawei Technologies Co., Ltd
*
* 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 "hybrid/node_executor/hccl/hccl_node_executor.h"
#include "graph/manager/util/hcom_util.h"
#include "framework/common/debug/ge_log.h"
#include "framework/common/fmk_error_codes.h"
#include "common/ge/ge_util.h"
#include "common/ge/plugin_manager.h"
#include "graph/attr_value.h"
#include "graph/debug/ge_attr_define.h"
#include "hccl/hcom.h"

namespace ge {
namespace hybrid {

REGISTER_NODE_EXECUTOR_BUILDER(NodeExecutorManager::ExecutorType::HCCL, HcclNodeExecutor);

Status HcclNodeTask::ExecuteAsync(TaskContext &context, std::function<void()> done_callback) {
GELOGI("[%s] HcclNodeTask::ExecuteAsync in.", context.GetNodeName());
if (context.handle_ == nullptr) {
GELOGE(FAILED, "hccl handle is nullptr! ");
return FAILED;
}
auto EnqueueHcomOpertion = (hcclResult_t(*)(HcomOpertion, std::function<void(hcclResult_t status)>))dlsym(
context.handle_, "EnqueueHcomOpertion");
if (EnqueueHcomOpertion == nullptr) {
GELOGE(FAILED, "Failed to invoke EnqueueHcomOpertion hcom unknown node function.");
if (dlclose(context.handle_) != 0) {
GELOGW("Failed to close handle %s", dlerror());
}
return FAILED;
}

vector<void *> inputs;
for (int i = 0; i < context.NumInputs(); ++i) {
TensorValue *tv = context.MutableInput(i);
GE_CHECK_NOTNULL(tv);
inputs.emplace_back(tv->MutableData());
}

vector<void *> outputs;
for (int i = 0; i < context.NumOutputs(); ++i) {
TensorValue *tv = context.MutableOutput(i);
GE_CHECK_NOTNULL(tv);
outputs.emplace_back(tv->MutableData());
}

const NodeItem &node_item = context.GetNodeItem();
const OpDescPtr op_desc = MakeShared<OpDesc>(*(node_item.op_desc));
GE_CHECK_NOTNULL(op_desc);

HcomOpertion op_info;
op_info.hcclType = op_desc->GetType();
op_info.inputPtr = inputs.empty() ? nullptr : inputs[0];
op_info.outputPtr = outputs.empty() ? nullptr : outputs[0];
ge::DataType src_data_type = op_desc->GetInputDescPtr(0)->GetDataType();
auto iter = kConstOpHcclDataType.find(static_cast<int64_t>(src_data_type));
if (iter == kConstOpHcclDataType.end()) {
GELOGE(PARAM_INVALID, "kConstOpHcclDataType find failed.");
return PARAM_INVALID;
}
op_info.dataType = iter->second;
hcclRedOp_t op_type = HCCL_REP_OP_SUM;
if (op_desc->GetType() == HCOMALLREDUCE || op_desc->GetType() == HCOMREDUCESCATTER ||
op_desc->GetType() == HVDCALLBACKALLREDUCE) {
GE_CHK_STATUS_RET(HcomOmeUtil::GetHcclOperationType(op_desc, op_type), "GetHcclOperationType failed");
op_info.opType = op_type;
}
int64_t root_id = 0;
if (op_desc->GetType() == HCOMBROADCAST) {
GE_CHK_STATUS_RET(HcomOmeUtil::GetHcclRootId(op_desc, root_id), "GetHcclRootId failed");
}
op_info.root = root_id;
auto callback = [this](hcclResult_t status) {
if (status != HCCL_SUCCESS) {
GELOGE(HCCL_E_INTERNAL, "Call HcomExcutorInitialize failed, ret: 0x%X", status);
}
std::lock_guard<std::mutex> lock(this->hccl_mutex_);
this->cond_.notify_all();
GELOGI("hccl callback success.");
};
int32_t count = 0;
GE_CHK_STATUS_RET(HcomOmeUtil::GetHcomCount(op_desc, static_cast<hcclDataType_t>(op_info.dataType), false, count),
"GetHcomCount failed");
GELOGI("[%s] HcclNodeTask::ExecuteAsync hccl_type %s, count %d, data_type %d, op_type %d, root %d.",
context.GetNodeName(), op_info.hcclType.c_str(), count, op_info.dataType, op_info.opType, op_info.root);
op_info.count = count;

hcclResult_t hccl_ret = EnqueueHcomOpertion(op_info, callback);
if (hccl_ret != HCCL_SUCCESS) {
GELOGE(HCCL_E_INTERNAL, "Call HcomExcutorInitialize failed, ret: 0x%X", hccl_ret);
return HCCL_E_INTERNAL;
}

// pending until hccl finished
std::unique_lock<std::mutex> ulock(hccl_mutex_);
cond_.wait(ulock);

context.RegisterCallback(done_callback);
GELOGI("[%s] HcclNodeTask::ExecuteAsync success.", context.GetNodeName());
return SUCCESS;
}

Status HcclNodeTask::UpdateArgs(TaskContext &context) { return SUCCESS; }

Status HcclNodeTask::Init(TaskContext &context) {
GELOGI("[%s] HcclNodeExecutor::Init success.", context.GetNodeName());
return SUCCESS;
}

Status HcclNodeExecutor::PrepareTask(NodeTask &task, TaskContext &context) const {
GELOGI("[%s] HcclNodeExecutor::PrepareTask in.", context.GetNodeName());

GE_CHK_STATUS_RET(task.Init(context), "hccl node load hccl so failed.");
// allocate output mem
GE_CHK_STATUS_RET(context.AllocateOutputs(), "hccl node task allocate output failed.");

GE_CHK_STATUS_RET(task.UpdateArgs(context), "hccl node task update args failed.");
GELOGI("[%s] HcclNodeExecutor::PrepareTask success.", context.GetNodeName());
return SUCCESS;
}

Status HcclNodeExecutor::LoadTask(const HybridModel &model, const NodePtr &node, shared_ptr<NodeTask> &task) const {
GELOGI("[%s] HcclNodeExecutor::LoadTask in.", node->GetName().c_str());
GE_CHECK_NOTNULL(node);

task = MakeShared<HcclNodeTask>();
GE_CHECK_NOTNULL(task);
GELOGI("[%s] HcclNodeExecutor::LoadTask success.", node->GetName().c_str());
return SUCCESS;
}

Status HcclNodeExecutor::ExecuteTask(NodeTask &task, TaskContext &context,
const std::function<void()> &callback) const {
context.handle_ = handle_;
GE_CHK_STATUS_RET(task.ExecuteAsync(context, callback), "Failed to execute task. node = %s",
context.GetNodeItem().NodeName().c_str());
return SUCCESS;
}

Status HcclNodeExecutor::Initialize() {
std::string file_name = "libhccl.so";
std::string path = PluginManager::GetPath();
path.append(file_name);
string canonical_path = RealPath(path.c_str());
if (canonical_path.empty()) {
GELOGW("failed to get realpath of %s", path.c_str());
return FAILED;
}

GELOGI("FileName:%s, Path:%s.", file_name.c_str(), canonical_path.c_str());
handle_ = dlopen(canonical_path.c_str(), RTLD_NOW | RTLD_GLOBAL);
if (handle_ == nullptr) {
GELOGE(GE_PLGMGR_SO_NOT_EXIST, "Failed in dlopen %s! ", dlerror());
return FAILED;
}
auto HcomExcutorInitialize = (hcclResult_t(*)())dlsym(handle_, "HcomExcutorInitialize");
if (HcomExcutorInitialize == nullptr) {
GELOGE(FAILED, "Failed to invoke HcomExcutorInitialize hcom unknown node function.");
return FAILED;
}
hcclResult_t hccl_ret = HcomExcutorInitialize();
if (hccl_ret == HCCL_E_PTR) {
GELOGI("Hccl comm is null, hcom executor initialize is not required.");
} else if (hccl_ret == HCCL_SUCCESS) {
GELOGI("Hcom executor initialize success.");
} else {
GELOGE(FAILED, "Call HcomExcutorInitialize failed, ret: 0x%X", hccl_ret);
return FAILED;
}
return SUCCESS;
}

Status HcclNodeExecutor::Finalize() {
auto HcomExcutorFinalize = (hcclResult_t(*)())dlsym(handle_, "HcomExcutorFinalize");
if (HcomExcutorFinalize == nullptr) {
GELOGE(FAILED, "Failed to invoke HcomExcutorFinalize hcom unknown node function.");
return FAILED;
}
hcclResult_t hccl_ret = HcomExcutorFinalize();
if (hccl_ret != HCCL_SUCCESS) {
GELOGE(FAILED, "Call HcomExcutorFinalize failed, ret: 0x%X", hccl_ret);
return FAILED;
}
// dlclose file handle
if (dlclose(handle_) != 0) {
GELOGW("Failed to close handle %s", dlerror());
}
GELOGI("Hcom executor finalize success.");
return SUCCESS;
}
} // namespace hybrid
} // namespace ge

+ 0
- 59
src/ge/hybrid/node_executor/hccl/hccl_node_executor.h View File

@@ -1,59 +0,0 @@
/**
* Copyright 2019-2020 Huawei Technologies Co., Ltd
*
* 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.
*/

#ifndef HYBRID_HCCL_NODE_EXECUTOR_H_
#define HYBRID_HCCL_NODE_EXECUTOR_H_
#include "hybrid/node_executor/node_executor.h"
#include "hybrid/model/hybrid_model.h"
#include "graph/op_desc.h"

namespace ge {
namespace hybrid {
class HybridModel;

class HcclNodeTask : public NodeTask {
public:
HcclNodeTask() {}

~HcclNodeTask() {}

Status UpdateArgs(TaskContext &context) override;
Status ExecuteAsync(TaskContext &context, std::function<void()> done_callback) override;
Status Init(TaskContext &context) override;

private:
std::shared_ptr<DavinciModel> davinci_model_ = nullptr;
bool load_flag_ = false;
std::mutex hccl_mutex_;
std::condition_variable cond_;
};

class HcclNodeExecutor : public NodeExecutor {
public:
Status LoadTask(const HybridModel &model, const NodePtr &node, shared_ptr<NodeTask> &task) const;
Status PrepareTask(NodeTask &task, TaskContext &context) const;
Status ExecuteTask(NodeTask &task, TaskContext &context, const std::function<void()> &callback) const;
Status Initialize() override;
Status Finalize() override;
~HcclNodeExecutor() {}

private:
void *handle_;
};
} // namespace hybrid
} // namespace ge

#endif // HYBRID_HCCL_NODE_EXECUTOR_H_

+ 4
- 27
src/ge/hybrid/node_executor/hostcpu/ge_local_node_executor.cc View File

@@ -17,12 +17,14 @@
#include "hybrid/node_executor/hostcpu/ge_local_node_executor.h"
#include "graph/debug/ge_attr_define.h"
#include "framework/common/util.h"
#include "hybrid/model/hybrid_model.h"
#include "framework/common/types.h"
#include "inc/kernel.h"
#include "inc/kernel_factory.h"
#include "common/ge/ge_util.h"

namespace ge {
namespace hybrid {

REGISTER_NODE_EXECUTOR_BUILDER(NodeExecutorManager::ExecutorType::GE_LOCAL, GeLocalNodeExecutor);

const std::unordered_map<std::string, std::vector<uint32_t>> RefInputTask::out_ref_input_index_ = {
@@ -130,7 +132,7 @@ Status DependInputShapeTask::Execute(TaskContext &context) {
}

// alloc output
GE_CHK_STATUS_RET_NOLOG(context.AllocateOutputs(NpuMemoryAllocator::AttrWithDefaultPadding()));
GE_CHK_STATUS_RET_NOLOG(context.AllocateOutputs());

// copy data to output
for (auto i = 0; i < output_num; ++i) {
@@ -192,16 +194,6 @@ Status GeLocalNodeExecutor::LoadTask(const HybridModel &model, const NodePtr &no
node_type.c_str());
return MEMALLOC_FAILED;
}
} else if (node_type == CONSTANTOP || node_type == VARIABLE) {
GELOGI("node %s type %s, use ConstantNodeTask.", node->GetName().c_str(), node_type.c_str());
auto tensor = model.GetVariable(node->GetName());
if (tensor == nullptr) {
GELOGE(INTERNAL_ERROR, "Failed to get tensor by name: %s", node->GetName().c_str());
return INTERNAL_ERROR;
}

task = MakeShared<ConstantNodeTask>(tensor);
GE_CHECK_NOTNULL(task);
} else {
GELOGE(UNSUPPORTED, "node %s type %s is not support in GeLocalNodeExecutor now.", node->GetName().c_str(),
node_type.c_str());
@@ -210,20 +202,5 @@ Status GeLocalNodeExecutor::LoadTask(const HybridModel &model, const NodePtr &no
return SUCCESS;
}

ConstantNodeTask::ConstantNodeTask(const TensorValue *tensor) : tensor_(tensor) {}

Status ConstantNodeTask::UpdateArgs(TaskContext &context) { return SUCCESS; }

Status ConstantNodeTask::ExecuteAsync(TaskContext &context, std::function<void()> done_callback) {
GELOGD("[%s] Start execute.", context.GetNodeName());
GE_CHK_STATUS_RET(context.SetOutput(0, *tensor_), "[%s] Failed to set output.", context.GetNodeName());
if (done_callback) {
GELOGD("[%s] Start invoke callback.", context.GetNodeName());
done_callback();
}

GELOGD("[%s] Done execute successfully.", context.GetNodeName());
return SUCCESS;
}
} // namespace hybrid
} // namespace ge

+ 1
- 12
src/ge/hybrid/node_executor/hostcpu/ge_local_node_executor.h View File

@@ -23,6 +23,7 @@

namespace ge {
namespace hybrid {

class RefInputTask : public NodeTask {
public:
explicit RefInputTask(const NodePtr &node) : node_name_(node->GetName()), node_type_(node->GetType()) {}
@@ -67,18 +68,6 @@ class DependInputShapeTask : public NodeTask {
static const std::unordered_set<std::string> depend_input_shape_ops_;
};

class ConstantNodeTask : public NodeTask {
public:
explicit ConstantNodeTask(const TensorValue *tensor);
~ConstantNodeTask() = default;
Status UpdateArgs(TaskContext &context) override;

Status ExecuteAsync(TaskContext &context, std::function<void()> done_callback) override;

private:
const TensorValue *tensor_;
};

class GeLocalNodeExecutor : public NodeExecutor {
public:
Status PrepareTask(NodeTask &task, TaskContext &context) const override;


+ 17
- 103
src/ge/hybrid/node_executor/node_executor.cc View File

@@ -16,7 +16,6 @@

#include "hybrid/node_executor/node_executor.h"
#include "framework/common/debug/log.h"
#include "graph/utils/node_utils.h"
#include "init/gelib.h"
#include "hybrid/model/hybrid_model.h"

@@ -26,11 +25,9 @@ namespace {
const char *const kEngineNameAiCore = "AIcoreEngine";
const char *const kEngineNameGeLocal = "DNN_VM_GE_LOCAL_OP_STORE";
const char *const kEngineNameAiCpu = "aicpu_kernel";
const char *const kEngineNameHccl = "ops_kernel_info_hccl";
} // namespace
Status NodeExecutor::PrepareTask(NodeTask &task, TaskContext &context) const {
GE_CHK_STATUS_RET_NOLOG(context.AllocateOutputs());
GE_CHK_STATUS_RET_NOLOG(task.UpdateTilingData(context)); // update op_desc before alloc ws
GE_CHK_STATUS_RET_NOLOG(context.AllocateWorkspaces());
GE_CHK_STATUS_RET_NOLOG(task.UpdateArgs(context));
return SUCCESS;
@@ -51,7 +48,6 @@ Status NodeExecutor::CompileTask(const HybridModel &model, const NodePtr &node,
}

Status NodeExecutorManager::EnsureInitialized() {
GE_CHK_STATUS_RET(InitializeExecutors());
std::lock_guard<std::mutex> lk(mu_);
if (initialized_) {
return SUCCESS;
@@ -60,7 +56,6 @@ Status NodeExecutorManager::EnsureInitialized() {
engine_mapping_.emplace(kEngineNameAiCore, NodeExecutorManager::ExecutorType::AICORE);
engine_mapping_.emplace(kEngineNameGeLocal, NodeExecutorManager::ExecutorType::GE_LOCAL);
engine_mapping_.emplace(kEngineNameAiCpu, NodeExecutorManager::ExecutorType::AICPU_TF);
engine_mapping_.emplace(kEngineNameHccl, NodeExecutorManager::ExecutorType::HCCL);

std::shared_ptr<GELib> instance_ptr = GELib::GetInstance();
if ((instance_ptr == nullptr) || (!instance_ptr->InitFlag())) {
@@ -74,6 +69,22 @@ Status NodeExecutorManager::EnsureInitialized() {
kernel_stores_.emplace(it.first, it.second);
}

GELOGI("Start to Initialize NodeExecutors");
for (auto &it : builders_) {
auto engine_type = it.first;
auto build_fn = it.second;
GE_CHECK_NOTNULL(build_fn);
auto executor = std::unique_ptr<NodeExecutor>(build_fn());
if (executor == nullptr) {
GELOGE(INTERNAL_ERROR, "Failed to create executor for engine type = %d", engine_type);
return INTERNAL_ERROR;
}

GELOGD("Executor of engine type = %d was created successfully", engine_type);
GE_CHK_STATUS_RET(executor->Initialize(), "Failed to initialize NodeExecutor of type = %d", engine_type);
executors_.emplace(engine_type, std::move(executor));
}

initialized_ = true;
GELOGI("Initializing NodeExecutors successfully");
return SUCCESS;
@@ -82,11 +93,6 @@ Status NodeExecutorManager::EnsureInitialized() {
NodeExecutorManager::ExecutorType NodeExecutorManager::ResolveExecutorType(Node &node) const {
auto op_type = node.GetType();
if (op_type == PARTITIONEDCALL) {
bool is_dynamic = false;
(void)NodeUtils::GetNodeUnknownShapeStatus(node, is_dynamic);
if (is_dynamic) {
return ExecutorType::DYNAMIC_SUBGRAPH;
}
return ExecutorType::COMPILED_SUBGRAPH;
}

@@ -95,10 +101,6 @@ NodeExecutorManager::ExecutorType NodeExecutorManager::ResolveExecutorType(Node
return ExecutorType::GE_LOCAL;
}

if (op_type == IF || op_type == CASE || op_type == WHILE) {
return ExecutorType::CONTROL_OP;
}

auto op_desc = node.GetOpDesc(); // checked before
const auto &lib_name = op_desc->GetOpKernelLibName();
auto it = engine_mapping_.find(lib_name);
@@ -114,11 +116,10 @@ Status NodeExecutorManager::GetExecutor(Node &node, const NodeExecutor **executo
auto executor_type = ResolveExecutorType(node);
const auto it = executors_.find(executor_type);
if (it == executors_.end()) {
GELOGE(INTERNAL_ERROR, "Failed to get executor by type: %d.", executor_type);
GELOGE(INTERNAL_ERROR, "Failed to get executor by type: %d", executor_type);
return INTERNAL_ERROR;
}

GELOGD("[%s] Set node executor by type: %d.", node.GetName().c_str(), executor_type);
*executor = it->second.get();
return SUCCESS;
}
@@ -131,11 +132,6 @@ void NodeExecutorManager::RegisterExecutorBuilder(NodeExecutorManager::ExecutorT
Status NodeExecutorManager::CalcOpRunningParam(Node &node) const {
auto op_desc = node.GetOpDesc();
GE_CHECK_NOTNULL(op_desc);
if (op_desc->GetType() == PARTITIONEDCALL) {
GELOGD("[%s] Skipping CalcOpRunningParam for PartitionedCall.", node.GetName().c_str());
return SUCCESS;
}

auto it = kernel_stores_.find(op_desc->GetOpKernelLibName());
if (it == kernel_stores_.end()) {
GELOGE(INTERNAL_ERROR, "Failed to get OpKernelStore. libName = %s, node = %s",
@@ -143,91 +139,9 @@ Status NodeExecutorManager::CalcOpRunningParam(Node &node) const {
return INTERNAL_ERROR;
}

// calc hccl output size independent, hccl ops kernel manager should GetSize for
// input which is the output size of input-op, but sometimes return error
// when multi-thread
if (op_desc->GetOpKernelLibName() == kEngineNameHccl) {
for (size_t i = 0; i < op_desc->GetOutputsSize(); ++i) {
GeTensorDesc output_tensor = op_desc->GetOutputDesc(static_cast<uint32_t>(i));
Format format = output_tensor.GetFormat();
DataType data_type = output_tensor.GetDataType();
GeShape output_shape = output_tensor.GetShape();
int64_t output_mem_size = 0;
GE_CHK_STATUS_RET(TensorUtils::CalcTensorMemSize(output_shape, format, data_type, output_mem_size),
"hccl calc tensor mem size failed.");
output_mem_size =
((output_mem_size + MEMORY_ALIGN_RATIO * MEMORY_ALIGN_SIZE - 1) / MEMORY_ALIGN_SIZE) * MEMORY_ALIGN_SIZE;
TensorUtils::SetSize(output_tensor, output_mem_size);
GE_CHK_STATUS_RET(op_desc->UpdateOutputDesc(static_cast<uint32_t>(i), output_tensor),
"hccl update output size failed.");
GELOGD("%s output desc[%u], dim_size: %zu, mem_size: %ld.", node.GetName().c_str(), i,
output_tensor.GetShape().GetDimNum(), output_mem_size);
}
return SUCCESS;
}
return it->second->CalcOpRunningParam(node);
}

Status NodeExecutorManager::InitializeExecutors() {
std::lock_guard<std::mutex> lk(mu_);
if (executor_initialized_) {
++ref_count_;
GELOGI("Executor is already initialized. add ref count to [%d]", ref_count_);
return SUCCESS;
}

GELOGI("Start to Initialize NodeExecutors");
for (auto &it : builders_) {
auto engine_type = it.first;
auto build_fn = it.second;
GE_CHECK_NOTNULL(build_fn);
auto executor = std::unique_ptr<NodeExecutor>(build_fn());
if (executor == nullptr) {
GELOGE(INTERNAL_ERROR, "Failed to create executor for engine type = %d", engine_type);
return INTERNAL_ERROR;
}

GELOGD("Executor of engine type = %d was created successfully", engine_type);
auto ret = executor->Initialize();
if (ret != SUCCESS) {
GELOGE(ret, "Failed to initialize NodeExecutor of type = %d, clear executors", engine_type);
for (auto &executor_it : executors_) {
executor_it.second->Finalize();
}
executors_.clear();
return ret;
}

executors_.emplace(engine_type, std::move(executor));
}

++ref_count_;
executor_initialized_ = true;
GELOGI("Initializing NodeExecutors successfully.");
return SUCCESS;
}

void NodeExecutorManager::FinalizeExecutors() {
std::lock_guard<std::mutex> lk(mu_);
if (!executor_initialized_) {
GELOGD("No need for finalizing for not initialized.");
return;
}

if (--ref_count_ > 0) {
GELOGD("Ref count = %d, do not finalize executors.", ref_count_);
return;
}

GELOGD("Start to invoke Finalize on executors.");
for (auto &it : executors_) {
it.second->Finalize();
}
executors_.clear();
executor_initialized_ = false;
GELOGD("Done invoking Finalize successfully.");
}

NodeExecutorRegistrar::NodeExecutorRegistrar(NodeExecutorManager::ExecutorType executor_type,
NodeExecutor *(*builder)()) {
NodeExecutorManager::GetInstance().RegisterExecutorBuilder(executor_type, builder);


+ 9
- 121
src/ge/hybrid/node_executor/node_executor.h View File

@@ -14,182 +14,70 @@
* limitations under the License.
*/

#ifndef GE_HYBRID_NODE_EXECUTOR_NODE_EXECUTOR_H_
#define GE_HYBRID_NODE_EXECUTOR_NODE_EXECUTOR_H_
#ifndef GE_HYBRID_KERNEL_NODE_EXECUTOR_H_
#define GE_HYBRID_KERNEL_NODE_EXECUTOR_H_

#include "external/ge/ge_api_error_codes.h"
#include "common/opskernel/ops_kernel_info_store.h"
#include "graph/node.h"
#include "proto/task.pb.h"
#include "task_context.h"

namespace ge {
const uint32_t MEMORY_ALIGN_RATIO = 2;
const uint32_t MEMORY_ALIGN_SIZE = 32;
namespace hybrid {
class HybridModel;
// Base class of Node Task
class NodeTask {
public:
NodeTask() = default;
virtual ~NodeTask() = default;

/**
* Update tiling data
* @param context instance of TaskContext
* @return SUCCESS on success, error code otherwise
*/
virtual Status UpdateTilingData(TaskContext &context) { return SUCCESS; }

/**
* Init
* @param context instance of TaskContext
* @return SUCCESS on success, error code otherwise
*/
virtual Status Init(TaskContext &context) { return SUCCESS; }

/**
* Whether this task supports dynamic shape
* @return true if this task supports dynamic shape, false otherwise
*/
virtual bool IsSupportDynamicShape() { return true; }

/**
* Update args for execution
* @param context instance of TaskContext
* @return SUCCESS on success, error code otherwise
*/
virtual Status UpdateArgs(TaskContext &context) = 0;

/**
* Execute task async
* @param context instance of TaskContext
* @param done_callback callback function, will be invoked after task is done
* @return SUCCESS on success, error code otherwise
*/
virtual Status ExecuteAsync(TaskContext &context, std::function<void()> done_callback) = 0;
virtual Status Init(TaskContext &context) { return SUCCESS; }
};

// Node executor
class NodeExecutor {
public:
NodeExecutor() = default;
virtual ~NodeExecutor() = default;

/**
* Initialize node executor
* @return SUCCESS on success, error code otherwise
*/
virtual Status Initialize() { return SUCCESS; }

/**
* Finalize node executor
* @return SUCCESS on success, error code otherwise
*/
virtual Status Finalize() { return SUCCESS; }

/**
* Load task in load stage
* @param model instance of HybridModel
* @param node node
* @param task generated node task
* @return SUCCESS on success, error code otherwise
*/
virtual Status LoadTask(const HybridModel &model, const NodePtr &node, std::shared_ptr<NodeTask> &task) const;

/**
* Compile task in run stage
* @param model instance of HybridModel
* @param node node
* @param task generated node task
* @return SUCCESS on success, error code otherwise
*/
virtual Status CompileTask(const HybridModel &model, const NodePtr &node, std::shared_ptr<NodeTask> &task) const;

/**
* Preparation actions before execution
* @param task instance of NodeTask
* @param context instance of TaskContext
* @return SUCCESS on success, error code otherwise
*/
virtual Status PrepareTask(NodeTask &task, TaskContext &context) const;

/**
* Execute task
* @param task instance of NodeTask
* @param context instance of TaskContext
* @param callback callback function which will be invoked after computation is done
* @return SUCCESS on success, error code otherwise
*/
virtual Status ExecuteTask(NodeTask &task, TaskContext &context, const std::function<void()> &callback) const;
};

class NodeExecutorManager {
public:
enum class ExecutorType {
AICORE,
AICPU_TF,
AICPU_CUSTOM,
COMPILED_SUBGRAPH,
DYNAMIC_SUBGRAPH,
GE_LOCAL,
CONTROL_OP,
HCCL,
RESERVED
};
enum class ExecutorType { AICORE, GE_LOCAL, AICPU_TF, AICPU_CUSTOM, COMPILED_SUBGRAPH, HCCL, RESERVED };

static NodeExecutorManager &GetInstance() {
static NodeExecutorManager instance;
return instance;
}

/**
* Register build of executor
* @param executor_type type of executor
* @param builder build function
*/
Status CalcOpRunningParam(Node &node) const;

void RegisterExecutorBuilder(ExecutorType executor_type, const std::function<NodeExecutor *()> &builder);

/**
* Initialize executor if needed
* @return SUCCESS on success, error code otherwise
*/
Status EnsureInitialized();

Status InitializeExecutors();

void FinalizeExecutors();

/**
* CalcOpRunningParam
* @param node node
* @return SUCCESS on success, error code otherwise
*/
Status CalcOpRunningParam(Node &node) const;

/**
* Get executor by node
* @param node node
* @param executor executor
* @return SUCCESS on success, error code otherwise
*/
Status GetExecutor(Node &node, const NodeExecutor **executor) const;

/**
* Resolve executor type by node
* @param node node
* @return executor type
*/
ExecutorType ResolveExecutorType(Node &node) const;

private:
std::map<ExecutorType, std::unique_ptr<NodeExecutor>> executors_;
std::map<ExecutorType, std::function<NodeExecutor *()>> builders_;
std::map<std::string, std::shared_ptr<OpsKernelInfoStore>> kernel_stores_;
std::map<std::string, NodeExecutorManager::ExecutorType> engine_mapping_;
std::mutex mu_;
bool initialized_ = false;
bool executor_initialized_ = false;
int ref_count_ = 0;
};

class NodeExecutorRegistrar {
@@ -211,4 +99,4 @@ class NodeExecutorRegistrar {
::ge::hybrid::NodeExecutorRegistrar( \
engine_type, []() -> ::ge::hybrid::NodeExecutor * { return new (std::nothrow) executor(); })

#endif // GE_HYBRID_NODE_EXECUTOR_NODE_EXECUTOR_H_
#endif // GE_HYBRID_KERNEL_NODE_EXECUTOR_H_

+ 0
- 81
src/ge/hybrid/node_executor/partitioned_call/partitioned_call_node_executor.cc View File

@@ -1,81 +0,0 @@
/**
* Copyright 2019-2020 Huawei Technologies Co., Ltd
*
* 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 "partitioned_call_node_executor.h"
#include "graph/utils/node_utils.h"

namespace ge {
namespace hybrid {
REGISTER_NODE_EXECUTOR_BUILDER(NodeExecutorManager::ExecutorType::DYNAMIC_SUBGRAPH, PartitionedCallNodeExecutor);

PartitionedCallNodeTask::PartitionedCallNodeTask(const GraphItem *graph_item) : graph_item_(graph_item) {}

PartitionedCallNodeTask::~PartitionedCallNodeTask() {
GELOGD("[%s] PartitionedCallNodeTask destroyed.", graph_item_->GetName().c_str());
}

Status PartitionedCallNodeTask::Init(TaskContext &context) {
auto execution_context = const_cast<GraphExecutionContext *>(context.GetExecutionContext());
subgraph_executor_.reset(new (std::nothrow) SubgraphExecutor(graph_item_, execution_context));
GE_CHECK_NOTNULL(subgraph_executor_);
return SUCCESS;
}

Status PartitionedCallNodeTask::ExecuteAsync(TaskContext &context, std::function<void()> done_callback) {
GE_CHK_STATUS_RET(subgraph_executor_->ExecuteAsync(context), "[%s] Failed to set inputs",
graph_item_->GetName().c_str());

auto callback = [=]() { Callback(done_callback); };

GE_CHK_STATUS_RET(context.RegisterCallback(callback), "[%s] Failed to register callback",
graph_item_->GetName().c_str());
GELOGD("[%s] Done executing subgraph successfully.", graph_item_->GetName().c_str());
return SUCCESS;
}

Status PartitionedCallNodeTask::Callback(const std::function<void()> &done_callback) {
GELOGD("[%s] On subgraph callback", graph_item_->GetName().c_str());
if (done_callback != nullptr) {
done_callback();
}

GELOGD("[%s] To release sub graph tensors.", graph_item_->GetName().c_str());
subgraph_executor_.reset();
GELOGD("[%s] Done releasing sub graph tensors.", graph_item_->GetName().c_str());
return SUCCESS;
}

Status PartitionedCallNodeTask::UpdateArgs(TaskContext &context) { return SUCCESS; }

Status PartitionedCallNodeExecutor::LoadTask(const ge::hybrid::HybridModel &model, const ge::NodePtr &node,
std::shared_ptr<NodeTask> &task) const {
GELOGD("Load dynamic partitioned call: [%s]", node->GetName().c_str());
auto subgraph = NodeUtils::GetSubgraph(*node, 0);
GE_CHECK_NOTNULL(subgraph);
auto partitioned_call = model.GetSubgraphItem(subgraph);
GE_CHECK_NOTNULL(partitioned_call);
task.reset(new (std::nothrow) PartitionedCallNodeTask(partitioned_call));
GE_CHECK_NOTNULL(task);
GELOGD("Done loading dynamic partitioned call: [%s]", node->GetName().c_str());
return SUCCESS;
}

Status PartitionedCallNodeExecutor::PrepareTask(NodeTask &task, TaskContext &context) const {
GE_CHK_STATUS_RET(task.Init(context), "[%s] Failed to init task.", context.GetNodeName());
return SUCCESS;
}
} // namespace hybrid
} // namespace ge

+ 0
- 54
src/ge/hybrid/node_executor/partitioned_call/partitioned_call_node_executor.h View File

@@ -1,54 +0,0 @@
/**
* Copyright 2019-2020 Huawei Technologies Co., Ltd
*
* 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.
*/

#ifndef GE_HYBRID_NODE_EXECUTOR_SUBGRAPH_SUBGRAPH_EXECUTOR_H_
#define GE_HYBRID_NODE_EXECUTOR_SUBGRAPH_SUBGRAPH_EXECUTOR_H_

#include "hybrid/node_executor/node_executor.h"
#include "hybrid/model/hybrid_model.h"
#include "hybrid/executor/node_state.h"
#include "hybrid/executor/subgraph_executor.h"
#include "common/thread_pool.h"

namespace ge {
namespace hybrid {
class PartitionedCallNodeTask : public NodeTask {
public:
explicit PartitionedCallNodeTask(const GraphItem *graph_item);
~PartitionedCallNodeTask() override;

Status Init(TaskContext &context) override;

Status UpdateArgs(TaskContext &context) override;

Status ExecuteAsync(TaskContext &context, std::function<void()> done_callback) override;

private:
Status Callback(const std::function<void()> &done_callback);

const GraphItem *graph_item_;
std::unique_ptr<SubgraphExecutor> subgraph_executor_;
GraphExecutionContext *context_ = nullptr;
};

class PartitionedCallNodeExecutor : public NodeExecutor {
public:
Status LoadTask(const HybridModel &model, const NodePtr &node, shared_ptr<NodeTask> &task) const override;
Status PrepareTask(NodeTask &task, TaskContext &context) const override;
};
} // namespace hybrid
} // namespace ge
#endif // GE_HYBRID_NODE_EXECUTOR_SUBGRAPH_SUBGRAPH_EXECUTOR_H_

+ 30
- 114
src/ge/hybrid/node_executor/task_context.cc View File

@@ -19,16 +19,12 @@
#include "framework/common/debug/log.h"
#include "graph/utils/tensor_utils.h"
#include "hybrid/executor/hybrid_execution_context.h"
#include "hybrid/executor/subgraph_executor.h"

namespace ge {
namespace hybrid {
TaskContext::TaskContext(GraphExecutionContext *execution_context, const NodeItem *node_item,
SubgraphContext *subgraph_context)
: node_item_(node_item), execution_context_(execution_context), subgraph_context_(subgraph_context) {}

TaskContext::TaskContext(GraphExecutionContext *execution_context) : execution_context_(execution_context) {}
TaskContext::~TaskContext() {
GELOGD("[%s] TaskContext destroyed.", node_item_->NodeName().c_str());
GELOGD("To execute ~TaskContext(). node = %s", node_item_->NodeName().c_str());
for (auto ws_addr : workspaces_) {
execution_context_->allocator->Deallocate(ws_addr);
}
@@ -42,28 +38,19 @@ TaskContext::~TaskContext() {
}
}

std::unique_ptr<TaskContext> TaskContext::Create(const NodeItem &node_item, GraphExecutionContext *execution_context,
SubgraphContext *subgraph_context) {
GELOGI("[%s] To create task context, input start = %d, num_inputs = %d, output start = %d, num_outputs = %d.",
std::unique_ptr<TaskContext> TaskContext::Create(const NodeItem &node_item, GraphExecutionContext *graph_context) {
GELOGI("To create task context for node %s, input start = %d, num_inputs = %d, output start = %d, num_outputs = %d",
node_item.NodeName().c_str(), node_item.input_start, node_item.num_inputs, node_item.output_start,
node_item.num_outputs);
if (node_item.input_start < 0 || node_item.output_start < 0) {
GELOGE(INTERNAL_ERROR, "NodeItem not property initialized. input_start = %d, output_start = %d",
node_item.input_start, node_item.output_start);
return nullptr;
}

auto task_context =
std::unique_ptr<TaskContext>(new (std::nothrow) TaskContext(execution_context, &node_item, subgraph_context));
auto task_context = std::unique_ptr<TaskContext>(new (std::nothrow) TaskContext(graph_context));
if (task_context == nullptr) {
GELOGE(MEMALLOC_FAILED, "[%s] Failed to create instance of TaskContext.", node_item.NodeName().c_str());
GELOGE(MEMALLOC_FAILED, "Failed to create instance of TaskContext. node = %s", node_item.NodeName().c_str());
return nullptr;
}

task_context->node_item_ = &node_item;
task_context->inputs_start_ = subgraph_context->all_inputs_.data() + node_item.input_start;
task_context->outputs_start_ = subgraph_context->all_outputs_.data() + node_item.output_start;
task_context->iteration_ = execution_context->iteration;
task_context->inputs_start_ = graph_context->all_inputs.data() + node_item.input_start;
task_context->outputs_start_ = graph_context->all_outputs.data() + node_item.output_start;
return task_context;
}

@@ -72,7 +59,7 @@ int TaskContext::NumInputs() const { return node_item_->num_inputs; }
int TaskContext::NumOutputs() const { return node_item_->num_outputs; }

TensorValue *TaskContext::MutableInput(int index) {
if (index < 0 || index >= node_item_->num_inputs) {
if (index < 0 || index > node_item_->num_inputs) {
GELOGE(PARAM_INVALID, "Index out of range. index = %d, num_inputs = %d", index, node_item_->num_inputs);
return nullptr;
}
@@ -81,7 +68,7 @@ TensorValue *TaskContext::MutableInput(int index) {
}

const TensorValue *TaskContext::GetOutput(int index) const {
if (index < 0 || index >= node_item_->num_outputs) {
if (index < 0 || index > node_item_->num_outputs) {
GELOGE(PARAM_INVALID, "Index out of range. index = %d, num_outputs = %d", index, node_item_->num_outputs);
return nullptr;
}
@@ -90,7 +77,7 @@ const TensorValue *TaskContext::GetOutput(int index) const {
}

TensorValue *TaskContext::MutableOutput(int index) {
if (index < 0 || index >= node_item_->num_outputs) {
if (index < 0 || index > node_item_->num_outputs) {
GELOGE(PARAM_INVALID, "Index out of range. index = %d, num_outputs = %d", index, node_item_->num_outputs);
return nullptr;
}
@@ -110,7 +97,7 @@ void *TaskContext::MutableWorkspace(int index) {
}

const TensorValue *TaskContext::GetInput(int index) const {
if (index < 0 || index >= node_item_->num_inputs) {
if (index < 0 || index > node_item_->num_inputs) {
GELOGE(PARAM_INVALID, "Index out of range. index = %d, num_inputs = %d", index, node_item_->num_inputs);
return nullptr;
}
@@ -133,14 +120,7 @@ Status TaskContext::AllocateWorkspaces() {
}

Status TaskContext::RegisterCallback(const std::function<void()> &callback_fun) const {
auto ret = execution_context_->callback_manager->RegisterCallback(callback_fun);
if (ret != SUCCESS) {
GELOGE(ret, "[%s] Failed to register callback", GetNodeName());
execution_context_->callback_manager->Destroy();
return ret;
}

return SUCCESS;
return execution_context_->callback_manager->RegisterCallback(callback_fun);
}

string TaskContext::TensorDesc2String(const GeTensorDesc &desc) {
@@ -157,7 +137,7 @@ string TaskContext::TensorDesc2String(const GeTensorDesc &desc) {
return ss.str();
}

Status TaskContext::AllocateTensor(const GeTensorDesc &tensor_desc, TensorValue &tensor, AllocationAttr *attr) {
Status TaskContext::AllocateTensor(const GeTensorDesc &tensor_desc, TensorValue &tensor) {
int64_t size = 0;
if (ge::TensorUtils::GetSize(tensor_desc, size) != GRAPH_SUCCESS) {
GELOGE(INTERNAL_ERROR, "Failed to get tensor size");
@@ -168,14 +148,13 @@ Status TaskContext::AllocateTensor(const GeTensorDesc &tensor_desc, TensorValue
GELOGW("size from tensor_desc == 0");
}

auto buffer = TensorBuffer::Create(execution_context_->allocator, size, attr);
auto buffer = TensorBuffer::Create(execution_context_->allocator, size);
GE_CHECK_NOTNULL(buffer);
tensor = TensorValue(shared_ptr<TensorBuffer>(buffer.release()));
return SUCCESS;
}

Status TaskContext::AllocateOutput(int index, const GeTensorDesc &tensor_desc, TensorValue **tensor,
AllocationAttr *attr) {
Status TaskContext::AllocateOutput(int index, const GeTensorDesc &tensor_desc, TensorValue **tensor) {
GELOGI("To allocate output for node: %s. index = %d, tensor desc = %s", node_item_->NodeName().c_str(), index,
TensorDesc2String(tensor_desc).c_str());

@@ -199,29 +178,9 @@ Status TaskContext::AllocateOutput(int index, const GeTensorDesc &tensor_desc, T
GE_CHECK_NOTNULL(ref_tensor);
outputs_start_[index] = *ref_tensor;
} else {
auto reuse_input = node_item_->reuse_inputs.find(index);
if (reuse_input != node_item_->reuse_inputs.end()) {
GELOGD("[%s] Output[%d] is referenced to input[%d]", GetNodeName(), index, reuse_input->second);
outputs_start_[index] = inputs_start_[reuse_input->second];
} else {
GE_CHK_STATUS_RET_NOLOG(AllocateTensor(tensor_desc, outputs_start_[index], attr));
GELOGD("Allocating output successfully. node: %s. index = %d, size = %zu", node_item_->NodeName().c_str(), index,
outputs_start_[index].GetSize());
}
}

// Temp modification
if (node_item_->node_type == "UnsortedSegmentSum" || node_item_->node_type == "UnsortedSegmentSumD" ||
node_item_->node_type == "ScatterNd") {
auto &out_tensor = outputs_start_[index];
GELOGD("[%s] clear output tensor: %s", GetNodeName(), out_tensor.DebugString().c_str());
auto *ctx = GetExecutionContext();
string name = "rtMemsetAsync" + node_item_->node_name;
RegisterCallback([ctx, name]() { RECORD_CALLBACK_EVENT(ctx, name.c_str(), "[Compute] Start"); });
RECORD_EXECUTION_EVENT(GetExecutionContext(), node_item_->node_name.c_str(), "[rtMemsetAsync] Start");
GE_CHK_RT_RET(rtMemsetAsync(out_tensor.MutableData(), out_tensor.GetSize(), 0, out_tensor.GetSize(), GetStream()));
RECORD_EXECUTION_EVENT(GetExecutionContext(), node_item_->node_name.c_str(), "[rtMemsetAsync] End");
RegisterCallback([ctx, name]() { RECORD_CALLBACK_EVENT(ctx, name.c_str(), "[Compute] End"); });
GE_CHK_STATUS_RET_NOLOG(AllocateTensor(tensor_desc, outputs_start_[index]));
GELOGD("Allocating output successfully. node: %s. index = %d, size = %zu", node_item_->NodeName().c_str(), index,
outputs_start_[index].GetSize());
}

if (execution_context_->trace_enabled) {
@@ -235,11 +194,11 @@ Status TaskContext::AllocateOutput(int index, const GeTensorDesc &tensor_desc, T
return SUCCESS;
}

Status TaskContext::AllocateOutputs(AllocationAttr *attr) {
Status TaskContext::AllocateOutputs() {
for (int i = 0; i < node_item_->num_outputs; ++i) {
const auto &output_desc = node_item_->op_desc->MutableOutputDesc(i);
GE_CHECK_NOTNULL(output_desc);
GE_CHK_STATUS_RET_NOLOG(AllocateOutput(i, *output_desc, nullptr, attr));
GE_CHK_STATUS_RET_NOLOG(AllocateOutput(i, *output_desc, nullptr));
}

return SUCCESS;
@@ -271,7 +230,7 @@ Status TaskContext::SetOutput(int index, const TensorValue &tensor) {

rtStream_t TaskContext::GetStream() { return execution_context_->stream; }

int64_t TaskContext::GetSessionId() const { return execution_context_->session_id; }
int64_t TaskContext::GetSessionId() { return execution_context_->session_id; }

Status TaskContext::GetStatus() const { return status_; }

@@ -279,13 +238,7 @@ void TaskContext::SetStatus(Status status) { status_ = status; }

Status TaskContext::AllocateWorkspace(size_t size, void **buffer, void *ori_addr) {
GE_CHECK_NOTNULL(buffer);
if (ori_addr == nullptr) {
*buffer = execution_context_->allocator->Allocate(size, nullptr);
} else {
AllocationAttr attr(ori_addr);
*buffer = execution_context_->allocator->Allocate(size, &attr);
}

*buffer = execution_context_->allocator->Allocate(size, ori_addr);
if (*buffer == nullptr) {
GELOGE(MEMALLOC_FAILED, "Failed to allocate workspace of size = %zu", size);
return MEMALLOC_FAILED;
@@ -308,21 +261,16 @@ Status TaskContext::PropagateOutputs() {
for (auto &dst_input_index_and_node : output_nodes) {
auto dst_input_idx = dst_input_index_and_node.first;
auto dst_node_item = dst_input_index_and_node.second;
auto input_offset = dst_node_item->input_start + dst_input_idx;
GELOGI(
"Propagate output of node %s, output index = %d, dst node = %s, "
"dst_input_index = %d, dst_input_offset = %d.",
node_item_->NodeName().c_str(), i, dst_node_item->NodeName().c_str(), dst_input_idx, input_offset);

if (subgraph_context_->all_inputs_.size() <= static_cast<size_t>(input_offset)) {
GELOGE(INTERNAL_ERROR, "[%s] input index out of range. index = %d, total input num = %zu", GetNodeName(),
input_offset, subgraph_context_->all_inputs_.size());
return INTERNAL_ERROR;
}

subgraph_context_->all_inputs_[input_offset] = *tensor;
"dst_input_index = %d, dst_input_offset = %d, addr = %p",
node_item_->NodeName().c_str(), i, dst_node_item->NodeName().c_str(), dst_input_idx,
dst_node_item->input_start + dst_input_idx,
execution_context_->all_inputs.data() + dst_node_item->input_start + dst_input_idx);
execution_context_->all_inputs[dst_node_item->input_start + dst_input_idx] = *tensor;
if (execution_context_->trace_enabled) {
subgraph_context_->all_inputs_[input_offset].SetName(node_item_->NodeName() + "_in_" + std::to_string(i));
execution_context_->all_inputs[dst_node_item->input_start + dst_input_idx].SetName(node_item_->NodeName() +
"_in_" + std::to_string(i));
}
}
}
@@ -341,37 +289,5 @@ void TaskContext::ReleaseInput(int index) {
GELOGD("[%s] Tensor of input[%d] released", GetNodeName(), index);
}
}

ConstGeTensorDescPtr TaskContext::GetOutputDesc(int index) {
return node_item_->op_desc->MutableOutputDesc(static_cast<uint32_t>(index));
}

ConstGeTensorDescPtr TaskContext::GetInputDesc(int index) {
return node_item_->op_desc->MutableInputDesc(static_cast<uint32_t>(index));
}

GeTensorDescPtr TaskContext::MutableInputDesc(int index) {
return node_item_->op_desc->MutableInputDesc(static_cast<uint32_t>(index));
}

GeTensorDescPtr TaskContext::MutableOutputDesc(int index) {
return node_item_->op_desc->MutableOutputDesc(static_cast<uint32_t>(index));
}

bool TaskContext::IsForceInferShape() const { return force_infer_shape_; }

void TaskContext::SetForceInferShape(bool force_infer_shape) { force_infer_shape_ = force_infer_shape; }

void TaskContext::NodeDone() { subgraph_context_->NodeDone(node_item_->node); }

void TaskContext::OnError(Status error) { subgraph_context_->OnError(error); }

bool TaskContext::IsTraceEnabled() const { return execution_context_->trace_enabled; }

TensorValue *TaskContext::GetVariable(const std::string &name) { return execution_context_->model->GetVariable(name); }

uint64_t TaskContext::GetIterationNumber() const { return iteration_; }

bool TaskContext::IsDumpEnabled() const { return execution_context_->dump_enabled; }
} // namespace hybrid
} // namespace ge

+ 9
- 34
src/ge/hybrid/node_executor/task_context.h View File

@@ -22,19 +22,16 @@
#include <vector>
#include "external/ge/ge_api_error_codes.h"
#include "hybrid/common/tensor_value.h"
#include "hybrid/common/npu_memory_allocator.h"
#include "hybrid/executor/rt_callback_manager.h"
#include "hybrid/model/node_item.h"

namespace ge {
namespace hybrid {
class GraphExecutionContext;
class SubgraphContext;

class TaskContext {
public:
static std::unique_ptr<TaskContext> Create(const NodeItem &node_item, GraphExecutionContext *execution_context,
SubgraphContext *subgraph_context);
static std::unique_ptr<TaskContext> Create(const NodeItem &node_item, GraphExecutionContext *graph_context);

~TaskContext();

@@ -44,33 +41,19 @@ class TaskContext {
const NodeItem &GetNodeItem() const;
const char *GetNodeName() const;
TensorValue *MutableInput(int index);
ConstGeTensorDescPtr GetInputDesc(int index);
ConstGeTensorDescPtr GetOutputDesc(int index);
GeTensorDescPtr MutableInputDesc(int index);
GeTensorDescPtr MutableOutputDesc(int index);
void ReleaseInput(int index);
const TensorValue *GetInput(int index) const;
const TensorValue *GetOutput(int index) const;
TensorValue *MutableOutput(int index);
TensorValue *GetVariable(const std::string &name);
rtStream_t GetStream();
int64_t GetSessionId() const;
uint64_t GetIterationNumber() const;

void NodeDone();
void OnError(Status error);
int64_t GetSessionId();

Status SetOutput(int index, const TensorValue &tensor);
Status AllocateOutput(int index, const GeTensorDesc &tensor_desc, TensorValue **tensor,
AllocationAttr *attr = nullptr);
Status AllocateOutputs(AllocationAttr *attr = nullptr);
Status AllocateOutput(int index, const GeTensorDesc &tensor_desc, TensorValue **tensor);
Status AllocateOutputs();
Status AllocateWorkspaces();
Status AllocateWorkspace(size_t size, void **buffer, void *ori_addr = nullptr);

bool IsTraceEnabled() const;

bool IsDumpEnabled() const;

const GraphExecutionContext *GetExecutionContext() { return execution_context_; }

Status AllocateTemp(size_t size, TensorValue &tensor);
@@ -85,25 +68,17 @@ class TaskContext {

void SetStatus(Status status);

bool IsForceInferShape() const;
void SetForceInferShape(bool force_infer_shape);
void *handle_ = nullptr;

private:
TaskContext(GraphExecutionContext *execution_context, const NodeItem *node_item, SubgraphContext *subgraph_context);

explicit TaskContext(GraphExecutionContext *execution_context);
TensorValue *inputs_start_ = nullptr;
TensorValue *outputs_start_ = nullptr;
static string TensorDesc2String(const GeTensorDesc &desc);
Status AllocateTensor(const GeTensorDesc &tensor_desc, TensorValue &tensor, AllocationAttr *attr);
Status AllocateTensor(const GeTensorDesc &tensor_desc, TensorValue &tensor);

const NodeItem *node_item_ = nullptr;
bool force_infer_shape_ = false;
GraphExecutionContext *execution_context_;
SubgraphContext *subgraph_context_;
TensorValue *inputs_start_ = nullptr;
TensorValue *outputs_start_ = nullptr;
const NodeItem *node_item_ = nullptr;
Status status_ = SUCCESS;
std::vector<void *> workspaces_;
uint64_t iteration_ = 0;
};
} // namespace hybrid
} // namespace ge


Loading…
Cancel
Save