@@ -29,7 +29,7 @@ bool IsNoOp(const NodeItem &node_item) { | |||||
const auto &tensor_desc = node_item.MutableOutputDesc(i); | const auto &tensor_desc = node_item.MutableOutputDesc(i); | ||||
GE_CHECK_NOTNULL(tensor_desc); | GE_CHECK_NOTNULL(tensor_desc); | ||||
const auto &shape = tensor_desc->MutableShape(); | const auto &shape = tensor_desc->MutableShape(); | ||||
if (shape.IsScalar() || shape.GetShapeSize() > 0) { | |||||
if (shape.IsScalar() || shape.GetShapeSize() > 0 || (node_item.shape_inference_type == DEPEND_SHAPE_RANGE)) { | |||||
return false; | return false; | ||||
} | } | ||||
} | } | ||||
@@ -219,12 +219,28 @@ Status AiCoreNodeTask::ExecuteAsync(TaskContext &context, std::function<void()> | |||||
RECORD_EXECUTION_EVENT(context.GetExecutionContext(), context.GetNodeName(), "[AiCoreNodeLaunchKernel] End"); | RECORD_EXECUTION_EVENT(context.GetExecutionContext(), context.GetNodeName(), "[AiCoreNodeLaunchKernel] End"); | ||||
} | } | ||||
if (done_callback != nullptr) { | |||||
auto callback = done_callback; | |||||
if (!tasks_.empty()) { | |||||
// only last task need update outputs shape | |||||
auto task = tasks_.back().get(); | |||||
if (task->GetUnknownShapeOpType() == DEPEND_SHAPE_RANGE) { | |||||
callback = [=, &context]() { | |||||
Status callback_ret = SUCCESS; | |||||
GELOGD("Node[%s] need update outputs shape.", context.GetNodeName()); | |||||
callback_ret = task->UpdateOutputsShape(context); | |||||
if (done_callback != nullptr) { | |||||
context.SetStatus(callback_ret); | |||||
done_callback(); | |||||
} | |||||
}; | |||||
} | |||||
} | |||||
if (callback != nullptr) { | |||||
RECORD_EXECUTION_EVENT(context.GetExecutionContext(), context.GetNodeName(), "[AiCoreNodeRegisterCallback] Start"); | RECORD_EXECUTION_EVENT(context.GetExecutionContext(), context.GetNodeName(), "[AiCoreNodeRegisterCallback] Start"); | ||||
GE_CHK_STATUS_RET_NOLOG(context.RegisterCallback(done_callback)); | |||||
GE_CHK_STATUS_RET_NOLOG(context.RegisterCallback(callback)); | |||||
RECORD_EXECUTION_EVENT(context.GetExecutionContext(), context.GetNodeName(), "[AiCoreNodeRegisterCallback] End"); | RECORD_EXECUTION_EVENT(context.GetExecutionContext(), context.GetNodeName(), "[AiCoreNodeRegisterCallback] End"); | ||||
} | } | ||||
GELOGD("[%s] ExecuteAsync End.", context.GetNodeName()); | GELOGD("[%s] ExecuteAsync End.", context.GetNodeName()); | ||||
RECORD_EXECUTION_EVENT(context.GetExecutionContext(), context.GetNodeName(), "[AiCoreNodeTaskExecuteAsync] End"); | RECORD_EXECUTION_EVENT(context.GetExecutionContext(), context.GetNodeName(), "[AiCoreNodeTaskExecuteAsync] End"); | ||||
return SUCCESS; | return SUCCESS; | ||||
@@ -15,13 +15,15 @@ | |||||
*/ | */ | ||||
#include "hybrid/node_executor/aicore/aicore_op_task.h" | #include "hybrid/node_executor/aicore/aicore_op_task.h" | ||||
#include "framework/common/taskdown_common.h" | |||||
#include "common/formats/formats.h" | |||||
#include "external/graph/types.h" | |||||
#include "framework/common/debug/log.h" | #include "framework/common/debug/log.h" | ||||
#include "framework/common/taskdown_common.h" | |||||
#include "graph/ge_context.h" | #include "graph/ge_context.h" | ||||
#include "graph/load/model_manager/tbe_handle_store.h" | |||||
#include "hybrid/executor/hybrid_execution_context.h" | #include "hybrid/executor/hybrid_execution_context.h" | ||||
#include "hybrid/node_executor/aicore/aicore_task_builder.h" | #include "hybrid/node_executor/aicore/aicore_task_builder.h" | ||||
#include "graph/load/model_manager/tbe_handle_store.h" | |||||
#include "external/graph/types.h" | |||||
#include "single_op/task/build_task_utils.h" | #include "single_op/task/build_task_utils.h" | ||||
#include "single_op/task/tbe_task_builder.h" | #include "single_op/task/tbe_task_builder.h" | ||||
@@ -35,6 +37,9 @@ constexpr char const *kAttrOpParamSize = "op_para_size"; | |||||
constexpr char const *kAttrAtomicOpParamSize = "atomic_op_para_size"; | constexpr char const *kAttrAtomicOpParamSize = "atomic_op_para_size"; | ||||
const string kAtomicOpType = "DynamicAtomicAddrClean"; | const string kAtomicOpType = "DynamicAtomicAddrClean"; | ||||
std::atomic<std::uint64_t> log_id(0); | std::atomic<std::uint64_t> log_id(0); | ||||
const uint32_t kMaxDimNum = 8; | |||||
// size,dim1,...,dim8: 9*4=36 | |||||
const size_t kShapeBufferSize = sizeof(uint32_t) * (1 + kMaxDimNum); | |||||
} // namespace | } // namespace | ||||
TbeHandleHolder::TbeHandleHolder(void *bin_handle) | TbeHandleHolder::TbeHandleHolder(void *bin_handle) | ||||
@@ -52,6 +57,30 @@ bool TbeHandleRegistry::AddHandle(std::unique_ptr<TbeHandleHolder> &&holder) { | |||||
} | } | ||||
Status AiCoreOpTask::Init(const OpDesc &op_desc, const domi::TaskDef &task_def) { | Status AiCoreOpTask::Init(const OpDesc &op_desc, const domi::TaskDef &task_def) { | ||||
GE_CHK_STATUS_RET_NOLOG(DoInit(op_desc, task_def)); | |||||
int32_t unknown_shape_op_type_val = static_cast<int32_t>(DEPEND_IN_SHAPE); | |||||
(void)AttrUtils::GetInt(op_desc, ::ge::ATTR_NAME_UNKNOWN_SHAPE_TYPE, unknown_shape_op_type_val); | |||||
unknown_shape_op_type_ = static_cast<UnknowShapeOpType>(unknown_shape_op_type_val); | |||||
GELOGD("Op [%s] unknown shape type is %d", op_desc.GetName().c_str(), unknown_shape_op_type_); | |||||
if (unknown_shape_op_type_ == DEPEND_SHAPE_RANGE) { | |||||
size_t size = kShapeBufferSize * op_desc.GetOutputsSize(); | |||||
if (size == 0) { | |||||
GELOGE(PARAM_INVALID, "Op [%s] unknown shape type is %d, but outputs size is 0.", op_desc.GetName().c_str(), | |||||
unknown_shape_op_type_); | |||||
return PARAM_INVALID; | |||||
} | |||||
auto allocator = NpuMemoryAllocator::GetAllocator(); | |||||
GE_CHECK_NOTNULL(allocator); | |||||
shape_buffer_ = TensorBuffer::Create(allocator, size); | |||||
GE_CHECK_NOTNULL(shape_buffer_); | |||||
GELOGD("Op [%s] allocate memory for outputs shape success, size=%zu", op_desc.GetName().c_str(), size); | |||||
host_shape_buffer_.reset(new (std::nothrow) uint8_t[shape_buffer_->GetSize()]); | |||||
GE_CHECK_NOTNULL(host_shape_buffer_); | |||||
} | |||||
return SUCCESS; | |||||
} | |||||
Status AiCoreOpTask::DoInit(const OpDesc &op_desc, const domi::TaskDef &task_def) { | |||||
op_type_ = op_desc.GetType(); | op_type_ = op_desc.GetType(); | ||||
log_name_ = op_desc.GetName() + "_tvmbin"; | log_name_ = op_desc.GetName() + "_tvmbin"; | ||||
log_id_ = log_id++; | log_id_ = log_id++; | ||||
@@ -81,6 +110,74 @@ Status AiCoreOpTask::Init(const OpDesc &op_desc, const domi::TaskDef &task_def) | |||||
return SUCCESS; | return SUCCESS; | ||||
} | } | ||||
Status AiCoreOpTask::UpdateOutputsShape(TaskContext &context) const { | |||||
GELOGD("Node[%s] start update outputs shape.", context.GetNodeName()); | |||||
GE_CHECK_NOTNULL(shape_buffer_); | |||||
GE_CHECK_NOTNULL(host_shape_buffer_); | |||||
GE_CHK_RT_RET(rtMemcpy(host_shape_buffer_.get(), shape_buffer_->GetSize(), shape_buffer_->GetData(), | |||||
shape_buffer_->GetSize(), RT_MEMCPY_DEVICE_TO_HOST)); | |||||
int num_outputs = context.NumOutputs(); | |||||
auto outputs_shape = reinterpret_cast<uint32_t(*)[num_outputs]>(host_shape_buffer_.get()); | |||||
for (int i = 0; i < num_outputs; ++i) { | |||||
if (outputs_shape[i][0] != 0) { | |||||
uint32_t dim_num = outputs_shape[i][0]; | |||||
GE_CHECK_LE(dim_num, kMaxDimNum); | |||||
vector<int64_t> dims; | |||||
for (uint32_t j = 1; j <= dim_num; ++j) { | |||||
dims.emplace_back(static_cast<int64_t>(outputs_shape[i][j])); | |||||
} | |||||
auto shape_new = GeShape(dims); | |||||
GELOGD("Node[%s] output[%d] shape:%s.", context.GetNodeName(), i, ToString(dims).c_str()); | |||||
GE_CHK_STATUS_RET_NOLOG(UpdateShapeToOutputDesc(context, shape_new, i)); | |||||
} | |||||
} | |||||
return SUCCESS; | |||||
} | |||||
Status AiCoreOpTask::UpdateShapeToOutputDesc(TaskContext &context, const GeShape &shape, const int output_index) const { | |||||
auto output_desc = context.MutableOutputDesc(output_index); | |||||
GE_CHECK_NOTNULL(output_desc); | |||||
auto shape_old = output_desc->GetShape(); | |||||
auto origin_shape_old = output_desc->GetOriginShape(); | |||||
auto origin_format = output_desc->GetOriginFormat(); | |||||
auto format = output_desc->GetFormat(); | |||||
auto node_state = context.GetNodeState(); | |||||
GE_CHECK_NOTNULL(node_state); | |||||
if (origin_format == format) { | |||||
GELOGD( | |||||
"Node[%s] try to update output[%d] shape from [%s] to [%s], origin_shape " | |||||
"from [%s] to [%s].", | |||||
context.GetNodeName(), output_index, shape_old.ToString().c_str(), shape.ToString().c_str(), | |||||
origin_shape_old.ToString().c_str(), shape.ToString().c_str()); | |||||
GE_CHK_STATUS_RET(node_state->UpdateOutputShapes(output_index, shape, shape), | |||||
"Node[%s] try to update output[%d] shape from [%s] to [%s], origin_shape " | |||||
"from [%s] to [%s] failed.", | |||||
context.GetNodeName(), output_index, shape_old.ToString().c_str(), shape.ToString().c_str(), | |||||
origin_shape_old.ToString().c_str(), shape.ToString().c_str()); | |||||
return SUCCESS; | |||||
} | |||||
// if format is not same need convert shape | |||||
std::vector<int64_t> origin_dims_new; | |||||
auto trans_ret = | |||||
formats::TransShape(format, shape.GetDims(), output_desc->GetDataType(), origin_format, origin_dims_new); | |||||
GE_CHK_STATUS_RET(trans_ret, | |||||
"[Trans][Shape] failed for node[%s] output[%d], origin_format[%d] " | |||||
"is not same as format[%d], shape=[%s].", | |||||
context.GetNodeName(), output_index, origin_format, format, shape.ToString().c_str()); | |||||
auto origin_shape_new = GeShape(origin_dims_new); | |||||
GE_CHK_STATUS_RET(node_state->UpdateOutputShapes(output_index, shape, origin_shape_new), | |||||
"Node[%s] try to update output[%d] shape from [%s] to [%s], origin_shape " | |||||
"from [%s] to [%s] failed.", | |||||
context.GetNodeName(), output_index, shape_old.ToString().c_str(), shape.ToString().c_str(), | |||||
origin_shape_old.ToString().c_str(), origin_shape_new.ToString().c_str()); | |||||
GELOGD( | |||||
"Node[%s] update output[%d] shape from [%s] to [%s], origin_shape " | |||||
"from [%s] to [%s].", | |||||
context.GetNodeName(), output_index, shape_old.ToString().c_str(), shape.ToString().c_str(), | |||||
origin_shape_old.ToString().c_str(), origin_shape_new.ToString().c_str()); | |||||
return SUCCESS; | |||||
} | |||||
Status AiCoreOpTask::RegisterTbeHandle(const OpDesc &op_desc) { | Status AiCoreOpTask::RegisterTbeHandle(const OpDesc &op_desc) { | ||||
rtError_t rt_ret = rtQueryFunctionRegistered(stub_name_.c_str()); | rtError_t rt_ret = rtQueryFunctionRegistered(stub_name_.c_str()); | ||||
if (rt_ret != RT_ERROR_NONE) { | if (rt_ret != RT_ERROR_NONE) { | ||||
@@ -429,6 +526,11 @@ Status AiCoreOpTask::UpdateArgs(TaskContext &task_context) { | |||||
if (tiling_buffer_ != nullptr) { | if (tiling_buffer_ != nullptr) { | ||||
++expected_arg_count; | ++expected_arg_count; | ||||
} | } | ||||
if (shape_buffer_ != nullptr) { | |||||
++expected_arg_count; | |||||
} | |||||
if (expected_arg_count > max_arg_count_) { | if (expected_arg_count > max_arg_count_) { | ||||
GELOGD("Need to reset size of args_ from %u to %zu.", max_arg_count_, expected_arg_count); | GELOGD("Need to reset size of args_ from %u to %zu.", max_arg_count_, expected_arg_count); | ||||
auto length = expected_arg_count * sizeof(uintptr_t) + offset_; | auto length = expected_arg_count * sizeof(uintptr_t) + offset_; | ||||
@@ -465,6 +567,12 @@ Status AiCoreOpTask::UpdateArgs(TaskContext &task_context) { | |||||
arg_base_[index++] = reinterpret_cast<uintptr_t>(output->GetData()); | arg_base_[index++] = reinterpret_cast<uintptr_t>(output->GetData()); | ||||
} | } | ||||
if (shape_buffer_ != nullptr) { | |||||
GE_CHK_RT_RET(rtMemset(shape_buffer_->GetData(), shape_buffer_->GetSize(), 0, shape_buffer_->GetSize())); | |||||
arg_base_[index++] = reinterpret_cast<uintptr_t>(shape_buffer_->GetData()); | |||||
GELOGD("Node:%s add shape buffer addr to args.", task_context.GetNodeName()); | |||||
} | |||||
int workspace_num = static_cast<int>(task_context.NumWorkspaces()); | int workspace_num = static_cast<int>(task_context.NumWorkspaces()); | ||||
for (int i = 0; i < workspace_num; ++i) { | for (int i = 0; i < workspace_num; ++i) { | ||||
const auto workspace = task_context.MutableWorkspace(i); | const auto workspace = task_context.MutableWorkspace(i); | ||||
@@ -567,7 +675,7 @@ std::string AiCoreOpTask::GetKeyForKernelName(const OpDesc &op_desc) const { | |||||
} | } | ||||
Status AtomicAddrCleanOpTask::Init(const OpDesc &op_desc, const domi::TaskDef &task_def) { | Status AtomicAddrCleanOpTask::Init(const OpDesc &op_desc, const domi::TaskDef &task_def) { | ||||
GE_CHK_STATUS_RET_NOLOG(AiCoreOpTask::Init(op_desc, task_def)); | |||||
GE_CHK_STATUS_RET_NOLOG(AiCoreOpTask::DoInit(op_desc, task_def)); | |||||
return InitAtomicAddrCleanIndices(op_desc); | return InitAtomicAddrCleanIndices(op_desc); | ||||
} | } | ||||
@@ -82,6 +82,12 @@ class AiCoreOpTask { | |||||
virtual const std::string& GetOpType() const; | virtual const std::string& GetOpType() const; | ||||
const UnknowShapeOpType GetUnknownShapeOpType() const { | |||||
return unknown_shape_op_type_; | |||||
} | |||||
Status UpdateOutputsShape(TaskContext &context) const; | |||||
protected: | protected: | ||||
Status UpdateTilingInfo(TaskContext &context); | Status UpdateTilingInfo(TaskContext &context); | ||||
virtual std::string GetKeyForOpParamSize() const; | virtual std::string GetKeyForOpParamSize() const; | ||||
@@ -90,6 +96,7 @@ class AiCoreOpTask { | |||||
virtual std::string GetKeyForTvmMetaData() const; | virtual std::string GetKeyForTvmMetaData() const; | ||||
virtual std::string GetKeyForKernelName(const OpDesc &op_desc) const; | virtual std::string GetKeyForKernelName(const OpDesc &op_desc) const; | ||||
virtual Status CalcTilingInfo(const NodePtr &node, optiling::utils::OpRunInfo &tiling_info); | virtual Status CalcTilingInfo(const NodePtr &node, optiling::utils::OpRunInfo &tiling_info); | ||||
Status DoInit(const OpDesc &op_desc, const domi::TaskDef &task_def); | |||||
std::unique_ptr<TensorBuffer> tiling_buffer_ = nullptr; | std::unique_ptr<TensorBuffer> tiling_buffer_ = nullptr; | ||||
std::string tiling_data_; | std::string tiling_data_; | ||||
@@ -104,6 +111,7 @@ class AiCoreOpTask { | |||||
Status RegisterKernelHandle(const OpDesc &op_desc); | Status RegisterKernelHandle(const OpDesc &op_desc); | ||||
Status InitWithKernelDef(const OpDesc &op_desc, const domi::TaskDef &task_def); | Status InitWithKernelDef(const OpDesc &op_desc, const domi::TaskDef &task_def); | ||||
Status InitWithKernelDefWithHandle(const OpDesc &node, const domi::TaskDef &task_def); | Status InitWithKernelDefWithHandle(const OpDesc &node, const domi::TaskDef &task_def); | ||||
Status UpdateShapeToOutputDesc(TaskContext &context, const GeShape &shape, const int output_index) const; | |||||
std::string stub_name_; | std::string stub_name_; | ||||
void *stub_func_ = nullptr; | void *stub_func_ = nullptr; | ||||
@@ -122,6 +130,9 @@ class AiCoreOpTask { | |||||
std::string log_name_; | std::string log_name_; | ||||
uint32_t offset_ = 0; | uint32_t offset_ = 0; | ||||
std::string op_type_; | std::string op_type_; | ||||
UnknowShapeOpType unknown_shape_op_type_ = DEPEND_IN_SHAPE; | |||||
std::unique_ptr<TensorBuffer> shape_buffer_ = nullptr; | |||||
std::unique_ptr<uint8_t[]> host_shape_buffer_ = nullptr; | |||||
}; | }; | ||||
class AtomicAddrCleanOpTask : public AiCoreOpTask { | class AtomicAddrCleanOpTask : public AiCoreOpTask { | ||||
@@ -735,6 +735,8 @@ set(HYBRID_TEST_FILES | |||||
"hybrid/executor/hybrid_model_async_executor_unittest.cc" | "hybrid/executor/hybrid_model_async_executor_unittest.cc" | ||||
"hybrid/executor/hybrid_model_pipeline_executor_unittest.cc" | "hybrid/executor/hybrid_model_pipeline_executor_unittest.cc" | ||||
"hybrid/node_executor/aicore/aicore_task_compiler_unittest.cc" | "hybrid/node_executor/aicore/aicore_task_compiler_unittest.cc" | ||||
"hybrid/node_executor/aicore/aicore_op_task_unittest.cc" | |||||
"hybrid/node_executor/aicore/aicore_node_executor_unittest.cc" | |||||
) | ) | ||||
set(OTHERS_TEST_FILES | set(OTHERS_TEST_FILES | ||||
@@ -821,6 +821,8 @@ TEST_F(UtestGeHybrid, TestTaskExecuteAsync) { | |||||
node_item->output_start = 0; | node_item->output_start = 0; | ||||
GraphExecutionContext execution_context; | GraphExecutionContext execution_context; | ||||
execution_context.callback_manager = | |||||
std::unique_ptr<CallbackManager>(new (std::nothrow) CallbackManager()); | |||||
GraphItem graph_item; | GraphItem graph_item; | ||||
SubgraphContext subgraph_context(&graph_item, &execution_context); | SubgraphContext subgraph_context(&graph_item, &execution_context); | ||||
ASSERT_EQ(subgraph_context.Init(), SUCCESS); | ASSERT_EQ(subgraph_context.Init(), SUCCESS); | ||||
@@ -0,0 +1,144 @@ | |||||
/** | |||||
* Copyright 2021-2021 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 <gmock/gmock.h> | |||||
#include <gtest/gtest.h> | |||||
#include <vector> | |||||
#define private public | |||||
#define protected public | |||||
#include "framework/common/taskdown_common.h" | |||||
#include "hybrid/executor/rt_callback_manager.h" | |||||
#include "hybrid/executor/subgraph_context.h" | |||||
#include "hybrid/node_executor/aicore/aicore_node_executor.h" | |||||
#include "init/gelib.h" | |||||
#undef private | |||||
#undef protected | |||||
using namespace std; | |||||
using namespace testing; | |||||
namespace ge { | |||||
using namespace hybrid; | |||||
class UtestAiCoreNodeExecutor : public testing::Test { | |||||
protected: | |||||
void SetUp() {} | |||||
void TearDown() {} | |||||
}; | |||||
static ge::OpDescPtr CreateOpDesc(string name = "", string type = "", | |||||
int in_num = 0, int out_num = 0) { | |||||
auto op_desc = std::make_shared<ge::OpDesc>(name, type); | |||||
op_desc->SetStreamId(0); | |||||
static int32_t index = 0; | |||||
op_desc->SetId(index++); | |||||
GeTensorDesc tensor(GeShape(), FORMAT_ND, DT_INT64); | |||||
TensorUtils::SetSize(tensor, 64); | |||||
vector<int64_t> input_offset; | |||||
for (int i = 0; i < in_num; ++i) { | |||||
op_desc->AddInputDesc(tensor); | |||||
input_offset.emplace_back(index * 64 + i * 64); | |||||
} | |||||
op_desc->SetInputOffset(input_offset); | |||||
vector<int64_t> output_offset; | |||||
for (int i = 0; i < out_num; ++i) { | |||||
op_desc->AddOutputDesc(tensor); | |||||
output_offset.emplace_back(index * 64 + in_num * 64 + i * 64); | |||||
} | |||||
op_desc->SetOutputOffset(output_offset); | |||||
op_desc->SetWorkspace({}); | |||||
op_desc->SetWorkspaceBytes({}); | |||||
ge::AttrUtils::SetStr(op_desc, ge::TVM_ATTR_NAME_MAGIC, | |||||
"RT_DEV_BINARY_MAGIC_ELF_AIVEC"); | |||||
bool support_dynamic = true; | |||||
ge::AttrUtils::GetBool(op_desc, "support_dynamicshape", support_dynamic); | |||||
return op_desc; | |||||
} | |||||
TEST_F(UtestAiCoreNodeExecutor, callback_success) { | |||||
dlog_setlevel(0, 0, 0); | |||||
std::unique_ptr<AiCoreOpTask> task1(new AiCoreOpTask()); | |||||
OpDescPtr op_desc = CreateOpDesc("Add", "Add", 2, 1); | |||||
ge::AttrUtils::SetInt(*op_desc, ge::ATTR_NAME_UNKNOWN_SHAPE_TYPE, | |||||
DEPEND_SHAPE_RANGE); | |||||
domi::TaskDef task_def; | |||||
task_def.set_type(RT_MODEL_TASK_KERNEL); | |||||
std::vector<uint8_t> args(100, 0); | |||||
task_def.mutable_kernel()->set_args(args.data(), args.size()); | |||||
task_def.mutable_kernel()->set_args_size(100); | |||||
task_def.mutable_kernel()->mutable_context()->set_kernel_type( | |||||
ccKernelType::TE); | |||||
uint16_t args_offset = 20; | |||||
char *a = reinterpret_cast<char *>(&args_offset); | |||||
task_def.mutable_kernel()->mutable_context()->set_args_offset( | |||||
a, 2 * sizeof(uint16_t)); | |||||
EXPECT_EQ(task1->Init(*op_desc, task_def), ge::SUCCESS); | |||||
ComputeGraphPtr graph = std::make_shared<ComputeGraph>("test"); | |||||
NodePtr node = graph->AddNode(op_desc); | |||||
std::unique_ptr<NodeItem> new_node; | |||||
ASSERT_EQ(NodeItem::Create(node, new_node), SUCCESS); | |||||
NodeItem *node_item = new_node.get(); | |||||
node_item->input_start = 0; | |||||
node_item->output_start = 0; | |||||
node_item->is_dynamic = true; | |||||
node_item->shape_inference_type = DEPEND_SHAPE_RANGE; | |||||
GraphItem graph_item; | |||||
graph_item.node_items_.emplace_back(node_item); | |||||
graph_item.total_inputs_ = 2; | |||||
graph_item.total_outputs_ = 1; | |||||
GeRootModelPtr ge_root_model = std::make_shared<GeRootModel>(graph); | |||||
ge_root_model->SetModelName("test_name"); | |||||
HybridModel hybrid_model(ge_root_model); | |||||
GraphExecutionContext graph_context; | |||||
graph_context.model = &hybrid_model; | |||||
SubgraphContext subgraph_context(&graph_item, &graph_context); | |||||
ASSERT_EQ(subgraph_context.Init(), SUCCESS); | |||||
graph_context.callback_manager = | |||||
std::unique_ptr<CallbackManager>(new CallbackManager()); | |||||
auto node_state = subgraph_context.GetOrCreateNodeState(node_item); | |||||
ASSERT_NE(node_state, nullptr); | |||||
auto outputs_shape = | |||||
reinterpret_cast<uint32_t(*)[1]>(task1->shape_buffer_->GetData()); | |||||
outputs_shape[0][0] = 2; | |||||
outputs_shape[0][1] = 1; | |||||
outputs_shape[0][2] = 2; | |||||
std::vector<std::unique_ptr<AiCoreOpTask>> tasks; | |||||
tasks.emplace_back(std::move(task1)); | |||||
std::unique_ptr<AiCoreNodeTask> aicore_node_task; | |||||
aicore_node_task.reset(new (std::nothrow) AiCoreNodeTask(std::move(tasks))); | |||||
ASSERT_EQ( | |||||
aicore_node_task->ExecuteAsync(*node_state->GetTaskContext(), nullptr), | |||||
SUCCESS); | |||||
std::pair<rtEvent_t, std::pair<rtCallback_t, void *>> entry; | |||||
node_state->GetTaskContext() | |||||
->execution_context_->callback_manager->callback_queue_.Pop(entry); | |||||
auto cb_func = entry.second.first; | |||||
auto cb_args = entry.second.second; | |||||
cb_func(cb_args); | |||||
dlog_setlevel(0, 3, 0); | |||||
} | |||||
} // namespace ge |
@@ -0,0 +1,168 @@ | |||||
/** | |||||
* Copyright 2021-2021 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 <gmock/gmock.h> | |||||
#include <gtest/gtest.h> | |||||
#include <vector> | |||||
#define private public | |||||
#define protected public | |||||
#include "framework/common/taskdown_common.h" | |||||
#include "hybrid/executor/subgraph_context.h" | |||||
#include "hybrid/node_executor/aicore/aicore_op_task.h" | |||||
#include "init/gelib.h" | |||||
#undef private | |||||
#undef protected | |||||
using namespace std; | |||||
using namespace testing; | |||||
namespace ge { | |||||
using namespace hybrid; | |||||
class UtestAiCoreOpTask : public testing::Test { | |||||
protected: | |||||
void SetUp() {} | |||||
void TearDown() {} | |||||
}; | |||||
static ge::OpDescPtr CreateOpDesc(string name = "", string type = "", | |||||
int in_num = 0, int out_num = 0) { | |||||
auto op_desc = std::make_shared<ge::OpDesc>(name, type); | |||||
op_desc->SetStreamId(0); | |||||
static int32_t index = 0; | |||||
op_desc->SetId(index++); | |||||
GeTensorDesc tensor(GeShape(), FORMAT_ND, DT_INT64); | |||||
TensorUtils::SetSize(tensor, 64); | |||||
vector<int64_t> input_offset; | |||||
for (int i = 0; i < in_num; ++i) { | |||||
op_desc->AddInputDesc(tensor); | |||||
input_offset.emplace_back(index * 64 + i * 64); | |||||
} | |||||
op_desc->SetInputOffset(input_offset); | |||||
vector<int64_t> output_offset; | |||||
for (int i = 0; i < out_num; ++i) { | |||||
op_desc->AddOutputDesc(tensor); | |||||
output_offset.emplace_back(index * 64 + in_num * 64 + i * 64); | |||||
} | |||||
op_desc->SetOutputOffset(output_offset); | |||||
op_desc->SetWorkspace({}); | |||||
op_desc->SetWorkspaceBytes({}); | |||||
ge::AttrUtils::SetStr(op_desc, ge::TVM_ATTR_NAME_MAGIC, | |||||
"RT_DEV_BINARY_MAGIC_ELF_AIVEC"); | |||||
bool support_dynamic = true; | |||||
ge::AttrUtils::GetBool(op_desc, "support_dynamicshape", support_dynamic); | |||||
return op_desc; | |||||
} | |||||
TEST_F(UtestAiCoreOpTask, Init_failed) { | |||||
dlog_setlevel(0, 0, 0); | |||||
std::unique_ptr<AiCoreOpTask> task1(new AiCoreOpTask()); | |||||
OpDescPtr op_desc = CreateOpDesc("Add", "Add"); | |||||
ge::AttrUtils::SetInt(*op_desc, ge::ATTR_NAME_UNKNOWN_SHAPE_TYPE, | |||||
DEPEND_SHAPE_RANGE); | |||||
domi::TaskDef task_def; | |||||
task_def.set_type(RT_MODEL_TASK_KERNEL); | |||||
std::vector<uint8_t> args(100, 0); | |||||
task_def.mutable_kernel()->set_args(args.data(), args.size()); | |||||
task_def.mutable_kernel()->set_args_size(100); | |||||
task_def.mutable_kernel()->mutable_context()->set_kernel_type( | |||||
ccKernelType::TE); | |||||
uint16_t args_offset = 20; | |||||
char *a = reinterpret_cast<char *>(&args_offset); | |||||
task_def.mutable_kernel()->mutable_context()->set_args_offset( | |||||
a, 2 * sizeof(uint16_t)); | |||||
EXPECT_EQ(task1->Init(*op_desc, task_def), ge::PARAM_INVALID); | |||||
dlog_setlevel(0, 3, 0); | |||||
} | |||||
TEST_F(UtestAiCoreOpTask, Init_success) { | |||||
dlog_setlevel(0, 0, 0); | |||||
std::unique_ptr<AiCoreOpTask> task1(new AiCoreOpTask()); | |||||
OpDescPtr op_desc = CreateOpDesc("Add", "Add", 2, 1); | |||||
ge::AttrUtils::SetInt(*op_desc, ge::ATTR_NAME_UNKNOWN_SHAPE_TYPE, | |||||
DEPEND_SHAPE_RANGE); | |||||
domi::TaskDef task_def; | |||||
task_def.set_type(RT_MODEL_TASK_KERNEL); | |||||
std::vector<uint8_t> args(100, 0); | |||||
task_def.mutable_kernel()->set_args(args.data(), args.size()); | |||||
task_def.mutable_kernel()->set_args_size(100); | |||||
task_def.mutable_kernel()->mutable_context()->set_kernel_type( | |||||
ccKernelType::TE); | |||||
uint16_t args_offset = 20; | |||||
char *a = reinterpret_cast<char *>(&args_offset); | |||||
task_def.mutable_kernel()->mutable_context()->set_args_offset( | |||||
a, 2 * sizeof(uint16_t)); | |||||
EXPECT_EQ(task1->Init(*op_desc, task_def), ge::SUCCESS); | |||||
dlog_setlevel(0, 3, 0); | |||||
} | |||||
TEST_F(UtestAiCoreOpTask, UpdateOutputsShape_success) { | |||||
dlog_setlevel(0, 0, 0); | |||||
std::unique_ptr<AiCoreOpTask> task1(new AiCoreOpTask()); | |||||
OpDescPtr op_desc = CreateOpDesc("Add", "Add", 2, 1); | |||||
ge::AttrUtils::SetInt(*op_desc, ge::ATTR_NAME_UNKNOWN_SHAPE_TYPE, | |||||
DEPEND_SHAPE_RANGE); | |||||
domi::TaskDef task_def; | |||||
task_def.set_type(RT_MODEL_TASK_KERNEL); | |||||
std::vector<uint8_t> args(100, 0); | |||||
task_def.mutable_kernel()->set_args(args.data(), args.size()); | |||||
task_def.mutable_kernel()->set_args_size(100); | |||||
task_def.mutable_kernel()->mutable_context()->set_kernel_type( | |||||
ccKernelType::TE); | |||||
uint16_t args_offset = 20; | |||||
char *a = reinterpret_cast<char *>(&args_offset); | |||||
task_def.mutable_kernel()->mutable_context()->set_args_offset( | |||||
a, 2 * sizeof(uint16_t)); | |||||
EXPECT_EQ(task1->Init(*op_desc, task_def), ge::SUCCESS); | |||||
ComputeGraphPtr graph = std::make_shared<ComputeGraph>("test"); | |||||
NodePtr node = graph->AddNode(op_desc); | |||||
std::unique_ptr<NodeItem> new_node; | |||||
ASSERT_EQ(NodeItem::Create(node, new_node), SUCCESS); | |||||
NodeItem *node_item = new_node.get(); | |||||
node_item->input_start = 0; | |||||
node_item->output_start = 0; | |||||
node_item->is_dynamic = true; | |||||
node_item->shape_inference_type = DEPEND_SHAPE_RANGE; | |||||
GraphItem graph_item; | |||||
graph_item.node_items_.emplace_back(node_item); | |||||
graph_item.total_inputs_ = 2; | |||||
graph_item.total_outputs_ = 1; | |||||
GraphExecutionContext graph_context; | |||||
SubgraphContext subgraph_context(&graph_item, &graph_context); | |||||
ASSERT_EQ(subgraph_context.Init(), SUCCESS); | |||||
graph_context.callback_manager = | |||||
std::unique_ptr<CallbackManager>(new CallbackManager()); | |||||
auto node_state = subgraph_context.GetOrCreateNodeState(node_item); | |||||
ASSERT_NE(node_state, nullptr); | |||||
auto outputs_shape = | |||||
reinterpret_cast<uint32_t(*)[1]>(task1->shape_buffer_->GetData()); | |||||
outputs_shape[0][0] = 2; | |||||
outputs_shape[0][1] = 1; | |||||
outputs_shape[0][2] = 2; | |||||
ASSERT_EQ(task1->UpdateOutputsShape(*node_state->GetTaskContext()), SUCCESS); | |||||
dlog_setlevel(0, 3, 0); | |||||
} | |||||
} // namespace ge |