From a93a9eef8f6c6c022dfc3ee9ba229e47cf505906 Mon Sep 17 00:00:00 2001 From: sneaxiy Date: Fri, 22 Mar 2019 04:38:10 +0000 Subject: [PATCH] add op registry type refine gc code test=develop --- paddle/fluid/framework/CMakeLists.txt | 7 +- .../details/early_delete_op_handle.h | 140 ------------- paddle/fluid/framework/details/op_registry.h | 104 ++++++++-- .../framework/details/reference_count_pass.cc | 94 ++++++++- paddle/fluid/framework/executor.cc | 116 ++--------- paddle/fluid/framework/executor.h | 16 +- paddle/fluid/framework/executor_gc_helper.cc | 185 ++++++++++++++++++ paddle/fluid/framework/executor_gc_helper.h | 40 ++++ .../no_need_buffer_vars_inference.cc | 23 +++ .../framework/no_need_buffer_vars_inference.h | 60 ++++++ paddle/fluid/framework/op_info.h | 6 + paddle/fluid/framework/operator.cc | 36 +++- paddle/fluid/framework/operator.h | 11 +- paddle/fluid/framework/type_defs.h | 5 + paddle/fluid/operators/clip_op.cc | 19 +- .../elementwise/elementwise_add_op.cc | 3 +- .../operators/elementwise/elementwise_op.h | 38 ++-- .../elementwise/elementwise_sub_op.cc | 3 +- paddle/fluid/pybind/pybind.cc | 5 +- .../test_eager_deletion_delete_vars.py | 165 ++++++++++++++++ 20 files changed, 776 insertions(+), 300 deletions(-) delete mode 100644 paddle/fluid/framework/details/early_delete_op_handle.h create mode 100644 paddle/fluid/framework/executor_gc_helper.cc create mode 100644 paddle/fluid/framework/executor_gc_helper.h create mode 100644 paddle/fluid/framework/no_need_buffer_vars_inference.cc create mode 100644 paddle/fluid/framework/no_need_buffer_vars_inference.h create mode 100644 python/paddle/fluid/tests/unittests/test_eager_deletion_delete_vars.py diff --git a/paddle/fluid/framework/CMakeLists.txt b/paddle/fluid/framework/CMakeLists.txt index ad19d729e..796a0795a 100644 --- a/paddle/fluid/framework/CMakeLists.txt +++ b/paddle/fluid/framework/CMakeLists.txt @@ -126,7 +126,7 @@ cc_test(operator_test SRCS operator_test.cc DEPS operator op_registry device_con cc_library(version SRCS version.cc) cc_test(version_test SRCS version_test.cc DEPS version) -cc_library(proto_desc SRCS var_desc.cc op_desc.cc block_desc.cc program_desc.cc DEPS shape_inference op_info operator glog version) +cc_library(proto_desc SRCS var_desc.cc op_desc.cc block_desc.cc program_desc.cc no_need_buffer_vars_inference.cc DEPS shape_inference op_info operator glog version) cc_library(op_registry SRCS op_registry.cc DEPS op_proto_maker op_info operator glog proto_desc memory_optimize_helper) nv_test(op_registry_test SRCS op_registry_test.cc DEPS op_registry) @@ -164,6 +164,8 @@ else() set(NGRAPH_EXE_DEPS) endif() +cc_library(executor_gc_helper SRCS executor_gc_helper.cc DEPS scope proto_desc operator garbage_collector) + if(WITH_DISTRIBUTE) cc_library(executor SRCS executor.cc DEPS op_registry device_context scope framework_proto glog lod_rank_table feed_fetch_method sendrecvop_rpc ${GLOB_DISTRIBUTE_DEPS} graph_to_program_pass variable_helper ${NGRAPH_EXE_DEPS}) @@ -174,7 +176,7 @@ else() cc_test(test_naive_executor SRCS naive_executor_test.cc DEPS naive_executor elementwise_add_op) endif() -target_link_libraries(executor garbage_collector while_op_helper) +target_link_libraries(executor while_op_helper executor_gc_helper) cc_library(parallel_executor SRCS parallel_executor.cc DEPS threaded_ssa_graph_executor scope_buffered_ssa_graph_executor parallel_ssa_graph_executor @@ -194,6 +196,7 @@ cc_test(prune_test SRCS prune_test.cc DEPS op_info prune recurrent_op device_con cc_test(var_type_inference_test SRCS var_type_inference_test.cc DEPS op_registry proto_desc) cc_test(inplace_op_inference_test SRCS inplace_op_inference_test.cc DEPS op_registry proto_desc op_info memory_optimize_helper) + cc_library(selected_rows SRCS selected_rows.cc DEPS tensor) cc_test(selected_rows_test SRCS selected_rows_test.cc DEPS selected_rows) diff --git a/paddle/fluid/framework/details/early_delete_op_handle.h b/paddle/fluid/framework/details/early_delete_op_handle.h deleted file mode 100644 index c8382d34b..000000000 --- a/paddle/fluid/framework/details/early_delete_op_handle.h +++ /dev/null @@ -1,140 +0,0 @@ -// Copyright (c) 2018 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. - -#pragma once -#include -#include -#include "paddle/fluid/framework/details/computation_op_handle.h" -#include "paddle/fluid/framework/details/op_handle_base.h" -#include "paddle/fluid/framework/details/var_handle.h" -#include "paddle/fluid/framework/garbage_collector.h" -#include "paddle/fluid/framework/lod_tensor_array.h" -#include "paddle/fluid/framework/scope.h" -#include "paddle/fluid/framework/selected_rows.h" -#include "paddle/fluid/framework/tensor.h" - -namespace paddle { -namespace framework { -namespace details { - -class EarlyDeleteOpHandle : public OpHandleBase { - public: - EarlyDeleteOpHandle(ir::Node* node, const Scope* scope, - const platform::Place& place, - const std::vector& names, - GarbageCollector* gc) - : OpHandleBase(node), - scope_(scope), - place_(place), - names_(names), - gc_(gc) { -#ifdef PADDLE_WITH_CUDA - if (IsStreamGarabageCollector()) { - auto gpu_place = boost::get(place); - PADDLE_ENFORCE(cudaSetDevice(gpu_place.device)); - PADDLE_ENFORCE(cudaEventCreateWithFlags(&event_, cudaEventDisableTiming)); - } -#endif - } - ~EarlyDeleteOpHandle() { -#ifdef PADDLE_WITH_CUDA - if (IsStreamGarabageCollector()) { - auto gpu_place = boost::get(dev_ctx_->GetPlace()); - PADDLE_ENFORCE(cudaSetDevice(gpu_place.device)); - PADDLE_ENFORCE(cudaEventDestroy(event_)); - } -#endif - } - - std::string Name() const override { return "early_delete"; } - - protected: - void RunImpl() override { - std::vector> tensors; - auto* local_scope = scope_->FindVar(kLocalExecScopeName)->Get(); - for (auto& var_name : names_) { - auto* var = local_scope->FindVar(var_name); - PADDLE_ENFORCE(var != nullptr, - string::Sprintf("Local Scope not has var %s", var_name)); - if (var->IsType()) { - tensors.emplace_back(var->GetMutable()->MoveMemoryHolder()); - } else if (var->IsType()) { - tensors.emplace_back(var->GetMutable() - ->mutable_value() - ->MoveMemoryHolder()); - } else if (var->IsType()) { - LoDTensorArray* tensor_array = var->GetMutable(); - for (auto& tensor : *tensor_array) { - tensors.emplace_back(tensor.MoveMemoryHolder()); - } - } - } - if (!tensors.empty()) { - ClearTensors(tensors); - } - } - - private: - void ClearTensors( - const std::vector>& tensors) { - if (platform::is_cpu_place(place_)) { - ClearCPUTensors(tensors); - } else { - ClearGPUTensors(tensors); - } - } - - void ClearCPUTensors( - const std::vector>& tensors) { - auto* gc = dynamic_cast(gc_); - if (gc != nullptr) { - gc->Add(tensors); - } - } - - void ClearGPUTensors( - const std::vector>& tensors) { -#ifdef PADDLE_WITH_CUDA - auto* gc = dynamic_cast(gc_); - if (gc != nullptr) { - auto compute_stream = dev_ctx_->stream(); - auto callback_stream = gc->stream(); - auto callback_func = [=]() { - PADDLE_ENFORCE(cudaEventRecord(event_, compute_stream)); - PADDLE_ENFORCE(cudaStreamWaitEvent(callback_stream, event_, 0)); - }; - gc_->Add(tensors, callback_func); - } else { - gc_->Add(tensors); - } - } - - bool IsStreamGarabageCollector() const { - return dynamic_cast(gc_) != nullptr; -#endif - } - - const Scope* scope_; - const platform::Place place_; - std::vector names_; - GarbageCollector* gc_; -#ifdef PADDLE_WITH_CUDA - platform::CUDADeviceContext* dev_ctx_; - cudaEvent_t event_; -#endif -}; - -} // namespace details -} // namespace framework -} // namespace paddle diff --git a/paddle/fluid/framework/details/op_registry.h b/paddle/fluid/framework/details/op_registry.h index e13ff99f3..7f5dfad7d 100644 --- a/paddle/fluid/framework/details/op_registry.h +++ b/paddle/fluid/framework/details/op_registry.h @@ -21,6 +21,7 @@ limitations under the License. */ #include #include "paddle/fluid/framework/grad_op_desc_maker.h" #include "paddle/fluid/framework/inplace_op_inference.h" +#include "paddle/fluid/framework/no_need_buffer_vars_inference.h" #include "paddle/fluid/framework/op_info.h" #include "paddle/fluid/framework/op_proto_maker.h" #include "paddle/fluid/framework/operator.h" @@ -36,27 +37,86 @@ enum OpInfoFillType { kGradOpDescMaker = 2, kVarTypeInference = 3, kShapeInference = 4, - kInplaceOpInference = 5 + kInplaceOpInference = 5, + kNoNeedBufferVarsInference = 6, + kUnknown = -1 }; +namespace internal { +template +struct TypePair { + using Type = T; + static constexpr OpInfoFillType kFillType = kType; +}; + +using OpRegistryClasses = std::tuple< // NOLINT + TypePair, // NOLINT + TypePair, // NOLINT + TypePair, // NOLINT + TypePair, // NOLINT + TypePair, // NOLINT + TypePair, // NOLINT + TypePair // NOLINT + >; + +static constexpr int kOpRegistryClassNumber = + std::tuple_size::value; + +template +struct IsMatchedBaseTypeImpl { + using PairType = typename std::tuple_element::type; + static constexpr bool kValue = + std::is_base_of::value; +}; + +template +struct IsMatchedBaseTypeImpl { + static constexpr bool kValue = false; +}; + +template +static inline constexpr bool IsMatchedBaseType() { + return IsMatchedBaseTypeImpl< + T, kPos, (kPos >= 0 && kPos < kOpRegistryClassNumber)>::kValue; +} + +template +struct OpInfoFillTypeGetterImpl {}; + +// This case should not happen +template +struct OpInfoFillTypeGetterImpl {}; + +template +struct OpInfoFillTypeGetterImpl { + static constexpr OpInfoFillType kType = kUnknown; +}; + +template +struct OpInfoFillTypeGetterImpl { + static constexpr OpInfoFillType kType = + OpInfoFillTypeGetterImpl()>::kType; +}; + +template +struct OpInfoFillTypeGetterImpl { + using PairType = typename std::tuple_element::type; + static constexpr OpInfoFillType kType = PairType::kFillType; +}; + +template +using OpInfoFillTypeGetter = + OpInfoFillTypeGetterImpl()>; + +} // namespace internal + template struct OpInfoFillTypeID { static constexpr OpInfoFillType ID() { - return std::is_base_of::value - ? kOperator - : (std::is_base_of::value - ? kOpProtoAndCheckerMaker - : (std::is_base_of::value - ? kGradOpDescMaker - : (std::is_base_of::value - ? kVarTypeInference - : (std::is_base_of::value - ? kShapeInference - : (std::is_base_of< - InplaceOpInference, T>::value - ? kInplaceOpInference - : static_cast( - -1)))))); + return internal::OpInfoFillTypeGetter::kType; } }; @@ -156,6 +216,18 @@ struct OpInfoFiller { } }; +template +struct OpInfoFiller { + void operator()(const char* op_type, OpInfo* info) const { + info->infer_no_need_buffer_vars_ = [](const VariableNameMap& inputs, + const VariableNameMap& outputs, + const AttributeMap& attrs) { + T infer(inputs, outputs, attrs); + return infer(); + }; + } +}; + } // namespace details } // namespace framework diff --git a/paddle/fluid/framework/details/reference_count_pass.cc b/paddle/fluid/framework/details/reference_count_pass.cc index 609214344..cad0c5ba3 100644 --- a/paddle/fluid/framework/details/reference_count_pass.cc +++ b/paddle/fluid/framework/details/reference_count_pass.cc @@ -193,6 +193,65 @@ ExtractComputationOpFromLastLivedVar(VarHandle *var, size_t scope_idx, return shrink_func(computation_op); } +static bool CanPrecede(const std::string &var_name, + std::unordered_set *op_handles) { + std::vector skip_ops; + for (auto *op_handle : *op_handles) { + auto *op_base = op_handle->GetOp(); + auto &inferer = op_base->Info().NoNeedBufferVarsInferer(); + if (!inferer) { + continue; + } + + std::unordered_set no_need_buffer_vars = + inferer(op_base->Inputs(), op_base->Outputs(), op_base->Attrs()); + + // Check whether var_name occurs in other inputs or outputs of the op + // If it occurs, we cannot precede reference count to previous op + bool occurred_in_other_vars = false; + for (auto &in_pair : op_base->Inputs()) { + if (no_need_buffer_vars.count(in_pair.first) > 0) { + continue; + } + + auto &args = in_pair.second; + auto iter = std::find(args.begin(), args.end(), var_name); + if (iter != args.end()) { + occurred_in_other_vars = true; + break; + } + } + + if (occurred_in_other_vars) { + continue; + } + + for (auto &out_pair : op_base->Outputs()) { + auto &args = out_pair.second; + auto iter = std::find(args.begin(), args.end(), var_name); + if (iter != args.end()) { + occurred_in_other_vars = true; + break; + } + } + + if (!occurred_in_other_vars) { + VLOG(2) << "Shrink var " << var_name << " in op " << op_handle->Name(); + skip_ops.emplace_back(op_handle); + } + } + + if (skip_ops.size() == op_handles->size()) { + op_handles->clear(); + return true; + } else { + for (auto *skip_op : skip_ops) { + op_handles->erase(skip_op); + } + return false; + } +} + std::unique_ptr ReferenceCountPass::ApplyImpl( std::unique_ptr graph) const { auto &ref_cnts = Get>(kGlobalReferenceCount); @@ -229,17 +288,42 @@ std::unique_ptr ReferenceCountPass::ApplyImpl( continue; } - bool ok; - auto result = ExtractComputationOpFromLastLivedVar( - name_var_pair.second.back(), i, shrink_func, &ok); + auto &var_name = name_var_pair.first; + auto &var_handles = name_var_pair.second; + + for (auto iter = var_handles.rbegin(); iter != var_handles.rend(); + ++iter) { + bool ok; + auto result = + ExtractComputationOpFromLastLivedVar(*iter, i, shrink_func, &ok); + + // Seldomly, some vars may have no pending or preceding computation ops + // Just break; + if (!ok) break; + VLOG(10) << "Extract " << result.size() << " ops of var " << var_name; + + size_t original_op_deps = result.size(); + // If reference count can be calculated precedingly, just precede + if (CanPrecede(var_name, &result)) { + VLOG(10) << "Try to precede reference count computing at var " + << var_name; + continue; + } + + size_t final_op_deps = result.size(); + if (final_op_deps < original_op_deps) { + VLOG(5) << "Shrink op deps from " << original_op_deps << " to " + << final_op_deps; + } - if (ok) { - auto &var_name = name_var_pair.first; PADDLE_ENFORCE(!result.empty(), "Last living ops of %s cannot be empty", var_name); ref_cnts[i].emplace(var_name, result.size()); last_live_ops_of_vars[i].emplace(var_name, std::move(result)); } + + // Seldomly, all preceding trying failed. + // Just skip this corner case } } diff --git a/paddle/fluid/framework/executor.cc b/paddle/fluid/framework/executor.cc index 99192292b..0d4334f19 100644 --- a/paddle/fluid/framework/executor.cc +++ b/paddle/fluid/framework/executor.cc @@ -19,6 +19,7 @@ limitations under the License. */ #include #include +#include "paddle/fluid/framework/executor_gc_helper.h" #include "paddle/fluid/framework/feed_fetch_method.h" #include "paddle/fluid/framework/lod_rank_table.h" #include "paddle/fluid/framework/lod_tensor_array.h" @@ -48,97 +49,23 @@ namespace { int kProgramId = -1; } // namespace -static std::unordered_map GetNonPersistableReferenceCounts( - const BlockDesc& block, const std::vector& skip_var_list) { - std::unordered_map ref_cnts; - std::unordered_set skip_vars(skip_var_list.begin(), - skip_var_list.end()); - - auto update_ref_cnts = [&](OpDesc* op_desc, const VariableNameMap& name_map) { - for (auto& name_pair : name_map) { - for (auto& name : name_pair.second) { - if (skip_vars.count(name)) continue; - auto* var_desc = block.FindVar(name); - if (var_desc == nullptr || var_desc->Persistable()) continue; - auto type = var_desc->Proto()->type().type(); - if (type != proto::VarType::LOD_TENSOR && - type != proto::VarType::SELECTED_ROWS && - type != proto::VarType::LOD_TENSOR_ARRAY) { - continue; - } - ++ref_cnts[name]; - } - } - }; - - for (auto op_desc : block.AllOps()) { - update_ref_cnts(op_desc, op_desc->Inputs()); - update_ref_cnts(op_desc, op_desc->Outputs()); - } - return ref_cnts; -} - ExecutorPrepareContext::ExecutorPrepareContext( - const framework::ProgramDesc& prog, size_t block_id, - const std::vector& keep_vars, bool force_disable_gc) - : prog_(prog), block_id_(block_id), force_disable_gc_(force_disable_gc) { - if (GetEagerDeletionThreshold() >= 0 && !force_disable_gc_) { - global_ref_cnts_ = - GetNonPersistableReferenceCounts(prog.Block(block_id), keep_vars); + const framework::ProgramDesc& prog, size_t block_id) + : prog_(prog), block_id_(block_id) {} + +void ExecutorPrepareContext::PrepareUnusedVars( + const std::vector& keep_vars, bool force_disable_gc) { + force_disable_gc_ = force_disable_gc; + if (GetEagerDeletionThreshold() < 0 || force_disable_gc_) { + return; } + unused_vars_ = GetUnusedVars(prog_.Block(block_id_), ops_, keep_vars); } ExecutorPrepareContext::~ExecutorPrepareContext() { VLOG(5) << "destroy ExecutorPrepareContext"; } -static void DeleteUnusedTensors( - const Scope& scope, const OperatorBase* op, GarbageCollector* gc, - std::unordered_map* ref_cnts) { - std::deque> garbages; - - auto handler = [&](const VariableNameMap& name_map) { - for (auto& name_pair : name_map) { - for (auto& name : name_pair.second) { - auto it = ref_cnts->find(name); - if (it == ref_cnts->end()) continue; - if (--(it->second) != 0) { - continue; - } - auto* var = scope.FindVar(name); - if (var == nullptr) { - continue; - } - - VLOG(2) << "Erase variable " << name; - if (var->IsType()) { - garbages.emplace_back( - var->GetMutable()->MoveMemoryHolder()); - } else if (var->IsType()) { - garbages.emplace_back(var->GetMutable() - ->mutable_value() - ->MoveMemoryHolder()); - } else if (var->IsType()) { - auto* lod_tensor_arr = var->GetMutable(); - for (auto& t : *lod_tensor_arr) { - garbages.emplace_back(t.MoveMemoryHolder()); - } - } else { - PADDLE_THROW("Type %s of %s is not supported eager deletion", - framework::ToTypeName(var->Type()), name); - } - } - } - }; - - handler(op->Inputs()); - handler(op->Outputs()); - - if (!garbages.empty()) { - gc->Add(std::move(garbages)); - } -} - Executor::Executor(const platform::Place& place) : place_(place) {} void Executor::Close() { @@ -362,8 +289,8 @@ void Executor::Run(const ProgramDesc& program, Scope* scope, std::unique_ptr Executor::Prepare( const ProgramDesc& program, int block_id, const std::vector& skip_ref_cnt_vars, bool force_disable_gc) { - std::unique_ptr ctx(new ExecutorPrepareContext( - program, block_id, skip_ref_cnt_vars, force_disable_gc)); + std::unique_ptr ctx( + new ExecutorPrepareContext(program, block_id)); PADDLE_ENFORCE_LT(static_cast(block_id), program.Size()); auto& block = program.Block(block_id); for (auto& op_desc : block.AllOps()) { @@ -375,6 +302,7 @@ std::unique_ptr Executor::Prepare( ctx->prog_.Block(ctx->block_id_), &ctx->ops_); } #endif + ctx->PrepareUnusedVars(skip_ref_cnt_vars, force_disable_gc); return ctx; } @@ -389,19 +317,17 @@ std::vector> Executor::Prepare( std::vector> result; size_t idx = 0; for (auto& bid : block_ids) { - ExecutorPrepareContext* ctx; - if (skip_ref_cnt_vars.empty()) { - ctx = new ExecutorPrepareContext(program, bid, std::vector(), - force_disable_gc); - } else { - ctx = new ExecutorPrepareContext(program, bid, skip_ref_cnt_vars[idx], - force_disable_gc); - } PADDLE_ENFORCE_LT(static_cast(bid), program.Size()); + auto* ctx = new ExecutorPrepareContext(program, bid); auto& block = program.Block(bid); for (auto& op_desc : block.AllOps()) { ctx->ops_.push_back(OpRegistry::CreateOp(*op_desc)); } + if (skip_ref_cnt_vars.empty()) { + ctx->PrepareUnusedVars(std::vector(), force_disable_gc); + } else { + ctx->PrepareUnusedVars(skip_ref_cnt_vars[idx], force_disable_gc); + } result.push_back(std::shared_ptr(ctx)); ++idx; } @@ -425,7 +351,6 @@ void Executor::RunPreparedContext(ExecutorPrepareContext* ctx, Scope* scope, // FIXME(zjl): recurrent_op is rather complex, we would // disable gc forcely in recurrent_op if (!ctx->force_disable_gc_ && max_memory_size >= 0) { - ctx->ResetReferenceCount(); #ifdef PADDLE_WITH_CUDA if (platform::is_gpu_place(place_)) { if (IsFastEagerDeletionModeEnabled()) { @@ -453,8 +378,7 @@ void Executor::RunPreparedContext(ExecutorPrepareContext* ctx, Scope* scope, op->Run(*local_scope, place_); if (gc) { - DeleteUnusedTensors(*local_scope, op.get(), gc.get(), - &(ctx->runtime_ref_cnts_)); + DeleteUnusedTensors(*local_scope, op.get(), ctx->unused_vars_, gc.get()); } } diff --git a/paddle/fluid/framework/executor.h b/paddle/fluid/framework/executor.h index 65cb9e51a..825224437 100644 --- a/paddle/fluid/framework/executor.h +++ b/paddle/fluid/framework/executor.h @@ -30,22 +30,20 @@ namespace paddle { namespace framework { struct ExecutorPrepareContext { - ExecutorPrepareContext(const framework::ProgramDesc& prog, size_t block_id, - const std::vector& skip_ref_cnt_vars = - std::vector(), - bool force_disable_gc = false); + ExecutorPrepareContext(const framework::ProgramDesc& prog, size_t block_id); ~ExecutorPrepareContext(); - void ResetReferenceCount() { runtime_ref_cnts_ = global_ref_cnts_; } + void PrepareUnusedVars(const std::vector& keep_vars, + bool force_disable_gc = false); const framework::ProgramDesc& prog_; - size_t block_id_; - bool force_disable_gc_; + const size_t block_id_; + std::vector> ops_; - std::unordered_map global_ref_cnts_; - std::unordered_map runtime_ref_cnts_; + std::unordered_map> unused_vars_; + bool force_disable_gc_{false}; }; class Executor { diff --git a/paddle/fluid/framework/executor_gc_helper.cc b/paddle/fluid/framework/executor_gc_helper.cc new file mode 100644 index 000000000..ce22664ad --- /dev/null +++ b/paddle/fluid/framework/executor_gc_helper.cc @@ -0,0 +1,185 @@ +// Copyright (c) 2019 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 "paddle/fluid/framework/executor_gc_helper.h" +#include +#include +#include +#include +#include +#include +#include "glog/logging.h" +#include "paddle/fluid/framework/lod_tensor.h" +#include "paddle/fluid/framework/lod_tensor_array.h" +#include "paddle/fluid/framework/selected_rows.h" +#include "paddle/fluid/platform/enforce.h" + +namespace paddle { +namespace framework { + +struct OpInOutInfo { + public: + void Build(const OperatorBase *op) { + is_built_ = true; + auto &inferer = op->Info().NoNeedBufferVarsInferer(); + if (inferer) { + no_need_buffer_ins_ = inferer(op->Inputs(), op->Outputs(), op->Attrs()); + + if (no_need_buffer_ins_.empty()) return; + + for (auto &in_name_pair : op->Inputs()) { + if (no_need_buffer_ins_.count(in_name_pair.first) != 0) { + continue; + } + + for (auto &in_arg_name : in_name_pair.second) { + other_args_set_.insert(in_arg_name); + } + } + + for (auto &out_name_pair : op->Outputs()) { + for (auto &out_arg_name : out_name_pair.second) { + other_args_set_.insert(out_arg_name); + } + } + } + } + + bool IsBuilt() const { return is_built_; } + + bool IsInArgBufferNeeded(const std::string &in_arg_name) const { + return no_need_buffer_ins_.empty() || + other_args_set_.count(in_arg_name) != 0; + } + + private: + std::unordered_set no_need_buffer_ins_; + std::unordered_set other_args_set_; + bool is_built_{false}; +}; + +static bool VarCanBeDeleted(const std::string &name, const BlockDesc &block, + const std::unordered_set &skip_vars) { + if (skip_vars.count(name) != 0) { + return false; + } + + auto *var_desc = block.FindVar(name); + if (var_desc == nullptr || var_desc->Persistable()) { + return false; + } + + auto type = var_desc->Proto()->type().type(); + + return type == proto::VarType::LOD_TENSOR || + type == proto::VarType::SELECTED_ROWS || + type == proto::VarType::LOD_TENSOR_ARRAY; +} + +std::unordered_map> GetUnusedVars( + const BlockDesc &block, + const std::vector> &ops, + const std::vector &skip_var_list) { + std::unordered_set skip_vars(skip_var_list.begin(), + skip_var_list.end()); + + std::unordered_map var_op_idx_map; + + for (size_t i = 0; i < ops.size(); ++i) { + auto *op = ops[i].get(); + + OpInOutInfo info; + for (auto &name_pair : op->Inputs()) { + for (auto &name : name_pair.second) { + if (!VarCanBeDeleted(name, block, skip_vars)) { + continue; + } + + // var can be gc-ed + if (!info.IsBuilt()) { + info.Build(op); + } + + if (info.IsInArgBufferNeeded(name)) { + var_op_idx_map[name] = i; + } else { + VLOG(10) << "Skip reference count computing of variable " + << name_pair.first << "(" << name << ") in Operator " + << op->Type(); + } + } + } + + for (auto &name_pair : op->Outputs()) { + for (auto &name : name_pair.second) { + if (VarCanBeDeleted(name, block, skip_vars)) { + var_op_idx_map[name] = i; + } + } + } + } + + std::unordered_map> result; + for (auto &name_op_idx_pair : var_op_idx_map) { + auto &name = name_op_idx_pair.first; + size_t op_idx = name_op_idx_pair.second; + result[ops[op_idx].get()].emplace_back(name); + } + return result; +} + +void DeleteUnusedTensors( + const Scope &scope, OperatorBase *op, + const std::unordered_map> + &delete_vars_map, + GarbageCollector *gc) { + auto iter = delete_vars_map.find(op); + if (iter == delete_vars_map.end()) { + return; + } + + auto &delete_vars = iter->second; + + std::deque> garbages; + + for (auto &var_name : delete_vars) { + auto *var = scope.FindVar(var_name); + if (var == nullptr) { + continue; + } + + VLOG(2) << "Erase variable " << var_name; + if (var->IsType()) { + garbages.emplace_back(var->GetMutable()->MoveMemoryHolder()); + } else if (var->IsType()) { + garbages.emplace_back( + var->GetMutable()->mutable_value()->MoveMemoryHolder()); + } else if (var->IsType()) { + auto *lod_tensor_arr = var->GetMutable(); + for (auto &t : *lod_tensor_arr) { + garbages.emplace_back(t.MoveMemoryHolder()); + } + } else { + PADDLE_THROW("Type %s of %s is not supported eager deletion", + framework::ToTypeName(var->Type()), var_name); + } + } + + if (!garbages.empty()) { + gc->Add(std::move(garbages)); + } +} + +} // namespace framework +} // namespace paddle diff --git a/paddle/fluid/framework/executor_gc_helper.h b/paddle/fluid/framework/executor_gc_helper.h new file mode 100644 index 000000000..6a71273b5 --- /dev/null +++ b/paddle/fluid/framework/executor_gc_helper.h @@ -0,0 +1,40 @@ +// Copyright (c) 2019 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. + +#pragma once + +#include +#include +#include +#include +#include "paddle/fluid/framework/garbage_collector.h" +#include "paddle/fluid/framework/operator.h" +#include "paddle/fluid/framework/scope.h" + +namespace paddle { +namespace framework { + +std::unordered_map> GetUnusedVars( + const BlockDesc &block, + const std::vector> &ops, + const std::vector &skip_vars); + +void DeleteUnusedTensors( + const Scope &scope, OperatorBase *op, + const std::unordered_map> + &delete_vars_map, + GarbageCollector *gc); + +} // namespace framework +} // namespace paddle diff --git a/paddle/fluid/framework/no_need_buffer_vars_inference.cc b/paddle/fluid/framework/no_need_buffer_vars_inference.cc new file mode 100644 index 000000000..747d82ed7 --- /dev/null +++ b/paddle/fluid/framework/no_need_buffer_vars_inference.cc @@ -0,0 +1,23 @@ +// Copyright (c) 2019 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 "paddle/fluid/framework/no_need_buffer_vars_inference.h" +#include +#include + +namespace paddle { +namespace framework { +// Reserve empty source file +} // namespace framework +} // namespace paddle diff --git a/paddle/fluid/framework/no_need_buffer_vars_inference.h b/paddle/fluid/framework/no_need_buffer_vars_inference.h new file mode 100644 index 000000000..2c9336598 --- /dev/null +++ b/paddle/fluid/framework/no_need_buffer_vars_inference.h @@ -0,0 +1,60 @@ +// Copyright (c) 2019 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. + +#pragma once + +#include +#include +#include +#include "paddle/fluid/framework/op_desc.h" + +namespace paddle { +namespace framework { + +class NoNeedBufferVarsInference { + public: + NoNeedBufferVarsInference(const VariableNameMap &inputs, + const VariableNameMap &outputs, + const AttributeMap &attrs) + : inputs_(inputs), outputs_(outputs), attrs_(attrs) {} + + virtual ~NoNeedBufferVarsInference() = default; + + const VariableNameMap &Inputs() const { return inputs_; } + + const VariableNameMap &Outputs() const { return outputs_; } + + const AttributeMap &Attrs() const { return attrs_; } + + virtual std::unordered_set operator()() const = 0; + + private: + const VariableNameMap &inputs_; + const VariableNameMap &outputs_; + const AttributeMap &attrs_; +}; + +#define DECLARE_NO_NEED_BUFFER_VARS_INFERENCE(class_type, ...) \ + class class_type : public ::paddle::framework::NoNeedBufferVarsInference { \ + public: \ + using ::paddle::framework::NoNeedBufferVarsInference:: \ + NoNeedBufferVarsInference; \ + \ + std::unordered_set operator()() const override { \ + return {__VA_ARGS__}; \ + } \ + } + +} // namespace framework +} // namespace paddle diff --git a/paddle/fluid/framework/op_info.h b/paddle/fluid/framework/op_info.h index 4b55bd070..e200d188b 100644 --- a/paddle/fluid/framework/op_info.h +++ b/paddle/fluid/framework/op_info.h @@ -19,6 +19,7 @@ limitations under the License. */ #include #include "paddle/fluid/framework/attribute.h" +#include "paddle/fluid/framework/no_need_buffer_vars_inference.h" #include "paddle/fluid/framework/type_defs.h" #include "paddle/fluid/platform/macros.h" @@ -39,6 +40,7 @@ struct OpInfo { InferVarTypeFN infer_var_type_; InferShapeFN infer_shape_; InferInplaceOpFN infer_inplace_; + InferNoNeedBufferVarsFN infer_no_need_buffer_vars_; bool HasOpProtoAndChecker() const { return proto_ != nullptr && checker_ != nullptr; @@ -64,6 +66,10 @@ struct OpInfo { } const OpAttrChecker* Checker() const { return checker_; } + + const InferNoNeedBufferVarsFN& NoNeedBufferVarsInferer() const { + return infer_no_need_buffer_vars_; + } }; class OpInfoMap { diff --git a/paddle/fluid/framework/operator.cc b/paddle/fluid/framework/operator.cc index 1ba2bed88..2581fc1ca 100644 --- a/paddle/fluid/framework/operator.cc +++ b/paddle/fluid/framework/operator.cc @@ -18,6 +18,7 @@ limitations under the License. */ #include #include #include +#include #include #include "paddle/fluid/framework/data_transform.h" #include "paddle/fluid/framework/executor.h" @@ -326,7 +327,12 @@ OperatorBase::OperatorBase(const std::string& type, const VariableNameMap& inputs, const VariableNameMap& outputs, const AttributeMap& attrs) - : type_(type), inputs_(inputs), outputs_(outputs), attrs_(attrs) { + : type_(type), + inputs_(inputs), + outputs_(outputs), + attrs_(attrs), + // NOTE(zjl): why op_info may be nullptr? + info_(OpInfoMap::Instance().GetNullable(type)) { GenerateTemporaryNames(); CheckAllInputOutputSet(); } @@ -350,7 +356,7 @@ std::vector OperatorBase::OutputVars(bool has_intermediate) const { } return ret_val; } - auto& info = OpInfoMap::Instance().Get(Type()); + auto& info = Info(); // get all OpProto::Var for outputs for (auto& o : info.Proto().outputs()) { @@ -366,18 +372,16 @@ std::vector OperatorBase::OutputVars(bool has_intermediate) const { } void OperatorBase::CheckAllInputOutputSet() const { - auto& info_map = OpInfoMap::Instance(); - auto* op_info = info_map.GetNullable(Type()); - if (op_info == nullptr || op_info->proto_ == nullptr) return; + if (info_ == nullptr || info_->proto_ == nullptr) return; - for (auto& in : op_info->Proto().inputs()) { + for (auto& in : info_->Proto().inputs()) { if (!in.dispensable()) { PADDLE_ENFORCE(inputs_.find(in.name()) != inputs_.end(), "Operator %s's input, %s, is not set", Type(), in.name()); } } - for (auto& out : op_info->Proto().outputs()) { + for (auto& out : info_->Proto().outputs()) { if (!out.dispensable()) { PADDLE_ENFORCE(outputs_.find(out.name()) != outputs_.end(), "Operator %s's output, %s, is not set", Type(), @@ -997,7 +1001,25 @@ Scope* OperatorWithKernel::PrepareData( std::vector* transfered_inplace_vars, RuntimeContext* ctx) const { Scope* new_scope = nullptr; + + std::unordered_set no_buffer_ins; + if (info_) { + auto& no_buffer_inferer = info_->NoNeedBufferVarsInferer(); + // Some op may not register NoNeedBufferVarsInferer + if (no_buffer_inferer) { + no_buffer_ins = no_buffer_inferer(Inputs(), Outputs(), Attrs()); + } + } + for (auto& var_name_item : Inputs()) { + // NOTE(zjl): STL does not guarantee fast std::unordered_set::count when set + // is empty. At least STL implemented on my mac does calculate hash code + // of search key even though the set is empty. + if (!no_buffer_ins.empty() && + no_buffer_ins.count(var_name_item.first) > 0) { + continue; + } + std::vector& input_vars = ctx->inputs[var_name_item.first]; for (size_t i = 0; i < var_name_item.second.size(); ++i) { diff --git a/paddle/fluid/framework/operator.h b/paddle/fluid/framework/operator.h index 684960c23..a02e53dcf 100644 --- a/paddle/fluid/framework/operator.h +++ b/paddle/fluid/framework/operator.h @@ -160,6 +160,11 @@ class OperatorBase { const VariableNameMap& Inputs() const { return inputs_; } const VariableNameMap& Outputs() const { return outputs_; } + const OpInfo& Info() const { + PADDLE_ENFORCE_NOT_NULL(info_, "OpInfo of %s is not found", type_); + return *info_; + } + bool HasInputs(const std::string& name) const; //! Get a input with argument's name described in `op_proto` std::string Input(const std::string& name) const; @@ -194,6 +199,10 @@ class OperatorBase { // IG (Inputs Gradients) VariableNameMap outputs_; AttributeMap attrs_; + + // OpInfo + const OpInfo* info_; + // Whether this operator executes in an Executor. bool run_by_executor_{true}; @@ -444,7 +453,7 @@ class OperatorWithKernel : public OperatorBase { } virtual void InferShape(InferShapeContext* ctx) const { - OpInfoMap::Instance().Get(Type()).infer_shape_(ctx); + Info().infer_shape_(ctx); } void RuntimeInferShape(const Scope& scope, const platform::Place& place, diff --git a/paddle/fluid/framework/type_defs.h b/paddle/fluid/framework/type_defs.h index f55520901..db8a3baca 100644 --- a/paddle/fluid/framework/type_defs.h +++ b/paddle/fluid/framework/type_defs.h @@ -30,6 +30,7 @@ class InferShapeContext; class InferVarTypeContext; class BlockDesc; class Variable; +class NoNeedBufferVarsInference; using VariableNameMap = std::map>; // TODO(panyx0718): Replace vector with something like gtl::Vector. @@ -61,5 +62,9 @@ using InferShapeFN = std::function; using InplacePair = std::unordered_map; using InferInplaceOpFN = std::function; +using InferNoNeedBufferVarsFN = std::function( + const VariableNameMap& /*inputs*/, const VariableNameMap& /*outputs*/, + const AttributeMap& /*attrs*/)>; + } // namespace framework } // namespace paddle diff --git a/paddle/fluid/operators/clip_op.cc b/paddle/fluid/operators/clip_op.cc index a679f7e25..4fc6ae365 100644 --- a/paddle/fluid/operators/clip_op.cc +++ b/paddle/fluid/operators/clip_op.cc @@ -13,6 +13,7 @@ See the License for the specific language governing permissions and limitations under the License. */ #include "paddle/fluid/operators/clip_op.h" +#include namespace paddle { namespace operators { @@ -76,12 +77,28 @@ class ClipOpGrad : public framework::OperatorWithKernel { } }; +class ClipGradOpDescMaker : public framework::SingleGradOpDescMaker { + public: + using framework::SingleGradOpDescMaker::SingleGradOpDescMaker; + + protected: + std::unique_ptr Apply() const override { + std::unique_ptr op(new framework::OpDesc()); + op->SetType("clip_grad"); + op->SetInput("X", Input("X")); + op->SetInput(framework::GradVarName("Out"), OutputGrad("Out")); + op->SetOutput(framework::GradVarName("X"), InputGrad("X")); + op->SetAttrMap(Attrs()); + return op; + } +}; + } // namespace operators } // namespace paddle namespace ops = paddle::operators; REGISTER_OPERATOR(clip, ops::ClipOp, ops::ClipOpMaker, - paddle::framework::DefaultGradOpDescMaker); + ops::ClipGradOpDescMaker); REGISTER_OPERATOR(clip_grad, ops::ClipOpGrad); REGISTER_OP_CPU_KERNEL( clip, ops::ClipKernel); diff --git a/paddle/fluid/operators/elementwise/elementwise_add_op.cc b/paddle/fluid/operators/elementwise/elementwise_add_op.cc index c6c658236..2b3fc06dc 100644 --- a/paddle/fluid/operators/elementwise/elementwise_add_op.cc +++ b/paddle/fluid/operators/elementwise/elementwise_add_op.cc @@ -16,8 +16,7 @@ limitations under the License. */ #include "paddle/fluid/operators/elementwise/elementwise_op.h" namespace ops = paddle::operators; REGISTER_ELEMWISE_GRAD_MAKER(elementwise_add, Add); -REGISTER_ELEMWISE_EXPLICIT_OP(elementwise_add, "Add", "Out = X + Y", "Out", - "X"); +REGISTER_ELEMWISE_EXPLICIT_OP(elementwise_add, "Add", "Out = X + Y"); REGISTER_OP_CPU_KERNEL( elementwise_add, diff --git a/paddle/fluid/operators/elementwise/elementwise_op.h b/paddle/fluid/operators/elementwise/elementwise_op.h index 91e441526..b66498794 100644 --- a/paddle/fluid/operators/elementwise/elementwise_op.h +++ b/paddle/fluid/operators/elementwise/elementwise_op.h @@ -14,7 +14,9 @@ limitations under the License. */ #pragma once +#include #include +#include #include "paddle/fluid/framework/data_layout.h" #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/framework/operator.h" @@ -281,12 +283,11 @@ class ElementwiseGradOpInplace : public framework::InplaceInToOut { } }; +DECLARE_NO_NEED_BUFFER_VARS_INFERENCE(ElementwiseGradNoBufVarsInference, "Y"); + } // namespace operators } // namespace paddle -/* -*/ - #define REGISTER_ELEMWISE_GRAD_MAKER(kernel_type, op_name) \ class kernel_type##GradMaker \ : public paddle::framework::SingleGradOpDescMaker { \ @@ -320,18 +321,19 @@ class ElementwiseGradOpInplace : public framework::InplaceInToOut { ::paddle::framework::DefaultGradOpDescMaker); \ REGISTER_OPERATOR(op_type##_grad, ::paddle::operators::ElementwiseOpGrad) -#define REGISTER_ELEMWISE_EXPLICIT_OP(op_type, op_name, equation, ...) \ - class __ElemwiseOp##op_type##Maker__ \ - : public ::paddle::operators::ElementwiseOpMaker { \ - protected: \ - virtual std::string GetName() const { return op_name; } \ - virtual std::string GetEquation() const { return equation; } \ - }; \ - REGISTER_OPERATOR(op_type, ::paddle::operators::ElementwiseOp, \ - __ElemwiseOp##op_type##Maker__, \ - ::paddle::operators::ElementwiseOpInferVarType, \ - op_type##GradMaker, \ - ::paddle::operators::ElementwiseOpInplace); \ - REGISTER_OPERATOR(op_type##_grad, \ - ::paddle::operators::ElementwiseOpExplicitGrad, \ - ::paddle::operators::ElementwiseGradOpInplace) +#define REGISTER_ELEMWISE_EXPLICIT_OP(op_type, op_name, equation) \ + class __ElemwiseOp##op_type##Maker__ \ + : public ::paddle::operators::ElementwiseOpMaker { \ + protected: \ + virtual std::string GetName() const { return op_name; } \ + virtual std::string GetEquation() const { return equation; } \ + }; \ + REGISTER_OPERATOR(op_type, ::paddle::operators::ElementwiseOp, \ + __ElemwiseOp##op_type##Maker__, \ + ::paddle::operators::ElementwiseOpInferVarType, \ + op_type##GradMaker, \ + ::paddle::operators::ElementwiseOpInplace); \ + REGISTER_OPERATOR(op_type##_grad, \ + ::paddle::operators::ElementwiseOpExplicitGrad, \ + ::paddle::operators::ElementwiseGradOpInplace, \ + ::paddle::operators::ElementwiseGradNoBufVarsInference) diff --git a/paddle/fluid/operators/elementwise/elementwise_sub_op.cc b/paddle/fluid/operators/elementwise/elementwise_sub_op.cc index efc66374c..04c87c1b2 100644 --- a/paddle/fluid/operators/elementwise/elementwise_sub_op.cc +++ b/paddle/fluid/operators/elementwise/elementwise_sub_op.cc @@ -16,8 +16,7 @@ limitations under the License. */ #include "paddle/fluid/operators/elementwise/elementwise_op.h" namespace ops = paddle::operators; REGISTER_ELEMWISE_GRAD_MAKER(elementwise_sub, Sub); -REGISTER_ELEMWISE_EXPLICIT_OP(elementwise_sub, "Sub", "Out = X - Y", "Out", - "X"); +REGISTER_ELEMWISE_EXPLICIT_OP(elementwise_sub, "Sub", "Out = X - Y"); REGISTER_OP_CPU_KERNEL( elementwise_sub, diff --git a/paddle/fluid/pybind/pybind.cc b/paddle/fluid/pybind/pybind.cc index a57083a14..5a6766d1b 100644 --- a/paddle/fluid/pybind/pybind.cc +++ b/paddle/fluid/pybind/pybind.cc @@ -274,6 +274,8 @@ PYBIND11_MODULE(core, m) { py::class_(m, "Tensor", py::buffer_protocol()) .def_buffer( [](Tensor &self) -> py::buffer_info { return CastToPyBuffer(self); }) + .def("_is_initialized", + [](const Tensor &self) { return self.IsInitialized(); }) .def("_get_dims", [](const Tensor &self) { return vectorize(self.dims()); }) .def("_set_dims", @@ -666,7 +668,8 @@ All parameter, weight, gradient are variables in Paddle. .def("drop_kids", &Scope::DropKids, R"DOC( Delete all sub-scopes of the current scope. - )DOC"); + )DOC") + .def("_kids", &Scope::kids); m.def("Scope", []() -> Scope * { diff --git a/python/paddle/fluid/tests/unittests/test_eager_deletion_delete_vars.py b/python/paddle/fluid/tests/unittests/test_eager_deletion_delete_vars.py new file mode 100644 index 000000000..effd0d31b --- /dev/null +++ b/python/paddle/fluid/tests/unittests/test_eager_deletion_delete_vars.py @@ -0,0 +1,165 @@ +# Copyright (c) 2019 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. + +import os +import numpy as np +os.environ['FLAGS_eager_delete_tensor_gb'] = '0.0' +os.environ['CPU_NUM'] = '4' + +import paddle.fluid as fluid +import six +import unittest +import multiprocessing + + +def simple_fc_net(): + image = fluid.layers.data(name='image', shape=[784], dtype='float32') + label = fluid.layers.data(name='label', shape=[1], dtype='int64') + hidden = image + for _ in range(4): + hidden = fluid.layers.fc( + hidden, + size=200, + act='tanh', + bias_attr=fluid.ParamAttr( + initializer=fluid.initializer.Constant(value=1.0))) + prediction = fluid.layers.fc(hidden, size=10, act='softmax') + loss = fluid.layers.cross_entropy(input=prediction, label=label) + loss = fluid.layers.mean(loss) + optimizer = fluid.optimizer.Adam(learning_rate=1e-3) + optimizer.minimize(loss) + return image, label, loss + + +def get_persistables_and_non_persistables(prog, fetch_list): + num_block = prog.num_blocks + persitables = set() + non_persistables = set() + for bid in six.moves.range(num_block): + block = prog.block(bid) + for _, var in block.vars.items(): + if var.persistable or var.name in fetch_list: + persitables.add(var.name) + else: + non_persistables.add(var.name) + + return persitables, non_persistables + + +class TestExecutor(unittest.TestCase): + def setUp(self): + self.place = fluid.CPUPlace() + + def test_executor_main(self): + with fluid.program_guard(fluid.Program(), fluid.Program()): + with fluid.scope_guard(fluid.Scope()): + self.executor_main() + + def test_parallel_executor_main(self): + with fluid.program_guard(fluid.Program(), fluid.Program()): + with fluid.scope_guard(fluid.Scope()): + self.pe_main() + + def prepare_feed(self, image, label, dev_cnt=1): + batch_size = 32 * dev_cnt + image_shape = (batch_size, ) + tuple(image.shape[1:]) + label_shape = (batch_size, ) + tuple(label.shape[1:]) + + image_np = np.random.random(size=image_shape).astype('float32') + label_np = np.random.random_integers( + low=0, high=9, size=label_shape).astype('int64') + + return image_np, label_np + + def assertScopeVar(self, scope, persitables, non_persistables): + for name in persitables: + var = scope.find_var(name) + self.assertTrue(var is not None) + t = var.get_tensor() + self.assertTrue(t._is_initialized()) + + for name in non_persistables: + var = scope.find_var(name) + self.assertTrue(var is not None) + t = var.get_tensor() + if t._is_initialized(): + print('WARNING: Variable {} is alive'.format(name)) + self.assertTrue(not t._is_initialized()) + + def executor_main(self): + image, label, loss = simple_fc_net() + loss.persistable = False + persistables, non_persistables = get_persistables_and_non_persistables( + fluid.default_main_program(), [loss.name]) + + exe = fluid.Executor(self.place) + exe.run(fluid.default_startup_program()) + + p = fluid.core.Place() + p.set_place(self.place) + exe = fluid.core.Executor(p) + + for _ in six.moves.range(10): + image_np, label_np = self.prepare_feed(image, label) + fluid.global_scope().var(image.name).get_tensor().set(image_np, + self.place) + fluid.global_scope().var(label.name).get_tensor().set(label_np, + self.place) + # exe.run would not create local scope + # so that we can detect whether gc clears temporary variables + exe.run(fluid.default_main_program().desc, + fluid.global_scope(), 0, False, True, [loss.name]) + self.assertScopeVar(fluid.global_scope(), persistables, + non_persistables) + + def pe_main(self): + image, label, loss = simple_fc_net() + loss.persistable = False + persitables, non_persistables = get_persistables_and_non_persistables( + fluid.default_main_program(), [loss.name]) + + exe = fluid.Executor(self.place) + exe.run(fluid.default_startup_program()) + + exec_strategy = fluid.ExecutionStrategy() + exec_strategy.num_iteration_per_drop_scope = 100 + + prog = fluid.CompiledProgram(fluid.default_main_program( + )).with_data_parallel( + loss_name=loss.name, exec_strategy=exec_strategy) + + dev_cnt = fluid.core.get_cuda_device_count() if isinstance(self.place, fluid.CUDAPlace) \ + else int(os.environ.get('CPU_NUM', multiprocessing.cpu_count())) + + for idx in six.moves.range(10): + image_np, label_np = self.prepare_feed(image, label, dev_cnt) + feed = {image.name: image_np, label.name: label_np} + + exe.run(program=prog, feed=feed, fetch_list=[loss]) + + local_scopes = prog._local_scopes + for scope in local_scopes: + kids = scope._kids() + self.assertTrue(len(kids) == 1) + self.assertScopeVar(kids[0], persistables, non_persistables) + + +class TestExecutor2(TestExecutor): + def setUp(self): + self.place = fluid.CPUPlace() if not fluid.core.is_compiled_with_cuda() \ + else fluid.CUDAPlace(0) + + +if __name__ == '__main__': + unittest.main() -- GitLab