未验证 提交 aba759ba 编写于 作者: L Leo Chen 提交者: GitHub

[Feature] Enhance inplace addto strategy for gradient accumulation in static graph (#27112)

* support use add instead of sum to do gradient accumulation

* add inplace addto pass

* add grad_add op and inplace addto pass

* remove debug code

* code refine

* fix bug when sereral sum ops inserts at same op_idx

* fix Flags type

* add addto attribute for conv3d

* fix ut

* code clean

* fix type
上级 669efb98
...@@ -74,6 +74,7 @@ set(SSA_GRAPH_EXECUTOR_DEPS graph framework_proto ...@@ -74,6 +74,7 @@ set(SSA_GRAPH_EXECUTOR_DEPS graph framework_proto
eager_deletion_pass eager_deletion_pass
buffer_shared_inplace_op_pass buffer_shared_inplace_op_pass
buffer_shared_cross_op_memory_reuse_pass buffer_shared_cross_op_memory_reuse_pass
inplace_addto_op_pass
set_reader_device_info_utils set_reader_device_info_utils
add_reader_dependency_pass) add_reader_dependency_pass)
cc_library(ssa_graph_executor SRCS ssa_graph_executor.cc DEPS ${SSA_GRAPH_EXECUTOR_DEPS}) cc_library(ssa_graph_executor SRCS ssa_graph_executor.cc DEPS ${SSA_GRAPH_EXECUTOR_DEPS})
......
...@@ -19,6 +19,7 @@ ...@@ -19,6 +19,7 @@
#include <unordered_set> #include <unordered_set>
#include <utility> #include <utility>
#include <vector> #include <vector>
#include "boost/optional.hpp" #include "boost/optional.hpp"
#include "paddle/fluid/framework/ir/pass_builder.h" #include "paddle/fluid/framework/ir/pass_builder.h"
#include "paddle/fluid/framework/program_desc.h" #include "paddle/fluid/framework/program_desc.h"
...@@ -119,6 +120,9 @@ struct BuildStrategy { ...@@ -119,6 +120,9 @@ struct BuildStrategy {
// Turn on inplace by default. // Turn on inplace by default.
bool enable_inplace_{true}; bool enable_inplace_{true};
// Turn off inplace addto by default.
bool enable_addto_{false};
// FIXME(zcd): is_distribution_ is a temporary field, because in pserver mode, // FIXME(zcd): is_distribution_ is a temporary field, because in pserver mode,
// num_trainers is 1, so the current fields of build_strategy doesn't tell if // num_trainers is 1, so the current fields of build_strategy doesn't tell if
// it's distributed model. // it's distributed model.
......
...@@ -12,6 +12,7 @@ ...@@ -12,6 +12,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
#include "paddle/fluid/framework/details/op_handle_base.h" #include "paddle/fluid/framework/details/op_handle_base.h"
#include <map> #include <map>
#include <unordered_set> #include <unordered_set>
...@@ -88,6 +89,12 @@ void OpHandleBase::Run(bool use_cuda) { ...@@ -88,6 +89,12 @@ void OpHandleBase::Run(bool use_cuda) {
PADDLE_ENFORCE(!use_cuda); PADDLE_ENFORCE(!use_cuda);
#endif #endif
// skip running current op, used with inplace_addto_op_pass
if (skip_running_) {
VLOG(4) << "skip running: " << Name();
return;
}
RunImpl(); RunImpl();
} }
......
...@@ -18,6 +18,7 @@ ...@@ -18,6 +18,7 @@
#include <unordered_map> #include <unordered_map>
#include <unordered_set> #include <unordered_set>
#include <vector> #include <vector>
#include "paddle/fluid/framework/details/var_handle.h" #include "paddle/fluid/framework/details/var_handle.h"
#include "paddle/fluid/framework/ir/node.h" #include "paddle/fluid/framework/ir/node.h"
#include "paddle/fluid/platform/device_context.h" #include "paddle/fluid/platform/device_context.h"
...@@ -52,6 +53,10 @@ class OpHandleBase { ...@@ -52,6 +53,10 @@ class OpHandleBase {
virtual Priority GetPriority() const { return kNormal; } virtual Priority GetPriority() const { return kNormal; }
virtual bool GetSkipRunning() const { return skip_running_; }
virtual void SetSkipRunning(bool skip_runing) { skip_running_ = skip_runing; }
virtual std::string Name() const = 0; virtual std::string Name() const = 0;
void Run(bool use_cuda); void Run(bool use_cuda);
...@@ -131,6 +136,7 @@ class OpHandleBase { ...@@ -131,6 +136,7 @@ class OpHandleBase {
std::map<platform::Place, platform::DeviceContext *> dev_ctxes_; std::map<platform::Place, platform::DeviceContext *> dev_ctxes_;
std::vector<Scope *> local_exec_scopes_; std::vector<Scope *> local_exec_scopes_;
bool skip_running_ = false;
#ifdef PADDLE_WITH_CUDA #ifdef PADDLE_WITH_CUDA
std::unordered_map<int, cudaEvent_t> events_; std::unordered_map<int, cudaEvent_t> events_;
......
...@@ -48,12 +48,13 @@ static inline Tensor *GetMutableTensorFromVar(Variable *var) { ...@@ -48,12 +48,13 @@ static inline Tensor *GetMutableTensorFromVar(Variable *var) {
ShareTensorBufferFunctor::ShareTensorBufferFunctor( ShareTensorBufferFunctor::ShareTensorBufferFunctor(
Scope *scope, size_t scope_idx, const std::string &op_type, Scope *scope, size_t scope_idx, const std::string &op_type,
const std::vector<const ir::MemOptVarInfo *> &in_var_infos, const std::vector<const ir::MemOptVarInfo *> &in_var_infos,
const std::vector<std::string> &out_var_names) const std::vector<std::string> &out_var_names, bool share_dims)
: scope_(scope), : scope_(scope),
scope_idx_(scope_idx), scope_idx_(scope_idx),
op_type_(op_type), op_type_(op_type),
in_var_infos_(in_var_infos), in_var_infos_(in_var_infos),
out_var_names_(out_var_names) { out_var_names_(out_var_names),
share_dims_(share_dims) {
PADDLE_ENFORCE_EQ(in_var_infos_.size(), out_var_names_.size(), PADDLE_ENFORCE_EQ(in_var_infos_.size(), out_var_names_.size(),
platform::errors::PreconditionNotMet( platform::errors::PreconditionNotMet(
"The number of input variables and output variables " "The number of input variables and output variables "
...@@ -151,6 +152,13 @@ void ShareTensorBufferFunctor::operator()(Scope *exec_scope) { ...@@ -151,6 +152,13 @@ void ShareTensorBufferFunctor::operator()(Scope *exec_scope) {
} else { } else {
out_tensor->ShareBufferWith(in_tensor); out_tensor->ShareBufferWith(in_tensor);
// NOTE(zhiqiu): In the case of inplace addto, if the operator of
// the in_out_vars is skipped during running, we should set the dims of
// output as the same as input.
if (share_dims_) {
out_tensor->Resize(in_tensor.dims());
}
VLOG(2) << "Share tensor buffer when running " << op_type_ << " : " VLOG(2) << "Share tensor buffer when running " << op_type_ << " : "
<< in_var_info->Name() << " -> " << out_var_names_[i]; << in_var_info->Name() << " -> " << out_var_names_[i];
} }
......
...@@ -19,6 +19,7 @@ ...@@ -19,6 +19,7 @@
#include <unordered_set> #include <unordered_set>
#include <utility> #include <utility>
#include <vector> #include <vector>
#include "paddle/fluid/framework/details/op_handle_base.h" #include "paddle/fluid/framework/details/op_handle_base.h"
#include "paddle/fluid/framework/ir/memory_optimize_pass/memory_optimization_var_info.h" #include "paddle/fluid/framework/ir/memory_optimize_pass/memory_optimization_var_info.h"
#include "paddle/fluid/framework/scope.h" #include "paddle/fluid/framework/scope.h"
...@@ -40,11 +41,13 @@ class ShareTensorBufferFunctor { ...@@ -40,11 +41,13 @@ class ShareTensorBufferFunctor {
ShareTensorBufferFunctor( ShareTensorBufferFunctor(
Scope *scope, size_t scope_idx, const std::string &op_type, Scope *scope, size_t scope_idx, const std::string &op_type,
const std::vector<const ir::MemOptVarInfo *> &in_var_infos, const std::vector<const ir::MemOptVarInfo *> &in_var_infos,
const std::vector<std::string> &out_var_names); const std::vector<std::string> &out_var_names, bool share_dims = false);
void AddReuseVarPair(const ir::MemOptVarInfo *in_var_info, void AddReuseVarPair(const ir::MemOptVarInfo *in_var_info,
const std::string &out_var_name); const std::string &out_var_name);
void SetShareDims(bool share_dims) { share_dims_ = share_dims; }
void operator()(Scope *exec_scope); void operator()(Scope *exec_scope);
std::unordered_map<std::string, std::string> ReusedVars() const; std::unordered_map<std::string, std::string> ReusedVars() const;
...@@ -66,6 +69,11 @@ class ShareTensorBufferFunctor { ...@@ -66,6 +69,11 @@ class ShareTensorBufferFunctor {
std::vector<std::string> out_var_names_; std::vector<std::string> out_var_names_;
std::vector<std::pair<const Variable *, Variable *>> in_out_vars_; std::vector<std::pair<const Variable *, Variable *>> in_out_vars_;
// NOTE(zhiqiu): In the case of inplace addto, if the operator of
// the in_out_vars is skipped during running, we should set the dims of output
// as the same as input.
bool share_dims_{false};
}; };
} // namespace details } // namespace details
......
...@@ -59,9 +59,10 @@ ComputationOpHandle *GetUniquePendingComputationOpHandle( ...@@ -59,9 +59,10 @@ ComputationOpHandle *GetUniquePendingComputationOpHandle(
ShareTensorBufferOpHandle::ShareTensorBufferOpHandle( ShareTensorBufferOpHandle::ShareTensorBufferOpHandle(
ir::Node *node, Scope *scope, size_t scope_idx, const std::string &op_type, ir::Node *node, Scope *scope, size_t scope_idx, const std::string &op_type,
const std::vector<const ir::MemOptVarInfo *> &in_var_infos, const std::vector<const ir::MemOptVarInfo *> &in_var_infos,
const std::vector<std::string> &out_var_names) const std::vector<std::string> &out_var_names, bool share_dims)
: OpHandleBase(node), : OpHandleBase(node),
functor_(scope, scope_idx, op_type, in_var_infos, out_var_names) {} functor_(scope, scope_idx, op_type, in_var_infos, out_var_names,
share_dims) {}
std::unordered_map<std::string, std::string> std::unordered_map<std::string, std::string>
ShareTensorBufferOpHandle::ReusedVars() const { ShareTensorBufferOpHandle::ReusedVars() const {
...@@ -73,6 +74,10 @@ void ShareTensorBufferOpHandle::AddReuseVarPair( ...@@ -73,6 +74,10 @@ void ShareTensorBufferOpHandle::AddReuseVarPair(
functor_.AddReuseVarPair(in_var_info, out_var_name); functor_.AddReuseVarPair(in_var_info, out_var_name);
} }
void ShareTensorBufferOpHandle::SetShareDims(bool share_dims) {
functor_.SetShareDims(share_dims);
}
void ShareTensorBufferOpHandle::InitCUDA() { void ShareTensorBufferOpHandle::InitCUDA() {
#ifdef PADDLE_WITH_CUDA #ifdef PADDLE_WITH_CUDA
int dev_id = int dev_id =
......
...@@ -17,6 +17,7 @@ ...@@ -17,6 +17,7 @@
#include <unordered_map> #include <unordered_map>
#include <utility> #include <utility>
#include <vector> #include <vector>
#include "paddle/fluid/framework/details/computation_op_handle.h" #include "paddle/fluid/framework/details/computation_op_handle.h"
#include "paddle/fluid/framework/details/op_handle_base.h" #include "paddle/fluid/framework/details/op_handle_base.h"
#include "paddle/fluid/framework/details/share_tensor_buffer_functor.h" #include "paddle/fluid/framework/details/share_tensor_buffer_functor.h"
...@@ -31,7 +32,7 @@ class ShareTensorBufferOpHandle : public OpHandleBase { ...@@ -31,7 +32,7 @@ class ShareTensorBufferOpHandle : public OpHandleBase {
ir::Node *node, Scope *scope, size_t scope_idx, ir::Node *node, Scope *scope, size_t scope_idx,
const std::string &op_type, const std::string &op_type,
const std::vector<const ir::MemOptVarInfo *> &in_vars_infos, const std::vector<const ir::MemOptVarInfo *> &in_vars_infos,
const std::vector<std::string> &out_var_names); const std::vector<std::string> &out_var_names, bool share_dims = false);
std::unordered_map<std::string, std::string> ReusedVars() const; std::unordered_map<std::string, std::string> ReusedVars() const;
...@@ -42,6 +43,8 @@ class ShareTensorBufferOpHandle : public OpHandleBase { ...@@ -42,6 +43,8 @@ class ShareTensorBufferOpHandle : public OpHandleBase {
void AddReuseVarPair(const ir::MemOptVarInfo *in_var_info, void AddReuseVarPair(const ir::MemOptVarInfo *in_var_info,
const std::string &out_var_name); const std::string &out_var_name);
void SetShareDims(bool share_dims);
const ShareTensorBufferFunctor &Functor() const { return functor_; } const ShareTensorBufferFunctor &Functor() const { return functor_; }
protected: protected:
......
...@@ -13,4 +13,6 @@ cc_library(memory_reuse_pass SRCS memory_reuse_pass.cc DEPS computation_op_handl ...@@ -13,4 +13,6 @@ cc_library(memory_reuse_pass SRCS memory_reuse_pass.cc DEPS computation_op_handl
cc_library(buffer_shared_inplace_op_pass SRCS buffer_shared_inplace_op_pass.cc DEPS memory_reuse_pass) cc_library(buffer_shared_inplace_op_pass SRCS buffer_shared_inplace_op_pass.cc DEPS memory_reuse_pass)
cc_library(buffer_shared_cross_op_memory_reuse_pass SRCS buffer_shared_cross_op_memory_reuse_pass.cc DEPS memory_reuse_pass) cc_library(buffer_shared_cross_op_memory_reuse_pass SRCS buffer_shared_cross_op_memory_reuse_pass.cc DEPS memory_reuse_pass)
cc_library(inplace_addto_op_pass SRCS inplace_addto_op_pass.cc DEPS memory_reuse_pass)
cc_test(test_reference_count_pass_last_lived_ops SRCS test_reference_count_pass_last_lived_ops.cc DEPS parallel_executor elementwise_mul_op elementwise_add_op scale_op) cc_test(test_reference_count_pass_last_lived_ops SRCS test_reference_count_pass_last_lived_ops.cc DEPS parallel_executor elementwise_mul_op elementwise_add_op scale_op)
...@@ -16,6 +16,7 @@ ...@@ -16,6 +16,7 @@
#include <unordered_map> #include <unordered_map>
#include <unordered_set> #include <unordered_set>
#include <vector> #include <vector>
#include "paddle/fluid/framework/details/computation_op_handle.h" #include "paddle/fluid/framework/details/computation_op_handle.h"
#include "paddle/fluid/framework/details/multi_devices_helper.h" #include "paddle/fluid/framework/details/multi_devices_helper.h"
#include "paddle/fluid/framework/details/share_tensor_buffer_op_handle.h" #include "paddle/fluid/framework/details/share_tensor_buffer_op_handle.h"
...@@ -141,11 +142,12 @@ void BufferSharedInplaceOpPass::Run(Graph *graph) const { ...@@ -141,11 +142,12 @@ void BufferSharedInplaceOpPass::Run(Graph *graph) const {
VLOG(4) << "Inplace performed in op " << op_type << ": " VLOG(4) << "Inplace performed in op " << op_type << ": "
<< in_var_handle_ptr->Name() << " -> " << in_var_handle_ptr->Name() << " -> "
<< out_var_handle_ptr->Name() << out_var_handle_ptr->Name()
<< ". Debug String is: " << op->GetOp()->DebugString(); << ". Debug String is: " << op->GetOp()->DebugString()
<< ". ReuseType: " << ReuseType();
} else { } else {
VLOG(3) << "Inplace failed in op " << op_type << ": " VLOG(3) << "Inplace failed in op " << op_type << ": "
<< in_var_handle_ptr->Name() << " -> " << in_var_handle_ptr->Name() << " -> "
<< out_var_handle_ptr->Name(); << out_var_handle_ptr->Name() << ". ReuseType: " << ReuseType();
} }
} }
} }
......
// Copyright (c) 2020 PaddlePaddle Authors. All Rights Reserved.
//
// 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 <string>
#include <unordered_map>
#include <unordered_set>
#include <vector>
#include "paddle/fluid/framework/details/computation_op_handle.h"
#include "paddle/fluid/framework/details/multi_devices_helper.h"
#include "paddle/fluid/framework/details/share_tensor_buffer_op_handle.h"
#include "paddle/fluid/framework/ir/memory_optimize_pass/memory_optimization_var_info.h"
#include "paddle/fluid/framework/ir/memory_optimize_pass/memory_reuse_pass.h"
#include "paddle/fluid/framework/ir/memory_optimize_pass/reference_count_pass_helper.h"
#include "paddle/fluid/framework/ir/pass.h"
namespace paddle {
namespace framework {
namespace ir {
class InplaceAddToOpPass : public MemoryReusePass {
protected:
std::string ReuseType() const override { return "inplace_addto"; }
void Run(Graph *graph) const override;
private:
// 1. Add last living op of in_var, add any last living op of out_var
// 2. Set reference count of in_var to be 2
void UpdateLastLiveOpOfVar(details::ComputationOpHandle *op,
details::VarHandle *in_var,
details::VarHandle *out_var) const override {
size_t scope_idx = op->GetScopeIdx();
auto *last_live_ops_of_vars_ =
&Get<std::vector<LastLiveOpsOfVars>>(kLastLiveOpsOfVars);
auto *var_infos_ = &(Get<MemOptVarInfoMapList>(kMemOptVarInfoMapList));
auto out_var_op_iter =
(*last_live_ops_of_vars_)[scope_idx].find(out_var->Name());
// In Reduce mode, some output variable(gradient of parameter) does not have
// last live ops
details::ComputationOpHandle *last_live_op_of_in_var = nullptr;
if (out_var_op_iter == (*last_live_ops_of_vars_)[scope_idx].end()) {
last_live_op_of_in_var = op;
} else {
PADDLE_ENFORCE_EQ(
out_var_op_iter->second.ops().empty(), false,
platform::errors::InvalidArgument(
"Var(%s)'s last live op should not empty.", out_var->Name()));
last_live_op_of_in_var = *(out_var_op_iter->second.ops().begin());
}
auto *last_live_ops_of_in_var =
(*last_live_ops_of_vars_)[scope_idx][in_var->Name()].mutable_ops();
// last_live_ops_of_in_var->clear();
last_live_ops_of_in_var->insert(last_live_op_of_in_var);
auto in_var_info_iter = (*var_infos_)[scope_idx].find(in_var->Name());
PADDLE_ENFORCE_NE(
in_var_info_iter, (*var_infos_)[scope_idx].end(),
platform::errors::NotFound("Cannot find variable %s.", in_var->Name()));
in_var_info_iter->second->SetRefCnt(2); // before inplace, it is 1
}
};
void InplaceAddToOpPass::Run(Graph *graph) const {
const auto &last_live_ops =
Get<std::vector<LastLiveOpsOfVars>>(kLastLiveOpsOfVars);
bool use_cuda = Get<bool>(kUseCuda);
// Currently, only perform InplaceAddToOpPass on cuda place
if (!use_cuda) {
return;
}
// Step 1: Build a reverse map of last_live_ops
// i.e.: op -> vars
std::unordered_map<details::ComputationOpHandle *,
std::unordered_map<std::string, ir::Node *>>
candidate_ops;
for (auto &each_scope_ops : last_live_ops) {
for (auto &pair : each_scope_ops) {
// If variable has more than 1 last lived ops, this variable cannot
// be inplaced.
if (pair.second.ops().size() != 1) {
continue;
}
auto *op = *(pair.second.ops().begin());
const std::string &op_type = op->GetOp()->Type();
const framework::OpDesc *op_desc = op->Node()->Op();
PADDLE_ENFORCE_NOT_NULL(
op_desc, platform::errors::NotFound("Op(%s) can not find opdesc.",
op->Name()));
// only grad op should be processed.
if (op_type != "grad_add") {
continue;
}
const std::string &var_name = pair.first;
auto in_nodes = this->FindNodesByName(var_name, op->Node()->inputs);
if (in_nodes.size() == 1) {
candidate_ops[op][var_name] = *in_nodes.begin();
}
VLOG(4) << "Find op " << op_type << " with input(" << var_name
<< ") that can do inplace add to";
}
}
// Step 2: Check which vars can be inplaced indeed
for (auto &op_vars_pair : candidate_ops) {
auto *op = op_vars_pair.first;
// The original gradient accumulation is g = sum(g_0, g_1,..., g_n), and it
// could be changed as follws if inplace addto is enabled:
// g_sum_0 = g_0
// g_sum_1 = grad_add(g_sum_0, g_1)
// g_sum_2 = grad_add(g_sum_1, g_2)
// ...
// g_sum_n = grad_add(g_sum_n-1, g_n)
// here we will add inplace for each grad_add, for example, for the first
// grad_add, g_sum_0 -> g1, g_sum_1 -> g1, and set grad_add as skipped.
const std::string &op_type = op->GetOp()->Type();
PADDLE_ENFORCE_EQ(op->Node()->inputs.size(), 2,
platform::errors::InvalidArgument(
"The size of inputs of %s should be 2, but got %d",
op_type, op->Node()->inputs.size()));
PADDLE_ENFORCE_EQ(op->Node()->outputs.size(), 1,
platform::errors::InvalidArgument(
"The size of outputs of %s should be 1, but got %d",
op_type, op->Node()->outputs.size()));
auto *left_var_ptr = dynamic_cast<details::VarHandle *>(
&(op->Node()->inputs[0]->Wrapper<details::VarHandleBase>()));
auto *right_var_ptr = dynamic_cast<details::VarHandle *>(
&(op->Node()->inputs[1]->Wrapper<details::VarHandleBase>()));
auto *out_var_ptr = dynamic_cast<details::VarHandle *>(
&(op->Node()->outputs[0]->Wrapper<details::VarHandleBase>()));
if (left_var_ptr == nullptr || right_var_ptr == nullptr ||
out_var_ptr == nullptr) {
continue;
}
// auto *left_generated_op = dynamic_cast<details::ComputationOpHandle *>(
// left_var_ptr->GeneratedOp());
auto *right_generated_op = dynamic_cast<details::ComputationOpHandle *>(
right_var_ptr->GeneratedOp());
auto *out_generated_op = dynamic_cast<details::ComputationOpHandle *>(
out_var_ptr->GeneratedOp());
// NOTE(zhiqiu): currently, only conv2d_grad supports addto strategy
if (right_generated_op->Name() != "conv2d_grad") {
continue;
}
// NOTE(zhiqiu): Normally, if we inplace a->b, we should let a generated
// before b. However, in the situation of inplace addto, we do not care
// the order, since a+b is equal to b+a. Is there any exception for that?
// AddDependencyVar(right_generated_op, left_generated_op);
// no need, as discussed above.
// step (a): inplace right_var->left_var of grad_add
this->AddReuseVar(right_generated_op, left_var_ptr, right_var_ptr);
UpdateLastLiveOpOfVar(right_generated_op, left_var_ptr, right_var_ptr);
VLOG(4) << "Inplace performed in op " << right_generated_op->GetOp()->Type()
<< ": " << left_var_ptr->Name() << " -> " << right_var_ptr->Name()
<< ". Debug String is: "
<< right_generated_op->GetOp()->DebugString()
<< ". ReuseType: " << ReuseType();
// step (b): inplace out -> right_var of grad_add
this->AddReuseVar(out_generated_op, right_var_ptr, out_var_ptr, true);
VLOG(4) << "Inplace performed in op " << op_type << ": "
<< left_var_ptr->Name() << " -> " << out_var_ptr->Name()
<< ". Debug String is: " << op->GetOp()->DebugString()
<< ". ReuseType: " << ReuseType();
// step (c): make right_var cannot inplace afterwards. canbe done
// aotomatically since CollectReusedVars is called before any reuse.
// step (d): make right_var's generated op use addto
right_generated_op->GetOp()->SetAttr("use_addto", true);
// step (e): make grad_add skip running
op->SetSkipRunning(true);
}
}
} // namespace ir
} // namespace framework
} // namespace paddle
REGISTER_PASS(inplace_addto_op_pass, paddle::framework::ir::InplaceAddToOpPass)
.RequirePassAttr(paddle::framework::ir::kMemOptVarInfoMapList)
.RequirePassAttr(paddle::framework::ir::kLastLiveOpsOfVars)
.RequirePassAttr(paddle::framework::ir::kUseCuda);
...@@ -13,6 +13,7 @@ ...@@ -13,6 +13,7 @@
// limitations under the License. // limitations under the License.
#include "paddle/fluid/framework/ir/memory_optimize_pass/memory_reuse_pass.h" #include "paddle/fluid/framework/ir/memory_optimize_pass/memory_reuse_pass.h"
#include <functional> #include <functional>
#include <map> #include <map>
#include <string> #include <string>
...@@ -73,6 +74,7 @@ bool MemoryReusePass::TryReuseVar(details::VarHandle *in_var, ...@@ -73,6 +74,7 @@ bool MemoryReusePass::TryReuseVar(details::VarHandle *in_var,
out_var->Name())); out_var->Name()));
if (IsVarPairReusable(*in_var, *out_var)) { if (IsVarPairReusable(*in_var, *out_var)) {
AddReuseVar(op, in_var, out_var); AddReuseVar(op, in_var, out_var);
UpdateLastLiveOpOfVar(op, in_var, out_var);
return true; return true;
} else { } else {
return false; return false;
...@@ -324,7 +326,8 @@ bool MemoryReusePass::IsVarPairReusable( ...@@ -324,7 +326,8 @@ bool MemoryReusePass::IsVarPairReusable(
void MemoryReusePass::AddReuseVar(details::ComputationOpHandle *op, void MemoryReusePass::AddReuseVar(details::ComputationOpHandle *op,
details::VarHandle *in_var, details::VarHandle *in_var,
details::VarHandle *out_var) const { details::VarHandle *out_var,
bool share_dims) const {
PADDLE_ENFORCE_GT( PADDLE_ENFORCE_GT(
(*var_infos_)[op->GetScopeIdx()].count(in_var->Name()), 0, (*var_infos_)[op->GetScopeIdx()].count(in_var->Name()), 0,
platform::errors::NotFound("Var(%s) does not in mem opt var infos.", platform::errors::NotFound("Var(%s) does not in mem opt var infos.",
...@@ -344,13 +347,15 @@ void MemoryReusePass::AddReuseVar(details::ComputationOpHandle *op, ...@@ -344,13 +347,15 @@ void MemoryReusePass::AddReuseVar(details::ComputationOpHandle *op,
share_buffer_op->AddInput(in_var); share_buffer_op->AddInput(in_var);
} }
if (share_dims) {
share_buffer_op->SetShareDims(true);
}
share_buffer_op->AddReuseVarPair( share_buffer_op->AddReuseVarPair(
(*var_infos_)[op->GetScopeIdx()].at(in_var->Name()).get(), (*var_infos_)[op->GetScopeIdx()].at(in_var->Name()).get(),
out_var->Name()); out_var->Name());
reused_in_var_names_[op->GetScopeIdx()].insert(in_var->Name()); reused_in_var_names_[op->GetScopeIdx()].insert(in_var->Name());
reused_out_var_names_[op->GetScopeIdx()].insert(out_var->Name()); reused_out_var_names_[op->GetScopeIdx()].insert(out_var->Name());
UpdateLastLiveOpOfVar(op, in_var, out_var);
} }
// 1. Set last living op of in_var to be any last living op of out_var // 1. Set last living op of in_var to be any last living op of out_var
......
...@@ -18,6 +18,7 @@ ...@@ -18,6 +18,7 @@
#include <unordered_map> #include <unordered_map>
#include <unordered_set> #include <unordered_set>
#include <vector> #include <vector>
#include "paddle/fluid/framework/details/computation_op_handle.h" #include "paddle/fluid/framework/details/computation_op_handle.h"
#include "paddle/fluid/framework/details/multi_devices_helper.h" #include "paddle/fluid/framework/details/multi_devices_helper.h"
#include "paddle/fluid/framework/details/share_tensor_buffer_op_handle.h" #include "paddle/fluid/framework/details/share_tensor_buffer_op_handle.h"
...@@ -92,6 +93,12 @@ class MemoryReusePass : public Pass { ...@@ -92,6 +93,12 @@ class MemoryReusePass : public Pass {
int64_t GetMemorySize(const details::VarHandle &var) const; int64_t GetMemorySize(const details::VarHandle &var) const;
void AddReuseVar(details::ComputationOpHandle *op, details::VarHandle *in_var,
details::VarHandle *out_var, bool share_dims = false) const;
virtual void UpdateLastLiveOpOfVar(details::ComputationOpHandle *op,
details::VarHandle *in_var,
details::VarHandle *out_var) const;
private: private:
VarDesc *GetVarDesc(const details::VarHandle &var) const; VarDesc *GetVarDesc(const details::VarHandle &var) const;
...@@ -109,13 +116,6 @@ class MemoryReusePass : public Pass { ...@@ -109,13 +116,6 @@ class MemoryReusePass : public Pass {
void CollectReusedVars() const; void CollectReusedVars() const;
void AddReuseVar(details::ComputationOpHandle *op, details::VarHandle *in_var,
details::VarHandle *out_var) const;
void UpdateLastLiveOpOfVar(details::ComputationOpHandle *op,
details::VarHandle *in_var,
details::VarHandle *out_var) const;
private: private:
mutable Graph *graph_; mutable Graph *graph_;
mutable bool use_cuda_; mutable bool use_cuda_;
......
...@@ -157,6 +157,14 @@ class OperatorBase { ...@@ -157,6 +157,14 @@ class OperatorBase {
platform::errors::NotFound("(%s) is not found in AttributeMap.", name)); platform::errors::NotFound("(%s) is not found in AttributeMap.", name));
return BOOST_GET_CONST(T, attrs_.at(name)); return BOOST_GET_CONST(T, attrs_.at(name));
} }
void SetAttr(const std::string& name, const Attribute& v) {
PADDLE_ENFORCE_EQ(
HasAttr(name), true,
platform::errors::NotFound(
"The attribute %s is not found in operator %s", name, Type()));
attrs_[name] = v;
}
const AttributeMap& Attrs() const { return attrs_; } const AttributeMap& Attrs() const { return attrs_; }
const VariableNameMap& Inputs() const { return inputs_; } const VariableNameMap& Inputs() const { return inputs_; }
......
...@@ -13,12 +13,14 @@ See the License for the specific language governing permissions and ...@@ -13,12 +13,14 @@ See the License for the specific language governing permissions and
limitations under the License. */ limitations under the License. */
#include "paddle/fluid/framework/parallel_executor.h" #include "paddle/fluid/framework/parallel_executor.h"
#include <algorithm> #include <algorithm>
#include <memory> #include <memory>
#include <string> #include <string>
#include <tuple> #include <tuple>
#include <utility> #include <utility>
#include <vector> #include <vector>
#include "paddle/fluid/framework/details/async_ssa_graph_executor.h" #include "paddle/fluid/framework/details/async_ssa_graph_executor.h"
#include "paddle/fluid/framework/details/fast_threaded_ssa_graph_executor.h" #include "paddle/fluid/framework/details/fast_threaded_ssa_graph_executor.h"
#include "paddle/fluid/framework/details/multi_devices_helper.h" #include "paddle/fluid/framework/details/multi_devices_helper.h"
...@@ -108,6 +110,11 @@ class ParallelExecutorPrivate { ...@@ -108,6 +110,11 @@ class ParallelExecutorPrivate {
* them. * them.
*/ */
inline void SetSkipMemoryReuse(size_t scope_idx, const std::string &name) { inline void SetSkipMemoryReuse(size_t scope_idx, const std::string &name) {
if (mem_opt_var_infos_.size() == 0) {
VLOG(4) << "The mem_opt_var_infos_ is empty, maybe no memory "
"optimization strategy is enabled";
return;
}
auto iter = mem_opt_var_infos_[scope_idx].find(name); auto iter = mem_opt_var_infos_[scope_idx].find(name);
if (iter != mem_opt_var_infos_[scope_idx].end()) { if (iter != mem_opt_var_infos_[scope_idx].end()) {
iter->second->SetSkipMemoryReuse(true); iter->second->SetSkipMemoryReuse(true);
...@@ -308,6 +315,7 @@ ir::Graph *ParallelExecutorPrivate::ApplyMemoryOptimizePass(ir::Graph *graph) { ...@@ -308,6 +315,7 @@ ir::Graph *ParallelExecutorPrivate::ApplyMemoryOptimizePass(ir::Graph *graph) {
} }
bool need_mem_opt = build_strategy_.enable_inplace_ || bool need_mem_opt = build_strategy_.enable_inplace_ ||
build_strategy_.enable_addto_ ||
build_strategy_.memory_optimize_.get() || is_gc_enabled; build_strategy_.memory_optimize_.get() || is_gc_enabled;
if (!need_mem_opt) return graph; if (!need_mem_opt) return graph;
...@@ -320,6 +328,16 @@ ir::Graph *ParallelExecutorPrivate::ApplyMemoryOptimizePass(ir::Graph *graph) { ...@@ -320,6 +328,16 @@ ir::Graph *ParallelExecutorPrivate::ApplyMemoryOptimizePass(ir::Graph *graph) {
graph = ref_cnt_pass->Apply(graph); graph = ref_cnt_pass->Apply(graph);
VLOG(10) << "ReferenceCountPass Applied"; VLOG(10) << "ReferenceCountPass Applied";
if (build_strategy_.enable_addto_) {
auto addto_pass = ir::PassRegistry::Instance().Get("inplace_addto_op_pass");
addto_pass->SetNotOwned(ir::kMemOptVarInfoMapList, &mem_opt_var_infos_);
addto_pass->SetNotOwned(ir::kLastLiveOpsOfVars, &last_live_ops_of_vars);
addto_pass->SetNotOwned(ir::kUseCuda, &use_cuda_);
VLOG(10) << "Start to apply inplace_addto_op_pass";
graph = addto_pass->Apply(graph);
VLOG(10) << "inplace_addto_op_pass Applied";
}
if (build_strategy_.enable_inplace_) { if (build_strategy_.enable_inplace_) {
auto inplace_pass = auto inplace_pass =
ir::PassRegistry::Instance().Get("buffer_shared_inplace_pass"); ir::PassRegistry::Instance().Get("buffer_shared_inplace_pass");
...@@ -1068,3 +1086,4 @@ USE_PASS(reference_count_pass); ...@@ -1068,3 +1086,4 @@ USE_PASS(reference_count_pass);
USE_PASS(eager_deletion_pass); USE_PASS(eager_deletion_pass);
USE_PASS(buffer_shared_inplace_pass); USE_PASS(buffer_shared_inplace_pass);
USE_PASS(buffer_shared_cross_op_memory_reuse_pass); USE_PASS(buffer_shared_cross_op_memory_reuse_pass);
USE_PASS(inplace_addto_op_pass);
...@@ -14,6 +14,7 @@ limitations under the License. */ ...@@ -14,6 +14,7 @@ limitations under the License. */
#include <utility> #include <utility>
#include <vector> #include <vector>
#include "paddle/fluid/framework/eigen.h" #include "paddle/fluid/framework/eigen.h"
#include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/framework/op_registry.h"
#include "paddle/fluid/framework/tensor.h" #include "paddle/fluid/framework/tensor.h"
...@@ -287,7 +288,9 @@ class CUDNNConvOpKernel : public framework::OpKernel<T> { ...@@ -287,7 +288,9 @@ class CUDNNConvOpKernel : public framework::OpKernel<T> {
#endif #endif
// ------------------- cudnn conv forward --------------------- // ------------------- cudnn conv forward ---------------------
ScalingParamType<T> alpha = 1.0f, beta = 0.0f; ScalingParamType<T> alpha = 1.0f;
ScalingParamType<T> beta = ctx.Attr<bool>("use_addto") ? 1.0f : 0.0f;
VLOG(4) << "Conv: use_addto = " << ctx.Attr<bool>("use_addto");
for (int i = 0; i < groups; i++) { for (int i = 0; i < groups; i++) {
workspace_handle.RunFunc( workspace_handle.RunFunc(
[&](void* workspace_ptr) { [&](void* workspace_ptr) {
...@@ -609,9 +612,13 @@ class CUDNNConvGradOpKernel : public framework::OpKernel<T> { ...@@ -609,9 +612,13 @@ class CUDNNConvGradOpKernel : public framework::OpKernel<T> {
} }
// ------------------- cudnn conv backward data --------------------- // ------------------- cudnn conv backward data ---------------------
ScalingParamType<T> alpha = 1.0f, beta = 0.0f; ScalingParamType<T> alpha = 1.0f;
ScalingParamType<T> beta = ctx.Attr<bool>("use_addto") ? 1.0f : 0.0f;
VLOG(4) << "Conv_grad: use_addto = " << ctx.Attr<bool>("use_addto");
if (input_grad) { if (input_grad) {
// Because beta is zero, it is unnecessary to reset input_grad. // When beta is 0, it is unnecessary to reset input_grad.
// When beta is 1, the output cannot be reset since addt strategy used.
for (int i = 0; i < groups; i++) { for (int i = 0; i < groups; i++) {
workspace_handle.RunFunc( workspace_handle.RunFunc(
[&](void* cudnn_workspace_ptr) { [&](void* cudnn_workspace_ptr) {
...@@ -653,6 +660,9 @@ class CUDNNConvGradOpKernel : public framework::OpKernel<T> { ...@@ -653,6 +660,9 @@ class CUDNNConvGradOpKernel : public framework::OpKernel<T> {
ctx, &transformed_input_grad_channel, input_grad); ctx, &transformed_input_grad_channel, input_grad);
} }
} }
// filter_grad do not use inplace addto.
ScalingParamType<T> beta_filter = 0.0f;
// ------------------- cudnn conv backward filter --------------------- // ------------------- cudnn conv backward filter ---------------------
if (filter_grad) { if (filter_grad) {
// Because beta is zero, it is unnecessary to reset filter_grad. // Because beta is zero, it is unnecessary to reset filter_grad.
...@@ -665,7 +675,7 @@ class CUDNNConvGradOpKernel : public framework::OpKernel<T> { ...@@ -665,7 +675,7 @@ class CUDNNConvGradOpKernel : public framework::OpKernel<T> {
input_data + i * group_offset_in, args2.odesc.desc(), input_data + i * group_offset_in, args2.odesc.desc(),
output_grad_data + i * group_offset_out, output_grad_data + i * group_offset_out,
args2.cdesc.desc(), filter_algo, cudnn_workspace_ptr, args2.cdesc.desc(), filter_algo, cudnn_workspace_ptr,
workspace_size, &beta, args2.wdesc.desc(), workspace_size, &beta_filter, args2.wdesc.desc(),
filter_grad_data + i * group_offset_filter)); filter_grad_data + i * group_offset_filter));
}, },
workspace_size); workspace_size);
...@@ -1017,7 +1027,14 @@ class CUDNNConvDoubleGradOpKernel : public framework::OpKernel<T> { ...@@ -1017,7 +1027,14 @@ class CUDNNConvDoubleGradOpKernel : public framework::OpKernel<T> {
int group_offset_out = o_c / groups * o_h * o_w * o_d; int group_offset_out = o_c / groups * o_h * o_w * o_d;
int group_offset_filter = W->numel() / groups; int group_offset_filter = W->numel() / groups;
ScalingParamType<T> alpha = 1.0f, beta = 0.0f; ScalingParamType<T> alpha = 1.0f;
ScalingParamType<T> beta = 0.0f;
// NOTE(zhiqiu): inplace addto is not supportted in double grad yet.
// ScalingParamType<T> beta = ctx.Attr<bool>("use_addto") ? 1.0f :
// 0.0f;
// VLOG(4) << "Conv_grad_grad: use_addto = " << ctx.Attr<bool>("use_addto");
auto wkspace_handle = dev_ctx.cudnn_workspace_handle(); auto wkspace_handle = dev_ctx.cudnn_workspace_handle();
if (ddO) { if (ddO) {
......
...@@ -305,6 +305,11 @@ void Conv2DOpMaker::Make() { ...@@ -305,6 +305,11 @@ void Conv2DOpMaker::Make() {
.SetDefault(0.0f); .SetDefault(0.0f);
AddAttr<float>("fuse_beta", "(float, default 0.0) Only used in mkldnn kernel") AddAttr<float>("fuse_beta", "(float, default 0.0) Only used in mkldnn kernel")
.SetDefault(0.0f); .SetDefault(0.0f);
AddAttr<bool>(
"use_addto",
"(bool, default false) If use addto strategy or not, only used in "
"cudnn kernel")
.SetDefault(false);
AddAttr<bool>("fuse_residual_connection", AddAttr<bool>("fuse_residual_connection",
"(bool, default false) Only used in mkldnn kernel. Used " "(bool, default false) Only used in mkldnn kernel. Used "
"whenever convolution output is as an input to residual " "whenever convolution output is as an input to residual "
...@@ -460,6 +465,11 @@ void Conv3DOpMaker::Make() { ...@@ -460,6 +465,11 @@ void Conv3DOpMaker::Make() {
.SetDefault(0.0f); .SetDefault(0.0f);
AddAttr<float>("fuse_beta", "(float, default 0.0) Only used in mkldnn kernel") AddAttr<float>("fuse_beta", "(float, default 0.0) Only used in mkldnn kernel")
.SetDefault(0.0f); .SetDefault(0.0f);
AddAttr<bool>(
"use_addto",
"(bool, default false) If use addto strategy or not, only used in "
"cudnn kernel")
.SetDefault(false);
AddAttr<bool>("fuse_residual_connection", AddAttr<bool>("fuse_residual_connection",
"(bool, default false) Only used in mkldnn kernel. Used " "(bool, default false) Only used in mkldnn kernel. Used "
"whenever convolution output is as an input to residual " "whenever convolution output is as an input to residual "
......
...@@ -13,8 +13,11 @@ See the License for the specific language governing permissions and ...@@ -13,8 +13,11 @@ See the License for the specific language governing permissions and
limitations under the License. */ limitations under the License. */
#include "paddle/fluid/operators/elementwise/elementwise_add_op.h" #include "paddle/fluid/operators/elementwise/elementwise_add_op.h"
#include <memory> #include <memory>
#include <string> #include <string>
#include "paddle/fluid/framework/op_version_registry.h"
#include "paddle/fluid/operators/elementwise/elementwise_op.h" #include "paddle/fluid/operators/elementwise/elementwise_op.h"
namespace paddle { namespace paddle {
...@@ -129,3 +132,18 @@ REGISTER_OP_CPU_KERNEL( ...@@ -129,3 +132,18 @@ REGISTER_OP_CPU_KERNEL(
int>, int>,
ops::ElementwiseAddDoubleGradKernel<paddle::platform::CPUDeviceContext, ops::ElementwiseAddDoubleGradKernel<paddle::platform::CPUDeviceContext,
int64_t>); int64_t>);
// A specialization elementwise_add operator, used in gradient accumulation with
// inplace addto.
REGISTER_OPERATOR(
grad_add, paddle::operators::ElementwiseOp,
paddle::operators::ElementwiseAddOpMaker,
paddle::framework::EmptyGradOpMaker<paddle::framework::OpDesc>,
paddle::framework::EmptyGradOpMaker<paddle::imperative::OpBase>);
REGISTER_OP_CPU_KERNEL(
grad_add,
ops::ElementwiseAddKernel<paddle::platform::CPUDeviceContext, float>,
ops::ElementwiseAddKernel<paddle::platform::CPUDeviceContext, double>,
ops::ElementwiseAddKernel<paddle::platform::CPUDeviceContext, int>,
ops::ElementwiseAddKernel<paddle::platform::CPUDeviceContext, int64_t>);
...@@ -111,3 +111,10 @@ REGISTER_OP_CUDA_KERNEL( ...@@ -111,3 +111,10 @@ REGISTER_OP_CUDA_KERNEL(
ops::ElementwiseAddDoubleGradKernel<plat::CUDADeviceContext, int64_t>, ops::ElementwiseAddDoubleGradKernel<plat::CUDADeviceContext, int64_t>,
ops::ElementwiseAddDoubleGradKernel<plat::CUDADeviceContext, ops::ElementwiseAddDoubleGradKernel<plat::CUDADeviceContext,
plat::float16>); plat::float16>);
REGISTER_OP_CUDA_KERNEL(
grad_add, ops::ElementwiseAddKernel<plat::CUDADeviceContext, float>,
ops::ElementwiseAddKernel<plat::CUDADeviceContext, double>,
ops::ElementwiseAddKernel<plat::CUDADeviceContext, int>,
ops::ElementwiseAddKernel<plat::CUDADeviceContext, int64_t>,
ops::ElementwiseAddKernel<plat::CUDADeviceContext, plat::float16>);
...@@ -521,3 +521,18 @@ DEFINE_int32( ...@@ -521,3 +521,18 @@ DEFINE_int32(
DEFINE_bool(sort_sum_gradient, false, DEFINE_bool(sort_sum_gradient, false,
"Sum gradients by the reverse order of " "Sum gradients by the reverse order of "
"the forward execution sequence."); "the forward execution sequence.");
/**
* Performance related FLAG
* Name: max_inplace_grad_add
* Since Version: 2.0.0
* Value Range: int32, default=0
* Example:
* Note: The maximum number of inplace grad_add.
*/
DEFINE_int32(
max_inplace_grad_add, 0,
"The maximum number of inplace grad_add. When doing "
"gradient accumulation, if the number of gradients need to that "
"less FLAGS_max_inplace_grad_add, than it will be use several grad_add"
"instead of sum. Default is 0.");
...@@ -62,6 +62,7 @@ DECLARE_bool(use_system_allocator); ...@@ -62,6 +62,7 @@ DECLARE_bool(use_system_allocator);
// others // others
DECLARE_bool(benchmark); DECLARE_bool(benchmark);
DECLARE_int32(inner_op_parallelism); DECLARE_int32(inner_op_parallelism);
DECLARE_int32(max_inplace_grad_add);
DECLARE_string(tracer_profile_fname); DECLARE_string(tracer_profile_fname);
#ifdef PADDLE_WITH_CUDA #ifdef PADDLE_WITH_CUDA
// cudnn // cudnn
...@@ -348,7 +349,7 @@ static void RegisterGlobalVarGetterSetter() { ...@@ -348,7 +349,7 @@ static void RegisterGlobalVarGetterSetter() {
FLAGS_init_allocated_mem, FLAGS_initial_cpu_memory_in_mb, FLAGS_init_allocated_mem, FLAGS_initial_cpu_memory_in_mb,
FLAGS_memory_fraction_of_eager_deletion, FLAGS_use_pinned_memory, FLAGS_memory_fraction_of_eager_deletion, FLAGS_use_pinned_memory,
FLAGS_benchmark, FLAGS_inner_op_parallelism, FLAGS_tracer_profile_fname, FLAGS_benchmark, FLAGS_inner_op_parallelism, FLAGS_tracer_profile_fname,
FLAGS_paddle_num_threads, FLAGS_use_mkldnn); FLAGS_paddle_num_threads, FLAGS_use_mkldnn, FLAGS_max_inplace_grad_add);
#ifdef PADDLE_WITH_CUDA #ifdef PADDLE_WITH_CUDA
REGISTER_PUBLIC_GLOBAL_VAR( REGISTER_PUBLIC_GLOBAL_VAR(
......
...@@ -12,6 +12,7 @@ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. ...@@ -12,6 +12,7 @@ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and See the License for the specific language governing permissions and
limitations under the License. */ limitations under the License. */
#include <Python.h> #include <Python.h>
#include <algorithm> #include <algorithm>
#include <cstdlib> #include <cstdlib>
#include <map> #include <map>
...@@ -22,6 +23,7 @@ limitations under the License. */ ...@@ -22,6 +23,7 @@ limitations under the License. */
#include <unordered_set> #include <unordered_set>
#include <utility> #include <utility>
#include <vector> #include <vector>
#include "paddle/fluid/framework/executor.h" #include "paddle/fluid/framework/executor.h"
#include "paddle/fluid/framework/feed_fetch_method.h" #include "paddle/fluid/framework/feed_fetch_method.h"
#include "paddle/fluid/framework/feed_fetch_type.h" #include "paddle/fluid/framework/feed_fetch_type.h"
...@@ -2528,6 +2530,10 @@ All parameter, weight, gradient are variables in Paddle. ...@@ -2528,6 +2530,10 @@ All parameter, weight, gradient are variables in Paddle.
"enable_inplace", "enable_inplace",
[](const BuildStrategy &self) { return self.enable_inplace_; }, [](const BuildStrategy &self) { return self.enable_inplace_; },
[](BuildStrategy &self, bool b) { self.enable_inplace_ = b; }) [](BuildStrategy &self, bool b) { self.enable_inplace_ = b; })
.def_property(
"enable_addto",
[](const BuildStrategy &self) { return self.enable_addto_; },
[](BuildStrategy &self, bool b) { self.enable_addto_ = b; })
.def_property( .def_property(
"fuse_all_reduce_ops", "fuse_all_reduce_ops",
[](const BuildStrategy &self) { [](const BuildStrategy &self) {
......
...@@ -197,6 +197,7 @@ def __bootstrap__(): ...@@ -197,6 +197,7 @@ def __bootstrap__():
'free_when_no_cache_hit', 'free_when_no_cache_hit',
'call_stack_level', 'call_stack_level',
'sort_sum_gradient', 'sort_sum_gradient',
'max_inplace_grad_add',
] ]
if 'Darwin' not in sysstr: if 'Darwin' not in sysstr:
read_env_flags.append('use_pinned_memory') read_env_flags.append('use_pinned_memory')
......
...@@ -251,12 +251,19 @@ def _rename_arg_(op_descs, old_name, new_name, begin_idx=None, end_idx=None): ...@@ -251,12 +251,19 @@ def _rename_arg_(op_descs, old_name, new_name, begin_idx=None, end_idx=None):
begin_idx = 0 begin_idx = 0
if end_idx is None: if end_idx is None:
end_idx = len(op_descs) end_idx = len(op_descs)
for i in range(begin_idx, end_idx): if isinstance(op_descs, (list, tuple)):
op_desc = op_descs[i] for i in range(begin_idx, end_idx):
if isinstance(op_desc, tuple): op_desc = op_descs[i]
op_desc = op_desc[0] if isinstance(op_desc, tuple):
op_desc._rename_input(old_name, new_name) op_desc = op_desc[0]
op_desc._rename_output(old_name, new_name) op_desc._rename_input(old_name, new_name)
op_desc._rename_output(old_name, new_name)
if isinstance(op_descs, collections.OrderedDict):
for key, value in op_descs.items():
if isinstance(value, (list, tuple)):
for op_desc in value:
op_desc._rename_input(old_name, new_name)
op_desc._rename_output(old_name, new_name)
def _create_op_desc_(op_type, inputs, outputs, attrs): def _create_op_desc_(op_type, inputs, outputs, attrs):
...@@ -369,6 +376,41 @@ def _append_grad_suffix_(name): ...@@ -369,6 +376,41 @@ def _append_grad_suffix_(name):
return cpt.to_text(name) + core.grad_var_suffix() return cpt.to_text(name) + core.grad_var_suffix()
def _accumulate_gradients_by_sum_op_(var_name, renamed_vars, pending_sum_ops,
op_idx):
"""
Use sum op to accumulate_gradients, the gradients are stored in renamed_vars.
"""
if op_idx not in pending_sum_ops.keys():
pending_sum_ops[op_idx] = []
pending_sum_ops[op_idx].append(
_create_op_desc_("sum", {"X": renamed_vars[var_name]},
{"Out": [var_name]}, {"use_mkldnn": False}))
renamed_vars[var_name] = [var_name]
def _accumulate_gradients_by_add_ops_(var_name, renamed_vars, pending_sum_ops,
op_idx):
"""
Use several inplace add op to accumulate_gradients, the gradients are stored in renamed_vars.
"""
if op_idx not in pending_sum_ops.keys():
pending_sum_ops[op_idx] = []
out_name = renamed_vars[var_name][0]
for i in range(1, len(renamed_vars[var_name])):
x_name = out_name
y_name = renamed_vars[var_name][i]
if i != len(renamed_vars[var_name]) - 1:
out_name = var_name + '@ADD@' + str(i)
else:
out_name = var_name
pending_sum_ops[op_idx].append(
_create_op_desc_("grad_add", {"X": [x_name],
"Y": [y_name]}, {"Out": [out_name]},
{"use_mkldnn": False}))
renamed_vars[var_name] = [var_name]
def _addup_repetitive_outputs_(op_descs, block_idx): def _addup_repetitive_outputs_(op_descs, block_idx):
""" """
In backward part, an variable may be the output of more than one ops. In backward part, an variable may be the output of more than one ops.
...@@ -376,7 +418,9 @@ def _addup_repetitive_outputs_(op_descs, block_idx): ...@@ -376,7 +418,9 @@ def _addup_repetitive_outputs_(op_descs, block_idx):
In these cases, the variable should be the accumulation of all the outputs. In these cases, the variable should be the accumulation of all the outputs.
`sum_op`s are added to implement the accumulate. `sum_op`s are added to implement the accumulate.
""" """
pending_sum_ops = [] _MAX_ADD_NUM_ = core.globals()['FLAGS_max_inplace_grad_add']
#pending_sum_ops = []
pending_sum_ops = collections.OrderedDict()
var_rename_count = collections.defaultdict(int) var_rename_count = collections.defaultdict(int)
renamed_vars = collections.defaultdict(list) renamed_vars = collections.defaultdict(list)
renamed_var_start_idx = collections.defaultdict(list) renamed_var_start_idx = collections.defaultdict(list)
...@@ -385,10 +429,13 @@ def _addup_repetitive_outputs_(op_descs, block_idx): ...@@ -385,10 +429,13 @@ def _addup_repetitive_outputs_(op_descs, block_idx):
if "@GRAD" not in var_name: if "@GRAD" not in var_name:
continue continue
if len(renamed_vars[var_name]) > 1: if len(renamed_vars[var_name]) > 1:
pending_sum_ops.append((_create_op_desc_( if len(renamed_vars[var_name]) > _MAX_ADD_NUM_:
"sum", {"X": renamed_vars[var_name]}, {"Out": [var_name]}, _accumulate_gradients_by_sum_op_(var_name, renamed_vars,
{"use_mkldnn": False}), idx)) pending_sum_ops, idx)
renamed_vars[var_name] = [var_name] else:
_accumulate_gradients_by_add_ops_(var_name, renamed_vars,
pending_sum_ops, idx)
for param_idx, param_name in enumerate(op_desc.output_names()): for param_idx, param_name in enumerate(op_desc.output_names()):
arg_names = op_desc.output(param_name) arg_names = op_desc.output(param_name)
for arg_idx, var_name in enumerate(arg_names): for arg_idx, var_name in enumerate(arg_names):
...@@ -440,13 +487,26 @@ def _addup_repetitive_outputs_(op_descs, block_idx): ...@@ -440,13 +487,26 @@ def _addup_repetitive_outputs_(op_descs, block_idx):
renamed_vars[var_name].append(new_name) renamed_vars[var_name].append(new_name)
for var_name, inputs in six.iteritems(renamed_vars): for var_name, inputs in six.iteritems(renamed_vars):
if len(inputs) > 1: if len(renamed_vars[var_name]) > 1:
pending_sum_ops.append( if len(renamed_vars[var_name]) > _MAX_ADD_NUM_:
(_create_op_desc_("sum", {"X": inputs}, {"Out": [var_name]}, _accumulate_gradients_by_sum_op_(var_name, renamed_vars,
{"use_mkldnn": False}), len(op_descs))) pending_sum_ops, len(op_descs))
else:
_accumulate_gradients_by_add_ops_(var_name, renamed_vars,
pending_sum_ops,
len(op_descs))
# sum_op descs are sorted according to their insert position # sum_op descs are sorted according to their insert position
for p in reversed(pending_sum_ops): for key, value in collections.OrderedDict(
op_descs.insert(p[1], p[0]) reversed(list(pending_sum_ops.items()))).items():
# NOTE(zhiqiu): Since reversed, the idx of op_descs to be inserted will remains correct.
# For example, [0, 1, 2], and we want to insert 'a' at idx 1, 'b' at idx 2, and the expected result is [0, 1, 'a', 2, 'b'].
# If reversed, we first insert 'b' at idx 2, it becomes [0, 1, 2, 'b'], and then insert 'a' at idx 1, it becomes [0, 1, 'a', 2, 'b'].
# If not reverse, we first insert 'a' at idx 1, it becomes [0, 1, 'a', 2], and then insert 'b' at idx 2, it becomes [0, 1, 'a', 'b', 2].
idx = key
for i, op in enumerate(value):
op_descs.insert(idx + i, op)
return op_descs return op_descs
......
# Copyright (c) 2020 PaddlePaddle Authors. All Rights Reserved.
#
# 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.
from __future__ import print_function
import unittest
import paddle
import paddle.fluid as fluid
import paddle.fluid.layers as layers
from paddle.fluid.backward import calc_gradient
import numpy as np
class ConvBNLayer(fluid.Layer):
def __init__(self,
num_channels,
num_filters,
filter_size,
stride=1,
groups=1,
act=None,
use_cudnn=False):
super(ConvBNLayer, self).__init__()
self._conv = fluid.dygraph.Conv2D(
num_channels=num_channels,
num_filters=num_filters,
filter_size=filter_size,
stride=stride,
padding=(filter_size - 1) // 2,
groups=groups,
act=None,
bias_attr=False,
use_cudnn=use_cudnn)
self._batch_norm = fluid.dygraph.BatchNorm(num_filters, act=act)
def forward(self, inputs):
y = self._conv(inputs)
y = self._batch_norm(y)
return y
def create_program():
main = fluid.Program()
startup = fluid.Program()
with fluid.program_guard(main, startup):
x = fluid.data(name='img', shape=[-1, 3, 224, 224])
x.stop_gradient = False
x = fluid.layers.prelu(x, mode="channel")
conv = ConvBNLayer(
num_channels=3,
num_filters=3,
filter_size=1,
act='relu',
use_cudnn=True)
y = conv(x) + x
loss = fluid.layers.reduce_sum(y)
sgd = fluid.optimizer.SGD(learning_rate=0.01)
sgd.minimize(loss)
return loss, main, startup, conv._conv.weight
class TestInplaceAddto(unittest.TestCase):
def test_result(self):
def run_program(enable_addto):
np.random.seed(10)
paddle.manual_seed(10)
paddle.framework.random._manual_program_seed(10)
if fluid.core.is_compiled_with_cuda():
fluid.set_flags({"FLAGS_cudnn_deterministic": True})
fluid.set_flags({"FLAGS_max_inplace_grad_add": 2})
loss, main, startup, w = create_program()
place = fluid.CUDAPlace(0) if fluid.core.is_compiled_with_cuda(
) else fluid.CPUPlace()
exe = fluid.Executor(place)
strategy = fluid.BuildStrategy()
strategy.enable_addto = enable_addto
compiled = fluid.CompiledProgram(main).with_data_parallel(
loss_name=loss.name, build_strategy=strategy)
exe.run(startup)
img = np.random.uniform(-128, 128,
[8, 3, 224, 224]).astype(np.float32)
for i in range(2):
res = exe.run(compiled,
feed={'img': img},
fetch_list=[loss.name, w.name])
return res
res1, w1 = run_program(True)
res2, w2 = run_program(False)
print(res1, res2)
self.assertTrue(np.array_equal(res1, res2))
if __name__ == "__main__":
unittest.main()
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册