From 1571c9b7315035141d474ae5c4ad894844b6de33 Mon Sep 17 00:00:00 2001 From: zhangxiaokun Date: Thu, 10 Jun 2021 20:30:26 +0800 Subject: [PATCH 1/4] Fix dynamic shape partition --- ge/graph/partition/dynamic_shape_partition.cc | 18 ++++++- ge/graph/partition/dynamic_shape_partition.h | 2 +- .../passes/mark_force_unknown_for_cond_pass.cc | 34 ++++++------- ge/graph/passes/mark_graph_unknown_status_pass.cc | 6 +++ ge/graph/passes/next_iteration_pass.cc | 10 +++- ge/hybrid/executor/node_state.cc | 57 +++++++++++++++++++--- ge/hybrid/executor/node_state.h | 4 ++ ge/hybrid/executor/subgraph_context.cc | 2 +- ge/hybrid/executor/subgraph_context.h | 4 +- ge/hybrid/executor/subgraph_executor.cc | 18 ++----- ge/hybrid/executor/subgraph_executor.h | 1 - ge/hybrid/model/node_item.cc | 5 +- ge/hybrid/model/node_item.h | 4 +- ge/hybrid/node_executor/aicore/aicore_op_task.cc | 6 +-- ge/hybrid/node_executor/task_context.cc | 17 +++---- ge/hybrid/node_executor/task_context.h | 4 +- 16 files changed, 123 insertions(+), 69 deletions(-) diff --git a/ge/graph/partition/dynamic_shape_partition.cc b/ge/graph/partition/dynamic_shape_partition.cc index 055b2aa4..a01fa62f 100755 --- a/ge/graph/partition/dynamic_shape_partition.cc +++ b/ge/graph/partition/dynamic_shape_partition.cc @@ -364,6 +364,7 @@ static std::string ToString(const std::vector &clusters) { } void DynamicShapePartitioner::MergeClustersControlFlow() { + std::unordered_set all_merged_clusters; for (const auto &item : control_clusters_) { const auto &control_cluster = item.second; auto rit = control_cluster.rbegin(); @@ -373,17 +374,27 @@ void DynamicShapePartitioner::MergeClustersControlFlow() { } const auto &cluster = *rit; + if (all_merged_clusters.count(cluster) > 0) { + continue; + } + + bool is_unknown_cluster = cluster->IsUnknownShape(); for (++rit; rit != control_cluster.rend(); ++rit) { const auto &cluster_from = *rit; auto merged_clusters = cluster->MergeAllPathFrom(cluster_from); GELOGD("Merge all path cluster from %lu to %lu %s.", cluster_from->Id(), cluster->Id(), ToString(merged_clusters).c_str()); for (const auto &merged_cluster : merged_clusters) { + all_merged_clusters.emplace(merged_cluster); for (const auto &node : merged_cluster->Nodes()) { node_2_cluster_[node] = cluster; } } } + + if (!is_unknown_cluster && cluster->IsUnknownShape()) { + ordered_cluster_.push_back(cluster); + } } } @@ -703,7 +714,12 @@ void Cluster::Merge(ClusterPtr other) { if (other->min_ < min_) { min_ = other->min_; } -}; + + if (!IsUnknownShape() && other->IsUnknownShape()) { + type_ = UNKNOWN_SHAPE; + } +} + bool Cluster::TryMerge(ClusterPtr other) { std::queue forward_reached; forward_reached.push(other); diff --git a/ge/graph/partition/dynamic_shape_partition.h b/ge/graph/partition/dynamic_shape_partition.h index a17c4e4b..bd3b128f 100644 --- a/ge/graph/partition/dynamic_shape_partition.h +++ b/ge/graph/partition/dynamic_shape_partition.h @@ -161,7 +161,7 @@ class DynamicShapePartitioner { ge::ComputeGraphPtr root_graph_; // The original graph to partition std::unordered_map> node_2_cluster_; // Record nodes and the cluster it belongs to // V1 control flow cluster, need merge to one Graph. - std::unordered_map>> control_clusters_; + std::map>> control_clusters_; // topological sorted clusters, this field will change with the splitting. // When partitioning UNKNOWN_SHAPE cluster, it is a collection of all topological sorted UNKNOWN_SHAPE clusters // When partitioning KNOWN_SHAPE cluster, it is a collection of all topological sorted KNOWN_SHAPE clusters diff --git a/ge/graph/passes/mark_force_unknown_for_cond_pass.cc b/ge/graph/passes/mark_force_unknown_for_cond_pass.cc index 08b358ee..b9ee26db 100644 --- a/ge/graph/passes/mark_force_unknown_for_cond_pass.cc +++ b/ge/graph/passes/mark_force_unknown_for_cond_pass.cc @@ -143,26 +143,24 @@ void MarkForceUnknownForCondPass::MarkUnknownForSwitch(const std::mapGetOutputDesc(0))) { - int64_t group_index = op_desc1->GetId(); - GELOGI("Mark %s as unknown shape control flow, group index: %ld", op_desc1->GetName().c_str(), group_index); - MarkForceUnknownShape(op_node1, true, group_index); - for (const auto &n : it1->second) { - MarkForceUnknownShape(n, true, group_index); - } + int64_t group_index = op_desc1->GetId(); + GELOGI("Mark %s as unknown shape control flow, group index: %ld", op_desc1->GetName().c_str(), group_index); + SetControlFlowGroup(op_node1, group_index); + for (const auto &n : it1->second) { + SetControlFlowGroup(n, group_index); + } - for (auto it2 = switch_groups.begin(); it2 != switch_groups.end(); ++it2) { - const auto &op_node2 = it2->first; - const auto &op_desc2 = op_node2->GetOpDesc(); - if (op_desc2->HasAttr(ATTR_NAME_CONTROL_FLOW_GROUP)) { - continue; - } + for (auto it2 = switch_groups.begin(); it2 != switch_groups.end(); ++it2) { + const auto &op_node2 = it2->first; + const auto &op_desc2 = op_node2->GetOpDesc(); + if (op_desc2->HasAttr(ATTR_NAME_CONTROL_FLOW_GROUP)) { + continue; + } - if (std::any_of(it2->second.begin(), it2->second.end(), callback)) { - MarkForceUnknownShape(op_node2, true, group_index); - for (const auto &n : it2->second) { - MarkForceUnknownShape(n, true, group_index); - } + if (std::any_of(it2->second.begin(), it2->second.end(), callback)) { + SetControlFlowGroup(op_node2, group_index); + for (const auto &n : it2->second) { + SetControlFlowGroup(n, group_index); } } } diff --git a/ge/graph/passes/mark_graph_unknown_status_pass.cc b/ge/graph/passes/mark_graph_unknown_status_pass.cc index 2d7b179b..9e460fc7 100644 --- a/ge/graph/passes/mark_graph_unknown_status_pass.cc +++ b/ge/graph/passes/mark_graph_unknown_status_pass.cc @@ -40,6 +40,12 @@ Status MarkGraphUnknownStatusPass::Run(ComputeGraphPtr graph) { } } + const auto &node = graph->GetParentNode(); + if (!is_unknown_shape && node != nullptr && node->GetType() == PARTITIONEDCALL) { + GE_CHK_GRAPH_STATUS_RET(NodeUtils::GetNodeUnknownShapeStatus(*node, is_unknown_shape), + "[Get][ShapeStatus] of node[%s] failed!", node->GetName().c_str()); + } + for (const auto &node : graph->GetDirectNode()) { GELOGD("Set OwnerGraphIsUnknown attr to node[%s]", node->GetName().c_str()); (void)AttrUtils::SetBool(node->GetOpDesc(), kOwnerGraphIsUnknown, is_unknown_shape); diff --git a/ge/graph/passes/next_iteration_pass.cc b/ge/graph/passes/next_iteration_pass.cc index 67735b8b..fb8f8627 100644 --- a/ge/graph/passes/next_iteration_pass.cc +++ b/ge/graph/passes/next_iteration_pass.cc @@ -284,13 +284,21 @@ Status NextIterationPass::HandleWhileGroup(ComputeGraphPtr &graph) { /// @return void /// void NextIterationPass::HandleSwitchExitNodes(const LoopCondGroup &loop_group, int64_t group_index) { + std::string node_type; for (const auto &switch_node : loop_group.switch_nodes) { SetControlFlowGroup(switch_node, group_index); for (const auto &node : switch_node->GetOutDataNodes()) { - std::string node_type; (void)GetOriginalType(node, node_type); if (kExitOpTypes.count(node_type) > 0) { SetControlFlowGroup(node, group_index); + } else { + // For: Switch -> Cast -> Exit + for (const auto &n : node->GetOutDataNodes()) { + (void)GetOriginalType(n, node_type); + if (kExitOpTypes.count(node_type) > 0) { + SetControlFlowGroup(n, group_index); + } + } } } } diff --git a/ge/hybrid/executor/node_state.cc b/ge/hybrid/executor/node_state.cc index 313a2934..ddded35e 100644 --- a/ge/hybrid/executor/node_state.cc +++ b/ge/hybrid/executor/node_state.cc @@ -19,8 +19,9 @@ #include "framework/common/debug/log.h" #include "graph/compute_graph.h" #include "graph/utils/tensor_utils.h" -#include "hybrid_execution_context.h" -#include "subgraph_context.h" +#include "hybrid/executor/hybrid_execution_context.h" +#include "hybrid/executor/subgraph_context.h" +#include "hybrid/node_executor/task_context.h" #define INC_ITERATION_COUNT(iteration) \ do { \ @@ -258,6 +259,8 @@ ShapeFuture::ShapeFuture(NodeState *src_node, 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(); + auto unique_task_context = TaskContext::Create(this, subgraph_context_); + task_context_ = std::shared_ptr(unique_task_context.release()); } Status NodeState::AwaitInputTensors(GraphExecutionContext &context) const { @@ -314,15 +317,53 @@ std::shared_ptr NodeState::GetTaskContext() { return task_context_; } +void NodeState::SaveRootTensor(int input_idx, const TensorValue &tensor) { + if (node_item_->root_data_.count(input_idx) > 0) { + GELOGD("[%s] Save Const input tensor: %d", GetName().c_str(), input_idx); + root_tensor_value_[input_idx] = tensor; + } + + if (node_item_->enter_data_.count(input_idx) > 0) { + GELOGD("[%s] Save Enter input tensor: %d", GetName().c_str(), input_idx); + root_tensor_value_[input_idx] = tensor; + } +} + +void NodeState::UpdateRootTensor(int input_idx) { + const auto it = root_tensor_value_.find(input_idx); + if (it == root_tensor_value_.end()) { + GELOGW("[%s] Not found saved tensor: %d", GetName().c_str(), input_idx); + return; + } + + auto tensor = task_context_->MutableInput(input_idx); + if (tensor == nullptr) { + GELOGW("[%s] Not found input tensor: %d", GetName().c_str(), input_idx); + return; + } + + *tensor = it->second; + GELOGW("[%s] Update input tensor: %d", GetName().c_str(), input_idx); +} + void NodeState::ResetContext(uint64_t iteration) { switch_index_ = -1; subgraph_context_->ResetContext(node_item_->node); - if (iteration == 0) { - data_scheduled_ = static_cast(node_item_->root_data_.size()); - ctrl_scheduled_ = static_cast(node_item_->root_ctrl_.size()); - } else { - data_scheduled_ = static_cast(node_item_->root_data_.size() + node_item_->enter_data_.size()); - ctrl_scheduled_ = static_cast(node_item_->root_ctrl_.size() + node_item_->enter_ctrl_.size()); + auto unique_task_context = TaskContext::Create(this, subgraph_context_); + task_context_ = std::shared_ptr(unique_task_context.release()); + + data_scheduled_ = static_cast(node_item_->root_data_.size()); + ctrl_scheduled_ = static_cast(node_item_->root_ctrl_.size()); + for (auto item : node_item_->root_data_) { + UpdateRootTensor(item.first); + } + + if (iteration > 0) { + data_scheduled_ += static_cast(node_item_->enter_data_.size()); + ctrl_scheduled_ += static_cast(node_item_->enter_ctrl_.size()); + for (auto item : node_item_->enter_data_) { + UpdateRootTensor(item.first); + } } iteration_count_ = iteration; diff --git a/ge/hybrid/executor/node_state.h b/ge/hybrid/executor/node_state.h index 9dd29846..98dfbf0b 100644 --- a/ge/hybrid/executor/node_state.h +++ b/ge/hybrid/executor/node_state.h @@ -129,6 +129,8 @@ struct NodeState { void RunStreamActive(); void RunNextIteration(); + void SaveRootTensor(int input_idx, const TensorValue &tensor); + Status NodeScheduled(const std::function &ready) const; void SetScheduleFuture(std::future &&future); @@ -187,6 +189,7 @@ struct NodeState { void SetCtrlSchedule(const NodeState &node_state, const std::function &ready); void ResetContext(uint64_t iteration); void ScheduleContext(const NodeState &node_state); + void UpdateRootTensor(int input_idx); const NodeItem *node_item_ = nullptr; std::shared_ptr kernel_task_ = nullptr; @@ -199,6 +202,7 @@ struct NodeState { std::future schedule_future_; std::shared_ptr frame_state_; + std::map root_tensor_value_; uint64_t active_count_ = 0; uint64_t iteration_count_ = 0; uint32_t ctrl_scheduled_ = 0; diff --git a/ge/hybrid/executor/subgraph_context.cc b/ge/hybrid/executor/subgraph_context.cc index b6763ffd..41ada9af 100644 --- a/ge/hybrid/executor/subgraph_context.cc +++ b/ge/hybrid/executor/subgraph_context.cc @@ -19,7 +19,7 @@ namespace ge { namespace hybrid { -SubgraphContext::SubgraphContext(const GraphItem *graph_item, const GraphExecutionContext *execution_context) +SubgraphContext::SubgraphContext(const GraphItem *graph_item, GraphExecutionContext *execution_context) : graph_item_(graph_item), execution_context_(execution_context) { } diff --git a/ge/hybrid/executor/subgraph_context.h b/ge/hybrid/executor/subgraph_context.h index a43cd210..d11d00d7 100644 --- a/ge/hybrid/executor/subgraph_context.h +++ b/ge/hybrid/executor/subgraph_context.h @@ -30,7 +30,7 @@ namespace ge { namespace hybrid { class SubgraphContext { public: - explicit SubgraphContext(const GraphItem *graph_item, const GraphExecutionContext *execution_context); + explicit SubgraphContext(const GraphItem *graph_item, GraphExecutionContext *execution_context); ~SubgraphContext(); Status Init(); @@ -54,7 +54,7 @@ class SubgraphContext { FrameStatePtr GetOrCreateFrameState(const NodeItem &node_item); // no lock friend class TaskContext; const GraphItem *graph_item_; - const GraphExecutionContext *execution_context_; + GraphExecutionContext *execution_context_; mmRWLock_t rw_lock_; std::vector all_inputs_; std::vector all_outputs_; diff --git a/ge/hybrid/executor/subgraph_executor.cc b/ge/hybrid/executor/subgraph_executor.cc index 612e7565..7429acc5 100644 --- a/ge/hybrid/executor/subgraph_executor.cc +++ b/ge/hybrid/executor/subgraph_executor.cc @@ -175,16 +175,12 @@ Status SubgraphExecutor::ExecuteAsyncForKnownShape(const std::vectorSetKernelTask(node_item->kernel_task); - known_shape_task_context_ = TaskContext::Create(node_state.get(), context_, subgraph_context_.get()); - GE_CHECK_NOTNULL(known_shape_task_context_); - node_state->SetTaskContext(known_shape_task_context_); - std::function callback; GE_CHK_STATUS_RET_NOLOG(InitCallback(node_state.get(), callback)); - HYBRID_CHK_STATUS_RET(ExecutionEngine::ExecuteAsync(*node_state, known_shape_task_context_, *context_, callback), + HYBRID_CHK_STATUS_RET(ExecutionEngine::ExecuteAsync(*node_state, node_state->GetTaskContext(), *context_, callback), "[%s] Failed to execute node [%s] for known subgraph.", graph_item_->GetName().c_str(), - known_shape_task_context_->GetNodeName()); + node_state->GetName().c_str()); GELOGD("[%s] Done execute non-dynamic subgraph successfully.", graph_item_->GetName().c_str()); return SUCCESS; @@ -271,16 +267,12 @@ Status SubgraphExecutor::PrepareNode(const NodeItem &node_item, int group) { } else { node_state->SetKernelTask(node_item.kernel_task); } - auto unique_task_context = TaskContext::Create(node_state.get(), context_, subgraph_context_.get()); - GE_CHECK_NOTNULL(unique_task_context); const auto &task = node_state->GetKernelTask(); if (task == nullptr) { GELOGE(INTERNAL_ERROR, "[Get][KernelTask] failed for[%s], NodeTask is null.", node_state->GetName().c_str()); REPORT_CALL_ERROR("E19999", "GetKernelTask failed for %s, nodetask is null.", node_state->GetName().c_str()); return INTERNAL_ERROR; } - auto shared_task_context = std::shared_ptr(unique_task_context.release()); - node_state->SetTaskContext(shared_task_context); GE_CHK_STATUS_RET_NOLOG(NodeEnqueue(p_node_state)); return AfterPrepared(p_node_state); } @@ -480,19 +472,15 @@ Status SubgraphExecutor::PrepareForExecution(GraphExecutionContext *ctx, NodeSta } else { node_state.SetKernelTask(node_item.kernel_task); } - auto unique_task_context = TaskContext::Create(&node_state, context_, subgraph_context_.get()); - GE_CHECK_NOTNULL(unique_task_context); const auto &task = node_state.GetKernelTask(); if (task == nullptr) { GELOGE(INTERNAL_ERROR, "[Invoke][GetKernelTask] failed for[%s], NodeTask is null.", node_state.GetName().c_str()); REPORT_CALL_ERROR("E19999", "invoke GetKernelTask failed for %s, NodeTask is null.", node_state.GetName().c_str()); return INTERNAL_ERROR; } - auto shared_task_context = std::shared_ptr(unique_task_context.release()); - node_state.SetTaskContext(shared_task_context); GE_CHK_RT_RET(rtCtxSetCurrent(ctx->rt_context)); RECORD_COMPILE_EVENT(ctx, node_item.NodeName().c_str(), "[UpdateTilingData] start"); - GE_CHK_STATUS_RET_NOLOG(task->UpdateTilingData(*shared_task_context)); // update op_desc before alloc ws + GE_CHK_STATUS_RET_NOLOG(task->UpdateTilingData(*node_state.GetTaskContext())); // update op_desc before alloc ws RECORD_COMPILE_EVENT(ctx, node_item.NodeName().c_str(), "[UpdateTilingData] end"); return SUCCESS; } diff --git a/ge/hybrid/executor/subgraph_executor.h b/ge/hybrid/executor/subgraph_executor.h index 758bf426..e4c0debe 100644 --- a/ge/hybrid/executor/subgraph_executor.h +++ b/ge/hybrid/executor/subgraph_executor.h @@ -125,7 +125,6 @@ class SubgraphExecutor { ThreadPool pre_run_pool_; BlockingQueue ready_queue_; std::unique_ptr shape_inference_engine_; - std::shared_ptr known_shape_task_context_; std::mutex mu_; // Guard for prepare_queues_. std::map> prepare_queues_; diff --git a/ge/hybrid/model/node_item.cc b/ge/hybrid/model/node_item.cc index b339e630..cef06fc6 100644 --- a/ge/hybrid/model/node_item.cc +++ b/ge/hybrid/model/node_item.cc @@ -398,12 +398,11 @@ void NodeItem::SetDataSend(NodeItem *node_item, int anchor_index) { data_send_.emplace(node_item); node_item->data_recv_[this] = anchor_index; if (is_root_node_) { - node_item->root_data_.emplace(this); + node_item->root_data_[anchor_index] = this; } // If Enter feed Not Merge, take as root Node. if (IsEnterOp() && (node_item->node_type != STREAMMERGE)) { - node_item->enter_data_.emplace(this); - node_item->enter_inside_.emplace(anchor_index); + node_item->enter_data_[anchor_index] = this; } GELOGI("Node[%s] will control node[%s]", NodeName().c_str(), node_item->NodeName().c_str()); } diff --git a/ge/hybrid/model/node_item.h b/ge/hybrid/model/node_item.h index 8de15952..ae0b1b47 100644 --- a/ge/hybrid/model/node_item.h +++ b/ge/hybrid/model/node_item.h @@ -148,9 +148,9 @@ struct NodeItem { int64_t frame_index_ = -1; int64_t parent_frame_ = -1; std::set root_ctrl_; // Recv ctrl from root node - std::set root_data_; // Recv data from root node + std::map root_data_; // Recv data from root node std::set enter_ctrl_; // Recv ctrl from Enter node - std::set enter_data_; // Recv data from Enter node + std::map enter_data_; // Recv data from Enter node std::set data_send_; // Send data notify to std::map data_recv_; // Recv data notify from std::set ctrl_send_; // Send ctrl notify to diff --git a/ge/hybrid/node_executor/aicore/aicore_op_task.cc b/ge/hybrid/node_executor/aicore/aicore_op_task.cc index 8cd24bd1..5ed57621 100644 --- a/ge/hybrid/node_executor/aicore/aicore_op_task.cc +++ b/ge/hybrid/node_executor/aicore/aicore_op_task.cc @@ -306,7 +306,7 @@ Status AiCoreOpTask::InitWithKernelDefWithHandle(const OpDesc &op_desc, const do } Status AiCoreOpTask::InitWithTaskDef(const OpDesc &op_desc, const domi::TaskDef &task_def) { - + auto rt_ret = ValidateTaskDef(task_def); if (rt_ret != SUCCESS) { REPORT_CALL_ERROR("E19999", "op:%s(op_type:%s) failed to validate task def:%s", @@ -315,7 +315,7 @@ Status AiCoreOpTask::InitWithTaskDef(const OpDesc &op_desc, const domi::TaskDef op_desc.GetName().c_str(), op_desc.GetType().c_str(), task_def.DebugString().c_str()); return rt_ret; } - + if (task_def.type() != RT_MODEL_TASK_ALL_KERNEL) { GE_CHK_STATUS_RET(InitWithKernelDef(op_desc, task_def)); } else { @@ -474,7 +474,7 @@ Status AiCoreOpTask::UpdateArgs(TaskContext &task_context) { if (task_context.IsTraceEnabled()) { for (int i = 0; i < index; ++i) { - GELOGD("[%s] Arg[%d] = %lu", stub_name_.c_str(), i, arg_base_[i]); + GELOGD("[%s] Arg[%d] = %p", stub_name_.c_str(), i, arg_base_[i]); } } diff --git a/ge/hybrid/node_executor/task_context.cc b/ge/hybrid/node_executor/task_context.cc index 14eb1222..417fcd96 100644 --- a/ge/hybrid/node_executor/task_context.cc +++ b/ge/hybrid/node_executor/task_context.cc @@ -52,9 +52,7 @@ void TaskContext::ReleaseWorkspace() { } } -std::unique_ptr TaskContext::Create(NodeState *node_state, - GraphExecutionContext *execution_context, - SubgraphContext *subgraph_context) { +std::unique_ptr TaskContext::Create(NodeState *node_state, SubgraphContext *subgraph_context) { const NodeItem &node_item = *node_state->GetNodeItem(); GELOGI("[%s] To create task context, input start = %d, num_inputs = %d, output start = %d, num_outputs = %d.", node_item.NodeName().c_str(), @@ -75,7 +73,7 @@ std::unique_ptr TaskContext::Create(NodeState *node_state, } auto task_context = std::unique_ptr( - new(std::nothrow)TaskContext(execution_context, node_state, subgraph_context)); + new(std::nothrow)TaskContext(subgraph_context->execution_context_, node_state, subgraph_context)); if (task_context == nullptr) { REPORT_CALL_ERROR("E19999", "Create TaskContext failed for [%s].", node_item.NodeName().c_str()); GELOGE(MEMALLOC_FAILED, "[Create][TaskContext] failed for [%s].", node_item.NodeName().c_str()); @@ -85,7 +83,7 @@ std::unique_ptr TaskContext::Create(NodeState *node_state, 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->iteration_ = subgraph_context->execution_context_->iteration; return task_context; } @@ -460,6 +458,10 @@ Status TaskContext::PropagateOutputs() { subgraph_context_->all_inputs_[input_offset].SetName( node_item_->NodeName() + "_in_" + std::to_string(dst_input_idx)); } + + auto dst_node_state = subgraph_context_->GetOrCreateNodeState(dst_node_item); + GE_CHECK_NOTNULL(dst_node_state); + dst_node_state->SaveRootTensor(dst_input_idx, *tensor); } } (void)guard; @@ -489,11 +491,6 @@ void TaskContext::ReleaseInputsAndOutputs() { } void TaskContext::ReleaseInput(int index) { - if (node_item_->enter_inside_.count(index) > 0) { - GELOGD("[%s] Tensor of input[%d] is enter, keep it", GetNodeName(), index); - return; - } - auto input_tensor = MutableInput(index); if (input_tensor != nullptr) { input_tensor->Destroy(); diff --git a/ge/hybrid/node_executor/task_context.h b/ge/hybrid/node_executor/task_context.h index ba4c62e6..c96e194e 100644 --- a/ge/hybrid/node_executor/task_context.h +++ b/ge/hybrid/node_executor/task_context.h @@ -36,9 +36,7 @@ class SubgraphContext; class TaskContext { public: - static std::unique_ptr Create(NodeState *node_state, - GraphExecutionContext *execution_context, - SubgraphContext *subgraph_context); + static std::unique_ptr Create(NodeState *node_state, SubgraphContext *subgraph_context); ~TaskContext(); From 747ae4bbe11814505d8c0ce840c9fdfa4a26d82a Mon Sep 17 00:00:00 2001 From: zhangxiaokun Date: Thu, 10 Jun 2021 20:46:28 +0800 Subject: [PATCH 2/4] Fix dynamic shape partition --- ge/graph/partition/dynamic_shape_partition.cc | 5 +++++ ge/hybrid/executor/node_state.cc | 10 +++++----- ge/hybrid/executor/node_state.h | 2 +- ge/hybrid/model/node_item.h | 1 - ge/hybrid/node_executor/aicore/aicore_op_task.cc | 6 +++--- 5 files changed, 14 insertions(+), 10 deletions(-) diff --git a/ge/graph/partition/dynamic_shape_partition.cc b/ge/graph/partition/dynamic_shape_partition.cc index a01fa62f..1db47498 100755 --- a/ge/graph/partition/dynamic_shape_partition.cc +++ b/ge/graph/partition/dynamic_shape_partition.cc @@ -381,6 +381,10 @@ void DynamicShapePartitioner::MergeClustersControlFlow() { bool is_unknown_cluster = cluster->IsUnknownShape(); for (++rit; rit != control_cluster.rend(); ++rit) { const auto &cluster_from = *rit; + if (all_merged_clusters.count(cluster_from) > 0) { + continue; + } + auto merged_clusters = cluster->MergeAllPathFrom(cluster_from); GELOGD("Merge all path cluster from %lu to %lu %s.", cluster_from->Id(), cluster->Id(), ToString(merged_clusters).c_str()); @@ -393,6 +397,7 @@ void DynamicShapePartitioner::MergeClustersControlFlow() { } if (!is_unknown_cluster && cluster->IsUnknownShape()) { + GELOGD("Add to ordered cluster: %s", cluster->DebugString().c_str()); ordered_cluster_.push_back(cluster); } } diff --git a/ge/hybrid/executor/node_state.cc b/ge/hybrid/executor/node_state.cc index ddded35e..c2760f04 100644 --- a/ge/hybrid/executor/node_state.cc +++ b/ge/hybrid/executor/node_state.cc @@ -320,18 +320,18 @@ std::shared_ptr NodeState::GetTaskContext() { void NodeState::SaveRootTensor(int input_idx, const TensorValue &tensor) { if (node_item_->root_data_.count(input_idx) > 0) { GELOGD("[%s] Save Const input tensor: %d", GetName().c_str(), input_idx); - root_tensor_value_[input_idx] = tensor; + root_tensor_values_[input_idx] = tensor; } if (node_item_->enter_data_.count(input_idx) > 0) { GELOGD("[%s] Save Enter input tensor: %d", GetName().c_str(), input_idx); - root_tensor_value_[input_idx] = tensor; + root_tensor_values_[input_idx] = tensor; } } void NodeState::UpdateRootTensor(int input_idx) { - const auto it = root_tensor_value_.find(input_idx); - if (it == root_tensor_value_.end()) { + const auto it = root_tensor_values_.find(input_idx); + if (it == root_tensor_values_.end()) { GELOGW("[%s] Not found saved tensor: %d", GetName().c_str(), input_idx); return; } @@ -343,7 +343,7 @@ void NodeState::UpdateRootTensor(int input_idx) { } *tensor = it->second; - GELOGW("[%s] Update input tensor: %d", GetName().c_str(), input_idx); + GELOGD("[%s] Update input tensor: %d", GetName().c_str(), input_idx); } void NodeState::ResetContext(uint64_t iteration) { diff --git a/ge/hybrid/executor/node_state.h b/ge/hybrid/executor/node_state.h index 98dfbf0b..c6468bbd 100644 --- a/ge/hybrid/executor/node_state.h +++ b/ge/hybrid/executor/node_state.h @@ -202,7 +202,7 @@ struct NodeState { std::future schedule_future_; std::shared_ptr frame_state_; - std::map root_tensor_value_; + std::map root_tensor_values_; uint64_t active_count_ = 0; uint64_t iteration_count_ = 0; uint32_t ctrl_scheduled_ = 0; diff --git a/ge/hybrid/model/node_item.h b/ge/hybrid/model/node_item.h index ae0b1b47..ec66f094 100644 --- a/ge/hybrid/model/node_item.h +++ b/ge/hybrid/model/node_item.h @@ -156,7 +156,6 @@ struct NodeItem { std::set ctrl_send_; // Send ctrl notify to std::set ctrl_recv_; // Recv ctrl notify from std::vector> switch_groups_; // Send ctrl notify to - std::set enter_inside_; // Enter feed loop inside Node, Not cross Merge. std::shared_ptr kernel_task; std::unique_ptr fused_subgraph; diff --git a/ge/hybrid/node_executor/aicore/aicore_op_task.cc b/ge/hybrid/node_executor/aicore/aicore_op_task.cc index 5ed57621..8cd24bd1 100644 --- a/ge/hybrid/node_executor/aicore/aicore_op_task.cc +++ b/ge/hybrid/node_executor/aicore/aicore_op_task.cc @@ -306,7 +306,7 @@ Status AiCoreOpTask::InitWithKernelDefWithHandle(const OpDesc &op_desc, const do } Status AiCoreOpTask::InitWithTaskDef(const OpDesc &op_desc, const domi::TaskDef &task_def) { - + auto rt_ret = ValidateTaskDef(task_def); if (rt_ret != SUCCESS) { REPORT_CALL_ERROR("E19999", "op:%s(op_type:%s) failed to validate task def:%s", @@ -315,7 +315,7 @@ Status AiCoreOpTask::InitWithTaskDef(const OpDesc &op_desc, const domi::TaskDef op_desc.GetName().c_str(), op_desc.GetType().c_str(), task_def.DebugString().c_str()); return rt_ret; } - + if (task_def.type() != RT_MODEL_TASK_ALL_KERNEL) { GE_CHK_STATUS_RET(InitWithKernelDef(op_desc, task_def)); } else { @@ -474,7 +474,7 @@ Status AiCoreOpTask::UpdateArgs(TaskContext &task_context) { if (task_context.IsTraceEnabled()) { for (int i = 0; i < index; ++i) { - GELOGD("[%s] Arg[%d] = %p", stub_name_.c_str(), i, arg_base_[i]); + GELOGD("[%s] Arg[%d] = %lu", stub_name_.c_str(), i, arg_base_[i]); } } From fbc4adf527f9555d8687ca9987342c766f82017e Mon Sep 17 00:00:00 2001 From: zhangxiaokun Date: Thu, 10 Jun 2021 21:01:59 +0800 Subject: [PATCH 3/4] Fix ut --- .../executor/worker/execution_engine_unittest.cc | 8 +---- tests/ut/ge/hybrid/ge_hybrid_unittest.cc | 8 ++--- .../ge_local/ge_local_node_executor_unittest.cc | 5 --- .../hccl/hccl_node_executor_unittest.cc | 12 +------ .../node_executor/rts/rts_node_task_unittest.cc | 40 ---------------------- 5 files changed, 5 insertions(+), 68 deletions(-) diff --git a/tests/ut/ge/hybrid/executor/worker/execution_engine_unittest.cc b/tests/ut/ge/hybrid/executor/worker/execution_engine_unittest.cc index 07022230..e0ccbfa5 100644 --- a/tests/ut/ge/hybrid/executor/worker/execution_engine_unittest.cc +++ b/tests/ut/ge/hybrid/executor/worker/execution_engine_unittest.cc @@ -84,9 +84,6 @@ TEST_F(UtestExecutionEngine, ExecuteAsync_without_kernel_task) { SubgraphContext subgraph_context(nullptr, &execution_context); NodeState node_state(*node_item, &subgraph_context); - auto task_context = TaskContext::Create(&node_state, &execution_context, &subgraph_context); - auto shared_task_context = std::shared_ptr(task_context.release()); - node_state.SetTaskContext(shared_task_context); ExecutionEngine execution_engine; ASSERT_TRUE(node_state.GetTaskContext() != nullptr); @@ -119,14 +116,11 @@ TEST_F(UtestExecutionEngine, ExecuteAsync_without_callback_and_kernel_task) { SubgraphContext subgraph_context(nullptr, &execution_context); NodeState node_state(*node_item, &subgraph_context); - auto task_context = TaskContext::Create(&node_state, &execution_context, &subgraph_context); uint32_t task_id = 0; uint32_t stream_id = 1; std::string task_type = "rts"; uint32_t block_dim = 0; - task_context->SaveProfilingTaskDescInfo(task_id, stream_id, task_type, block_dim); - auto shared_task_context = std::shared_ptr(task_context.release()); - node_state.SetTaskContext(shared_task_context); + node_state.GetTaskContext()->SaveProfilingTaskDescInfo(task_id, stream_id, task_type, block_dim); ExecutionEngine execution_engine; ASSERT_TRUE(node_state.GetTaskContext() != nullptr); diff --git a/tests/ut/ge/hybrid/ge_hybrid_unittest.cc b/tests/ut/ge/hybrid/ge_hybrid_unittest.cc index 7a2a5dfe..ea93949d 100644 --- a/tests/ut/ge/hybrid/ge_hybrid_unittest.cc +++ b/tests/ut/ge/hybrid/ge_hybrid_unittest.cc @@ -160,10 +160,8 @@ TEST_F(UtestGeHybrid, task_update_tiling_info) { GraphExecutionContext execution_context; SubgraphContext subgraph_context(nullptr, &execution_context); NodeState node_state(*node_item, &subgraph_context); - auto task_context = TaskContext::Create(&node_state, &execution_context, &subgraph_context); - ASSERT_TRUE(task_context != nullptr); ASSERT_EQ(aicore_task->InitTilingInfo(*op_desc), SUCCESS); - ASSERT_EQ(aicore_task->UpdateTilingInfo(*task_context), SUCCESS); + ASSERT_EQ(aicore_task->UpdateTilingInfo(*node_state.GetTaskContext()), SUCCESS); } TEST_F(UtestGeHybrid, index_taskdefs_failed) { @@ -481,7 +479,7 @@ TEST_F(UtestGeHybrid, TestTaskContext) { subgraph_context.all_outputs_.resize(1); NodeState node_state(*node_item, &subgraph_context); - auto task_context = TaskContext::Create(&node_state, &execution_context, &subgraph_context); + auto task_context = node_state.GetTaskContext(); ASSERT_TRUE(task_context != nullptr); auto desc = task_context->MutableInputDesc(2); ASSERT_TRUE(desc == nullptr); @@ -526,7 +524,7 @@ TEST_F(UtestGeHybrid, hybrid_model_executor_update_args) { subgraph_context.all_outputs_.resize(1); NodeState node_state(*node_item, &subgraph_context); - auto task_context = TaskContext::Create(&node_state, &execution_context, &subgraph_context); + auto task_context = node_state.GetTaskContext(); int32_t buffer[1]; aicore_task->tiling_buffer_ = TensorBuffer::Create(buffer, sizeof(buffer)); diff --git a/tests/ut/ge/hybrid/node_executor/ge_local/ge_local_node_executor_unittest.cc b/tests/ut/ge/hybrid/node_executor/ge_local/ge_local_node_executor_unittest.cc index a7a407a4..e4d211f9 100644 --- a/tests/ut/ge/hybrid/node_executor/ge_local/ge_local_node_executor_unittest.cc +++ b/tests/ut/ge/hybrid/node_executor/ge_local/ge_local_node_executor_unittest.cc @@ -97,11 +97,6 @@ TEST_F(UtestGeLocalNodeExecutor, test_no_op_task) { auto node_state = subgraph_context.GetOrCreateNodeState(node_item); ASSERT_NE(node_state, nullptr); - auto unique_task_context = TaskContext::Create(node_state.get(), &graph_context, &subgraph_context); - ASSERT_NE(unique_task_context, nullptr); - auto shared_task_context = std::shared_ptr(unique_task_context.release()); - node_state->SetTaskContext(shared_task_context); - NodeTaskPtr task = nullptr; GeLocalNodeExecutor node_executor; ASSERT_EQ(node_executor.LoadTask(hybrid_model, node, task), SUCCESS); diff --git a/tests/ut/ge/hybrid/node_executor/hccl/hccl_node_executor_unittest.cc b/tests/ut/ge/hybrid/node_executor/hccl/hccl_node_executor_unittest.cc index afaf067e..2b23a4c1 100644 --- a/tests/ut/ge/hybrid/node_executor/hccl/hccl_node_executor_unittest.cc +++ b/tests/ut/ge/hybrid/node_executor/hccl/hccl_node_executor_unittest.cc @@ -94,7 +94,7 @@ TEST_F(UtestHcclNodeExecutor, test_rdmatask_extract_tensor) { tensor.SetData(data); ctx->SetTensor(1, 0, tensor.Clone()); - auto unique_task_context = TaskContext::Create(node_state.get(), &graph_context, &subgraph_context); + auto unique_task_context = node_state->GetTaskContext(); vector addr_infos; shared_ptr task = MakeShared(); task->remote_index_ = {1, 0}; @@ -140,11 +140,6 @@ TEST_F(UtestHcclNodeExecutor, gatheralltoallv_execute) { auto node_state = subgraph_context.GetOrCreateNodeState(node_item); ASSERT_NE(node_state, nullptr); - auto unique_task_context = TaskContext::Create(node_state.get(), &graph_context, &subgraph_context); - ASSERT_NE(unique_task_context, nullptr); - auto shared_task_context = std::shared_ptr(unique_task_context.release()); - node_state->SetTaskContext(shared_task_context); - for (int i=0; i<4; ++i) { uint64_t value_0 = 512; TensorValue in_tensor0(&value_0, sizeof(value_0)); @@ -206,11 +201,6 @@ TEST_F(UtestHcclNodeExecutor, alltoallv_execute) { auto node_state = subgraph_context.GetOrCreateNodeState(node_item); ASSERT_NE(node_state, nullptr); - auto unique_task_context = TaskContext::Create(node_state.get(), &graph_context, &subgraph_context); - ASSERT_NE(unique_task_context, nullptr); - auto shared_task_context = std::shared_ptr(unique_task_context.release()); - node_state->SetTaskContext(shared_task_context); - for (int i=0; i<5; ++i) { uint64_t value_0 = 512; TensorValue in_tensor0(&value_0, sizeof(value_0)); diff --git a/tests/ut/ge/hybrid/node_executor/rts/rts_node_task_unittest.cc b/tests/ut/ge/hybrid/node_executor/rts/rts_node_task_unittest.cc index 44b2f37f..109e5192 100644 --- a/tests/ut/ge/hybrid/node_executor/rts/rts_node_task_unittest.cc +++ b/tests/ut/ge/hybrid/node_executor/rts/rts_node_task_unittest.cc @@ -96,11 +96,6 @@ TEST_F(UtestRtsNodeTask, test_stream_switch_task) { auto node_state = subgraph_context.GetOrCreateNodeState(node_item); ASSERT_NE(node_state, nullptr); - auto unique_task_context = TaskContext::Create(node_state.get(), &graph_context, &subgraph_context); - ASSERT_NE(unique_task_context, nullptr); - auto shared_task_context = std::shared_ptr(unique_task_context.release()); - node_state->SetTaskContext(shared_task_context); - uint64_t value_0 = 110; uint64_t value_1 = 120; TensorValue in_tensor0(&value_0, sizeof(value_0)); @@ -153,11 +148,6 @@ TEST_F(UtestRtsNodeTask, test_stream_active_task) { auto node_state = subgraph_context.GetOrCreateNodeState(node_item); ASSERT_NE(node_state, nullptr); - auto unique_task_context = TaskContext::Create(node_state.get(), &graph_context, &subgraph_context); - ASSERT_NE(unique_task_context, nullptr); - auto shared_task_context = std::shared_ptr(unique_task_context.release()); - node_state->SetTaskContext(shared_task_context); - NodeTaskPtr task = nullptr; RtsNodeExecutor node_executor; ASSERT_EQ(node_executor.LoadTask(hybrid_model, node, task), SUCCESS); @@ -203,11 +193,6 @@ TEST_F(UtestRtsNodeTask, test_stream_merge_task) { auto node_state = subgraph_context.GetOrCreateNodeState(node_item); ASSERT_NE(node_state, nullptr); - auto unique_task_context = TaskContext::Create(node_state.get(), &graph_context, &subgraph_context); - ASSERT_NE(unique_task_context, nullptr); - auto shared_task_context = std::shared_ptr(unique_task_context.release()); - node_state->SetTaskContext(shared_task_context); - uint64_t value_0 = 110; TensorValue in_tensor0(&value_0, sizeof(value_0)); subgraph_context.SetInput(*node_item, 0, in_tensor0); @@ -271,11 +256,6 @@ TEST_F(UtestRtsNodeTask, test_memcpy_async_task) { auto node_state = subgraph_context.GetOrCreateNodeState(node_item); ASSERT_NE(node_state, nullptr); - auto unique_task_context = TaskContext::Create(node_state.get(), &graph_context, &subgraph_context); - ASSERT_NE(unique_task_context, nullptr); - auto shared_task_context = std::shared_ptr(unique_task_context.release()); - node_state->SetTaskContext(shared_task_context); - uint64_t value_0 = 110; TensorValue in_tensor0(&value_0, sizeof(value_0)); subgraph_context.SetInput(*node_item, 0, in_tensor0); @@ -328,11 +308,6 @@ TEST_F(UtestRtsNodeTask, test_pass_through_task) { auto node_state = subgraph_context.GetOrCreateNodeState(node_item); ASSERT_NE(node_state, nullptr); - auto unique_task_context = TaskContext::Create(node_state.get(), &graph_context, &subgraph_context); - ASSERT_NE(unique_task_context, nullptr); - auto shared_task_context = std::shared_ptr(unique_task_context.release()); - node_state->SetTaskContext(shared_task_context); - uint64_t value_0 = 110; TensorValue in_tensor0(&value_0, sizeof(value_0)); subgraph_context.SetInput(*node_item, 0, in_tensor0); @@ -384,11 +359,6 @@ TEST_F(UtestRtsNodeTask, test_unsupport_label_set) { auto node_state = subgraph_context.GetOrCreateNodeState(node_item); ASSERT_NE(node_state, nullptr); - auto unique_task_context = TaskContext::Create(node_state.get(), &graph_context, &subgraph_context); - ASSERT_NE(unique_task_context, nullptr); - auto shared_task_context = std::shared_ptr(unique_task_context.release()); - node_state->SetTaskContext(shared_task_context); - NodeTaskPtr task = nullptr; RtsNodeExecutor node_executor; ASSERT_EQ(node_executor.LoadTask(hybrid_model, node, task), SUCCESS); @@ -428,11 +398,6 @@ TEST_F(UtestRtsNodeTask, test_unsupport_label_goto) { auto node_state = subgraph_context.GetOrCreateNodeState(node_item); ASSERT_NE(node_state, nullptr); - auto unique_task_context = TaskContext::Create(node_state.get(), &graph_context, &subgraph_context); - ASSERT_NE(unique_task_context, nullptr); - auto shared_task_context = std::shared_ptr(unique_task_context.release()); - node_state->SetTaskContext(shared_task_context); - NodeTaskPtr task = nullptr; RtsNodeExecutor node_executor; ASSERT_EQ(node_executor.LoadTask(hybrid_model, node, task), SUCCESS); @@ -472,11 +437,6 @@ TEST_F(UtestRtsNodeTask, test_unsupport_label_switch) { auto node_state = subgraph_context.GetOrCreateNodeState(node_item); ASSERT_NE(node_state, nullptr); - auto unique_task_context = TaskContext::Create(node_state.get(), &graph_context, &subgraph_context); - ASSERT_NE(unique_task_context, nullptr); - auto shared_task_context = std::shared_ptr(unique_task_context.release()); - node_state->SetTaskContext(shared_task_context); - NodeTaskPtr task = nullptr; RtsNodeExecutor node_executor; ASSERT_EQ(node_executor.LoadTask(hybrid_model, node, task), SUCCESS); From 34d47b8e9ad2e74aa46c1825c925959fdc3ee747 Mon Sep 17 00:00:00 2001 From: zhangxiaokun Date: Fri, 11 Jun 2021 17:20:28 +0800 Subject: [PATCH 4/4] Fix ut --- ge/graph/common/omg_util.cc | 15 ---------- ge/graph/common/omg_util.h | 9 ------ .../passes/mark_force_unknown_for_cond_pass.cc | 34 +++++----------------- ge/graph/passes/merge_to_stream_merge_pass.cc | 5 ++-- ge/graph/passes/switch_to_stream_switch_pass.cc | 16 +++++----- ge/hybrid/executor/node_state.cc | 10 +++---- ge/hybrid/executor/node_state.h | 4 +-- ge/hybrid/node_executor/task_context.cc | 2 +- 8 files changed, 25 insertions(+), 70 deletions(-) diff --git a/ge/graph/common/omg_util.cc b/ge/graph/common/omg_util.cc index 52e6cb9c..b2017e4d 100644 --- a/ge/graph/common/omg_util.cc +++ b/ge/graph/common/omg_util.cc @@ -275,21 +275,6 @@ bool IsUnknownShapeTensor(const GeTensorDesc &tensor_desc) { } /// -/// @brief Set Op _force_unknown_shape flag -/// @param [in] node -/// @param [in] force_unknown, set attribute if true -/// @param [in] group_index, condition group index of node. -/// @return -/// -void MarkForceUnknownShape(const NodePtr &node, bool force_unknown, int64_t group_index) { - if (!force_unknown) { - return; - } - - SetControlFlowGroup(node, group_index); -} - -/// /// @brief Set Op _control_flow_group flag /// @param [in] node /// @param [in] group, condition group index of node. diff --git a/ge/graph/common/omg_util.h b/ge/graph/common/omg_util.h index 148e4102..edaafa45 100644 --- a/ge/graph/common/omg_util.h +++ b/ge/graph/common/omg_util.h @@ -126,15 +126,6 @@ Status GetMemorySize(const NodePtr &node, int64_t &output_size); bool IsUnknownShapeTensor(const GeTensorDesc &tensor_desc); /// -/// @brief Set Op _force_unknown_shape flag -/// @param [in] node -/// @param [in] force_unknown, set attribute if true -/// @param [in] group_index, condition group index of node. -/// @return -/// -void MarkForceUnknownShape(const NodePtr &node, bool force_unknown, int64_t group_index); - -/// /// @brief Set Op _control_flow_group flag /// @param [in] node /// @param [in] group, condition group index of node. diff --git a/ge/graph/passes/mark_force_unknown_for_cond_pass.cc b/ge/graph/passes/mark_force_unknown_for_cond_pass.cc index b9ee26db..74babadc 100644 --- a/ge/graph/passes/mark_force_unknown_for_cond_pass.cc +++ b/ge/graph/passes/mark_force_unknown_for_cond_pass.cc @@ -132,38 +132,18 @@ void MarkForceUnknownForCondPass::MarkUnknownForSwitch(const NodePtr &node, std: /// @return /// void MarkForceUnknownForCondPass::MarkUnknownForSwitch(const std::map> &switch_groups) { - std::function callback = [](const NodePtr &n) { - return n->GetOpDesc()->HasAttr(ATTR_NAME_CONTROL_FLOW_GROUP); - }; - - for (auto it1 = switch_groups.begin(); it1 != switch_groups.end(); ++it1) { - const auto &op_node1 = it1->first; - const auto &op_desc1 = op_node1->GetOpDesc(); - if (op_desc1->HasAttr(ATTR_NAME_CONTROL_FLOW_GROUP)) { + for (auto it = switch_groups.begin(); it != switch_groups.end(); ++it) { + const auto &op_node = it->first; + const auto &op_desc = op_node->GetOpDesc(); + if (op_desc->HasAttr(ATTR_NAME_CONTROL_FLOW_GROUP)) { continue; } - int64_t group_index = op_desc1->GetId(); - GELOGI("Mark %s as unknown shape control flow, group index: %ld", op_desc1->GetName().c_str(), group_index); - SetControlFlowGroup(op_node1, group_index); - for (const auto &n : it1->second) { + int64_t group_index = op_desc->GetId(); + SetControlFlowGroup(op_node, group_index); + for (const auto &n : it->second) { SetControlFlowGroup(n, group_index); } - - for (auto it2 = switch_groups.begin(); it2 != switch_groups.end(); ++it2) { - const auto &op_node2 = it2->first; - const auto &op_desc2 = op_node2->GetOpDesc(); - if (op_desc2->HasAttr(ATTR_NAME_CONTROL_FLOW_GROUP)) { - continue; - } - - if (std::any_of(it2->second.begin(), it2->second.end(), callback)) { - SetControlFlowGroup(op_node2, group_index); - for (const auto &n : it2->second) { - SetControlFlowGroup(n, group_index); - } - } - } } } } // namespace ge diff --git a/ge/graph/passes/merge_to_stream_merge_pass.cc b/ge/graph/passes/merge_to_stream_merge_pass.cc index 0b383911..dbcff620 100644 --- a/ge/graph/passes/merge_to_stream_merge_pass.cc +++ b/ge/graph/passes/merge_to_stream_merge_pass.cc @@ -89,8 +89,7 @@ Status MergeToStreamMergePass::AddActiveNodes(const ComputeGraphPtr &graph, cons REPORT_INNER_ERROR("E19999", "Param node is nullptr, check invalid"); return FAILED, "[Check][Param] Param of pre node is nullptr."); int64_t group_index = -1; - bool force_unknown = AttrUtils::GetInt(node->GetOpDesc(), ATTR_NAME_CONTROL_FLOW_GROUP, group_index); - MarkForceUnknownShape(node, force_unknown, group_index); + (void)AttrUtils::GetInt(node->GetOpDesc(), ATTR_NAME_CONTROL_FLOW_GROUP, group_index); for (const InDataAnchorPtr &in_data_anchor : node->GetAllInDataAnchors()) { OutDataAnchorPtr peer_out_anchor = in_data_anchor->GetPeerOutAnchor(); GE_IF_BOOL_EXEC(peer_out_anchor == nullptr, continue); @@ -109,7 +108,7 @@ Status MergeToStreamMergePass::AddActiveNodes(const ComputeGraphPtr &graph, cons GELOGE(FAILED, "[Set][ActiveLabelList] for node %s failed.", active_node->GetName().c_str()); return FAILED; } - MarkForceUnknownShape(active_node, force_unknown, group_index); + SetControlFlowGroup(active_node, group_index); } return SUCCESS; diff --git a/ge/graph/passes/switch_to_stream_switch_pass.cc b/ge/graph/passes/switch_to_stream_switch_pass.cc index e7743130..e4ab0111 100644 --- a/ge/graph/passes/switch_to_stream_switch_pass.cc +++ b/ge/graph/passes/switch_to_stream_switch_pass.cc @@ -395,8 +395,8 @@ NodePtr SwitchToStreamSwitchPass::CreateStreamSwitchNode(const ComputeGraphPtr & peer_cond_anchor->GetOwnerNode()->GetName().c_str(), stream_switch->GetName().c_str()); int64_t group_index = -1; - bool force_unknown = AttrUtils::GetInt(switch_node->GetOpDesc(), ATTR_NAME_CONTROL_FLOW_GROUP, group_index); - MarkForceUnknownShape(stream_switch, force_unknown, group_index); + (void)AttrUtils::GetInt(switch_node->GetOpDesc(), ATTR_NAME_CONTROL_FLOW_GROUP, group_index); + SetControlFlowGroup(stream_switch, group_index); return stream_switch; } @@ -491,8 +491,8 @@ int64_t SwitchToStreamSwitchPass::GetGroupId(const NodePtr &node) { Status SwitchToStreamSwitchPass::CombineSwitchNode(const ComputeGraphPtr &graph) { for (auto iter = cond_node_map_.begin(); iter != cond_node_map_.end(); ++iter) { for (auto group_iter = iter->second.begin(); group_iter != iter->second.end(); ++group_iter) { - std::list false_switch_list = group_iter->second[SWITCH_FALSE_OUTPUT]; - std::list true_switch_list = group_iter->second[SWITCH_TRUE_OUTPUT]; + const std::list &false_switch_list = group_iter->second[SWITCH_FALSE_OUTPUT]; + const std::list &true_switch_list = group_iter->second[SWITCH_TRUE_OUTPUT]; std::set same_cond_switch; same_cond_switch.insert(false_switch_list.begin(), false_switch_list.end()); same_cond_switch.insert(true_switch_list.begin(), true_switch_list.end()); @@ -524,13 +524,13 @@ Status SwitchToStreamSwitchPass::CombineSwitchNode(const ComputeGraphPtr &graph) std::function callback = [&group_index](const NodePtr &n) { return AttrUtils::GetInt(n->GetOpDesc(), ATTR_NAME_CONTROL_FLOW_GROUP, group_index); }; - bool is_unknown_shape = std::any_of(same_cond_switch.begin(), same_cond_switch.end(), callback); - MarkForceUnknownShape(active_node, is_unknown_shape, group_index); + (void)std::any_of(same_cond_switch.begin(), same_cond_switch.end(), callback); + SetControlFlowGroup(active_node, group_index); const std::string &cond_group = cond_node->GetName(); for (uint32_t i = 0; i < SWITCH_OUTPUT_NUM; ++i) { bool true_branch_flag = (i == SWITCH_TRUE_OUTPUT); - std::list &switch_list = (true_branch_flag ? true_switch_list : false_switch_list); + const std::list &switch_list = (true_branch_flag ? true_switch_list : false_switch_list); GE_IF_BOOL_EXEC(switch_list.empty(), continue); // select first stream_switch @@ -559,7 +559,7 @@ Status SwitchToStreamSwitchPass::CombineSwitchNode(const ComputeGraphPtr &graph) "[Add][Edge] between %s and %s failed.", cast_node->GetName().c_str(), stream_switch->GetName().c_str()); - MarkForceUnknownShape(stream_switch, is_unknown_shape, group_index); + SetControlFlowGroup(stream_switch, group_index); for (const NodePtr &node : switch_list) { GE_IF_BOOL_EXEC(node != stream_switch, { GE_CHK_STATUS(GraphUtils::RemoveEdge(peer_cond_anchor, node->GetInDataAnchor(0)), diff --git a/ge/hybrid/executor/node_state.cc b/ge/hybrid/executor/node_state.cc index c2760f04..42e08811 100644 --- a/ge/hybrid/executor/node_state.cc +++ b/ge/hybrid/executor/node_state.cc @@ -317,9 +317,9 @@ std::shared_ptr NodeState::GetTaskContext() { return task_context_; } -void NodeState::SaveRootTensor(int input_idx, const TensorValue &tensor) { +void NodeState::SavePersistTensor(int input_idx, const TensorValue &tensor) { if (node_item_->root_data_.count(input_idx) > 0) { - GELOGD("[%s] Save Const input tensor: %d", GetName().c_str(), input_idx); + GELOGD("[%s] Save Root input tensor: %d", GetName().c_str(), input_idx); root_tensor_values_[input_idx] = tensor; } @@ -329,7 +329,7 @@ void NodeState::SaveRootTensor(int input_idx, const TensorValue &tensor) { } } -void NodeState::UpdateRootTensor(int input_idx) { +void NodeState::UpdatePersistTensor(int input_idx) { const auto it = root_tensor_values_.find(input_idx); if (it == root_tensor_values_.end()) { GELOGW("[%s] Not found saved tensor: %d", GetName().c_str(), input_idx); @@ -355,14 +355,14 @@ void NodeState::ResetContext(uint64_t iteration) { data_scheduled_ = static_cast(node_item_->root_data_.size()); ctrl_scheduled_ = static_cast(node_item_->root_ctrl_.size()); for (auto item : node_item_->root_data_) { - UpdateRootTensor(item.first); + UpdatePersistTensor(item.first); } if (iteration > 0) { data_scheduled_ += static_cast(node_item_->enter_data_.size()); ctrl_scheduled_ += static_cast(node_item_->enter_ctrl_.size()); for (auto item : node_item_->enter_data_) { - UpdateRootTensor(item.first); + UpdatePersistTensor(item.first); } } diff --git a/ge/hybrid/executor/node_state.h b/ge/hybrid/executor/node_state.h index c6468bbd..72e2b90e 100644 --- a/ge/hybrid/executor/node_state.h +++ b/ge/hybrid/executor/node_state.h @@ -129,7 +129,7 @@ struct NodeState { void RunStreamActive(); void RunNextIteration(); - void SaveRootTensor(int input_idx, const TensorValue &tensor); + void SavePersistTensor(int input_idx, const TensorValue &tensor); Status NodeScheduled(const std::function &ready) const; @@ -189,7 +189,7 @@ struct NodeState { void SetCtrlSchedule(const NodeState &node_state, const std::function &ready); void ResetContext(uint64_t iteration); void ScheduleContext(const NodeState &node_state); - void UpdateRootTensor(int input_idx); + void UpdatePersistTensor(int input_idx); const NodeItem *node_item_ = nullptr; std::shared_ptr kernel_task_ = nullptr; diff --git a/ge/hybrid/node_executor/task_context.cc b/ge/hybrid/node_executor/task_context.cc index 417fcd96..fe580c1e 100644 --- a/ge/hybrid/node_executor/task_context.cc +++ b/ge/hybrid/node_executor/task_context.cc @@ -461,7 +461,7 @@ Status TaskContext::PropagateOutputs() { auto dst_node_state = subgraph_context_->GetOrCreateNodeState(dst_node_item); GE_CHECK_NOTNULL(dst_node_state); - dst_node_state->SaveRootTensor(dst_input_idx, *tensor); + dst_node_state->SavePersistTensor(dst_input_idx, *tensor); } } (void)guard;