diff --git a/paddle/fluid/framework/CMakeLists.txt b/paddle/fluid/framework/CMakeLists.txt index dbd375aa31bfbdcb109b6302acf23b3bb3b6befe..627370cd2df7317b4d32aa967565aaf9cf0c7a08 100644 --- a/paddle/fluid/framework/CMakeLists.txt +++ b/paddle/fluid/framework/CMakeLists.txt @@ -87,7 +87,7 @@ cc_library(executor SRCS executor.cc DEPS op_registry device_context scope framework_proto glog lod_rank_table feed_fetch_method) -cc_library(parallel_executor SRCS parallel_executor.cc DEPS multi_devices_graph_builder threaded_ssa_graph_executor scope_buffered_ssa_graph_executor) +cc_library(parallel_executor SRCS parallel_executor.cc DEPS graph_builder_factory threaded_ssa_graph_executor scope_buffered_ssa_graph_executor) cc_library(prune SRCS prune.cc DEPS framework_proto) cc_test(prune_test SRCS prune_test.cc DEPS op_info prune recurrent_op device_context) diff --git a/paddle/fluid/framework/details/CMakeLists.txt b/paddle/fluid/framework/details/CMakeLists.txt index c026e6c100a303b43650f08cd12d7260258c8f7e..c106761f72e689ff53867ecad8e36b6038173d0e 100644 --- a/paddle/fluid/framework/details/CMakeLists.txt +++ b/paddle/fluid/framework/details/CMakeLists.txt @@ -7,6 +7,7 @@ cc_library(rpc_op_handle SRCS rpc_op_handle.cc DEPS framework_proto scope place cc_library(ssa_graph SRCS ssa_graph.cc DEPS var_handle op_handle_base) cc_library(ssa_graph_builder SRCS ssa_graph_builder.cc DEPS ssa_graph) +cc_library(ssa_graph_printer SRCS ssa_graph_printer.cc DEPS ssa_graph_builder) cc_library(variable_visitor SRCS variable_visitor.cc DEPS lod_tensor selected_rows) @@ -28,6 +29,9 @@ cc_library(gather_op_handle SRCS gather_op_handle.cc DEPS op_handle_base scope d cc_library(multi_devices_graph_builder SRCS multi_devices_graph_builder.cc DEPS ssa_graph_builder computation_op_handle scale_loss_grad_op_handle rpc_op_handle ${multi_devices_graph_builder_deps} reduce_op_handle broadcast_op_handle) + +cc_library(graph_builder_factory SRCS graph_builder_factory.cc DEPS multi_devices_graph_builder ssa_graph_printer) + cc_library(ssa_graph_executor SRCS ssa_graph_executor.cc DEPS ssa_graph framework_proto) cc_library(threaded_ssa_graph_executor SRCS threaded_ssa_graph_executor.cc DEPS fetch_op_handle ssa_graph_executor scope simple_threadpool device_context) diff --git a/paddle/fluid/framework/details/broadcast_op_handle.h b/paddle/fluid/framework/details/broadcast_op_handle.h index 629aa00cb817c4b1446e7b750ca62a7c6b1db670..8036f756b6d6506684c109ab881d546f38176a10 100644 --- a/paddle/fluid/framework/details/broadcast_op_handle.h +++ b/paddle/fluid/framework/details/broadcast_op_handle.h @@ -59,8 +59,8 @@ struct BroadcastOpHandle : public OpHandleBase { void RunImpl() override; private: - const std::vector &local_scopes_; - const std::vector &places_; + std::vector local_scopes_; + std::vector places_; #ifdef PADDLE_WITH_CUDA const platform::NCCLContextMap *nccl_ctxs_; #endif diff --git a/paddle/fluid/framework/details/build_strategy.h b/paddle/fluid/framework/details/build_strategy.h index 91bdfe6134ffbd1404336c9d6d1222a505084b2b..64e83acb4dc1995800c4ca3caf81668b24a7c9fe 100644 --- a/paddle/fluid/framework/details/build_strategy.h +++ b/paddle/fluid/framework/details/build_strategy.h @@ -14,6 +14,8 @@ #pragma once +#include + namespace paddle { namespace framework { namespace details { @@ -29,6 +31,8 @@ struct BuildStrategy { ReduceStrategy reduce_{ReduceStrategy::kAllReduce}; GradientScaleStrategy gradient_scale_{GradientScaleStrategy::kCoeffNumDevice}; + + std::string debug_graphviz_path_{""}; }; } // namespace details diff --git a/paddle/fluid/framework/details/graph_builder_factory.cc b/paddle/fluid/framework/details/graph_builder_factory.cc new file mode 100644 index 0000000000000000000000000000000000000000..a04b9bb63c06b40ff5c30c9792cdfad5d64d404c --- /dev/null +++ b/paddle/fluid/framework/details/graph_builder_factory.cc @@ -0,0 +1,47 @@ +// 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. + +#include "paddle/fluid/framework/details/graph_builder_factory.h" +#include +#include "paddle/fluid/framework/details/multi_devices_graph_builder.h" +#include "paddle/fluid/framework/details/ssa_graph_printer.h" + +namespace paddle { +namespace framework { +namespace details { +std::unique_ptr SSAGraphBuilderFactory::Create() { + std::unique_ptr res( +#ifdef PADDLE_WITH_CUDA + new MultiDevSSAGraphBuilder(places_, loss_var_name_, param_names_, + local_scopes_, nccl_ctxs_, strategy_) +#else + new MultiDevSSAGraphBuilder(places_, loss_var_name_, param_names_, + local_scopes_, strategy_) +#endif + ); // NOLINT + + if (!strategy_.debug_graphviz_path_.empty()) { + std::unique_ptr fout( + new std::ofstream(strategy_.debug_graphviz_path_)); + PADDLE_ENFORCE(fout->good()); + std::unique_ptr graphviz_printer( + new GraphvizSSAGraphPrinter()); + res.reset(new SSAGraghBuilderWithPrinter( + std::move(fout), std::move(graphviz_printer), std::move(res))); + } + return res; +} +} // namespace details +} // namespace framework +} // namespace paddle diff --git a/paddle/fluid/framework/details/graph_builder_factory.h b/paddle/fluid/framework/details/graph_builder_factory.h new file mode 100644 index 0000000000000000000000000000000000000000..857ab12d684e19788597e144fc0c46571d06aafc --- /dev/null +++ b/paddle/fluid/framework/details/graph_builder_factory.h @@ -0,0 +1,67 @@ +// 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 +#include "paddle/fluid/framework/details/build_strategy.h" +#include "paddle/fluid/framework/details/ssa_graph_builder.h" +#include "paddle/fluid/platform/place.h" + +#ifdef PADDLE_WITH_CUDA +#include "paddle/fluid/platform/nccl_helper.h" +#endif + +namespace paddle { +namespace framework { +class Scope; +namespace details { + +class SSAGraphBuilderFactory { + public: + SSAGraphBuilderFactory(const std::vector& places, + const std::string& loss_var_name, + const std::unordered_set& param_names, + const std::vector& local_scopes, + const BuildStrategy& strategy) + : places_(places), + loss_var_name_(loss_var_name), + param_names_(param_names), + local_scopes_(local_scopes), + strategy_(strategy) {} + +#ifdef PADDLE_WITH_CUDA + void SetNCCLContextMap(platform::NCCLContextMap* nccl_ctxs) { + nccl_ctxs_ = nccl_ctxs; + } +#endif + + std::unique_ptr Create(); + + private: + std::vector places_; + std::string loss_var_name_; + std::unordered_set param_names_; + std::vector local_scopes_; + BuildStrategy strategy_; + +#ifdef PADDLE_WITH_CUDA + platform::NCCLContextMap* nccl_ctxs_; +#endif +}; + +} // namespace details +} // namespace framework +} // namespace paddle diff --git a/paddle/fluid/framework/details/multi_devices_graph_builder.cc b/paddle/fluid/framework/details/multi_devices_graph_builder.cc index 17baacd13eecac8f410631fe9e94788da4fff848..0c4d369e889cf2cca7722dac14a5268fdacabeb4 100644 --- a/paddle/fluid/framework/details/multi_devices_graph_builder.cc +++ b/paddle/fluid/framework/details/multi_devices_graph_builder.cc @@ -30,10 +30,6 @@ #include "paddle/fluid/framework/details/nccl_all_reduce_op_handle.h" #endif -DEFINE_string(ssa_graph_path, "/tmp/ssa_graph.dot", - "the ssa graph path only print with GLOG_v=10," - "default /tmp/graph.dot"); - namespace paddle { namespace framework { namespace details { @@ -277,11 +273,6 @@ std::unique_ptr MultiDevSSAGraphBuilder::Build( */ AddOutputToLeafOps(&result); - if (VLOG_IS_ON(10)) { - std::ofstream fout(FLAGS_ssa_graph_path); - PrintGraphviz(*graph, fout); - } - return std::unique_ptr(graph); } diff --git a/paddle/fluid/framework/details/nccl_all_reduce_op_handle.h b/paddle/fluid/framework/details/nccl_all_reduce_op_handle.h index a0c321843e3fc5abcbd1ef2ce2e153250269aa7d..8e98d894b828b4162059b30f5c6a74cfc06f402e 100644 --- a/paddle/fluid/framework/details/nccl_all_reduce_op_handle.h +++ b/paddle/fluid/framework/details/nccl_all_reduce_op_handle.h @@ -41,8 +41,8 @@ struct NCCLAllReduceOpHandle : public OpHandleBase { void RunImpl() override; private: - const std::vector &local_scopes_; - const std::vector &places_; + std::vector local_scopes_; + std::vector places_; const platform::NCCLContextMap &nccl_ctxs_; }; diff --git a/paddle/fluid/framework/details/reduce_op_handle.h b/paddle/fluid/framework/details/reduce_op_handle.h index c652a2f4eb0f9b73cb19ebbd9d0809210b280ad3..4d14334cdfe06e2e805c2577458d6689e6324cc7 100644 --- a/paddle/fluid/framework/details/reduce_op_handle.h +++ b/paddle/fluid/framework/details/reduce_op_handle.h @@ -32,8 +32,8 @@ namespace framework { namespace details { struct ReduceOpHandle : public OpHandleBase { - const std::vector &local_scopes_; - const std::vector &places_; + std::vector local_scopes_; + std::vector places_; #ifdef PADDLE_WITH_CUDA const platform::NCCLContextMap *nccl_ctxs_; diff --git a/paddle/fluid/framework/details/ssa_graph_builder.cc b/paddle/fluid/framework/details/ssa_graph_builder.cc index 6a567527550883add08031e50aa8de2b204cf13d..211113c7979ee95d896c0a57879f7b3ad13b36ef 100644 --- a/paddle/fluid/framework/details/ssa_graph_builder.cc +++ b/paddle/fluid/framework/details/ssa_graph_builder.cc @@ -73,64 +73,6 @@ void SSAGraphBuilder::CreateOpOutput(SSAGraph *graph, OpHandleBase *op_handle, op_handle->AddOutput(var); } -template -void IterAllVar(const SSAGraph &graph, Callback callback) { - for (auto &each : graph.vars_) { - for (auto &pair1 : each) { - for (auto &pair2 : pair1.second) { - callback(*pair2); - } - } - } - - for (auto &var : graph.dep_vars_) { - callback(*var); - } -} - -void SSAGraphBuilder::PrintGraphviz(const SSAGraph &graph, std::ostream &sout) { - size_t var_id = 0; - std::unordered_map vars; - - sout << "digraph G {\n"; - - IterAllVar(graph, [&](const VarHandleBase &var) { - auto *var_ptr = &var; - auto *var_handle_ptr = dynamic_cast(var_ptr); - auto *dummy_ptr = dynamic_cast(var_ptr); - - size_t cur_var_id = var_id++; - vars[var_ptr] = cur_var_id; - - if (var_handle_ptr) { - sout << "var_" << cur_var_id << " [label=\"" << var_handle_ptr->name_ - << "\\n" - << var_handle_ptr->place_ << "\\n" - << var_handle_ptr->version_ << "\"]" << std::endl; - } else if (dummy_ptr) { - sout << "var_" << cur_var_id << " [label=\"dummy\"]" << std::endl; - } - }); - - size_t op_id = 0; - for (auto &op : graph.ops_) { - std::string op_name = "op_" + std::to_string(op_id++); - sout << op_name << " [label=\"" << op->Name() << "\", shape=rect]" - << std::endl; - for (auto in : op->Inputs()) { - std::string var_name = "var_" + std::to_string(vars[in]); - sout << var_name << " -> " << op_name << std::endl; - } - - for (auto out : op->Outputs()) { - std::string var_name = "var_" + std::to_string(vars[out]); - sout << op_name << " -> " << var_name << std::endl; - } - } - - sout << "}\n"; -} - void SSAGraphBuilder::AddOutputToLeafOps(SSAGraph *graph) { for (auto &op : graph->ops_) { if (!op->Outputs().empty()) { diff --git a/paddle/fluid/framework/details/ssa_graph_builder.h b/paddle/fluid/framework/details/ssa_graph_builder.h index 64e5d93081eb76c56898bbeb530e37364619fdbb..5fc12a44b51fae26e5a8f5fdba952d3879e82d0f 100644 --- a/paddle/fluid/framework/details/ssa_graph_builder.h +++ b/paddle/fluid/framework/details/ssa_graph_builder.h @@ -55,8 +55,6 @@ class SSAGraphBuilder { const platform::Place &place, size_t place_offset); static void AddOutputToLeafOps(SSAGraph *graph); - - static void PrintGraphviz(const SSAGraph &graph, std::ostream &sout); }; } // namespace details } // namespace framework diff --git a/paddle/fluid/framework/details/ssa_graph_printer.cc b/paddle/fluid/framework/details/ssa_graph_printer.cc new file mode 100644 index 0000000000000000000000000000000000000000..22a40ca4b25cdd8ed9856b6c71bffc79561edcac --- /dev/null +++ b/paddle/fluid/framework/details/ssa_graph_printer.cc @@ -0,0 +1,83 @@ +// 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. + +#include "paddle/fluid/framework/details/ssa_graph_printer.h" +#include +#include "paddle/fluid/framework/details/ssa_graph.h" + +namespace paddle { +namespace framework { +namespace details { + +template +static inline void IterAllVar(const SSAGraph &graph, Callback callback) { + for (auto &each : graph.vars_) { + for (auto &pair1 : each) { + for (auto &pair2 : pair1.second) { + callback(*pair2); + } + } + } + + for (auto &var : graph.dep_vars_) { + callback(*var); + } +} + +void GraphvizSSAGraphPrinter::Print(const SSAGraph &graph, + std::ostream &sout) const { + size_t var_id = 0; + std::unordered_map vars; + + sout << "digraph G {\n"; + + IterAllVar(graph, [&](const VarHandleBase &var) { + auto *var_ptr = &var; + auto *var_handle_ptr = dynamic_cast(var_ptr); + auto *dummy_ptr = dynamic_cast(var_ptr); + + size_t cur_var_id = var_id++; + vars[var_ptr] = cur_var_id; + + if (var_handle_ptr) { + sout << "var_" << cur_var_id << " [label=\"" << var_handle_ptr->name_ + << "\\n" + << var_handle_ptr->place_ << "\\n" + << var_handle_ptr->version_ << "\"]" << std::endl; + } else if (dummy_ptr) { + sout << "var_" << cur_var_id << " [label=\"dummy\"]" << std::endl; + } + }); + + size_t op_id = 0; + for (auto &op : graph.ops_) { + std::string op_name = "op_" + std::to_string(op_id++); + sout << op_name << " [label=\"" << op->Name() << "\", shape=rect]" + << std::endl; + for (auto in : op->Inputs()) { + std::string var_name = "var_" + std::to_string(vars[in]); + sout << var_name << " -> " << op_name << std::endl; + } + + for (auto out : op->Outputs()) { + std::string var_name = "var_" + std::to_string(vars[out]); + sout << op_name << " -> " << var_name << std::endl; + } + } + + sout << "}\n"; +} +} // namespace details +} // namespace framework +} // namespace paddle diff --git a/paddle/fluid/framework/details/ssa_graph_printer.h b/paddle/fluid/framework/details/ssa_graph_printer.h new file mode 100644 index 0000000000000000000000000000000000000000..5287be3b6a05ec7067ca433ba976b0314d05fe02 --- /dev/null +++ b/paddle/fluid/framework/details/ssa_graph_printer.h @@ -0,0 +1,67 @@ +// 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 "paddle/fluid/framework/details/ssa_graph_builder.h" + +namespace paddle { +namespace framework { +namespace details { +class SSAGraph; +class SSAGraphPrinter { + public: + virtual ~SSAGraphPrinter() {} + virtual void Print(const SSAGraph& graph, std::ostream& sout) const = 0; +}; + +class GraphvizSSAGraphPrinter : public SSAGraphPrinter { + public: + void Print(const SSAGraph& graph, std::ostream& sout) const override; +}; + +class SSAGraghBuilderWithPrinter : public SSAGraphBuilder { + public: + SSAGraghBuilderWithPrinter(std::ostream& sout, + std::unique_ptr&& printer, + std::unique_ptr&& builder) + : printer_(std::move(printer)), + builder_(std::move(builder)), + stream_ref_(sout) {} + + SSAGraghBuilderWithPrinter(std::unique_ptr&& sout, + std::unique_ptr&& printer, + std::unique_ptr&& builder) + : printer_(std::move(printer)), + builder_(std::move(builder)), + stream_ptr_(std::move(sout)), + stream_ref_(*stream_ptr_) {} + + std::unique_ptr Build(const ProgramDesc& program) const override { + auto graph = builder_->Build(program); + printer_->Print(*graph, stream_ref_); + return graph; + } + + private: + std::unique_ptr printer_; + std::unique_ptr builder_; + std::unique_ptr stream_ptr_; + std::ostream& stream_ref_; +}; + +} // namespace details +} // namespace framework +} // namespace paddle diff --git a/paddle/fluid/framework/parallel_executor.cc b/paddle/fluid/framework/parallel_executor.cc index 003304b85af00165d54efbb199be01b2c5106768..ce56f55e4195a0625cd0754152285b80e4282183 100644 --- a/paddle/fluid/framework/parallel_executor.cc +++ b/paddle/fluid/framework/parallel_executor.cc @@ -22,7 +22,7 @@ limitations under the License. */ #include "paddle/fluid/platform/nccl_helper.h" #endif -#include "paddle/fluid/framework/details/multi_devices_graph_builder.h" +#include "paddle/fluid/framework/details/graph_builder_factory.h" #include "paddle/fluid/framework/details/scope_buffered_ssa_graph_executor.h" #include "paddle/fluid/framework/details/threaded_ssa_graph_executor.h" #include "paddle/fluid/platform/profiler.h" @@ -102,22 +102,19 @@ ParallelExecutor::ParallelExecutor( var_infos.back().persistable_ = var->Persistable(); } -// Step 3. Convert main_program to SSA form and dependency graph. Also, insert -// ncclOp -#ifdef PADDLE_WITH_CUDA - details::MultiDevSSAGraphBuilder builder( + // Step 3. Convert main_program to SSA form and dependency graph. Also, insert + // ncclOp + + details::SSAGraphBuilderFactory builder_factory( member_->places_, loss_var_name, params, member_->local_scopes_, - member_->nccl_ctxs_.get(), build_strategy); -#else - details::MultiDevSSAGraphBuilder builder(member_->places_, loss_var_name, - params, member_->local_scopes_, - build_strategy); + build_strategy); +#ifdef PADDLE_WITH_CUDA + builder_factory.SetNCCLContextMap(member_->nccl_ctxs_.get()); #endif - auto graph = builder.Build(main_program); - member_->executor_.reset(new details::ThreadedSSAGraphExecutor( - exec_strategy, member_->local_scopes_, places, std::move(graph))); + exec_strategy, member_->local_scopes_, places, + builder_factory.Create()->Build(main_program))); member_->executor_.reset(new details::ScopeBufferedSSAGraphExecutor( exec_strategy, member_->local_scopes_, std::move(var_infos), diff --git a/paddle/fluid/framework/tensor.cc b/paddle/fluid/framework/tensor.cc index e97ada06f06d0538f17160220e3aa3f4ffc55520..c7286dacf01659f3af0927a71856e5a6496cb877 100644 --- a/paddle/fluid/framework/tensor.cc +++ b/paddle/fluid/framework/tensor.cc @@ -15,5 +15,102 @@ limitations under the License. */ #include "paddle/fluid/framework/tensor.h" namespace paddle { -namespace framework {} +namespace framework { +extern size_t SizeOfType(std::type_index type); +void Tensor::check_memory_size() const { + PADDLE_ENFORCE_NOT_NULL( + holder_, "Tensor holds no memory. Call Tensor::mutable_data first."); + PADDLE_ENFORCE_LE( + numel() * SizeOfType(type()), memory_size(), + "Tensor's dims_ is out of bound. Call Tensor::mutable_data " + "first to re-allocate memory.\n" + "or maybe the required data-type mismatches the data already stored."); +} + +size_t Tensor::memory_size() const { + return holder_ == nullptr ? 0UL : holder_->size() - offset_; +} + +void* Tensor::mutable_data(platform::Place place, std::type_index type) { + if (holder_ != nullptr) { + holder_->set_type(type); + } + PADDLE_ENFORCE_GE(numel(), 0, + "When calling this method, the Tensor's numel must be " + "equal or larger than zero. " + "Please check Tensor::Resize has been called first."); + int64_t size = numel() * SizeOfType(type); + /* some versions of boost::variant don't have operator!= */ + if (holder_ == nullptr || !(holder_->place() == place) || + holder_->size() < size + offset_) { + if (platform::is_cpu_place(place)) { + holder_.reset(new PlaceholderImpl( + boost::get(place), size, type)); + } else if (platform::is_gpu_place(place) || + platform::is_cuda_pinned_place(place)) { +#ifndef PADDLE_WITH_CUDA + PADDLE_THROW( + "CUDAPlace or CUDAPinnedPlace is not supported in CPU-only mode."); + } +#else + if (platform::is_gpu_place(place)) { + holder_.reset(new PlaceholderImpl( + boost::get(place), size, type)); + } else if (platform::is_cuda_pinned_place(place)) { + holder_.reset(new PlaceholderImpl( + boost::get(place), size, type)); + } + } +#endif + offset_ = 0; + } + return reinterpret_cast(reinterpret_cast(holder_->ptr()) + + offset_); +} + +void* Tensor::mutable_data(platform::Place place) { + PADDLE_ENFORCE(this->holder_ != nullptr, + "Cannot invoke mutable data if current hold nothing."); + return mutable_data(place, holder_->type()); +} + +Tensor& Tensor::ShareDataWith(const Tensor& src) { + src.check_memory_size(); + *this = src; + return *this; +} + +Tensor Tensor::Slice(int begin_idx, int end_idx) const { + check_memory_size(); + PADDLE_ENFORCE_GE(begin_idx, 0, + "The start row index must be greater than 0."); + PADDLE_ENFORCE_LE(end_idx, dims_[0], "The end row index is out of bound."); + PADDLE_ENFORCE_LT( + begin_idx, end_idx, + "The start row index must be lesser than the end row index."); + + if (dims_[0] == 1) { + return *this; + } else { + size_t base = numel() / dims_[0]; + Tensor dst; + dst.holder_ = holder_; + dst.set_layout(layout_); + DDim dst_dims = dims_; + dst_dims[0] = end_idx - begin_idx; + dst.Resize(dst_dims); + dst.offset_ = offset_ + begin_idx * base * SizeOfType(type()); + return dst; + } +} + +Tensor& Tensor::Resize(const DDim& dims) { + dims_ = dims; + return *this; +} + +const DDim& Tensor::dims() const { return dims_; } + +int64_t Tensor::numel() const { return product(dims_); } +} // namespace framework } // namespace paddle diff --git a/paddle/fluid/framework/tensor.h b/paddle/fluid/framework/tensor.h index 6f878541e6de1deec1829145b1b325ecd176a034..29566aaa53370b1fffc9ff9a90ae9b740b24f69e 100644 --- a/paddle/fluid/framework/tensor.h +++ b/paddle/fluid/framework/tensor.h @@ -54,26 +54,24 @@ class Tensor { /*! Return a pointer to mutable memory block. */ template - inline T* data(); + T* data(); /*! Return a pointer to constant memory block. */ template - inline const T* data() const; + const T* data() const; - inline bool IsInitialized() const; - - inline void switch_place(platform::Place new_place); + bool IsInitialized() const; /** * @brief Return a pointer to mutable memory block. * @note If not exist, then allocation. */ template - inline T* mutable_data(platform::Place place); + T* mutable_data(platform::Place place); - inline void* mutable_data(platform::Place place, std::type_index type); + void* mutable_data(platform::Place place, std::type_index type); - inline void* mutable_data(platform::Place place); + void* mutable_data(platform::Place place); /** * @brief Return a pointer to mutable memory block. @@ -84,19 +82,19 @@ class Tensor { * @note If not exist, then allocation. */ template - inline T* mutable_data(DDim dims, platform::Place place); + T* mutable_data(DDim dims, platform::Place place); /*! Return the dimensions of the memory block. */ - inline const DDim& dims() const; + const DDim& dims() const; /*! Return the numel of the memory block. */ - inline int64_t numel() const; + int64_t numel() const; /*! Resize the dimensions of the memory block. */ - inline Tensor& Resize(const DDim& dims); + Tensor& Resize(const DDim& dims); /*! The internal of two tensors share the same memory block. */ - inline Tensor& ShareDataWith(const Tensor& src); + Tensor& ShareDataWith(const Tensor& src); /** * @brief Return a sub-tensor of the given tensor. @@ -106,7 +104,7 @@ class Tensor { * @param[in] end_idx The index of the end row(exclusive) to slice. * The index number begins from 0. */ - inline Tensor Slice(int begin_idx, int end_idx) const; + Tensor Slice(int begin_idx, int end_idx) const; platform::Place place() const { PADDLE_ENFORCE_NOT_NULL( @@ -123,11 +121,11 @@ class Tensor { // memory size returns the holding memory size in byte. size_t memory_size() const; - inline void check_memory_size() const; + void check_memory_size() const; - inline DataLayout layout() const { return layout_; } + DataLayout layout() const { return layout_; } - inline void set_layout(const DataLayout layout) { layout_ = layout; } + void set_layout(const DataLayout layout) { layout_ = layout; } private: /** @@ -210,15 +208,6 @@ class Tensor { size_t offset_; }; -inline void Tensor::switch_place(platform::Place new_place) { - if (holder_->place() == new_place) { - return; - } - - // TODO(tonyyang-svail): do memcpy here. - PADDLE_THROW("Not Implemented"); -} - } // namespace framework } // namespace paddle diff --git a/paddle/fluid/framework/tensor_impl.h b/paddle/fluid/framework/tensor_impl.h index 2f19ec0f0a9338e2b96d1f64eac45387bae4d1eb..96114678a9992f2975c4173c7cc003114f04d8df 100644 --- a/paddle/fluid/framework/tensor_impl.h +++ b/paddle/fluid/framework/tensor_impl.h @@ -20,21 +20,6 @@ limitations under the License. */ namespace paddle { namespace framework { -extern size_t SizeOfType(std::type_index type); -inline void Tensor::check_memory_size() const { - PADDLE_ENFORCE_NOT_NULL( - holder_, "Tensor holds no memory. Call Tensor::mutable_data first."); - PADDLE_ENFORCE_LE( - numel() * SizeOfType(type()), memory_size(), - "Tensor's dims_ is out of bound. Call Tensor::mutable_data " - "first to re-allocate memory.\n" - "or maybe the required data-type mismatches the data already stored."); -} - -inline size_t Tensor::memory_size() const { - return holder_ == nullptr ? 0UL : holder_->size() - offset_; -} - template inline const T* Tensor::data() const { check_memory_size(); @@ -73,88 +58,6 @@ inline T* Tensor::mutable_data(platform::Place place) { return reinterpret_cast(mutable_data(place, typeid(T))); } -inline void* Tensor::mutable_data(platform::Place place, std::type_index type) { - if (holder_ != nullptr) { - holder_->set_type(type); - } - PADDLE_ENFORCE_GE(numel(), 0, - "When calling this method, the Tensor's numel must be " - "equal or larger than zero. " - "Please check Tensor::Resize has been called first."); - int64_t size = numel() * SizeOfType(type); - /* some versions of boost::variant don't have operator!= */ - if (holder_ == nullptr || !(holder_->place() == place) || - holder_->size() < size + offset_) { - if (platform::is_cpu_place(place)) { - holder_.reset(new PlaceholderImpl( - boost::get(place), size, type)); - } else if (platform::is_gpu_place(place) || - platform::is_cuda_pinned_place(place)) { -#ifndef PADDLE_WITH_CUDA - PADDLE_THROW( - "CUDAPlace or CUDAPinnedPlace is not supported in CPU-only mode."); - } -#else - if (platform::is_gpu_place(place)) { - holder_.reset(new PlaceholderImpl( - boost::get(place), size, type)); - } else if (platform::is_cuda_pinned_place(place)) { - holder_.reset(new PlaceholderImpl( - boost::get(place), size, type)); - } - } -#endif - offset_ = 0; - } - return reinterpret_cast(reinterpret_cast(holder_->ptr()) + - offset_); -} - -inline void* Tensor::mutable_data(platform::Place place) { - PADDLE_ENFORCE(this->holder_ != nullptr, - "Cannot invoke mutable data if current hold nothing."); - return mutable_data(place, holder_->type()); -} - -inline Tensor& Tensor::ShareDataWith(const Tensor& src) { - src.check_memory_size(); - *this = src; - return *this; -} - -inline Tensor Tensor::Slice(int begin_idx, int end_idx) const { - check_memory_size(); - PADDLE_ENFORCE_GE(begin_idx, 0, - "The start row index must be greater than 0."); - PADDLE_ENFORCE_LE(end_idx, dims_[0], "The end row index is out of bound."); - PADDLE_ENFORCE_LT( - begin_idx, end_idx, - "The start row index must be lesser than the end row index."); - - if (dims_[0] == 1) { - return *this; - } else { - size_t base = numel() / dims_[0]; - Tensor dst; - dst.holder_ = holder_; - dst.set_layout(layout_); - DDim dst_dims = dims_; - dst_dims[0] = end_idx - begin_idx; - dst.Resize(dst_dims); - dst.offset_ = offset_ + begin_idx * base * SizeOfType(type()); - return dst; - } -} - -inline Tensor& Tensor::Resize(const DDim& dims) { - dims_ = dims; - return *this; -} - -inline const DDim& Tensor::dims() const { return dims_; } - -inline int64_t Tensor::numel() const { return product(dims_); } - inline Tensor ReshapeToMatrix(const Tensor& src, int num_col_dims) { Tensor res; res.ShareDataWith(src); diff --git a/paddle/fluid/operators/reverse_op.cc b/paddle/fluid/operators/reverse_op.cc new file mode 100644 index 0000000000000000000000000000000000000000..a20f7d231fa9ea313581ac0629a87fa5f4a88ce5 --- /dev/null +++ b/paddle/fluid/operators/reverse_op.cc @@ -0,0 +1,107 @@ +// 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. + +#include "paddle/fluid/operators/reverse_op.h" +#include + +namespace paddle { +namespace operators { + +class ReverseOp : public framework::OperatorWithKernel { + public: + using framework::OperatorWithKernel::OperatorWithKernel; + + void InferShape(framework::InferShapeContext* ctx) const override { + PADDLE_ENFORCE(ctx->HasInput("X"), "Input(X) should not be null"); + PADDLE_ENFORCE(ctx->HasOutput("Out"), "Output(Out) should not be null"); + const auto& x_dims = ctx->GetInputDim("X"); + const auto& axis = ctx->Attrs().Get>("axis"); + PADDLE_ENFORCE(!axis.empty(), "'axis' can not be empty."); + for (int a : axis) { + PADDLE_ENFORCE_LT(a, x_dims.size(), + "The axis must be less than input tensor's rank."); + } + ctx->SetOutputDim("Out", x_dims); + } +}; + +class ReverseOpMaker : public framework::OpProtoAndCheckerMaker { + public: + void Make() override { + AddInput("X", "The LoDTensor to be flipped."); + AddOutput("Out", "The LoDTensor after flipping."); + AddAttr>( + "axis", "The axises that along which order of elements is reversed."); + AddComment(R"DOC( + Reverse Operator. + + Reverse the order of elements in the input LoDTensor along given axises. + + Case 1: + Given + X = [[1, 2, 3, 4, 5] + [6, 7, 8, 9, 10] + [11, 12, 13, 14, 15]], + and + axis = [0], + we get: + Out = [[11, 12, 13, 14, 15] + [6, 7, 8, 9, 10] + [1, 2, 3, 4, 5]]. + + Case 2: + Given + X = [[[1, 2, 3, 4] + [5, 6, 7, 8]] + [[9, 10, 11, 12] + [13, 14, 15, 16]]], + and + axis = [0, 2], + we get: + Out = [[[12, 11, 10, 9] + [16, 15, 14, 13]] + [[4, 3, 2, 1] + [8, 7, 6, 5]]], + )DOC"); + } +}; + +class ReverseGradMaker : public framework::SingleGradOpDescMaker { + public: + using framework::SingleGradOpDescMaker::SingleGradOpDescMaker; + + std::unique_ptr Apply() const override { + auto* grad_op = new framework::OpDesc(); + grad_op->SetType("reverse"); + grad_op->SetInput("X", OutputGrad("Out")); + grad_op->SetOutput("Out", InputGrad("X")); + grad_op->SetAttr("axis", GetAttr("axis")); + return std::unique_ptr(grad_op); + } +}; + +} // namespace operators +} // namespace paddle + +namespace ops = paddle::operators; +REGISTER_OPERATOR(reverse, ops::ReverseOp, ops::ReverseOpMaker, + ops::ReverseGradMaker); +REGISTER_OPERATOR(reverse_grad, ops::ReverseOp); +REGISTER_OP_CPU_KERNEL( + reverse, ops::ReverseKernel, + ops::ReverseKernel, + ops::ReverseKernel, + ops::ReverseKernel, + ops::ReverseKernel, + ops::ReverseKernel) diff --git a/paddle/fluid/operators/reverse_op.cu b/paddle/fluid/operators/reverse_op.cu new file mode 100644 index 0000000000000000000000000000000000000000..635c41529b38f2dd287b00ed2e5659e11f619e78 --- /dev/null +++ b/paddle/fluid/operators/reverse_op.cu @@ -0,0 +1,24 @@ +// 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. + +#include "paddle/fluid/operators/reverse_op.h" + +namespace ops = paddle::operators; +REGISTER_OP_CUDA_KERNEL( + reverse, ops::ReverseKernel, + ops::ReverseKernel, + ops::ReverseKernel, + ops::ReverseKernel, + ops::ReverseKernel, + ops::ReverseKernel) diff --git a/paddle/fluid/operators/reverse_op.h b/paddle/fluid/operators/reverse_op.h new file mode 100644 index 0000000000000000000000000000000000000000..9063cd59bba5c6307b55a500455908a5fd278390 --- /dev/null +++ b/paddle/fluid/operators/reverse_op.h @@ -0,0 +1,87 @@ +// 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 "paddle/fluid/framework/eigen.h" +#include "paddle/fluid/framework/op_registry.h" + +namespace paddle { +namespace operators { +template +struct ReverseFunctor { + void operator()(const DeviceContext& context, const framework::LoDTensor& in, + framework::LoDTensor* out, const std::vector& axis) { + Eigen::array reverse_axis; + for (int i = 0; i < Rank; ++i) { + reverse_axis[i] = false; + } + for (int a : axis) { + reverse_axis[a] = true; + } + + auto in_eigen = framework::EigenTensor::From(in); + auto out_eigen = framework::EigenTensor::From(*out); + auto* dev = context.eigen_device(); + + out_eigen.device(*dev) = in_eigen.reverse(reverse_axis); + } +}; + +template +class ReverseKernel : public framework::OpKernel { + public: + void Compute(const framework::ExecutionContext& context) const override { + auto* x = context.Input("X"); + auto* out = context.Output("Out"); + out->mutable_data(context.GetPlace()); + const auto& axis = context.Attr>("axis"); + int rank = x->dims().size(); + auto& dev_ctx = context.template device_context(); + + switch (rank) { + case 1: + ReverseFunctor functor1; + functor1(dev_ctx, *x, out, axis); + break; + case 2: + ReverseFunctor functor2; + functor2(dev_ctx, *x, out, axis); + break; + case 3: + ReverseFunctor functor3; + functor3(dev_ctx, *x, out, axis); + break; + case 4: + ReverseFunctor functor4; + functor4(dev_ctx, *x, out, axis); + break; + case 5: + ReverseFunctor functor5; + functor5(dev_ctx, *x, out, axis); + break; + case 6: + ReverseFunctor functor6; + functor6(dev_ctx, *x, out, axis); + break; + default: + PADDLE_THROW( + "Reserve operator doesn't supports tensors whose ranks are greater " + "than 6."); + } + } +}; + +} // namespace operators +} // namespace paddle diff --git a/paddle/fluid/pybind/pybind.cc b/paddle/fluid/pybind/pybind.cc index 03cf417b62f96fd6812b3eac497ffdf9a484f5eb..669d1bdaa3ec194be817cdc5e1f8484770c70c68 100644 --- a/paddle/fluid/pybind/pybind.cc +++ b/paddle/fluid/pybind/pybind.cc @@ -553,6 +553,12 @@ All parameter, weight, gradient are variables in Paddle. [](BuildStrategy &self, BuildStrategy::GradientScaleStrategy strategy) { self.gradient_scale_ = strategy; + }) + .def_property( + "debug_graphviz_path", + [](const BuildStrategy &self) { return self.debug_graphviz_path_; }, + [](BuildStrategy &self, const std::string &path) { + self.debug_graphviz_path_ = path; }); pe.def(py::init &, diff --git a/paddle/scripts/paddle_build.sh b/paddle/scripts/paddle_build.sh index 8d99e660e7e34e40b905563a0cf87176b0cf619d..55959197e7cd82253fb0c604604b4302ca0a3dc7 100755 --- a/paddle/scripts/paddle_build.sh +++ b/paddle/scripts/paddle_build.sh @@ -447,7 +447,7 @@ EOF # run paddle version to install python packages first RUN apt-get update &&\ ${NCCL_DEPS}\ - apt-get install -y wget python-pip python-opencv dmidecode python-tk && easy_install -U pip && \ + apt-get install -y wget python-pip python-opencv libgtk2.0-dev dmidecode python-tk && easy_install -U pip && \ pip install /*.whl; apt-get install -f -y && \ apt-get clean -y && \ rm -f /*.whl && \ diff --git a/python/paddle/fluid/layers/nn.py b/python/paddle/fluid/layers/nn.py index 56f6f26803919a171f6459c909e6bb71ab63b180..221f3ddae589d9992ba7fb92975a698ca4306249 100644 --- a/python/paddle/fluid/layers/nn.py +++ b/python/paddle/fluid/layers/nn.py @@ -1182,19 +1182,19 @@ def conv2d(input, - Input: - Input shape: $(N, C_{in}, H_{in}, W_{in})$ + Input shape: :math:`(N, C_{in}, H_{in}, W_{in})` - Filter shape: $(C_{out}, C_{in}, H_f, W_f)$ + Filter shape: :math:`(C_{out}, C_{in}, H_f, W_f)` - Output: - Output shape: $(N, C_{out}, H_{out}, W_{out})$ + Output shape: :math:`(N, C_{out}, H_{out}, W_{out})` Where .. math:: - H_{out}&= \\frac{(H_{in} + 2 * paddings[0] - (dilations[0] * (H_f - 1) + 1))}{strides[0]} + 1 \\\\ - W_{out}&= \\frac{(W_{in} + 2 * paddings[1] - (dilations[1] * (W_f - 1) + 1))}{strides[1]} + 1 + H_{out}&= \\frac{(H_{in} + 2 * paddings[0] - (dilations[0] * (H_f - 1) + 1))}{strides[0]} + 1 \\\\ + W_{out}&= \\frac{(W_{in} + 2 * paddings[1] - (dilations[1] * (W_f - 1) + 1))}{strides[1]} + 1 Args: input(Variable): The input image with [N, C, H, W] format. diff --git a/python/paddle/fluid/layers/tensor.py b/python/paddle/fluid/layers/tensor.py index be34cc81a5d5ca0e781e5984b6c3eeaa4e25eb90..75d3bf879703a1db1108eae45d879164e0024156 100644 --- a/python/paddle/fluid/layers/tensor.py +++ b/python/paddle/fluid/layers/tensor.py @@ -363,6 +363,40 @@ def zeros(shape, dtype, force_cpu=False): return fill_constant(value=0.0, **locals()) +def reverse(x, axis): + """ + **reverse** + + This function reverse the input 'x' along given axises. + + Args: + x(Vairbale): the input to be reversed. + axis(int|tuple|list): Axis that along which order of elements + is reversed. If it is a tuple or a list, reversing + will be apply on each axis in the tuple or list. + + Returns: + Variable: The reversed tensor. + + Examples: + .. code-block:: python + + out = fluid.layers.reverse(x=in, axis=0) + # or: + out = fluid.layers.reverse(x=in, axis=[0,1]) + """ + if isinstance(axis, int): + axis = [axis] + helper = LayerHelper("reverse", **locals()) + out = helper.create_tmp_variable(dtype=x.dtype) + helper.append_op( + type='reverse', + inputs={'Input': x}, + outputs={'Out': [out]}, + attrs={'axis': axis}) + return out + + def save(x, file_path, overwrite=True): """ Saves a variable as a file. diff --git a/python/paddle/fluid/tests/unittests/test_elementwise_add_op.py b/python/paddle/fluid/tests/unittests/test_elementwise_add_op.py index 1f52bd90d0d49bda6c180019e90ebd923c91439c..96d47906a0606bba4b1d2207f7da85b058e42a2b 100644 --- a/python/paddle/fluid/tests/unittests/test_elementwise_add_op.py +++ b/python/paddle/fluid/tests/unittests/test_elementwise_add_op.py @@ -252,5 +252,25 @@ class TestFP16ElementwiseAddOp_rowwise_add_1(TestFP16ElementwiseAddOp): self.axis = 1 +class TestElementwiseAddOp_channelwise_add(TestElementwiseAddOp): + def init_input_output(self): + self.x = np.random.rand(3, 20, 20).astype(self.dtype) + self.y = np.random.rand(3, 1, 1).astype(self.dtype) + self.out = self.x + self.y + + def init_axis(self): + self.axis = -1 + + +class TestFP16ElementwiseAddOp_channelwise_add(TestFP16ElementwiseAddOp): + def init_input_output(self): + self.x = np.random.rand(3, 10, 20).astype(self.dtype) + self.y = np.random.rand(3, 1, 1).astype(self.dtype) + self.out = self.x + self.y + + def init_axis(self): + self.axis = -1 + + if __name__ == '__main__': unittest.main() diff --git a/python/paddle/fluid/tests/unittests/test_reverse_op.py b/python/paddle/fluid/tests/unittests/test_reverse_op.py new file mode 100644 index 0000000000000000000000000000000000000000..f845575a02869f08299d76b5600074598ca27f6c --- /dev/null +++ b/python/paddle/fluid/tests/unittests/test_reverse_op.py @@ -0,0 +1,67 @@ +# 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. + +import unittest +import numpy as np +from op_test import OpTest + + +class TestReverseOp(OpTest): + def initTestCase(self): + self.x = np.random.random((3, 4)).astype('float32') + self.axis = [0] + + def setUp(self): + self.initTestCase() + self.op_type = "reverse" + self.inputs = {"X": self.x} + self.attrs = {'axis': self.axis} + out = self.x + for a in self.axis: + out = np.flip(out, axis=a) + self.outputs = {'Out': out} + + def test_check_output(self): + self.check_output() + + def test_check_grad(self): + self.check_grad(['X'], 'Out') + + +class TestCase0(TestReverseOp): + def initTestCase(self): + self.x = np.random.random((3, 4)).astype('float32') + self.axis = [1] + + +class TestCase1(TestReverseOp): + def initTestCase(self): + self.x = np.random.random((3, 4)).astype('float32') + self.axis = [0, 1] + + +class TestCase2(TestReverseOp): + def initTestCase(self): + self.x = np.random.random((3, 4, 5)).astype('float32') + self.axis = [0, 2] + + +class TestCase3(TestReverseOp): + def initTestCase(self): + self.x = np.random.random((3, 4, 5)).astype('float32') + self.axis = [1, 2] + + +if __name__ == '__main__': + unittest.main()