Browse Source

!1162 parallel group

From: @chen-hua-baker
Reviewed-by: 
Signed-off-by:
tags/v1.3.0
mindspore-ci-bot Gitee 3 years ago
parent
commit
0731fc4026
11 changed files with 831 additions and 1 deletions
  1. +2
    -0
      ge/CMakeLists.txt
  2. +43
    -0
      ge/graph/build/logical_stream_allocator.cc
  3. +7
    -0
      ge/graph/build/logical_stream_allocator.h
  4. +7
    -0
      ge/graph/manager/graph_manager.cc
  5. +8
    -1
      ge/graph/passes/next_iteration_pass.cc
  6. +354
    -0
      ge/graph/passes/parallel_group_pass.cc
  7. +53
    -0
      ge/graph/passes/parallel_group_pass.h
  8. +7
    -0
      ge/graph/passes/switch_to_stream_switch_pass.cc
  9. +3
    -0
      tests/ut/ge/CMakeLists.txt
  10. +43
    -0
      tests/ut/ge/graph/build/logical_stream_allocator_unittest.cc
  11. +304
    -0
      tests/ut/ge/graph/passes/parallel_group_pass_unittest.cc

+ 2
- 0
ge/CMakeLists.txt View File

@@ -321,6 +321,7 @@ set(TRAIN_SRC_LIST
"graph/passes/variable_ref_useless_control_out_delete_pass.cc"
"graph/passes/end_of_sequence_add_control_pass.cc"
"graph/passes/memcpy_addr_async_pass.cc"
"graph/passes/parallel_group_pass.cc"
"graph/passes/set_input_output_offset_pass.cc"
"graph/preprocess/graph_preprocess.cc"
"graph/preprocess/insert_op/ge_aipp_op.cc"
@@ -608,6 +609,7 @@ set(INFER_SRC_LIST
"graph/passes/hccl_group_pass.cc"
"graph/passes/memcpy_addr_async_pass.cc"
"graph/passes/set_input_output_offset_pass.cc"
"graph/passes/parallel_group_pass.cc"
"graph/manager/model_manager/event_manager.cc"
"graph/manager/util/rt_context_util.cc"
"graph/manager/util/variable_accelerate_ctrl.cc"


+ 43
- 0
ge/graph/build/logical_stream_allocator.cc View File

@@ -377,6 +377,48 @@ Status NodeStreamUpdatePass::Run(ComputeGraphPtr graph, const vector<SubgraphPtr
return SUCCESS;
}

Status UpdateForParallelGroupPass::Run(ComputeGraphPtr graph, const vector<SubgraphPtr> &subgraphs, Context &context) {
std::map<int, vector<OpDescPtr>> stream_op_map;
for (const SubgraphPtr &subgraph : subgraphs) {
auto compute_graph = subgraph->subgraph_info.GetSubGraph();
for (const NodePtr &node : compute_graph->GetDirectNode()) {
OpDescPtr op_desc = node->GetOpDesc();
GE_CHECK_NOTNULL(op_desc);
if (op_desc->HasAttr(ATTR_NAME_PARALLEL_GROUP)) {
int64_t op_desc_stream_id = op_desc->GetStreamId();
stream_op_map[op_desc_stream_id].push_back(op_desc);
}
}
}
for (const auto &itr : stream_op_map) {
if (itr.first == kInvalidStream) {
continue;
}
std::map<std::string, int64_t> group_2_stream_id;
for (const auto &op_desc : itr.second) {
std::string group_name;
if (!AttrUtils::GetStr(op_desc, ATTR_NAME_PARALLEL_GROUP, group_name)) {
GELOGE(FAILED, "[GetAttr][OpDesc]Get node %s ATTR_NAME_PARALLEL_GROUP failed.", op_desc->GetName().c_str());
REPORT_INNER_ERROR("E19999", "Get node %s ATTR_NAME_PARALLEL_GROUP failed.", op_desc->GetName().c_str());
return FAILED;
}
const auto &itr = group_2_stream_id.find(group_name);
int64_t new_stream_id = kInvalidStream;
int64_t old_stream_id = op_desc->GetStreamId();
if (itr != group_2_stream_id.end()) {
new_stream_id = itr->second;
} else {
new_stream_id = context.next_stream++;
group_2_stream_id[group_name] = new_stream_id;
}
op_desc->SetStreamId(new_stream_id);
GELOGD("Node %s assigned stream %ld from stream %ld.",
op_desc->GetName().c_str(), new_stream_id, old_stream_id);
}
}
return SUCCESS;
}

int64_t UpdateForSkippedEnginePass::GetSingleInoutStream(const NodePtr &node) const {
set<int64_t> stream_ids;

@@ -666,6 +708,7 @@ Status LogicalStreamAllocator::RunPasses(const ComputeGraphPtr &graph, const vec
passes.emplace_back(MakeShared<IndependentStreamPass>());
passes.emplace_back(MakeShared<AssignByDependencyPass>());
passes.emplace_back(MakeShared<NodeStreamUpdatePass>());
passes.emplace_back(MakeShared<UpdateForParallelGroupPass>());
passes.emplace_back(MakeShared<AllReduceParallelPass>());
passes.emplace_back(MakeShared<UpdateForSkippedEnginePass>());
}


+ 7
- 0
ge/graph/build/logical_stream_allocator.h View File

@@ -149,6 +149,13 @@ class NodeStreamUpdatePass : public LogicalStreamPass {
Status Run(ComputeGraphPtr graph, const std::vector<SubgraphPtr> &subgraphs, Context &context) override;
};

// assign stream by parallel group
class UpdateForParallelGroupPass : public LogicalStreamPass {
public:
STREAM_PASS_DEFAULT_FUNC(UpdateForParallelGroupPass);
Status Run(ComputeGraphPtr graph, const std::vector<SubgraphPtr> &subgraphs, Context &context) override;
};

// Update the stream of subgraphs to nodes.
class UpdateForSkippedEnginePass : public LogicalStreamPass {
public:


+ 7
- 0
ge/graph/manager/graph_manager.cc View File

@@ -93,6 +93,7 @@
#include "graph/passes/global_step_insert_pass.h"
#include "graph/passes/memcpy_addr_async_pass.h"
#include "graph/passes/hccl_continuous_memcpy_pass.h"
#include "graph/passes/parallel_group_pass.h"
#include "graph/build/label_allocator.h"
#include "graph/utils/tensor_adapter.h"
#include "inc/pass_manager.h"
@@ -2381,6 +2382,12 @@ Status GraphManager::OptimizeStage2(ge::ComputeGraphPtr &compute_graph) {
GE_CHK_STATUS_RET(memcpy_addr.Run(compute_graph), "Add memcpy_addr_async node failed.");
GE_TIMESTAMP_END(AddMemcpyAddrAsyncNode, "MemcpyAddrAsyncPass::Run.");

// Handle parallel group .
GE_TIMESTAMP_START(ParallelGroup);
ParallelGroupPass parallel_group_pass;
GE_CHK_STATUS_RET(parallel_group_pass.Run(compute_graph), "Handle parallel group failed.");
GE_TIMESTAMP_END(ParallelGroup, "ParallelGroupPass::Run.");

// After while sub graph handle, mark all node rw type
auto result = GetCompilerStages(compute_graph->GetGraphID()).optimizer.HandleMemoryRWConflict(compute_graph);
if (result != SUCCESS) {


+ 8
- 1
ge/graph/passes/next_iteration_pass.cc View File

@@ -22,6 +22,10 @@
using std::string;

namespace ge {
namespace {
const int64_t kLoopType = 1;
}

Status NextIterationPass::Run(ComputeGraphPtr graph) {
GELOGD("NextIterationPass Enter");
/// Enter-----------+
@@ -121,7 +125,10 @@ Status NextIterationPass::FindWhileGroups() {
if (switch_node == nullptr) {
continue;
}

if (!AttrUtils::SetInt(switch_node->GetOpDesc(), ATTR_NAME_STREAM_SWITCH_TYPE, kLoopType)) {
GELOGE(INTERNAL_ERROR, "set int failed");
return INTERNAL_ERROR;
}
NodePtr loop_cond = nullptr;
if (FindTargetNode(switch_node, LOOPCOND, true, loop_cond) != SUCCESS) {
GELOGE(INTERNAL_ERROR, "Get LoopCond node failed, frame_name: %s.", frame_name.c_str());


+ 354
- 0
ge/graph/passes/parallel_group_pass.cc View File

@@ -0,0 +1,354 @@
/**
* Copyright 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 "graph/passes/parallel_group_pass.h"

#include "framework/common/debug/ge_log.h"
#include "common/ge/ge_util.h"
#include "framework/common/ge_inner_error_codes.h"
#include "graph/debug/ge_attr_define.h"
#include "graph/utils/graph_utils.h"
#include "graph/utils/node_utils.h"

namespace ge {
namespace {
const int32_t kMaxRecursionDepth = 10;
const int64_t kLoopType = 1;
}

Status ParallelGroupPass::Run(ComputeGraphPtr graph) {
GELOGD("ParallelGroupPass running");
if (graph == nullptr) {
GELOGE(PARAM_INVALID, "[Check][Graph]Input param graph is null, skip ParallelGroupPass.");
REPORT_INNER_ERROR("E19999", "Input param graph is null, skip ParallelGroupPass.");
return PARAM_INVALID;
}

if (graph->GetParentGraph() != nullptr) {
GELOGD("Current graph %s is a subgraph, this pass only support root graph.",
graph->GetName().c_str());
return SUCCESS;
}

if (graph->TopologicalSorting() != GRAPH_SUCCESS) {
GELOGE(FAILED, "[TopoSort][Graph]Graph:%s topological sort failed.", graph->GetName().c_str());
REPORT_CALL_ERROR("E19999", "Graph:%s topological sort failed when ParallelGroupPass run.",
graph->GetName().c_str());
return FAILED;
}

std::unordered_set<std::string> parallel_groups;
int depth = 0;
if (ProcessGraphGroupNodes(graph, depth, parallel_groups) != SUCCESS) {
GELOGE(INTERNAL_ERROR, "[Process][Graph]Process group nodes of graph %s failed.", graph->GetName().c_str());
return INTERNAL_ERROR;
}

if (graph->TopologicalSorting() != GRAPH_SUCCESS) {
GELOGE(FAILED, "[TopoSort][Graph]Graph:%s topological sort failed.", graph->GetName().c_str());
REPORT_CALL_ERROR("E19999", "Graph:%s topological sort failed when ParallelGroupPass run.",
graph->GetName().c_str());
return FAILED;
}

return SUCCESS;
}

Status ParallelGroupPass::ProcessGraphGroupNodes(ComputeGraphPtr graph, int32_t depth,
std::unordered_set<std::string> &parallel_groups) {
if (depth >= kMaxRecursionDepth) {
GELOGE(FAILED, "[Process][SubGraph]There are too much subgraphs:%d > %d(max subgraphs)", depth, kMaxRecursionDepth);
REPORT_INNER_ERROR("E19999", "There are too much subgraphs:%d > %d(max subgraphs)", depth, kMaxRecursionDepth);
return FAILED;
}
std::map<std::string, vector<NodePtr>> group_nodes;
auto candidates = graph->GetDirectNode();
auto root_graph = GraphUtils::FindRootGraph(graph);
for (const auto &node : candidates) {
OpDescPtr op_desc = node->GetOpDesc();
if (op_desc == nullptr) {
continue;
}
std::string group_name;
if (AttrUtils::GetStr(op_desc, ATTR_NAME_PARALLEL_GROUP, group_name)) {
group_nodes[group_name].push_back(node);
parallel_groups.insert(group_name);
GELOGD("Find group node:%s, group_name:%s", node->GetName().c_str(), group_name.c_str());
}

const auto &subgraph_name = op_desc->GetSubgraphInstanceNames();
GE_CHECK_NOTNULL(root_graph);
for (auto name_iter = subgraph_name.rbegin(); name_iter != subgraph_name.rend(); ++name_iter) {
const auto &sub_graph = root_graph->GetSubgraph(*name_iter);
GE_CHECK_NOTNULL(sub_graph);
// if the pass add control edge for known and unknown graph, then the known graph will become unknown graph
// the order between known and unknown graph is guaranteed by dynamic shape executor
// so the parallel group pass do nothing for unknown graph
if (sub_graph->GetGraphUnknownFlag()) {
continue;
}
std::unordered_set<std::string> sub_parallel_groups;
auto ret = ProcessGraphGroupNodes(sub_graph, depth + 1, sub_parallel_groups);
if (ret != SUCCESS) {
GELOGE(FAILED, "[Process][SubGraph]Process sub graph %s failed.", sub_graph->GetName().c_str());
return FAILED;
}
for (const auto &sub_parallel_group : sub_parallel_groups) {
parallel_groups.insert(sub_parallel_group);
group_nodes[sub_parallel_group].emplace_back(node);
}
}
}

std::map<NodePtr, std::pair<std::set<NodePtr>, NodePtr>> node_2_switch_merge;
if (ProcessGroupNodeInSwitch(graph, node_2_switch_merge) != SUCCESS) {
GELOGE(FAILED, "[Process][Node]Process group node in switch failed, graph:%s.", graph->GetName().c_str());
return FAILED;
}

for (const auto &itr : group_nodes) {
const auto &nodes = itr.second;
if (nodes.empty()) {
continue;
}
NodePtr pre_node = nodes[0];
NodePtr cur_node = nullptr;
for (std::size_t i = 1; i < nodes.size(); i++) {
cur_node = nodes[i];
GELOGD("Original add ctrl anchor for node:%s->%s", pre_node->GetName().c_str(),
cur_node->GetName().c_str());
if (ReplaceWithSwitchAndMerge(pre_node, cur_node, node_2_switch_merge) != SUCCESS) {
GELOGE(FAILED, "[Replace][Node]Replace switch and merges for nodes: %s and %s failed.",
pre_node->GetName().c_str(), cur_node->GetName().c_str());
return FAILED;
}
pre_node = cur_node;
}
}

return SUCCESS;
}

Status ParallelGroupPass::AddCtrlEdge(NodePtr pre_node, NodePtr cur_node) {
if (pre_node == cur_node) {
GELOGD("Pre_node and cur_node are same, no need add anchor");
return SUCCESS;
}
auto in_nodes = cur_node->GetInAllNodes();
for (const auto &node : in_nodes) {
if (pre_node == node) {
GELOGD("Node:%s and %s already linked", pre_node->GetName().c_str(),
cur_node->GetName().c_str());
return SUCCESS;
}
}
GELOGD("Finally add ctrl anchor for node:%s->%s", pre_node->GetName().c_str(),
cur_node->GetName().c_str());
return GraphUtils::AddEdge(pre_node->GetOutControlAnchor(),
cur_node->GetInControlAnchor());
}

Status ParallelGroupPass::ProcessGroupNodeInSwitch(ComputeGraphPtr graph,
std::map<NodePtr, std::pair<std::set<NodePtr>, NodePtr>> &node_2_switch_merge) {

std::string type;
auto direct_nodes = graph->GetDirectNode();
for (const auto &node : direct_nodes) {
type = node->GetType();
if (type != STREAMSWITCH) {
continue;
}

if (IsBigSmallLoopStreamSwitch(node->GetOpDesc()) ||
IsWhileStreamSwitch(node->GetOpDesc())) {
continue;
}

std::vector<NodePtr> merge_nodes;
std::set<NodePtr> group_nodes;
std::set<std::string> stream_labels;

FindGroupNodeAndMerge(node, group_nodes, merge_nodes, stream_labels);

if (merge_nodes.empty() || (!group_nodes.empty() && stream_labels.size() > 1)) {
GELOGE(FAILED, "[Process][Node]Cannot find merge node or exist switch nestification, switch node:%s,"
"merge_vec size:%zu, stream_labels size:%zu, graph:%s.", node->GetName().c_str(),
merge_nodes.size(), stream_labels.size(), graph->GetName().c_str());
REPORT_INNER_ERROR("E19999", "Cannot find merge node or exist switch nest, switch node:%s,"
"merge_vec size: %zu, stream_labels size: %zu, graph:%s.", node->GetName().c_str(),
merge_nodes.size(), stream_labels.size(), graph->GetName().c_str());
return FAILED;
}

std::sort(merge_nodes.begin(), merge_nodes.end(),
[] (NodePtr a, NodePtr b) -> bool {
return (a->GetOpDesc()->GetId() < b->GetOpDesc()->GetId());
});

NodePtr cast_node = NodeUtils::GetInDataNodeByIndex(*node, 0);
GE_CHECK_NOTNULL(cast_node);
if (MappingNodeToSwitchAndMerge(group_nodes, merge_nodes,
cast_node, node,
node_2_switch_merge) != SUCCESS) {
GELOGE(FAILED, "[Mapping][Node]Mapping node to switch and merge failed, graph:%s.", graph->GetName().c_str());
REPORT_CALL_ERROR("E19999", "[Mapping][Node]Mapping node to switch and merge failed, graph:%s.",
graph->GetName().c_str());
return FAILED;
}
}

return SUCCESS;
}

void ParallelGroupPass::FindGroupNodeAndMerge(NodePtr stream_switch_node, std::set<NodePtr> &group_nodes,
std::vector<NodePtr> &merge_nodes, std::set<std::string> &stream_labels) {
std::string type;
std::deque<NodePtr> candidates;
std::set<NodePtr> visited;

candidates.push_back(stream_switch_node);
while (!candidates.empty()) {
NodePtr tmp_node = candidates.front();
candidates.pop_front();
for (const auto &out_node : tmp_node->GetOutAllNodes()) {
type = out_node->GetType();
if (type == STREAMMERGE) {
merge_nodes.emplace_back(out_node);
continue;
}
const auto &op = out_node->GetOpDesc();
if (op != nullptr && op->HasAttr(ATTR_NAME_PARALLEL_GROUP)) {
group_nodes.emplace(out_node);
}
if (visited.count(out_node) > 0) {
continue;
}
candidates.push_back(out_node);
visited.insert(out_node);
std::string stream_label;
if (ge::AttrUtils::GetStr(out_node->GetOpDesc(), ATTR_NAME_STREAM_LABEL, stream_label)) {
stream_labels.insert(stream_label);
}
}
}
}

Status ParallelGroupPass::MappingNodeToSwitchAndMerge(const std::set<NodePtr> &group_nodes,
const std::vector<NodePtr> &merge_nodes,
const NodePtr &cast_node, const NodePtr &switch_node,
std::map<NodePtr, std::pair<std::set<NodePtr>, NodePtr>> &node_2_switch_merge) {
for (const auto &group_node : group_nodes) {
auto itr = node_2_switch_merge.find(group_node);
if (itr != node_2_switch_merge.end()) {
auto &tmp = itr->second;
auto &switch_set = tmp.first;
const auto &merge_node = tmp.second;
GELOGD("Find group node: %s in switch %s and merge %s.",
group_node->GetName().c_str(), switch_node->GetName().c_str(), merge_node->GetName().c_str());
if (merge_node != merge_nodes.back()) {
GELOGE(FAILED, "[Mapping][Node]Has two different merge nodes: %s and %s, graph's structure is invalid",
merge_node->GetName().c_str(), merge_nodes.back()->GetName().c_str());
REPORT_INNER_ERROR("E19999", "Has two different merge nodes: %s and %s,"
"graph's structure is invalid",
merge_node->GetName().c_str(), merge_nodes.back()->GetName().c_str());
return FAILED;
}
switch_set.insert(cast_node);
} else {
node_2_switch_merge.emplace(group_node,
std::make_pair(std::set<NodePtr>{cast_node}, merge_nodes.back()));
}
}
return SUCCESS;
}

Status ParallelGroupPass::ReplaceWithSwitchAndMerge(NodePtr pre_node, NodePtr cur_node,
const std::map<NodePtr, std::pair<std::set<NodePtr>, NodePtr>> &node_2_switch_merge) {
auto pre_itr = node_2_switch_merge.find(pre_node);
auto cur_itr = node_2_switch_merge.find(cur_node);
if (pre_itr != node_2_switch_merge.end()) {
if (cur_itr != node_2_switch_merge.end()) {
const auto &pre_set = pre_itr->second.first;
const auto &cur_set = cur_itr->second.first;
if (!HasSameSwitch(pre_set, cur_set)) {
pre_node = pre_itr->second.second;
for (const auto &switch_node : cur_itr->second.first) {
if (AddCtrlEdge(pre_node, switch_node) != SUCCESS) {
GELOGE(FAILED, "[AddEdge][Node]Add edge for nodes: %s->%s failed.",
pre_node->GetName().c_str(), switch_node->GetName().c_str());
REPORT_CALL_ERROR("E19999", "[AddEdge][Node]Add edge for nodes: %s->%s failed.",
pre_node->GetName().c_str(), switch_node->GetName().c_str());
return FAILED;
}
}
}
return SUCCESS;
} else {
pre_node = pre_itr->second.second;
return AddCtrlEdge(pre_node, cur_node);
}
} else {
if (cur_itr != node_2_switch_merge.end()) {
for (const auto &switch_node : cur_itr->second.first) {
int64_t pre_id = pre_node->GetOpDesc()->GetId();
int64_t switch_id = switch_node->GetOpDesc()->GetId();
// avoid ring
if (pre_id > switch_id) {
auto merge_node = cur_itr->second.second;
if (AddCtrlEdge(merge_node, pre_node) != SUCCESS) {
GELOGE(FAILED, "[AddEdge][Node]Add edge for nodes: %s->%s failed.",
pre_node->GetName().c_str(), switch_node->GetName().c_str());
REPORT_CALL_ERROR("E19999", "[AddEdge][Node]Add edge for nodes: %s->%s failed.",
pre_node->GetName().c_str(), switch_node->GetName().c_str());
return FAILED;
}
} else {
if (AddCtrlEdge(pre_node, switch_node) != SUCCESS) {
GELOGE(FAILED, "[AddEdge][Node]Add edge for nodes: %s->%s failed.",
pre_node->GetName().c_str(), switch_node->GetName().c_str());
REPORT_CALL_ERROR("E19999", "[AddEdge][Node]Add edge for nodes: %s->%s failed.",
pre_node->GetName().c_str(), switch_node->GetName().c_str());
return FAILED;
}
}
}
} else {
return AddCtrlEdge(pre_node, cur_node);
}
}
return SUCCESS;
}

bool ParallelGroupPass::HasSameSwitch(const std::set<NodePtr> &switch_set1, const std::set<NodePtr> &switch_set2) {
for (const auto &node1 : switch_set1) {
auto itr = switch_set2.find(node1);
if (itr != switch_set2.end()) {
return true;
}
}
return false;
}

bool ParallelGroupPass::IsBigSmallLoopStreamSwitch(OpDescPtr switch_op_desc) {
return !AttrUtils::HasAttr(switch_op_desc, ATTR_NAME_SWITCH_TRUE_BRANCH_FLAG);
}

bool ParallelGroupPass::IsWhileStreamSwitch(OpDescPtr switch_op_desc) {
int64_t stream_switch_type = -1;
return (AttrUtils::GetInt(switch_op_desc, ATTR_NAME_STREAM_SWITCH_TYPE, stream_switch_type) &&
stream_switch_type == kLoopType);
}
} // namespace ge

+ 53
- 0
ge/graph/passes/parallel_group_pass.h View File

@@ -0,0 +1,53 @@
/**
* Copyright 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_GRAPH_PASSES_PARALLEL_GROUP_PASS_H
#define GE_GRAPH_PASSES_PARALLEL_GROUP_PASS_H

#include <map>
#include <unordered_set>
#include "graph/graph.h"
#include "inc/graph_pass.h"

namespace ge {
class ParallelGroupPass : public GraphPass {
public:
Status Run(ComputeGraphPtr graph) override;
private:
Status ProcessGraphGroupNodes(ComputeGraphPtr graph, int32_t depth, std::unordered_set<std::string> &parallel_group);

Status AddCtrlEdge(NodePtr pre_node, NodePtr cur_node);

Status ReplaceWithSwitchAndMerge(NodePtr pre_node, NodePtr cur_node,
const std::map<NodePtr, std::pair<std::set<NodePtr>, NodePtr>> &node_2_switch_merge);

bool HasSameSwitch(const std::set<NodePtr> &a, const std::set<NodePtr> &b);

Status ProcessGroupNodeInSwitch(ComputeGraphPtr graph,
std::map<NodePtr, std::pair<std::set<NodePtr>, NodePtr>> &node_2_switch_merge);

void FindGroupNodeAndMerge(NodePtr stream_switch_node, std::set<NodePtr> &group_nodes,
std::vector<NodePtr> &merge_nodes, std::set<std::string> &stream_labels);

Status MappingNodeToSwitchAndMerge(const std::set<NodePtr> &group_set, const std::vector<NodePtr> &merge_vec,
const NodePtr &cast_node, const NodePtr &switch_node,
std::map<NodePtr, std::pair<std::set<NodePtr>, NodePtr>> &node_2_switch_merge);

bool IsBigSmallLoopStreamSwitch(OpDescPtr switch_op_desc);
bool IsWhileStreamSwitch(OpDescPtr switch_op_desc);
};
} // namespace ge
#endif // GE_GRAPH_PASSES_PARALLEL_GROUP_PASS_H

+ 7
- 0
ge/graph/passes/switch_to_stream_switch_pass.cc View File

@@ -307,6 +307,13 @@ NodePtr SwitchToStreamSwitchPass::CreateStreamSwitchNode(const ComputeGraphPtr &
hccl_group_id.c_str());
}

int64_t switch_type;
if (AttrUtils::GetInt(switch_node->GetOpDesc(), ATTR_NAME_STREAM_SWITCH_TYPE, switch_type)) {
(void)AttrUtils::SetInt(op_desc, ATTR_NAME_STREAM_SWITCH_TYPE, switch_type);
GELOGD("Set attr ATTR_NAME_STREAM_SWITCH_TYPE for Stream_Switch %s, value is %ld.", node_name.c_str(),
switch_type);
}

if (!AttrUtils::SetInt(op_desc, ATTR_NAME_SWITCH_DATA_TYPE, RT_SWITCH_INT32) ||
!AttrUtils::SetInt(op_desc, ATTR_NAME_STREAM_SWITCH_COND, (int64_t)RT_EQUAL)) {
GELOGE(INTERNAL_ERROR, "set int failed");


+ 3
- 0
tests/ut/ge/CMakeLists.txt View File

@@ -274,6 +274,7 @@ set(COMMON_SRC_FILES
"${GE_CODE_DIR}/ge/graph/passes/set_input_output_offset_pass.cc"
"${GE_CODE_DIR}/ge/graph/passes/remove_same_const_pass.cc"
"${GE_CODE_DIR}/ge/graph/passes/useless_control_out_remove_pass.cc"
"${GE_CODE_DIR}/ge/graph/passes/parallel_group_pass.cc"
"${GE_CODE_DIR}/ge/model/ge_model.cc"
"${GE_CODE_DIR}/ge/common/cust_aicpu_kernel_store.cc"
"${GE_CODE_DIR}/ge/graph/load/model_manager/model_utils.cc"
@@ -519,6 +520,7 @@ set(GRAPH_PASS_COMMON_SRC_FILES
"${GE_CODE_DIR}/ge/graph/passes/compile_nodes_pass.cc"
"${GE_CODE_DIR}/ge/graph/common/transop_util.cc"
"${GE_CODE_DIR}/ge/graph/passes/flow_ctrl_pass.cc"
"${GE_CODE_DIR}/ge/graph/passes/parallel_group_pass.cc"
#"${GE_CODE_DIR}/ge/graph/optimize/optimizer/allreduce_fusion_pass.cc"
"${GE_CODE_DIR}/ge/graph/passes/folding_pass.cc"
"${GE_CODE_DIR}/ge/graph/passes/variable_op_pass.cc"
@@ -697,6 +699,7 @@ set(PASS_TEST_FILES
"graph/passes/replace_with_empty_const_pass_unittest.cc"
"graph/passes/link_gen_mask_nodes_pass_unittest.cc"
"graph/passes/transpose_transdata_pass_unittest.cc"
"graph/passes/parallel_group_pass_unittest.cc"
)

set(KERNEL_TEST_FILES


+ 43
- 0
tests/ut/ge/graph/build/logical_stream_allocator_unittest.cc View File

@@ -32,6 +32,7 @@
#include "graph/compute_graph.h"
#include "graph/utils/attr_utils.h"
#include "graph/utils/graph_utils.h"
#include "graph/debug/ge_attr_define.h"

using namespace std;

@@ -153,6 +154,22 @@ class UtestLogicalStreamAllocator : public testing::Test {
return CreateSubgraphWithName("graph", engine, stream_label, in_num, out_num);
}

SubGraphInfoPtr CreateParallelGroupSubgraphWithName(const string &name, const string &engine,
const string &stream_label = "",
std::string group_name = "1") {
ComputeGraphPtr compute_graph = make_shared<ComputeGraph>(name);
OpDescPtr op_desc = std::make_shared<OpDesc>("relu", "Relu");
op_desc->AddInputDesc(GeTensorDesc());
op_desc->AddOutputDesc(GeTensorDesc());
AttrUtils::SetStr(op_desc, ATTR_NAME_PARALLEL_GROUP, group_name);
compute_graph->AddNode(op_desc);

SubGraphInfoPtr subgraph = BuildSubGraph(compute_graph, engine, stream_label);
AddPlaceHolderAndEnd(subgraph, 1, 1);

return subgraph;
}

void LinkSubGraph(SubGraphInfoPtr subgraph1, const string &end_name, SubGraphInfoPtr subgraph2,
const string &placeholder_name) {
NodePtr end_node = subgraph1->GetSubGraph()->FindNode(end_name);
@@ -878,4 +895,30 @@ TEST_F(UtestLogicalStreamAllocator, test_all_reduce_parallel_pass) {
EXPECT_EQ(ret, NOT_CHANGED);
}

TEST_F(UtestLogicalStreamAllocator, test_parallel_group) {
SubGraphInfoPtr data = CreateDataSubgraph();
SubGraphInfoPtr subgraph1 = CreateParallelGroupSubgraphWithName("graph1", "engine1", "");
SubGraphInfoPtr subgraph2 = CreateParallelGroupSubgraphWithName("graph2", "engine2", "", "2");
SubGraphInfoPtr subgraph3 = CreateParallelGroupSubgraphWithName("graph3", "engine3", "", "3");
SubGraphInfoPtr subgraph4 = CreateParallelGroupSubgraphWithName("graph4", "engine4", "", "4");
LinkSubGraph(data, "end", subgraph1, "placeholder");
LinkSubGraph(subgraph1, "end", subgraph2, "placeholder");
LinkSubGraph(subgraph2, "end", subgraph3, "placeholder");
LinkSubGraph(subgraph3, "end", subgraph4, "placeholder");

EngineConfPtr conf1 = make_shared<EngineConf>();
conf1->id = subgraph1->GetEngineName();
EngineConfPtr conf2 = make_shared<EngineConf>();
conf2->id = subgraph2->GetEngineName();
conf2->attach = false;
EngineConfPtr conf3 = make_shared<EngineConf>();
conf3->id = subgraph3->GetEngineName();
conf3->attach = false;
EngineConfPtr conf4 = make_shared<EngineConf>();
conf4->id = subgraph4->GetEngineName();

Status status = AssignLogicalStreams({subgraph1, subgraph2, subgraph3, subgraph4}, {conf1, conf2, conf3, conf4});
EXPECT_EQ(status, ge::SUCCESS);
}

} // namespace ge

+ 304
- 0
tests/ut/ge/graph/passes/parallel_group_pass_unittest.cc View File

@@ -0,0 +1,304 @@
/**
* 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 <gtest/gtest.h>
#include <cstdint>
#include <string>

#define private public

#include "common/ge_inner_error_codes.h"
#include "inc/pass_manager.h"
#include "utils/graph_utils.h"
#include "graph/passes/parallel_group_pass.h"
#undef private

namespace ge {
namespace {

class UtestGraphPassesParallelGgroupPass : public testing::Test {
protected:
UtestGraphPassesParallelGgroupPass() {
graph_ = std::make_shared<ComputeGraph>("test");
sub_graph_ = std::make_shared<ComputeGraph>("test_subgraph");
vector<int64_t> shape_vec{1, 1, 1, 1};
GeShape shape = GeShape(shape_vec);
default_tensor_desc_ = std::make_shared<GeTensorDesc>();
default_tensor_desc_->SetShape(shape);
default_tensor_desc_->SetFormat(FORMAT_NCHW);
default_tensor_desc_->SetDataType(DT_FLOAT);
}

NodePtr NewNode(const std::string &name, const std::string &type,
int input_cnt, int output_cnt, bool isSubgraph = false) {
OpDescPtr op_desc = std::make_shared<OpDesc>(name, type);
for (int i = 0; i < input_cnt; ++i) {
op_desc->AddInputDesc(default_tensor_desc_->Clone());
}

for (int i = 0; i < output_cnt; ++i) {
op_desc->AddOutputDesc(default_tensor_desc_->Clone());
}
NodePtr node = nullptr;
if (isSubgraph) {
node = sub_graph_->AddNode(op_desc);
(void)node->SetOwnerComputeGraph(sub_graph_);
} else {
node = graph_->AddNode(op_desc);
(void)node->SetOwnerComputeGraph(graph_);
}

return node;
}

void BuildDefaultGraph() {
/// input
/// \
/// sqrt pred
/// \ /
/// cast
/// / \
/// switch_t switch_f
/// | |
/// F T
/// | |
/// Merge
/// |
/// relu
/// |
/// sqrt1
input_node_ = NewNode("input", RELU, 0, 1);
sqrt_node_ = NewNode("sqrt", SQRT, 1, 1);
pred_node_ = NewNode("pred", GREATER, 2, 1);
cast_node_ = NewNode("cast", CAST, 2, 2);
AttrUtils::SetStr(input_node_->GetOpDesc(), ATTR_NAME_PARALLEL_GROUP, "1");

switch_node_t = NewNode("switch_t", STREAMSWITCH, 1, 1);
AttrUtils::SetBool(switch_node_t->GetOpDesc(), ATTR_NAME_SWITCH_TRUE_BRANCH_FLAG, true);
switch_node_f = NewNode("switch_f", STREAMSWITCH, 1, 1);
AttrUtils::SetBool(switch_node_f->GetOpDesc(), ATTR_NAME_SWITCH_TRUE_BRANCH_FLAG, false);
output_false_node_ = NewNode("false_output", RELU, 1, 1);
AttrUtils::SetStr(output_false_node_->GetOpDesc(), ATTR_NAME_PARALLEL_GROUP, "1");
output_true_node_ = NewNode("true_output", RELU, 1, 1);
AttrUtils::SetStr(output_true_node_->GetOpDesc(), ATTR_NAME_PARALLEL_GROUP, "1");
merge_node_ = NewNode("merge", STREAMMERGE, 2, 1);
relu_node_ = NewNode("relu", RELU, 1, 1);
sqrt_node1_ = NewNode("sqrt1", SQRT, 1, 1);
AttrUtils::SetStr(sqrt_node1_->GetOpDesc(), ATTR_NAME_PARALLEL_GROUP, "1");

GraphUtils::AddEdge(input_node_->GetOutDataAnchor(0), sqrt_node_->GetInDataAnchor(0));
GraphUtils::AddEdge(pred_node_->GetOutDataAnchor(0), cast_node_->GetInDataAnchor(0));
GraphUtils::AddEdge(sqrt_node_->GetOutDataAnchor(0), cast_node_->GetInDataAnchor(1));
GraphUtils::AddEdge(cast_node_->GetOutDataAnchor(0), switch_node_t->GetInDataAnchor(0));
GraphUtils::AddEdge(cast_node_->GetOutDataAnchor(1), switch_node_f->GetInDataAnchor(0));
GraphUtils::AddEdge(switch_node_f->GetOutDataAnchor(0), output_false_node_->GetInDataAnchor(0));
GraphUtils::AddEdge(switch_node_t->GetOutDataAnchor(0), output_true_node_->GetInDataAnchor(0));

GraphUtils::AddEdge(output_false_node_->GetOutDataAnchor(0), merge_node_->GetInDataAnchor(0));
GraphUtils::AddEdge(output_true_node_->GetOutDataAnchor(0), merge_node_->GetInDataAnchor(1));
GraphUtils::AddEdge(merge_node_->GetOutDataAnchor(0), relu_node_->GetInDataAnchor(0));
GraphUtils::AddEdge(relu_node_->GetOutDataAnchor(0), sqrt_node1_->GetInDataAnchor(0));

output_false_node_->GetOpDesc()->SetIsInputConst({false});
output_true_node_->GetOpDesc()->SetIsInputConst({false});
}

void BuildDefaultGraph1() {
/// input
/// \
/// sqrt pred
/// \ /
/// Switch
/// | |
/// ----F T----
/// \ | / \
/// \ Merge1 Merge2
/// \_________|
input_node_ = NewNode("input", RELU, 0, 1);
AttrUtils::SetStr(input_node_->GetOpDesc(), ATTR_NAME_PARALLEL_GROUP, "1");
pred_node_ = NewNode("pred", GREATER, 2, 1);
sqrt_node_ = NewNode("sqrt", SQRT, 1, 1);
cast_node_ = NewNode("cast", CAST, 2, 2);

switch_node_t = NewNode("switch_t", STREAMSWITCH, 1, 1);
AttrUtils::SetBool(switch_node_t->GetOpDesc(), ATTR_NAME_SWITCH_TRUE_BRANCH_FLAG, true);
switch_node_f = NewNode("switch_f", STREAMSWITCH, 1, 1);
AttrUtils::SetBool(switch_node_f->GetOpDesc(), ATTR_NAME_SWITCH_TRUE_BRANCH_FLAG, false);
output_false_node_ = NewNode("false_output", RELU, 1, 2);
AttrUtils::SetStr(output_false_node_->GetOpDesc(), ATTR_NAME_PARALLEL_GROUP, "1");
output_true_node_ = NewNode("true_output", RELU, 1, 2);
AttrUtils::SetStr(output_true_node_->GetOpDesc(), ATTR_NAME_PARALLEL_GROUP, "1");
merge_node_ = NewNode("merge", STREAMMERGE, 2, 1);
merge_node1_ = NewNode("merge1", STREAMMERGE, 2, 1);

GraphUtils::AddEdge(input_node_->GetOutDataAnchor(0), sqrt_node_->GetInDataAnchor(0));
GraphUtils::AddEdge(pred_node_->GetOutDataAnchor(0), cast_node_->GetInDataAnchor(0));
GraphUtils::AddEdge(sqrt_node_->GetOutDataAnchor(0), cast_node_->GetInDataAnchor(1));
GraphUtils::AddEdge(cast_node_->GetOutDataAnchor(0), switch_node_t->GetInDataAnchor(0));
GraphUtils::AddEdge(cast_node_->GetOutDataAnchor(1), switch_node_f->GetInDataAnchor(0));
GraphUtils::AddEdge(switch_node_f->GetOutDataAnchor(0), output_false_node_->GetInDataAnchor(0));
GraphUtils::AddEdge(switch_node_t->GetOutDataAnchor(0), output_true_node_->GetInDataAnchor(0));

GraphUtils::AddEdge(output_false_node_->GetOutDataAnchor(0), merge_node_->GetInDataAnchor(0));
GraphUtils::AddEdge(output_true_node_->GetOutDataAnchor(0), merge_node_->GetInDataAnchor(1));
GraphUtils::AddEdge(output_false_node_->GetOutDataAnchor(1), merge_node1_->GetInDataAnchor(0));
GraphUtils::AddEdge(output_true_node_->GetOutDataAnchor(1), merge_node1_->GetInDataAnchor(1));

output_false_node_->GetOpDesc()->SetIsInputConst({false});
output_true_node_->GetOpDesc()->SetIsInputConst({false});
}


void BuildDefaultGraph2() {
/// input input1
/// \ \
/// sqrt pred sqrt1 pred1
/// \ / \ /
/// Switch Switch1
/// | | _______|
/// | | /
/// ____F T____
/// \ | / \
/// \ Merge1 Merge2
/// \__________|
input_node_ = NewNode("input", RELU, 0, 2);
input_node1_ = NewNode("input_1", RELU, 0, 2);
sqrt_node_ = NewNode("sqrt", SQRT, 1, 1);
pred_node_ = NewNode("pred", GREATER, 2, 1);
sqrt_node1_ = NewNode("sqrt_1", SQRT, 1, 1);
pred_node1_ = NewNode("pred_1", LESS, 2, 1);
cast_node_ = NewNode("cast", CAST, 2, 2);
cast_node1_ = NewNode("cast_1", CAST, 2, 2);
AttrUtils::SetStr(input_node_->GetOpDesc(), ATTR_NAME_PARALLEL_GROUP, "1");
AttrUtils::SetStr(input_node1_->GetOpDesc(), ATTR_NAME_PARALLEL_GROUP, "2");

switch_node_t = NewNode("switch_t", STREAMSWITCH, 1, 1);
AttrUtils::SetBool(switch_node_t->GetOpDesc(), ATTR_NAME_SWITCH_TRUE_BRANCH_FLAG, true);
switch_node_f = NewNode("switch_f", STREAMSWITCH, 1, 1);
AttrUtils::SetBool(switch_node_f->GetOpDesc(), ATTR_NAME_SWITCH_TRUE_BRANCH_FLAG, false);
switch_node1_t = NewNode("switch1_t", STREAMSWITCH, 1, 1);
AttrUtils::SetBool(switch_node1_t->GetOpDesc(), ATTR_NAME_SWITCH_TRUE_BRANCH_FLAG, true);
switch_node1_f = NewNode("switch1_f", STREAMSWITCH, 1, 1);
AttrUtils::SetBool(switch_node1_f->GetOpDesc(), ATTR_NAME_SWITCH_TRUE_BRANCH_FLAG, false);
output_false_node_ = NewNode("false_output", RELU, 2, 2);
AttrUtils::SetStr(output_false_node_->GetOpDesc(), ATTR_NAME_PARALLEL_GROUP, "1");
output_true_node_ = NewNode("true_output", RELU, 2, 2);
AttrUtils::SetStr(output_true_node_->GetOpDesc(), ATTR_NAME_PARALLEL_GROUP, "2");
merge_node_ = NewNode("merge", STREAMMERGE, 2, 1);
merge_node1_ = NewNode("merge1", STREAMMERGE, 2, 1);

GraphUtils::AddEdge(input_node_->GetOutDataAnchor(0), sqrt_node_->GetInDataAnchor(0));
GraphUtils::AddEdge(pred_node_->GetOutDataAnchor(0), cast_node_->GetInDataAnchor(0));
GraphUtils::AddEdge(sqrt_node_->GetOutDataAnchor(0), cast_node_->GetInDataAnchor(1));
GraphUtils::AddEdge(cast_node_->GetOutDataAnchor(0), switch_node_t->GetInDataAnchor(0));
GraphUtils::AddEdge(cast_node_->GetOutDataAnchor(1), switch_node_f->GetInDataAnchor(0));
GraphUtils::AddEdge(switch_node_f->GetOutDataAnchor(0), output_false_node_->GetInDataAnchor(0));
GraphUtils::AddEdge(switch_node_t->GetOutDataAnchor(0), output_true_node_->GetInDataAnchor(0));

GraphUtils::AddEdge(input_node1_->GetOutDataAnchor(0), sqrt_node1_->GetInDataAnchor(0));
GraphUtils::AddEdge(pred_node1_->GetOutDataAnchor(0), cast_node1_->GetInDataAnchor(0));
GraphUtils::AddEdge(sqrt_node1_->GetOutDataAnchor(0), cast_node1_->GetInDataAnchor(1));
GraphUtils::AddEdge(cast_node1_->GetOutDataAnchor(0), switch_node1_t->GetInDataAnchor(0));
GraphUtils::AddEdge(cast_node1_->GetOutDataAnchor(1), switch_node1_f->GetInDataAnchor(0));
GraphUtils::AddEdge(switch_node1_f->GetOutDataAnchor(0), output_false_node_->GetInDataAnchor(1));
GraphUtils::AddEdge(switch_node1_t->GetOutDataAnchor(0), output_true_node_->GetInDataAnchor(1));

GraphUtils::AddEdge(output_false_node_->GetOutDataAnchor(0), merge_node_->GetInDataAnchor(0));
GraphUtils::AddEdge(output_true_node_->GetOutDataAnchor(0), merge_node_->GetInDataAnchor(1));
GraphUtils::AddEdge(output_false_node_->GetOutDataAnchor(1), merge_node1_->GetInDataAnchor(0));
GraphUtils::AddEdge(output_true_node_->GetOutDataAnchor(1), merge_node1_->GetInDataAnchor(1));

output_false_node_->GetOpDesc()->SetIsInputConst({false});
output_true_node_->GetOpDesc()->SetIsInputConst({false});
}

ComputeGraphPtr graph_;
ComputeGraphPtr sub_graph_;
GeTensorDescPtr default_tensor_desc_;
ParallelGroupPass pass_;
NodePtr pred_node_;
NodePtr pred_node1_;
NodePtr cast_node_;
NodePtr cast_node1_;
NodePtr sqrt_node_;
NodePtr sqrt_node1_;
NodePtr input_node_;
NodePtr input_node1_;
NodePtr switch_node_t;
NodePtr switch_node_f;
NodePtr switch_node1_t;
NodePtr switch_node1_f;
NodePtr output_false_node_;
NodePtr output_true_node_;
NodePtr merge_node_;
NodePtr merge_node1_;
NodePtr relu_node_;
};

TEST_F(UtestGraphPassesParallelGgroupPass, null_graph) {
ComputeGraphPtr graph = nullptr;
auto ret = pass_.Run(graph);
EXPECT_EQ(ret, PARAM_INVALID);
}

TEST_F(UtestGraphPassesParallelGgroupPass, normal_graph) {
BuildDefaultGraph();
auto ret = pass_.Run(graph_);
EXPECT_EQ(ret, GRAPH_SUCCESS);
EXPECT_EQ(true, input_node_->GetOutControlAnchor()->IsLinkedWith(cast_node_->GetInControlAnchor()));
EXPECT_EQ(true, merge_node_->GetOutControlAnchor()->IsLinkedWith(sqrt_node1_->GetInControlAnchor()));
EXPECT_EQ(false, output_false_node_->GetOutControlAnchor()->IsLinkedWith(output_true_node_->GetInControlAnchor()));
}

TEST_F(UtestGraphPassesParallelGgroupPass, normal_graph1) {
BuildDefaultGraph1();
auto ret = pass_.Run(graph_);
EXPECT_EQ(ret, GRAPH_SUCCESS);
EXPECT_EQ(true, input_node_->GetOutControlAnchor()->IsLinkedWith(cast_node_->GetInControlAnchor()));
}

TEST_F(UtestGraphPassesParallelGgroupPass, normal_graph2) {
BuildDefaultGraph2();
auto ret = pass_.Run(graph_);
EXPECT_EQ(ret, GRAPH_SUCCESS);
EXPECT_EQ(true, input_node_->GetOutControlAnchor()->IsLinkedWith(cast_node_->GetInControlAnchor()));
EXPECT_EQ(true, input_node1_->GetOutControlAnchor()->IsLinkedWith(cast_node1_->GetInControlAnchor()));
}

TEST_F(UtestGraphPassesParallelGgroupPass, normal_subgraph) {
BuildDefaultGraph1();
NodePtr input_node1 = NewNode("input1", RELU, 0, 1, true);
NodePtr input_node2 = NewNode("input2", RELU, 0, 1, true);
NodePtr add = NewNode("add", ADD, 2, 1, true);
AttrUtils::SetStr(input_node1->GetOpDesc(), ATTR_NAME_PARALLEL_GROUP, "1");
AttrUtils::SetStr(input_node2->GetOpDesc(), ATTR_NAME_PARALLEL_GROUP, "1");

sub_graph_->SetParentNode(input_node_);
sub_graph_->SetParentGraph(graph_);
auto ret = graph_->AddSubgraph(sub_graph_->GetName(), sub_graph_);
EXPECT_EQ(ret, GRAPH_SUCCESS);
ret = input_node_->GetOpDesc()->AddSubgraphName(sub_graph_->GetName());
EXPECT_EQ(ret, GRAPH_SUCCESS);
ret = input_node_->GetOpDesc()->SetSubgraphInstanceName(0, sub_graph_->GetName());
EXPECT_EQ(ret, GRAPH_SUCCESS);
ret = pass_.Run(sub_graph_);
EXPECT_EQ(ret, GRAPH_SUCCESS);
ret = pass_.Run(graph_);
EXPECT_EQ(ret, GRAPH_SUCCESS);
}

} // namespace
} // namespace ge

Loading…
Cancel
Save