From 7160cb0f322aa5a0f3478bc2957ea704a907c30b Mon Sep 17 00:00:00 2001 From: sneaxiy Date: Tue, 19 Feb 2019 11:15:23 +0000 Subject: [PATCH 01/44] decoupled reader test=develop --- paddle/fluid/framework/reader.h | 53 +++++- paddle/fluid/operators/reader/CMakeLists.txt | 7 +- .../fluid/operators/reader/blocking_queue.h | 7 +- .../fluid/operators/reader/buffered_reader.cc | 32 ++-- .../fluid/operators/reader/buffered_reader.h | 6 +- .../fluid/operators/reader/compose_reader.cc | 39 +++++ .../fluid/operators/reader/compose_reader.h | 34 ++++ .../operators/reader/create_py_reader_op.cc | 26 +-- paddle/fluid/operators/reader/py_reader.cc | 78 +++++++++ paddle/fluid/operators/reader/py_reader.h | 62 +++++++ paddle/fluid/pybind/CMakeLists.txt | 2 +- paddle/fluid/pybind/pybind.cc | 54 +++++- paddle/fluid/pybind/reader_py.cc | 132 +++++++++++++++ paddle/fluid/pybind/reader_py.h | 25 +++ python/paddle/fluid/compiler.py | 43 +++-- python/paddle/fluid/executor.py | 4 + python/paddle/fluid/framework.py | 36 ++++ python/paddle/fluid/io.py | 4 +- python/paddle/fluid/reader.py | 141 ++++++++++++++++ .../unittests/test_decoupled_py_reader.py | 157 ++++++++++++++++++ 20 files changed, 869 insertions(+), 73 deletions(-) create mode 100644 paddle/fluid/operators/reader/compose_reader.cc create mode 100644 paddle/fluid/operators/reader/compose_reader.h create mode 100644 paddle/fluid/operators/reader/py_reader.cc create mode 100644 paddle/fluid/operators/reader/py_reader.h create mode 100644 paddle/fluid/pybind/reader_py.cc create mode 100644 paddle/fluid/pybind/reader_py.h create mode 100644 python/paddle/fluid/reader.py create mode 100644 python/paddle/fluid/tests/unittests/test_decoupled_py_reader.py diff --git a/paddle/fluid/framework/reader.h b/paddle/fluid/framework/reader.h index 82562bf883..61120dcf12 100644 --- a/paddle/fluid/framework/reader.h +++ b/paddle/fluid/framework/reader.h @@ -54,6 +54,7 @@ class ReaderBase { private: friend class DecoratedReader; + friend class MultiDecoratedReader; // These methods can be only invoked inside DecoratedReader to record the // decorating chain. void InsertDecoratedReader( @@ -62,15 +63,20 @@ class ReaderBase { std::vector> decorated_readers_; }; -class DecoratedReader : public ReaderBase, +class DecoratedReaderBase : public ReaderBase { + public: + virtual void RegisterDecorateChain() = 0; +}; + +class DecoratedReader : public DecoratedReaderBase, public std::enable_shared_from_this { public: explicit DecoratedReader(const std::shared_ptr& reader) - : ReaderBase(), reader_(reader) { + : DecoratedReaderBase(), reader_(reader) { PADDLE_ENFORCE_NOT_NULL(reader_); } - void RegisterDecorateChain() { + void RegisterDecorateChain() final { reader_->InsertDecoratedReader(shared_from_this()); } @@ -84,6 +90,41 @@ class DecoratedReader : public ReaderBase, std::shared_ptr reader_; }; +class MultiDecoratedReader + : public DecoratedReaderBase, + public std::enable_shared_from_this { + public: + explicit MultiDecoratedReader( + const std::vector>& readers) + : readers_(readers) { + PADDLE_ENFORCE(!readers_.empty()); + for (auto& r : readers_) { + PADDLE_ENFORCE_NOT_NULL(r); + } + } + + void RegisterDecorateChain() final { + for (auto& r : readers_) { + r->InsertDecoratedReader(shared_from_this()); + } + } + + protected: + void ShutdownImpl() override { + for (auto& r : readers_) { + r->Shutdown(); + } + } + + void StartImpl() override { + for (auto& r : readers_) { + r->Start(); + } + } + + std::vector> readers_; +}; + // FileReader is just a conceptual class. class FileReader : public ReaderBase {}; @@ -132,8 +173,10 @@ class ReaderHolder { }; template -inline std::shared_ptr MakeDecoratedReader(ARGS&&... args) { - std::shared_ptr reader(new T(std::forward(args)...)); +inline std::shared_ptr MakeDecoratedReader( + ARGS&&... args) { + std::shared_ptr reader( + new T(std::forward(args)...)); reader->RegisterDecorateChain(); return reader; } diff --git a/paddle/fluid/operators/reader/CMakeLists.txt b/paddle/fluid/operators/reader/CMakeLists.txt index 7c284312df..2701e10b30 100644 --- a/paddle/fluid/operators/reader/CMakeLists.txt +++ b/paddle/fluid/operators/reader/CMakeLists.txt @@ -17,7 +17,10 @@ function(reader_library TARGET_NAME) PARENT_SCOPE) endfunction() +cc_library(py_reader SRCS py_reader.cc DEPS reader) +cc_library(compose_reader SRCS compose_reader.cc DEPS reader) cc_library(buffered_reader SRCS buffered_reader.cc DEPS reader simple_threadpool) + reader_library(open_files_op SRCS open_files_op.cc DEPS buffered_reader) reader_library(create_random_data_generator_op SRCS create_random_data_generator_op.cc) reader_library(create_shuffle_reader_op SRCS create_shuffle_reader_op.cc) @@ -26,7 +29,7 @@ reader_library(create_recordio_file_reader_op SRCS create_recordio_file_reader_o reader_library(create_double_buffer_reader_op SRCS create_double_buffer_reader_op.cc DEPS buffered_reader) reader_library(create_multi_pass_reader_op SRCS create_multi_pass_reader_op.cc) reader_library(create_custom_reader_op SRCS create_custom_reader_op.cc) -reader_library(create_py_reader_op SRCS create_py_reader_op.cc) +reader_library(create_py_reader_op SRCS create_py_reader_op.cc DEPS py_reader) if (NOT WIN32 AND NOT ON_INFER) cc_library(ctr_reader SRCS ctr_reader.cc DEPS gzstream reader zlib) @@ -38,7 +41,7 @@ cc_test(reader_blocking_queue_test SRCS reader_blocking_queue_test.cc) # Export local libraries to parent # set(READER_LIBRARY ${LOCAL_READER_LIBS} PARENT_SCOPE) -op_library(read_op) +op_library(read_op DEPS py_reader compose_reader buffered_reader) foreach(src ${LOCAL_READER_LIBS}) set(OP_LIBRARY ${src} ${OP_LIBRARY} CACHE INTERNAL "op libs") diff --git a/paddle/fluid/operators/reader/blocking_queue.h b/paddle/fluid/operators/reader/blocking_queue.h index 51b980acb5..b76f482c57 100644 --- a/paddle/fluid/operators/reader/blocking_queue.h +++ b/paddle/fluid/operators/reader/blocking_queue.h @@ -34,7 +34,7 @@ class BlockingQueue { explicit BlockingQueue(size_t capacity, bool speed_test_mode = false) : capacity_(capacity), speed_test_mode_(speed_test_mode), closed_(false) { PADDLE_ENFORCE_GT( - capacity_, 0, + capacity_, static_cast(0), "The capacity of a reader::BlockingQueue must be greater than 0."); } @@ -114,6 +114,11 @@ class BlockingQueue { return queue_.size(); } + void Clear() { + std::lock_guard lock(mutex_); + queue_.clear(); + } + private: size_t capacity_; bool speed_test_mode_; diff --git a/paddle/fluid/operators/reader/buffered_reader.cc b/paddle/fluid/operators/reader/buffered_reader.cc index defc29b91f..b8c98ff5e7 100644 --- a/paddle/fluid/operators/reader/buffered_reader.cc +++ b/paddle/fluid/operators/reader/buffered_reader.cc @@ -28,8 +28,10 @@ BufferedReader::~BufferedReader() { #ifdef PADDLE_WITH_CUDA if (platform::is_gpu_place(place_)) { platform::SetDeviceId(boost::get(place_).device); - PADDLE_ENFORCE(cudaStreamDestroy(stream)); - for (auto &event : events) PADDLE_ENFORCE(cudaEventDestroy(event)); + PADDLE_ENFORCE(cudaStreamDestroy(stream_)); + for (auto &event : events_) { + PADDLE_ENFORCE(cudaEventDestroy(event)); + } } #endif } @@ -44,14 +46,15 @@ BufferedReader::BufferedReader( #ifdef PADDLE_WITH_CUDA if (platform::is_gpu_place(place_)) { platform::SetDeviceId(boost::get(place_).device); - compute_stream = + compute_stream_ = ((platform::CUDADeviceContext *)(platform::DeviceContextPool::Instance() .Get(place_))) ->stream(); - events.resize(buffer_size); - for (auto &event : events) + events_.resize(buffer_size); + for (auto &event : events_) { PADDLE_ENFORCE(cudaEventCreateWithFlags(&event, cudaEventDisableTiming)); - PADDLE_ENFORCE(cudaStreamCreateWithFlags(&stream, cudaStreamNonBlocking)); + } + PADDLE_ENFORCE(cudaStreamCreateWithFlags(&stream_, cudaStreamNonBlocking)); } #endif cpu_buffer_.resize(buffer_size); @@ -70,7 +73,7 @@ void BufferedReader::ReadAsync(size_t i) { #ifdef PADDLE_WITH_CUDA if (platform::is_gpu_place(place_)) { platform::SetDeviceId(boost::get(place_).device); - PADDLE_ENFORCE(cudaEventRecord(events[i], compute_stream)); + PADDLE_ENFORCE(cudaEventRecord(events_[i], compute_stream_)); } #endif position_.emplace(thread_pool_.enqueue([this, i]() -> size_t { @@ -86,7 +89,7 @@ void BufferedReader::ReadAsync(size_t i) { // TensorCopySync would block other stream if (platform::is_gpu_place(place_)) { platform::SetDeviceId(boost::get(place_).device); - PADDLE_ENFORCE(cudaStreamWaitEvent(stream, events[i], 0)); + PADDLE_ENFORCE(cudaStreamWaitEvent(stream_, events_[i], 0)); TensorVec &gpu = gpu_buffer_[i]; gpu.resize(cpu.size()); for (size_t i = 0; i < cpu.size(); ++i) { @@ -97,23 +100,24 @@ void BufferedReader::ReadAsync(size_t i) { auto gpu_ptr = gpu[i].mutable_data(place_, cpu[i].type()); auto size = cpu[i].numel() * paddle::framework::SizeOfType(cpu[i].type()); - if (platform::is_cuda_pinned_place(cpu_place)) + if (platform::is_cuda_pinned_place(cpu_place)) { memory::Copy(boost::get(place_), gpu_ptr, boost::get(cpu_place), - cpu_ptr, size, stream); - else if ((platform::is_gpu_place(cpu_place))) + cpu_ptr, size, stream_); + } else if ((platform::is_gpu_place(cpu_place))) { memory::Copy(boost::get(place_), gpu_ptr, boost::get(cpu_place), cpu_ptr, - size, stream); - else + size, stream_); + } else { // if cpu place is not pinned, async copy is slower than sync copy, // so we use sync copy instead. memory::Copy(boost::get(place_), gpu_ptr, boost::get(cpu_place), cpu_ptr, size, 0); + } gpu[i].set_lod(cpu[i].lod()); } - PADDLE_ENFORCE(cudaStreamSynchronize(stream)); + PADDLE_ENFORCE(cudaStreamSynchronize(stream_)); } #endif return i; diff --git a/paddle/fluid/operators/reader/buffered_reader.h b/paddle/fluid/operators/reader/buffered_reader.h index 87680da01a..6b21de0949 100644 --- a/paddle/fluid/operators/reader/buffered_reader.h +++ b/paddle/fluid/operators/reader/buffered_reader.h @@ -63,9 +63,9 @@ class BufferedReader : public framework::DecoratedReader { std::vector gpu_buffer_; size_t prev_pos_{-1UL}; #ifdef PADDLE_WITH_CUDA - cudaStream_t stream; - cudaStream_t compute_stream; - std::vector events; + cudaStream_t stream_; + cudaStream_t compute_stream_; + std::vector events_; #endif }; diff --git a/paddle/fluid/operators/reader/compose_reader.cc b/paddle/fluid/operators/reader/compose_reader.cc new file mode 100644 index 0000000000..4b88b9331c --- /dev/null +++ b/paddle/fluid/operators/reader/compose_reader.cc @@ -0,0 +1,39 @@ +// 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/operators/reader/compose_reader.h" + +namespace paddle { +namespace operators { +namespace reader { + +ComposeReader::ComposeReader( + const std::vector> &readers) + : framework::MultiDecoratedReader(readers) {} + +void ComposeReader::ReadNext(std::vector *out) { + out->clear(); + std::vector each_ret; + for (auto &r : readers_) { + r->ReadNext(&each_ret); + out->reserve(out->size() + each_ret.size()); + for (auto &data : each_ret) { + out->emplace_back(std::move(data)); + } + } +} + +} // namespace reader +} // namespace operators +} // namespace paddle diff --git a/paddle/fluid/operators/reader/compose_reader.h b/paddle/fluid/operators/reader/compose_reader.h new file mode 100644 index 0000000000..c9e2a2d72f --- /dev/null +++ b/paddle/fluid/operators/reader/compose_reader.h @@ -0,0 +1,34 @@ +// 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 "paddle/fluid/framework/reader.h" + +namespace paddle { +namespace operators { +namespace reader { + +class ComposeReader : public framework::MultiDecoratedReader { + public: + explicit ComposeReader( + const std::vector> &readers); + + void ReadNext(std::vector *out) override; +}; + +} // namespace reader +} // namespace operators +} // namespace paddle diff --git a/paddle/fluid/operators/reader/create_py_reader_op.cc b/paddle/fluid/operators/reader/create_py_reader_op.cc index 901a92ab5b..4a6581bbbd 100644 --- a/paddle/fluid/operators/reader/create_py_reader_op.cc +++ b/paddle/fluid/operators/reader/create_py_reader_op.cc @@ -12,37 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include "paddle/fluid/operators/reader/lod_tensor_blocking_queue.h" +#include "paddle/fluid/operators/reader/py_reader.h" #include "paddle/fluid/operators/reader/reader_op_registry.h" namespace paddle { namespace operators { namespace reader { -class PyReader : public framework::FileReader { - public: - explicit PyReader(const std::shared_ptr& queue) - : framework::FileReader() { - PADDLE_ENFORCE(queue != nullptr, "LoDTensorBlockingQueue must not be null"); - queue_ = queue; - } - - void ReadNext(std::vector* out) override { - bool success; - *out = queue_->Pop(&success); - if (!success) out->clear(); - } - - ~PyReader() { queue_->Close(); } - - void Shutdown() override { queue_->Close(); } - - void Start() override { queue_->ReOpen(); } - - private: - std::shared_ptr queue_; -}; - class CreatePyReaderOp : public framework::OperatorBase { public: using framework::OperatorBase::OperatorBase; diff --git a/paddle/fluid/operators/reader/py_reader.cc b/paddle/fluid/operators/reader/py_reader.cc new file mode 100644 index 0000000000..dc84faa974 --- /dev/null +++ b/paddle/fluid/operators/reader/py_reader.cc @@ -0,0 +1,78 @@ +// 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/operators/reader/py_reader.h" + +namespace paddle { +namespace operators { +namespace reader { + +PyReader::PyReader(const std::shared_ptr& queue) + : framework::FileReader() { + PADDLE_ENFORCE(queue != nullptr, "LoDTensorBlockingQueue must not be null"); + queue_ = queue; +} + +void PyReader::ReadNext(std::vector* out) { + bool success; + *out = queue_->Pop(&success); + if (!success) out->clear(); +} + +PyReader::~PyReader() { queue_->Close(); } + +void PyReader::Shutdown() { queue_->Close(); } + +void PyReader::Start() { queue_->ReOpen(); } + +MultiQueuePyReader::MultiQueuePyReader( + const std::vector>& queues) + : queues_(queues) { + PADDLE_ENFORCE(!queues_.empty()); + for (auto& q : queues_) { + PADDLE_ENFORCE_NOT_NULL(q); + } +} + +void MultiQueuePyReader::ReadNext(std::vector* out) { + auto idx = read_out_idx_.fetch_add(1) % queues_.size(); + for (size_t i = 0; i < queues_.size(); ++i) { + *out = queues_[idx]->Pop(); + if (!out->empty()) return; + idx = (idx + 1) % queues_.size(); + } +} + +MultiQueuePyReader::~MultiQueuePyReader() { + for (auto& q : queues_) { + q->Close(); + } +} + +void MultiQueuePyReader::Shutdown() { + for (auto& q : queues_) { + q->Close(); + } + read_out_idx_.store(0, std::memory_order::memory_order_seq_cst); +} + +void MultiQueuePyReader::Start() { + for (auto& q : queues_) { + q->ReOpen(); + } +} + +} // namespace reader +} // namespace operators +} // namespace paddle diff --git a/paddle/fluid/operators/reader/py_reader.h b/paddle/fluid/operators/reader/py_reader.h new file mode 100644 index 0000000000..146a2351e5 --- /dev/null +++ b/paddle/fluid/operators/reader/py_reader.h @@ -0,0 +1,62 @@ +// 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 "paddle/fluid/framework/reader.h" +#include "paddle/fluid/operators/reader/lod_tensor_blocking_queue.h" + +namespace paddle { +namespace operators { +namespace reader { + +class PyReader : public framework::FileReader { + public: + explicit PyReader(const std::shared_ptr& queue); + + void ReadNext(std::vector* out) override; + + ~PyReader(); + + void Shutdown() override; + + void Start() override; + + private: + std::shared_ptr queue_; +}; + +class MultiQueuePyReader : public framework::FileReader { + public: + explicit MultiQueuePyReader( + const std::vector>& queues); + + void ReadNext(std::vector* out) override; + + ~MultiQueuePyReader(); + + void Shutdown() override; + + void Start() override; + + private: + std::vector> queues_; + std::atomic read_out_idx_{0}; +}; + +} // namespace reader +} // namespace operators +} // namespace paddle diff --git a/paddle/fluid/pybind/CMakeLists.txt b/paddle/fluid/pybind/CMakeLists.txt index 4ac5b83c56..84da8491d3 100644 --- a/paddle/fluid/pybind/CMakeLists.txt +++ b/paddle/fluid/pybind/CMakeLists.txt @@ -5,7 +5,7 @@ set(PYBIND_DEPS pybind python proto_desc memory executor async_executor prune if(WITH_PYTHON) list(APPEND PYBIND_DEPS py_func_op) endif() -set(PYBIND_SRCS pybind.cc exception.cc protobuf.cc const_value.cc recordio.cc async_executor_py.cc imperative.cc ir.cc inference_api.cc) +set(PYBIND_SRCS pybind.cc exception.cc protobuf.cc const_value.cc recordio.cc reader_py.cc async_executor_py.cc imperative.cc ir.cc inference_api.cc) if(WITH_PYTHON) if(WITH_AMD_GPU) diff --git a/paddle/fluid/pybind/pybind.cc b/paddle/fluid/pybind/pybind.cc index a4a01ad647..f2000cc45e 100644 --- a/paddle/fluid/pybind/pybind.cc +++ b/paddle/fluid/pybind/pybind.cc @@ -54,6 +54,7 @@ limitations under the License. */ #include "paddle/fluid/pybind/ir.h" #include "paddle/fluid/pybind/protobuf.h" #include "paddle/fluid/pybind/pybind.h" // NOLINT +#include "paddle/fluid/pybind/reader_py.h" #include "paddle/fluid/pybind/recordio.h" #include "paddle/fluid/pybind/tensor_py.h" @@ -106,6 +107,16 @@ bool IsCompiledWithDIST() { #endif } +template +static inline bool IsSamePlace(const PlaceType1 &p1, const PlaceType2 &p2) { + return paddle::platform::Place(p1) == paddle::platform::Place(p2); +} + +template +static inline int PlaceIndex(const PlaceType &p) { + return static_cast(paddle::platform::Place(p).which()); +} + PYBIND11_MODULE(core, m) { // Not used, just make sure cpu_info.cc is linked. paddle::platform::CpuTotalPhysicalMemory(); @@ -452,6 +463,7 @@ PYBIND11_MODULE(core, m) { All parameter, weight, gradient are variables in Paddle. )DOC") + .def(py::init<>()) .def("is_int", [](const Variable &var) { return var.IsType(); }) .def("set_int", [](Variable &var, int val) -> void { *var.GetMutable() = val; }) @@ -493,9 +505,7 @@ All parameter, weight, gradient are variables in Paddle. }, py::return_value_policy::reference); - py::class_(m, "Reader", "") - .def("start", &framework::ReaderHolder::Start) - .def("reset", &framework::ReaderHolder::ResetAll); + BindReader(&m); using LoDTensorBlockingQueue = ::paddle::operators::reader::LoDTensorBlockingQueue; @@ -657,29 +667,65 @@ All parameter, weight, gradient are variables in Paddle. PADDLE_THROW("Cannot use CUDAPlace in CPU only version"); #endif }) + .def("_type", &PlaceIndex) + .def("_equals", &IsSamePlace) + .def("_equals", &IsSamePlace) + .def("_equals", &IsSamePlace) + .def("_equals", + &IsSamePlace) + .def("gpu_device_id", + [](platform::CUDAPlace &self) { return self.device; }) .def("__str__", string::to_string); py::class_(m, "CPUPlace") .def(py::init<>()) + .def("_type", &PlaceIndex) + .def("_equals", &IsSamePlace) + .def("_equals", &IsSamePlace) + .def("_equals", &IsSamePlace) + .def("_equals", + &IsSamePlace) .def("__str__", string::to_string); py::class_(m, "CUDAPinnedPlace") .def("__init__", - [](platform::CUDAPinnedPlace &) { + [](platform::CUDAPinnedPlace &self) { #ifndef PADDLE_WITH_CUDA PADDLE_THROW("Cannot use CUDAPinnedPlace in CPU only version"); #endif + new (&self) platform::CUDAPinnedPlace(); }) + .def("_type", &PlaceIndex) + .def("_equals", &IsSamePlace) + .def("_equals", + &IsSamePlace) + .def("_equals", + &IsSamePlace) + .def("_equals", + &IsSamePlace) .def("__str__", string::to_string); py::class_(m, "Place") .def(py::init<>()) + .def("_type", &PlaceIndex) + .def("_equals", &IsSamePlace) + .def("_equals", &IsSamePlace) + .def("_equals", &IsSamePlace) + .def("_equals", &IsSamePlace) .def("is_gpu_place", [](platform::Place &self) { return platform::is_gpu_place(self); }) + .def("is_cpu_place", + [](platform::Place &self) { return platform::is_cpu_place(self); }) + .def("is_cuda_pinned_place", + [](platform::Place &self) { + return platform::is_cuda_pinned_place(self); + }) .def("gpu_device_id", [](platform::Place &self) { return boost::get(self).device; }) + .def("set_place", [](platform::Place &self, + const platform::Place &other) { self = other; }) .def("set_place", [](platform::Place &self, const platform::CPUPlace &cpu_place) { self = cpu_place; diff --git a/paddle/fluid/pybind/reader_py.cc b/paddle/fluid/pybind/reader_py.cc new file mode 100644 index 0000000000..a09d18656f --- /dev/null +++ b/paddle/fluid/pybind/reader_py.cc @@ -0,0 +1,132 @@ +// 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/pybind/reader_py.h" +#include +#include +#include "paddle/fluid/framework/reader.h" +#include "paddle/fluid/operators/reader/buffered_reader.h" +#include "paddle/fluid/operators/reader/compose_reader.h" +#include "paddle/fluid/operators/reader/py_reader.h" +#include "paddle/fluid/platform/place.h" +#include "pybind11/stl.h" + +namespace paddle { +namespace pybind { + +class FeedReader { + using ResultDictList = + std::vector>; + + public: + FeedReader(std::unique_ptr reader, + const std::vector &names, size_t num_places, + bool drop_last = true) + : reader_(std::move(reader)), + names_(names), + num_places_(num_places), + drop_last_(drop_last) {} + + ResultDictList ReadNext() { + std::vector tensors; + reader_->ReadNext(&tensors); + if (tensors.empty()) return ResultDictList(); + + PADDLE_ENFORCE(tensors.size() % names_.size() == 0, + "Tensor size: %d, names size: %d", tensors.size(), + names_.size()); + + size_t read_place_num = tensors.size() / names_.size(); + + if (drop_last_ && read_place_num != num_places_) { + return ResultDictList(); + } + + ResultDictList ret(read_place_num); + for (size_t i = 0; i < tensors.size(); ++i) { + ret[i / names_.size()].emplace(names_[i % names_.size()], + std::move(tensors[i])); + } + return ret; + } + + void Start() { reader_->Start(); } + + void Reset() { reader_->ResetAll(); } + + private: + std::unique_ptr reader_; + std::vector names_; + size_t num_places_; + bool drop_last_; +}; + +static std::unique_ptr CreatePyReader( + const std::vector< + std::shared_ptr> &queues, + const std::vector &dst_places) { + std::shared_ptr reader; + if (queues.size() == 1) { + reader.reset(new operators::reader::PyReader(queues[0])); + } else { + reader.reset(new operators::reader::MultiQueuePyReader(queues)); + } + std::vector> buffered_reader; + buffered_reader.reserve(dst_places.size()); + for (auto &p : dst_places) { + buffered_reader.emplace_back( + framework::MakeDecoratedReader( + reader, p, 2)); + } + reader = framework::MakeDecoratedReader( + buffered_reader); + + auto *holder = new framework::ReaderHolder(); + holder->Reset(reader); + return std::unique_ptr(holder); +} + +namespace py = pybind11; + +void BindReader(py::module *module) { + auto &m = *module; + + namespace reader = ::paddle::operators::reader; + + py::class_(m, "Reader", "") + .def("start", &framework::ReaderHolder::Start) + .def("reset", &framework::ReaderHolder::ResetAll); + + py::class_(m, "FeedReader", "") + .def("read_next", &FeedReader::ReadNext, + py::call_guard()) + .def("start", &FeedReader::Start, + py::call_guard()) + .def("reset", &FeedReader::Reset, + py::call_guard()); + + m.def("create_py_reader", + [](const std::vector< + std::shared_ptr> + queues, + const std::vector &names, + const std::vector &dst_places, bool drop_last) { + return new FeedReader(CreatePyReader(queues, dst_places), names, + dst_places.size(), drop_last); + }, + py::return_value_policy::take_ownership); +} + +} // namespace pybind +} // namespace paddle diff --git a/paddle/fluid/pybind/reader_py.h b/paddle/fluid/pybind/reader_py.h new file mode 100644 index 0000000000..472ff65368 --- /dev/null +++ b/paddle/fluid/pybind/reader_py.h @@ -0,0 +1,25 @@ +// 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 "pybind11/pybind11.h" + +namespace paddle { +namespace pybind { + +void BindReader(pybind11::module *module); + +} // namespace pybind +} // namespace paddle diff --git a/python/paddle/fluid/compiler.py b/python/paddle/fluid/compiler.py index ef02429428..523894af7b 100644 --- a/python/paddle/fluid/compiler.py +++ b/python/paddle/fluid/compiler.py @@ -17,6 +17,7 @@ import os import six import sys from .. import compat as cpt +from .framework import cuda_places, cpu_places from . import core @@ -78,7 +79,8 @@ class CompiledProgram(object): loss_name=None, build_strategy=None, exec_strategy=None, - share_vars_from=None): + share_vars_from=None, + places=None): """Configs the program to run in data parallel way. Args: @@ -97,6 +99,12 @@ class CompiledProgram(object): will share variables from `share_vars_from`. `share_vars_from` must be run by the executor before this CompiledProgram so that vars are ready. + places(list(CUDAPlace)|list(CPUPlace)|None): If provide, only compile + program in the given places. Otherwise, the places used when compiled + is determined by the Executor, and the places used are controlled + by environment variables: FLAGS_selected_gpus or CUDA_VISIBLE_DEVICES + if using GPU; or CPU_NUM if using CPU. + Returns: self """ @@ -110,6 +118,12 @@ class CompiledProgram(object): self._exec_strategy = ExecutionStrategy() if self._build_strategy is None: self._build_strategy = BuildStrategy() + if places is not None: + if not isinstance(places, (list, tuple)): + places = [places] + self._places = [_place_obj(p) for p in places] + else: + self._places = None return self def with_inference_optimize(self, config): @@ -148,19 +162,16 @@ class CompiledProgram(object): self._local_scopes = [] self._exec_strategy.use_cuda = isinstance(self._place, core.CUDAPlace) - if self._exec_strategy.use_cuda: - gpus_env = os.getenv("FLAGS_selected_gpus") - if gpus_env: - gpus = [int(s) for s in gpus_env.split(",")] - else: - gpus = [ - i for i in six.moves.range(core.get_cuda_device_count()) - ] - self._places = [core.CUDAPlace(i) for i in gpus] + has_set_place = (self._places is not None) + if has_set_place: + desire_place = _place_obj(self._place) + for p in self._places: + assert p._type() == desire_place._type(), \ + "Place type not match. You may set the wrong type of places" else: - cpu_num = int( - os.environ.get('CPU_NUM', multiprocessing.cpu_count())) - self._places = [core.CPUPlace() for _ in six.moves.range(cpu_num)] + places = cuda_places( + ) if self._exec_strategy.use_cuda else cpu_places() + self._places = [_place_obj(p) for p in places] assert self._places, "no place for execution" if self._exec_strategy.num_threads == 0: @@ -169,9 +180,7 @@ class CompiledProgram(object): # performance. Worth tunning for other models in the future. self._exec_strategy.num_threads = len(self._places) * 4 else: - cpu_num = int( - os.environ.get('CPU_NUM', multiprocessing.cpu_count())) - self._exec_strategy.num_threads = cpu_num * 2 + self._exec_strategy.num_threads = len(self._places) * 2 trainers_endpoints = self._program._trainers_endpoints @@ -217,7 +226,7 @@ class CompiledProgram(object): if self._compiled: if scope and self._scope != scope: raise ValueError("Cannot compile with different scope") - if place and self._place != place: + if place and not self._place._equals(place): raise ValueError("Cannot compile with different place") return self self._compiled = True diff --git a/python/paddle/fluid/executor.py b/python/paddle/fluid/executor.py index 8815911eae..5454d12e2c 100644 --- a/python/paddle/fluid/executor.py +++ b/python/paddle/fluid/executor.py @@ -554,6 +554,10 @@ class Executor(object): if feed is None: feed = {} + elif isinstance(feed, (list, tuple)): + assert len(feed) == 1, "Not compiled with data parallel" + feed = feed[0] + if not isinstance(feed, dict): raise TypeError( "feed requires dict as its Parameter. But you passed in %s" % diff --git a/python/paddle/fluid/framework.py b/python/paddle/fluid/framework.py index ef304b1110..deb837d96c 100644 --- a/python/paddle/fluid/framework.py +++ b/python/paddle/fluid/framework.py @@ -26,6 +26,7 @@ import six import numpy as np import subprocess +import multiprocessing from .. import compat as cpt from .proto import framework_pb2 @@ -63,6 +64,9 @@ __all__ = [ 'default_main_program', 'program_guard', 'name_scope', + 'cuda_places', + 'cpu_places', + 'cuda_pinned_places', ] EMPTY_VAR_NAME = core.kEmptyVarName() @@ -87,6 +91,38 @@ def _current_expected_place(): return _imperative_current_expected_place_ +def _cpu_num(): + return int(os.environ.get('CPU_NUM', multiprocessing.cpu_count())) + + +def cuda_places(device_ids=None): + assert core.is_compiled_with_cuda(), \ + "Not compiled with CUDA" + if device_ids is None: + gpus_env = os.getenv("FLAGS_selected_gpus") + if gpus_env: + device_ids = [int(s) for s in gpus_env.split(",")] + else: + device_ids = six.moves.range(core.get_cuda_device_count()) + elif not isinstance(device_ids, (list, tuple)): + device_ids = [device_ids] + return [core.CUDAPlace(dev_id) for dev_id in device_ids] + + +def cpu_places(device_count=None): + if device_count is None: + device_count = _cpu_num() + return [core.CPUPlace()] * device_count + + +def cuda_pinned_places(device_count=None): + assert core.is_compiled_with_cuda(), \ + "Not compiled with CUDA" + if device_count is None: + device_count = _cpu_num() + return [core.cuda_pinned_places()] * device_count + + class NameScope(object): def __init__(self, name="", parent=None): self._children = dict() diff --git a/python/paddle/fluid/io.py b/python/paddle/fluid/io.py index a2abbf36c0..1e3f4f476f 100644 --- a/python/paddle/fluid/io.py +++ b/python/paddle/fluid/io.py @@ -26,12 +26,14 @@ from paddle.fluid import layers from paddle.fluid.executor import Executor from paddle.fluid.evaluator import Evaluator from paddle.fluid.framework import Program, Parameter, default_main_program, default_startup_program, Variable, program_guard +from . import reader +from .reader import * from . import core __all__ = [ 'save_vars', 'save_params', 'save_persistables', 'load_vars', 'load_params', 'load_persistables', 'save_inference_model', 'load_inference_model' -] +] + reader.__all__ def is_parameter(var): diff --git a/python/paddle/fluid/reader.py b/python/paddle/fluid/reader.py new file mode 100644 index 0000000000..b765430622 --- /dev/null +++ b/python/paddle/fluid/reader.py @@ -0,0 +1,141 @@ +# 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 core +import six +import threading +from .framework import Program, Variable, program_guard +from .data_feeder import DataFeeder + +__all__ = ['PyReader'] + + +def _convert_places(places): + if not isinstance(places, (list, tuple)): + places = [places] + + ret = [] + for p in places: + if not isinstance(p, core.Place): + tmp = core.Place() + tmp.set_place(p) + p = tmp + + ret.append(p) + return ret + + +class PyReader(object): + def __init__(self, feed_list, places, capacity, multi_queue=True): + self._tensor_reader = None + self._thread = None + + # TODO(zjl): to support drop_last = False + self._drop_last = True + + self._feed_list = feed_list + self._var_names = [v.name for v in feed_list] + + self._queues = [] + + self._places = _convert_places(places) + + self._queue_capacity = capacity + + queue_num = len(self._places) if multi_queue else 1 + for _ in six.moves.range(queue_num): + self._queues.append( + core.init_lod_tensor_blocking_queue(core.Variable(), + self._queue_capacity)) + + self._reader = core.create_py_reader(self._queues, self._var_names, + self._places, self._drop_last) + self._exited = True + + def __call__(self): + assert self._tensor_reader is not None, \ + "Data source of PyReader has not set yet" + + class Iterator(object): + def __init__(self, reader): + self._reader = reader + + def __iter__(self): + return self + + def next(self): + ret = self._reader._reader.read_next() + if len(ret): + return ret + else: + self._reader._restart_reader() + self._reader._reader.reset() + raise StopIteration + + return Iterator(self) + + def _restart_reader(self): + if not self._exited: + for q in self._queues: + q.close() + + self._thread.join() + + def __thread_main__(): + queue_num = len(self._queues) + idx = 0 + for tensors in self._tensor_reader(): + array = core.LoDTensorArray() + for item in tensors: + if not isinstance(item, core.LoDTensor): + tmp = core.LoDTensor() + tmp.set(item, core.CPUPlace()) + item = tmp + + array.append(item) + + if not self._queues[idx].push(array): + break + + idx = (idx + 1) % queue_num + + for q in self._queues: + q.close() + + self._exited = True + + self._thread = threading.Thread(target=__thread_main__) + self._thread.daemon = True + self._exited = False + self._thread.start() + + def set_numpy_reader(self, reader): + assert self._tensor_reader is None, \ + "Cannot reset the data source of PyReader" + with program_guard(Program(), Program()): + feeder = DataFeeder( + feed_list=self._feed_list, place=core.CPUPlace()) + paddle_reader = feeder.decorate_reader(reader, multi_devices=False) + + def __tensor_reader_impl__(): + for slots in paddle_reader(): + yield [slots[var.name] for var in self._feed_list] + + self.set_tensor_reader(__tensor_reader_impl__) + + def set_tensor_reader(self, reader): + assert self._tensor_reader is None, \ + "Cannot reset the data source of PyReader" + self._tensor_reader = reader + self._restart_reader() diff --git a/python/paddle/fluid/tests/unittests/test_decoupled_py_reader.py b/python/paddle/fluid/tests/unittests/test_decoupled_py_reader.py new file mode 100644 index 0000000000..807cbaf39d --- /dev/null +++ b/python/paddle/fluid/tests/unittests/test_decoupled_py_reader.py @@ -0,0 +1,157 @@ +# 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 paddle +import paddle.fluid as fluid +import numpy as np +import time +import six +import unittest + +EPOCH_NUM = 60 +BATCH_SIZE = 32 +CLASS_NUM = 10 + + +def random_reader(): + for i in range(BATCH_SIZE * 40): + image = np.random.random([784]) + label = np.random.random_integers(low=0, high=CLASS_NUM - 1) + yield image, label + + +def simple_fc_net(places, use_legacy_py_reader): + startup_prog = fluid.Program() + main_prog = fluid.Program() + startup_prog.random_seed = 1 + main_prog.random_seed = 1 + reader = paddle.batch(random_reader, batch_size=BATCH_SIZE) + + with fluid.unique_name.guard(): + with fluid.program_guard(main_prog, startup_prog): + if not use_legacy_py_reader: + image = fluid.layers.data( + name='image', shape=[784], dtype='float32') + label = fluid.layers.data( + name='label', shape=[1], dtype='int64') + py_reader = fluid.io.PyReader( + feed_list=[image, label], + places=places, + capacity=4, + multi_queue=False) + py_reader.set_numpy_reader(reader) + else: + py_reader = fluid.layers.py_reader( + capacity=4, + shapes=[(-1, 784), (-1, 1)], + dtypes=['float32', 'int64']) + image, label = fluid.layers.read_file(py_reader) + py_reader.decorate_paddle_reader(reader) + + hidden = image + for hidden_size in [10, 20, 30]: + hidden = fluid.layers.fc( + hidden, + size=hidden_size, + act='tanh', + bias_attr=fluid.ParamAttr( + initializer=fluid.initializer.Constant(value=1.0))) + + predict_label = fluid.layers.fc(hidden, + size=CLASS_NUM, + act='softmax') + loss = fluid.layers.mean( + fluid.layers.cross_entropy( + input=predict_label, label=label)) + + optimizer = fluid.optimizer.Adam() + optimizer.minimize(loss) + return startup_prog, main_prog, py_reader, loss + + +class TestBase(unittest.TestCase): + def run_main(self, use_legacy_py_reader, with_data_parallel, places): + with fluid.scope_guard(fluid.Scope()): + startup_prog, main_prog, py_reader, loss = simple_fc_net( + places, use_legacy_py_reader) + exe = fluid.Executor(place=places[0]) + exe.run(startup_prog) + + prog = fluid.CompiledProgram(main_prog) + if with_data_parallel: + prog = prog.with_data_parallel( + loss_name=loss.name, places=places) + + step = 0 + start_t = time.time() + if use_legacy_py_reader: + for _ in six.moves.range(EPOCH_NUM): + py_reader.start() + while True: + try: + L, = exe.run(program=prog, fetch_list=[loss]) + step += 1 + except fluid.core.EOFException: + py_reader.reset() + break + else: + for _ in six.moves.range(EPOCH_NUM): + for d in py_reader(): + ''' + assert len(d) == len(places) + for i, item in enumerate(d): + image = item['image'] + label = item['label'] + assert image.shape() == [BATCH_SIZE, 784] + assert label.shape() == [BATCH_SIZE, 1] + assert image._place()._equals(places[i]) + assert label._place()._equals(places[i]) + ''' + L, = exe.run(program=prog, feed=d, fetch_list=[loss]) + step += 1 + end_t = time.time() + return {"time": end_t - start_t, "step": step} + + def prepare_places(self, with_data_parallel): + places = [[fluid.CPUPlace()], ] + if with_data_parallel: + places.append([fluid.CPUPlace()] * 2) + + if fluid.core.is_compiled_with_cuda(): + tmp = fluid.cuda_places() + assert len(tmp) > 0, "no gpu detected" + if with_data_parallel: + places.append(tmp) + places.append([tmp[0]]) + return places + + def test_main(self): + for with_data_parallel in [True, False]: + for p in self.prepare_places(with_data_parallel): + t = [] + for use_legacy_py_reader in [False, True]: + ret = self.run_main( + use_legacy_py_reader=use_legacy_py_reader, + with_data_parallel=with_data_parallel, + places=p) + ret['legacy'] = use_legacy_py_reader + ret['data_parallel'] = with_data_parallel + ret['places'] = p + t.append(ret) + + print(t) + + +if __name__ == '__main__': + unittest.main() From 27e1a9ee5ac9c2df61f2c1fa7cd3ca96030a3bd1 Mon Sep 17 00:00:00 2001 From: sneaxiy Date: Wed, 20 Feb 2019 12:58:07 +0000 Subject: [PATCH 02/44] fix hang bug test=develop --- python/paddle/fluid/reader.py | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/python/paddle/fluid/reader.py b/python/paddle/fluid/reader.py index b765430622..8352587f23 100644 --- a/python/paddle/fluid/reader.py +++ b/python/paddle/fluid/reader.py @@ -69,29 +69,32 @@ class PyReader(object): class Iterator(object): def __init__(self, reader): - self._reader = reader + self._reader = reader._reader + self._reset = reader._reset def __iter__(self): return self def next(self): - ret = self._reader._reader.read_next() + ret = self._reader.read_next() if len(ret): return ret else: - self._reader._restart_reader() - self._reader._reader.reset() + self._reset() raise StopIteration return Iterator(self) - def _restart_reader(self): + def _reset(self): if not self._exited: for q in self._queues: q.close() + if self._thread: self._thread.join() + self._reader.reset() + def __thread_main__(): queue_num = len(self._queues) idx = 0 @@ -138,4 +141,4 @@ class PyReader(object): assert self._tensor_reader is None, \ "Cannot reset the data source of PyReader" self._tensor_reader = reader - self._restart_reader() + self._reset() From b17541a9c1fcf424dc5550f581b87a2bb26ad22c Mon Sep 17 00:00:00 2001 From: sneaxiy Date: Mon, 25 Feb 2019 08:31:11 +0000 Subject: [PATCH 03/44] fix hang bug --- paddle/fluid/pybind/pybind.cc | 33 +++- paddle/fluid/pybind/reader_py.cc | 154 +++++++++++------- python/paddle/fluid/layers/io.py | 18 +- python/paddle/fluid/reader.py | 36 ++-- .../unittests/test_decoupled_py_reader.py | 52 ++++-- 5 files changed, 181 insertions(+), 112 deletions(-) diff --git a/paddle/fluid/pybind/pybind.cc b/paddle/fluid/pybind/pybind.cc index c907cb48b8..1b53410d16 100644 --- a/paddle/fluid/pybind/pybind.cc +++ b/paddle/fluid/pybind/pybind.cc @@ -546,19 +546,39 @@ All parameter, weight, gradient are variables in Paddle. ::paddle::operators::reader::LoDTensorBlockingQueue; using LoDTensorBlockingQueueHolder = ::paddle::operators::reader::LoDTensorBlockingQueueHolder; + + using LockFreeLoDTensorBlockingQueue = + ::paddle::operators::reader::LockFreeLoDTensorBlockingQueue; + using LockFreeLoDTensorBlockingQueueHolder = + ::paddle::operators::reader::LockFreeLoDTensorBlockingQueueHolder; + py::class_>( m, "LoDTensorBlockingQueue", "") .def("push", [](LoDTensorBlockingQueue &self, - const std::vector &lod_tensor_vec) { + std::vector &lod_tensor_vec) { pybind11::gil_scoped_release release; - return self.Push(lod_tensor_vec); + return self.Push(std::move(lod_tensor_vec)); }) .def("size", &LoDTensorBlockingQueue::Size) .def("capacity", &LoDTensorBlockingQueue::Cap) .def("close", &LoDTensorBlockingQueue::Close) .def("is_closed", &LoDTensorBlockingQueue::IsClosed); + py::class_>( + m, "LockFreeLoDTensorBlockingQueue", "") + .def("push", + [](LockFreeLoDTensorBlockingQueue &self, + std::vector &lod_tensor_vec) { + pybind11::gil_scoped_release release; + return self.Push(std::move(lod_tensor_vec)); + }) + .def("size", &LockFreeLoDTensorBlockingQueue::Size) + .def("capacity", &LockFreeLoDTensorBlockingQueue::Cap) + .def("close", &LockFreeLoDTensorBlockingQueue::Close) + .def("is_closed", &LockFreeLoDTensorBlockingQueue::IsClosed); + m.def("init_lod_tensor_blocking_queue", [](Variable &var, size_t capacity) -> std::shared_ptr { @@ -568,6 +588,15 @@ All parameter, weight, gradient are variables in Paddle. }, py::return_value_policy::copy); + m.def("init_lock_free_lod_tensor_blocking_queue", + [](Variable &var, + size_t capacity) -> std::shared_ptr { + auto *holder = var.GetMutable(); + holder->InitOnce(capacity); + return holder->GetQueue(); + }, + py::return_value_policy::copy); + py::class_(m, "_Scope", R"DOC( Scope is an association of a name to Variable. All variables belong to Scope. diff --git a/paddle/fluid/pybind/reader_py.cc b/paddle/fluid/pybind/reader_py.cc index a09d18656f..22f67b38bb 100644 --- a/paddle/fluid/pybind/reader_py.cc +++ b/paddle/fluid/pybind/reader_py.cc @@ -25,77 +25,107 @@ namespace paddle { namespace pybind { -class FeedReader { +class MultiDeviceFeedReader { + public: using ResultDictList = std::vector>; - public: - FeedReader(std::unique_ptr reader, - const std::vector &names, size_t num_places, - bool drop_last = true) - : reader_(std::move(reader)), + MultiDeviceFeedReader( + const std::shared_ptr &queue, + const std::vector &names, + const std::vector &dst_places, bool use_double_buffer) + : queue_(queue), names_(names), - num_places_(num_places), - drop_last_(drop_last) {} - - ResultDictList ReadNext() { - std::vector tensors; - reader_->ReadNext(&tensors); - if (tensors.empty()) return ResultDictList(); + pool_(new ::ThreadPool(dst_places.size())) { + std::shared_ptr reader( + new operators::reader::PyReader(queue)); + + readers_.reserve(dst_places.size()); + for (auto &p : dst_places) { + auto *holder = new framework::ReaderHolder(); + if (use_double_buffer) { + holder->Reset( + framework::MakeDecoratedReader( + reader, p, 2)); + } else { + if (platform::is_gpu_place(p)) { + PADDLE_THROW( + "Place cannot be CUDAPlace when use_double_buffer is False"); + } + holder->Reset(reader); + } + readers_.emplace_back(holder); + } - PADDLE_ENFORCE(tensors.size() % names_.size() == 0, - "Tensor size: %d, names size: %d", tensors.size(), - names_.size()); + futures_.resize(dst_places.size()); + ret_.resize(dst_places.size()); + ReadAsync(); + } - size_t read_place_num = tensors.size() / names_.size(); + ResultDictList ReadNext() { + bool success = WaitFutures(); - if (drop_last_ && read_place_num != num_places_) { - return ResultDictList(); + if (!success) { + return {}; } - ResultDictList ret(read_place_num); - for (size_t i = 0; i < tensors.size(); ++i) { - ret[i / names_.size()].emplace(names_[i % names_.size()], - std::move(tensors[i])); + ResultDictList result(ret_.size()); + for (size_t i = 0; i < ret_.size(); ++i) { + for (size_t j = 0; j < names_.size(); ++j) { + result[i].emplace(names_[j], std::move(ret_[i][j])); + } } - return ret; + ReadAsync(); + return result; } - void Start() { reader_->Start(); } + void Reset() { + Shutdown(); + Start(); - void Reset() { reader_->ResetAll(); } + ReadAsync(); + } + + ~MultiDeviceFeedReader() { + queue_->Close(); + pool_.reset(); + } private: - std::unique_ptr reader_; - std::vector names_; - size_t num_places_; - bool drop_last_; -}; + bool WaitFutures() { + bool success = true; + for (auto &f : futures_) { + success &= f.get(); + } + return success; + } -static std::unique_ptr CreatePyReader( - const std::vector< - std::shared_ptr> &queues, - const std::vector &dst_places) { - std::shared_ptr reader; - if (queues.size() == 1) { - reader.reset(new operators::reader::PyReader(queues[0])); - } else { - reader.reset(new operators::reader::MultiQueuePyReader(queues)); + void Shutdown() { + for (auto &r : readers_) r->Shutdown(); } - std::vector> buffered_reader; - buffered_reader.reserve(dst_places.size()); - for (auto &p : dst_places) { - buffered_reader.emplace_back( - framework::MakeDecoratedReader( - reader, p, 2)); + + void Start() { + for (auto &r : readers_) r->Start(); } - reader = framework::MakeDecoratedReader( - buffered_reader); - auto *holder = new framework::ReaderHolder(); - holder->Reset(reader); - return std::unique_ptr(holder); -} + void ReadAsync() { + for (size_t i = 0; i < readers_.size(); ++i) { + futures_[i] = pool_->enqueue([this, i] { + readers_[i]->ReadNext(&ret_[i]); + return !ret_[i].empty(); + }); + } + } + + std::vector names_; + std::unique_ptr<::ThreadPool> pool_; + + std::shared_ptr queue_; + std::vector> readers_; + std::vector> futures_; + std::vector> ret_; + bool drop_last_; +}; namespace py = pybind11; @@ -108,22 +138,20 @@ void BindReader(py::module *module) { .def("start", &framework::ReaderHolder::Start) .def("reset", &framework::ReaderHolder::ResetAll); - py::class_(m, "FeedReader", "") - .def("read_next", &FeedReader::ReadNext, - py::call_guard()) - .def("start", &FeedReader::Start, + py::class_(m, "MultiDeviceFeedReader", "") + .def("read_next", &MultiDeviceFeedReader::ReadNext, py::call_guard()) - .def("reset", &FeedReader::Reset, + .def("reset", &MultiDeviceFeedReader::Reset, py::call_guard()); m.def("create_py_reader", - [](const std::vector< - std::shared_ptr> - queues, + [](const std::shared_ptr + &queue, const std::vector &names, - const std::vector &dst_places, bool drop_last) { - return new FeedReader(CreatePyReader(queues, dst_places), names, - dst_places.size(), drop_last); + const std::vector &dst_places, + bool use_double_buffer) { + return new MultiDeviceFeedReader(queues, names, dst_places, + use_double_buffer); }, py::return_value_policy::take_ownership); } diff --git a/python/paddle/fluid/layers/io.py b/python/paddle/fluid/layers/io.py index a9b391fd53..639be053b0 100644 --- a/python/paddle/fluid/layers/io.py +++ b/python/paddle/fluid/layers/io.py @@ -486,7 +486,8 @@ def _py_reader(capacity, lod_levels=None, name=None, use_double_buffer=True, - feed_list=None): + feed_list=None, + lock_free=False): if feed_list is not None: if not isinstance(feed_list, list): @@ -526,12 +527,17 @@ def _py_reader(capacity, double_buffer_name = "_".join([name, "double_buffer"]) var = global_scope().var(queue_name) - feed_queue = core.init_lod_tensor_blocking_queue(var, capacity) + if not lock_free: + feed_queue = core.init_lod_tensor_blocking_queue(var, capacity) + else: + feed_queue = core.init_lock_free_lod_tensor_blocking_queue(var, + capacity) startup_blk = default_startup_program().current_block() startup_var = startup_blk.create_var(name=reader_name) startup_blk.append_op( - type='create_py_reader', + type='create_py_reader' + if not lock_free else 'create_lock_free_py_reader', inputs={'blocking_queue': [queue_name]}, outputs={'Out': [startup_var]}, attrs={ @@ -638,7 +644,8 @@ def py_reader(capacity, dtypes, lod_levels=None, name=None, - use_double_buffer=True): + use_double_buffer=True, + lock_free=False): """ Create a Python reader for data feeding in Python @@ -763,7 +770,8 @@ def py_reader(capacity, dtypes=dtypes, lod_levels=lod_levels, name=name, - use_double_buffer=use_double_buffer) + use_double_buffer=use_double_buffer, + lock_free=lock_free) def create_py_reader_by_data(capacity, diff --git a/python/paddle/fluid/reader.py b/python/paddle/fluid/reader.py index 8352587f23..7c95ea20e3 100644 --- a/python/paddle/fluid/reader.py +++ b/python/paddle/fluid/reader.py @@ -17,6 +17,7 @@ import six import threading from .framework import Program, Variable, program_guard from .data_feeder import DataFeeder +import paddle.reader.decorator as decorator __all__ = ['PyReader'] @@ -36,8 +37,8 @@ def _convert_places(places): return ret -class PyReader(object): - def __init__(self, feed_list, places, capacity, multi_queue=True): +class PyReader(Reader): + def __init__(self, feed_list, places, capacity): self._tensor_reader = None self._thread = None @@ -53,15 +54,11 @@ class PyReader(object): self._queue_capacity = capacity - queue_num = len(self._places) if multi_queue else 1 - for _ in six.moves.range(queue_num): - self._queues.append( - core.init_lod_tensor_blocking_queue(core.Variable(), - self._queue_capacity)) + self.queue = core.init_lod_tensor_blocking_queue(core.Variable(), + self._queue_capacity) - self._reader = core.create_py_reader(self._queues, self._var_names, + self._reader = core.create_py_reader(self._queue, self._var_names, self._places, self._drop_last) - self._exited = True def __call__(self): assert self._tensor_reader is not None, \ @@ -77,7 +74,7 @@ class PyReader(object): def next(self): ret = self._reader.read_next() - if len(ret): + if ret: return ret else: self._reset() @@ -86,18 +83,11 @@ class PyReader(object): return Iterator(self) def _reset(self): - if not self._exited: - for q in self._queues: - q.close() - if self._thread: + self._reader.reset() self._thread.join() - self._reader.reset() - def __thread_main__(): - queue_num = len(self._queues) - idx = 0 for tensors in self._tensor_reader(): array = core.LoDTensorArray() for item in tensors: @@ -108,19 +98,13 @@ class PyReader(object): array.append(item) - if not self._queues[idx].push(array): + if not self.queue.push(array): break - idx = (idx + 1) % queue_num - - for q in self._queues: - q.close() - - self._exited = True + self.queue.close() self._thread = threading.Thread(target=__thread_main__) self._thread.daemon = True - self._exited = False self._thread.start() def set_numpy_reader(self, reader): diff --git a/python/paddle/fluid/tests/unittests/test_decoupled_py_reader.py b/python/paddle/fluid/tests/unittests/test_decoupled_py_reader.py index 807cbaf39d..dd64f10395 100644 --- a/python/paddle/fluid/tests/unittests/test_decoupled_py_reader.py +++ b/python/paddle/fluid/tests/unittests/test_decoupled_py_reader.py @@ -31,7 +31,7 @@ def random_reader(): yield image, label -def simple_fc_net(places, use_legacy_py_reader): +def simple_fc_net(places, use_legacy_py_reader, lock_free=False): startup_prog = fluid.Program() main_prog = fluid.Program() startup_prog.random_seed = 1 @@ -55,7 +55,8 @@ def simple_fc_net(places, use_legacy_py_reader): py_reader = fluid.layers.py_reader( capacity=4, shapes=[(-1, 784), (-1, 1)], - dtypes=['float32', 'int64']) + dtypes=['float32', 'int64'], + lock_free=lock_free) image, label = fluid.layers.read_file(py_reader) py_reader.decorate_paddle_reader(reader) @@ -82,7 +83,8 @@ def simple_fc_net(places, use_legacy_py_reader): class TestBase(unittest.TestCase): def run_main(self, use_legacy_py_reader, with_data_parallel, places): - with fluid.scope_guard(fluid.Scope()): + scope = fluid.Scope() + with fluid.scope_guard(scope): startup_prog, main_prog, py_reader, loss = simple_fc_net( places, use_legacy_py_reader) exe = fluid.Executor(place=places[0]) @@ -94,21 +96,29 @@ class TestBase(unittest.TestCase): loss_name=loss.name, places=places) step = 0 + step_list = [] start_t = time.time() if use_legacy_py_reader: for _ in six.moves.range(EPOCH_NUM): + step = 0 py_reader.start() while True: try: - L, = exe.run(program=prog, fetch_list=[loss]) + L, = exe.run(program=prog, + fetch_list=[loss], + use_program_cache=True) + # print('runned', step, py_reader.queue.is_closed(), py_reader.queue.size()) step += 1 except fluid.core.EOFException: + # print('try to reset') py_reader.reset() + # print('reseted') break + step_list.append(step) else: for _ in six.moves.range(EPOCH_NUM): + step = 0 for d in py_reader(): - ''' assert len(d) == len(places) for i, item in enumerate(d): image = item['image'] @@ -117,18 +127,25 @@ class TestBase(unittest.TestCase): assert label.shape() == [BATCH_SIZE, 1] assert image._place()._equals(places[i]) assert label._place()._equals(places[i]) - ''' - L, = exe.run(program=prog, feed=d, fetch_list=[loss]) + L, = exe.run(program=prog, + feed=d, + fetch_list=[loss], + use_program_cache=True) step += 1 + step_list.append(step) end_t = time.time() - return {"time": end_t - start_t, "step": step} - - def prepare_places(self, with_data_parallel): - places = [[fluid.CPUPlace()], ] - if with_data_parallel: - places.append([fluid.CPUPlace()] * 2) + ret = {"time": end_t - start_t, "step": step_list} + scope._remove_from_pool() + return ret + + def prepare_places(self, with_data_parallel, with_cpu=False, with_gpu=True): + places = [] + if with_cpu: + places.append([fluid.CPUPlace()]) + if with_data_parallel: + places.append([fluid.CPUPlace()] * 2) - if fluid.core.is_compiled_with_cuda(): + if with_gpu and fluid.core.is_compiled_with_cuda(): tmp = fluid.cuda_places() assert len(tmp) > 0, "no gpu detected" if with_data_parallel: @@ -140,7 +157,10 @@ class TestBase(unittest.TestCase): for with_data_parallel in [True, False]: for p in self.prepare_places(with_data_parallel): t = [] - for use_legacy_py_reader in [False, True]: + for use_legacy_py_reader in [ + False + ]: #[True, False]: #[False, True]: + print(p, use_legacy_py_reader) ret = self.run_main( use_legacy_py_reader=use_legacy_py_reader, with_data_parallel=with_data_parallel, @@ -148,7 +168,7 @@ class TestBase(unittest.TestCase): ret['legacy'] = use_legacy_py_reader ret['data_parallel'] = with_data_parallel ret['places'] = p - t.append(ret) + t.append([ret['step'], ]) #, ret['places']]) print(t) From c545f1ed8f0aa76053e78c165c441760871c8d03 Mon Sep 17 00:00:00 2001 From: sneaxiy Date: Mon, 25 Feb 2019 10:38:32 +0000 Subject: [PATCH 04/44] unify API test=develop --- paddle/fluid/API.spec | 10 +- paddle/fluid/framework/reader.h | 53 +------ paddle/fluid/operators/reader/CMakeLists.txt | 3 +- .../fluid/operators/reader/blocking_queue.h | 5 - .../fluid/operators/reader/compose_reader.cc | 39 ----- .../fluid/operators/reader/compose_reader.h | 34 ----- paddle/fluid/operators/reader/py_reader.cc | 37 ----- paddle/fluid/operators/reader/py_reader.h | 18 --- paddle/fluid/pybind/pybind.cc | 30 ---- paddle/fluid/pybind/reader_py.cc | 8 +- python/paddle/fluid/layers/io.py | 15 +- python/paddle/fluid/reader.py | 140 ++++++++++++++---- .../unittests/test_decoupled_py_reader.py | 76 ++++------ 13 files changed, 164 insertions(+), 304 deletions(-) delete mode 100644 paddle/fluid/operators/reader/compose_reader.cc delete mode 100644 paddle/fluid/operators/reader/compose_reader.h diff --git a/paddle/fluid/API.spec b/paddle/fluid/API.spec index 2544b7308c..946e264f05 100644 --- a/paddle/fluid/API.spec +++ b/paddle/fluid/API.spec @@ -10,6 +10,9 @@ paddle.fluid.default_startup_program ArgSpec(args=[], varargs=None, keywords=Non paddle.fluid.default_main_program ArgSpec(args=[], varargs=None, keywords=None, defaults=None) paddle.fluid.program_guard ArgSpec(args=['main_program', 'startup_program'], varargs=None, keywords=None, defaults=(None,)) paddle.fluid.name_scope ArgSpec(args=['prefix'], varargs=None, keywords=None, defaults=(None,)) +paddle.fluid.cuda_places ArgSpec(args=['device_ids'], varargs=None, keywords=None, defaults=(None,)) +paddle.fluid.cpu_places ArgSpec(args=['device_count'], varargs=None, keywords=None, defaults=(None,)) +paddle.fluid.cuda_pinned_places ArgSpec(args=['device_count'], varargs=None, keywords=None, defaults=(None,)) paddle.fluid.Executor.__init__ ArgSpec(args=['self', 'place'], varargs=None, keywords=None, defaults=None) paddle.fluid.Executor.close ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None) paddle.fluid.Executor.run ArgSpec(args=['self', 'program', 'feed', 'fetch_list', 'feed_var_name', 'fetch_var_name', 'scope', 'return_numpy', 'use_program_cache'], varargs=None, keywords=None, defaults=(None, None, None, 'feed', 'fetch', None, True, False)) @@ -44,7 +47,7 @@ paddle.fluid.AsyncExecutor.run ArgSpec(args=['self', 'program', 'data_feed', 'fi paddle.fluid.AsyncExecutor.save_model ArgSpec(args=['self', 'save_path'], varargs=None, keywords=None, defaults=None) paddle.fluid.AsyncExecutor.stop ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None) paddle.fluid.CompiledProgram.__init__ ArgSpec(args=['self', 'program'], varargs=None, keywords=None, defaults=None) -paddle.fluid.CompiledProgram.with_data_parallel ArgSpec(args=['self', 'loss_name', 'build_strategy', 'exec_strategy', 'share_vars_from'], varargs=None, keywords=None, defaults=(None, None, None, None)) +paddle.fluid.CompiledProgram.with_data_parallel ArgSpec(args=['self', 'loss_name', 'build_strategy', 'exec_strategy', 'share_vars_from', 'places'], varargs=None, keywords=None, defaults=(None, None, None, None, None)) paddle.fluid.CompiledProgram.with_inference_optimize ArgSpec(args=['self', 'config'], varargs=None, keywords=None, defaults=None) paddle.fluid.ExecutionStrategy.__init__ __init__(self: paddle.fluid.core.ParallelExecutor.ExecutionStrategy) -> None paddle.fluid.BuildStrategy.GradientScaleStrategy.__init__ __init__(self: paddle.fluid.core.ParallelExecutor.BuildStrategy.GradientScaleStrategy, arg0: int) -> None @@ -58,6 +61,11 @@ paddle.fluid.io.load_params ArgSpec(args=['executor', 'dirname', 'main_program', paddle.fluid.io.load_persistables ArgSpec(args=['executor', 'dirname', 'main_program', 'filename'], varargs=None, keywords=None, defaults=(None, None)) paddle.fluid.io.save_inference_model ArgSpec(args=['dirname', 'feeded_var_names', 'target_vars', 'executor', 'main_program', 'model_filename', 'params_filename', 'export_for_deployment'], varargs=None, keywords=None, defaults=(None, None, None, True)) paddle.fluid.io.load_inference_model ArgSpec(args=['dirname', 'executor', 'model_filename', 'params_filename', 'pserver_endpoints'], varargs=None, keywords=None, defaults=(None, None, None)) +paddle.fluid.io.PyReader.__init__ ArgSpec(args=['self', 'feed_list', 'capacity', 'use_double_buffer', 'iterable'], varargs=None, keywords=None, defaults=(True, True)) +paddle.fluid.io.PyReader.decorate_paddle_reader ArgSpec(args=['self', 'reader', 'places'], varargs=None, keywords=None, defaults=(None,)) +paddle.fluid.io.PyReader.decorate_tensor_provider ArgSpec(args=['self', 'reader', 'places'], varargs=None, keywords=None, defaults=(None,)) +paddle.fluid.io.PyReader.reset ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None) +paddle.fluid.io.PyReader.start ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None) paddle.fluid.initializer.ConstantInitializer.__init__ ArgSpec(args=['self', 'value', 'force_cpu'], varargs=None, keywords=None, defaults=(0.0, False)) paddle.fluid.initializer.UniformInitializer.__init__ ArgSpec(args=['self', 'low', 'high', 'seed'], varargs=None, keywords=None, defaults=(-1.0, 1.0, 0)) paddle.fluid.initializer.NormalInitializer.__init__ ArgSpec(args=['self', 'loc', 'scale', 'seed'], varargs=None, keywords=None, defaults=(0.0, 1.0, 0)) diff --git a/paddle/fluid/framework/reader.h b/paddle/fluid/framework/reader.h index 61120dcf12..82562bf883 100644 --- a/paddle/fluid/framework/reader.h +++ b/paddle/fluid/framework/reader.h @@ -54,7 +54,6 @@ class ReaderBase { private: friend class DecoratedReader; - friend class MultiDecoratedReader; // These methods can be only invoked inside DecoratedReader to record the // decorating chain. void InsertDecoratedReader( @@ -63,20 +62,15 @@ class ReaderBase { std::vector> decorated_readers_; }; -class DecoratedReaderBase : public ReaderBase { - public: - virtual void RegisterDecorateChain() = 0; -}; - -class DecoratedReader : public DecoratedReaderBase, +class DecoratedReader : public ReaderBase, public std::enable_shared_from_this { public: explicit DecoratedReader(const std::shared_ptr& reader) - : DecoratedReaderBase(), reader_(reader) { + : ReaderBase(), reader_(reader) { PADDLE_ENFORCE_NOT_NULL(reader_); } - void RegisterDecorateChain() final { + void RegisterDecorateChain() { reader_->InsertDecoratedReader(shared_from_this()); } @@ -90,41 +84,6 @@ class DecoratedReader : public DecoratedReaderBase, std::shared_ptr reader_; }; -class MultiDecoratedReader - : public DecoratedReaderBase, - public std::enable_shared_from_this { - public: - explicit MultiDecoratedReader( - const std::vector>& readers) - : readers_(readers) { - PADDLE_ENFORCE(!readers_.empty()); - for (auto& r : readers_) { - PADDLE_ENFORCE_NOT_NULL(r); - } - } - - void RegisterDecorateChain() final { - for (auto& r : readers_) { - r->InsertDecoratedReader(shared_from_this()); - } - } - - protected: - void ShutdownImpl() override { - for (auto& r : readers_) { - r->Shutdown(); - } - } - - void StartImpl() override { - for (auto& r : readers_) { - r->Start(); - } - } - - std::vector> readers_; -}; - // FileReader is just a conceptual class. class FileReader : public ReaderBase {}; @@ -173,10 +132,8 @@ class ReaderHolder { }; template -inline std::shared_ptr MakeDecoratedReader( - ARGS&&... args) { - std::shared_ptr reader( - new T(std::forward(args)...)); +inline std::shared_ptr MakeDecoratedReader(ARGS&&... args) { + std::shared_ptr reader(new T(std::forward(args)...)); reader->RegisterDecorateChain(); return reader; } diff --git a/paddle/fluid/operators/reader/CMakeLists.txt b/paddle/fluid/operators/reader/CMakeLists.txt index 2701e10b30..5ee1206175 100644 --- a/paddle/fluid/operators/reader/CMakeLists.txt +++ b/paddle/fluid/operators/reader/CMakeLists.txt @@ -18,7 +18,6 @@ function(reader_library TARGET_NAME) endfunction() cc_library(py_reader SRCS py_reader.cc DEPS reader) -cc_library(compose_reader SRCS compose_reader.cc DEPS reader) cc_library(buffered_reader SRCS buffered_reader.cc DEPS reader simple_threadpool) reader_library(open_files_op SRCS open_files_op.cc DEPS buffered_reader) @@ -41,7 +40,7 @@ cc_test(reader_blocking_queue_test SRCS reader_blocking_queue_test.cc) # Export local libraries to parent # set(READER_LIBRARY ${LOCAL_READER_LIBS} PARENT_SCOPE) -op_library(read_op DEPS py_reader compose_reader buffered_reader) +op_library(read_op DEPS py_reader buffered_reader) foreach(src ${LOCAL_READER_LIBS}) set(OP_LIBRARY ${src} ${OP_LIBRARY} CACHE INTERNAL "op libs") diff --git a/paddle/fluid/operators/reader/blocking_queue.h b/paddle/fluid/operators/reader/blocking_queue.h index b76f482c57..7962c0332d 100644 --- a/paddle/fluid/operators/reader/blocking_queue.h +++ b/paddle/fluid/operators/reader/blocking_queue.h @@ -114,11 +114,6 @@ class BlockingQueue { return queue_.size(); } - void Clear() { - std::lock_guard lock(mutex_); - queue_.clear(); - } - private: size_t capacity_; bool speed_test_mode_; diff --git a/paddle/fluid/operators/reader/compose_reader.cc b/paddle/fluid/operators/reader/compose_reader.cc deleted file mode 100644 index 4b88b9331c..0000000000 --- a/paddle/fluid/operators/reader/compose_reader.cc +++ /dev/null @@ -1,39 +0,0 @@ -// 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/operators/reader/compose_reader.h" - -namespace paddle { -namespace operators { -namespace reader { - -ComposeReader::ComposeReader( - const std::vector> &readers) - : framework::MultiDecoratedReader(readers) {} - -void ComposeReader::ReadNext(std::vector *out) { - out->clear(); - std::vector each_ret; - for (auto &r : readers_) { - r->ReadNext(&each_ret); - out->reserve(out->size() + each_ret.size()); - for (auto &data : each_ret) { - out->emplace_back(std::move(data)); - } - } -} - -} // namespace reader -} // namespace operators -} // namespace paddle diff --git a/paddle/fluid/operators/reader/compose_reader.h b/paddle/fluid/operators/reader/compose_reader.h deleted file mode 100644 index c9e2a2d72f..0000000000 --- a/paddle/fluid/operators/reader/compose_reader.h +++ /dev/null @@ -1,34 +0,0 @@ -// 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 "paddle/fluid/framework/reader.h" - -namespace paddle { -namespace operators { -namespace reader { - -class ComposeReader : public framework::MultiDecoratedReader { - public: - explicit ComposeReader( - const std::vector> &readers); - - void ReadNext(std::vector *out) override; -}; - -} // namespace reader -} // namespace operators -} // namespace paddle diff --git a/paddle/fluid/operators/reader/py_reader.cc b/paddle/fluid/operators/reader/py_reader.cc index dc84faa974..f2c28c1df8 100644 --- a/paddle/fluid/operators/reader/py_reader.cc +++ b/paddle/fluid/operators/reader/py_reader.cc @@ -36,43 +36,6 @@ void PyReader::Shutdown() { queue_->Close(); } void PyReader::Start() { queue_->ReOpen(); } -MultiQueuePyReader::MultiQueuePyReader( - const std::vector>& queues) - : queues_(queues) { - PADDLE_ENFORCE(!queues_.empty()); - for (auto& q : queues_) { - PADDLE_ENFORCE_NOT_NULL(q); - } -} - -void MultiQueuePyReader::ReadNext(std::vector* out) { - auto idx = read_out_idx_.fetch_add(1) % queues_.size(); - for (size_t i = 0; i < queues_.size(); ++i) { - *out = queues_[idx]->Pop(); - if (!out->empty()) return; - idx = (idx + 1) % queues_.size(); - } -} - -MultiQueuePyReader::~MultiQueuePyReader() { - for (auto& q : queues_) { - q->Close(); - } -} - -void MultiQueuePyReader::Shutdown() { - for (auto& q : queues_) { - q->Close(); - } - read_out_idx_.store(0, std::memory_order::memory_order_seq_cst); -} - -void MultiQueuePyReader::Start() { - for (auto& q : queues_) { - q->ReOpen(); - } -} - } // namespace reader } // namespace operators } // namespace paddle diff --git a/paddle/fluid/operators/reader/py_reader.h b/paddle/fluid/operators/reader/py_reader.h index 146a2351e5..7d760eca64 100644 --- a/paddle/fluid/operators/reader/py_reader.h +++ b/paddle/fluid/operators/reader/py_reader.h @@ -39,24 +39,6 @@ class PyReader : public framework::FileReader { std::shared_ptr queue_; }; -class MultiQueuePyReader : public framework::FileReader { - public: - explicit MultiQueuePyReader( - const std::vector>& queues); - - void ReadNext(std::vector* out) override; - - ~MultiQueuePyReader(); - - void Shutdown() override; - - void Start() override; - - private: - std::vector> queues_; - std::atomic read_out_idx_{0}; -}; - } // namespace reader } // namespace operators } // namespace paddle diff --git a/paddle/fluid/pybind/pybind.cc b/paddle/fluid/pybind/pybind.cc index 1b53410d16..2acedca245 100644 --- a/paddle/fluid/pybind/pybind.cc +++ b/paddle/fluid/pybind/pybind.cc @@ -547,11 +547,6 @@ All parameter, weight, gradient are variables in Paddle. using LoDTensorBlockingQueueHolder = ::paddle::operators::reader::LoDTensorBlockingQueueHolder; - using LockFreeLoDTensorBlockingQueue = - ::paddle::operators::reader::LockFreeLoDTensorBlockingQueue; - using LockFreeLoDTensorBlockingQueueHolder = - ::paddle::operators::reader::LockFreeLoDTensorBlockingQueueHolder; - py::class_>( m, "LoDTensorBlockingQueue", "") .def("push", @@ -565,20 +560,6 @@ All parameter, weight, gradient are variables in Paddle. .def("close", &LoDTensorBlockingQueue::Close) .def("is_closed", &LoDTensorBlockingQueue::IsClosed); - py::class_>( - m, "LockFreeLoDTensorBlockingQueue", "") - .def("push", - [](LockFreeLoDTensorBlockingQueue &self, - std::vector &lod_tensor_vec) { - pybind11::gil_scoped_release release; - return self.Push(std::move(lod_tensor_vec)); - }) - .def("size", &LockFreeLoDTensorBlockingQueue::Size) - .def("capacity", &LockFreeLoDTensorBlockingQueue::Cap) - .def("close", &LockFreeLoDTensorBlockingQueue::Close) - .def("is_closed", &LockFreeLoDTensorBlockingQueue::IsClosed); - m.def("init_lod_tensor_blocking_queue", [](Variable &var, size_t capacity) -> std::shared_ptr { @@ -588,15 +569,6 @@ All parameter, weight, gradient are variables in Paddle. }, py::return_value_policy::copy); - m.def("init_lock_free_lod_tensor_blocking_queue", - [](Variable &var, - size_t capacity) -> std::shared_ptr { - auto *holder = var.GetMutable(); - holder->InitOnce(capacity); - return holder->GetQueue(); - }, - py::return_value_policy::copy); - py::class_(m, "_Scope", R"DOC( Scope is an association of a name to Variable. All variables belong to Scope. @@ -777,8 +749,6 @@ All parameter, weight, gradient are variables in Paddle. .def("_equals", &IsSamePlace) .def("_equals", &IsSamePlace) - .def("gpu_device_id", - [](platform::CUDAPlace &self) { return self.device; }) .def("__str__", string::to_string); py::class_(m, "CPUPlace") diff --git a/paddle/fluid/pybind/reader_py.cc b/paddle/fluid/pybind/reader_py.cc index 22f67b38bb..8af0490310 100644 --- a/paddle/fluid/pybind/reader_py.cc +++ b/paddle/fluid/pybind/reader_py.cc @@ -17,7 +17,6 @@ #include #include "paddle/fluid/framework/reader.h" #include "paddle/fluid/operators/reader/buffered_reader.h" -#include "paddle/fluid/operators/reader/compose_reader.h" #include "paddle/fluid/operators/reader/py_reader.h" #include "paddle/fluid/platform/place.h" #include "pybind11/stl.h" @@ -82,7 +81,6 @@ class MultiDeviceFeedReader { void Reset() { Shutdown(); Start(); - ReadAsync(); } @@ -117,14 +115,14 @@ class MultiDeviceFeedReader { } } + std::shared_ptr queue_; std::vector names_; std::unique_ptr<::ThreadPool> pool_; - std::shared_ptr queue_; std::vector> readers_; + std::vector> futures_; std::vector> ret_; - bool drop_last_; }; namespace py = pybind11; @@ -150,7 +148,7 @@ void BindReader(py::module *module) { const std::vector &names, const std::vector &dst_places, bool use_double_buffer) { - return new MultiDeviceFeedReader(queues, names, dst_places, + return new MultiDeviceFeedReader(queue, names, dst_places, use_double_buffer); }, py::return_value_policy::take_ownership); diff --git a/python/paddle/fluid/layers/io.py b/python/paddle/fluid/layers/io.py index 639be053b0..6b9e000358 100644 --- a/python/paddle/fluid/layers/io.py +++ b/python/paddle/fluid/layers/io.py @@ -486,8 +486,7 @@ def _py_reader(capacity, lod_levels=None, name=None, use_double_buffer=True, - feed_list=None, - lock_free=False): + feed_list=None): if feed_list is not None: if not isinstance(feed_list, list): @@ -527,11 +526,7 @@ def _py_reader(capacity, double_buffer_name = "_".join([name, "double_buffer"]) var = global_scope().var(queue_name) - if not lock_free: - feed_queue = core.init_lod_tensor_blocking_queue(var, capacity) - else: - feed_queue = core.init_lock_free_lod_tensor_blocking_queue(var, - capacity) + feed_queue = core.init_lod_tensor_blocking_queue(var, capacity) startup_blk = default_startup_program().current_block() startup_var = startup_blk.create_var(name=reader_name) @@ -644,8 +639,7 @@ def py_reader(capacity, dtypes, lod_levels=None, name=None, - use_double_buffer=True, - lock_free=False): + use_double_buffer=True): """ Create a Python reader for data feeding in Python @@ -770,8 +764,7 @@ def py_reader(capacity, dtypes=dtypes, lod_levels=lod_levels, name=name, - use_double_buffer=use_double_buffer, - lock_free=lock_free) + use_double_buffer=use_double_buffer) def create_py_reader_by_data(capacity, diff --git a/python/paddle/fluid/reader.py b/python/paddle/fluid/reader.py index 7c95ea20e3..f29231589e 100644 --- a/python/paddle/fluid/reader.py +++ b/python/paddle/fluid/reader.py @@ -15,9 +15,11 @@ import core import six import threading -from .framework import Program, Variable, program_guard +from .framework import Program, Variable, program_guard, default_main_program, default_startup_program +from .executor import global_scope from .data_feeder import DataFeeder -import paddle.reader.decorator as decorator +from .layers.io import monkey_patch_reader_methods, _copy_reader_var_, double_buffer +import unique_name __all__ = ['PyReader'] @@ -37,30 +39,101 @@ def _convert_places(places): return ret -class PyReader(Reader): - def __init__(self, feed_list, places, capacity): +class PyReader(object): + unique_name_generator = unique_name.UniqueNameGenerator() + + def __init__(self, + feed_list, + capacity, + use_double_buffer=True, + iterable=True): self._tensor_reader = None self._thread = None - - # TODO(zjl): to support drop_last = False - self._drop_last = True - + self._iterable = iterable + self._use_double_buffer = use_double_buffer + self._capacity = capacity self._feed_list = feed_list - self._var_names = [v.name for v in feed_list] - - self._queues = [] + self._scope = global_scope() + if not self._iterable: + self._init_non_iterable() + def _init_iterable(self, places): + self._var_names = [v.name for v in self._feed_list] self._places = _convert_places(places) - - self._queue_capacity = capacity - - self.queue = core.init_lod_tensor_blocking_queue(core.Variable(), - self._queue_capacity) - - self._reader = core.create_py_reader(self._queue, self._var_names, - self._places, self._drop_last) + self._queue = core.init_lod_tensor_blocking_queue(core.Variable(), + self._capacity) + self._reader = core.create_py_reader( + self.queue, self._var_names, self._places, self._use_double_buffer) + + def _init_non_iterable(self): + lod_levels = [] + dtypes = [] + shape_concat = [] + ranks = [] + shapes = [] + + for feed_data in self._feed_list: + dtypes.append(feed_data.dtype) + shape_concat.extend(feed_data.shape) + ranks.append(len(feed_data.shape)) + shapes.append(feed_data.shape) + lod_levels.append(feed_data.lod_level) + + queue_name = PyReader.unique_name_generator('lod_tensor_blocking_queue') + reader_name = PyReader.unique_name_generator('create_py_reader') + double_buffer_name = PyReader.unique_name_generator('double_buffer') + + var = self._scope.var(queue_name) + self._queue = core.init_lod_tensor_blocking_queue(var, self._capacity) + + startup_blk = default_startup_program().current_block() + startup_var = startup_blk.create_var(name=reader_name) + + startup_blk.append_op( + type='create_py_reader', + inputs={'blocking_queue': [queue_name]}, + outputs={'Out': [startup_var]}, + attrs={ + 'shape_concat': shape_concat, + 'lod_levels': lod_levels, + 'ranks': ranks + }) + + startup_var.desc.set_dtypes(dtypes) + startup_var.persistable = True + + main_prog_var = _copy_reader_var_( + default_main_program().current_block(), startup_var) + + main_prog_var.stop_gradient = True + main_prog_var.persistable = True + + reader = monkey_patch_reader_methods(main_prog_var) + if self._use_double_buffer: + double_buffer_reader = double_buffer( + reader, name=double_buffer_name) + # we return a double buffer reader. However, the reset method comes from + # py_reader. + double_buffer_reader.reset = reader.reset + reader = double_buffer_reader + + self._reader = reader + + default_main_program().current_block().append_op( + type='read', + inputs={'Reader': [self._reader]}, + outputs={'Out': self._feed_list}) + + @property + def queue(self): + return self._queue + + @property + def iterable(self): + return self._iterable def __call__(self): + assert self.iterable, "PyReader is not iterable" assert self._tensor_reader is not None, \ "Data source of PyReader has not set yet" @@ -80,13 +153,22 @@ class PyReader(Reader): self._reset() raise StopIteration + self._start() return Iterator(self) def _reset(self): - if self._thread: - self._reader.reset() - self._thread.join() + self._reader.reset() + self._thread.join() + + def start(self): + assert not self._iterable, "start() cannot be called when PyReader is iterable" + self._start() + def reset(self): + assert not self._iterable, "reset() cannot be called when PyReader is iterable" + self._reset() + + def _start(self): def __thread_main__(): for tensors in self._tensor_reader(): array = core.LoDTensorArray() @@ -98,16 +180,16 @@ class PyReader(Reader): array.append(item) - if not self.queue.push(array): + if not self._queue.push(array): break - self.queue.close() + self._queue.close() self._thread = threading.Thread(target=__thread_main__) self._thread.daemon = True self._thread.start() - def set_numpy_reader(self, reader): + def decorate_paddle_reader(self, reader, places=None): assert self._tensor_reader is None, \ "Cannot reset the data source of PyReader" with program_guard(Program(), Program()): @@ -119,10 +201,12 @@ class PyReader(Reader): for slots in paddle_reader(): yield [slots[var.name] for var in self._feed_list] - self.set_tensor_reader(__tensor_reader_impl__) + self.decorate_tensor_provider(__tensor_reader_impl__, places) - def set_tensor_reader(self, reader): + def decorate_tensor_provider(self, reader, places=None): assert self._tensor_reader is None, \ "Cannot reset the data source of PyReader" self._tensor_reader = reader - self._reset() + if self._iterable: + assert places is not None, "Places cannot be None when py_reader is iterable" + self._init_iterable(places) diff --git a/python/paddle/fluid/tests/unittests/test_decoupled_py_reader.py b/python/paddle/fluid/tests/unittests/test_decoupled_py_reader.py index dd64f10395..96a11edd49 100644 --- a/python/paddle/fluid/tests/unittests/test_decoupled_py_reader.py +++ b/python/paddle/fluid/tests/unittests/test_decoupled_py_reader.py @@ -31,35 +31,22 @@ def random_reader(): yield image, label -def simple_fc_net(places, use_legacy_py_reader, lock_free=False): +def simple_fc_net(places, use_legacy_py_reader, use_double_buffer): startup_prog = fluid.Program() main_prog = fluid.Program() startup_prog.random_seed = 1 main_prog.random_seed = 1 - reader = paddle.batch(random_reader, batch_size=BATCH_SIZE) with fluid.unique_name.guard(): with fluid.program_guard(main_prog, startup_prog): - if not use_legacy_py_reader: - image = fluid.layers.data( - name='image', shape=[784], dtype='float32') - label = fluid.layers.data( - name='label', shape=[1], dtype='int64') - py_reader = fluid.io.PyReader( - feed_list=[image, label], - places=places, - capacity=4, - multi_queue=False) - py_reader.set_numpy_reader(reader) - else: - py_reader = fluid.layers.py_reader( - capacity=4, - shapes=[(-1, 784), (-1, 1)], - dtypes=['float32', 'int64'], - lock_free=lock_free) - image, label = fluid.layers.read_file(py_reader) - py_reader.decorate_paddle_reader(reader) - + image = fluid.layers.data( + name='image', shape=[784], dtype='float32') + label = fluid.layers.data(name='label', shape=[1], dtype='int64') + py_reader = fluid.io.PyReader( + feed_list=[image, label], + capacity=4, + iterable=not use_legacy_py_reader, + use_double_buffer=use_double_buffer) hidden = image for hidden_size in [10, 20, 30]: hidden = fluid.layers.fc( @@ -82,11 +69,19 @@ def simple_fc_net(places, use_legacy_py_reader, lock_free=False): class TestBase(unittest.TestCase): - def run_main(self, use_legacy_py_reader, with_data_parallel, places): + def run_main(self, use_legacy_py_reader, with_data_parallel, places, + use_double_buffer): scope = fluid.Scope() with fluid.scope_guard(scope): startup_prog, main_prog, py_reader, loss = simple_fc_net( - places, use_legacy_py_reader) + places, use_legacy_py_reader, use_double_buffer) + + reader = paddle.batch(random_reader, batch_size=BATCH_SIZE) + + ps = places if use_double_buffer else fluid.cpu_places(len(places)) + py_reader.decorate_paddle_reader( + reader, places=ps if py_reader.iterable else None) + exe = fluid.Executor(place=places[0]) exe.run(startup_prog) @@ -98,7 +93,7 @@ class TestBase(unittest.TestCase): step = 0 step_list = [] start_t = time.time() - if use_legacy_py_reader: + if not py_reader.iterable: for _ in six.moves.range(EPOCH_NUM): step = 0 py_reader.start() @@ -107,12 +102,9 @@ class TestBase(unittest.TestCase): L, = exe.run(program=prog, fetch_list=[loss], use_program_cache=True) - # print('runned', step, py_reader.queue.is_closed(), py_reader.queue.size()) step += 1 except fluid.core.EOFException: - # print('try to reset') py_reader.reset() - # print('reseted') break step_list.append(step) else: @@ -125,8 +117,8 @@ class TestBase(unittest.TestCase): label = item['label'] assert image.shape() == [BATCH_SIZE, 784] assert label.shape() == [BATCH_SIZE, 1] - assert image._place()._equals(places[i]) - assert label._place()._equals(places[i]) + assert image._place()._equals(ps[i]) + assert label._place()._equals(ps[i]) L, = exe.run(program=prog, feed=d, fetch_list=[loss], @@ -138,7 +130,7 @@ class TestBase(unittest.TestCase): scope._remove_from_pool() return ret - def prepare_places(self, with_data_parallel, with_cpu=False, with_gpu=True): + def prepare_places(self, with_data_parallel, with_cpu=True, with_gpu=True): places = [] if with_cpu: places.append([fluid.CPUPlace()]) @@ -156,21 +148,13 @@ class TestBase(unittest.TestCase): def test_main(self): for with_data_parallel in [True, False]: for p in self.prepare_places(with_data_parallel): - t = [] - for use_legacy_py_reader in [ - False - ]: #[True, False]: #[False, True]: - print(p, use_legacy_py_reader) - ret = self.run_main( - use_legacy_py_reader=use_legacy_py_reader, - with_data_parallel=with_data_parallel, - places=p) - ret['legacy'] = use_legacy_py_reader - ret['data_parallel'] = with_data_parallel - ret['places'] = p - t.append([ret['step'], ]) #, ret['places']]) - - print(t) + for use_double_buffer in [False, True]: + for use_legacy_py_reader in [False, True]: + ret = self.run_main( + use_legacy_py_reader=use_legacy_py_reader, + with_data_parallel=with_data_parallel, + places=p, + use_double_buffer=use_double_buffer) if __name__ == '__main__': From 2e67f8ae88db689494dff7ba71bfb7a7322d1447 Mon Sep 17 00:00:00 2001 From: sneaxiy Date: Tue, 26 Feb 2019 02:39:14 +0000 Subject: [PATCH 05/44] add doc test=develop --- python/paddle/fluid/framework.py | 49 +++++++++++++++ python/paddle/fluid/layers/io.py | 3 +- python/paddle/fluid/reader.py | 102 +++++++++++++++++++++++++++++++ 3 files changed, 152 insertions(+), 2 deletions(-) diff --git a/python/paddle/fluid/framework.py b/python/paddle/fluid/framework.py index 98200e2efb..00b6e7afa0 100644 --- a/python/paddle/fluid/framework.py +++ b/python/paddle/fluid/framework.py @@ -96,6 +96,27 @@ def _cpu_num(): def cuda_places(device_ids=None): + ''' + Create a list of :code:`fluid.CUDAPlace` objects. + + If :code:`device_ids` is None, environment variable of + :code:`FLAGS_selected_gpus` would be checked first. If + :code:`FLAGS_selected_gpus=0,1,2`, the returned list would + be [fluid.CUDAPlace(0), fluid.CUDAPlace(1), fluid.CUDAPlace(2)]. + If :code:`FLAGS_selected_gpus` is not set, all visible + gpu places would be returned. + + If :code:`device_ids` is not None, it should be the device + ids of gpus. For example, if :code:`device_ids=[0,1,2]`, + the returned list would be + [fluid.CUDAPlace(0), fluid.CUDAPlace(1), fluid.CUDAPlace(2)]. + + Args: + device_ids (None|list(int)|tuple(int)): gpu device id list. + + Returns: + out (list(fluid.CUDAPlace)): gpu place list. + ''' assert core.is_compiled_with_cuda(), \ "Not compiled with CUDA" if device_ids is None: @@ -110,12 +131,40 @@ def cuda_places(device_ids=None): def cpu_places(device_count=None): + ''' + Create a list of :code:`fluid.CPUPlace` objects. + + If :code:`device_count` is None, the device count would + be determined by environment variable :code:`CPU_NUM`. + If :code:`CPU_NUM` is not set, the device count would + be determined by :code:`multiprocessing.cpu_count()`. + + Args: + device_count (None|int): device number. + + Returns: + out (list(fluid.CPUPlace)): cpu place list. + ''' if device_count is None: device_count = _cpu_num() return [core.CPUPlace()] * device_count def cuda_pinned_places(device_count=None): + ''' + Create a list of :code:`fluid.CUDAPinnedPlace` objects. + + If :code:`device_count` is None, the device count would + be determined by environment variable :code:`CPU_NUM`. + If :code:`CPU_NUM` is not set, the device count would + be determined by :code:`multiprocessing.cpu_count()`. + + Args: + device_count (None|int): device number. + + Returns: + out (list(fluid.CUDAPinnedPlace)): cuda pinned place list. + ''' assert core.is_compiled_with_cuda(), \ "Not compiled with CUDA" if device_count is None: diff --git a/python/paddle/fluid/layers/io.py b/python/paddle/fluid/layers/io.py index 6b9e000358..a9b391fd53 100644 --- a/python/paddle/fluid/layers/io.py +++ b/python/paddle/fluid/layers/io.py @@ -531,8 +531,7 @@ def _py_reader(capacity, startup_blk = default_startup_program().current_block() startup_var = startup_blk.create_var(name=reader_name) startup_blk.append_op( - type='create_py_reader' - if not lock_free else 'create_lock_free_py_reader', + type='create_py_reader', inputs={'blocking_queue': [queue_name]}, outputs={'Out': [startup_var]}, attrs={ diff --git a/python/paddle/fluid/reader.py b/python/paddle/fluid/reader.py index f29231589e..a0ce914822 100644 --- a/python/paddle/fluid/reader.py +++ b/python/paddle/fluid/reader.py @@ -47,6 +47,76 @@ class PyReader(object): capacity, use_double_buffer=True, iterable=True): + """ + Create a reader object for data feeding in Python. + Data would be prefetched using Python thread and be pushed + into a queue asynchronously. Data in the queue would be extracted + automatically when `Executor.run(...)` is called. + + Args: + feed_list (list(Variable)|tuple(Variable)): feed variable list. + The variables should be created by :code:`fluid.layers.data()`. + capacity (int): capacity of the queue maintained in PyReader object. + use_double_buffer (bool): whether to use double_buffer_reader to + speed up data feeding. + iterable (bool): whether the created reader object is iterable. + + Returns: + reader (Reader): the created reader object. + + Examples: + 1. If iterable = False, the created PyReader object is almost the + same as :code:`fluid.layers.py_reader()`. Operators would be + inserted into the program. User should call :code:`start()` + before each epoch and catch :code:`fluid.core.EOFException` + thrown by :code:`Executor.run()` when epoch ends. Once the + exception is caught, user should call :code:`reset()` to reset + the reader manually. + + .. code-block:: python + + image = fluid.layers.data( + name='image', shape=[784], dtype='float32') + label = fluid.layers.data( + name='label', shape=[1], dtype='int64') + + reader = fluid.io.PyReader(feed_list=[image, label], + capacity=4, iterable=False) + reader.decorate_paddle_reader(user_defined_reader) + ... # definition of network is omitted + executor.run(fluid.default_main_program()) + for _ in range(EPOCH_NUM): + reader.start() + while True: + try: + executor.run(feed=None, ...) + except fluid.core.EOFException: + reader.reset() + break + + 2. If iterable=True, the created PyReader object is decoupled with + the program. No operator would be inserted into the program. + In this case, the created reader is a Python generator, which + is iterable. User should feed the data yielded from PyReader + object into :code:`Executor.run(feed=...)`. + + .. code-block:: python + + image = fluid.layers.data( + name='image', shape=[784], dtype='float32') + label = fluid.layers.data( + name='label', shape=[1], dtype='int64') + + reader = fluid.io.PyReader(feed_list=[image, label], + capacity=4, iterable=True) + reader.decorate_paddle_reader(user_defined_reader, + places=fluid.cuda_places()) + ... # definition of network is omitted + executor.run(fluid.default_main_program()) + for _ in range(EPOCH_NUM): + for data in reader(): + executor.run(feed=data, ...) + """ self._tensor_reader = None self._thread = None self._iterable = iterable @@ -161,10 +231,18 @@ class PyReader(object): self._thread.join() def start(self): + ''' + Start the data feeding thread. + Can only call when the reader object is not iterable. + ''' assert not self._iterable, "start() cannot be called when PyReader is iterable" self._start() def reset(self): + ''' + Reset the reader object when :code:`fluid.core.EOFException` raises. + Can only call when the reader object is not iterable. + ''' assert not self._iterable, "reset() cannot be called when PyReader is iterable" self._reset() @@ -190,6 +268,18 @@ class PyReader(object): self._thread.start() def decorate_paddle_reader(self, reader, places=None): + ''' + Set the data source of the PyReader object. + + The provided :code:`reader` should be a Python generator, + which yields numpy-typed batched data. + + :code:`places` must be set when the PyReader object is iterable. + + Args: + reader (generator): Python generator that yields numpy-typed + batched data. + ''' assert self._tensor_reader is None, \ "Cannot reset the data source of PyReader" with program_guard(Program(), Program()): @@ -204,6 +294,18 @@ class PyReader(object): self.decorate_tensor_provider(__tensor_reader_impl__, places) def decorate_tensor_provider(self, reader, places=None): + ''' + Set the data source of the PyReader object. + + The provided :code:`reader` should be a Python generator, + which yields LoDTensor-typed batched data. + + :code:`places` must be set when the PyReader object is iterable. + + Args: + reader (generator): Python generator that yields LoDTensor-typed + batched data. + ''' assert self._tensor_reader is None, \ "Cannot reset the data source of PyReader" self._tensor_reader = reader From 7b5a9d75d94c3b79d772a31a8185628355901c0d Mon Sep 17 00:00:00 2001 From: sneaxiy Date: Wed, 27 Feb 2019 03:54:45 +0000 Subject: [PATCH 06/44] add cache reader test=develop --- paddle/fluid/API.spec | 1 + python/paddle/fluid/reader.py | 6 ++++-- python/paddle/reader/decorator.py | 27 ++++++++++++++++++++++++++- 3 files changed, 31 insertions(+), 3 deletions(-) diff --git a/paddle/fluid/API.spec b/paddle/fluid/API.spec index dad729bd15..db3739e65b 100644 --- a/paddle/fluid/API.spec +++ b/paddle/fluid/API.spec @@ -511,6 +511,7 @@ paddle.fluid.unique_name.guard ArgSpec(args=['new_generator'], varargs=None, key paddle.fluid.recordio_writer.convert_reader_to_recordio_file ArgSpec(args=['filename', 'reader_creator', 'feeder', 'compressor', 'max_num_records', 'feed_order'], varargs=None, keywords=None, defaults=(Compressor.Snappy, 1000, None)) paddle.fluid.recordio_writer.convert_reader_to_recordio_files ArgSpec(args=['filename', 'batch_per_file', 'reader_creator', 'feeder', 'compressor', 'max_num_records', 'feed_order'], varargs=None, keywords=None, defaults=(Compressor.Snappy, 1000, None)) paddle.fluid.Scope Scope() -> paddle.fluid.core._Scope +paddle.reader.cache ArgSpec(args=['reader'], varargs=None, keywords=None, defaults=None) paddle.reader.map_readers ArgSpec(args=['func'], varargs='readers', keywords=None, defaults=None) paddle.reader.buffered ArgSpec(args=['reader', 'size'], varargs=None, keywords=None, defaults=None) paddle.reader.compose ArgSpec(args=[], varargs='readers', keywords='kwargs', defaults=None) diff --git a/python/paddle/fluid/reader.py b/python/paddle/fluid/reader.py index af340c03a4..7d08403d26 100644 --- a/python/paddle/fluid/reader.py +++ b/python/paddle/fluid/reader.py @@ -123,7 +123,6 @@ class PyReader(object): self._use_double_buffer = use_double_buffer self._capacity = capacity self._feed_list = feed_list - self._scope = global_scope() if not self._iterable: self._init_non_iterable() @@ -153,7 +152,7 @@ class PyReader(object): reader_name = PyReader.unique_name_generator('create_py_reader') double_buffer_name = PyReader.unique_name_generator('double_buffer') - var = self._scope.var(queue_name) + var = global_scope().var(queue_name) self._queue = core.init_lod_tensor_blocking_queue(var, self._capacity) startup_blk = default_startup_program().current_block() @@ -215,6 +214,9 @@ class PyReader(object): def __iter__(self): return self + def __next__(self): + return self.next() + def next(self): ret = self._reader.read_next() if ret: diff --git a/python/paddle/reader/decorator.py b/python/paddle/reader/decorator.py index b2ef9f7580..7443a6bb19 100644 --- a/python/paddle/reader/decorator.py +++ b/python/paddle/reader/decorator.py @@ -13,7 +13,7 @@ # limitations under the License. __all__ = [ - 'map_readers', 'buffered', 'compose', 'chain', 'shuffle', + 'cache', 'map_readers', 'buffered', 'compose', 'chain', 'shuffle', 'ComposeNotAligned', 'firstn', 'xmap_readers', 'PipeReader', 'multiprocess_reader', 'Fake' ] @@ -33,6 +33,31 @@ import zlib import paddle.compat as cpt +def cache(reader): + """ + Cache the reader data into memory. + + Be careful that this method may take long time to process, + and consume lots of memory. :code:`reader()` would only + call once. + + Args: + reader (generator): a reader object which yields + data each time. + + Returns: + reader (generator): a decorated reader object + which yields data from cached memory. + """ + all_data = tuple(reader()) + + def __impl__(): + for item in all_data: + yield item + + return __impl__ + + def map_readers(func, *readers): """ Creates a data reader that outputs return value of function using From 3334c279d098c884b915a56dedc7dd69c95d7c7e Mon Sep 17 00:00:00 2001 From: sneaxiy Date: Wed, 27 Feb 2019 10:40:26 +0000 Subject: [PATCH 07/44] add sample_generator test=develop --- paddle/fluid/API.spec | 3 +- .../fluid/operators/reader/blocking_queue.h | 1 + .../fluid/operators/reader/buffered_reader.cc | 1 + .../fluid/operators/reader/buffered_reader.h | 1 + paddle/fluid/operators/reader/py_reader.cc | 1 + paddle/fluid/operators/reader/py_reader.h | 1 + paddle/fluid/pybind/reader_py.cc | 3 + python/paddle/fluid/data_feeder.py | 82 ++++++++--- python/paddle/fluid/reader.py | 60 +++++++- .../unittests/test_decoupled_py_reader.py | 1 - .../test_py_reader_sample_generator.py | 137 ++++++++++++++++++ 11 files changed, 264 insertions(+), 27 deletions(-) create mode 100644 python/paddle/fluid/tests/unittests/test_py_reader_sample_generator.py diff --git a/paddle/fluid/API.spec b/paddle/fluid/API.spec index db3739e65b..746b1eecfe 100644 --- a/paddle/fluid/API.spec +++ b/paddle/fluid/API.spec @@ -61,8 +61,9 @@ paddle.fluid.io.load_params ArgSpec(args=['executor', 'dirname', 'main_program', paddle.fluid.io.load_persistables ArgSpec(args=['executor', 'dirname', 'main_program', 'filename'], varargs=None, keywords=None, defaults=(None, None)) paddle.fluid.io.save_inference_model ArgSpec(args=['dirname', 'feeded_var_names', 'target_vars', 'executor', 'main_program', 'model_filename', 'params_filename', 'export_for_deployment'], varargs=None, keywords=None, defaults=(None, None, None, True)) paddle.fluid.io.load_inference_model ArgSpec(args=['dirname', 'executor', 'model_filename', 'params_filename', 'pserver_endpoints'], varargs=None, keywords=None, defaults=(None, None, None)) -paddle.fluid.io.PyReader.__init__ ArgSpec(args=['self', 'feed_list', 'capacity', 'use_double_buffer', 'iterable'], varargs=None, keywords=None, defaults=(True, True)) +paddle.fluid.io.PyReader.__init__ ArgSpec(args=['self', 'feed_list', 'capacity', 'use_double_buffer', 'iterable'], varargs=None, keywords=None, defaults=(True, False)) paddle.fluid.io.PyReader.decorate_paddle_reader ArgSpec(args=['self', 'reader', 'places'], varargs=None, keywords=None, defaults=(None,)) +paddle.fluid.io.PyReader.decorate_sample_generator ArgSpec(args=['self', 'sample_generator', 'batch_size', 'drop_last', 'places'], varargs=None, keywords=None, defaults=(True, None)) paddle.fluid.io.PyReader.decorate_tensor_provider ArgSpec(args=['self', 'reader', 'places'], varargs=None, keywords=None, defaults=(None,)) paddle.fluid.io.PyReader.reset ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None) paddle.fluid.io.PyReader.start ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None) diff --git a/paddle/fluid/operators/reader/blocking_queue.h b/paddle/fluid/operators/reader/blocking_queue.h index 7962c0332d..78d238aa61 100644 --- a/paddle/fluid/operators/reader/blocking_queue.h +++ b/paddle/fluid/operators/reader/blocking_queue.h @@ -16,6 +16,7 @@ #include // NOLINT #include +#include #include "paddle/fluid/platform/enforce.h" diff --git a/paddle/fluid/operators/reader/buffered_reader.cc b/paddle/fluid/operators/reader/buffered_reader.cc index b8c98ff5e7..c9962b4ac2 100644 --- a/paddle/fluid/operators/reader/buffered_reader.cc +++ b/paddle/fluid/operators/reader/buffered_reader.cc @@ -13,6 +13,7 @@ // limitations under the License. #include "paddle/fluid/operators/reader/buffered_reader.h" +#include #include #include "paddle/fluid/framework/data_type.h" diff --git a/paddle/fluid/operators/reader/buffered_reader.h b/paddle/fluid/operators/reader/buffered_reader.h index 6b21de0949..5f8b2d47c2 100644 --- a/paddle/fluid/operators/reader/buffered_reader.h +++ b/paddle/fluid/operators/reader/buffered_reader.h @@ -15,6 +15,7 @@ #pragma once #include +#include #include #include #include "ThreadPool.h" diff --git a/paddle/fluid/operators/reader/py_reader.cc b/paddle/fluid/operators/reader/py_reader.cc index f2c28c1df8..155ae859de 100644 --- a/paddle/fluid/operators/reader/py_reader.cc +++ b/paddle/fluid/operators/reader/py_reader.cc @@ -13,6 +13,7 @@ // limitations under the License. #include "paddle/fluid/operators/reader/py_reader.h" +#include namespace paddle { namespace operators { diff --git a/paddle/fluid/operators/reader/py_reader.h b/paddle/fluid/operators/reader/py_reader.h index 7d760eca64..4307907514 100644 --- a/paddle/fluid/operators/reader/py_reader.h +++ b/paddle/fluid/operators/reader/py_reader.h @@ -15,6 +15,7 @@ #pragma once #include +#include #include #include "paddle/fluid/framework/reader.h" #include "paddle/fluid/operators/reader/lod_tensor_blocking_queue.h" diff --git a/paddle/fluid/pybind/reader_py.cc b/paddle/fluid/pybind/reader_py.cc index 8af0490310..af7d30552e 100644 --- a/paddle/fluid/pybind/reader_py.cc +++ b/paddle/fluid/pybind/reader_py.cc @@ -13,7 +13,10 @@ // limitations under the License. #include "paddle/fluid/pybind/reader_py.h" +#include #include +#include +#include #include #include "paddle/fluid/framework/reader.h" #include "paddle/fluid/operators/reader/buffered_reader.h" diff --git a/python/paddle/fluid/data_feeder.py b/python/paddle/fluid/data_feeder.py index a24e1d1300..83d7cef19c 100644 --- a/python/paddle/fluid/data_feeder.py +++ b/python/paddle/fluid/data_feeder.py @@ -26,6 +26,24 @@ from .framework import Variable, default_main_program __all__ = ['DataFeeder'] +def convert_dtype(dtype): + if dtype == core.VarDesc.VarType.FP32: + return 'float32' + elif dtype == core.VarDesc.VarType.INT64: + return 'int64' + elif dtype == core.VarDesc.VarType.FP64: + return 'float64' + elif dtype == core.VarDesc.VarType.FP16: + return 'float16' + elif dtype == core.VarDesc.VarType.INT32: + return 'int32' + elif dtype == core.VarDesc.VarType.UINT8: + return 'uint8' + else: + raise ValueError("dtype must be any of [int32, float32, int64, " + "float64, uint8]") + + class DataToLoDTensorConverter(object): def __init__(self, place, lod_level, shape, dtype): self.place = place @@ -38,27 +56,12 @@ class DataToLoDTensorConverter(object): if negtive_count > 1: self.shape = None break - if dtype == core.VarDesc.VarType.FP32: - self.dtype = 'float32' - elif dtype == core.VarDesc.VarType.INT64: - self.dtype = 'int64' - elif dtype == core.VarDesc.VarType.FP64: - self.dtype = 'float64' - elif dtype == core.VarDesc.VarType.FP16: - self.dtype = 'float16' - elif dtype == core.VarDesc.VarType.INT32: - self.dtype = 'int32' - elif dtype == core.VarDesc.VarType.UINT8: - self.dtype = 'uint8' - else: - raise ValueError("dtype must be any of [int32, float32, int64, " - "float64, uint8]") + self.dtype = convert_dtype(dtype) + self._reset() + def _reset(self): self.data = [] - self.lod = [] - - for i in six.moves.range(lod_level): - self.lod.append([]) + self.lod = [[] for _ in six.moves.range(self.lod_level)] def feed(self, data): self._feed_impl_(data, self.lod, self.lod_level) @@ -88,15 +91,52 @@ class DataToLoDTensorConverter(object): raise ValueError( "Reshape error. What is defined in data layer is {}, but receive {}" .format(self.shape, arr.shape)) - #else: - # self._check_shape(arr.shape) t = core.LoDTensor() t.set(arr, self.place) if self.lod_level > 0: t.set_recursive_sequence_lengths(self.lod) + self._reset() return t +class BatchedTensorProvider(object): + def __init__(self, feed_list, place, batch_size, generator, drop_last): + self.place = place + self.batch_size = batch_size + self.generator = generator + self.converters = [] + self.drop_last = drop_last + + for var in feed_list: + assert var.lod_level == 0, "lod_level must be 0" + self.converters.append( + DataToLoDTensorConverter( + place=self.place, + lod_level=0, + shape=var.shape, + dtype=var.dtype)) + + def _done(self): + return [c.done() for c in self.converters] + + def __call__(self): + idx = 0 + for each_sample in self.generator(): + for each_slot, each_converter in six.moves.zip(each_sample, + self.converters): + each_converter.data.append(each_slot) + + idx += 1 + if idx == self.batch_size: + idx = 0 + yield self._done() + + if not self.drop_last and idx > 0: + yield self._done() + else: + [c._reset() for c in self.converters] + + class DataFeeder(object): """ DataFeeder converts the data that returned by a reader into a data diff --git a/python/paddle/fluid/reader.py b/python/paddle/fluid/reader.py index 7d08403d26..49ea1b83b5 100644 --- a/python/paddle/fluid/reader.py +++ b/python/paddle/fluid/reader.py @@ -17,7 +17,7 @@ import six import threading from .framework import Program, Variable, program_guard, default_main_program, default_startup_program from .executor import global_scope -from .data_feeder import DataFeeder +from .data_feeder import DataFeeder, BatchedTensorProvider from .layers.io import monkey_patch_reader_methods, _copy_reader_var_, double_buffer from .unique_name import UniqueNameGenerator @@ -46,7 +46,7 @@ class PyReader(object): feed_list, capacity, use_double_buffer=True, - iterable=True): + iterable=False): """ Create a reader object for data feeding in Python. Data would be prefetched using Python thread and be pushed @@ -269,6 +269,54 @@ class PyReader(object): self._thread.daemon = True self._thread.start() + def decorate_sample_generator(self, + sample_generator, + batch_size, + drop_last=True, + places=None): + ''' + Set the data source of the PyReader object. + + The provided :code:`sample_generator` should be a Python generator, + which yields numpy.ndarray typed data of each sample. + + :code:`places` must be set when the PyReader object is iterable. + + If all inputs have no lods, this method is faster than + :code:`decorate_paddle_reader(paddle.batch(sample_generator, ...))` . + + Args: + sample_generator (generator): Python generator that yields + numpy.ndarray-typed sample data. + batch_size (int): batch size. Must be larger than 0. + drop_last (bool): Whether to drop the last batch when sample number + is less than batch_size. + places (None|list(CUDAPlace)|list(CPUPlace)): place list. Must + be provided when PyReader is iterable. + ''' + assert batch_size > 0, "batch_size must be larger than 0" + has_lod = False + for f in self._feed_list: + if f.lod_level != 0: + has_lod = True + break + + if has_lod: + self.decorate_paddle_reader( + paddle.batch( + sample_generator, + batch_size=batch_size, + drop_last=drop_last), + places=places) + else: + reader = BatchedTensorProvider( + feed_list=self._feed_list, + place=core.CPUPlace(), + batch_size=batch_size, + generator=sample_generator, + drop_last=drop_last) + self.decorate_tensor_provider(reader, places=places) + def decorate_paddle_reader(self, reader, places=None): ''' Set the data source of the PyReader object. @@ -279,8 +327,10 @@ class PyReader(object): :code:`places` must be set when the PyReader object is iterable. Args: - reader (generator): Python generator that yields numpy-typed - batched data. + reader (generator): Python generator that yields + list(numpy.ndarray)-typed batched data. + places (None|list(CUDAPlace)|list(CPUPlace)): place list. Must + be provided when PyReader is iterable. ''' assert self._tensor_reader is None, \ "Cannot reset the data source of PyReader" @@ -307,6 +357,8 @@ class PyReader(object): Args: reader (generator): Python generator that yields LoDTensor-typed batched data. + places (None|list(CUDAPlace)|list(CPUPlace)): place list. Must + be provided when PyReader is iterable. ''' assert self._tensor_reader is None, \ "Cannot reset the data source of PyReader" diff --git a/python/paddle/fluid/tests/unittests/test_decoupled_py_reader.py b/python/paddle/fluid/tests/unittests/test_decoupled_py_reader.py index 96a11edd49..7112a57743 100644 --- a/python/paddle/fluid/tests/unittests/test_decoupled_py_reader.py +++ b/python/paddle/fluid/tests/unittests/test_decoupled_py_reader.py @@ -127,7 +127,6 @@ class TestBase(unittest.TestCase): step_list.append(step) end_t = time.time() ret = {"time": end_t - start_t, "step": step_list} - scope._remove_from_pool() return ret def prepare_places(self, with_data_parallel, with_cpu=True, with_gpu=True): diff --git a/python/paddle/fluid/tests/unittests/test_py_reader_sample_generator.py b/python/paddle/fluid/tests/unittests/test_py_reader_sample_generator.py new file mode 100644 index 0000000000..2f8f0b1b6e --- /dev/null +++ b/python/paddle/fluid/tests/unittests/test_py_reader_sample_generator.py @@ -0,0 +1,137 @@ +# 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 paddle +import paddle.fluid as fluid +import math +import unittest +import numpy as np +import os + +os.environ['CPU_NUM'] = '1' + + +def random_reader(sample_num): + def __impl__(): + for _ in range(sample_num): + yield np.random.random( + size=[784]).astype('float32'), np.random.random_integers( + low=0, high=9, size=[1]).astype('int64') + + return paddle.reader.cache(__impl__) + + +class TestCaseBase(unittest.TestCase): + def setUp(self): + self.batch_size = 32 + self.epoch_num = 2 + self.sample_num = 165 + + def generate_all_data(self, reader): + ret = [] + for d in reader(): + slots = [[], []] + for item in d: + slots[0].append(item[0]) + slots[1].append(item[1]) + slots = [np.array(slot) for slot in slots] + ret.append(slots) + return ret + + def run_main(self, reader, use_sample_generator, iterable, drop_last): + image = fluid.layers.data(name='image', dtype='float32', shape=[784]) + label = fluid.layers.data(name='label', dtype='int64', shape=[1]) + py_reader = fluid.io.PyReader( + feed_list=[image, label], + capacity=16, + iterable=iterable, + use_double_buffer=False) + + batch_reader = paddle.batch(reader, self.batch_size, drop_last) + all_datas = self.generate_all_data(batch_reader) + + if not use_sample_generator: + py_reader.decorate_paddle_reader( + batch_reader, places=fluid.cpu_places()) + else: + py_reader.decorate_sample_generator( + reader, self.batch_size, drop_last, places=fluid.cpu_places()) + + if drop_last: + batch_num = int(self.sample_num / self.batch_size) + else: + batch_num = math.ceil(float(self.sample_num) / self.batch_size) + + exe = fluid.Executor(fluid.CPUPlace()) + exe.run(fluid.default_startup_program()) + for _ in range(self.epoch_num): + if py_reader.iterable: + step = 0 + for data in py_reader(): + img, lbl = exe.run(feed=data, fetch_list=[image, label]) + self.assertArrayEqual(img, all_datas[step][0]) + self.assertArrayEqual(lbl, all_datas[step][1]) + step += 1 + self.assertEqual(step, len(all_datas)) + else: + step = 0 + try: + py_reader.start() + while True: + img, lbl = exe.run(fetch_list=[image, label]) + self.assertArrayEqual(img, all_datas[step][0]) + self.assertArrayEqual(lbl, all_datas[step][1]) + step += 1 + except fluid.core.EOFException: + py_reader.reset() + self.assertEqual(step, len(all_datas)) + break + + def assertArrayEqual(self, arr1, arr2): + self.assertEqual(arr1.shape, arr2.shape) + self.assertTrue((arr1 == arr2).all()) + + def test_main(self): + reader = random_reader(self.sample_num) + for use_sample_generator in [False, True]: + for iterable in [False, True]: + for drop_last in [False, True]: + with fluid.program_guard(fluid.Program(), fluid.Program()): + self.run_main(reader, use_sample_generator, iterable, + drop_last) + + +class TestCase1(TestCaseBase): + def setUp(self): + self.batch_size = 32 + self.epoch_num = 10 + self.sample_num = 160 + + +class TestCase2(TestCaseBase): + def setUp(self): + self.batch_size = 32 + self.epoch_num = 2 + self.sample_num = 200 + + +class TestCase3(TestCaseBase): + def setUp(self): + self.batch_size = 32 + self.epoch_num = 2 + self.sample_num = 159 + + +if __name__ == '__main__': + unittest.main() From aa2335c218a45804b35b760896d894e4f4f46842 Mon Sep 17 00:00:00 2001 From: sneaxiy Date: Mon, 11 Mar 2019 04:14:21 +0000 Subject: [PATCH 08/44] add py_reader doc --- python/paddle/fluid/layers/io.py | 41 +++++++++++++++++++------------- python/paddle/fluid/reader.py | 32 ++++++++++++++----------- 2 files changed, 43 insertions(+), 30 deletions(-) diff --git a/python/paddle/fluid/layers/io.py b/python/paddle/fluid/layers/io.py index a9b391fd53..da3ffc9a60 100644 --- a/python/paddle/fluid/layers/io.py +++ b/python/paddle/fluid/layers/io.py @@ -563,22 +563,26 @@ def _py_reader(capacity, def start_provide_thread(func): def __provider_thread__(): - for tensors in func(): - array = core.LoDTensorArray() - for item in tensors: - if not isinstance(item, core.LoDTensor): - tmp = core.LoDTensor() - tmp.set(item, core.CPUPlace()) - item = tmp - - array.append(item) - - if reader.exited: - break - feed_queue.push(array) - if reader.exited: - break - feed_queue.close() + try: + for tensors in func(): + array = core.LoDTensorArray() + for item in tensors: + if not isinstance(item, core.LoDTensor): + tmp = core.LoDTensor() + tmp.set(item, core.CPUPlace()) + item = tmp + + array.append(item) + + if reader.exited: + break + feed_queue.push(array) + if reader.exited: + break + feed_queue.close() + except Exception as ex: + feed_queue.close() + raise ex reader.thread = threading.Thread(target=__provider_thread__) reader.thread.daemon = True @@ -692,6 +696,11 @@ def py_reader(capacity, >>> exe.run(fetch_list=[loss.name]) >>> except fluid.core.EOFException: >>> reader.reset() + >>> + >>> ... + >>> + >>> fluid.io.save_inference_model(dirname='./model', feeded_var_names=[img, label], + >>> target_vars=[loss], executor=fluid.Executor(fluid.CUDAPlace(0))) 2. When training and testing are both performed, two different :code:`py_reader` should be created with different names, e.g.: diff --git a/python/paddle/fluid/reader.py b/python/paddle/fluid/reader.py index 49ea1b83b5..ef212014b5 100644 --- a/python/paddle/fluid/reader.py +++ b/python/paddle/fluid/reader.py @@ -250,20 +250,24 @@ class PyReader(object): def _start(self): def __thread_main__(): - for tensors in self._tensor_reader(): - array = core.LoDTensorArray() - for item in tensors: - if not isinstance(item, core.LoDTensor): - tmp = core.LoDTensor() - tmp.set(item, core.CPUPlace()) - item = tmp - - array.append(item) - - if not self._queue.push(array): - break - - self._queue.close() + try: + for tensors in self._tensor_reader(): + array = core.LoDTensorArray() + for item in tensors: + if not isinstance(item, core.LoDTensor): + tmp = core.LoDTensor() + tmp.set(item, core.CPUPlace()) + item = tmp + + array.append(item) + + if not self._queue.push(array): + break + + self._queue.close() + except Exception as ex: + self._queue.close() + raise ex self._thread = threading.Thread(target=__thread_main__) self._thread.daemon = True From fc12f38394426c98cf31e3c44f6cbecc6ed7e2f2 Mon Sep 17 00:00:00 2001 From: sneaxiy Date: Mon, 11 Mar 2019 05:15:37 +0000 Subject: [PATCH 09/44] add API.spec test=develop --- paddle/fluid/API.spec | 535 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 535 insertions(+) create mode 100644 paddle/fluid/API.spec diff --git a/paddle/fluid/API.spec b/paddle/fluid/API.spec new file mode 100644 index 0000000000..fcdc98b0c4 --- /dev/null +++ b/paddle/fluid/API.spec @@ -0,0 +1,535 @@ +paddle.fluid.Program.__init__ (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.Program.block (ArgSpec(args=['self', 'index'], varargs=None, keywords=None, defaults=None), ('document', 'af5346376065ff4cf6832a8ac0ae0945')) +paddle.fluid.Program.clone (ArgSpec(args=['self', 'for_test'], varargs=None, keywords=None, defaults=(False,)), ('document', 'ebb7765b2962bd2be041d19720e49d0f')) +paddle.fluid.Program.current_block (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '5e162d3bf8dd625703463d9e4be36adb')) +paddle.fluid.Program.global_block (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', 'cfb7e05a002b2e64650778cabde7301c')) +paddle.fluid.Program.list_vars (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '1c8647b14fe57c7824b1c9562394dd3c')) +paddle.fluid.Program.parse_from_string (ArgSpec(args=['binary_str'], varargs=None, keywords=None, defaults=None), ('document', 'b6a7ffb239a30bf2ce58cfaca8d8b8d5')) +paddle.fluid.Program.to_string (ArgSpec(args=['self', 'throw_on_error', 'with_details'], varargs=None, keywords=None, defaults=(False,)), ('document', 'faec17e5a04af28e3776160e34504d15')) +paddle.fluid.default_startup_program (ArgSpec(args=[], varargs=None, keywords=None, defaults=None), ('document', '99e5d53d92d82797093332719c9e3ccd')) +paddle.fluid.default_main_program (ArgSpec(args=[], varargs=None, keywords=None, defaults=None), ('document', '5430f54ab4895f9f47db6bebbaf71659')) +paddle.fluid.program_guard (ArgSpec(args=['main_program', 'startup_program'], varargs=None, keywords=None, defaults=(None,)), ('document', 'b54f403e57825a1592aece03afe3afb6')) +paddle.fluid.name_scope (ArgSpec(args=['prefix'], varargs=None, keywords=None, defaults=(None,)), ('document', '0ef753f5cec69fef9ae6ad8b867b33a2')) +paddle.fluid.cuda_places (ArgSpec(args=['device_ids'], varargs=None, keywords=None, defaults=(None,)), ('document', '7d9a51fc9cf3c5245b5227080a8064c3')) +paddle.fluid.cpu_places (ArgSpec(args=['device_count'], varargs=None, keywords=None, defaults=(None,)), ('document', '4c0cd83f0b401fc2ff84c70974e5d210')) +paddle.fluid.cuda_pinned_places (ArgSpec(args=['device_count'], varargs=None, keywords=None, defaults=(None,)), ('document', 'd0c3ebd813c39958c92b78e3eef7e912')) +paddle.fluid.Executor.__init__ (ArgSpec(args=['self', 'place'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.Executor.close (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', 'f5369953dd0c443961cf79f7a00e1a03')) +paddle.fluid.Executor.run (ArgSpec(args=['self', 'program', 'feed', 'fetch_list', 'feed_var_name', 'fetch_var_name', 'scope', 'return_numpy', 'use_program_cache'], varargs=None, keywords=None, defaults=(None, None, None, 'feed', 'fetch', None, True, False)), ('document', 'aba8093edebf2d5c869b735b92811e45')) +paddle.fluid.global_scope (ArgSpec(args=[], varargs=None, keywords=None, defaults=None), ('document', 'e148d3ab1ed8edf3e928212a375959c0')) +paddle.fluid.scope_guard (ArgSpec(args=['scope'], varargs=None, keywords=None, defaults=None), ('document', 'b94d1f6bcc29c4fb58fc0058561250c2')) +paddle.fluid.DistributeTranspiler.__init__ (ArgSpec(args=['self', 'config'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.DistributeTranspiler.get_pserver_program (ArgSpec(args=['self', 'endpoint'], varargs=None, keywords=None, defaults=None), ('document', '292ab72977afbe58e6a3bde175452680')) +paddle.fluid.DistributeTranspiler.get_pserver_programs (ArgSpec(args=['self', 'endpoint'], varargs=None, keywords=None, defaults=None), ('document', '78f4949aedf317666a89ca74b3748ba8')) +paddle.fluid.DistributeTranspiler.get_startup_program (ArgSpec(args=['self', 'endpoint', 'pserver_program', 'startup_program'], varargs=None, keywords=None, defaults=(None, None)), ('document', 'd796fc0c8d51503b556fcf6dc15c4f0c')) +paddle.fluid.DistributeTranspiler.get_trainer_program (ArgSpec(args=['self', 'wait_port'], varargs=None, keywords=None, defaults=(True,)), ('document', '736330e31a7a54abccc0c7fd9119d9ff')) +paddle.fluid.DistributeTranspiler.transpile (ArgSpec(args=['self', 'trainer_id', 'program', 'pservers', 'trainers', 'sync_mode', 'startup_program', 'current_endpoint'], varargs=None, keywords=None, defaults=(None, '127.0.0.1:6174', 1, True, None, '127.0.0.1:6174')), ('document', '06ce55338dfe96311ad1078235ab3bf4')) +paddle.fluid.memory_optimize (ArgSpec(args=['input_program', 'skip_opt_set', 'print_log', 'level', 'skip_grads'], varargs=None, keywords=None, defaults=(None, False, 0, False)), ('document', 'eda17d0f1639bc6ca215cecf87f588a4')) +paddle.fluid.release_memory (ArgSpec(args=['input_program', 'skip_opt_set'], varargs=None, keywords=None, defaults=(None,)), ('document', 'ac4114d3df16264f1946deb3a8434a6f')) +paddle.fluid.DistributeTranspilerConfig.__init__ +paddle.fluid.ParallelExecutor.__init__ (ArgSpec(args=['self', 'use_cuda', 'loss_name', 'main_program', 'share_vars_from', 'exec_strategy', 'build_strategy', 'num_trainers', 'trainer_id', 'scope'], varargs=None, keywords=None, defaults=(None, None, None, None, None, 1, 0, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.ParallelExecutor.run (ArgSpec(args=['self', 'fetch_list', 'feed', 'feed_dict', 'return_numpy'], varargs=None, keywords=None, defaults=(None, None, True)), ('document', '2cb4bd74481861345c70228a0f57620c')) +paddle.fluid.create_lod_tensor (ArgSpec(args=['data', 'recursive_seq_lens', 'place'], varargs=None, keywords=None, defaults=None), ('document', '8e7bb21e83ff4604f5b379672e285b94')) +paddle.fluid.create_random_int_lodtensor (ArgSpec(args=['recursive_seq_lens', 'base_shape', 'place', 'low', 'high'], varargs=None, keywords=None, defaults=None), ('document', '368f638b99f1dfe59e9b02aa6f077752')) +paddle.fluid.DataFeedDesc.__init__ (ArgSpec(args=['self', 'proto_file'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.DataFeedDesc.desc (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '4294493e31c4bc9fc4bd48753044235f')) +paddle.fluid.DataFeedDesc.set_batch_size (ArgSpec(args=['self', 'batch_size'], varargs=None, keywords=None, defaults=None), ('document', '8d9f44601e0a99dd431f14fd9250cd21')) +paddle.fluid.DataFeedDesc.set_dense_slots (ArgSpec(args=['self', 'dense_slots_name'], varargs=None, keywords=None, defaults=None), ('document', 'eb894b464bbcd1b4bc8038398954f766')) +paddle.fluid.DataFeedDesc.set_use_slots (ArgSpec(args=['self', 'use_slots_name'], varargs=None, keywords=None, defaults=None), ('document', '415c56600ce4e198c071cad01409a690')) +paddle.fluid.AsyncExecutor.__init__ (ArgSpec(args=['self', 'place', 'run_mode'], varargs=None, keywords=None, defaults=(None, '')), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.AsyncExecutor.config_distributed_nodes (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '4810dbe1870452f16b3c60b6c5fd1459')) +paddle.fluid.AsyncExecutor.download_data (ArgSpec(args=['self', 'afs_path', 'local_path', 'fs_default_name', 'ugi', 'file_cnt', 'hadoop_home', 'process_num'], varargs=None, keywords=None, defaults=('$HADOOP_HOME', 12)), ('document', '799a2066cc26819f1ed31f47c15ad083')) +paddle.fluid.AsyncExecutor.get_instance (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', 'f8688f76a2db1243c7097a60c507b182')) +paddle.fluid.AsyncExecutor.init_model (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '504f39be2007404a17e5cabea1256c7d')) +paddle.fluid.AsyncExecutor.init_server (ArgSpec(args=['self', 'dist_desc'], varargs=None, keywords=None, defaults=None), ('document', 'c403ab46c5d3ef25c0f7e94ae75dcb68')) +paddle.fluid.AsyncExecutor.init_worker (ArgSpec(args=['self', 'dist_desc', 'startup_program'], varargs=None, keywords=None, defaults=None), ('document', 'dcf08f4bf2f3282acf11391f5d39c536')) +paddle.fluid.AsyncExecutor.run (ArgSpec(args=['self', 'program', 'data_feed', 'filelist', 'thread_num', 'fetch', 'mode', 'debug'], varargs=None, keywords=None, defaults=('', False)), ('document', '848fc53484e8326f6325feea87fe955c')) +paddle.fluid.AsyncExecutor.save_model (ArgSpec(args=['self', 'save_path'], varargs=None, keywords=None, defaults=None), ('document', 'c8ac0dfcb3b187aba25d03af7fea56b2')) +paddle.fluid.AsyncExecutor.stop (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '5f23d043607bb5d55e466ec3f578e093')) +paddle.fluid.CompiledProgram.__init__ (ArgSpec(args=['self', 'program_or_graph'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.CompiledProgram.with_data_parallel (ArgSpec(args=['self', 'loss_name', 'build_strategy', 'exec_strategy', 'share_vars_from', 'places'], varargs=None, keywords=None, defaults=(None, None, None, None, None)), ('document', 'dbf542d1384741650a1238ddb05daa37')) +paddle.fluid.CompiledProgram.with_inference_optimize (ArgSpec(args=['self', 'config'], varargs=None, keywords=None, defaults=None), ('document', '9e5b009d850191a010e859189c127fd8')) +paddle.fluid.ExecutionStrategy.__init__ __init__(self: paddle.fluid.core.ParallelExecutor.ExecutionStrategy) -> None +paddle.fluid.BuildStrategy.GradientScaleStrategy.__init__ __init__(self: paddle.fluid.core.ParallelExecutor.BuildStrategy.GradientScaleStrategy, arg0: int) -> None +paddle.fluid.BuildStrategy.ReduceStrategy.__init__ __init__(self: paddle.fluid.core.ParallelExecutor.BuildStrategy.ReduceStrategy, arg0: int) -> None +paddle.fluid.BuildStrategy.__init__ __init__(self: paddle.fluid.core.ParallelExecutor.BuildStrategy) -> None +paddle.fluid.io.save_vars (ArgSpec(args=['executor', 'dirname', 'main_program', 'vars', 'predicate', 'filename'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'b55d6193a1d4198d45b013fc5779e1f2')) +paddle.fluid.io.save_params (ArgSpec(args=['executor', 'dirname', 'main_program', 'filename'], varargs=None, keywords=None, defaults=(None, None)), ('document', '3a7a99abac3e1bf898871fe609354218')) +paddle.fluid.io.save_persistables (ArgSpec(args=['executor', 'dirname', 'main_program', 'filename'], varargs=None, keywords=None, defaults=(None, None)), ('document', '9141bb5f32caf7975eb3fd88c8a1b2da')) +paddle.fluid.io.load_vars (ArgSpec(args=['executor', 'dirname', 'main_program', 'vars', 'predicate', 'filename'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', '0a5308f496632ab1ec3ba1f1377e6f95')) +paddle.fluid.io.load_params (ArgSpec(args=['executor', 'dirname', 'main_program', 'filename'], varargs=None, keywords=None, defaults=(None, None)), ('document', '41779819cef32f2246e83aebc5a002e2')) +paddle.fluid.io.load_persistables (ArgSpec(args=['executor', 'dirname', 'main_program', 'filename'], varargs=None, keywords=None, defaults=(None, None)), ('document', '28df5bfe26ca7a077f91156abb0fe6d2')) +paddle.fluid.io.save_inference_model (ArgSpec(args=['dirname', 'feeded_var_names', 'target_vars', 'executor', 'main_program', 'model_filename', 'params_filename', 'export_for_deployment'], varargs=None, keywords=None, defaults=(None, None, None, True)), ('document', '582d87b8df75a5a639a107db8ff86f9c')) +paddle.fluid.io.load_inference_model (ArgSpec(args=['dirname', 'executor', 'model_filename', 'params_filename', 'pserver_endpoints'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '7a5255386075dac3c75b7058254fcdcb')) +paddle.fluid.io.PyReader.__init__ (ArgSpec(args=['self', 'feed_list', 'capacity', 'use_double_buffer', 'iterable'], varargs=None, keywords=None, defaults=(True, False)), ('document', '18211b287474b401bc460d3f73dbc1c7')) +paddle.fluid.io.PyReader.decorate_paddle_reader (ArgSpec(args=['self', 'reader', 'places'], varargs=None, keywords=None, defaults=(None,)), ('document', 'faef298f73e91aedcfaf5d184f3109b7')) +paddle.fluid.io.PyReader.decorate_sample_generator (ArgSpec(args=['self', 'sample_generator', 'batch_size', 'drop_last', 'places'], varargs=None, keywords=None, defaults=(True, None)), ('document', 'd3fe49fc342e7778ed086e965f41bf12')) +paddle.fluid.io.PyReader.decorate_tensor_provider (ArgSpec(args=['self', 'reader', 'places'], varargs=None, keywords=None, defaults=(None,)), ('document', 'd10224fef1095247063b6976da793021')) +paddle.fluid.io.PyReader.reset (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', 'ff1cc1e2beb8824d453656c72c28ddfb')) +paddle.fluid.io.PyReader.start (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', 'b7ea0a548991924e4cfe61a577b8e56d')) +paddle.fluid.initializer.ConstantInitializer.__init__ (ArgSpec(args=['self', 'value', 'force_cpu'], varargs=None, keywords=None, defaults=(0.0, False)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.initializer.UniformInitializer.__init__ (ArgSpec(args=['self', 'low', 'high', 'seed'], varargs=None, keywords=None, defaults=(-1.0, 1.0, 0)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.initializer.NormalInitializer.__init__ (ArgSpec(args=['self', 'loc', 'scale', 'seed'], varargs=None, keywords=None, defaults=(0.0, 1.0, 0)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.initializer.TruncatedNormalInitializer.__init__ (ArgSpec(args=['self', 'loc', 'scale', 'seed'], varargs=None, keywords=None, defaults=(0.0, 1.0, 0)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.initializer.XavierInitializer.__init__ (ArgSpec(args=['self', 'uniform', 'fan_in', 'fan_out', 'seed'], varargs=None, keywords=None, defaults=(True, None, None, 0)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.initializer.BilinearInitializer.__init__ (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', 'd389912dc079cbef432335a00017cec0')) +paddle.fluid.initializer.MSRAInitializer.__init__ (ArgSpec(args=['self', 'uniform', 'fan_in', 'seed'], varargs=None, keywords=None, defaults=(True, None, 0)), ('document', '53c757bed9345f2ad3361902531e7cf5')) +paddle.fluid.initializer.force_init_on_cpu (ArgSpec(args=[], varargs=None, keywords=None, defaults=None), ('document', '6d0f3e22c90d9d500d36ff57daf056ee')) +paddle.fluid.initializer.init_on_cpu (ArgSpec(args=[], varargs=None, keywords=None, defaults=None), ('document', 'a6d7011ca3d8c0d454dac3a56eae0c29')) +paddle.fluid.initializer.NumpyArrayInitializer.__init__ (ArgSpec(args=['self', 'value'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.layers.fc (ArgSpec(args=['input', 'size', 'num_flatten_dims', 'param_attr', 'bias_attr', 'act', 'is_test', 'name'], varargs=None, keywords=None, defaults=(1, None, None, None, False, None)), ('document', '1929058262994f212620599c63aea6bd')) +paddle.fluid.layers.embedding (ArgSpec(args=['input', 'size', 'is_sparse', 'is_distributed', 'padding_idx', 'param_attr', 'dtype'], varargs=None, keywords=None, defaults=(False, False, None, None, 'float32')), ('document', '89c2c55a0b0656b106064048e068e77a')) +paddle.fluid.layers.dynamic_lstm (ArgSpec(args=['input', 'size', 'h_0', 'c_0', 'param_attr', 'bias_attr', 'use_peepholes', 'is_reverse', 'gate_activation', 'cell_activation', 'candidate_activation', 'dtype', 'name'], varargs=None, keywords=None, defaults=(None, None, None, None, True, False, 'sigmoid', 'tanh', 'tanh', 'float32', None)), ('document', 'dfbb624f85015df29e994ca6999e8ff6')) +paddle.fluid.layers.dynamic_lstmp (ArgSpec(args=['input', 'size', 'proj_size', 'param_attr', 'bias_attr', 'use_peepholes', 'is_reverse', 'gate_activation', 'cell_activation', 'candidate_activation', 'proj_activation', 'dtype', 'name', 'h_0', 'c_0', 'cell_clip', 'proj_clip'], varargs=None, keywords=None, defaults=(None, None, True, False, 'sigmoid', 'tanh', 'tanh', 'tanh', 'float32', None, None, None, None, None)), ('document', 'b4b608b986eb9617aa0525e1be21d32d')) +paddle.fluid.layers.dynamic_gru (ArgSpec(args=['input', 'size', 'param_attr', 'bias_attr', 'is_reverse', 'gate_activation', 'candidate_activation', 'h_0', 'origin_mode'], varargs=None, keywords=None, defaults=(None, None, False, 'sigmoid', 'tanh', None, False)), ('document', '4ec4845fd7d991bcac822f8b0dfc101f')) +paddle.fluid.layers.gru_unit (ArgSpec(args=['input', 'hidden', 'size', 'param_attr', 'bias_attr', 'activation', 'gate_activation', 'origin_mode'], varargs=None, keywords=None, defaults=(None, None, 'tanh', 'sigmoid', False)), ('document', 'e0e2439f7af069b57badca18a6ba60b8')) +paddle.fluid.layers.linear_chain_crf (ArgSpec(args=['input', 'label', 'param_attr'], varargs=None, keywords=None, defaults=(None,)), ('document', '7c49ef4bbf0adfd4b9a1d98e2e5f3fea')) +paddle.fluid.layers.crf_decoding (ArgSpec(args=['input', 'param_attr', 'label'], varargs=None, keywords=None, defaults=(None,)), ('document', '7642373ab65d3fc3b96d16d10fef1538')) +paddle.fluid.layers.cos_sim (ArgSpec(args=['X', 'Y'], varargs=None, keywords=None, defaults=None), ('document', 'd740824aa7316b807c4b4a3c6c8c0bbe')) +paddle.fluid.layers.cross_entropy (ArgSpec(args=['input', 'label', 'soft_label', 'ignore_index'], varargs=None, keywords=None, defaults=(False, -100)), ('document', '025b364dafb4b7975c801eb33e7831a1')) +paddle.fluid.layers.bpr_loss (ArgSpec(args=['input', 'label', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '30add751a0f99347a6257634c03ff254')) +paddle.fluid.layers.square_error_cost (ArgSpec(args=['input', 'label'], varargs=None, keywords=None, defaults=None), ('document', '44b6eef4a0f2bc15f7d9745782406736')) +paddle.fluid.layers.chunk_eval (ArgSpec(args=['input', 'label', 'chunk_scheme', 'num_chunk_types', 'excluded_chunk_types'], varargs=None, keywords=None, defaults=(None,)), ('document', 'ee152a7ba3036e7b9ede9184545179b4')) +paddle.fluid.layers.sequence_conv (ArgSpec(args=['input', 'num_filters', 'filter_size', 'filter_stride', 'padding', 'bias_attr', 'param_attr', 'act', 'name'], varargs=None, keywords=None, defaults=(3, 1, None, None, None, None, None)), ('document', 'b6543768e1afaa2ecb869709d6e9c7e2')) +paddle.fluid.layers.conv2d (ArgSpec(args=['input', 'num_filters', 'filter_size', 'stride', 'padding', 'dilation', 'groups', 'param_attr', 'bias_attr', 'use_cudnn', 'act', 'name'], varargs=None, keywords=None, defaults=(1, 0, 1, None, None, None, True, None, None)), ('document', '8ca6121acd6d23cd8806a93f493c2e17')) +paddle.fluid.layers.conv3d (ArgSpec(args=['input', 'num_filters', 'filter_size', 'stride', 'padding', 'dilation', 'groups', 'param_attr', 'bias_attr', 'use_cudnn', 'act', 'name'], varargs=None, keywords=None, defaults=(1, 0, 1, None, None, None, True, None, None)), ('document', '37042620f9bd3a2da6e5d3138b2f724b')) +paddle.fluid.layers.sequence_pool (ArgSpec(args=['input', 'pool_type', 'is_test'], varargs=None, keywords=None, defaults=(False,)), ('document', 'a194fb80614023f543df3949fbd0d0b8')) +paddle.fluid.layers.sequence_softmax (ArgSpec(args=['input', 'use_cudnn', 'name'], varargs=None, keywords=None, defaults=(False, None)), ('document', '19ef6f9cdd27feac8a1ae060f19c10b4')) +paddle.fluid.layers.softmax (ArgSpec(args=['input', 'use_cudnn', 'name'], varargs=None, keywords=None, defaults=(False, None)), ('document', 'f19dd380864e61134ce3814e4be0de4b')) +paddle.fluid.layers.pool2d (ArgSpec(args=['input', 'pool_size', 'pool_type', 'pool_stride', 'pool_padding', 'global_pooling', 'use_cudnn', 'ceil_mode', 'name', 'exclusive'], varargs=None, keywords=None, defaults=(-1, 'max', 1, 0, False, True, False, None, True)), ('document', 'bbd84e855e660cd1084bb71a2fd0cdaa')) +paddle.fluid.layers.pool3d (ArgSpec(args=['input', 'pool_size', 'pool_type', 'pool_stride', 'pool_padding', 'global_pooling', 'use_cudnn', 'ceil_mode', 'name', 'exclusive'], varargs=None, keywords=None, defaults=(-1, 'max', 1, 0, False, True, False, None, True)), ('document', '043de7333b79ee0ac55053c14ed81625')) +paddle.fluid.layers.adaptive_pool2d (ArgSpec(args=['input', 'pool_size', 'pool_type', 'require_index', 'name'], varargs=None, keywords=None, defaults=('max', False, None)), ('document', '859b887174d06f361658f69cb7c06d95')) +paddle.fluid.layers.adaptive_pool3d (ArgSpec(args=['input', 'pool_size', 'pool_type', 'require_index', 'name'], varargs=None, keywords=None, defaults=('max', False, None)), ('document', '120f4323a3d7ed9c0916f15a59f0e497')) +paddle.fluid.layers.batch_norm (ArgSpec(args=['input', 'act', 'is_test', 'momentum', 'epsilon', 'param_attr', 'bias_attr', 'data_layout', 'in_place', 'name', 'moving_mean_name', 'moving_variance_name', 'do_model_average_for_mean_and_var', 'fuse_with_relu', 'use_global_stats'], varargs=None, keywords=None, defaults=(None, False, 0.9, 1e-05, None, None, 'NCHW', False, None, None, None, False, False, False)), ('document', 'c527b71b8a4c60dca8df8a745c2b598d')) +paddle.fluid.layers.data_norm (ArgSpec(args=['input', 'act', 'epsilon', 'param_attr', 'data_layout', 'in_place', 'name', 'moving_mean_name', 'moving_variance_name', 'do_model_average_for_mean_and_var'], varargs=None, keywords=None, defaults=(None, 1e-05, None, 'NCHW', False, None, None, None, False)), ('document', 'e45e09e65a2658e07cad987222f0d9ab')) +paddle.fluid.layers.beam_search_decode (ArgSpec(args=['ids', 'scores', 'beam_size', 'end_id', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', 'b0b8d53821716cd50c42e09b593f3feb')) +paddle.fluid.layers.conv2d_transpose (ArgSpec(args=['input', 'num_filters', 'output_size', 'filter_size', 'padding', 'stride', 'dilation', 'groups', 'param_attr', 'bias_attr', 'use_cudnn', 'act', 'name'], varargs=None, keywords=None, defaults=(None, None, 0, 1, 1, None, None, None, True, None, None)), ('document', '03993955ab1e6d3044c44e6f17fc85e9')) +paddle.fluid.layers.conv3d_transpose (ArgSpec(args=['input', 'num_filters', 'output_size', 'filter_size', 'padding', 'stride', 'dilation', 'groups', 'param_attr', 'bias_attr', 'use_cudnn', 'act', 'name'], varargs=None, keywords=None, defaults=(None, None, 0, 1, 1, None, None, None, True, None, None)), ('document', 'ec113c6a3686ac94f8fccd1a7953d445')) +paddle.fluid.layers.sequence_expand (ArgSpec(args=['x', 'y', 'ref_level', 'name'], varargs=None, keywords=None, defaults=(-1, None)), ('document', '79c375214fa427faac504043d162dae9')) +paddle.fluid.layers.sequence_expand_as (ArgSpec(args=['x', 'y', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '9d2611f84ab364c5da545e6a82f1770a')) +paddle.fluid.layers.sequence_pad (ArgSpec(args=['x', 'pad_value', 'maxlen', 'name'], varargs=None, keywords=None, defaults=(None, None)), ('document', '6a1adf3067b20f6e4bcb354d71c19184')) +paddle.fluid.layers.sequence_unpad (ArgSpec(args=['x', 'length', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', 'd12803c903c99aa36ec03aaac5f0cc5b')) +paddle.fluid.layers.lstm_unit (ArgSpec(args=['x_t', 'hidden_t_prev', 'cell_t_prev', 'forget_bias', 'param_attr', 'bias_attr', 'name'], varargs=None, keywords=None, defaults=(0.0, None, None, None)), ('document', '027723966f3ef0d7bc598f22287a96cc')) +paddle.fluid.layers.reduce_sum (ArgSpec(args=['input', 'dim', 'keep_dim', 'name'], varargs=None, keywords=None, defaults=(None, False, None)), ('document', 'b69998ce3ff4980fb21da0df05565f1b')) +paddle.fluid.layers.reduce_mean (ArgSpec(args=['input', 'dim', 'keep_dim', 'name'], varargs=None, keywords=None, defaults=(None, False, None)), ('document', 'd4d80dd98a1a5839f41eeb3a0f85f370')) +paddle.fluid.layers.reduce_max (ArgSpec(args=['input', 'dim', 'keep_dim', 'name'], varargs=None, keywords=None, defaults=(None, False, None)), ('document', '66a622db727551761ce4eb73eaa7f6a4')) +paddle.fluid.layers.reduce_min (ArgSpec(args=['input', 'dim', 'keep_dim', 'name'], varargs=None, keywords=None, defaults=(None, False, None)), ('document', 'd50ac552b5d131468ed466d08bb2d38c')) +paddle.fluid.layers.reduce_prod (ArgSpec(args=['input', 'dim', 'keep_dim', 'name'], varargs=None, keywords=None, defaults=(None, False, None)), ('document', 'fcd8301a0ce15f219c7a4bcd0c1e8eca')) +paddle.fluid.layers.sequence_first_step (ArgSpec(args=['input'], varargs=None, keywords=None, defaults=None), ('document', '2b290d3d77882bfe9bb8d331cac8cdd3')) +paddle.fluid.layers.sequence_last_step (ArgSpec(args=['input'], varargs=None, keywords=None, defaults=None), ('document', 'c16a892f44f7fe71bfa5afc32d3f34ce')) +paddle.fluid.layers.sequence_slice (ArgSpec(args=['input', 'offset', 'length', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', 'fdcea0e8b5bc7d8d4b1b072c521014e6')) +paddle.fluid.layers.dropout (ArgSpec(args=['x', 'dropout_prob', 'is_test', 'seed', 'name', 'dropout_implementation'], varargs=None, keywords=None, defaults=(False, None, None, 'downgrade_in_infer')), ('document', 'dc7042734c6d8b8ce97321f017f01d6f')) +paddle.fluid.layers.split (ArgSpec(args=['input', 'num_or_sections', 'dim', 'name'], varargs=None, keywords=None, defaults=(-1, None)), ('document', '652625345c2acb900029c78cc75f8aa6')) +paddle.fluid.layers.ctc_greedy_decoder (ArgSpec(args=['input', 'blank', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', 'ebbf2adbd79683dc93db03454dfa18c2')) +paddle.fluid.layers.edit_distance (ArgSpec(args=['input', 'label', 'normalized', 'ignored_tokens'], varargs=None, keywords=None, defaults=(True, None)), ('document', '97f0262f97602644c83142789d784571')) +paddle.fluid.layers.l2_normalize (ArgSpec(args=['x', 'axis', 'epsilon', 'name'], varargs=None, keywords=None, defaults=(1e-12, None)), ('document', '6e428384ce6a77207fa2c70d9f011990')) +paddle.fluid.layers.matmul (ArgSpec(args=['x', 'y', 'transpose_x', 'transpose_y', 'alpha', 'name'], varargs=None, keywords=None, defaults=(False, False, 1.0, None)), ('document', 'b4cbe1ac451005df6dad12e9ffdccca9')) +paddle.fluid.layers.topk (ArgSpec(args=['input', 'k', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', 'd3570c02f71bcd78e60b3f31dc8f5b32')) +paddle.fluid.layers.warpctc (ArgSpec(args=['input', 'label', 'blank', 'norm_by_times', 'use_cudnn'], varargs=None, keywords=None, defaults=(0, False, False)), ('document', 'aaba49c038ba927f0a8e45c0c9a686ab')) +paddle.fluid.layers.sequence_reshape (ArgSpec(args=['input', 'new_dim'], varargs=None, keywords=None, defaults=None), ('document', 'a10ab9bf88d4a7e328882d411abb6fd1')) +paddle.fluid.layers.transpose (ArgSpec(args=['x', 'perm', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', 'a1feac48b843d679db82312dc85885f4')) +paddle.fluid.layers.im2sequence (ArgSpec(args=['input', 'filter_size', 'stride', 'padding', 'input_image_size', 'out_stride', 'name'], varargs=None, keywords=None, defaults=(1, 1, 0, None, 1, None)), ('document', '3ce01160ede80b1c26f776f8fef9340f')) +paddle.fluid.layers.nce (ArgSpec(args=['input', 'label', 'num_total_classes', 'sample_weight', 'param_attr', 'bias_attr', 'num_neg_samples', 'name', 'sampler', 'custom_dist', 'seed', 'is_sparse'], varargs=None, keywords=None, defaults=(None, None, None, None, None, 'uniform', None, 0, False)), ('document', 'fddad4896dee5193e1cdf70882c2a347')) +paddle.fluid.layers.sampled_softmax_with_cross_entropy (ArgSpec(args=['logits', 'label', 'num_samples', 'num_true', 'remove_accidental_hits', 'use_customized_samples', 'customized_samples', 'customized_probabilities', 'seed'], varargs=None, keywords=None, defaults=(1, True, False, None, None, 0)), ('document', '5db30b8a74e8c93687943a3e8d221da0')) +paddle.fluid.layers.hsigmoid (ArgSpec(args=['input', 'label', 'num_classes', 'param_attr', 'bias_attr', 'name', 'path_table', 'path_code', 'is_custom', 'is_sparse'], varargs=None, keywords=None, defaults=(None, None, None, None, None, False, False)), ('document', '80641ee6810b1cdc3fd6e14fc89ecc9d')) +paddle.fluid.layers.beam_search (ArgSpec(args=['pre_ids', 'pre_scores', 'ids', 'scores', 'beam_size', 'end_id', 'level', 'is_accumulated', 'name', 'return_parent_idx'], varargs=None, keywords=None, defaults=(0, True, None, False)), ('document', 'b350b9a30a18e7efd7e1bb740eef6996')) +paddle.fluid.layers.row_conv (ArgSpec(args=['input', 'future_context_size', 'param_attr', 'act'], varargs=None, keywords=None, defaults=(None, None)), ('document', '17485788fffe4e2d36dc58c2ac8d174e')) +paddle.fluid.layers.multiplex (ArgSpec(args=['inputs', 'index'], varargs=None, keywords=None, defaults=None), ('document', '013795af319e2e86d3506741941078ee')) +paddle.fluid.layers.layer_norm (ArgSpec(args=['input', 'scale', 'shift', 'begin_norm_axis', 'epsilon', 'param_attr', 'bias_attr', 'act', 'name'], varargs=None, keywords=None, defaults=(True, True, 1, 1e-05, None, None, None, None)), ('document', 'de6a906950bae9f3c245cb744d22b94e')) +paddle.fluid.layers.group_norm (ArgSpec(args=['input', 'groups', 'epsilon', 'param_attr', 'bias_attr', 'act', 'data_layout', 'name'], varargs=None, keywords=None, defaults=(1e-05, None, None, None, 'NCHW', None)), ('document', '419c3a24a83cc89219a029cf4092788b')) +paddle.fluid.layers.spectral_norm (ArgSpec(args=['weight', 'dim', 'power_iters', 'eps', 'name'], varargs=None, keywords=None, defaults=(0, 1, 1e-12, None)), ('document', '3f536aafba30d793287b52d231baff1b')) +paddle.fluid.layers.softmax_with_cross_entropy (ArgSpec(args=['logits', 'label', 'soft_label', 'ignore_index', 'numeric_stable_mode', 'return_softmax'], varargs=None, keywords=None, defaults=(False, -100, True, False)), ('document', 'bce1b75e3d95b75cacd1099655cbb3c3')) +paddle.fluid.layers.smooth_l1 (ArgSpec(args=['x', 'y', 'inside_weight', 'outside_weight', 'sigma'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', 'c6b175d253c55baf4b9c0eca9b1dda88')) +paddle.fluid.layers.one_hot (ArgSpec(args=['input', 'depth'], varargs=None, keywords=None, defaults=None), ('document', '6148b6a555cbfb62fdcd030d8982c18c')) +paddle.fluid.layers.autoincreased_step_counter (ArgSpec(args=['counter_name', 'begin', 'step'], varargs=None, keywords=None, defaults=(None, 1, 1)), ('document', '3f6c828594720c9b2da89c464be94478')) +paddle.fluid.layers.reshape (ArgSpec(args=['x', 'shape', 'actual_shape', 'act', 'inplace', 'name'], varargs=None, keywords=None, defaults=(None, None, False, None)), ('document', '323c019f257e55ddea4a824a362de62f')) +paddle.fluid.layers.squeeze (ArgSpec(args=['input', 'axes', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '3229d06517f794e86ca3da14c38b1465')) +paddle.fluid.layers.unsqueeze (ArgSpec(args=['input', 'axes', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', 'bbd62da391b1df984a1909d069a759b2')) +paddle.fluid.layers.lod_reset (ArgSpec(args=['x', 'y', 'target_lod'], varargs=None, keywords=None, defaults=(None, None)), ('document', 'f122194c562bd674f6ecdccf33785f99')) +paddle.fluid.layers.lrn (ArgSpec(args=['input', 'n', 'k', 'alpha', 'beta', 'name'], varargs=None, keywords=None, defaults=(5, 1.0, 0.0001, 0.75, None)), ('document', '0795e9940e42dcd62953514ff7e09f77')) +paddle.fluid.layers.pad (ArgSpec(args=['x', 'paddings', 'pad_value', 'name'], varargs=None, keywords=None, defaults=(0.0, None)), ('document', '2f28153bdd2d5ea6f7bad5867bd03eeb')) +paddle.fluid.layers.pad_constant_like (ArgSpec(args=['x', 'y', 'pad_value', 'name'], varargs=None, keywords=None, defaults=(0.0, None)), ('document', 'd2e1f45fef51b2c214e3f2aa8976c46c')) +paddle.fluid.layers.label_smooth (ArgSpec(args=['label', 'prior_dist', 'epsilon', 'dtype', 'name'], varargs=None, keywords=None, defaults=(None, 0.1, 'float32', None)), ('document', '70c113658102a11cc5d8e3d45145737a')) +paddle.fluid.layers.roi_pool (ArgSpec(args=['input', 'rois', 'pooled_height', 'pooled_width', 'spatial_scale'], varargs=None, keywords=None, defaults=(1, 1, 1.0)), ('document', 'c317aa595deb31649083c8faa91cdb97')) +paddle.fluid.layers.roi_align (ArgSpec(args=['input', 'rois', 'pooled_height', 'pooled_width', 'spatial_scale', 'sampling_ratio', 'name'], varargs=None, keywords=None, defaults=(1, 1, 1.0, -1, None)), ('document', '12c5bbb8b38c42e623fbc47611d766e1')) +paddle.fluid.layers.dice_loss (ArgSpec(args=['input', 'label', 'epsilon'], varargs=None, keywords=None, defaults=(1e-05,)), ('document', '1ba0508d573f65feecf3564dce22aa1d')) +paddle.fluid.layers.image_resize (ArgSpec(args=['input', 'out_shape', 'scale', 'name', 'resample', 'actual_shape', 'align_corners', 'align_mode'], varargs=None, keywords=None, defaults=(None, None, None, 'BILINEAR', None, True, 1)), ('document', '7a1966d7c3a48f1fc0881cdaf5d83b0b')) +paddle.fluid.layers.image_resize_short (ArgSpec(args=['input', 'out_short_len', 'resample'], varargs=None, keywords=None, defaults=('BILINEAR',)), ('document', '06211aefc50c5a3e940d7204d859cdf7')) +paddle.fluid.layers.resize_bilinear (ArgSpec(args=['input', 'out_shape', 'scale', 'name', 'actual_shape', 'align_corners', 'align_mode'], varargs=None, keywords=None, defaults=(None, None, None, None, True, 1)), ('document', 'e4fb4ed511b2293b8f04f7e872afbfd7')) +paddle.fluid.layers.resize_nearest (ArgSpec(args=['input', 'out_shape', 'scale', 'name', 'actual_shape', 'align_corners'], varargs=None, keywords=None, defaults=(None, None, None, None, True)), ('document', '735fa9758a6d7ff3b47d7b827f961c1d')) +paddle.fluid.layers.gather (ArgSpec(args=['input', 'index'], varargs=None, keywords=None, defaults=None), ('document', '98f1c86716b9b7f4dda83f20e2adeee2')) +paddle.fluid.layers.scatter (ArgSpec(args=['input', 'index', 'updates', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '65f8e9d8ddfd0b412f940579c4faa342')) +paddle.fluid.layers.sequence_scatter (ArgSpec(args=['input', 'index', 'updates', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '15b522457dfef103f0c20ca9d397678b')) +paddle.fluid.layers.random_crop (ArgSpec(args=['x', 'shape', 'seed'], varargs=None, keywords=None, defaults=(None,)), ('document', 'c9ab9e460ef0a1823249935a30e82c66')) +paddle.fluid.layers.mean_iou (ArgSpec(args=['input', 'label', 'num_classes'], varargs=None, keywords=None, defaults=None), ('document', '35cbbdfa585d027bb490707c95a176b9')) +paddle.fluid.layers.relu (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '866ffa1cc93f29e23662b526a7596537')) +paddle.fluid.layers.selu (ArgSpec(args=['x', 'scale', 'alpha', 'name'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '9044c7fe667b76cb2d9264f2db11f417')) +paddle.fluid.layers.log (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '98247c59d1c9b40af6730001b2aea73d')) +paddle.fluid.layers.crop (ArgSpec(args=['x', 'shape', 'offsets', 'name'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '883104791204d3127e24234bb630b2e7')) +paddle.fluid.layers.rank_loss (ArgSpec(args=['label', 'left', 'right', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', 'c542e39ac6add24a6bef6e79bf5617e2')) +paddle.fluid.layers.margin_rank_loss (ArgSpec(args=['label', 'left', 'right', 'margin', 'name'], varargs=None, keywords=None, defaults=(0.1, None)), ('document', '6d19dcc19917080b7ff3e03bde451bc8')) +paddle.fluid.layers.elu (ArgSpec(args=['x', 'alpha', 'name'], varargs=None, keywords=None, defaults=(1.0, None)), ('document', '463258ee9f8b60760eb1e26357cc9bfa')) +paddle.fluid.layers.relu6 (ArgSpec(args=['x', 'threshold', 'name'], varargs=None, keywords=None, defaults=(6.0, None)), ('document', '6f367339caf6c7124bc262fe1475df70')) +paddle.fluid.layers.pow (ArgSpec(args=['x', 'factor', 'name'], varargs=None, keywords=None, defaults=(1.0, None)), ('document', 'a5117c1eb84aca2ac0b0abab337a4799')) +paddle.fluid.layers.stanh (ArgSpec(args=['x', 'scale_a', 'scale_b', 'name'], varargs=None, keywords=None, defaults=(0.6666666666666666, 1.7159, None)), ('document', '959936a477efc6c1447a9c8bf8ce94bb')) +paddle.fluid.layers.hard_sigmoid (ArgSpec(args=['x', 'slope', 'offset', 'name'], varargs=None, keywords=None, defaults=(0.2, 0.5, None)), ('document', 'c82059b6fea1aa730f9aac911807b756')) +paddle.fluid.layers.swish (ArgSpec(args=['x', 'beta', 'name'], varargs=None, keywords=None, defaults=(1.0, None)), ('document', 'ef745e55a48763ee7b46b21a81dc7e84')) +paddle.fluid.layers.prelu (ArgSpec(args=['x', 'mode', 'param_attr', 'name'], varargs=None, keywords=None, defaults=(None, None)), ('document', 'f6acef7ff7d887e49ff499fbb1dad4a9')) +paddle.fluid.layers.brelu (ArgSpec(args=['x', 't_min', 't_max', 'name'], varargs=None, keywords=None, defaults=(0.0, 24.0, None)), ('document', '3db337c195e156e6ef2b8b4a57113600')) +paddle.fluid.layers.leaky_relu (ArgSpec(args=['x', 'alpha', 'name'], varargs=None, keywords=None, defaults=(0.02, None)), ('document', 'f878486c82b576938151daad0de995a0')) +paddle.fluid.layers.soft_relu (ArgSpec(args=['x', 'threshold', 'name'], varargs=None, keywords=None, defaults=(40.0, None)), ('document', '869adce548c342d6cc1bd88a948d83c9')) +paddle.fluid.layers.flatten (ArgSpec(args=['x', 'axis', 'name'], varargs=None, keywords=None, defaults=(1, None)), ('document', 'cb295c13cb957db85cd9609269d7784d')) +paddle.fluid.layers.sequence_mask (ArgSpec(args=['x', 'maxlen', 'dtype', 'name'], varargs=None, keywords=None, defaults=(None, 'int64', None)), ('document', 'f0dd6eddd3bff015a3c05269d82fcbd8')) +paddle.fluid.layers.stack (ArgSpec(args=['x', 'axis'], varargs=None, keywords=None, defaults=(0,)), ('document', '367cfbb642839beacb5d117e2d2b4041')) +paddle.fluid.layers.pad2d (ArgSpec(args=['input', 'paddings', 'mode', 'pad_value', 'data_format', 'name'], varargs=None, keywords=None, defaults=([0, 0, 0, 0], 'constant', 0.0, 'NCHW', None)), ('document', '7f4d46320cc077ca2e8db600c35f4030')) +paddle.fluid.layers.unstack (ArgSpec(args=['x', 'axis', 'num'], varargs=None, keywords=None, defaults=(0, None)), ('document', '98eb9d633116efcfc6f90c114bd44fd6')) +paddle.fluid.layers.sequence_enumerate (ArgSpec(args=['input', 'win_size', 'pad_value', 'name'], varargs=None, keywords=None, defaults=(0, None)), ('document', 'f6028537085dc296103bbbd85fa7763d')) +paddle.fluid.layers.expand (ArgSpec(args=['x', 'expand_times', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '117d3607d1ffa0571835bbaebc7857ff')) +paddle.fluid.layers.sequence_concat (ArgSpec(args=['input', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '3a1d155dd1bf6e72a0a3e3e1519591d1')) +paddle.fluid.layers.scale (ArgSpec(args=['x', 'scale', 'bias', 'bias_after_scale', 'act', 'name'], varargs=None, keywords=None, defaults=(1.0, 0.0, True, None, None)), ('document', '30190413b2fa442e7466d6cf2ce5ea07')) +paddle.fluid.layers.elementwise_add (ArgSpec(args=['x', 'y', 'axis', 'act', 'name'], varargs=None, keywords=None, defaults=(-1, None, None)), ('document', '6bfbe72cbadc95ac7ab88c05ed5bf9f0')) +paddle.fluid.layers.elementwise_div (ArgSpec(args=['x', 'y', 'axis', 'act', 'name'], varargs=None, keywords=None, defaults=(-1, None, None)), ('document', 'cc6e6cc1cb942a152dde3ef08d5f165c')) +paddle.fluid.layers.elementwise_sub (ArgSpec(args=['x', 'y', 'axis', 'act', 'name'], varargs=None, keywords=None, defaults=(-1, None, None)), ('document', 'a12abdab09c3e57af5a6e1e9f138684a')) +paddle.fluid.layers.elementwise_mul (ArgSpec(args=['x', 'y', 'axis', 'act', 'name'], varargs=None, keywords=None, defaults=(-1, None, None)), ('document', '422c77dbfcff355a57b5fdd4ec876daa')) +paddle.fluid.layers.elementwise_max (ArgSpec(args=['x', 'y', 'axis', 'act', 'name'], varargs=None, keywords=None, defaults=(-1, None, None)), ('document', 'f0bb0b2c454541cfafa761021a5cc776')) +paddle.fluid.layers.elementwise_min (ArgSpec(args=['x', 'y', 'axis', 'act', 'name'], varargs=None, keywords=None, defaults=(-1, None, None)), ('document', '8a9cdefefbccbf9f6b0991c0946a21e9')) +paddle.fluid.layers.elementwise_pow (ArgSpec(args=['x', 'y', 'axis', 'act', 'name'], varargs=None, keywords=None, defaults=(-1, None, None)), ('document', '1aea4e197c552a284f83888a3c67a32e')) +paddle.fluid.layers.uniform_random_batch_size_like (ArgSpec(args=['input', 'shape', 'dtype', 'input_dim_idx', 'output_dim_idx', 'min', 'max', 'seed'], varargs=None, keywords=None, defaults=('float32', 0, 0, -1.0, 1.0, 0)), ('document', '129e0a3257f1d532a948eedf9d5bf671')) +paddle.fluid.layers.gaussian_random (ArgSpec(args=['shape', 'mean', 'std', 'seed', 'dtype'], varargs=None, keywords=None, defaults=(0.0, 1.0, 0, 'float32')), ('document', '389dafe36e099841b6a7fb18d11f1b4c')) +paddle.fluid.layers.sampling_id (ArgSpec(args=['x', 'min', 'max', 'seed', 'dtype'], varargs=None, keywords=None, defaults=(0.0, 1.0, 0, 'float32')), ('document', '840fdac643d1341c1cae218d4511dbb9')) +paddle.fluid.layers.gaussian_random_batch_size_like (ArgSpec(args=['input', 'shape', 'input_dim_idx', 'output_dim_idx', 'mean', 'std', 'seed', 'dtype'], varargs=None, keywords=None, defaults=(0, 0, 0.0, 1.0, 0, 'float32')), ('document', '840026b4766613c5705e06563cd103b6')) +paddle.fluid.layers.sum (ArgSpec(args=['x'], varargs=None, keywords=None, defaults=None), ('document', 'a418e3ccb5e2ac21bd60f5cc221d5860')) +paddle.fluid.layers.slice (ArgSpec(args=['input', 'axes', 'starts', 'ends'], varargs=None, keywords=None, defaults=None), ('document', '01dbb91e7c74cb11336cd531013de51a')) +paddle.fluid.layers.shape (ArgSpec(args=['input'], varargs=None, keywords=None, defaults=None), ('document', '17db0f814eb7bb5a3fac1ca6e60e16d8')) +paddle.fluid.layers.logical_and (ArgSpec(args=['x', 'y', 'out', 'name'], varargs=None, keywords=None, defaults=(None, None)), ('document', 'cdcf20c494c92060d10feb9374532f42')) +paddle.fluid.layers.logical_or (ArgSpec(args=['x', 'y', 'out', 'name'], varargs=None, keywords=None, defaults=(None, None)), ('document', '0eae3f726a4afe590757552fa3ced012')) +paddle.fluid.layers.logical_xor (ArgSpec(args=['x', 'y', 'out', 'name'], varargs=None, keywords=None, defaults=(None, None)), ('document', 'b0daaa3fa4a0aa62f9b58c43d959eb25')) +paddle.fluid.layers.logical_not (ArgSpec(args=['x', 'out', 'name'], varargs=None, keywords=None, defaults=(None, None)), ('document', 'cd1c8cf31e040427d4e05711044caeb6')) +paddle.fluid.layers.clip (ArgSpec(args=['x', 'min', 'max', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', 'b020b7aab59719be98a4ae229a76deba')) +paddle.fluid.layers.clip_by_norm (ArgSpec(args=['x', 'max_norm', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', 'a1ea0bc5a926f427458c4254ca022749')) +paddle.fluid.layers.mean (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', 'd638d915195ce86a8d7963b81110d4c8')) +paddle.fluid.layers.mul (ArgSpec(args=['x', 'y', 'x_num_col_dims', 'y_num_col_dims', 'name'], varargs=None, keywords=None, defaults=(1, 1, None)), ('document', 'ccd37fa6b53f074adbfb732d738c4c2d')) +paddle.fluid.layers.sigmoid_cross_entropy_with_logits (ArgSpec(args=['x', 'label', 'ignore_index', 'name', 'normalize'], varargs=None, keywords=None, defaults=(-100, None, False)), ('document', '180c284317ea45ef89a460d8d79c0b72')) +paddle.fluid.layers.maxout (ArgSpec(args=['x', 'groups', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '891870d069a6aea746d34cc53b61690c')) +paddle.fluid.layers.space_to_depth (ArgSpec(args=['x', 'blocksize', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '5f207ae10589ebe38a63575ef6ff8e1e')) +paddle.fluid.layers.affine_grid (ArgSpec(args=['theta', 'out_shape', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '51def402b8910e163cbace9d0c0526ed')) +paddle.fluid.layers.sequence_reverse (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '77a6d80aa5551ca70324fc975c44507f')) +paddle.fluid.layers.affine_channel (ArgSpec(args=['x', 'scale', 'bias', 'data_layout', 'name'], varargs=None, keywords=None, defaults=(None, None, 'NCHW', None)), ('document', '2f46f1ff39a13ab00857e7b9f44b2fa7')) +paddle.fluid.layers.similarity_focus (ArgSpec(args=['input', 'axis', 'indexes', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '70e3b5182a18b40b47ecabd7c8490a35')) +paddle.fluid.layers.hash (ArgSpec(args=['input', 'hash_size', 'num_hash', 'name'], varargs=None, keywords=None, defaults=(1, None)), ('document', '9bb77f8dc002dd2ce75d4769eaaf5007')) +paddle.fluid.layers.grid_sampler (ArgSpec(args=['x', 'grid', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', 'd256cba1c41a5ed92ce3f31e24a2ca6d')) +paddle.fluid.layers.log_loss (ArgSpec(args=['input', 'label', 'epsilon', 'name'], varargs=None, keywords=None, defaults=(0.0001, None)), ('document', '4b5a2341023afe63157a066c14254f98')) +paddle.fluid.layers.add_position_encoding (ArgSpec(args=['input', 'alpha', 'beta', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '4b9c2e8af5817937d831820874b5aa77')) +paddle.fluid.layers.bilinear_tensor_product (ArgSpec(args=['x', 'y', 'size', 'act', 'name', 'param_attr', 'bias_attr'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'aa7540a0fa73ff69a02e11b4091aab75')) +paddle.fluid.layers.merge_selected_rows (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', 'dc63315b84f591ac79ecca0c3632027a')) +paddle.fluid.layers.get_tensor_from_selected_rows (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '7ffc849e71f31dfe29030ff94e662de6')) +paddle.fluid.layers.lstm (ArgSpec(args=['input', 'init_h', 'init_c', 'max_len', 'hidden_size', 'num_layers', 'dropout_prob', 'is_bidirec', 'is_test', 'name', 'default_initializer', 'seed'], varargs=None, keywords=None, defaults=(0.0, False, False, None, None, -1)), ('document', 'd5e6c494ac35100e2ed4d4bd9a1ed932')) +paddle.fluid.layers.shuffle_channel (ArgSpec(args=['x', 'group', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '2fa6782d43d02ae64482d21235a82949')) +paddle.fluid.layers.py_func (ArgSpec(args=['func', 'x', 'out', 'backward_func', 'skip_vars_in_backward_input'], varargs=None, keywords=None, defaults=(None, None)), ('document', '8404e472ac12b4a30a505d3d3a3e5fdb')) +paddle.fluid.layers.psroi_pool (ArgSpec(args=['input', 'rois', 'output_channels', 'spatial_scale', 'pooled_height', 'pooled_width', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '1546136806fef5c08f6918544bd9151d')) +paddle.fluid.layers.teacher_student_sigmoid_loss (ArgSpec(args=['input', 'label', 'soft_max_up_bound', 'soft_max_lower_bound'], varargs=None, keywords=None, defaults=(15.0, -15.0)), ('document', '2f6ff96864054a31aa4bb659c6722c99')) +paddle.fluid.layers.huber_loss (ArgSpec(args=['input', 'label', 'delta'], varargs=None, keywords=None, defaults=None), ('document', '431a4301c35032166ec029f7432c80a7')) +paddle.fluid.layers.tree_conv (ArgSpec(args=['nodes_vector', 'edge_set', 'output_size', 'num_filters', 'max_depth', 'act', 'param_attr', 'bias_attr', 'name'], varargs=None, keywords=None, defaults=(1, 2, 'tanh', None, None, None)), ('document', '34ea12ac9f10a65dccbc50100d12e607')) +paddle.fluid.layers.npair_loss (ArgSpec(args=['anchor', 'positive', 'labels', 'l2_reg'], varargs=None, keywords=None, defaults=(0.002,)), ('document', '46994d10276dd4cb803b4062b5d14329')) +paddle.fluid.layers.data (ArgSpec(args=['name', 'shape', 'append_batch_size', 'dtype', 'lod_level', 'type', 'stop_gradient'], varargs=None, keywords=None, defaults=(True, 'float32', 0, VarType.LOD_TENSOR, True)), ('document', '33bbd42027d872b3818b3d64ec52e139')) +paddle.fluid.layers.open_files (ArgSpec(args=['filenames', 'shapes', 'lod_levels', 'dtypes', 'thread_num', 'buffer_size', 'pass_num', 'is_test'], varargs=None, keywords=None, defaults=(None, None, 1, None)), ('document', 'b1ae2e1cc0750e58726374061ea90ecc')) +paddle.fluid.layers.read_file (ArgSpec(args=['reader'], varargs=None, keywords=None, defaults=None), ('document', 'b0a1c2fc51c27a106da28f3308c41f5e')) +paddle.fluid.layers.shuffle (ArgSpec(args=['reader', 'buffer_size'], varargs=None, keywords=None, defaults=None), ('document', 'f967a73426db26f970bc70bfb03cffca')) +paddle.fluid.layers.batch (ArgSpec(args=['reader', 'batch_size'], varargs=None, keywords=None, defaults=None), ('document', 'f563d376d35e1a4c4db100fd11b381a0')) +paddle.fluid.layers.double_buffer (ArgSpec(args=['reader', 'place', 'name'], varargs=None, keywords=None, defaults=(None, None)), ('document', '07e5b796674796eb1ef3fee9c10d24e3')) +paddle.fluid.layers.random_data_generator (ArgSpec(args=['low', 'high', 'shapes', 'lod_levels', 'for_parallel'], varargs=None, keywords=None, defaults=(True,)), ('document', '9b7f0f86ec24bbc97643cadcb6499cff')) +paddle.fluid.layers.py_reader (ArgSpec(args=['capacity', 'shapes', 'dtypes', 'lod_levels', 'name', 'use_double_buffer'], varargs=None, keywords=None, defaults=(None, None, True)), ('document', '4357643685cfd65454ba5a15f0151709')) +paddle.fluid.layers.create_py_reader_by_data (ArgSpec(args=['capacity', 'feed_list', 'name', 'use_double_buffer'], varargs=None, keywords=None, defaults=(None, True)), ('document', '350f74d93fab9adb2ac4950f1c26416b')) +paddle.fluid.layers.Preprocessor.__init__ (ArgSpec(args=['self', 'reader', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.layers.Preprocessor.block (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.layers.Preprocessor.inputs (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.layers.Preprocessor.outputs (ArgSpec(args=['self'], varargs='outs', keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.layers.load (ArgSpec(args=['out', 'file_path', 'load_as_fp16'], varargs=None, keywords=None, defaults=(None,)), ('document', '9d1a4bc97bbce9fa1d4f7a4200a771ff')) +paddle.fluid.layers.create_tensor (ArgSpec(args=['dtype', 'name', 'persistable'], varargs=None, keywords=None, defaults=(None, False)), ('document', 'c0c3d0194f83fff8ea99ce0820657dae')) +paddle.fluid.layers.create_parameter (ArgSpec(args=['shape', 'dtype', 'name', 'attr', 'is_bias', 'default_initializer'], varargs=None, keywords=None, defaults=(None, None, False, None)), ('document', 'd62b866c899bc1fedb5385f95b88e1f8')) +paddle.fluid.layers.create_global_var (ArgSpec(args=['shape', 'value', 'dtype', 'persistable', 'force_cpu', 'name'], varargs=None, keywords=None, defaults=(False, False, None)), ('document', 'ab914fac893607e29ac6e52bbdbea1a4')) +paddle.fluid.layers.cast (ArgSpec(args=['x', 'dtype'], varargs=None, keywords=None, defaults=None), ('document', '992eb42590fc1c380841a6db72ce78b3')) +paddle.fluid.layers.tensor_array_to_tensor (ArgSpec(args=['input', 'axis', 'name'], varargs=None, keywords=None, defaults=(1, None)), ('document', 'b12717d3d4567e6119589f7f655b0cbb')) +paddle.fluid.layers.concat (ArgSpec(args=['input', 'axis', 'name'], varargs=None, keywords=None, defaults=(0, None)), ('document', 'b19b79be4f05e85d1d6cec642c9fb535')) +paddle.fluid.layers.sums (ArgSpec(args=['input', 'out'], varargs=None, keywords=None, defaults=(None,)), ('document', '42912092418620b4be07f36af31e7816')) +paddle.fluid.layers.assign (ArgSpec(args=['input', 'output'], varargs=None, keywords=None, defaults=(None,)), ('document', 'b690184f3537df5501e4d9d8f31152a5')) +paddle.fluid.layers.fill_constant_batch_size_like (ArgSpec(args=['input', 'shape', 'dtype', 'value', 'input_dim_idx', 'output_dim_idx'], varargs=None, keywords=None, defaults=(0, 0)), ('document', 'd4059a2f5763036b07018d76429f9acb')) +paddle.fluid.layers.fill_constant (ArgSpec(args=['shape', 'dtype', 'value', 'force_cpu', 'out'], varargs=None, keywords=None, defaults=(False, None)), ('document', '1d8b14729639fa38509c79b9784740fa')) +paddle.fluid.layers.argmin (ArgSpec(args=['x', 'axis'], varargs=None, keywords=None, defaults=(0,)), ('document', '2778a1d34be49263a51211885599ea37')) +paddle.fluid.layers.argmax (ArgSpec(args=['x', 'axis'], varargs=None, keywords=None, defaults=(0,)), ('document', '04114996cfb98994ba222804a1a6109f')) +paddle.fluid.layers.argsort (ArgSpec(args=['input', 'axis', 'name'], varargs=None, keywords=None, defaults=(-1, None)), ('document', '68ec45c6fb6b93e47de9c9a0945fb98e')) +paddle.fluid.layers.ones (ArgSpec(args=['shape', 'dtype', 'force_cpu'], varargs=None, keywords=None, defaults=(False,)), ('document', 'b402489c62e668df42e7daceb63c142b')) +paddle.fluid.layers.zeros (ArgSpec(args=['shape', 'dtype', 'force_cpu'], varargs=None, keywords=None, defaults=(False,)), ('document', 'c155e2efc56ffa5ed4658cca0272e491')) +paddle.fluid.layers.reverse (ArgSpec(args=['x', 'axis'], varargs=None, keywords=None, defaults=None), ('document', '8ee7cb6ca639e7460e825f953b65d94d')) +paddle.fluid.layers.has_inf (ArgSpec(args=['x'], varargs=None, keywords=None, defaults=None), ('document', '8f8c0306117ea441f20dcbbdba1f0ecc')) +paddle.fluid.layers.has_nan (ArgSpec(args=['x'], varargs=None, keywords=None, defaults=None), ('document', '2e53e83127dbfd86e7098bdfe9a549e8')) +paddle.fluid.layers.isfinite (ArgSpec(args=['x'], varargs=None, keywords=None, defaults=None), ('document', '0a437011c3906079fd8947ed3e52d292')) +paddle.fluid.layers.While.__init__ (ArgSpec(args=['self', 'cond', 'is_test', 'name'], varargs=None, keywords=None, defaults=(False, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.layers.While.block (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.layers.Switch.__init__ (ArgSpec(args=['self', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.layers.Switch.case (ArgSpec(args=['self', 'condition'], varargs=None, keywords=None, defaults=None), ('document', 'f7c7160014c1b46cfeda9dd5808d1789')) +paddle.fluid.layers.Switch.default (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '50853ae884df03d9c36703bb46d9ef07')) +paddle.fluid.layers.increment (ArgSpec(args=['x', 'value', 'in_place'], varargs=None, keywords=None, defaults=(1.0, True)), ('document', '73bb96ec4783ec1a11e760e8851b0e77')) +paddle.fluid.layers.array_write (ArgSpec(args=['x', 'i', 'array'], varargs=None, keywords=None, defaults=(None,)), ('document', '40b6d15f4c86b2b09df340d7778ad713')) +paddle.fluid.layers.create_array (ArgSpec(args=['dtype'], varargs=None, keywords=None, defaults=None), ('document', '2d4f20087080ba5105b55205ad5c5b6a')) +paddle.fluid.layers.less_than (ArgSpec(args=['x', 'y', 'force_cpu', 'cond'], varargs=None, keywords=None, defaults=(None, None)), ('document', '067bbc799c66289ca8b8924c26b6673f')) +paddle.fluid.layers.equal (ArgSpec(args=['x', 'y', 'cond'], varargs=None, keywords=None, defaults=(None,)), ('document', '80c29b1dc64718f0116de90d1ac88a77')) +paddle.fluid.layers.array_read (ArgSpec(args=['array', 'i'], varargs=None, keywords=None, defaults=None), ('document', '0275133f1dde2aed528b4d3230edf823')) +paddle.fluid.layers.array_length (ArgSpec(args=['array'], varargs=None, keywords=None, defaults=None), ('document', 'ffb8b9578ec66db565b223d313aa82a2')) +paddle.fluid.layers.IfElse.__init__ (ArgSpec(args=['self', 'cond', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.layers.IfElse.false_block (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.layers.IfElse.input (ArgSpec(args=['self', 'x'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.layers.IfElse.output (ArgSpec(args=['self'], varargs='outs', keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.layers.IfElse.true_block (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.layers.DynamicRNN.__init__ (ArgSpec(args=['self', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.layers.DynamicRNN.block (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6d3e0a5d9aa519a9773a36e1620ea9b7')) +paddle.fluid.layers.DynamicRNN.memory (ArgSpec(args=['self', 'init', 'shape', 'value', 'need_reorder', 'dtype'], varargs=None, keywords=None, defaults=(None, None, 0.0, False, 'float32')), ('document', 'b9174d4e91505b0c8ecc193eb51e248d')) +paddle.fluid.layers.DynamicRNN.output (ArgSpec(args=['self'], varargs='outputs', keywords=None, defaults=None), ('document', 'b439a176a3328de8a75bdc5c08eece4a')) +paddle.fluid.layers.DynamicRNN.static_input (ArgSpec(args=['self', 'x'], varargs=None, keywords=None, defaults=None), ('document', 'f29ad2478b6b2ad4f413d2936a331ea0')) +paddle.fluid.layers.DynamicRNN.step_input (ArgSpec(args=['self', 'x'], varargs=None, keywords=None, defaults=None), ('document', '169d694d2224f62b4f3afdc3dbc19e95')) +paddle.fluid.layers.DynamicRNN.update_memory (ArgSpec(args=['self', 'ex_mem', 'new_mem'], varargs=None, keywords=None, defaults=None), ('document', '5d83987da13b98363d6a807a52d8024f')) +paddle.fluid.layers.StaticRNN.__init__ (ArgSpec(args=['self', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.layers.StaticRNN.memory (ArgSpec(args=['self', 'init', 'shape', 'batch_ref', 'init_value', 'init_batch_dim_idx', 'ref_batch_dim_idx'], varargs=None, keywords=None, defaults=(None, None, None, 0.0, 0, 1)), ('document', 'c24e368e23afac1ed91a78a639d7a9c7')) +paddle.fluid.layers.StaticRNN.output (ArgSpec(args=['self'], varargs='outputs', keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.layers.StaticRNN.step (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.layers.StaticRNN.step_input (ArgSpec(args=['self', 'x'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.layers.StaticRNN.step_output (ArgSpec(args=['self', 'o'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.layers.StaticRNN.update_memory (ArgSpec(args=['self', 'mem', 'var'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.layers.reorder_lod_tensor_by_rank (ArgSpec(args=['x', 'rank_table'], varargs=None, keywords=None, defaults=None), ('document', '3545f529ef04e8f6ecb76b47fa3df01a')) +paddle.fluid.layers.Print (ArgSpec(args=['input', 'first_n', 'message', 'summarize', 'print_tensor_name', 'print_tensor_type', 'print_tensor_shape', 'print_tensor_lod', 'print_phase'], varargs=None, keywords=None, defaults=(-1, None, -1, True, True, True, True, 'both')), ('document', '5fef91b0e21c93610785f2b1f7161732')) +paddle.fluid.layers.is_empty (ArgSpec(args=['x', 'cond'], varargs=None, keywords=None, defaults=(None,)), ('document', 'bbe578dbb49ad13e15b014e98c22b519')) +paddle.fluid.layers.sigmoid (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '29a25ba78de79152076cacfc5443137d')) +paddle.fluid.layers.logsigmoid (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '81ccb7acafd06c7728e11581f5d342e3')) +paddle.fluid.layers.exp (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', 'e6b3e769413d96aab4176f96db25984b')) +paddle.fluid.layers.tanh (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', 'e9d586a0b5bd05f67ee78048f9d503b6')) +paddle.fluid.layers.tanh_shrink (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '1e521554b9fdda9061ec6d306f0709b7')) +paddle.fluid.layers.softshrink (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '9eef31597bbafa2bd49691e072296e13')) +paddle.fluid.layers.sqrt (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '072a8541e0f632366bba10f67cb0db27')) +paddle.fluid.layers.abs (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '64650ac42cf82e9920cb0b172b1d29fd')) +paddle.fluid.layers.ceil (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', 'c75d67dc5fe28f68e4cfffead4f698ad')) +paddle.fluid.layers.floor (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '647b16c5da5ef909649ae02abb434973')) +paddle.fluid.layers.cos (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '485f2686bcc2fe37a4bd893769c8a3e2')) +paddle.fluid.layers.sin (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '01f1766aa76eff1df30147505b59f7c4')) +paddle.fluid.layers.round (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', 'b47f5da13913d3e56bdb1e612a73f3f2')) +paddle.fluid.layers.reciprocal (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', 'cc6ac2f14f03c52aaa83a59bf83b8d26')) +paddle.fluid.layers.square (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '48dfb45d773dbc30126c3a7f777de5ee')) +paddle.fluid.layers.softplus (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '459c5781e9d1dd88283b7c5769d7872a')) +paddle.fluid.layers.softsign (ArgSpec(args=['x', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '80846bcd4bd457207457a6d5411f4148')) +paddle.fluid.layers.uniform_random (ArgSpec(args=['shape', 'dtype', 'min', 'max', 'seed'], varargs=None, keywords=None, defaults=('float32', -1.0, 1.0, 0)), ('document', '308b619af849caa82bbc31e897f5e641')) +paddle.fluid.layers.hard_shrink (ArgSpec(args=['x', 'threshold'], varargs=None, keywords=None, defaults=(None,)), ('document', 'c142f5884f3255e0d6075c286bbd531e')) +paddle.fluid.layers.cumsum (ArgSpec(args=['x', 'axis', 'exclusive', 'reverse'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '944d7c03057f5fc88bc78acd4d82f926')) +paddle.fluid.layers.thresholded_relu (ArgSpec(args=['x', 'threshold'], varargs=None, keywords=None, defaults=(None,)), ('document', '90566ea449ea4c681435546e2f70610a')) +paddle.fluid.layers.prior_box (ArgSpec(args=['input', 'image', 'min_sizes', 'max_sizes', 'aspect_ratios', 'variance', 'flip', 'clip', 'steps', 'offset', 'name', 'min_max_aspect_ratios_order'], varargs=None, keywords=None, defaults=(None, [1.0], [0.1, 0.1, 0.2, 0.2], False, False, [0.0, 0.0], 0.5, None, False)), ('document', '14cac0ee643fa6e026ad82aeeee75bd8')) +paddle.fluid.layers.density_prior_box (ArgSpec(args=['input', 'image', 'densities', 'fixed_sizes', 'fixed_ratios', 'variance', 'clip', 'steps', 'offset', 'flatten_to_2d', 'name'], varargs=None, keywords=None, defaults=(None, None, None, [0.1, 0.1, 0.2, 0.2], False, [0.0, 0.0], 0.5, False, None)), ('document', 'a0d762bb08de9ce93bc780aa57cd5cd9')) +paddle.fluid.layers.multi_box_head (ArgSpec(args=['inputs', 'image', 'base_size', 'num_classes', 'aspect_ratios', 'min_ratio', 'max_ratio', 'min_sizes', 'max_sizes', 'steps', 'step_w', 'step_h', 'offset', 'variance', 'flip', 'clip', 'kernel_size', 'pad', 'stride', 'name', 'min_max_aspect_ratios_order'], varargs=None, keywords=None, defaults=(None, None, None, None, None, None, None, 0.5, [0.1, 0.1, 0.2, 0.2], True, False, 1, 0, 1, None, False)), ('document', 'a6ab47a2fe681e52fabb7057ddf0efdd')) +paddle.fluid.layers.bipartite_match (ArgSpec(args=['dist_matrix', 'match_type', 'dist_threshold', 'name'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '3ddb9b966f193900193a95a3df77c3c1')) +paddle.fluid.layers.target_assign (ArgSpec(args=['input', 'matched_indices', 'negative_indices', 'mismatch_value', 'name'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', 'c0b334f917828f95056f6ebe10907b1c')) +paddle.fluid.layers.detection_output (ArgSpec(args=['loc', 'scores', 'prior_box', 'prior_box_var', 'background_label', 'nms_threshold', 'nms_top_k', 'keep_top_k', 'score_threshold', 'nms_eta'], varargs=None, keywords=None, defaults=(0, 0.3, 400, 200, 0.01, 1.0)), ('document', 'c33093a82a46e3091e789e5572588db1')) +paddle.fluid.layers.ssd_loss (ArgSpec(args=['location', 'confidence', 'gt_box', 'gt_label', 'prior_box', 'prior_box_var', 'background_label', 'overlap_threshold', 'neg_pos_ratio', 'neg_overlap', 'loc_loss_weight', 'conf_loss_weight', 'match_type', 'mining_type', 'normalize', 'sample_size'], varargs=None, keywords=None, defaults=(None, 0, 0.5, 3.0, 0.5, 1.0, 1.0, 'per_prediction', 'max_negative', True, None)), ('document', '6d5028fd09d01ab82d296adc0ea95aee')) +paddle.fluid.layers.detection_map (ArgSpec(args=['detect_res', 'label', 'class_num', 'background_label', 'overlap_threshold', 'evaluate_difficult', 'has_state', 'input_states', 'out_states', 'ap_version'], varargs=None, keywords=None, defaults=(0, 0.3, True, None, None, None, 'integral')), ('document', '1467d91b50c22cd52103b4aa1ee9d0a1')) +paddle.fluid.layers.rpn_target_assign (ArgSpec(args=['bbox_pred', 'cls_logits', 'anchor_box', 'anchor_var', 'gt_boxes', 'is_crowd', 'im_info', 'rpn_batch_size_per_im', 'rpn_straddle_thresh', 'rpn_fg_fraction', 'rpn_positive_overlap', 'rpn_negative_overlap', 'use_random'], varargs=None, keywords=None, defaults=(256, 0.0, 0.5, 0.7, 0.3, True)), ('document', '1dddef3eb4b3cbd4df8e03ac480dbf97')) +paddle.fluid.layers.anchor_generator (ArgSpec(args=['input', 'anchor_sizes', 'aspect_ratios', 'variance', 'stride', 'offset', 'name'], varargs=None, keywords=None, defaults=(None, None, [0.1, 0.1, 0.2, 0.2], None, 0.5, None)), ('document', '23337cc57bbf5be73884b6bd0f849603')) +paddle.fluid.layers.roi_perspective_transform (ArgSpec(args=['input', 'rois', 'transformed_height', 'transformed_width', 'spatial_scale'], varargs=None, keywords=None, defaults=(1.0,)), ('document', '5761f9ed83654314416e24372b33bb84')) +paddle.fluid.layers.generate_proposal_labels (ArgSpec(args=['rpn_rois', 'gt_classes', 'is_crowd', 'gt_boxes', 'im_info', 'batch_size_per_im', 'fg_fraction', 'fg_thresh', 'bg_thresh_hi', 'bg_thresh_lo', 'bbox_reg_weights', 'class_nums', 'use_random'], varargs=None, keywords=None, defaults=(256, 0.25, 0.25, 0.5, 0.0, [0.1, 0.1, 0.2, 0.2], None, True)), ('document', '87863717edeb7fe87a1268976cbc015d')) +paddle.fluid.layers.generate_proposals (ArgSpec(args=['scores', 'bbox_deltas', 'im_info', 'anchors', 'variances', 'pre_nms_top_n', 'post_nms_top_n', 'nms_thresh', 'min_size', 'eta', 'name'], varargs=None, keywords=None, defaults=(6000, 1000, 0.5, 0.1, 1.0, None)), ('document', '57ab49f3f324f310b7eed322e7c1057a')) +paddle.fluid.layers.generate_mask_labels (ArgSpec(args=['im_info', 'gt_classes', 'is_crowd', 'gt_segms', 'rois', 'labels_int32', 'num_classes', 'resolution'], varargs=None, keywords=None, defaults=None), ('document', 'f73706a65468e9ca3e0bee4a31521b0a')) +paddle.fluid.layers.iou_similarity (ArgSpec(args=['x', 'y', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '587845f60c5d97ffdf2dfd21da52eca1')) +paddle.fluid.layers.box_coder (ArgSpec(args=['prior_box', 'prior_box_var', 'target_box', 'code_type', 'box_normalized', 'name', 'axis'], varargs=None, keywords=None, defaults=('encode_center_size', True, None, 0)), ('document', '032d0f4b7d8f6235ee5d91e473344f0e')) +paddle.fluid.layers.polygon_box_transform (ArgSpec(args=['input', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '0e5ac2507723a0b5adec473f9556799b')) +paddle.fluid.layers.yolov3_loss (ArgSpec(args=['x', 'gtbox', 'gtlabel', 'anchors', 'anchor_mask', 'class_num', 'ignore_thresh', 'downsample_ratio', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '991e934c3e09abf0edec7c9c978b4691')) +paddle.fluid.layers.box_clip (ArgSpec(args=['input', 'im_info', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '397e9e02b451d99c56e20f268fa03f2e')) +paddle.fluid.layers.multiclass_nms (ArgSpec(args=['bboxes', 'scores', 'score_threshold', 'nms_top_k', 'keep_top_k', 'nms_threshold', 'normalized', 'nms_eta', 'background_label', 'name'], varargs=None, keywords=None, defaults=(0.3, True, 1.0, 0, None)), ('document', 'ca7d1107b6c5d2d6d8221039a220fde0')) +paddle.fluid.layers.distribute_fpn_proposals (ArgSpec(args=['fpn_rois', 'min_level', 'max_level', 'refer_level', 'refer_scale', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '7bb011ec26bace2bc23235aa4a17647d')) +paddle.fluid.layers.box_decoder_and_assign (ArgSpec(args=['prior_box', 'prior_box_var', 'target_box', 'box_score', 'box_clip', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '005a5ae47d6c8fff721931d69d072b9f')) +paddle.fluid.layers.accuracy (ArgSpec(args=['input', 'label', 'k', 'correct', 'total'], varargs=None, keywords=None, defaults=(1, None, None)), ('document', '9808534c12c5e739a10f73ebb0b4eafd')) +paddle.fluid.layers.auc (ArgSpec(args=['input', 'label', 'curve', 'num_thresholds', 'topk', 'slide_steps'], varargs=None, keywords=None, defaults=('ROC', 4095, 1, 1)), ('document', 'e0e95334fce92d16c2d9db6e7caffc47')) +paddle.fluid.layers.exponential_decay (ArgSpec(args=['learning_rate', 'decay_steps', 'decay_rate', 'staircase'], varargs=None, keywords=None, defaults=(False,)), ('document', '98a5050bee8522fcea81aa795adaba51')) +paddle.fluid.layers.natural_exp_decay (ArgSpec(args=['learning_rate', 'decay_steps', 'decay_rate', 'staircase'], varargs=None, keywords=None, defaults=(False,)), ('document', '676a7bc2a218691db50bca233903d21e')) +paddle.fluid.layers.inverse_time_decay (ArgSpec(args=['learning_rate', 'decay_steps', 'decay_rate', 'staircase'], varargs=None, keywords=None, defaults=(False,)), ('document', 'd07e767d59c4a5e6c930f3e6756d3f82')) +paddle.fluid.layers.polynomial_decay (ArgSpec(args=['learning_rate', 'decay_steps', 'end_learning_rate', 'power', 'cycle'], varargs=None, keywords=None, defaults=(0.0001, 1.0, False)), ('document', '882634f420f626642f0874481263da40')) +paddle.fluid.layers.piecewise_decay (ArgSpec(args=['boundaries', 'values'], varargs=None, keywords=None, defaults=None), ('document', 'c717d9d1d78a53c809d01b8bc56f3cae')) +paddle.fluid.layers.noam_decay (ArgSpec(args=['d_model', 'warmup_steps'], varargs=None, keywords=None, defaults=None), ('document', 'd9a95746353fd574be36dc28d8726c28')) +paddle.fluid.layers.append_LARS (ArgSpec(args=['params_grads', 'learning_rate', 'weight_decay'], varargs=None, keywords=None, defaults=None), ('document', 'd24fa1e7d62ac8a534fc6a86002f84f8')) +paddle.fluid.layers.cosine_decay (ArgSpec(args=['learning_rate', 'step_each_epoch', 'epochs'], varargs=None, keywords=None, defaults=None), ('document', '9588c64c26ffaef3c466e404a6af9d9b')) +paddle.fluid.contrib.InitState.__init__ (ArgSpec(args=['self', 'init', 'shape', 'value', 'init_boot', 'need_reorder', 'dtype'], varargs=None, keywords=None, defaults=(None, None, 0.0, None, False, 'float32')), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.contrib.StateCell.__init__ (ArgSpec(args=['self', 'inputs', 'states', 'out_state', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.contrib.StateCell.compute_state (ArgSpec(args=['self', 'inputs'], varargs=None, keywords=None, defaults=None), ('document', '92973b3f222081a1d17069c683cf4a99')) +paddle.fluid.contrib.StateCell.get_input (ArgSpec(args=['self', 'input_name'], varargs=None, keywords=None, defaults=None), ('document', '6f24a007cfa184e32f01a960703bfd70')) +paddle.fluid.contrib.StateCell.get_state (ArgSpec(args=['self', 'state_name'], varargs=None, keywords=None, defaults=None), ('document', '630a4945cfe659ea4f307598fbbce5d2')) +paddle.fluid.contrib.StateCell.out_state (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '7ad681dff0393ddf13a724194e720f28')) +paddle.fluid.contrib.StateCell.set_state (ArgSpec(args=['self', 'state_name', 'state_value'], varargs=None, keywords=None, defaults=None), ('document', 'd4e0e08cd5d9d9a571cbc52d114f5ae9')) +paddle.fluid.contrib.StateCell.state_updater (ArgSpec(args=['self', 'updater'], varargs=None, keywords=None, defaults=None), ('document', 'd5afe1b7665d94fb023b15cf913ca510')) +paddle.fluid.contrib.StateCell.update_states (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', 'fe0b0f1338723516a35a30247899c81b')) +paddle.fluid.contrib.TrainingDecoder.__init__ (ArgSpec(args=['self', 'state_cell', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.contrib.TrainingDecoder.block (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '98d88fa1c989748410a12517c6a585bf')) +paddle.fluid.contrib.TrainingDecoder.output (ArgSpec(args=['self'], varargs='outputs', keywords=None, defaults=None), ('document', 'f0a457dee586559036202087ce2eff69')) +paddle.fluid.contrib.TrainingDecoder.static_input (ArgSpec(args=['self', 'x'], varargs=None, keywords=None, defaults=None), ('document', 'a024c72664fe815068423ba630b7658a')) +paddle.fluid.contrib.TrainingDecoder.step_input (ArgSpec(args=['self', 'x'], varargs=None, keywords=None, defaults=None), ('document', '4659db7a888a2495e71c1838a0483909')) +paddle.fluid.contrib.BeamSearchDecoder.__init__ (ArgSpec(args=['self', 'state_cell', 'init_ids', 'init_scores', 'target_dict_dim', 'word_dim', 'input_var_dict', 'topk_size', 'sparse_emb', 'max_len', 'beam_size', 'end_id', 'name'], varargs=None, keywords=None, defaults=({}, 50, True, 100, 1, 1, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.contrib.BeamSearchDecoder.block (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '98d88fa1c989748410a12517c6a585bf')) +paddle.fluid.contrib.BeamSearchDecoder.decode (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '1e47c60f080c1343ebb6ceaef89656b2')) +paddle.fluid.contrib.BeamSearchDecoder.early_stop (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '3a84a7454ed6707f79b9e954d92a7575')) +paddle.fluid.contrib.BeamSearchDecoder.read_array (ArgSpec(args=['self', 'init', 'is_ids', 'is_scores'], varargs=None, keywords=None, defaults=(False, False)), ('document', 'aa89eb8fd5e4cabaf5cc1bcae14665a4')) +paddle.fluid.contrib.BeamSearchDecoder.update_array (ArgSpec(args=['self', 'array', 'value'], varargs=None, keywords=None, defaults=None), ('document', '5754e9b3212b7c09497151516a0de5a7')) +paddle.fluid.contrib.memory_usage (ArgSpec(args=['program', 'batch_size'], varargs=None, keywords=None, defaults=None), ('document', '8fcb2f93bb743693baa8d4860a5ccc47')) +paddle.fluid.contrib.op_freq_statistic (ArgSpec(args=['program'], varargs=None, keywords=None, defaults=None), ('document', '4d43687113c4bf5b29d15aee2f4e4afa')) +paddle.fluid.contrib.QuantizeTranspiler.__init__ (ArgSpec(args=['self', 'weight_bits', 'activation_bits', 'activation_quantize_type', 'weight_quantize_type', 'window_size'], varargs=None, keywords=None, defaults=(8, 8, 'abs_max', 'abs_max', 10000)), ('document', '14b39f1fcd5667ff556b1aad94357d1d')) +paddle.fluid.contrib.QuantizeTranspiler.convert_to_int8 (ArgSpec(args=['self', 'program', 'place', 'scope'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.contrib.QuantizeTranspiler.freeze_program (ArgSpec(args=['self', 'program', 'place', 'fuse_bn', 'scope'], varargs=None, keywords=None, defaults=(False, None)), ('document', '909675a1ab055c69b436a7893fcae4fd')) +paddle.fluid.contrib.QuantizeTranspiler.training_transpile (ArgSpec(args=['self', 'program', 'startup_program'], varargs=None, keywords=None, defaults=(None, None)), ('document', '6dd9909f10b283ba2892a99058a72884')) +paddle.fluid.contrib.Calibrator.__init__ (ArgSpec(args=['self'], varargs='args', keywords='kwargs', defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.contrib.Calibrator.sample_data (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '3b8c85ca1e2cf753cc8c90a6c6992958')) +paddle.fluid.contrib.Calibrator.save_int8_model (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.contrib.reader.ctr_reader.ctr_reader (ArgSpec(args=['feed_dict', 'file_type', 'file_format', 'dense_slot_index', 'sparse_slot_index', 'capacity', 'thread_num', 'batch_size', 'file_list', 'slots', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', 'b2ebf3de2a6ef1af2c3b88d2db7591ab')) +paddle.fluid.contrib.build_compressor (ArgSpec(args=['place', 'data_reader', 'data_feeder', 'scope', 'metrics', 'epoch', 'config'], varargs=None, keywords=None, defaults=(None, None, None, None, None, None, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.contrib.CompressPass.__init__ (ArgSpec(args=['self', 'place', 'data_reader', 'data_feeder', 'scope', 'metrics', 'epoch', 'program_exe'], varargs=None, keywords=None, defaults=(None, None, None, None, None, None, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.contrib.CompressPass.add_strategy (ArgSpec(args=['self', 'strategy'], varargs=None, keywords=None, defaults=None), ('document', '3bf6010b6f47d3c86df0ec8957be95e0')) +paddle.fluid.contrib.CompressPass.apply (ArgSpec(args=['self', 'graph'], varargs=None, keywords=None, defaults=None), ('document', 'a92bf85d4b59bd4f2ac1706d7c4899a6')) +paddle.fluid.contrib.ImitationGraph.__init__ (ArgSpec(args=['self', 'program'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.contrib.ImitationGraph.all_parameters (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.contrib.SensitivePruneStrategy.__init__ (ArgSpec(args=['self', 'pruner', 'start_epoch', 'end_epoch', 'delta_rate', 'acc_loss_threshold', 'sensitivities'], varargs=None, keywords=None, defaults=(None, 0, 10, 0.2, 0.2, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.contrib.SensitivePruneStrategy.on_batch_begin (ArgSpec(args=['self', 'context'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.contrib.SensitivePruneStrategy.on_batch_end (ArgSpec(args=['self', 'context'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.contrib.SensitivePruneStrategy.on_compress_begin (ArgSpec(args=['self', 'context'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.contrib.SensitivePruneStrategy.on_compress_end (ArgSpec(args=['self', 'context'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.contrib.SensitivePruneStrategy.on_epoch_begin (ArgSpec(args=['self', 'context'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.contrib.SensitivePruneStrategy.on_epoch_end (ArgSpec(args=['self', 'context'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.contrib.MagnitudePruner.__init__ (ArgSpec(args=['self', 'threshold'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.contrib.MagnitudePruner.prune (ArgSpec(args=['self', 'param', 'threshold'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.contrib.RatioPruner.__init__ (ArgSpec(args=['self', 'ratios'], varargs=None, keywords=None, defaults=(None,)), ('document', 'e7a81a325b296a9ca502ee5adb4fc85d')) +paddle.fluid.contrib.RatioPruner.prune (ArgSpec(args=['self', 'param', 'ratio'], varargs=None, keywords=None, defaults=(None,)), ('document', '358cbf2978c91028fb96a195a9884645')) +paddle.fluid.contrib.load_persistables_for_increment (ArgSpec(args=['dirname', 'executor', 'program', 'lookup_table_var', 'lookup_table_var_path'], varargs=None, keywords=None, defaults=None), ('document', '11fbf7e8dd2289805de291b453a33ee7')) +paddle.fluid.contrib.load_persistables_for_inference (ArgSpec(args=['dirname', 'executor', 'program', 'lookup_table_var_name'], varargs=None, keywords=None, defaults=None), ('document', '5b5577bb3d24070da819674255d16196')) +paddle.fluid.contrib.convert_dist_to_sparse_program (ArgSpec(args=['program'], varargs=None, keywords=None, defaults=None), ('document', '4efbd93876832d4d35497cdbc7a1e6d8')) +paddle.fluid.contrib.HDFSClient.__init__ (ArgSpec(args=['self', 'hadoop_home', 'configs'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.contrib.HDFSClient.delete (ArgSpec(args=['self', 'hdfs_path'], varargs=None, keywords=None, defaults=None), ('document', 'c3721aa2d4d9ef5a857dd47b2681c03e')) +paddle.fluid.contrib.HDFSClient.download (ArgSpec(args=['self', 'hdfs_path', 'local_path', 'overwrite', 'unzip'], varargs=None, keywords=None, defaults=(False, False)), ('document', 'ca55bde92184d3fd0f9f5c963b25e634')) +paddle.fluid.contrib.HDFSClient.is_dir (ArgSpec(args=['self', 'hdfs_path'], varargs=None, keywords=None, defaults=(None,)), ('document', '45bde1bae02605a205c8245b58b9156d')) +paddle.fluid.contrib.HDFSClient.is_exist (ArgSpec(args=['self', 'hdfs_path'], varargs=None, keywords=None, defaults=(None,)), ('document', 'be9c94bccff7ba0c1d95883ac62b5864')) +paddle.fluid.contrib.HDFSClient.ls (ArgSpec(args=['self', 'hdfs_path'], varargs=None, keywords=None, defaults=None), ('document', '808acac504870c7e46594b95674f8a86')) +paddle.fluid.contrib.HDFSClient.lsr (ArgSpec(args=['self', 'hdfs_path', 'only_file', 'sort'], varargs=None, keywords=None, defaults=(True, True)), ('document', 'fae835aa3354eb6a0434c0f9ba3c2747')) +paddle.fluid.contrib.HDFSClient.make_local_dirs (ArgSpec(args=['local_path'], varargs=None, keywords=None, defaults=None), ('document', 'e76b89c8e7f019b5da576c0026fcf689')) +paddle.fluid.contrib.HDFSClient.makedirs (ArgSpec(args=['self', 'hdfs_path'], varargs=None, keywords=None, defaults=None), ('document', '44d9972aae390aedf40aaea731a37e4b')) +paddle.fluid.contrib.HDFSClient.rename (ArgSpec(args=['self', 'hdfs_src_path', 'hdfs_dst_path', 'overwrite'], varargs=None, keywords=None, defaults=(False,)), ('document', '0eb133644d9a9f4da45bb39261ff0955')) +paddle.fluid.contrib.HDFSClient.upload (ArgSpec(args=['self', 'hdfs_path', 'local_path', 'overwrite', 'retry_times'], varargs=None, keywords=None, defaults=(False, 5)), ('document', '7d053b4bfd6dcfdd2c9dda0e0dbd9665')) +paddle.fluid.contrib.multi_download (ArgSpec(args=['client', 'hdfs_path', 'local_path', 'trainer_id', 'trainers', 'multi_processes'], varargs=None, keywords=None, defaults=(5,)), ('document', '100927be598ed8f9eaa1f3ef1b23568a')) +paddle.fluid.contrib.multi_upload (ArgSpec(args=['client', 'hdfs_path', 'local_path', 'multi_processes', 'overwrite', 'sync'], varargs=None, keywords=None, defaults=(5, False, True)), ('document', '183f34c83d30dbe16e09e8716c41958a')) +paddle.fluid.transpiler.DistributeTranspiler.__init__ (ArgSpec(args=['self', 'config'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.transpiler.DistributeTranspiler.get_pserver_program (ArgSpec(args=['self', 'endpoint'], varargs=None, keywords=None, defaults=None), ('document', '292ab72977afbe58e6a3bde175452680')) +paddle.fluid.transpiler.DistributeTranspiler.get_pserver_programs (ArgSpec(args=['self', 'endpoint'], varargs=None, keywords=None, defaults=None), ('document', '78f4949aedf317666a89ca74b3748ba8')) +paddle.fluid.transpiler.DistributeTranspiler.get_startup_program (ArgSpec(args=['self', 'endpoint', 'pserver_program', 'startup_program'], varargs=None, keywords=None, defaults=(None, None)), ('document', 'd796fc0c8d51503b556fcf6dc15c4f0c')) +paddle.fluid.transpiler.DistributeTranspiler.get_trainer_program (ArgSpec(args=['self', 'wait_port'], varargs=None, keywords=None, defaults=(True,)), ('document', '736330e31a7a54abccc0c7fd9119d9ff')) +paddle.fluid.transpiler.DistributeTranspiler.transpile (ArgSpec(args=['self', 'trainer_id', 'program', 'pservers', 'trainers', 'sync_mode', 'startup_program', 'current_endpoint'], varargs=None, keywords=None, defaults=(None, '127.0.0.1:6174', 1, True, None, '127.0.0.1:6174')), ('document', '06ce55338dfe96311ad1078235ab3bf4')) +paddle.fluid.transpiler.memory_optimize (ArgSpec(args=['input_program', 'skip_opt_set', 'print_log', 'level', 'skip_grads'], varargs=None, keywords=None, defaults=(None, False, 0, False)), ('document', 'eda17d0f1639bc6ca215cecf87f588a4')) +paddle.fluid.transpiler.release_memory (ArgSpec(args=['input_program', 'skip_opt_set'], varargs=None, keywords=None, defaults=(None,)), ('document', 'ac4114d3df16264f1946deb3a8434a6f')) +paddle.fluid.transpiler.HashName.__init__ (ArgSpec(args=['self', 'pserver_endpoints'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.transpiler.HashName.dispatch (ArgSpec(args=['self', 'varlist'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.transpiler.HashName.reset (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.transpiler.RoundRobin.__init__ (ArgSpec(args=['self', 'pserver_endpoints'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.transpiler.RoundRobin.dispatch (ArgSpec(args=['self', 'varlist'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.transpiler.RoundRobin.reset (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.transpiler.DistributeTranspilerConfig.__init__ +paddle.fluid.nets.simple_img_conv_pool (ArgSpec(args=['input', 'num_filters', 'filter_size', 'pool_size', 'pool_stride', 'pool_padding', 'pool_type', 'global_pooling', 'conv_stride', 'conv_padding', 'conv_dilation', 'conv_groups', 'param_attr', 'bias_attr', 'act', 'use_cudnn'], varargs=None, keywords=None, defaults=(0, 'max', False, 1, 0, 1, 1, None, None, None, True)), ('document', 'e0f67f35abf27f666f81003113b90244')) +paddle.fluid.nets.sequence_conv_pool (ArgSpec(args=['input', 'num_filters', 'filter_size', 'param_attr', 'act', 'pool_type', 'bias_attr'], varargs=None, keywords=None, defaults=(None, 'sigmoid', 'max', None)), ('document', '48c434dd7bb827f69d90e5135d77470f')) +paddle.fluid.nets.glu (ArgSpec(args=['input', 'dim'], varargs=None, keywords=None, defaults=(-1,)), ('document', '08c1c57e1db6b20bf87b264cb7cf3ca8')) +paddle.fluid.nets.scaled_dot_product_attention (ArgSpec(args=['queries', 'keys', 'values', 'num_heads', 'dropout_rate'], varargs=None, keywords=None, defaults=(1, 0.0)), ('document', '921714c9bfb351b41403418265393203')) +paddle.fluid.nets.img_conv_group (ArgSpec(args=['input', 'conv_num_filter', 'pool_size', 'conv_padding', 'conv_filter_size', 'conv_act', 'param_attr', 'conv_with_batchnorm', 'conv_batchnorm_drop_rate', 'pool_stride', 'pool_type', 'use_cudnn'], varargs=None, keywords=None, defaults=(1, 3, None, None, False, 0.0, 1, 'max', True)), ('document', '3802be78fbfb206dae64a2d9f8480970')) +paddle.fluid.optimizer.SGDOptimizer.__init__ (ArgSpec(args=['self', 'learning_rate', 'regularization', 'name'], varargs=None, keywords=None, defaults=(None, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.optimizer.SGDOptimizer.apply_gradients (ArgSpec(args=['self', 'params_grads'], varargs=None, keywords=None, defaults=None), ('document', 'bfe7305918552aaecfdaa22411dbe871')) +paddle.fluid.optimizer.SGDOptimizer.backward (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'ba3a113d0229ff7bc9d39bda0a6d947f')) +paddle.fluid.optimizer.SGDOptimizer.minimize (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '35fd5d3330c97903528c7e0dacc7f6ea')) +paddle.fluid.optimizer.MomentumOptimizer.__init__ (ArgSpec(args=['self', 'learning_rate', 'momentum', 'use_nesterov', 'regularization', 'name'], varargs=None, keywords=None, defaults=(False, None, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.optimizer.MomentumOptimizer.apply_gradients (ArgSpec(args=['self', 'params_grads'], varargs=None, keywords=None, defaults=None), ('document', 'bfe7305918552aaecfdaa22411dbe871')) +paddle.fluid.optimizer.MomentumOptimizer.backward (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'ba3a113d0229ff7bc9d39bda0a6d947f')) +paddle.fluid.optimizer.MomentumOptimizer.minimize (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '35fd5d3330c97903528c7e0dacc7f6ea')) +paddle.fluid.optimizer.AdagradOptimizer.__init__ (ArgSpec(args=['self', 'learning_rate', 'epsilon', 'regularization', 'name', 'initial_accumulator_value'], varargs=None, keywords=None, defaults=(1e-06, None, None, 0.0)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.optimizer.AdagradOptimizer.apply_gradients (ArgSpec(args=['self', 'params_grads'], varargs=None, keywords=None, defaults=None), ('document', 'bfe7305918552aaecfdaa22411dbe871')) +paddle.fluid.optimizer.AdagradOptimizer.backward (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'ba3a113d0229ff7bc9d39bda0a6d947f')) +paddle.fluid.optimizer.AdagradOptimizer.minimize (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '35fd5d3330c97903528c7e0dacc7f6ea')) +paddle.fluid.optimizer.AdamOptimizer.__init__ (ArgSpec(args=['self', 'learning_rate', 'beta1', 'beta2', 'epsilon', 'regularization', 'name', 'lazy_mode'], varargs=None, keywords=None, defaults=(0.001, 0.9, 0.999, 1e-08, None, None, False)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.optimizer.AdamOptimizer.apply_gradients (ArgSpec(args=['self', 'params_grads'], varargs=None, keywords=None, defaults=None), ('document', 'bfe7305918552aaecfdaa22411dbe871')) +paddle.fluid.optimizer.AdamOptimizer.backward (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'ba3a113d0229ff7bc9d39bda0a6d947f')) +paddle.fluid.optimizer.AdamOptimizer.minimize (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '35fd5d3330c97903528c7e0dacc7f6ea')) +paddle.fluid.optimizer.AdamaxOptimizer.__init__ (ArgSpec(args=['self', 'learning_rate', 'beta1', 'beta2', 'epsilon', 'regularization', 'name'], varargs=None, keywords=None, defaults=(0.001, 0.9, 0.999, 1e-08, None, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.optimizer.AdamaxOptimizer.apply_gradients (ArgSpec(args=['self', 'params_grads'], varargs=None, keywords=None, defaults=None), ('document', 'bfe7305918552aaecfdaa22411dbe871')) +paddle.fluid.optimizer.AdamaxOptimizer.backward (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'ba3a113d0229ff7bc9d39bda0a6d947f')) +paddle.fluid.optimizer.AdamaxOptimizer.minimize (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '35fd5d3330c97903528c7e0dacc7f6ea')) +paddle.fluid.optimizer.DecayedAdagradOptimizer.__init__ (ArgSpec(args=['self', 'learning_rate', 'decay', 'epsilon', 'regularization', 'name'], varargs=None, keywords=None, defaults=(0.95, 1e-06, None, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.optimizer.DecayedAdagradOptimizer.apply_gradients (ArgSpec(args=['self', 'params_grads'], varargs=None, keywords=None, defaults=None), ('document', 'bfe7305918552aaecfdaa22411dbe871')) +paddle.fluid.optimizer.DecayedAdagradOptimizer.backward (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'ba3a113d0229ff7bc9d39bda0a6d947f')) +paddle.fluid.optimizer.DecayedAdagradOptimizer.minimize (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '35fd5d3330c97903528c7e0dacc7f6ea')) +paddle.fluid.optimizer.FtrlOptimizer.__init__ (ArgSpec(args=['self', 'learning_rate', 'l1', 'l2', 'lr_power', 'regularization', 'name'], varargs=None, keywords=None, defaults=(0.0, 0.0, -0.5, None, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.optimizer.FtrlOptimizer.apply_gradients (ArgSpec(args=['self', 'params_grads'], varargs=None, keywords=None, defaults=None), ('document', 'bfe7305918552aaecfdaa22411dbe871')) +paddle.fluid.optimizer.FtrlOptimizer.backward (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'ba3a113d0229ff7bc9d39bda0a6d947f')) +paddle.fluid.optimizer.FtrlOptimizer.minimize (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '35fd5d3330c97903528c7e0dacc7f6ea')) +paddle.fluid.optimizer.RMSPropOptimizer.__init__ (ArgSpec(args=['self', 'learning_rate', 'rho', 'epsilon', 'momentum', 'centered', 'regularization', 'name'], varargs=None, keywords=None, defaults=(0.95, 1e-06, 0.0, False, None, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.optimizer.RMSPropOptimizer.apply_gradients (ArgSpec(args=['self', 'params_grads'], varargs=None, keywords=None, defaults=None), ('document', 'bfe7305918552aaecfdaa22411dbe871')) +paddle.fluid.optimizer.RMSPropOptimizer.backward (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'ba3a113d0229ff7bc9d39bda0a6d947f')) +paddle.fluid.optimizer.RMSPropOptimizer.minimize (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '35fd5d3330c97903528c7e0dacc7f6ea')) +paddle.fluid.optimizer.AdadeltaOptimizer.__init__ (ArgSpec(args=['self', 'learning_rate', 'epsilon', 'rho', 'regularization', 'name'], varargs=None, keywords=None, defaults=(1e-06, 0.95, None, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.optimizer.AdadeltaOptimizer.apply_gradients (ArgSpec(args=['self', 'params_grads'], varargs=None, keywords=None, defaults=None), ('document', 'bfe7305918552aaecfdaa22411dbe871')) +paddle.fluid.optimizer.AdadeltaOptimizer.backward (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'ba3a113d0229ff7bc9d39bda0a6d947f')) +paddle.fluid.optimizer.AdadeltaOptimizer.minimize (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '35fd5d3330c97903528c7e0dacc7f6ea')) +paddle.fluid.optimizer.ModelAverage.__init__ (ArgSpec(args=['self', 'average_window_rate', 'min_average_window', 'max_average_window', 'regularization', 'name'], varargs=None, keywords=None, defaults=(10000, 10000, None, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.optimizer.ModelAverage.apply (ArgSpec(args=['self', 'executor', 'need_restore'], varargs=None, keywords=None, defaults=(True,)), ('document', '46234a5470590feb336346f70a3db715')) +paddle.fluid.optimizer.ModelAverage.apply_gradients (ArgSpec(args=['self', 'params_grads'], varargs=None, keywords=None, defaults=None), ('document', 'bfe7305918552aaecfdaa22411dbe871')) +paddle.fluid.optimizer.ModelAverage.backward (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'ba3a113d0229ff7bc9d39bda0a6d947f')) +paddle.fluid.optimizer.ModelAverage.minimize (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '35fd5d3330c97903528c7e0dacc7f6ea')) +paddle.fluid.optimizer.ModelAverage.restore (ArgSpec(args=['self', 'executor'], varargs=None, keywords=None, defaults=None), ('document', '18db9c70be9c4dd466f9844457b21bfe')) +paddle.fluid.optimizer.LarsMomentumOptimizer.__init__ (ArgSpec(args=['self', 'learning_rate', 'momentum', 'lars_coeff', 'lars_weight_decay', 'regularization', 'name'], varargs=None, keywords=None, defaults=(0.001, 0.0005, None, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.optimizer.LarsMomentumOptimizer.apply_gradients (ArgSpec(args=['self', 'params_grads'], varargs=None, keywords=None, defaults=None), ('document', 'bfe7305918552aaecfdaa22411dbe871')) +paddle.fluid.optimizer.LarsMomentumOptimizer.backward (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'ba3a113d0229ff7bc9d39bda0a6d947f')) +paddle.fluid.optimizer.LarsMomentumOptimizer.minimize (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '35fd5d3330c97903528c7e0dacc7f6ea')) +paddle.fluid.backward.append_backward (ArgSpec(args=['loss', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '1a79bd7d10ae54ca763ec81bca36ba24')) +paddle.fluid.regularizer.L1DecayRegularizer.__init__ (ArgSpec(args=['self', 'regularization_coeff'], varargs=None, keywords=None, defaults=(0.0,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.regularizer.L2DecayRegularizer.__init__ (ArgSpec(args=['self', 'regularization_coeff'], varargs=None, keywords=None, defaults=(0.0,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.LoDTensor.__init__ 1. __init__(self: paddle.fluid.core.LoDTensor, arg0: List[List[int]]) -> None 2. __init__(self: paddle.fluid.core.LoDTensor) -> None +paddle.fluid.LoDTensor.has_valid_recursive_sequence_lengths has_valid_recursive_sequence_lengths(self: paddle.fluid.core.LoDTensor) -> bool +paddle.fluid.LoDTensor.lod lod(self: paddle.fluid.core.LoDTensor) -> List[List[int]] +paddle.fluid.LoDTensor.recursive_sequence_lengths recursive_sequence_lengths(self: paddle.fluid.core.LoDTensor) -> List[List[int]] +paddle.fluid.LoDTensor.set 1. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[float32], arg1: paddle::platform::CPUPlace) -> None 2. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[int32], arg1: paddle::platform::CPUPlace) -> None 3. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[float64], arg1: paddle::platform::CPUPlace) -> None 4. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[int64], arg1: paddle::platform::CPUPlace) -> None 5. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[bool], arg1: paddle::platform::CPUPlace) -> None 6. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[uint16], arg1: paddle::platform::CPUPlace) -> None 7. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[uint8], arg1: paddle::platform::CPUPlace) -> None 8. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[int8], arg1: paddle::platform::CPUPlace) -> None 9. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[float32], arg1: paddle::platform::CUDAPlace) -> None 10. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[int32], arg1: paddle::platform::CUDAPlace) -> None 11. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[float64], arg1: paddle::platform::CUDAPlace) -> None 12. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[int64], arg1: paddle::platform::CUDAPlace) -> None 13. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[bool], arg1: paddle::platform::CUDAPlace) -> None 14. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[uint16], arg1: paddle::platform::CUDAPlace) -> None 15. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[uint8], arg1: paddle::platform::CUDAPlace) -> None 16. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[int8], arg1: paddle::platform::CUDAPlace) -> None 17. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[float32], arg1: paddle::platform::CUDAPinnedPlace) -> None 18. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[int32], arg1: paddle::platform::CUDAPinnedPlace) -> None 19. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[float64], arg1: paddle::platform::CUDAPinnedPlace) -> None 20. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[int64], arg1: paddle::platform::CUDAPinnedPlace) -> None 21. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[bool], arg1: paddle::platform::CUDAPinnedPlace) -> None 22. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[uint16], arg1: paddle::platform::CUDAPinnedPlace) -> None 23. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[uint8], arg1: paddle::platform::CUDAPinnedPlace) -> None 24. set(self: paddle.fluid.core.Tensor, arg0: numpy.ndarray[int8], arg1: paddle::platform::CUDAPinnedPlace) -> None +paddle.fluid.LoDTensor.set_lod set_lod(self: paddle.fluid.core.LoDTensor, lod: List[List[int]]) -> None +paddle.fluid.LoDTensor.set_recursive_sequence_lengths set_recursive_sequence_lengths(self: paddle.fluid.core.LoDTensor, recursive_sequence_lengths: List[List[int]]) -> None +paddle.fluid.LoDTensor.shape shape(self: paddle.fluid.core.Tensor) -> List[int] +paddle.fluid.LoDTensorArray.__init__ __init__(self: paddle.fluid.core.LoDTensorArray) -> None +paddle.fluid.LoDTensorArray.append append(self: paddle.fluid.core.LoDTensorArray, tensor: paddle.fluid.core.LoDTensor) -> None +paddle.fluid.CPUPlace.__init__ __init__(self: paddle.fluid.core.CPUPlace) -> None +paddle.fluid.CUDAPlace.__init__ __init__(self: paddle.fluid.core.CUDAPlace, arg0: int) -> None +paddle.fluid.CUDAPinnedPlace.__init__ __init__(self: paddle.fluid.core.CUDAPinnedPlace) -> None +paddle.fluid.ParamAttr.__init__ (ArgSpec(args=['self', 'name', 'initializer', 'learning_rate', 'regularizer', 'trainable', 'gradient_clip', 'do_model_average'], varargs=None, keywords=None, defaults=(None, None, 1.0, None, True, None, False)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.WeightNormParamAttr.__init__ (ArgSpec(args=['self', 'dim', 'name', 'initializer', 'learning_rate', 'regularizer', 'trainable', 'gradient_clip', 'do_model_average'], varargs=None, keywords=None, defaults=(None, None, None, 1.0, None, True, None, False)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.DataFeeder.__init__ (ArgSpec(args=['self', 'feed_list', 'place', 'program'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.DataFeeder.decorate_reader (ArgSpec(args=['self', 'reader', 'multi_devices', 'num_places', 'drop_last'], varargs=None, keywords=None, defaults=(None, True)), ('document', '0eed2f198dc73c08a41b61edbc755753')) +paddle.fluid.DataFeeder.feed (ArgSpec(args=['self', 'iterable'], varargs=None, keywords=None, defaults=None), ('document', '459e316301279dfd82001b46f0b8ffca')) +paddle.fluid.DataFeeder.feed_parallel (ArgSpec(args=['self', 'iterable', 'num_places'], varargs=None, keywords=None, defaults=(None,)), ('document', '543863d1f9d4853758adb613b8659e85')) +paddle.fluid.clip.ErrorClipByValue.__init__ (ArgSpec(args=['self', 'max', 'min'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.clip.GradientClipByValue.__init__ (ArgSpec(args=['self', 'max', 'min'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.clip.GradientClipByNorm.__init__ (ArgSpec(args=['self', 'clip_norm'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.clip.GradientClipByGlobalNorm.__init__ (ArgSpec(args=['self', 'clip_norm', 'group_name'], varargs=None, keywords=None, defaults=('default_group',)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.profiler.cuda_profiler (ArgSpec(args=['output_file', 'output_mode', 'config'], varargs=None, keywords=None, defaults=(None, None)), ('document', '2e2fb1cfc469a67f19fb578a2ed6be79')) +paddle.fluid.profiler.reset_profiler (ArgSpec(args=[], varargs=None, keywords=None, defaults=None), ('document', '397ce757fabbe5c622e0c3458c41fcd0')) +paddle.fluid.profiler.profiler (ArgSpec(args=['state', 'sorted_key', 'profile_path'], varargs=None, keywords=None, defaults=(None, '/tmp/profile')), ('document', 'bd3a07eeb68e384f4d2d416cb2e28d86')) +paddle.fluid.profiler.start_profiler (ArgSpec(args=['state'], varargs=None, keywords=None, defaults=None), ('document', '88da8fb6dbebaee2f7520188a09574f9')) +paddle.fluid.profiler.stop_profiler (ArgSpec(args=['sorted_key', 'profile_path'], varargs=None, keywords=None, defaults=(None, '/tmp/profile')), ('document', 'a7500e39dd033f1e64f562e909333a8a')) +paddle.fluid.unique_name.generate (ArgSpec(args=['key'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.unique_name.switch (ArgSpec(args=['new_generator'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.unique_name.guard (ArgSpec(args=['new_generator'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.recordio_writer.convert_reader_to_recordio_file (ArgSpec(args=['filename', 'reader_creator', 'feeder', 'compressor', 'max_num_records', 'feed_order'], varargs=None, keywords=None, defaults=(Compressor.Snappy, 1000, None)), ('document', '65c7523e86f0c50bb729b01667f36310')) +paddle.fluid.recordio_writer.convert_reader_to_recordio_files (ArgSpec(args=['filename', 'batch_per_file', 'reader_creator', 'feeder', 'compressor', 'max_num_records', 'feed_order'], varargs=None, keywords=None, defaults=(Compressor.Snappy, 1000, None)), ('document', 'bc643f0f5f1b9db57ff0d8a57d379bd7')) +paddle.fluid.Scope Scope() -> paddle.fluid.core._Scope +paddle.reader.cache (ArgSpec(args=['reader'], varargs=None, keywords=None, defaults=None), ('document', '83b94750674c6a04b5f96599d4bf3105')) +paddle.reader.map_readers (ArgSpec(args=['func'], varargs='readers', keywords=None, defaults=None), ('document', '77cbadb09df588e21e5cc0819b69c87d')) +paddle.reader.buffered (ArgSpec(args=['reader', 'size'], varargs=None, keywords=None, defaults=None), ('document', '0d6186f109feceb99f60ec50a0a624cb')) +paddle.reader.compose (ArgSpec(args=[], varargs='readers', keywords='kwargs', defaults=None), ('document', '884291104e1c3f37f33aae44b7deeb0d')) +paddle.reader.chain (ArgSpec(args=[], varargs='readers', keywords=None, defaults=None), ('document', 'd22c34e379a53901ae67a6bca7f4def4')) +paddle.reader.shuffle (ArgSpec(args=['reader', 'buf_size'], varargs=None, keywords=None, defaults=None), ('document', 'e42ea6fee23ce26b23cb142cd1d6522d')) +paddle.reader.ComposeNotAligned.__init__ +paddle.reader.firstn (ArgSpec(args=['reader', 'n'], varargs=None, keywords=None, defaults=None), ('document', 'c5bb8f7dd4f917f1569a368aab5b8aad')) +paddle.reader.xmap_readers (ArgSpec(args=['mapper', 'reader', 'process_num', 'buffer_size', 'order'], varargs=None, keywords=None, defaults=(False,)), ('document', '283bc0b8a0e26ae186b8b9bee4aec560')) +paddle.reader.PipeReader.__init__ (ArgSpec(args=['self', 'command', 'bufsize', 'file_type'], varargs=None, keywords=None, defaults=(8192, 'plain')), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.reader.PipeReader.get_line (ArgSpec(args=['self', 'cut_lines', 'line_break'], varargs=None, keywords=None, defaults=(True, '\n')), ('document', '5f80a7ed70052f01665e4c74acccfa69')) +paddle.reader.multiprocess_reader (ArgSpec(args=['readers', 'use_pipe', 'queue_size'], varargs=None, keywords=None, defaults=(True, 1000)), ('document', '7d8b3a96e592107c893d5d51ce968ba0')) +paddle.reader.Fake.__init__ (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.reader.creator.np_array (ArgSpec(args=['x'], varargs=None, keywords=None, defaults=None), ('document', '28d457fbc9a71efa4ac91a3be179cada')) +paddle.reader.creator.text_file (ArgSpec(args=['path'], varargs=None, keywords=None, defaults=None), ('document', '44fe286ab6175a5464d3a961a68c266a')) +paddle.reader.creator.recordio (ArgSpec(args=['paths', 'buf_size'], varargs=None, keywords=None, defaults=(100,)), ('document', '11b3704ea42cfd537953387a7e58dae8')) From 4b073c95dc398ca0d4895fbad4413b3768bb87db Mon Sep 17 00:00:00 2001 From: sneaxiy Date: Fri, 15 Mar 2019 03:48:21 +0000 Subject: [PATCH 10/44] fix compiler test=develop --- paddle/fluid/API.spec | 1 - python/paddle/fluid/compiler.py | 5 ++--- python/paddle/fluid/parallel_executor.py | 3 ++- 3 files changed, 4 insertions(+), 5 deletions(-) diff --git a/paddle/fluid/API.spec b/paddle/fluid/API.spec index 6479e30f6a..20817d52bf 100644 --- a/paddle/fluid/API.spec +++ b/paddle/fluid/API.spec @@ -526,7 +526,6 @@ paddle.reader.buffered (ArgSpec(args=['reader', 'size'], varargs=None, keywords= paddle.reader.compose (ArgSpec(args=[], varargs='readers', keywords='kwargs', defaults=None), ('document', '884291104e1c3f37f33aae44b7deeb0d')) paddle.reader.chain (ArgSpec(args=[], varargs='readers', keywords=None, defaults=None), ('document', 'd22c34e379a53901ae67a6bca7f4def4')) paddle.reader.shuffle (ArgSpec(args=['reader', 'buf_size'], varargs=None, keywords=None, defaults=None), ('document', 'e42ea6fee23ce26b23cb142cd1d6522d')) -paddle.reader.ComposeNotAligned.__init__ paddle.reader.firstn (ArgSpec(args=['reader', 'n'], varargs=None, keywords=None, defaults=None), ('document', 'c5bb8f7dd4f917f1569a368aab5b8aad')) paddle.reader.xmap_readers (ArgSpec(args=['mapper', 'reader', 'process_num', 'buffer_size', 'order'], varargs=None, keywords=None, defaults=(False,)), ('document', '283bc0b8a0e26ae186b8b9bee4aec560')) paddle.reader.PipeReader.__init__ (ArgSpec(args=['self', 'command', 'bufsize', 'file_type'], varargs=None, keywords=None, defaults=(8192, 'plain')), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) diff --git a/python/paddle/fluid/compiler.py b/python/paddle/fluid/compiler.py index 504bfb4103..8615518009 100644 --- a/python/paddle/fluid/compiler.py +++ b/python/paddle/fluid/compiler.py @@ -149,8 +149,7 @@ class CompiledProgram(object): self._places = [_place_obj(p) for p in places] else: self._places = None - self._build_strategy.is_distribution = framework.is_pserver_mode( - self._program) + self._build_strategy.is_distribution = _is_pserver_mode(self._program) return self def with_inference_optimize(self, config): @@ -191,7 +190,7 @@ class CompiledProgram(object): assert scope is not None, "" self._local_scopes = [] - self._exec_strategy.use_cuda = isinstance(self._place, core.CUDAPlace) + self._exec_strategy.use_cuda = use_cuda has_set_place = (self._places is not None) if has_set_place: desire_place = _place_obj(self._place) diff --git a/python/paddle/fluid/parallel_executor.py b/python/paddle/fluid/parallel_executor.py index 517418da1c..6702fc808b 100644 --- a/python/paddle/fluid/parallel_executor.py +++ b/python/paddle/fluid/parallel_executor.py @@ -99,7 +99,8 @@ class ParallelExecutor(object): build_strategy.num_trainers = num_trainers build_strategy.trainer_id = trainer_id - self._places = compiler.get_available_places(use_cuda) + self._places = framework.cuda_places( + ) if use_cuda else framework.cpu_places() self._scope = scope if scope is not None else executor.global_scope() main_program = main_program if main_program is not None \ From 3a09693f5c73035e762e5d296dbee8a2f2aaff7a Mon Sep 17 00:00:00 2001 From: sneaxiy Date: Mon, 18 Mar 2019 12:39:38 +0000 Subject: [PATCH 11/44] change API name test=develop --- paddle/fluid/API.spec | 8 ++++---- python/paddle/fluid/compiler.py | 4 +++- python/paddle/fluid/layers/io.py | 3 +++ python/paddle/fluid/reader.py | 16 ++++++++-------- .../unittests/test_decoupled_py_reader.py | 19 +++++++++++++++++-- .../test_py_reader_sample_generator.py | 2 +- 6 files changed, 36 insertions(+), 16 deletions(-) diff --git a/paddle/fluid/API.spec b/paddle/fluid/API.spec index c14eaa4b8b..f168d5608d 100644 --- a/paddle/fluid/API.spec +++ b/paddle/fluid/API.spec @@ -61,10 +61,10 @@ paddle.fluid.io.load_params (ArgSpec(args=['executor', 'dirname', 'main_program' paddle.fluid.io.load_persistables (ArgSpec(args=['executor', 'dirname', 'main_program', 'filename'], varargs=None, keywords=None, defaults=(None, None)), ('document', '28df5bfe26ca7a077f91156abb0fe6d2')) paddle.fluid.io.save_inference_model (ArgSpec(args=['dirname', 'feeded_var_names', 'target_vars', 'executor', 'main_program', 'model_filename', 'params_filename', 'export_for_deployment'], varargs=None, keywords=None, defaults=(None, None, None, True)), ('document', '582d87b8df75a5a639a107db8ff86f9c')) paddle.fluid.io.load_inference_model (ArgSpec(args=['dirname', 'executor', 'model_filename', 'params_filename', 'pserver_endpoints'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '7a5255386075dac3c75b7058254fcdcb')) -paddle.fluid.io.PyReader.__init__ (ArgSpec(args=['self', 'feed_list', 'capacity', 'use_double_buffer', 'iterable'], varargs=None, keywords=None, defaults=(True, False)), ('document', '18211b287474b401bc460d3f73dbc1c7')) -paddle.fluid.io.PyReader.decorate_paddle_reader (ArgSpec(args=['self', 'reader', 'places'], varargs=None, keywords=None, defaults=(None,)), ('document', 'faef298f73e91aedcfaf5d184f3109b7')) -paddle.fluid.io.PyReader.decorate_sample_generator (ArgSpec(args=['self', 'sample_generator', 'batch_size', 'drop_last', 'places'], varargs=None, keywords=None, defaults=(True, None)), ('document', 'd3fe49fc342e7778ed086e965f41bf12')) -paddle.fluid.io.PyReader.decorate_tensor_provider (ArgSpec(args=['self', 'reader', 'places'], varargs=None, keywords=None, defaults=(None,)), ('document', 'd10224fef1095247063b6976da793021')) +paddle.fluid.io.PyReader.__init__ (ArgSpec(args=['self', 'feed_list', 'capacity', 'use_double_buffer', 'iterable'], varargs=None, keywords=None, defaults=(True, False)), ('document', 'b3d72958b2568aae3f90f72abdcb7d1a')) +paddle.fluid.io.PyReader.decorate_batch_generator (ArgSpec(args=['self', 'reader', 'places'], varargs=None, keywords=None, defaults=(None,)), ('document', 'd10224fef1095247063b6976da793021')) +paddle.fluid.io.PyReader.decorate_sample_generator (ArgSpec(args=['self', 'sample_generator', 'batch_size', 'drop_last', 'places'], varargs=None, keywords=None, defaults=(True, None)), ('document', '7abd9cf7d695bab5bb6cf7ded5903cb2')) +paddle.fluid.io.PyReader.decorate_sample_list_generator (ArgSpec(args=['self', 'reader', 'places'], varargs=None, keywords=None, defaults=(None,)), ('document', 'faef298f73e91aedcfaf5d184f3109b7')) paddle.fluid.io.PyReader.reset (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', 'ff1cc1e2beb8824d453656c72c28ddfb')) paddle.fluid.io.PyReader.start (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', 'b7ea0a548991924e4cfe61a577b8e56d')) paddle.fluid.initializer.ConstantInitializer.__init__ (ArgSpec(args=['self', 'value', 'force_cpu'], varargs=None, keywords=None, defaults=(0.0, False)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) diff --git a/python/paddle/fluid/compiler.py b/python/paddle/fluid/compiler.py index a12fde47d8..894220d96c 100644 --- a/python/paddle/fluid/compiler.py +++ b/python/paddle/fluid/compiler.py @@ -127,7 +127,9 @@ class CompiledProgram(object): program in the given places. Otherwise, the places used when compiled is determined by the Executor, and the places used are controlled by environment variables: FLAGS_selected_gpus or CUDA_VISIBLE_DEVICES - if using GPU; or CPU_NUM if using CPU. + if using GPU; or CPU_NUM if using CPU. For example, if you want to + run on GPU 0 and 1, set places=[fluid.CUDAPlace(0), fluid.CUDAPlace(1)]. + If you want to run on 2 CPU cores, set places=[fluid.CPUPlace()]*2. Returns: self diff --git a/python/paddle/fluid/layers/io.py b/python/paddle/fluid/layers/io.py index da3ffc9a60..94fd9f3ea5 100644 --- a/python/paddle/fluid/layers/io.py +++ b/python/paddle/fluid/layers/io.py @@ -632,6 +632,9 @@ def _py_reader(capacity, reader.reset = __reset__ reader.decorate_tensor_provider = __set_tensor_provider__ reader.decorate_paddle_reader = __set_paddle_reader__ + + reader.decorate_batch_generator = __set_tensor_provider__ + reader.decorate_sample_list_generator = __set_paddle_reader__ reader.start = __start__ return reader diff --git a/python/paddle/fluid/reader.py b/python/paddle/fluid/reader.py index ef212014b5..5893798069 100644 --- a/python/paddle/fluid/reader.py +++ b/python/paddle/fluid/reader.py @@ -82,7 +82,7 @@ class PyReader(object): reader = fluid.io.PyReader(feed_list=[image, label], capacity=4, iterable=False) - reader.decorate_paddle_reader(user_defined_reader) + reader.decorate_sample_list_generator(user_defined_reader) ... # definition of network is omitted executor.run(fluid.default_main_program()) for _ in range(EPOCH_NUM): @@ -109,7 +109,7 @@ class PyReader(object): reader = fluid.io.PyReader(feed_list=[image, label], capacity=4, iterable=True) - reader.decorate_paddle_reader(user_defined_reader, + reader.decorate_sample_list_generator(user_defined_reader, places=fluid.cuda_places()) ... # definition of network is omitted executor.run(fluid.default_main_program()) @@ -287,7 +287,7 @@ class PyReader(object): :code:`places` must be set when the PyReader object is iterable. If all inputs have no lods, this method is faster than - :code:`decorate_paddle_reader(paddle.batch(sample_generator, ...))` . + :code:`decorate_sample_list_generator(paddle.batch(sample_generator, ...))` . Args: sample_generator (generator): Python generator that yields @@ -306,7 +306,7 @@ class PyReader(object): break if has_lod: - self.decorate_paddle_reader( + self.decorate_sample_list_generator( paddle.batch( sample_generator, batch_size=batch_size, @@ -319,9 +319,9 @@ class PyReader(object): batch_size=batch_size, generator=sample_generator, drop_last=drop_last) - self.decorate_tensor_provider(reader, places=places) + self.decorate_batch_generator(reader, places=places) - def decorate_paddle_reader(self, reader, places=None): + def decorate_sample_list_generator(self, reader, places=None): ''' Set the data source of the PyReader object. @@ -347,9 +347,9 @@ class PyReader(object): for slots in paddle_reader(): yield [slots[var.name] for var in self._feed_list] - self.decorate_tensor_provider(__tensor_reader_impl__, places) + self.decorate_batch_generator(__tensor_reader_impl__, places) - def decorate_tensor_provider(self, reader, places=None): + def decorate_batch_generator(self, reader, places=None): ''' Set the data source of the PyReader object. diff --git a/python/paddle/fluid/tests/unittests/test_decoupled_py_reader.py b/python/paddle/fluid/tests/unittests/test_decoupled_py_reader.py index 7112a57743..377014510b 100644 --- a/python/paddle/fluid/tests/unittests/test_decoupled_py_reader.py +++ b/python/paddle/fluid/tests/unittests/test_decoupled_py_reader.py @@ -25,6 +25,7 @@ CLASS_NUM = 10 def random_reader(): + np.random.seed(1) for i in range(BATCH_SIZE * 40): image = np.random.random([784]) label = np.random.random_integers(low=0, high=CLASS_NUM - 1) @@ -79,7 +80,8 @@ class TestBase(unittest.TestCase): reader = paddle.batch(random_reader, batch_size=BATCH_SIZE) ps = places if use_double_buffer else fluid.cpu_places(len(places)) - py_reader.decorate_paddle_reader( + + py_reader.decorate_sample_list_generator( reader, places=ps if py_reader.iterable else None) exe = fluid.Executor(place=places[0]) @@ -92,6 +94,7 @@ class TestBase(unittest.TestCase): step = 0 step_list = [] + loss_list = [] start_t = time.time() if not py_reader.iterable: for _ in six.moves.range(EPOCH_NUM): @@ -102,6 +105,7 @@ class TestBase(unittest.TestCase): L, = exe.run(program=prog, fetch_list=[loss], use_program_cache=True) + loss_list.append(np.mean(L)) step += 1 except fluid.core.EOFException: py_reader.reset() @@ -123,10 +127,15 @@ class TestBase(unittest.TestCase): feed=d, fetch_list=[loss], use_program_cache=True) + loss_list.append(np.mean(L)) step += 1 step_list.append(step) end_t = time.time() - ret = {"time": end_t - start_t, "step": step_list} + ret = { + "time": end_t - start_t, + "step": step_list, + "loss": np.array(loss_list) + } return ret def prepare_places(self, with_data_parallel, with_cpu=True, with_gpu=True): @@ -148,12 +157,18 @@ class TestBase(unittest.TestCase): for with_data_parallel in [True, False]: for p in self.prepare_places(with_data_parallel): for use_double_buffer in [False, True]: + results = [] for use_legacy_py_reader in [False, True]: ret = self.run_main( use_legacy_py_reader=use_legacy_py_reader, with_data_parallel=with_data_parallel, places=p, use_double_buffer=use_double_buffer) + results.append(ret) + if not use_double_buffer: + diff = np.max( + np.abs(results[0]['loss'] - results[1]['loss'])) + self.assertLess(diff, 1e-3) if __name__ == '__main__': diff --git a/python/paddle/fluid/tests/unittests/test_py_reader_sample_generator.py b/python/paddle/fluid/tests/unittests/test_py_reader_sample_generator.py index 2f8f0b1b6e..4efca5e2aa 100644 --- a/python/paddle/fluid/tests/unittests/test_py_reader_sample_generator.py +++ b/python/paddle/fluid/tests/unittests/test_py_reader_sample_generator.py @@ -62,7 +62,7 @@ class TestCaseBase(unittest.TestCase): all_datas = self.generate_all_data(batch_reader) if not use_sample_generator: - py_reader.decorate_paddle_reader( + py_reader.decorate_sample_list_generator( batch_reader, places=fluid.cpu_places()) else: py_reader.decorate_sample_generator( From 22715487dc222bc52aa88beb8b41137f7724068f Mon Sep 17 00:00:00 2001 From: zhhsplendid Date: Tue, 19 Mar 2019 11:39:47 +0000 Subject: [PATCH 12/44] add allocator flags test=develop --- CMakeLists.txt | 2 + paddle/fluid/API.spec | 21 +- .../fluid/framework/details/graph_test_base.h | 10 +- paddle/fluid/framework/details/op_registry.h | 6 +- paddle/fluid/framework/grad_op_desc_maker.h | 8 +- paddle/fluid/framework/ir/CMakeLists.txt | 9 +- .../fluid/framework/ir/cpu_quantize_pass.cc | 239 ++++++++++++++ paddle/fluid/framework/ir/cpu_quantize_pass.h | 66 ++++ .../framework/ir/cpu_quantize_pass_tester.cc | 211 +++++++++++++ .../ir/cpu_quantize_placement_pass.cc | 58 ++++ .../ir/cpu_quantize_placement_pass.h | 34 ++ .../ir/cpu_quantize_placement_pass_tester.cc | 129 ++++++++ .../framework/ir/graph_pattern_detector.cc | 51 ++- .../framework/ir/graph_pattern_detector.h | 29 ++ paddle/fluid/framework/ir/graph_test.cc | 14 +- .../ir/runtime_context_cache_pass.cc | 39 +++ .../framework/ir/runtime_context_cache_pass.h | 32 ++ paddle/fluid/framework/op_desc.cc | 4 +- paddle/fluid/framework/operator.cc | 28 +- paddle/fluid/framework/operator.h | 11 + paddle/fluid/framework/tensor_util.cc | 5 + paddle/fluid/framework/type_defs.h | 3 +- paddle/fluid/framework/var_type_inference.h | 117 ++++++- .../framework/var_type_inference_test.cc | 12 +- paddle/fluid/imperative/CMakeLists.txt | 1 + paddle/fluid/imperative/layer.cc | 100 ++++-- paddle/fluid/imperative/layer.h | 201 ++++++++++-- paddle/fluid/imperative/profiler.cc | 62 ++++ paddle/fluid/imperative/profiler.h | 25 ++ paddle/fluid/imperative/tracer.cc | 78 ++--- paddle/fluid/imperative/tracer.h | 2 +- paddle/fluid/imperative/type_defs.h | 1 + paddle/fluid/inference/CMakeLists.txt | 2 +- paddle/fluid/inference/analysis/argument.h | 6 + .../inference/analysis/ir_pass_manager.cc | 11 +- paddle/fluid/inference/api/analysis_config.cc | 17 +- .../inference/api/paddle_analysis_config.h | 26 ++ .../fluid/inference/tests/api/CMakeLists.txt | 2 +- .../tests/api/analyzer_pyramid_dnn_tester.cc | 1 + .../tests/api/analyzer_transformer_tester.cc | 20 +- .../inference/tests/api/config_printer.h | 3 +- .../allocation/allocator_facade_test.cc | 37 ++- .../memory/allocation/legacy_allocator.cc | 20 +- paddle/fluid/memory/detail/buddy_allocator.cc | 26 +- paddle/fluid/memory/detail/buddy_allocator.h | 2 + .../fluid/memory/detail/system_allocator.cc | 20 +- paddle/fluid/operators/CMakeLists.txt | 6 +- .../fluid/operators/beam_search_decode_op.cc | 21 +- paddle/fluid/operators/beam_search_op.cc | 15 +- paddle/fluid/operators/concat_op.cc | 16 +- .../operators/controlflow/get_places_op.cc | 8 +- .../controlflow/tensor_array_read_write_op.cc | 15 +- .../fluid/operators/controlflow/while_op.cc | 17 +- paddle/fluid/operators/conv_op.cc | 7 + .../fluid/operators/detection/CMakeLists.txt | 1 + .../fluid/operators/detection/yolo_box_op.cc | 167 ++++++++++ .../fluid/operators/detection/yolo_box_op.cu | 120 +++++++ .../fluid/operators/detection/yolo_box_op.h | 149 +++++++++ .../operators/distributed_ops/fake_init_op.cc | 3 +- .../operators/distributed_ops/merge_ids_op.cc | 9 +- .../operators/distributed_ops/split_ids_op.cc | 11 +- paddle/fluid/operators/fc_op.cc | 27 +- paddle/fluid/operators/fc_op.h | 16 + paddle/fluid/operators/fill_constant_op.cc | 9 +- .../fused/fused_embedding_seq_pool_op.cc | 17 +- .../get_tensor_from_selected_rows_op.cc | 15 +- paddle/fluid/operators/hash_op.cc | 3 +- .../operators/hierarchical_sigmoid_op.cc | 24 +- paddle/fluid/operators/lod_rank_table_op.cc | 8 +- .../fluid/operators/lod_tensor_to_array_op.cc | 7 +- paddle/fluid/operators/lookup_table_op.cc | 14 +- .../fluid/operators/mkldnn/conv_mkldnn_op.cc | 1 + paddle/fluid/operators/mkldnn/fc_mkldnn_op.cc | 24 +- .../operators/mkldnn/transpose_mkldnn_op.cc | 28 +- paddle/fluid/operators/nccl/nccl_op.cc | 9 +- paddle/fluid/operators/nce_op.cc | 14 +- .../operators/ngraph/ngraph_engine_op.cc | 3 +- paddle/fluid/operators/optimizers/adam_op.h | 49 +-- .../operators/optimizers/lars_momentum_op.cc | 7 +- .../fluid/operators/optimizers/momentum_op.cc | 18 +- .../fluid/operators/optimizers/momentum_op.h | 19 +- .../fluid/operators/optimizers/rmsprop_op.h | 18 +- paddle/fluid/operators/optimizers/sgd_op.cc | 14 +- paddle/fluid/operators/pool_op.cc | 7 + paddle/fluid/operators/py_func_op.cc | 41 ++- .../reader/create_custom_reader_op.cc | 23 +- paddle/fluid/operators/reader/read_op.cc | 17 +- .../operators/reader/reader_op_registry.cc | 21 +- .../operators/reader/reader_op_registry.h | 8 +- paddle/fluid/operators/save_op.cc | 9 +- paddle/fluid/operators/scale_op.cc | 15 +- .../sequence_ops/sequence_enumerate_op.cc | 10 +- .../sequence_ops/sequence_enumerate_op.h | 39 ++- paddle/fluid/operators/slice_op.cu | 14 +- .../softmax_with_cross_entropy_op.cu | 3 +- .../fluid/operators/split_selected_rows_op.cc | 9 +- paddle/fluid/operators/squeeze_op.cc | 1 + paddle/fluid/operators/sum_op.cc | 32 +- .../operators/tensor_array_to_tensor_op.cc | 7 +- .../operators/tensorrt/tensorrt_engine_op.cc | 3 +- paddle/fluid/operators/uniform_random_op.cc | 15 +- paddle/fluid/platform/device_context.cc | 2 + paddle/fluid/platform/device_context.h | 4 + paddle/fluid/platform/gpu_info.cc | 61 +++- paddle/fluid/platform/gpu_info.h | 6 + paddle/fluid/pybind/CMakeLists.txt | 2 +- paddle/fluid/pybind/imperative.cc | 6 +- paddle/fluid/pybind/inference_api.cc | 4 + paddle/fluid/pybind/pybind.cc | 8 +- paddle/testing/paddle_gtest_main.cc | 2 + python/paddle/fluid/__init__.py | 6 +- .../fluid/contrib/utils/lookup_table_utils.py | 294 ++++++++++++++---- python/paddle/fluid/data_feeder.py | 6 +- python/paddle/fluid/executor.py | 20 +- python/paddle/fluid/framework.py | 5 + python/paddle/fluid/imperative/__init__.py | 4 + python/paddle/fluid/imperative/profiler.py | 30 ++ python/paddle/fluid/layers/detection.py | 78 +++++ python/paddle/fluid/layers/nn.py | 67 +++- python/paddle/fluid/tests/test_detection.py | 10 + .../mkldnn/test_transpose_int8_mkldnn_op.py | 78 +++++ .../tests/unittests/test_imperative_gnn.py | 144 +++++++++ .../fluid/tests/unittests/test_layers.py | 75 +++++ .../fluid/tests/unittests/test_slice_op.py | 26 ++ .../fluid/tests/unittests/test_yolo_box_op.py | 117 +++++++ python/paddle/reader/__init__.py | 7 +- python/paddle/reader/creator.py | 20 +- python/paddle/reader/decorator.py | 28 +- tools/manylinux1/build_scripts/build.sh | 6 + 129 files changed, 3392 insertions(+), 708 deletions(-) create mode 100644 paddle/fluid/framework/ir/cpu_quantize_pass.cc create mode 100644 paddle/fluid/framework/ir/cpu_quantize_pass.h create mode 100644 paddle/fluid/framework/ir/cpu_quantize_pass_tester.cc create mode 100644 paddle/fluid/framework/ir/cpu_quantize_placement_pass.cc create mode 100644 paddle/fluid/framework/ir/cpu_quantize_placement_pass.h create mode 100644 paddle/fluid/framework/ir/cpu_quantize_placement_pass_tester.cc create mode 100644 paddle/fluid/framework/ir/runtime_context_cache_pass.cc create mode 100644 paddle/fluid/framework/ir/runtime_context_cache_pass.h create mode 100644 paddle/fluid/imperative/profiler.cc create mode 100644 paddle/fluid/imperative/profiler.h create mode 100644 paddle/fluid/operators/detection/yolo_box_op.cc create mode 100644 paddle/fluid/operators/detection/yolo_box_op.cu create mode 100644 paddle/fluid/operators/detection/yolo_box_op.h create mode 100644 python/paddle/fluid/imperative/profiler.py create mode 100644 python/paddle/fluid/tests/unittests/mkldnn/test_transpose_int8_mkldnn_op.py create mode 100644 python/paddle/fluid/tests/unittests/test_imperative_gnn.py create mode 100644 python/paddle/fluid/tests/unittests/test_yolo_box_op.py diff --git a/CMakeLists.txt b/CMakeLists.txt index 8e7ffe72b5..6bb0e5f51f 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -24,6 +24,8 @@ message(STATUS "CXX compiler: ${CMAKE_CXX_COMPILER}, version: " "${CMAKE_CXX_COMPILER_ID} ${CMAKE_CXX_COMPILER_VERSION}") message(STATUS "C compiler: ${CMAKE_C_COMPILER}, version: " "${CMAKE_C_COMPILER_ID} ${CMAKE_C_COMPILER_VERSION}") +message(STATUS "AR tools: ${CMAKE_AR}") + if(WIN32) set(CMAKE_SUPPRESS_REGENERATION ON) set(CMAKE_STATIC_LIBRARY_PREFIX lib) diff --git a/paddle/fluid/API.spec b/paddle/fluid/API.spec index fdd23681af..9a6d0d1c08 100644 --- a/paddle/fluid/API.spec +++ b/paddle/fluid/API.spec @@ -12,7 +12,7 @@ paddle.fluid.program_guard (ArgSpec(args=['main_program', 'startup_program'], va paddle.fluid.name_scope (ArgSpec(args=['prefix'], varargs=None, keywords=None, defaults=(None,)), ('document', '0ef753f5cec69fef9ae6ad8b867b33a2')) paddle.fluid.Executor.__init__ (ArgSpec(args=['self', 'place'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.Executor.close (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', 'f5369953dd0c443961cf79f7a00e1a03')) -paddle.fluid.Executor.run (ArgSpec(args=['self', 'program', 'feed', 'fetch_list', 'feed_var_name', 'fetch_var_name', 'scope', 'return_numpy', 'use_program_cache'], varargs=None, keywords=None, defaults=(None, None, None, 'feed', 'fetch', None, True, False)), ('document', 'aba8093edebf2d5c869b735b92811e45')) +paddle.fluid.Executor.run (ArgSpec(args=['self', 'program', 'feed', 'fetch_list', 'feed_var_name', 'fetch_var_name', 'scope', 'return_numpy', 'use_program_cache'], varargs=None, keywords=None, defaults=(None, None, None, 'feed', 'fetch', None, True, False)), ('document', 'f482e93b38b4018796969a2e1dde479d')) paddle.fluid.global_scope (ArgSpec(args=[], varargs=None, keywords=None, defaults=None), ('document', 'e148d3ab1ed8edf3e928212a375959c0')) paddle.fluid.scope_guard (ArgSpec(args=['scope'], varargs=None, keywords=None, defaults=None), ('document', 'b94d1f6bcc29c4fb58fc0058561250c2')) paddle.fluid.DistributeTranspiler.__init__ (ArgSpec(args=['self', 'config'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) @@ -68,7 +68,7 @@ paddle.fluid.initializer.MSRAInitializer.__init__ (ArgSpec(args=['self', 'unifor paddle.fluid.initializer.force_init_on_cpu (ArgSpec(args=[], varargs=None, keywords=None, defaults=None), ('document', '6d0f3e22c90d9d500d36ff57daf056ee')) paddle.fluid.initializer.init_on_cpu (ArgSpec(args=[], varargs=None, keywords=None, defaults=None), ('document', 'a6d7011ca3d8c0d454dac3a56eae0c29')) paddle.fluid.initializer.NumpyArrayInitializer.__init__ (ArgSpec(args=['self', 'value'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) -paddle.fluid.layers.fc (ArgSpec(args=['input', 'size', 'num_flatten_dims', 'param_attr', 'bias_attr', 'act', 'is_test', 'name'], varargs=None, keywords=None, defaults=(1, None, None, None, False, None)), ('document', '1929058262994f212620599c63aea6bd')) +paddle.fluid.layers.fc (ArgSpec(args=['input', 'size', 'num_flatten_dims', 'param_attr', 'bias_attr', 'act', 'is_test', 'name'], varargs=None, keywords=None, defaults=(1, None, None, None, False, None)), ('document', '424e898365195e3ccbc2e7dc8b63605e')) paddle.fluid.layers.embedding (ArgSpec(args=['input', 'size', 'is_sparse', 'is_distributed', 'padding_idx', 'param_attr', 'dtype'], varargs=None, keywords=None, defaults=(False, False, None, None, 'float32')), ('document', '89c2c55a0b0656b106064048e068e77a')) paddle.fluid.layers.dynamic_lstm (ArgSpec(args=['input', 'size', 'h_0', 'c_0', 'param_attr', 'bias_attr', 'use_peepholes', 'is_reverse', 'gate_activation', 'cell_activation', 'candidate_activation', 'dtype', 'name'], varargs=None, keywords=None, defaults=(None, None, None, None, True, False, 'sigmoid', 'tanh', 'tanh', 'float32', None)), ('document', 'dfbb624f85015df29e994ca6999e8ff6')) paddle.fluid.layers.dynamic_lstmp (ArgSpec(args=['input', 'size', 'proj_size', 'param_attr', 'bias_attr', 'use_peepholes', 'is_reverse', 'gate_activation', 'cell_activation', 'candidate_activation', 'proj_activation', 'dtype', 'name', 'h_0', 'c_0', 'cell_clip', 'proj_clip'], varargs=None, keywords=None, defaults=(None, None, True, False, 'sigmoid', 'tanh', 'tanh', 'tanh', 'float32', None, None, None, None, None)), ('document', 'b4b608b986eb9617aa0525e1be21d32d')) @@ -331,6 +331,7 @@ paddle.fluid.layers.iou_similarity (ArgSpec(args=['x', 'y', 'name'], varargs=Non paddle.fluid.layers.box_coder (ArgSpec(args=['prior_box', 'prior_box_var', 'target_box', 'code_type', 'box_normalized', 'name', 'axis'], varargs=None, keywords=None, defaults=('encode_center_size', True, None, 0)), ('document', '032d0f4b7d8f6235ee5d91e473344f0e')) paddle.fluid.layers.polygon_box_transform (ArgSpec(args=['input', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '0e5ac2507723a0b5adec473f9556799b')) paddle.fluid.layers.yolov3_loss (ArgSpec(args=['x', 'gtbox', 'gtlabel', 'anchors', 'anchor_mask', 'class_num', 'ignore_thresh', 'downsample_ratio', 'gtscore', 'use_label_smooth', 'name'], varargs=None, keywords=None, defaults=(None, True, None)), ('document', '57fa96922e42db8f064c3fb77f2255e8')) +paddle.fluid.layers.yolo_box (ArgSpec(args=['x', 'img_size', 'anchors', 'class_num', 'conf_thresh', 'downsample_ratio', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '5566169a5ab993d177792c023c7fb340')) paddle.fluid.layers.box_clip (ArgSpec(args=['input', 'im_info', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '397e9e02b451d99c56e20f268fa03f2e')) paddle.fluid.layers.multiclass_nms (ArgSpec(args=['bboxes', 'scores', 'score_threshold', 'nms_top_k', 'keep_top_k', 'nms_threshold', 'normalized', 'nms_eta', 'background_label', 'name'], varargs=None, keywords=None, defaults=(0.3, True, 1.0, 0, None)), ('document', 'ca7d1107b6c5d2d6d8221039a220fde0')) paddle.fluid.layers.distribute_fpn_proposals (ArgSpec(args=['fpn_rois', 'min_level', 'max_level', 'refer_level', 'refer_scale', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '7bb011ec26bace2bc23235aa4a17647d')) @@ -392,9 +393,9 @@ paddle.fluid.contrib.MagnitudePruner.__init__ (ArgSpec(args=['self', 'threshold' paddle.fluid.contrib.MagnitudePruner.prune (ArgSpec(args=['self', 'param', 'threshold'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.contrib.RatioPruner.__init__ (ArgSpec(args=['self', 'ratios'], varargs=None, keywords=None, defaults=(None,)), ('document', 'e7a81a325b296a9ca502ee5adb4fc85d')) paddle.fluid.contrib.RatioPruner.prune (ArgSpec(args=['self', 'param', 'ratio'], varargs=None, keywords=None, defaults=(None,)), ('document', '358cbf2978c91028fb96a195a9884645')) -paddle.fluid.contrib.load_persistables_for_increment (ArgSpec(args=['dirname', 'executor', 'program', 'lookup_table_var', 'lookup_table_var_path'], varargs=None, keywords=None, defaults=None), ('document', '11fbf7e8dd2289805de291b453a33ee7')) -paddle.fluid.contrib.load_persistables_for_inference (ArgSpec(args=['dirname', 'executor', 'program', 'lookup_table_var_name'], varargs=None, keywords=None, defaults=None), ('document', '5b5577bb3d24070da819674255d16196')) -paddle.fluid.contrib.convert_dist_to_sparse_program (ArgSpec(args=['program'], varargs=None, keywords=None, defaults=None), ('document', '4efbd93876832d4d35497cdbc7a1e6d8')) +paddle.fluid.contrib.load_persistables_for_increment (ArgSpec(args=['dirname', 'executor', 'program', 'lookup_table_var', 'lookup_table_var_path'], varargs=None, keywords=None, defaults=None), ('document', '2ab36d4f7a564f5f65e455807ad06c67')) +paddle.fluid.contrib.load_persistables_for_inference (ArgSpec(args=['dirname', 'executor', 'program', 'lookup_table_var_name'], varargs=None, keywords=None, defaults=None), ('document', '59066bac9db0ac6ce414d05780b7333f')) +paddle.fluid.contrib.convert_dist_to_sparse_program (ArgSpec(args=['program'], varargs=None, keywords=None, defaults=None), ('document', '74c39c595dc70d6be2f16d8e462d282b')) paddle.fluid.contrib.HDFSClient.__init__ (ArgSpec(args=['self', 'hadoop_home', 'configs'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.contrib.HDFSClient.delete (ArgSpec(args=['self', 'hdfs_path'], varargs=None, keywords=None, defaults=None), ('document', 'c3721aa2d4d9ef5a857dd47b2681c03e')) paddle.fluid.contrib.HDFSClient.download (ArgSpec(args=['self', 'hdfs_path', 'local_path', 'overwrite', 'unzip'], varargs=None, keywords=None, defaults=(False, False)), ('document', 'ca55bde92184d3fd0f9f5c963b25e634')) @@ -493,7 +494,7 @@ paddle.fluid.CUDAPinnedPlace.__init__ __init__(self: paddle.fluid.core.CUDAPinne paddle.fluid.ParamAttr.__init__ (ArgSpec(args=['self', 'name', 'initializer', 'learning_rate', 'regularizer', 'trainable', 'gradient_clip', 'do_model_average'], varargs=None, keywords=None, defaults=(None, None, 1.0, None, True, None, False)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.WeightNormParamAttr.__init__ (ArgSpec(args=['self', 'dim', 'name', 'initializer', 'learning_rate', 'regularizer', 'trainable', 'gradient_clip', 'do_model_average'], varargs=None, keywords=None, defaults=(None, None, None, 1.0, None, True, None, False)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.DataFeeder.__init__ (ArgSpec(args=['self', 'feed_list', 'place', 'program'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) -paddle.fluid.DataFeeder.decorate_reader (ArgSpec(args=['self', 'reader', 'multi_devices', 'num_places', 'drop_last'], varargs=None, keywords=None, defaults=(None, True)), ('document', '0eed2f198dc73c08a41b61edbc755753')) +paddle.fluid.DataFeeder.decorate_reader (ArgSpec(args=['self', 'reader', 'multi_devices', 'num_places', 'drop_last'], varargs=None, keywords=None, defaults=(None, True)), ('document', 'f8f3df23c5633c614db781a91b81fb62')) paddle.fluid.DataFeeder.feed (ArgSpec(args=['self', 'iterable'], varargs=None, keywords=None, defaults=None), ('document', '459e316301279dfd82001b46f0b8ffca')) paddle.fluid.DataFeeder.feed_parallel (ArgSpec(args=['self', 'iterable', 'num_places'], varargs=None, keywords=None, defaults=(None,)), ('document', '543863d1f9d4853758adb613b8659e85')) paddle.fluid.clip.ErrorClipByValue.__init__ (ArgSpec(args=['self', 'max', 'min'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) @@ -517,11 +518,11 @@ paddle.reader.compose (ArgSpec(args=[], varargs='readers', keywords='kwargs', de paddle.reader.chain (ArgSpec(args=[], varargs='readers', keywords=None, defaults=None), ('document', 'd22c34e379a53901ae67a6bca7f4def4')) paddle.reader.shuffle (ArgSpec(args=['reader', 'buf_size'], varargs=None, keywords=None, defaults=None), ('document', 'e42ea6fee23ce26b23cb142cd1d6522d')) paddle.reader.firstn (ArgSpec(args=['reader', 'n'], varargs=None, keywords=None, defaults=None), ('document', 'c5bb8f7dd4f917f1569a368aab5b8aad')) -paddle.reader.xmap_readers (ArgSpec(args=['mapper', 'reader', 'process_num', 'buffer_size', 'order'], varargs=None, keywords=None, defaults=(False,)), ('document', '283bc0b8a0e26ae186b8b9bee4aec560')) +paddle.reader.xmap_readers (ArgSpec(args=['mapper', 'reader', 'process_num', 'buffer_size', 'order'], varargs=None, keywords=None, defaults=(False,)), ('document', '9c804a42f8a4dbaa76b3c98e0ab7f796')) paddle.reader.PipeReader.__init__ (ArgSpec(args=['self', 'command', 'bufsize', 'file_type'], varargs=None, keywords=None, defaults=(8192, 'plain')), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) -paddle.reader.PipeReader.get_line (ArgSpec(args=['self', 'cut_lines', 'line_break'], varargs=None, keywords=None, defaults=(True, '\n')), ('document', '5f80a7ed70052f01665e4c74acccfa69')) +paddle.reader.PipeReader.get_line (ArgSpec(args=['self', 'cut_lines', 'line_break'], varargs=None, keywords=None, defaults=(True, '\n')), ('document', '9621ae612e595b6c34eb3bb5f3eb1a45')) paddle.reader.multiprocess_reader (ArgSpec(args=['readers', 'use_pipe', 'queue_size'], varargs=None, keywords=None, defaults=(True, 1000)), ('document', '7d8b3a96e592107c893d5d51ce968ba0')) paddle.reader.Fake.__init__ (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.reader.creator.np_array (ArgSpec(args=['x'], varargs=None, keywords=None, defaults=None), ('document', '28d457fbc9a71efa4ac91a3be179cada')) -paddle.reader.creator.text_file (ArgSpec(args=['path'], varargs=None, keywords=None, defaults=None), ('document', '44fe286ab6175a5464d3a961a68c266a')) -paddle.reader.creator.recordio (ArgSpec(args=['paths', 'buf_size'], varargs=None, keywords=None, defaults=(100,)), ('document', '11b3704ea42cfd537953387a7e58dae8')) +paddle.reader.creator.text_file (ArgSpec(args=['path'], varargs=None, keywords=None, defaults=None), ('document', 'f45fcb7add066c8e042c6774fc7c3db2')) +paddle.reader.creator.recordio (ArgSpec(args=['paths', 'buf_size'], varargs=None, keywords=None, defaults=(100,)), ('document', 'b4a94ee0e2cefb495619275c2f8c61d2')) diff --git a/paddle/fluid/framework/details/graph_test_base.h b/paddle/fluid/framework/details/graph_test_base.h index 126959bcd8..d139f84883 100644 --- a/paddle/fluid/framework/details/graph_test_base.h +++ b/paddle/fluid/framework/details/graph_test_base.h @@ -68,11 +68,11 @@ class SplitOpMaker : public OpProtoAndCheckerMaker { class DummyVarTypeInference : public VarTypeInference { public: - void operator()(const OpDesc& op_desc, BlockDesc* block) const override { - auto& inputs = op_desc.Input("X"); - auto type = block->Var(inputs.front())->GetType(); - auto out_var_name = op_desc.Output("Out").front(); - block->Var(out_var_name)->SetType(type); + void operator()(framework::InferVarTypeContext* ctx) const override { + auto& inputs = ctx->Input("X"); + auto type = ctx->GetType(inputs.front()); + auto out_var_name = ctx->Output("Out").front(); + ctx->SetType(out_var_name, type); } }; diff --git a/paddle/fluid/framework/details/op_registry.h b/paddle/fluid/framework/details/op_registry.h index 0901e59f97..e13ff99f3f 100644 --- a/paddle/fluid/framework/details/op_registry.h +++ b/paddle/fluid/framework/details/op_registry.h @@ -16,6 +16,8 @@ limitations under the License. */ #include #include +#include +#include #include #include "paddle/fluid/framework/grad_op_desc_maker.h" #include "paddle/fluid/framework/inplace_op_inference.h" @@ -127,9 +129,9 @@ struct OpInfoFiller { template struct OpInfoFiller { void operator()(const char* op_type, OpInfo* info) const { - info->infer_var_type_ = [](const OpDesc& fwd_op, BlockDesc* block) { + info->infer_var_type_ = [](InferVarTypeContext* context) { T inference; - inference(fwd_op, block); + inference(context); }; } }; diff --git a/paddle/fluid/framework/grad_op_desc_maker.h b/paddle/fluid/framework/grad_op_desc_maker.h index 9bccb1a32b..f2f4c53eea 100644 --- a/paddle/fluid/framework/grad_op_desc_maker.h +++ b/paddle/fluid/framework/grad_op_desc_maker.h @@ -14,7 +14,9 @@ limitations under the License. */ #pragma once #include +#include #include +#include #include #include #include "paddle/fluid/framework/op_desc.h" @@ -55,11 +57,11 @@ class GradOpDescMakerBase { std::back_inserter(ret_val), [this](const std::string& fwd_var_name) -> std::string { auto g_name = GradVarName(fwd_var_name); - if (no_grad_set_.count(g_name)) { - return kEmptyVarName; - } else { + if (no_grad_set_.empty() || !no_grad_set_.count(g_name)) { (*this->grad_to_var_)[g_name] = fwd_var_name; return g_name; + } else { + return kEmptyVarName; } }); if (!drop_empty_grad) { diff --git a/paddle/fluid/framework/ir/CMakeLists.txt b/paddle/fluid/framework/ir/CMakeLists.txt index faf7768a7b..a79a53867d 100644 --- a/paddle/fluid/framework/ir/CMakeLists.txt +++ b/paddle/fluid/framework/ir/CMakeLists.txt @@ -46,6 +46,8 @@ cc_library(fuse_pass_base SRCS fuse_pass_base.cc DEPS pass) pass_library(graph_to_program_pass base) pass_library(graph_viz_pass base) pass_library(lock_free_optimize_pass base) +pass_library(cpu_quantize_placement_pass base) +pass_library(cpu_quantize_pass inference) pass_library(cpu_quantize_squash_pass inference) pass_library(fc_fuse_pass inference) pass_library(attention_lstm_fuse_pass inference) @@ -68,6 +70,7 @@ pass_library(conv_affine_channel_fuse_pass inference) pass_library(transpose_flatten_concat_fuse_pass inference) pass_library(identity_scale_op_clean_pass base) pass_library(sync_batch_norm_pass base) +pass_library(runtime_context_cache_pass base) # There may be many transpose-flatten structures in a model, and the output of # these structures will be used as inputs to the concat Op. This pattern will @@ -102,8 +105,12 @@ cc_test(test_graph_pattern_detector SRCS graph_pattern_detector_tester.cc DEPS g cc_test(test_fc_fuse_pass SRCS fc_fuse_pass_tester.cc DEPS fc_fuse_pass framework_proto) cc_test(test_seqpool_concat_fuse_pass SRCS seqpool_concat_fuse_pass_tester.cc DEPS seqpool_concat_fuse_pass framework_proto) cc_test(test_is_test_pass SRCS is_test_pass_tester.cc DEPS is_test_pass) -cc_test(test_sync_batch_norm_pass SRCS sync_batch_norm_pass_tester.cc DEPS sync_batch_norm_pass) +cc_test(test_cpu_quantize_placement_pass SRCS cpu_quantize_placement_pass_tester.cc DEPS cpu_quantize_placement_pass) +cc_test(test_cpu_quantize_pass SRCS cpu_quantize_pass_tester.cc DEPS cpu_quantize_pass naive_executor) cc_test(test_cpu_quantize_squash_pass SRCS cpu_quantize_squash_pass_tester.cc DEPS cpu_quantize_squash_pass naive_executor) +if(NOT WIN32) + cc_test(test_sync_batch_norm_pass SRCS sync_batch_norm_pass_tester.cc DEPS sync_batch_norm_pass) +endif() if (WITH_MKLDNN) cc_test(test_depthwise_conv_mkldnn_pass SRCS mkldnn/depthwise_conv_mkldnn_pass_tester.cc DEPS depthwise_conv_mkldnn_pass) cc_test(test_conv_bias_mkldnn_fuse_pass SRCS mkldnn/conv_bias_mkldnn_fuse_pass_tester.cc DEPS conv_bias_mkldnn_fuse_pass naive_executor) diff --git a/paddle/fluid/framework/ir/cpu_quantize_pass.cc b/paddle/fluid/framework/ir/cpu_quantize_pass.cc new file mode 100644 index 0000000000..edfaf47f01 --- /dev/null +++ b/paddle/fluid/framework/ir/cpu_quantize_pass.cc @@ -0,0 +1,239 @@ +// 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/ir/cpu_quantize_pass.h" +#include +#include +#include "paddle/fluid/framework/eigen.h" +#include "paddle/fluid/string/pretty_log.h" + +namespace paddle { +namespace framework { +namespace ir { + +namespace { + +void UnlinkNodes(ir::Node* a, ir::Node* b) { + a->outputs.erase(std::remove(a->outputs.begin(), a->outputs.end(), b), + a->outputs.end()); + b->inputs.erase(std::remove(b->inputs.begin(), b->inputs.end(), a), + b->inputs.end()); +} + +} // namespace + +enum { U8_MAX = 255, S8_MAX = 127 }; + +using EigenVectorArrayMap = Eigen::Map>; +using string::PrettyLogDetail; + +void CPUQuantizePass::QuantizeInput(Graph* g, Node* op, Node* input, + std::string input_name, double scale_to_one, + bool is_unsigned, + std::string scale_attr_name) const { + unsigned max = is_unsigned ? U8_MAX : S8_MAX; + float scale = scale_to_one * max; + + // Create quantize output variable + VarDesc quantize_out_desc(patterns::PDNodeName("quantize", "out")); + auto* quantize_out_node = g->CreateVarNode(&quantize_out_desc); + + // create a quantize op node + OpDesc q_desc; + q_desc.SetType("quantize"); + q_desc.SetInput("Input", std::vector({input->Name()})); + q_desc.SetOutput("Output", + std::vector({quantize_out_node->Name()})); + q_desc.SetAttr("Scale", scale); + q_desc.SetAttr("is_negative_input", !is_unsigned); + auto quantize_op = g->CreateOpNode(&q_desc); // OpDesc will be copied. + + // update op's input + op->Op()->SetInput(input_name, + std::vector({quantize_out_node->Name()})); + + // link quantize op + UnlinkNodes(input, op); + IR_NODE_LINK_TO(input, quantize_op); + IR_NODE_LINK_TO(quantize_op, quantize_out_node); + IR_NODE_LINK_TO(quantize_out_node, op); + + if (!scale_attr_name.empty()) op->Op()->SetAttr(scale_attr_name, scale); +} + +void CPUQuantizePass::DequantizeOutput(Graph* g, Node* op, Node* output, + std::string output_name, + double scale_to_one, bool is_unsigned, + std::string scale_attr_name) const { + unsigned max = is_unsigned ? U8_MAX : S8_MAX; + float scale = scale_to_one * max; + + // Create dequantize input variable + VarDesc dequantize_in_desc(patterns::PDNodeName("dequantize", "in")); + auto* dequantize_in_node = g->CreateVarNode(&dequantize_in_desc); + + // create a dequantize op node for output. + OpDesc deq_desc; + deq_desc.SetType("dequantize"); + deq_desc.SetInput("Input", + std::vector({dequantize_in_node->Name()})); + deq_desc.SetOutput("Output", std::vector({output->Name()})); + deq_desc.SetAttr("Scale", scale); + auto dequantize_op = g->CreateOpNode(&deq_desc); // OpDesc will be copied. + + // update op's output + op->Op()->SetOutput(output_name, + std::vector({dequantize_in_node->Name()})); + + // link dequantize op + UnlinkNodes(op, output); + IR_NODE_LINK_TO(op, dequantize_in_node); + IR_NODE_LINK_TO(dequantize_in_node, dequantize_op); + IR_NODE_LINK_TO(dequantize_op, output); + + if (!scale_attr_name.empty()) op->Op()->SetAttr(scale_attr_name, scale); +} + +void CPUQuantizePass::QuantizeConv(Graph* graph, + bool with_residual_data) const { + GraphPatternDetector gpd; + auto pattern = gpd.mutable_pattern(); + patterns::ConvResidual conv_pattern{pattern, name_scope_}; + conv_pattern(with_residual_data); + + int quantize_conv_count = 0; + auto handler = [&](const GraphPatternDetector::subgraph_t& subgraph, + Graph* g) { + VLOG(4) << "Quantize conv2d op"; + GET_IR_NODE_FROM_SUBGRAPH(conv_op, conv_op, conv_pattern); + auto* conv_op_desc = conv_op->Op(); + + // skip if should not be quantized + if (!conv_op_desc->HasAttr("use_quantizer") || + !boost::get(conv_op_desc->GetAttr("use_quantizer"))) + return; + + GET_IR_NODE_FROM_SUBGRAPH(conv_filter, conv_filter, conv_pattern); + GET_IR_NODE_FROM_SUBGRAPH(conv_input, conv_input, conv_pattern); + GET_IR_NODE_FROM_SUBGRAPH(conv_output, conv_output, conv_pattern); + + // get scales calculated after warmup, they scale variables to MAX=1.0 + auto scales = Get("quant_var_scales"); + + auto input_scale = scales[conv_input->Name()].second.data()[0]; + bool is_input_unsigned = scales[conv_input->Name()].first; + QuantizeInput(g, conv_op, conv_input, "Input", input_scale, + is_input_unsigned, "Scale_in"); + + auto filter_scale_tensor = scales[conv_filter->Name()].second; + EigenVectorArrayMap eigen_tensor{filter_scale_tensor.data(), + filter_scale_tensor.numel(), 1}; + eigen_tensor *= static_cast(S8_MAX); + std::vector filter_scale{ + filter_scale_tensor.data(), + filter_scale_tensor.data() + filter_scale_tensor.numel()}; + + conv_op->Op()->SetAttr("Scale_weights", filter_scale); + + if (with_residual_data) { + GET_IR_NODE_FROM_SUBGRAPH(conv_residual_data, conv_residual_data, + conv_pattern); + auto residual_scale = + scales[conv_residual_data->Name()].second.data()[0]; + bool is_residual_unsigned = scales[conv_residual_data->Name()].first; + + QuantizeInput(g, conv_op, conv_residual_data, "ResidualData", + residual_scale, is_residual_unsigned, "Scale_in_eltwise"); + } + + auto output_scale = scales[conv_output->Name()].second.data()[0]; + bool is_output_unsigned = scales[conv_output->Name()].first; + DequantizeOutput(g, conv_op, conv_output, "Output", output_scale, + is_output_unsigned, "Scale_out"); + + ++quantize_conv_count; + }; + + gpd(graph, handler); + AddStatis(quantize_conv_count); + + std::stringstream msg_ss; + msg_ss << "--- quantized " << quantize_conv_count << " conv2d ops"; + if (with_residual_data) msg_ss << " with residual connection"; + PrettyLogDetail(msg_ss.str().c_str()); +} + +void CPUQuantizePass::QuantizePool(Graph* graph) const { + GraphPatternDetector gpd; + auto pattern = gpd.mutable_pattern(); + patterns::Pool pool_pattern{pattern, name_scope_}; + pool_pattern(); + + int quantize_pool_count = 0; + auto handler = [&](const GraphPatternDetector::subgraph_t& subgraph, + Graph* g) { + VLOG(4) << "Quantize pool2d op"; + GET_IR_NODE_FROM_SUBGRAPH(pool_op, pool_op, pool_pattern); + auto* pool_op_desc = pool_op->Op(); + + // skip if should not be quantized + if (!pool_op_desc->HasAttr("use_quantizer") || + !boost::get(pool_op_desc->GetAttr("use_quantizer"))) + return; + + GET_IR_NODE_FROM_SUBGRAPH(pool_input, pool_input, pool_pattern); + GET_IR_NODE_FROM_SUBGRAPH(pool_output, pool_output, pool_pattern); + + // get scales calculated after warmup, they scale variables to MAX=1.0 + auto scales = Get("quant_var_scales"); + + auto input_scale = scales[pool_input->Name()].second.data()[0]; + bool is_input_unsigned = scales[pool_input->Name()].first; + QuantizeInput(g, pool_op, pool_input, "X", input_scale, is_input_unsigned); + + auto output_scale = scales[pool_output->Name()].second.data()[0]; + bool is_output_unsigned = scales[pool_output->Name()].first; + DequantizeOutput(g, pool_op, pool_output, "Out", output_scale, + is_output_unsigned); + + ++quantize_pool_count; + }; + + gpd(graph, handler); + AddStatis(quantize_pool_count); + + PrettyLogDetail("--- quantized %d pool2d ops", quantize_pool_count); +} + +std::unique_ptr CPUQuantizePass::ApplyImpl( + std::unique_ptr graph) const { + VLOG(3) << "Quantizing the graph."; + PADDLE_ENFORCE(graph.get()); + FusePassBase::Init(name_scope_, graph.get()); + + PADDLE_ENFORCE(param_scope()); + + QuantizeConv(graph.get(), true /* with_residual_data */); + QuantizeConv(graph.get()); + QuantizePool(graph.get()); + + return graph; +} + +} // namespace ir +} // namespace framework +} // namespace paddle + +REGISTER_PASS(cpu_quantize_pass, paddle::framework::ir::CPUQuantizePass) + .RequirePassAttr("quant_var_scales"); diff --git a/paddle/fluid/framework/ir/cpu_quantize_pass.h b/paddle/fluid/framework/ir/cpu_quantize_pass.h new file mode 100644 index 0000000000..9873bb04e1 --- /dev/null +++ b/paddle/fluid/framework/ir/cpu_quantize_pass.h @@ -0,0 +1,66 @@ +// 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/ir/fuse_pass_base.h" +#include "paddle/fluid/framework/ir/graph.h" +#include "paddle/fluid/framework/ir/graph_pattern_detector.h" + +namespace paddle { +namespace framework { +namespace ir { + +/* + * Map variable name to tensor of scaling factors scaling it to MAX=1.0. + * bool denotes whether quantization of the variable should be done to unsigned + * type. + */ +using VarQuantScale = + std::unordered_map>; + +/* + * Quantize all supported operators. + */ +class CPUQuantizePass : public FusePassBase { + public: + virtual ~CPUQuantizePass() {} + + protected: + std::unique_ptr ApplyImpl( + std::unique_ptr graph) const override; + + void QuantizeConv(Graph* graph, bool with_residual_data = false) const; + + void QuantizePool(Graph* graph) const; + + void QuantizeInput(Graph* g, Node* op, Node* input, std::string input_name, + double scale_to_one, bool is_unsigned, + std::string scale_attr_name = "") const; + + void DequantizeOutput(Graph* g, Node* op, Node* output, + std::string output_name, double scale_to_one, + bool is_unsigned, + std::string scale_attr_name = "") const; + + const std::string name_scope_{"quantize"}; +}; + +} // namespace ir +} // namespace framework +} // namespace paddle diff --git a/paddle/fluid/framework/ir/cpu_quantize_pass_tester.cc b/paddle/fluid/framework/ir/cpu_quantize_pass_tester.cc new file mode 100644 index 0000000000..89601be7d1 --- /dev/null +++ b/paddle/fluid/framework/ir/cpu_quantize_pass_tester.cc @@ -0,0 +1,211 @@ +// 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/ir/cpu_quantize_pass.h" +#include +#include "paddle/fluid/framework/naive_executor.h" +#include "paddle/fluid/platform/place.h" + +namespace paddle { +namespace framework { +namespace ir { + +void SetOp(ProgramDesc* prog, const std::string& type, const std::string& name, + const std::vector& inputs, + const std::vector& outputs, bool use_mkldnn, + bool use_quantizer = false) { + auto* op = prog->MutableBlock(0)->AppendOp(); + op->SetType(type); + op->SetAttr("use_mkldnn", use_mkldnn); + op->SetAttr("name", name); + if (type == "conv2d") { + op->SetInput("Input", {inputs[0]}); + op->SetInput("Filter", {inputs[1]}); + if (inputs.size() > 2) + op->SetInput("Bias", {inputs[2]}); + else + op->SetInput("Bias", {}); + if (inputs.size() > 3) { + op->SetInput("ResidualData", {inputs[3]}); + op->SetAttr("fuse_residual_connection", true); + } else { + op->SetInput("ResidualData", {}); + op->SetAttr("fuse_residual_connection", false); + } + op->SetOutput("Output", {outputs[0]}); + op->SetAttr("use_quantizer", use_quantizer); + op->SetAttr("Scale_in", 1.0f); + op->SetAttr("Scale_out", 1.0f); + op->SetAttr("Scale_weights", std::vector{1.0f}); + } else if (type == "pool2d") { + op->SetInput("X", {inputs[0]}); + op->SetOutput("Out", {outputs[0]}); + op->SetAttr("use_quantizer", use_quantizer); + } else if (type == "dropout") { + op->SetInput("X", {inputs[0]}); + op->SetOutput("Out", {outputs[0]}); + } else if (type == "fc") { + op->SetInput("Input", {inputs[0]}); + if (inputs.size() > 1) op->SetInput("W", {inputs[1]}); + if (inputs.size() > 2) op->SetInput("Bias", {inputs[2]}); + op->SetOutput("Out", {outputs[0]}); + } +} + +static const std::initializer_list variable_names{ + "a", "w1", "c", "d", "w2", "e", "f", "g", + "h", "w3", "b1", "i", "j", "w4", "b2"}; +// (a,w1)->Conv1->c and c->Pool1->d +// +// (d,w2)->Conv2->e and e->Pool2->f +// +// d->Dropout1->g and g->Fc1->h and (h,w3,b1,i)->Conv3->j +// +// (d,w4, b2)->Conv4->i +ProgramDesc BuildProgramDesc(bool use_mkldnn, bool use_quantizer) { + ProgramDesc prog; + for (auto& v : variable_names) { + auto* var = prog.MutableBlock(0)->Var(v); + if (v.find("w") == 0 || v.find("b") == 0) { + var->SetPersistable(true); + } + } + + SetOp(&prog, "conv2d", "Conv1", {"a", "w1"}, {"c"}, use_mkldnn, + use_quantizer); + SetOp(&prog, "pool2d", "Pool1", {"c"}, {"d"}, use_mkldnn, use_quantizer); + + SetOp(&prog, "conv2d", "Conv2", {"d", "w2"}, {"e"}, use_mkldnn, + use_quantizer); + SetOp(&prog, "pool2d", "Pool2", {"e"}, {"f"}, use_mkldnn, use_quantizer); + + SetOp(&prog, "dropout", "Dropout1", {"d"}, {"g"}, use_mkldnn); + SetOp(&prog, "fc", "Fc1", {"g"}, {"h"}, use_mkldnn); + SetOp(&prog, "conv2d", "Conv3", {"h", "w3", "b1", "i"}, {"j"}, use_mkldnn, + use_quantizer); + + SetOp(&prog, "conv2d", "Conv4", {"c", "w4", "b2"}, {"i"}, use_mkldnn, + use_quantizer); + + return prog; +} + +void InitTensorHolder(Scope* scope, const paddle::platform::Place& place, + const char* var_name) { + auto x = scope->Var(var_name); + auto tensor = x->GetMutable(); + tensor->mutable_data(place, proto::VarType::FP32, + ::paddle::memory::Allocator::kDefault, 1); +} + +void MainTest(const ProgramDesc& prog, int conv_count, int pool_count, + int quant_count, int dequant_count, int added_nodes_count, + float scale) { + std::unique_ptr graph(new ir::Graph(prog)); + + // Init scope, as it is used in pass + auto place = paddle::platform::CPUPlace(); + NaiveExecutor exe{place}; + Scope scope; + exe.CreateVariables(prog, 0, true, &scope); + + auto* scales = new VarQuantScale(); + + for (auto& v : variable_names) { + InitTensorHolder(&scope, place, v.c_str()); + LoDTensor tensor; + tensor.Resize({1}); + auto* ptr = tensor.mutable_data(place); + ptr[0] = 2.0; + + (*scales)[v] = std::make_pair(false, std::move(tensor)); + } + + graph->Set(kParamScopeAttr, new framework::Scope*(&scope)); + + auto pass = PassRegistry::Instance().Get("cpu_quantize_pass"); + pass->Set("quant_var_scales", scales); + + int original_nodes_num = graph->Nodes().size(); + + graph = pass->Apply(std::move(graph)); + + int current_nodes_num = graph->Nodes().size(); + + int quantize_nodes_count = 0; + int dequantize_nodes_count = 0; + int conv2d_nodes_count = 0; + int pool2d_nodes_count = 0; + for (auto* node : graph->Nodes()) { + if (node->IsOp()) { + auto* op = node->Op(); + if (op->Type() == "conv2d") { + conv2d_nodes_count++; + auto op_name = boost::get(op->GetAttr("name")); + EXPECT_EQ(boost::get(op->GetAttr("Scale_in")), scale) + << "Scale_in for node '" + op_name + "'."; + EXPECT_EQ(boost::get(op->GetAttr("Scale_out")), scale) + << "Scale_out for node '" + op_name + "'."; + EXPECT_EQ( + boost::get>(op->GetAttr("Scale_weights"))[0], + scale) + << "Scale_weights for node '" + op_name + "'."; + } else if (op->Type() == "pool2d") { + pool2d_nodes_count++; + } else if (op->Type() == "quantize") { + quantize_nodes_count++; + } else if (op->Type() == "dequantize") { + dequantize_nodes_count++; + } + } + } + EXPECT_EQ(conv2d_nodes_count, conv_count); + EXPECT_EQ(pool2d_nodes_count, pool_count); + EXPECT_EQ(quantize_nodes_count, quant_count); + EXPECT_EQ(dequantize_nodes_count, dequant_count); + EXPECT_EQ(original_nodes_num + added_nodes_count, current_nodes_num); +} + +TEST(CpuQuantizePass, quantize) { + bool use_mkldnn = true; + bool use_quantizer = true; + // (a->QUANT1->IN1,w1)->Conv1->OUT1->DEQUANT1->c and + // c->QUANT2->IN2->Pool1->OUT2->DEQUANT2->d + // + // (d->QUANT3->IN3,w2)->Conv2->OUT3->DEQUANT3->e and + // e->QUANT4->IN4->Pool2->OUT4->DEQUANT4->f + // + // d->Dropout1->g and g->Fc1->h and + // (h->QUANT5->IN5,w3,b1,i->QUANT6->IN6)->Conv3->OUT5->DEQUANT5->j + // + // (d->QUANT7->IN7,w4, b2)->Conv4->DEQUANT6->OUT6->i + // Insert nodes: 7 Quant + 7 IN + 6 OUT + 6 DEQUANT + int added_nodes = 7 + 7 + 6 + 6; + MainTest(BuildProgramDesc(use_mkldnn, use_quantizer), 4, 2, 7, 6, added_nodes, + 2.0f * 127); +} + +TEST(CpuQuantizePass, do_not_quantize) { + bool use_mkldnn = true; + bool use_quantizer = false; + int added_nodes = 0; + MainTest(BuildProgramDesc(use_mkldnn, use_quantizer), 4, 2, 0, 0, added_nodes, + 1.0f); +} + +} // namespace ir +} // namespace framework +} // namespace paddle + +USE_PASS(cpu_quantize_pass); diff --git a/paddle/fluid/framework/ir/cpu_quantize_placement_pass.cc b/paddle/fluid/framework/ir/cpu_quantize_placement_pass.cc new file mode 100644 index 0000000000..50bbe4915b --- /dev/null +++ b/paddle/fluid/framework/ir/cpu_quantize_placement_pass.cc @@ -0,0 +1,58 @@ +/* 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/ir/cpu_quantize_placement_pass.h" +#include +#include + +namespace paddle { +namespace framework { +namespace ir { + +std::unique_ptr CPUQuantizePlacementPass::ApplyImpl( + std::unique_ptr graph) const { + VLOG(3) << "Marks operators which are to be quantized."; + const auto& excluded_ids_list = + Get>("quantize_excluded_op_ids"); + const auto& op_types_list = + Get>("quantize_enabled_op_types"); + for (const Node* n : graph->Nodes()) { + if (n->IsOp()) { + if (std::find(excluded_ids_list.begin(), excluded_ids_list.end(), + n->id()) != excluded_ids_list.end()) + continue; + auto* op = n->Op(); + if (op->HasAttr("use_quantizer") || op->HasProtoAttr("use_quantizer")) { + if (op_types_list.empty()) { + op->SetAttr("use_quantizer", true); + } else if (std::find(op_types_list.begin(), op_types_list.end(), + n->Name()) != op_types_list.end()) { + op->SetAttr("use_quantizer", true); + } + } + } + } + return graph; +} + +} // namespace ir +} // namespace framework +} // namespace paddle + +REGISTER_PASS(cpu_quantize_placement_pass, + paddle::framework::ir::CPUQuantizePlacementPass) + // a vector of operator type names to be quantized ("conv2d" etc.) + .RequirePassAttr("quantize_enabled_op_types") + // a vector of operator ids that are to be excluded from quantization + .RequirePassAttr("quantize_excluded_op_ids"); diff --git a/paddle/fluid/framework/ir/cpu_quantize_placement_pass.h b/paddle/fluid/framework/ir/cpu_quantize_placement_pass.h new file mode 100644 index 0000000000..ef3861b249 --- /dev/null +++ b/paddle/fluid/framework/ir/cpu_quantize_placement_pass.h @@ -0,0 +1,34 @@ +/* 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 "paddle/fluid/framework/ir/pass.h" + +namespace paddle { +namespace framework { +namespace ir { +/* + * Specifies which operators should be quantized. + */ +class CPUQuantizePlacementPass : public Pass { + protected: + std::unique_ptr ApplyImpl( + std::unique_ptr graph) const override; +}; + +} // namespace ir +} // namespace framework +} // namespace paddle diff --git a/paddle/fluid/framework/ir/cpu_quantize_placement_pass_tester.cc b/paddle/fluid/framework/ir/cpu_quantize_placement_pass_tester.cc new file mode 100644 index 0000000000..5a4d622645 --- /dev/null +++ b/paddle/fluid/framework/ir/cpu_quantize_placement_pass_tester.cc @@ -0,0 +1,129 @@ +// 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/ir/cpu_quantize_placement_pass.h" + +#include +#include + +namespace paddle { +namespace framework { +namespace ir { + +void SetOp(ProgramDesc* prog, const std::string& type, const std::string& name, + const std::vector& inputs, + const std::vector& outputs, + boost::tribool use_quantizer) { + auto* op = prog->MutableBlock(0)->AppendOp(); + + op->SetType(type); + + if (!boost::indeterminate(use_quantizer)) + op->SetAttr("use_quantizer", use_quantizer); + + if (type == "conv2d") { + op->SetAttr("name", name); + op->SetInput("Input", {inputs[0]}); + op->SetInput("Filter", {inputs[1]}); + op->SetInput("Bias", {inputs[2]}); + } else if (type == "relu") { + op->SetInput("X", inputs); + } else if (type == "concat") { + op->SetAttr("axis", 1); + op->SetInput("X", {inputs[0], inputs[1]}); + } else if (type == "pool2d") { + op->SetInput("X", {inputs[0]}); + } else { + FAIL() << "Unexpected operator type."; + } + op->SetOutput("Out", {outputs[0]}); +} + +// operator use_quantizer +// --------------------------------------- +// (a,b)->concat->c none +// (c,weights,bias)->conv->f false +// f->relu->g none +// g->pool->h false +// (h,weights2,bias2)->conv->k false +// k->pool->l false +ProgramDesc BuildProgramDesc() { + ProgramDesc prog; + + for (auto& v : + std::vector({"a", "b", "c", "weights", "bias", "f", "g", + "h", "weights2", "bias2", "k", "l"})) { + auto* var = prog.MutableBlock(0)->Var(v); + var->SetType(proto::VarType::SELECTED_ROWS); + if (v == "weights" || v == "bias") { + var->SetPersistable(true); + } + } + + SetOp(&prog, "concat", "concat1", {"a", "b"}, {"c"}, boost::indeterminate); + SetOp(&prog, "conv2d", "conv1", {"c", "weights", "bias"}, {"f"}, false); + SetOp(&prog, "relu", "relu1", {"f"}, {"g"}, boost::indeterminate); + SetOp(&prog, "pool2d", "pool1", {"g"}, {"h"}, false); + SetOp(&prog, "conv2d", "conv2", {"h", "weights2", "bias2"}, {"k"}, false); + SetOp(&prog, "pool2d", "pool2", {"k"}, {"l"}, false); + + return prog; +} + +void MainTest(std::initializer_list quantize_enabled_op_types, + std::initializer_list quantize_excluded_op_ids, + unsigned expected_use_quantizer_true_count) { + auto prog = BuildProgramDesc(); + + std::unique_ptr graph(new ir::Graph(prog)); + + auto pass = PassRegistry::Instance().Get("cpu_quantize_placement_pass"); + pass->Set("quantize_enabled_op_types", + new std::unordered_set(quantize_enabled_op_types)); + pass->Set("quantize_excluded_op_ids", + new std::unordered_set(quantize_excluded_op_ids)); + + graph = pass->Apply(std::move(graph)); + + unsigned use_quantizer_true_count = 0; + + for (auto* node : graph->Nodes()) { + if (node->IsOp()) { + auto* op = node->Op(); + if (op->HasAttr("use_quantizer") && + boost::get(op->GetAttr("use_quantizer"))) { + ++use_quantizer_true_count; + } + } + } + + EXPECT_EQ(use_quantizer_true_count, expected_use_quantizer_true_count); +} + +TEST(QuantizerPlacementPass, enabled_pool) { MainTest({"pool2d"}, {}, 2); } + +TEST(QuantizerPlacementPass, enabled_conv_excluded_one) { + MainTest({"conv2d"}, {4}, 1); +} + +TEST(QuantizerPlacementPass, excluded_none) { + // 2 conv + 2 pool + MainTest({}, {}, 4); +} + +} // namespace ir +} // namespace framework +} // namespace paddle + +USE_PASS(cpu_quantize_placement_pass); diff --git a/paddle/fluid/framework/ir/graph_pattern_detector.cc b/paddle/fluid/framework/ir/graph_pattern_detector.cc index 08354b526a..b653e5a521 100644 --- a/paddle/fluid/framework/ir/graph_pattern_detector.cc +++ b/paddle/fluid/framework/ir/graph_pattern_detector.cc @@ -90,7 +90,8 @@ void GraphPatternDetector::operator()(Graph *graph, ValidateByNodeRole(&subgraphs); if (subgraphs.empty()) return; - PrettyLogEndl(Style::detail(), "--- detect %d subgraphs", subgraphs.size()); + PrettyLogEndl(Style::detail(), "--- detected %d subgraphs", + subgraphs.size()); int id = 0; for (auto &g : subgraphs) { VLOG(3) << "optimizing #" << id++ << " subgraph"; @@ -1074,9 +1075,53 @@ PDNode *patterns::Conv::operator()() { ->AsOutput() ->assert_is_op_output("conv2d", "Output"); - conv_op->LinksFrom({input_var, filter_var}); - conv_op->LinksTo({output_var}); + conv_op->LinksFrom({input_var, filter_var}).LinksTo({output_var}); + return output_var; +} + +PDNode *patterns::ConvResidual::operator()(bool with_residual_data) { + auto conv_op = pattern->NewNode(conv_op_repr())->assert_is_op("conv2d"); + + if (!with_residual_data) + conv_op->assert_op_attr("fuse_residual_connection", false); + + auto input_var = pattern->NewNode(conv_input_repr()) + ->AsInput() + ->assert_is_op_input("conv2d", "Input"); + + auto filter_var = pattern->NewNode(conv_filter_repr()) + ->AsInput() + ->assert_is_op_input("conv2d", "Filter"); + + auto output_var = pattern->NewNode(conv_output_repr()) + ->AsOutput() + ->assert_is_op_output("conv2d", "Output"); + + std::vector links_from{input_var, filter_var}; + + if (with_residual_data) { + auto res_conn_var = pattern->NewNode(conv_residual_data_repr()) + ->AsInput() + ->assert_is_op_input("conv2d", "ResidualData"); + links_from.push_back(res_conn_var); + } + + conv_op->LinksFrom(links_from).LinksTo({output_var}); + return output_var; +} + +PDNode *patterns::Pool::operator()() { + auto pool_op = pattern->NewNode(pool_op_repr())->assert_is_op("pool2d"); + + auto input_var = pattern->NewNode(pool_input_repr()) + ->AsInput() + ->assert_is_op_input("pool2d", "X"); + + auto output_var = pattern->NewNode(pool_output_repr()) + ->AsOutput() + ->assert_is_op_output("pool2d", "Out"); + pool_op->LinksFrom({input_var}).LinksTo({output_var}); return output_var; } diff --git a/paddle/fluid/framework/ir/graph_pattern_detector.h b/paddle/fluid/framework/ir/graph_pattern_detector.h index 3db4bba10d..fc30b5b21c 100644 --- a/paddle/fluid/framework/ir/graph_pattern_detector.h +++ b/paddle/fluid/framework/ir/graph_pattern_detector.h @@ -659,6 +659,35 @@ struct Conv : public PatternBase { PATTERN_DECL_NODE(conv_output); }; +// Convolution op with residual data +struct ConvResidual : public PatternBase { + ConvResidual(PDPattern* pattern, const std::string& name_scope) + : PatternBase(pattern, name_scope, "conv_residual") {} + + PDNode* operator()(bool with_residual_data); + + PATTERN_DECL_NODE(conv_op); + PATTERN_DECL_NODE(conv_input); + PATTERN_DECL_NODE(conv_filter); + PATTERN_DECL_NODE(conv_residual_data); + PATTERN_DECL_NODE(conv_output); +}; + +// Pool op +// Forward pass for pooling. +// pool_input is the input. +// pool_output is a result of the operator. +struct Pool : public PatternBase { + Pool(PDPattern* pattern, const std::string& name_scope) + : PatternBase(pattern, name_scope, "pooling") {} + + PDNode* operator()(); + + PATTERN_DECL_NODE(pool_op); + PATTERN_DECL_NODE(pool_input); + PATTERN_DECL_NODE(pool_output); +}; + // ElementwiseAdd used in residual connections. // y_var is used and convolution output. // The operator is removed, when residual diff --git a/paddle/fluid/framework/ir/graph_test.cc b/paddle/fluid/framework/ir/graph_test.cc index 7ed2f96eb2..a95588a57b 100644 --- a/paddle/fluid/framework/ir/graph_test.cc +++ b/paddle/fluid/framework/ir/graph_test.cc @@ -43,20 +43,20 @@ class SumOpMaker : public OpProtoAndCheckerMaker { class SumOpVarTypeInference : public VarTypeInference { public: - void operator()(const OpDesc &op_desc, BlockDesc *block) const override { - auto &inputs = op_desc.Input("X"); + void operator()(InferVarTypeContext *ctx) const override { + auto &inputs = ctx->Input("X"); auto default_var_type = proto::VarType::SELECTED_ROWS; bool any_input_is_lod_tensor = std::any_of( - inputs.begin(), inputs.end(), [block](const std::string &name) { - return block->Var(name)->GetType() == proto::VarType::LOD_TENSOR; + inputs.begin(), inputs.end(), [&ctx](const std::string &name) { + return ctx->GetType(name) == proto::VarType::LOD_TENSOR; }); if (any_input_is_lod_tensor) { default_var_type = proto::VarType::LOD_TENSOR; } - auto out_var_name = op_desc.Output("Out").front(); - block->Var(out_var_name)->SetType(default_var_type); + auto out_var_name = ctx->Output("Out").front(); + ctx->SetType(out_var_name, default_var_type); } }; @@ -71,7 +71,7 @@ class DummyOpMaker : public OpProtoAndCheckerMaker { class DummyOpVarTypeInference : public VarTypeInference { public: - void operator()(const OpDesc &op_desc, BlockDesc *block) const override {} + void operator()(framework::InferVarTypeContext *ctx) const override {} }; } // namespace framework } // namespace paddle diff --git a/paddle/fluid/framework/ir/runtime_context_cache_pass.cc b/paddle/fluid/framework/ir/runtime_context_cache_pass.cc new file mode 100644 index 0000000000..67b29512c4 --- /dev/null +++ b/paddle/fluid/framework/ir/runtime_context_cache_pass.cc @@ -0,0 +1,39 @@ +/* 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/ir/runtime_context_cache_pass.h" +#include +#include "paddle/fluid/framework/operator.h" + +namespace paddle { +namespace framework { +namespace ir { + +std::unique_ptr RuntimeContextCachePass::ApplyImpl( + std::unique_ptr graph) const { + VLOG(3) << "Applies Runtime Context Cache strategy."; + for (const Node* n : graph->Nodes()) { + if (n->IsOp()) { + n->Op()->SetAttr(kEnableCacheRuntimeContext, true); + } + } + return graph; +} + +} // namespace ir +} // namespace framework +} // namespace paddle + +REGISTER_PASS(runtime_context_cache_pass, + paddle::framework::ir::RuntimeContextCachePass); diff --git a/paddle/fluid/framework/ir/runtime_context_cache_pass.h b/paddle/fluid/framework/ir/runtime_context_cache_pass.h new file mode 100644 index 0000000000..a6cf1a9ae5 --- /dev/null +++ b/paddle/fluid/framework/ir/runtime_context_cache_pass.h @@ -0,0 +1,32 @@ +/* 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 "paddle/fluid/framework/ir/pass.h" + +namespace paddle { +namespace framework { +namespace ir { + +class RuntimeContextCachePass : public Pass { + protected: + std::unique_ptr ApplyImpl( + std::unique_ptr graph) const override; +}; + +} // namespace ir +} // namespace framework +} // namespace paddle diff --git a/paddle/fluid/framework/op_desc.cc b/paddle/fluid/framework/op_desc.cc index 0e7b0cbeb9..8f9c6cb5e9 100644 --- a/paddle/fluid/framework/op_desc.cc +++ b/paddle/fluid/framework/op_desc.cc @@ -24,6 +24,7 @@ limitations under the License. */ #include "paddle/fluid/framework/operator.h" #include "paddle/fluid/framework/program_desc.h" #include "paddle/fluid/framework/shape_inference.h" +#include "paddle/fluid/framework/var_type_inference.h" namespace paddle { namespace framework { @@ -677,7 +678,8 @@ void OpDesc::InferVarType(BlockDesc *block) const { // var type inference. Hence, we don't do any "default" setting here. auto &info = OpInfoMap::Instance().Get(this->Type()); if (info.infer_var_type_) { - info.infer_var_type_(*this, block); + InferVarTypeContext context(this, block); + info.infer_var_type_(&context); } } diff --git a/paddle/fluid/framework/operator.cc b/paddle/fluid/framework/operator.cc index 44821aadf6..ab96201b33 100644 --- a/paddle/fluid/framework/operator.cc +++ b/paddle/fluid/framework/operator.cc @@ -874,9 +874,23 @@ std::vector* OperatorWithKernel::GetKernelConfig( return kernel_configs; } +RuntimeContext* OperatorWithKernel::GetRuntimeContext( + const Scope& scope) const { + if (!HasAttr(kEnableCacheRuntimeContext)) { + return new RuntimeContext(Inputs(), Outputs(), scope); + } else { + const Scope* cur_scope = &scope; + if (!runtime_ctx_ || pre_scope_ != cur_scope) { + runtime_ctx_.reset(new RuntimeContext(Inputs(), Outputs(), scope)); + pre_scope_ = cur_scope; + } + return runtime_ctx_.get(); + } +} + void OperatorWithKernel::RunImpl(const Scope& scope, const platform::Place& place) const { - RuntimeContext ctx(Inputs(), Outputs(), scope); + auto runtime_ctx = GetRuntimeContext(scope); platform::DeviceContextPool& pool = platform::DeviceContextPool::Instance(); auto* dev_ctx = pool.Get(place); @@ -891,7 +905,7 @@ void OperatorWithKernel::RunImpl(const Scope& scope, OpKernelMap& kernels = kernels_iter->second; auto expected_kernel_key = this->GetExpectedKernelType( - ExecutionContext(*this, scope, *dev_ctx, ctx, nullptr)); + ExecutionContext(*this, scope, *dev_ctx, *runtime_ctx, nullptr)); VLOG(3) << "expected_kernel_key:" << expected_kernel_key; auto kernel_iter = kernels.find(expected_kernel_key); @@ -915,8 +929,8 @@ void OperatorWithKernel::RunImpl(const Scope& scope, // do data transformScope &transfer_scope; std::vector transfered_inplace_vars; - auto* transfer_scope = - PrepareData(scope, expected_kernel_key, &transfered_inplace_vars, &ctx); + auto* transfer_scope = PrepareData(scope, expected_kernel_key, + &transfered_inplace_vars, runtime_ctx); // exec scope is the scope that kernel actually executed on. const Scope& exec_scope = @@ -927,13 +941,13 @@ void OperatorWithKernel::RunImpl(const Scope& scope, } if (!HasAttr(kAllKernelsMustComputeRuntimeShape)) { - RuntimeInferShapeContext infer_shape_ctx(*this, exec_scope, ctx); + RuntimeInferShapeContext infer_shape_ctx(*this, exec_scope, *runtime_ctx); this->InferShape(&infer_shape_ctx); } // TODO(panyx0718): ExecutionContext should only depend on RuntimeContext // not Scope. Imperative mode only pass inputs and get outputs. - kernel_iter->second( - ExecutionContext(*this, exec_scope, *dev_ctx, ctx, kernel_configs)); + kernel_iter->second(ExecutionContext(*this, exec_scope, *dev_ctx, + *runtime_ctx, kernel_configs)); if (!transfered_inplace_vars.empty()) { // there is inplace variable has been transfered. diff --git a/paddle/fluid/framework/operator.h b/paddle/fluid/framework/operator.h index 822bf5c9ce..ca5f0e27b3 100644 --- a/paddle/fluid/framework/operator.h +++ b/paddle/fluid/framework/operator.h @@ -62,6 +62,14 @@ constexpr char kZeroVarSuffix[] = "@ZERO"; /// Variables with this suffix are the new Gradient. constexpr char kNewGradSuffix[] = "@NEWGRAD@"; +/// RuntimeContext is used to relate input/output names of Operator with +/// the corresponding variables in name scope. +/// If an Op has attribute kEnableCacheRuntimeContext, it means that in a same +/// name scope, since the input/output names of this Op do not change in the +/// execution, RuntimeContext could be created only at the first iteration of +/// this Op's execution to save the elapsed time. +constexpr char kEnableCacheRuntimeContext[] = "@ENABLE_CACHE_RUNTIME_CONTEXT@"; + /// If an Op has this attribute, all its kernels should calculate output /// variable's shape in the corresponding Compute() function. And /// OperatorWithKernel::RunImpl() would skip call this Op's InferShape() @@ -456,6 +464,7 @@ class OperatorWithKernel : public OperatorBase { // same. proto::VarType::Type IndicateDataType(const ExecutionContext& ctx) const; void RunImpl(const Scope& scope, const platform::Place& place) const final; + RuntimeContext* GetRuntimeContext(const Scope& scope) const; /** * Transfer data from scope to a transfered scope. If there is no data need to @@ -474,6 +483,8 @@ class OperatorWithKernel : public OperatorBase { protected: mutable OpKernelConfigsMap kernel_configs_map_; + mutable std::unique_ptr runtime_ctx_; + mutable const Scope* pre_scope_ = nullptr; }; extern bool OpSupportGPU(const std::string& op_type); diff --git a/paddle/fluid/framework/tensor_util.cc b/paddle/fluid/framework/tensor_util.cc index a7f09df491..5f21dae605 100644 --- a/paddle/fluid/framework/tensor_util.cc +++ b/paddle/fluid/framework/tensor_util.cc @@ -44,6 +44,11 @@ void TensorCopy(const Tensor& src, const platform::Place& dst_place, << dst_place; return; } +#ifdef PADDLE_WITH_MKLDNN + if (src.layout() == DataLayout::kMKLDNN) { + dst->set_mkldnn_prim_desc(src.get_mkldnn_prim_desc()); + } +#endif memory::Copy(boost::get(dst_place), dst_ptr, boost::get(src_place), src_ptr, size); } diff --git a/paddle/fluid/framework/type_defs.h b/paddle/fluid/framework/type_defs.h index d02c699b97..f55520901c 100644 --- a/paddle/fluid/framework/type_defs.h +++ b/paddle/fluid/framework/type_defs.h @@ -27,6 +27,7 @@ namespace framework { class OperatorBase; class OpDesc; class InferShapeContext; +class InferVarTypeContext; class BlockDesc; class Variable; @@ -53,7 +54,7 @@ using GradOpMakerFN = std::function>( const std::vector& grad_block)>; using InferVarTypeFN = - std::function; + std::function; using InferShapeFN = std::function; diff --git a/paddle/fluid/framework/var_type_inference.h b/paddle/fluid/framework/var_type_inference.h index 64236b78d2..2e9c64d3e6 100644 --- a/paddle/fluid/framework/var_type_inference.h +++ b/paddle/fluid/framework/var_type_inference.h @@ -14,6 +14,8 @@ limitations under the License. */ #pragma once #include +#include +#include #include "paddle/fluid/framework/block_desc.h" #include "paddle/fluid/framework/op_desc.h" #include "paddle/fluid/framework/type_defs.h" @@ -21,26 +23,123 @@ limitations under the License. */ namespace paddle { namespace framework { +class OpDesc; +class BlockDesc; +// default infer var type context +class InferVarTypeContext { + public: + InferVarTypeContext(const OpDesc* op, BlockDesc* block) + : op_(op), block_(block) {} + + virtual ~InferVarTypeContext() {} + + virtual Attribute GetAttr(const std::string& name) const { + PADDLE_ENFORCE_NOT_NULL(op_); + return op_->GetAttr(name); + } + + virtual bool HasVar(const std::string& name) const { + PADDLE_ENFORCE_NOT_NULL(block_); + return block_->FindVarRecursive(name) != nullptr; + } + + virtual bool HasInput(const std::string& name) const { + PADDLE_ENFORCE_NOT_NULL(op_); + return op_->Inputs().count(name) > 0; + } + + virtual bool HasOutput(const std::string& name) const { + PADDLE_ENFORCE_NOT_NULL(op_); + return op_->Outputs().count(name) > 0; + } + + virtual const std::vector& Input(const std::string& name) const { + PADDLE_ENFORCE_NOT_NULL(op_); + return op_->Input(name); + } + + virtual const std::vector& Output( + const std::string& name) const { + PADDLE_ENFORCE_NOT_NULL(op_); + return op_->Output(name); + } + + virtual proto::VarType::Type GetType(const std::string& name) const { + PADDLE_ENFORCE_NOT_NULL(block_); + return block_->FindRecursiveOrCreateVar(name).GetType(); + } + + virtual void SetType(const std::string& name, proto::VarType::Type type) { + PADDLE_ENFORCE_NOT_NULL(block_); + block_->FindRecursiveOrCreateVar(name).SetType(type); + } + + virtual proto::VarType::Type GetDataType(const std::string& name) const { + PADDLE_ENFORCE_NOT_NULL(block_); + return block_->FindRecursiveOrCreateVar(name).GetDataType(); + } + + virtual void SetDataType(const std::string& name, proto::VarType::Type type) { + PADDLE_ENFORCE_NOT_NULL(block_); + block_->FindRecursiveOrCreateVar(name).SetDataType(type); + } + + virtual std::vector GetDataTypes( + const std::string& name) const { + PADDLE_ENFORCE_NOT_NULL(block_); + return block_->FindRecursiveOrCreateVar(name).GetDataTypes(); + } + + virtual void SetDataTypes( + const std::string& name, + const std::vector& multiple_data_type) { + PADDLE_ENFORCE_NOT_NULL(block_); + block_->FindRecursiveOrCreateVar(name).SetDataTypes(multiple_data_type); + } + + virtual std::vector GetShape(const std::string& name) const { + PADDLE_ENFORCE_NOT_NULL(block_); + return block_->FindRecursiveOrCreateVar(name).GetShape(); + } + + virtual void SetShape(const std::string& name, + const std::vector& dims) { + PADDLE_ENFORCE_NOT_NULL(block_); + block_->FindRecursiveOrCreateVar(name).SetShape(dims); + } + + virtual int32_t GetLoDLevel(const std::string& name) const { + PADDLE_ENFORCE_NOT_NULL(block_); + return block_->FindRecursiveOrCreateVar(name).GetLoDLevel(); + } + + virtual void SetLoDLevel(const std::string& name, int32_t lod_level) { + PADDLE_ENFORCE_NOT_NULL(block_); + block_->FindRecursiveOrCreateVar(name).SetLoDLevel(lod_level); + } + + protected: + const OpDesc* op_; + BlockDesc* block_; +}; + class VarTypeInference { public: virtual ~VarTypeInference() {} - virtual void operator()(const OpDesc& op_desc, BlockDesc* block) const = 0; + virtual void operator()(InferVarTypeContext* context) const = 0; // NOLINT }; class PassInDtypeAndVarTypeToOutput : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc& op_desc, - framework::BlockDesc* block) const final { + void operator()(framework::InferVarTypeContext* ctx) const final { // NOLINT auto in_out_var_names = this->GetInputOutputWithSameType(); for (auto& i_o_n : in_out_var_names) { - auto& x_name = op_desc.Input(i_o_n.first).at(0); - auto& out_name = op_desc.Output(i_o_n.second).at(0); + auto& x_name = ctx->Input(i_o_n.first).at(0); + auto& out_name = ctx->Output(i_o_n.second).at(0); - auto& x = block->FindRecursiveOrCreateVar(x_name); - auto& out = block->FindRecursiveOrCreateVar(out_name); - out.SetType(x.GetType()); - out.SetDataType(x.GetDataType()); + ctx->SetType(out_name, ctx->GetType(x_name)); + ctx->SetDataType(out_name, ctx->GetDataType(x_name)); } } diff --git a/paddle/fluid/framework/var_type_inference_test.cc b/paddle/fluid/framework/var_type_inference_test.cc index 2a75394fca..6bbb25a573 100644 --- a/paddle/fluid/framework/var_type_inference_test.cc +++ b/paddle/fluid/framework/var_type_inference_test.cc @@ -44,20 +44,20 @@ class SumOpMaker : public OpProtoAndCheckerMaker { class SumOpVarTypeInference : public VarTypeInference { public: - void operator()(const OpDesc &op_desc, BlockDesc *block) const override { - auto &inputs = op_desc.Input("X"); + void operator()(framework::InferVarTypeContext *ctx) const override { + auto &inputs = ctx->Input("X"); auto default_var_type = proto::VarType::SELECTED_ROWS; bool any_input_is_lod_tensor = std::any_of( - inputs.begin(), inputs.end(), [block](const std::string &name) { - return block->Var(name)->GetType() == proto::VarType::LOD_TENSOR; + inputs.begin(), inputs.end(), [&ctx](const std::string &name) { + return ctx->GetType(name) == proto::VarType::LOD_TENSOR; }); if (any_input_is_lod_tensor) { default_var_type = proto::VarType::LOD_TENSOR; } - auto out_var_name = op_desc.Output("Out").front(); - block->Var(out_var_name)->SetType(default_var_type); + auto out_var_name = ctx->Output("Out").front(); + ctx->SetType(out_var_name, default_var_type); } }; } // namespace framework diff --git a/paddle/fluid/imperative/CMakeLists.txt b/paddle/fluid/imperative/CMakeLists.txt index ec8dedd605..0d116a6495 100644 --- a/paddle/fluid/imperative/CMakeLists.txt +++ b/paddle/fluid/imperative/CMakeLists.txt @@ -2,4 +2,5 @@ if(WITH_PYTHON) cc_library(layer SRCS layer.cc DEPS proto_desc operator device_context blas pybind) cc_library(tracer SRCS tracer.cc DEPS proto_desc device_context pybind) cc_library(engine SRCS engine.cc) +cc_library(imperative_profiler SRCS profiler.cc) endif() diff --git a/paddle/fluid/imperative/layer.cc b/paddle/fluid/imperative/layer.cc index 5530823b90..3d1de95f58 100644 --- a/paddle/fluid/imperative/layer.cc +++ b/paddle/fluid/imperative/layer.cc @@ -214,13 +214,11 @@ framework::LoDTensor& VarBase::GradValue() { } std::map> OpBase::ApplyGrad() { - if (grad_op_descs_.empty() && backward_id_ <= 0) { - VLOG(3) << "op with no grad: " << Type(); - return {}; - } + PADDLE_ENFORCE(!grad_op_descs_.empty() || backward_id_ > 0, + "%s has no backward implementation", Type()); VLOG(3) << "apply op grad: " << Type(); - std::vector tmp_grad_outputs; + std::vector tmp_grad_outputs; if (backward_id_ > 0) { VLOG(3) << "py_layer_grad"; tmp_grad_outputs.resize(1); @@ -239,30 +237,66 @@ std::map> OpBase::ApplyGrad() { VLOG(3) << "apply grad op " << grad_op_desc->Type(); // Allocate tmp grad output variable - for (auto it : grad_output_variable_map) { + for (const auto& it : grad_output_variable_map) { auto& outputs = tmp_grad_outputs[k][it.first]; outputs.reserve(it.second.size()); for (size_t i = 0; i < it.second.size(); ++i) { + VarBase* origin_grad_var_base = it.second[i]; + // Allocate a new variable - Variable* tmp_var = new framework::Variable(); - tmp_var->GetMutable(); - outputs.emplace_back(tmp_var); + VarBase* tmp_grad_var_base = new VarBase( + string::Sprintf("%s@IGrad", origin_grad_var_base->Name()), + origin_grad_var_base->DataType(), origin_grad_var_base->Dims(), + place_, true, false); + outputs.emplace_back(tmp_grad_var_base); } } - // Run grad op - framework::RuntimeContext ctx(grad_input_vars_[k], tmp_grad_outputs[k]); - // No need to do compile time infer shape here. // grad_op_desc_->InferShape(*block_); // grad_op_desc->InferVarType(block_); std::unique_ptr opbase = framework::OpRegistry::CreateOp(*grad_op_desc); + + auto& info = framework::OpInfoMap::Instance().Get(grad_op_desc->Type()); + if (info.infer_var_type_) { + RuntimeInferVarTypeContext infer_var_type_ctx( + &grad_input_vars_[k], &tmp_grad_outputs[k], &attrs_); + info.infer_var_type_(&infer_var_type_ctx); + } + framework::OperatorWithKernel* op_kernel = dynamic_cast(opbase.get()); PADDLE_ENFORCE_NOT_NULL(op_kernel, "only support op with kernel"); + // Run grad op + framework::VariableValueMap grad_invars_map; + framework::VariableValueMap grad_outvars_map; + + for (const auto& it : grad_input_vars_[k]) { + auto& grad_invars = grad_invars_map[it.first]; + grad_invars.reserve(it.second.size()); + for (const VarBase* grad_inp : it.second) { + PADDLE_ENFORCE_NOT_NULL(grad_inp->var_, "op %s input %s nullptr", + grad_op_desc->Type(), grad_inp->Name()); + + grad_invars.emplace_back(grad_inp->var_); + } + } + + for (const auto& it : tmp_grad_outputs[k]) { + auto& grad_outvars = grad_outvars_map[it.first]; + grad_outvars.reserve(it.second.size()); + for (VarBase* grad_out : it.second) { + PADDLE_ENFORCE_NOT_NULL(grad_out->var_, "op %s output %s nullptr", + grad_op_desc->Type(), grad_out->Name()); + + grad_outvars.emplace_back(grad_out->var_); + } + } + + framework::RuntimeContext ctx(grad_invars_map, grad_outvars_map); framework::Scope scope; PreparedOp p = PreparedOp::Prepare(ctx, *op_kernel, place_); p.op.RuntimeInferShape(scope, place_, ctx); @@ -273,14 +307,14 @@ std::map> OpBase::ApplyGrad() { // Add tmp grad outputs to original grad vars for (size_t k = 0; k < grad_output_vars_.size(); ++k) { - for (auto it : grad_output_vars_[k]) { + for (const auto& it : grad_output_vars_[k]) { auto& outputs = tmp_grad_outputs[k][it.first]; - auto& origin_outputs = it.second; + const auto& origin_outputs = it.second; PADDLE_ENFORCE_EQ(outputs.size(), origin_outputs.size()); for (size_t i = 0; i < outputs.size(); ++i) { - framework::Variable* grad = outputs[i]; - framework::Variable* orig_grad = origin_outputs[i]; + framework::Variable* grad = outputs[i]->var_; + framework::Variable* orig_grad = origin_outputs[i]->var_; AddTo(grad, orig_grad, place_); delete grad; } @@ -328,28 +362,35 @@ void PyLayer::RegisterFunc(int func_id, const py::object& py_func) { int PyLayer::NumFuncs() { return py_funcs_.size(); } -std::vector PyLayer::Apply(int func_id, - const std::vector& inputs) { - std::vector invars; - for (const VarBase* in : inputs) { - invars.push_back(in->var_); - } +std::vector PyLayer::Apply( + int func_id, const std::vector& inputs) { PADDLE_ENFORCE(py_funcs_.find(func_id) != py_funcs_.end()); - return CallPythonFunc(py_funcs_[func_id], invars); + return CallPythonFunc(py_funcs_[func_id], inputs); } -std::vector PyLayer::ApplyGrad( - int func_id, const std::vector& inputs) { +std::vector PyLayer::ApplyGrad(int func_id, + const std::vector& inputs) { PADDLE_ENFORCE(py_funcs_.find(func_id) != py_funcs_.end()); - return CallPythonFunc(py_funcs_[func_id], inputs); + auto rets = CallPythonFunc(py_funcs_[func_id], inputs); + + std::vector outs; + outs.reserve(rets.size()); + for (size_t i = 0U; i != rets.size(); ++i) { + outs.emplace_back(new VarBase( + string::Sprintf("%s_out_%d", framework::GradVarName(PyLayer::kFwdOut), + i), + rets[i], nullptr, true)); + } + + return outs; } std::vector PyLayer::CallPythonFunc( - const py::object& callable, const std::vector& ins) { + const py::object& callable, const std::vector& ins) { py::gil_scoped_acquire guard; py::tuple in_args(ins.size()); for (size_t i = 0; i < ins.size(); ++i) { - const framework::LoDTensor& t = ins[i]->Get(); + const framework::LoDTensor& t = ins[i]->var_->Get(); in_args[i] = t.IsInitialized() ? py::cast(t) : py::cast(nullptr); } VLOG(3) << "pyfunc in " << py::len(in_args); @@ -359,6 +400,7 @@ std::vector PyLayer::CallPythonFunc( auto ret_tuple = py::cast(ret); size_t ret_num = py::len(ret_tuple); std::vector outs; + outs.reserve(ret_num); VLOG(3) << "pyfunc out " << ret_num; for (size_t i = 0; i < ret_num; ++i) { try { @@ -369,7 +411,7 @@ std::vector PyLayer::CallPythonFunc( auto* tensor = var->GetMutable(); tensor->ShareDataWith(*py_out_tensor); tensor->set_lod(py_out_tensor->lod()); - outs.push_back(var); + outs.emplace_back(var); } catch (py::cast_error&) { PADDLE_THROW("The %d-th output must be LoDTensor", i); } diff --git a/paddle/fluid/imperative/layer.h b/paddle/fluid/imperative/layer.h index 618a5b7a03..72c548d5e9 100644 --- a/paddle/fluid/imperative/layer.h +++ b/paddle/fluid/imperative/layer.h @@ -18,14 +18,16 @@ #include "paddle/fluid/framework/python_headers.h" // clang-format on -#include // NOLINT -#include // NOLINT -#include // NOLINT -#include // NOLINT +#include // NOLINT +#include // NOLINT +#include // NOLINT +#include // NOLINT +#include // NOLINT #include "paddle/fluid/framework/op_desc.h" #include "paddle/fluid/framework/operator.h" #include "paddle/fluid/framework/var_desc.h" +#include "paddle/fluid/framework/var_type_inference.h" #include "paddle/fluid/platform/enforce.h" #include "paddle/fluid/platform/device_context.h" #include "paddle/fluid/operators/math/math_function.h" @@ -135,13 +137,13 @@ class VarBase { persistable) {} private: + // TODO(minqiyang): need support SelectedRows VarBase(const std::string& name, framework::proto::VarType::Type dtype, const framework::DDim& shape, const platform::Place& place, framework::Variable* var, VarBase* grad, bool stop_gradient, bool persistable) : name_(name), - dtype_(dtype), - place_(place), + type_(framework::proto::VarType::LOD_TENSOR), var_(var), grads_(grad), stop_gradient_(stop_gradient), @@ -151,10 +153,12 @@ class VarBase { pre_op_out_idx_(-1) { if (!var_) { var_ = new framework::Variable(); - auto tensor = var_->GetMutable(); - tensor->Resize(shape); - tensor->mutable_data(place_, dtype_); } + auto tensor = var_->GetMutable(); + tensor->Resize(shape); + tensor->mutable_data(place, dtype); + VLOG(10) << "create varbase: " << name_ << " type: " << dtype + << " place: " << place; } public: @@ -184,7 +188,23 @@ class VarBase { } } - inline framework::proto::VarType::Type DType() const { return dtype_; } + inline framework::DDim Dims() const { + return var_->Get().dims(); + } + + // data type. e.g.. FP32 + inline void SetDataType(framework::proto::VarType::Type type) { + auto tensor = var_->GetMutable(); + tensor->mutable_data(tensor->place(), type); + } + inline framework::proto::VarType::Type DataType() const { + auto tensor = var_->Get(); + return tensor.type(); + } + + // tensor type. e.g.. LoDTensor + inline void SetType(framework::proto::VarType::Type type) { type_ = type; } + inline framework::proto::VarType::Type Type() const { return type_; } inline void SetStopGradient(bool stop_gradient) { stop_gradient_ = stop_gradient; @@ -238,7 +258,7 @@ class VarBase { } std::string name_; - framework::proto::VarType::Type dtype_; + framework::proto::VarType::Type type_; platform::Place place_; framework::Variable* var_; @@ -294,17 +314,23 @@ class PYBIND11_HIDDEN OpBase { void InvokeBackwardHooks(); - void TrackPreOp(const VarBase* inp_var, const std::string& inp_name) { - if (inp_var->PreOp() && !inp_var->IsStopGradient()) { - VLOG(3) << "add pre op " << inp_var->PreOp()->Type() << " in slot " - << inp_name; - pre_ops_[inp_name].push_back(inp_var->PreOp()); - pre_ops_out_idx_[inp_name].push_back(inp_var->PreOpOutIdx()); - } else { - VLOG(3) << "no pre op in slot " << inp_name - << " input var stop_gradient: " << inp_var->IsStopGradient(); - pre_ops_[inp_name].push_back(nullptr); - // pre_ops_out_idx_[inp_name].push_back(-1); + void TrackPreOp(const std::string& inp_name, + const std::vector& inputs) { + auto& pre_ops_list = pre_ops_[inp_name]; + pre_ops_list.reserve(inputs.size()); + auto& pre_ops_out_idx_list = pre_ops_out_idx_[inp_name]; + for (VarBase* inp_var : inputs) { + if (inp_var->PreOp() && !inp_var->IsStopGradient()) { + VLOG(3) << "add pre op " << inp_var->PreOp()->Type() << " in slot " + << inp_name; + pre_ops_list.emplace_back(inp_var->PreOp()); + pre_ops_out_idx_list.push_back(inp_var->PreOpOutIdx()); + } else { + VLOG(3) << "no pre op in slot " << inp_name + << " input var stop_gradient: " << inp_var->IsStopGradient(); + pre_ops_list.emplace_back(nullptr); + // pre_ops_out_idx_list.push_back(-1); + } } } @@ -328,11 +354,13 @@ class PYBIND11_HIDDEN OpBase { std::map> pre_ops_out_idx_; // Inputs to a vector of bwd ops. - std::vector grad_input_vars_; + std::vector grad_input_vars_; // Outputs to a vector of bwd ops. - std::vector grad_output_vars_; + std::vector grad_output_vars_; std::vector backward_hooks_; + + framework::AttributeMap attrs_; }; class Layer { @@ -359,12 +387,131 @@ class PyLayer { static std::vector Apply( int func_id, const std::vector& inputs); - static std::vector ApplyGrad( - int func_id, const std::vector& inputs); + static std::vector ApplyGrad(int func_id, + const std::vector& inputs); private: static std::vector CallPythonFunc( - const py::object& callable, const std::vector& ins); + const py::object& callable, const std::vector& ins); +}; + +// infer var type context for imperative mode +class PYBIND11_HIDDEN RuntimeInferVarTypeContext + : public framework::InferVarTypeContext { + public: + RuntimeInferVarTypeContext(const imperative::VarBasePtrMap* inputs, + imperative::VarBasePtrMap* outputs, + const framework::AttributeMap* attrs_map) + : InferVarTypeContext(nullptr, nullptr), + inputs_(inputs), + outputs_(outputs), + attrs_(attrs_map), + input_names_(), + output_names_(), + var_set_() { + input_names_.reserve(inputs_->size()); + for (auto& it : *inputs_) { + for (imperative::VarBase* var : it.second) { + input_names_[it.first].emplace_back(var->Name()); + var_set_[var->Name()] = var; + } + } + + output_names_.reserve(outputs_->size()); + for (auto& it : *outputs_) { + for (imperative::VarBase* var : it.second) { + output_names_[it.first].emplace_back(var->Name()); + var_set_[var->Name()] = var; + } + } + } + + virtual ~RuntimeInferVarTypeContext() {} + + framework::Attribute GetAttr(const std::string& name) const override { + PADDLE_ENFORCE_NOT_NULL(attrs_); + return attrs_->at(name); + } + + bool HasVar(const std::string& name) const override { + return var_set_.count(name) > 0; + } + + bool HasInput(const std::string& name) const override { + PADDLE_ENFORCE_NOT_NULL(inputs_); + return inputs_->count(name) > 0; + } + + bool HasOutput(const std::string& name) const override { + PADDLE_ENFORCE_NOT_NULL(outputs_); + return outputs_->count(name) > 0; + } + + const std::vector& Input( + const std::string& name) const override { + return input_names_.at(name); + } + + const std::vector& Output( + const std::string& name) const override { + return output_names_.at(name); + } + + framework::proto::VarType::Type GetType( + const std::string& name) const override { + return var_set_.at(name)->Type(); + } + + void SetType(const std::string& name, + framework::proto::VarType::Type type) override { + var_set_[name]->SetType(type); + } + + framework::proto::VarType::Type GetDataType( + const std::string& name) const override { + return var_set_.at(name)->DataType(); + } + + void SetDataType(const std::string& name, + framework::proto::VarType::Type type) override { + var_set_[name]->SetDataType(type); + } + + std::vector GetDataTypes( + const std::string& name) const override { + PADDLE_THROW("GetDataTypes is not supported in runtime InferVarType"); + } + + void SetDataTypes(const std::string& name, + const std::vector& + multiple_data_type) override { + PADDLE_THROW("SetDataTypes is not supported in runtime InferVarType"); + } + + std::vector GetShape(const std::string& name) const override { + PADDLE_THROW("Do not handle Shape in runtime InferVarType"); + } + + void SetShape(const std::string& name, + const std::vector& dims) override { + PADDLE_THROW("Do not handle Shape in runtime InferVarType"); + } + + int32_t GetLoDLevel(const std::string& name) const override { + PADDLE_THROW("Do not handle LoDLevel in runtime InferVarType"); + } + + void SetLoDLevel(const std::string& name, int32_t lod_level) override { + PADDLE_THROW("Do not handle LoDLevel in runtime InferVarType"); + } + + private: + const imperative::VarBasePtrMap* inputs_; + imperative::VarBasePtrMap* outputs_; + const framework::AttributeMap* attrs_; + std::unordered_map> input_names_; + std::unordered_map> output_names_; + std::unordered_map var_set_; }; } // namespace imperative diff --git a/paddle/fluid/imperative/profiler.cc b/paddle/fluid/imperative/profiler.cc new file mode 100644 index 0000000000..34570b3a60 --- /dev/null +++ b/paddle/fluid/imperative/profiler.cc @@ -0,0 +1,62 @@ +// 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/imperative/profiler.h" + +#ifdef WITH_GPERFTOOLS +#include "gperftools/profiler.h" +#endif +#include +#include +#include // NOLINT +#include // NOLINT + +DEFINE_string( + tracer_profile_fname, "xxgperf", + "Profiler filename for imperative tracer, which generated by gperftools." + "Only valid when compiled `WITH_PROFILER=ON`. Empty if disable."); + +namespace paddle { +namespace imperative { + +static std::once_flag gTracerProfileOnce; +#ifdef WITH_GPERFTOOLS +static bool gTracerProfilerStarted = false; +#endif + +void StartProfile() { + if (!FLAGS_tracer_profile_fname.empty()) { + std::call_once(gTracerProfileOnce, [] { +#ifdef WITH_GPERFTOOLS + ProfilerStart(FLAGS_tracer_profile_fname.c_str()); + gTracerProfilerStarted = true; +#else + LOG(WARNING) << "Paddle is not compiled with gperftools. " + "FLAGS_tracer_profile_fname will be ignored"; +#endif + }); + } +} + +void StopProfile() { +#ifdef WITH_GPERFTOOLS + ProfilerFlush(); +#else + LOG(WARNING) << "Paddle is not compiled with gperftools. " + "FLAGS_tracer_profile_fname will be ignored"; +#endif +} + +} // namespace imperative +} // namespace paddle diff --git a/paddle/fluid/imperative/profiler.h b/paddle/fluid/imperative/profiler.h new file mode 100644 index 0000000000..d52aeed4e8 --- /dev/null +++ b/paddle/fluid/imperative/profiler.h @@ -0,0 +1,25 @@ +// 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 + +namespace paddle { +namespace imperative { + +extern void StartProfile(); + +extern void StopProfile(); + +} // namespace imperative +} // namespace paddle diff --git a/paddle/fluid/imperative/tracer.cc b/paddle/fluid/imperative/tracer.cc index 7ee92b4d8c..0cfdea030e 100644 --- a/paddle/fluid/imperative/tracer.cc +++ b/paddle/fluid/imperative/tracer.cc @@ -19,38 +19,26 @@ #include #include +#include "paddle/fluid/framework/var_type_inference.h" #include "paddle/fluid/operators/math/math_function.h" #include "paddle/fluid/platform/device_context.h" #include "paddle/fluid/platform/enforce.h" -#ifdef WITH_GPERFTOOLS -#include "gperftools/profiler.h" -#endif - -DEFINE_string( - tracer_profile_fname, "", - "Profiler filename for imperative tracer, which generated by gperftools." - "Only valid when compiled `WITH_PROFILER=ON`. Empty if disable."); - namespace paddle { namespace imperative { -static std::once_flag gTracerProfileOnce; -#ifdef WITH_GPERFTOOLS -static bool gTracerProfilerStarted = false; -#endif - void CreateGradOp(const framework::OpDesc& op_desc, const std::unordered_set& no_grad_set, const std::vector& grad_sub_block, std::vector* grad_op_descs, std::unordered_map* grad_to_var) { PADDLE_ENFORCE(grad_op_descs->empty()); - std::vector> descs = - framework::OpInfoMap::Instance() - .Get(op_desc.Type()) - .GradOpMaker()(op_desc, no_grad_set, grad_to_var, grad_sub_block); + const framework::OpInfo& op_info = + framework::OpInfoMap::Instance().Get(op_desc.Type()); + if (!op_info.grad_op_maker_) return; + std::vector> descs = + op_info.GradOpMaker()(op_desc, no_grad_set, grad_to_var, grad_sub_block); for (auto& desc : descs) { grad_op_descs->emplace_back(desc.release()); } @@ -145,31 +133,13 @@ framework::VariableNameMap CreateOutputVarNameMap( return result; } -Tracer::Tracer(framework::BlockDesc* root_block) : root_block_(root_block) { - if (!FLAGS_tracer_profile_fname.empty()) { - std::call_once(gTracerProfileOnce, [] { -#ifdef WITH_GPERFTOOLS - ProfilerStart(FLAGS_tracer_profile_fname.c_str()); - gTracerProfilerStarted = true; -#else - LOG(WARNING) << "Paddle is not compiled with gperftools. " - "FLAGS_tracer_profile_fname will be ignored"; -#endif - }); - } -} +Tracer::Tracer(framework::BlockDesc* root_block) : root_block_(root_block) {} std::set Tracer::Trace(OpBase* op, const VarBasePtrMap& inputs, - const VarBasePtrMap& outputs, + VarBasePtrMap* outputs, framework::AttributeMap attrs_map, const platform::Place expected_place, const bool stop_gradient) { -#ifdef WITH_GPERFTOOLS - if (gTracerProfilerStarted) { - ProfilerFlush(); - } -#endif - framework::VariableValueMap invars_map; framework::VariableValueMap outvars_map; @@ -184,7 +154,6 @@ std::set Tracer::Trace(OpBase* op, const VarBasePtrMap& inputs, inp->Name()); invars.emplace_back(inp->var_); - op->TrackPreOp(inp, it.first); if (!stop_gradient) { current_vars_map[inp->Name()] = inp; } @@ -192,9 +161,10 @@ std::set Tracer::Trace(OpBase* op, const VarBasePtrMap& inputs, << " inited: " << inp->var_->IsInitialized() << " stop_grad: " << inp->IsStopGradient(); } + op->TrackPreOp(it.first, it.second); } - op->output_vars_ = outputs; + op->output_vars_ = *outputs; for (auto it : op->output_vars_) { auto& outvars = outvars_map[it.first]; const std::vector& outputs = it.second; @@ -217,7 +187,7 @@ std::set Tracer::Trace(OpBase* op, const VarBasePtrMap& inputs, framework::VariableNameMap invars_name_map = CreateInputVarNameMap(op, inputs); framework::VariableNameMap outvars_name_map = - CreateOutputVarNameMap(op, outputs); + CreateOutputVarNameMap(op, *outputs); auto& info = framework::OpInfoMap::Instance().Get(op->Type()); if (info.Checker() != nullptr) { @@ -228,6 +198,11 @@ std::set Tracer::Trace(OpBase* op, const VarBasePtrMap& inputs, framework::OpRegistry::CreateOp(op->Type(), invars_name_map, outvars_name_map, attrs_map); + if (info.infer_var_type_) { + RuntimeInferVarTypeContext infer_var_type_ctx(&inputs, outputs, &attrs_map); + info.infer_var_type_(&infer_var_type_ctx); + } + // TODO(minqiyang): Support infer var type in imperative mode // Run forward op VLOG(3) << "tracer running " << op->Type(); @@ -252,6 +227,7 @@ std::set Tracer::Trace(OpBase* op, const VarBasePtrMap& inputs, VLOG(5) << "start construct backward op"; // construct grad op descs + op->attrs_ = attrs_map; std::unique_ptr fwd_op_desc(new framework::OpDesc( op->Type(), invars_name_map, outvars_name_map, attrs_map)); std::unique_ptr> grad_to_var( @@ -278,12 +254,12 @@ std::set Tracer::Trace(OpBase* op, const VarBasePtrMap& inputs, auto fwd_var_it = current_vars_map.find(grad_invar); PADDLE_ENFORCE(fwd_var_it != current_vars_map.end()); // Forward inputs or outputs. - grad_in_vars.emplace_back(fwd_var_it->second->var_); + grad_in_vars.emplace_back(fwd_var_it->second); } else { VarBase* var = current_vars_map[var_it->second]; InitGrad(var, prepared_op.GetDeviceContext()); // Douts. - grad_in_vars.emplace_back(var->grads_->var_); + grad_in_vars.emplace_back(var->grads_); } vars_saved_for_backward.insert(it.first); @@ -300,7 +276,7 @@ std::set Tracer::Trace(OpBase* op, const VarBasePtrMap& inputs, op->Type()); VarBase* var = current_vars_map[var_it->second]; InitGrad(var, prepared_op.GetDeviceContext()); - grad_out_vars.push_back(var->grads_->var_); + grad_out_vars.push_back(var->grads_); } } } @@ -319,9 +295,7 @@ std::vector Tracer::PyTrace(OpBase* op, std::vector ret_vars = PyLayer::Apply(op->forward_id_, inputs); - for (VarBase* inp : inputs) { - op->TrackPreOp(inp, PyLayer::kFwdInp); - } + op->TrackPreOp(PyLayer::kFwdInp, inputs); std::vector& outputs = op->output_vars_[PyLayer::kFwdOut]; outputs.reserve(ret_vars.size()); @@ -342,23 +316,23 @@ std::vector Tracer::PyTrace(OpBase* op, auto& grad_output_vars = op->grad_output_vars_[0][framework::GradVarName(PyLayer::kFwdOut)]; - for (const VarBase* inp : inputs) { - grad_input_vars.push_back(inp->var_); + for (VarBase* inp : inputs) { + grad_input_vars.push_back(inp); } for (VarBase* out : outputs) { - grad_input_vars.push_back(out->var_); + grad_input_vars.push_back(out); } // TODO(minqiyang): Add GPU support for PyLayer, only support CPU now platform::CPUPlace place; for (VarBase* out : outputs) { InitGrad(out, platform::DeviceContextPool::Instance().Get(place)); - grad_input_vars.push_back(out->grads_->var_); + grad_input_vars.push_back(out->grads_); } for (VarBase* inp : inputs) { InitGrad(inp, platform::DeviceContextPool::Instance().Get(place)); - grad_output_vars.push_back(inp->grads_->var_); + grad_output_vars.push_back(inp->grads_); } } return outputs; diff --git a/paddle/fluid/imperative/tracer.h b/paddle/fluid/imperative/tracer.h index 7b65d55e9e..a87f3b8009 100644 --- a/paddle/fluid/imperative/tracer.h +++ b/paddle/fluid/imperative/tracer.h @@ -48,7 +48,7 @@ class Tracer { virtual ~Tracer() {} std::set Trace(OpBase* op, const VarBasePtrMap& inputs, - const VarBasePtrMap& outputs, + VarBasePtrMap* outputs, // NOLINT framework::AttributeMap attrs_map, const platform::Place expected_place, const bool stop_gradient = false); diff --git a/paddle/fluid/imperative/type_defs.h b/paddle/fluid/imperative/type_defs.h index fc9e42f8d0..c51ce931de 100644 --- a/paddle/fluid/imperative/type_defs.h +++ b/paddle/fluid/imperative/type_defs.h @@ -25,6 +25,7 @@ class VarBase; class OpBase; typedef std::map> VarBasePtrMap; +typedef std::map> ConstVarBasePtrMap; typedef std::map> OpBasePtrMap; } // namespace imperative diff --git a/paddle/fluid/inference/CMakeLists.txt b/paddle/fluid/inference/CMakeLists.txt index 762640d6d1..d27ef8fe3c 100644 --- a/paddle/fluid/inference/CMakeLists.txt +++ b/paddle/fluid/inference/CMakeLists.txt @@ -91,5 +91,5 @@ if(WITH_TESTING) add_subdirectory(tests/book) if(WITH_INFERENCE_API_TEST) add_subdirectory(tests/api) - endif() + endif() endif() diff --git a/paddle/fluid/inference/analysis/argument.h b/paddle/fluid/inference/analysis/argument.h index 89e934ae27..321deccf86 100644 --- a/paddle/fluid/inference/analysis/argument.h +++ b/paddle/fluid/inference/analysis/argument.h @@ -27,6 +27,7 @@ #include #include #include +#include #include #include "paddle/fluid/framework/ir/graph.h" @@ -38,7 +39,10 @@ namespace paddle { namespace inference { namespace analysis { + using framework::ir::Graph; +using VarQuantScale = + std::unordered_map>; /* * The argument definition of both Pass and PassManagers. @@ -127,6 +131,8 @@ struct Argument { // Pass a set of op types to enable its mkldnn kernel DECL_ARGUMENT_FIELD(mkldnn_enabled_op_types, MKLDNNEnabledOpTypes, std::unordered_set); + // Scales for variables to be quantized + DECL_ARGUMENT_FIELD(quant_var_scales, QuantVarScales, VarQuantScale); // Passed from config. DECL_ARGUMENT_FIELD(use_gpu, UseGPU, bool); diff --git a/paddle/fluid/inference/analysis/ir_pass_manager.cc b/paddle/fluid/inference/analysis/ir_pass_manager.cc index 1cdb4881fb..8fd86b2cc5 100644 --- a/paddle/fluid/inference/analysis/ir_pass_manager.cc +++ b/paddle/fluid/inference/analysis/ir_pass_manager.cc @@ -14,6 +14,7 @@ #include "paddle/fluid/inference/analysis/ir_pass_manager.h" #include +#include #include #include "paddle/fluid/framework/ir/fuse_pass_base.h" #include "paddle/fluid/framework/ir/graph.h" @@ -55,14 +56,14 @@ void IRPassManager::CreatePasses(Argument *argument, ".dot"; pass->Set("graph_viz_path", new std::string(std::move(dot_file_path))); pass_num++; - } - if (pass_name == "mkldnn_placement_pass") { + } else if (pass_name == "mkldnn_placement_pass") { pass->Set("mkldnn_enabled_op_types", new std::unordered_set( argument->mkldnn_enabled_op_types())); - } - - if (pass_name == "tensorrt_subgraph_pass") { + } else if (pass_name == "cpu_quantize_pass") { + pass->Set("quant_var_scales", + new VarQuantScale(argument->quant_var_scales())); + } else if (pass_name == "tensorrt_subgraph_pass") { pass->Set("workspace_size", new int(argument->tensorrt_workspace_size())); pass->Set("max_batch_size", new int(argument->tensorrt_max_batch_size())); pass->Set("min_subgraph_size", diff --git a/paddle/fluid/inference/api/analysis_config.cc b/paddle/fluid/inference/api/analysis_config.cc index 7741111222..4cad8a9dfc 100644 --- a/paddle/fluid/inference/api/analysis_config.cc +++ b/paddle/fluid/inference/api/analysis_config.cc @@ -118,6 +118,9 @@ AnalysisConfig::AnalysisConfig(const AnalysisConfig &other) { CP_MEMBER(serialized_info_cache_); + // framework related. + CP_MEMBER(enable_runtime_context_cache_); + if (use_gpu_) { pass_builder_.reset(new GpuPassStrategy( *static_cast(other.pass_builder()))); @@ -219,12 +222,23 @@ void AnalysisConfig::Update() { } if (enable_memory_optim_) { - pass_builder()->AppendAnalysisPass("memory_optimize_pass"); + auto analysis_passes = pass_builder()->AnalysisPasses(); + auto memory_opti_pass_name = "memory_optimize_pass"; + bool already_exists = + std::find(analysis_passes.begin(), analysis_passes.end(), + memory_opti_pass_name) != analysis_passes.end(); + if (!already_exists) { + pass_builder()->AppendAnalysisPass(memory_opti_pass_name); + } } if (ir_debug_) { pass_builder()->TurnOnDebug(); } + + if (enable_runtime_context_cache_) { + pass_builder()->AppendPass("runtime_context_cache_pass"); + } } std::string AnalysisConfig::SerializeInfoCache() { @@ -258,6 +272,7 @@ std::string AnalysisConfig::SerializeInfoCache() { ss << specify_input_name_; ss << cpu_math_library_num_threads_; + ss << enable_runtime_context_cache_; return ss.str(); } diff --git a/paddle/fluid/inference/api/paddle_analysis_config.h b/paddle/fluid/inference/api/paddle_analysis_config.h index 9b05c33504..3b7faa5400 100644 --- a/paddle/fluid/inference/api/paddle_analysis_config.h +++ b/paddle/fluid/inference/api/paddle_analysis_config.h @@ -194,6 +194,23 @@ struct AnalysisConfig { /** Tell whether the memory optimization is activated. */ bool enable_memory_optim() const; + // framework related + /** \brief Control whether to perform runtime context cache optimization. + * + * If turned off, in Op's every execution, RuntimeContext would be called to + * relate input/output names of this Op with the corresponding variables in + * Scope. + */ + void SwitchRuntimeContextCache(int x = true) { + enable_runtime_context_cache_ = x; + } + /** A boolean state tell whether the runtime context cache optimization is + * actived. + */ + bool runtime_context_cache_enabled() const { + return enable_runtime_context_cache_; + } + friend class ::paddle::AnalysisPredictor; /** NOTE just for developer, not an official API, easily to be broken. @@ -254,6 +271,15 @@ struct AnalysisConfig { int cpu_math_library_num_threads_{1}; + // framework related + // RuntimeContext is used to relate input/output names of Operator with + // the corresponding variables in Scope. + // If enable_runtime_context_cache_ is true, it means that in a same Scope, + // since the input/output names of this Op do not change in the execution, + // RuntimeContext could be created only at the first iteration of this Op's + // execution to save the elapsed time. + bool enable_runtime_context_cache_{false}; + // A runtime cache, shouldn't be transferred to others. std::string serialized_info_cache_; diff --git a/paddle/fluid/inference/tests/api/CMakeLists.txt b/paddle/fluid/inference/tests/api/CMakeLists.txt index 8f7b6f31de..d9ac73b063 100644 --- a/paddle/fluid/inference/tests/api/CMakeLists.txt +++ b/paddle/fluid/inference/tests/api/CMakeLists.txt @@ -110,7 +110,7 @@ set(TRANSFORMER_INSTALL_DIR "${INFERENCE_DEMO_INSTALL_DIR}/transformer") download_model_and_data(${TRANSFORMER_INSTALL_DIR} "temp%2Ftransformer_model.tar.gz" "temp%2Ftransformer_data.txt.tar.gz") inference_analysis_test(test_analyzer_transformer SRCS analyzer_transformer_tester.cc EXTRA_DEPS ${INFERENCE_EXTRA_DEPS} - ARGS --infer_model=${TRANSFORMER_INSTALL_DIR}/model --infer_data=${TRANSFORMER_INSTALL_DIR}/data.txt --batch_size=8) + ARGS --infer_model=${TRANSFORMER_INSTALL_DIR}/model --infer_data=${TRANSFORMER_INSTALL_DIR}/data.txt --batch_size=8 SERIAL) # ocr set(OCR_INSTALL_DIR "${INFERENCE_DEMO_INSTALL_DIR}/ocr") diff --git a/paddle/fluid/inference/tests/api/analyzer_pyramid_dnn_tester.cc b/paddle/fluid/inference/tests/api/analyzer_pyramid_dnn_tester.cc index 5157bd280d..e1787a7177 100644 --- a/paddle/fluid/inference/tests/api/analyzer_pyramid_dnn_tester.cc +++ b/paddle/fluid/inference/tests/api/analyzer_pyramid_dnn_tester.cc @@ -107,6 +107,7 @@ void SetConfig(AnalysisConfig *cfg) { cfg->DisableGpu(); cfg->SwitchSpecifyInputNames(); cfg->SwitchIrOptim(); + cfg->SwitchRuntimeContextCache(); if (FLAGS_zero_copy) { cfg->SwitchUseFeedFetchOps(false); } diff --git a/paddle/fluid/inference/tests/api/analyzer_transformer_tester.cc b/paddle/fluid/inference/tests/api/analyzer_transformer_tester.cc index 9d17f38ab7..f765f55611 100644 --- a/paddle/fluid/inference/tests/api/analyzer_transformer_tester.cc +++ b/paddle/fluid/inference/tests/api/analyzer_transformer_tester.cc @@ -183,10 +183,13 @@ void SetInput(std::vector> *inputs) { } // Easy for profiling independently. -TEST(Analyzer_Transformer, profile) { +void profile(bool use_mkldnn = false) { AnalysisConfig cfg; SetConfig(&cfg); std::vector outputs; + if (use_mkldnn) { + cfg.EnableMKLDNN(); + } std::vector> input_slots_all; SetInput(&input_slots_all); @@ -194,6 +197,11 @@ TEST(Analyzer_Transformer, profile) { input_slots_all, &outputs, FLAGS_num_threads); } +TEST(Analyzer_Transformer, profile) { profile(); } +#ifdef PADDLE_WITH_MKLDNN +TEST(Analyzer_Transformer, profile_mkldnn) { profile(true); } +#endif + // Check the fuse status TEST(Analyzer_Transformer, fuse_statis) { AnalysisConfig cfg; @@ -206,9 +214,12 @@ TEST(Analyzer_Transformer, fuse_statis) { } // Compare result of NativeConfig and AnalysisConfig -TEST(Analyzer_Transformer, compare) { +void compare(bool use_mkldnn = false) { AnalysisConfig cfg; SetConfig(&cfg); + if (use_mkldnn) { + cfg.EnableMKLDNN(); + } std::vector> input_slots_all; SetInput(&input_slots_all); @@ -216,5 +227,10 @@ TEST(Analyzer_Transformer, compare) { reinterpret_cast(&cfg), input_slots_all); } +TEST(Analyzer_Transformer, compare) { compare(); } +#ifdef PADDLE_WITH_MKLDNN +TEST(Analyzer_Transformer, compare_mkldnn) { compare(true /* use_mkldnn */); } +#endif + } // namespace inference } // namespace paddle diff --git a/paddle/fluid/inference/tests/api/config_printer.h b/paddle/fluid/inference/tests/api/config_printer.h index b0c23fbd53..b7b39d4dd4 100644 --- a/paddle/fluid/inference/tests/api/config_printer.h +++ b/paddle/fluid/inference/tests/api/config_printer.h @@ -72,7 +72,8 @@ std::ostream &operator<<(std::ostream &os, const AnalysisConfig &config) { } os << GenSpaces(num_spaces) << "enable_ir_optim: " << config.ir_optim() << "\n"; - os << GenSpaces(num_spaces) << "enable_ir_optim: " << config.ir_optim() + os << GenSpaces(num_spaces) + << "use_runtime_context_cache: " << config.runtime_context_cache_enabled() << "\n"; os << GenSpaces(num_spaces) << "use_feed_fetch_ops: " << config.use_feed_fetch_ops_enabled() << "\n"; diff --git a/paddle/fluid/memory/allocation/allocator_facade_test.cc b/paddle/fluid/memory/allocation/allocator_facade_test.cc index 802d79e15d..2df1486c91 100644 --- a/paddle/fluid/memory/allocation/allocator_facade_test.cc +++ b/paddle/fluid/memory/allocation/allocator_facade_test.cc @@ -19,6 +19,8 @@ #ifdef PADDLE_WITH_CUDA DECLARE_double(fraction_of_gpu_memory_to_use); DECLARE_double(fraction_of_cuda_pinned_memory_to_use); +DECLARE_uint64(gpu_init_memory_in_mb); +DECLARE_uint64(gpu_reallocate_memory_in_mb); DECLARE_int64(gpu_allocator_retry_time); #endif @@ -26,13 +28,8 @@ namespace paddle { namespace memory { namespace allocation { -TEST(allocator, allocator) { -#ifdef PADDLE_WITH_CUDA - FLAGS_fraction_of_gpu_memory_to_use = 0.01; - FLAGS_gpu_allocator_retry_time = 500; - FLAGS_fraction_of_cuda_pinned_memory_to_use = 0.5; -#endif - +//! Run allocate test cases for different places +void AllocateTestCases() { auto &instance = AllocatorFacade::Instance(); platform::Place place; size_t size = 1024; @@ -82,6 +79,32 @@ TEST(allocator, allocator) { #endif } +TEST(allocator, allocator) { +#ifdef PADDLE_WITH_CUDA + FLAGS_fraction_of_gpu_memory_to_use = 0.01; + FLAGS_gpu_allocator_retry_time = 500; + FLAGS_fraction_of_cuda_pinned_memory_to_use = 0.5; +#endif + + AllocateTestCases(); +} + +TEST(allocator, specify_gpu_memory) { +#ifdef PADDLE_WITH_CUDA + // Set to 0.0 to test FLAGS_gpu_init_memory_in_mb and + // FLAGS_gpu_reallocate_memory_in_mb + FLAGS_fraction_of_gpu_memory_to_use = 0.0; + // 512 MB + FLAGS_gpu_init_memory_in_mb = 512; + // 4 MB + FLAGS_gpu_reallocate_memory_in_mb = 4; + FLAGS_gpu_allocator_retry_time = 500; + FLAGS_fraction_of_cuda_pinned_memory_to_use = 0.5; +#endif + + AllocateTestCases(); +} + } // namespace allocation } // namespace memory } // namespace paddle diff --git a/paddle/fluid/memory/allocation/legacy_allocator.cc b/paddle/fluid/memory/allocation/legacy_allocator.cc index c233bf4edf..5f30b2221d 100644 --- a/paddle/fluid/memory/allocation/legacy_allocator.cc +++ b/paddle/fluid/memory/allocation/legacy_allocator.cc @@ -37,6 +37,8 @@ DEFINE_bool(init_allocated_mem, false, "that initializing the allocated memory with a small value " "during unit testing."); DECLARE_double(fraction_of_gpu_memory_to_use); +DECLARE_uint64(gpu_init_memory_in_mb); +DECLARE_uint64(gpu_reallocate_memory_in_mb); DECLARE_bool(benchmark); namespace paddle { @@ -153,12 +155,18 @@ BuddyAllocator *GetGPUBuddyAllocator(int gpu_id) { platform::GpuMinChunkSize(), platform::GpuMaxChunkSize()); - VLOG(10) << "\n\nNOTE: each GPU device use " - << FLAGS_fraction_of_gpu_memory_to_use * 100 - << "% of GPU memory.\n" - << "You can set GFlags environment variable '" - << "FLAGS_fraction_of_gpu_memory_to_use" - << "' to change the fraction of GPU usage.\n\n"; + VLOG(10) << "\n\nNOTE:\n" + << "You can set GFlags environment variable " + << "'FLAGS_fraction_of_gpu_memory_to_use' " + << "or 'FLAGS_gpu_init_memory_in_mb' " + << "or 'FLAGS_gpu_reallocate_memory_in_mb' " + << "to change the memory size for GPU usage.\n" + << "Current 'FLAGS_fraction_of_gpu_memory_to_use' value is " + << FLAGS_fraction_of_gpu_memory_to_use + << ". Current 'FLAGS_gpu_init_memory_in_mb' value is " + << FLAGS_gpu_init_memory_in_mb + << ". Current 'FLAGS_gpu_reallocate_memory_in_mb' value is " + << FLAGS_gpu_reallocate_memory_in_mb << "\n\n"; } }); diff --git a/paddle/fluid/memory/detail/buddy_allocator.cc b/paddle/fluid/memory/detail/buddy_allocator.cc index 26ef27c3ca..2898185a34 100644 --- a/paddle/fluid/memory/detail/buddy_allocator.cc +++ b/paddle/fluid/memory/detail/buddy_allocator.cc @@ -13,6 +13,9 @@ See the License for the specific language governing permissions and limitations under the License. */ #include "paddle/fluid/memory/detail/buddy_allocator.h" + +#include // for std::move + #include "glog/logging.h" DEFINE_bool(free_idle_memory, false, @@ -185,18 +188,27 @@ void* BuddyAllocator::SystemAlloc(size_t size) { } BuddyAllocator::PoolSet::iterator BuddyAllocator::RefillPool() { + size_t allocate_bytes = max_chunk_size_; + size_t index = 0; + #ifdef PADDLE_WITH_CUDA if (system_allocator_->UseGpu()) { if ((total_used_ + total_free_) == 0) { - // Compute the maximum allocation size for the first allocation. + // Compute the allocation size for gpu for the first allocation. max_chunk_size_ = platform::GpuMaxChunkSize(); + allocate_bytes = platform::GpuInitAllocSize(); + } else { + // Reallocation size + if (realloc_size_ == 0) { + realloc_size_ = platform::GpuReallocSize(); + } + allocate_bytes = realloc_size_; } } #endif - // Allocate a new maximum sized block - size_t index = 0; - void* p = system_allocator_->Alloc(&index, max_chunk_size_); + // Allocate a new block + void* p = system_allocator_->Alloc(&index, allocate_bytes); if (p == nullptr) return pool_.end(); @@ -204,7 +216,7 @@ BuddyAllocator::PoolSet::iterator BuddyAllocator::RefillPool() { << " from system allocator"; static_cast(p)->init(&cache_, MemoryBlock::FREE_CHUNK, index, - max_chunk_size_, nullptr, nullptr); + allocate_bytes, nullptr, nullptr); // gpu fallback allocation if (system_allocator_->UseGpu() && @@ -212,10 +224,10 @@ BuddyAllocator::PoolSet::iterator BuddyAllocator::RefillPool() { fallback_alloc_count_++; } - total_free_ += max_chunk_size_; + total_free_ += allocate_bytes; // dump the block into pool - return pool_.insert(IndexSizeAddress(index, max_chunk_size_, p)).first; + return pool_.insert(IndexSizeAddress(index, allocate_bytes, p)).first; } BuddyAllocator::PoolSet::iterator BuddyAllocator::FindExistChunk(size_t size) { diff --git a/paddle/fluid/memory/detail/buddy_allocator.h b/paddle/fluid/memory/detail/buddy_allocator.h index 3f86a51f0d..7a785daba8 100644 --- a/paddle/fluid/memory/detail/buddy_allocator.h +++ b/paddle/fluid/memory/detail/buddy_allocator.h @@ -89,6 +89,8 @@ class BuddyAllocator { size_t min_chunk_size_; // the minimum size of each chunk size_t max_chunk_size_; // the maximum size of each chunk + size_t realloc_size_ = 0; // the size of re-allocated chunk + private: /** * \brief A list of free allocation diff --git a/paddle/fluid/memory/detail/system_allocator.cc b/paddle/fluid/memory/detail/system_allocator.cc index 197d1c2f21..70fa203df7 100644 --- a/paddle/fluid/memory/detail/system_allocator.cc +++ b/paddle/fluid/memory/detail/system_allocator.cc @@ -32,6 +32,9 @@ limitations under the License. */ DECLARE_bool(use_pinned_memory); DECLARE_double(fraction_of_gpu_memory_to_use); +DECLARE_uint64(gpu_init_memory_in_mb); +DECLARE_uint64(gpu_reallocate_memory_in_mb); + namespace paddle { namespace memory { namespace detail { @@ -119,11 +122,18 @@ void* GPUAllocator::Alloc(size_t* index, size_t size) { gpu_alloc_size_ += size; return p; } else { - LOG(WARNING) - << "Cannot malloc " << size / 1024.0 / 1024.0 - << " MB GPU memory. Please shrink FLAGS_fraction_of_gpu_memory_to_use " - "environment variable to a lower value. Current value is " - << FLAGS_fraction_of_gpu_memory_to_use; + LOG(WARNING) << "Cannot malloc " << size / 1024.0 / 1024.0 + << " MB GPU memory. Please shrink " + "FLAGS_fraction_of_gpu_memory_to_use or " + "FLAGS_gpu_init_memory_in_mb or " + "FLAGS_gpu_reallocate_memory_in_mb" + "environment variable to a lower value. " + << "Current FLAGS_fraction_of_gpu_memory_to_use value is " + << FLAGS_fraction_of_gpu_memory_to_use + << ". Current FLAGS_gpu_init_memory_in_mb value is " + << FLAGS_gpu_init_memory_in_mb + << ". Current FLAGS_gpu_reallocate_memory_in_mb value is " + << FLAGS_gpu_reallocate_memory_in_mb; return nullptr; } } diff --git a/paddle/fluid/operators/CMakeLists.txt b/paddle/fluid/operators/CMakeLists.txt index 2f8e0b3a30..651c5e6e75 100644 --- a/paddle/fluid/operators/CMakeLists.txt +++ b/paddle/fluid/operators/CMakeLists.txt @@ -58,8 +58,10 @@ if (WITH_GPU) op_library(conv_fusion_op) file(APPEND ${pybind_file} "USE_CUDA_ONLY_OP(conv2d_fusion);\n") endif() - op_library(sync_batch_norm_op) - file(APPEND ${pybind_file} "USE_CUDA_ONLY_OP(sync_batch_norm);\n") + if (NOT WIN32) + op_library(sync_batch_norm_op) + file(APPEND ${pybind_file} "USE_CUDA_ONLY_OP(sync_batch_norm);\n") + endif() else() op_library(warpctc_op DEPS dynload_warpctc sequence_padding sequence_scale) endif() diff --git a/paddle/fluid/operators/beam_search_decode_op.cc b/paddle/fluid/operators/beam_search_decode_op.cc index cf78c83297..4cef49280d 100644 --- a/paddle/fluid/operators/beam_search_decode_op.cc +++ b/paddle/fluid/operators/beam_search_decode_op.cc @@ -178,10 +178,10 @@ Beam Search Decode Operator. This Operator constructs the full hypotheses for each source sentence by walking back along the LoDTensorArray Input(ids) whose lods can be used to restore the path in the beam search tree. -The Output(SentenceIds) and Output(SentenceScores) separately contain the -generated id sequences and the corresponding scores. The shapes and lods of the -two LodTensor are same. The lod level is 2 and the two levels separately -indicate how many hypotheses each source sentence has and how many ids each +The Output(SentenceIds) and Output(SentenceScores) separately contain the +generated id sequences and the corresponding scores. The shapes and lods of the +two LodTensor are same. The lod level is 2 and the two levels separately +indicate how many hypotheses each source sentence has and how many ids each hypothesis has. )DOC"); } @@ -203,15 +203,12 @@ class BeamSearchDecodeInferShape : public framework::InferShapeBase { class BeamSearchDecodeInferVarType : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc& op_desc, - framework::BlockDesc* block) const override { - for (auto& o : op_desc.Output("SentenceIds")) { - auto& sentence_ids = block->FindRecursiveOrCreateVar(o); - sentence_ids.SetType(framework::proto::VarType::LOD_TENSOR); + void operator()(framework::InferVarTypeContext* ctx) const override { + for (auto& o : ctx->Output("SentenceIds")) { + ctx->SetType(o, framework::proto::VarType::LOD_TENSOR); } - for (auto& o : op_desc.Output("SentenceScores")) { - auto& sentence_scores = block->FindRecursiveOrCreateVar(o); - sentence_scores.SetType(framework::proto::VarType::LOD_TENSOR); + for (auto& o : ctx->Output("SentenceScores")) { + ctx->SetType(o, framework::proto::VarType::LOD_TENSOR); } } }; diff --git a/paddle/fluid/operators/beam_search_op.cc b/paddle/fluid/operators/beam_search_op.cc index fa6b09b4e7..a6aa35e056 100644 --- a/paddle/fluid/operators/beam_search_op.cc +++ b/paddle/fluid/operators/beam_search_op.cc @@ -65,7 +65,7 @@ class BeamSearchOpMaker : public framework::OpProtoAndCheckerMaker { .SetDefault(true); AddComment(R"DOC( -This operator does the search in beams for one time step. +This operator does the search in beams for one time step. Specifically, it selects the top-K candidate word ids of current step from Input(ids) according to their Input(scores) for all source sentences, where K is Attr(beam_size) and Input(ids), Input(scores) are predicted results @@ -120,15 +120,12 @@ class BeamSearchOp : public framework::OperatorWithKernel { class BeamSearchInferVarType : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc &op_desc, - framework::BlockDesc *block) const override { - for (auto &o : op_desc.Output("selected_ids")) { - auto &selected_ids = block->FindRecursiveOrCreateVar(o); - selected_ids.SetType(framework::proto::VarType::LOD_TENSOR); + void operator()(framework::InferVarTypeContext *ctx) const override { + for (auto &o : ctx->Output("selected_ids")) { + ctx->SetType(o, framework::proto::VarType::LOD_TENSOR); } - for (auto &o : op_desc.Output("selected_scores")) { - auto &selected_scores = block->FindRecursiveOrCreateVar(o); - selected_scores.SetType(framework::proto::VarType::LOD_TENSOR); + for (auto &o : ctx->Output("selected_scores")) { + ctx->SetType(o, framework::proto::VarType::LOD_TENSOR); } } }; diff --git a/paddle/fluid/operators/concat_op.cc b/paddle/fluid/operators/concat_op.cc index 194f9cf503..5d5ad9e711 100644 --- a/paddle/fluid/operators/concat_op.cc +++ b/paddle/fluid/operators/concat_op.cc @@ -50,9 +50,19 @@ class ConcatOp : public framework::OperatorWithKernel { if (j == axis) { out_dims[axis] += ins[i][j]; } else { - PADDLE_ENFORCE_EQ(out_dims[j], ins[i][j], - "Input tensors should have the same " - "elements except the specify axis."); + if (ctx->IsRuntime()) { + // check all shape in run time + PADDLE_ENFORCE_EQ(out_dims[j], ins[i][j], + "Input tensors should have the same " + "elements except the specify axis."); + } else { + // not check -1 with other in compile time + if (out_dims[j] != -1 && ins[i][j] != -1) { + PADDLE_ENFORCE_EQ(out_dims[j], ins[i][j], + "Input tensors should have the same " + "elements except the specify axis."); + } + } } } } diff --git a/paddle/fluid/operators/controlflow/get_places_op.cc b/paddle/fluid/operators/controlflow/get_places_op.cc index 1a157688f3..fa77f97419 100644 --- a/paddle/fluid/operators/controlflow/get_places_op.cc +++ b/paddle/fluid/operators/controlflow/get_places_op.cc @@ -93,11 +93,9 @@ execution. class GetPlacesInferVarType : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc &op_desc, - framework::BlockDesc *block) const override { - for (auto &o_name : op_desc.Output("Out")) { - block->FindRecursiveOrCreateVar(o_name).SetType( - framework::proto::VarType::PLACE_LIST); + void operator()(framework::InferVarTypeContext *ctx) const override { + for (auto &o_name : ctx->Output("Out")) { + ctx->SetType(o_name, framework::proto::VarType::PLACE_LIST); } } }; diff --git a/paddle/fluid/operators/controlflow/tensor_array_read_write_op.cc b/paddle/fluid/operators/controlflow/tensor_array_read_write_op.cc index fa18ade323..45f18ac925 100644 --- a/paddle/fluid/operators/controlflow/tensor_array_read_write_op.cc +++ b/paddle/fluid/operators/controlflow/tensor_array_read_write_op.cc @@ -100,16 +100,13 @@ class WriteToArrayInferShape : public framework::InferShapeBase { class WriteToArrayInferVarType : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc &op_desc, - framework::BlockDesc *block) const override { - auto x_name = op_desc.Input("X")[0]; - auto out_name = op_desc.Output("Out")[0]; + void operator()(framework::InferVarTypeContext *ctx) const override { + auto x_name = ctx->Input("X")[0]; + auto out_name = ctx->Output("Out")[0]; VLOG(10) << "Set Variable " << out_name << " as LOD_TENSOR_ARRAY"; - auto &out = block->FindRecursiveOrCreateVar(out_name); - out.SetType(framework::proto::VarType::LOD_TENSOR_ARRAY); - auto *x = block->FindVarRecursive(x_name); - if (x != nullptr) { - out.SetDataType(x->GetDataType()); + ctx->SetType(out_name, framework::proto::VarType::LOD_TENSOR_ARRAY); + if (ctx->HasVar(x_name)) { + ctx->SetDataType(out_name, ctx->GetDataType(x_name)); } } }; diff --git a/paddle/fluid/operators/controlflow/while_op.cc b/paddle/fluid/operators/controlflow/while_op.cc index 8352ba4f2b..deb8ec3bb2 100644 --- a/paddle/fluid/operators/controlflow/while_op.cc +++ b/paddle/fluid/operators/controlflow/while_op.cc @@ -365,19 +365,16 @@ class WhileGradOpDescMaker : public framework::SingleGradOpDescMaker { class WhileGradOpVarTypeInference : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc &op_desc, - framework::BlockDesc *block) const override { - auto p_names = op_desc.Input(kX); - auto pg_ig_names = op_desc.Output(framework::GradVarName(kX)); + void operator()(framework::InferVarTypeContext *ctx) const override { + auto p_names = ctx->Input(kX); + auto pg_ig_names = ctx->Output(framework::GradVarName(kX)); for (size_t i = 0; i < p_names.size(); ++i) { - auto &p_var = detail::Ref(block->FindVarRecursive(p_names[i])); - auto *g_var = block->FindVarRecursive(pg_ig_names[i]); - if (g_var != nullptr) { // Gradient could be @EMPTY@ + if (ctx->HasVar(pg_ig_names[i])) { VLOG(5) << "Setting " << pg_ig_names[i] << " following " << p_names[i] - << " type: " << p_var.GetType(); - g_var->SetType(p_var.GetType()); - g_var->SetDataType(p_var.GetDataType()); + << " type: " << ctx->GetType(p_names[i]); + ctx->SetType(pg_ig_names[i], ctx->GetType(p_names[i])); + ctx->SetDataType(pg_ig_names[i], ctx->GetDataType(p_names[i])); } } } diff --git a/paddle/fluid/operators/conv_op.cc b/paddle/fluid/operators/conv_op.cc index ca6bc4df0f..c6121d00da 100644 --- a/paddle/fluid/operators/conv_op.cc +++ b/paddle/fluid/operators/conv_op.cc @@ -14,6 +14,7 @@ limitations under the License. */ #include "paddle/fluid/operators/conv_op.h" +#include #include #include @@ -194,6 +195,12 @@ void Conv2DOpMaker::Make() { AddAttr("use_mkldnn", "(bool, default false) Only used in mkldnn kernel") .SetDefault(false); + AddAttr("use_quantizer", + "(bool, default false) " + "Set to true for operators that should be quantized and use " + "int8 kernel. " + "Only used on CPU.") + .SetDefault(false); AddAttr("fuse_relu", "(bool, default false) Only used in mkldnn kernel") .SetDefault(false); AddAttr("fuse_residual_connection", diff --git a/paddle/fluid/operators/detection/CMakeLists.txt b/paddle/fluid/operators/detection/CMakeLists.txt index c87837e694..94a2016aa5 100644 --- a/paddle/fluid/operators/detection/CMakeLists.txt +++ b/paddle/fluid/operators/detection/CMakeLists.txt @@ -33,6 +33,7 @@ detection_library(rpn_target_assign_op SRCS rpn_target_assign_op.cc) detection_library(generate_proposal_labels_op SRCS generate_proposal_labels_op.cc) detection_library(box_clip_op SRCS box_clip_op.cc box_clip_op.cu) detection_library(yolov3_loss_op SRCS yolov3_loss_op.cc) +detection_library(yolo_box_op SRCS yolo_box_op.cc yolo_box_op.cu) detection_library(box_decoder_and_assign_op SRCS box_decoder_and_assign_op.cc box_decoder_and_assign_op.cu) if(WITH_GPU) diff --git a/paddle/fluid/operators/detection/yolo_box_op.cc b/paddle/fluid/operators/detection/yolo_box_op.cc new file mode 100644 index 0000000000..e0d7e25d94 --- /dev/null +++ b/paddle/fluid/operators/detection/yolo_box_op.cc @@ -0,0 +1,167 @@ +/* Copyright (c) 2019 PaddlePaddle Authors. All Rights Reserve. + 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/detection/yolo_box_op.h" +#include "paddle/fluid/framework/op_registry.h" + +namespace paddle { +namespace operators { + +using framework::Tensor; + +class YoloBoxOp : public framework::OperatorWithKernel { + public: + using framework::OperatorWithKernel::OperatorWithKernel; + void InferShape(framework::InferShapeContext* ctx) const override { + PADDLE_ENFORCE(ctx->HasInput("X"), + "Input(X) of YoloBoxOp should not be null."); + PADDLE_ENFORCE(ctx->HasInput("ImgSize"), + "Input(ImgSize) of YoloBoxOp should not be null."); + PADDLE_ENFORCE(ctx->HasOutput("Boxes"), + "Output(Boxes) of YoloBoxOp should not be null."); + PADDLE_ENFORCE(ctx->HasOutput("Scores"), + "Output(Scores) of YoloBoxOp should not be null."); + + auto dim_x = ctx->GetInputDim("X"); + auto dim_imgsize = ctx->GetInputDim("ImgSize"); + auto anchors = ctx->Attrs().Get>("anchors"); + int anchor_num = anchors.size() / 2; + auto class_num = ctx->Attrs().Get("class_num"); + + PADDLE_ENFORCE_EQ(dim_x.size(), 4, "Input(X) should be a 4-D tensor."); + PADDLE_ENFORCE_EQ( + dim_x[1], anchor_num * (5 + class_num), + "Input(X) dim[1] should be equal to (anchor_mask_number * (5 " + "+ class_num))."); + PADDLE_ENFORCE_EQ(dim_imgsize.size(), 2, + "Input(ImgSize) should be a 2-D tensor."); + PADDLE_ENFORCE_EQ( + dim_imgsize[0], dim_x[0], + "Input(ImgSize) dim[0] and Input(X) dim[0] should be same."); + PADDLE_ENFORCE_EQ(dim_imgsize[1], 2, "Input(ImgSize) dim[1] should be 2."); + PADDLE_ENFORCE_GT(anchors.size(), 0, + "Attr(anchors) length should be greater than 0."); + PADDLE_ENFORCE_EQ(anchors.size() % 2, 0, + "Attr(anchors) length should be even integer."); + PADDLE_ENFORCE_GT(class_num, 0, + "Attr(class_num) should be an integer greater than 0."); + + int box_num = dim_x[2] * dim_x[3] * anchor_num; + std::vector dim_boxes({dim_x[0], box_num, 4}); + ctx->SetOutputDim("Boxes", framework::make_ddim(dim_boxes)); + + std::vector dim_scores({dim_x[0], box_num, class_num}); + ctx->SetOutputDim("Scores", framework::make_ddim(dim_scores)); + } + + protected: + framework::OpKernelType GetExpectedKernelType( + const framework::ExecutionContext& ctx) const override { + return framework::OpKernelType(ctx.Input("X")->type(), + ctx.GetPlace()); + } +}; + +class YoloBoxOpMaker : public framework::OpProtoAndCheckerMaker { + public: + void Make() override { + AddInput("X", + "The input tensor of YoloBox operator is a 4-D tensor with " + "shape of [N, C, H, W]. The second dimension(C) stores " + "box locations, confidence score and classification one-hot " + "keys of each anchor box. Generally, X should be the output " + "of YOLOv3 network."); + AddInput("ImgSize", + "The image size tensor of YoloBox operator, " + "This is a 2-D tensor with shape of [N, 2]. This tensor holds " + "height and width of each input image used for resizing output " + "box in input image scale."); + AddOutput("Boxes", + "The output tensor of detection boxes of YoloBox operator, " + "This is a 3-D tensor with shape of [N, M, 4], N is the " + "batch num, M is output box number, and the 3rd dimension " + "stores [xmin, ymin, xmax, ymax] coordinates of boxes."); + AddOutput("Scores", + "The output tensor of detection boxes scores of YoloBox " + "operator, This is a 3-D tensor with shape of " + "[N, M, :attr:`class_num`], N is the batch num, M is " + "output box number."); + + AddAttr("class_num", "The number of classes to predict."); + AddAttr>("anchors", + "The anchor width and height, " + "it will be parsed pair by pair.") + .SetDefault(std::vector{}); + AddAttr("downsample_ratio", + "The downsample ratio from network input to YoloBox operator " + "input, so 32, 16, 8 should be set for the first, second, " + "and thrid YoloBox operators.") + .SetDefault(32); + AddAttr("conf_thresh", + "The confidence scores threshold of detection boxes. " + "Boxes with confidence scores under threshold should " + "be ignored.") + .SetDefault(0.01); + AddComment(R"DOC( + This operator generates YOLO detection boxes from output of YOLOv3 network. + + The output of previous network is in shape [N, C, H, W], while H and W + should be the same, H and W specify the grid size, each grid point predict + given number boxes, this given number, which following will be represented as S, + is specified by the number of anchors. In the second dimension(the channel + dimension), C should be equal to S * (5 + class_num), class_num is the object + category number of source dataset(such as 80 in coco dataset), so the + second(channel) dimension, apart from 4 box location coordinates x, y, w, h, + also includes confidence score of the box and class one-hot key of each anchor + box. + + Assume the 4 location coordinates are :math:`t_x, t_y, t_w, t_h`, the box + predictions should be as follows: + + $$ + b_x = \\sigma(t_x) + c_x + $$ + $$ + b_y = \\sigma(t_y) + c_y + $$ + $$ + b_w = p_w e^{t_w} + $$ + $$ + b_h = p_h e^{t_h} + $$ + + in the equation above, :math:`c_x, c_y` is the left top corner of current grid + and :math:`p_w, p_h` is specified by anchors. + + The logistic regression value of the 5th channel of each anchor prediction boxes + represents the confidence score of each prediction box, and the logistic + regression value of the last :attr:`class_num` channels of each anchor prediction + boxes represents the classifcation scores. Boxes with confidence scores less than + :attr:`conf_thresh` should be ignored, and box final scores is the product of + confidence scores and classification scores. + + $$ + score_{pred} = score_{conf} * score_{class} + $$ + + )DOC"); + } +}; + +} // namespace operators +} // namespace paddle + +namespace ops = paddle::operators; +REGISTER_OPERATOR(yolo_box, ops::YoloBoxOp, ops::YoloBoxOpMaker, + paddle::framework::EmptyGradOpMaker); +REGISTER_OP_CPU_KERNEL(yolo_box, ops::YoloBoxKernel, + ops::YoloBoxKernel); diff --git a/paddle/fluid/operators/detection/yolo_box_op.cu b/paddle/fluid/operators/detection/yolo_box_op.cu new file mode 100644 index 0000000000..5a882958e6 --- /dev/null +++ b/paddle/fluid/operators/detection/yolo_box_op.cu @@ -0,0 +1,120 @@ +/* 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/operators/detection/yolo_box_op.h" +#include "paddle/fluid/operators/math/math_function.h" + +namespace paddle { +namespace operators { + +using Tensor = framework::Tensor; + +template +__global__ void KeYoloBoxFw(const T* input, const int* imgsize, T* boxes, + T* scores, const float conf_thresh, + const int* anchors, const int n, const int h, + const int w, const int an_num, const int class_num, + const int box_num, int input_size) { + int tid = blockIdx.x * blockDim.x + threadIdx.x; + int stride = blockDim.x * gridDim.x; + T box[4]; + for (; tid < n * box_num; tid += stride) { + int grid_num = h * w; + int i = tid / box_num; + int j = (tid % box_num) / grid_num; + int k = (tid % grid_num) / w; + int l = tid % w; + + int an_stride = (5 + class_num) * grid_num; + int img_height = imgsize[2 * i]; + int img_width = imgsize[2 * i + 1]; + + int obj_idx = + GetEntryIndex(i, j, k * w + l, an_num, an_stride, grid_num, 4); + T conf = sigmoid(input[obj_idx]); + if (conf < conf_thresh) { + continue; + } + + int box_idx = + GetEntryIndex(i, j, k * w + l, an_num, an_stride, grid_num, 0); + GetYoloBox(box, input, anchors, l, k, j, h, input_size, box_idx, + grid_num, img_height, img_width); + box_idx = (i * box_num + j * grid_num + k * w + l) * 4; + CalcDetectionBox(boxes, box, box_idx, img_height, img_width); + + int label_idx = + GetEntryIndex(i, j, k * w + l, an_num, an_stride, grid_num, 5); + int score_idx = (i * box_num + j * grid_num + k * w + l) * class_num; + CalcLabelScore(scores, input, label_idx, score_idx, class_num, conf, + grid_num); + } +} + +template +class YoloBoxOpCUDAKernel : public framework::OpKernel { + public: + void Compute(const framework::ExecutionContext& ctx) const override { + auto* input = ctx.Input("X"); + auto* img_size = ctx.Input("ImgSize"); + auto* boxes = ctx.Output("Boxes"); + auto* scores = ctx.Output("Scores"); + + auto anchors = ctx.Attr>("anchors"); + int class_num = ctx.Attr("class_num"); + float conf_thresh = ctx.Attr("conf_thresh"); + int downsample_ratio = ctx.Attr("downsample_ratio"); + + const int n = input->dims()[0]; + const int h = input->dims()[2]; + const int w = input->dims()[3]; + const int box_num = boxes->dims()[1]; + const int an_num = anchors.size() / 2; + int input_size = downsample_ratio * h; + + auto& dev_ctx = ctx.cuda_device_context(); + auto& allocator = + platform::DeviceTemporaryAllocator::Instance().Get(dev_ctx); + int bytes = sizeof(int) * anchors.size(); + auto anchors_ptr = allocator.Allocate(sizeof(int) * anchors.size()); + int* anchors_data = reinterpret_cast(anchors_ptr->ptr()); + const auto gplace = boost::get(ctx.GetPlace()); + const auto cplace = platform::CPUPlace(); + memory::Copy(gplace, anchors_data, cplace, anchors.data(), bytes, + dev_ctx.stream()); + + const T* input_data = input->data(); + const int* imgsize_data = img_size->data(); + T* boxes_data = boxes->mutable_data({n, box_num, 4}, ctx.GetPlace()); + T* scores_data = + scores->mutable_data({n, box_num, class_num}, ctx.GetPlace()); + math::SetConstant set_zero; + set_zero(dev_ctx, boxes, static_cast(0)); + set_zero(dev_ctx, scores, static_cast(0)); + + int grid_dim = (n * box_num + 512 - 1) / 512; + grid_dim = grid_dim > 8 ? 8 : grid_dim; + + KeYoloBoxFw<<>>( + input_data, imgsize_data, boxes_data, scores_data, conf_thresh, + anchors_data, n, h, w, an_num, class_num, box_num, input_size); + } +}; + +} // namespace operators +} // namespace paddle + +namespace ops = paddle::operators; +REGISTER_OP_CUDA_KERNEL(yolo_box, ops::YoloBoxOpCUDAKernel, + ops::YoloBoxOpCUDAKernel); diff --git a/paddle/fluid/operators/detection/yolo_box_op.h b/paddle/fluid/operators/detection/yolo_box_op.h new file mode 100644 index 0000000000..8b7c7df0f3 --- /dev/null +++ b/paddle/fluid/operators/detection/yolo_box_op.h @@ -0,0 +1,149 @@ +/* Copyright (c) 2019 PaddlePaddle Authors. All Rights Reserve. + 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/op_registry.h" +#include "paddle/fluid/platform/hostdevice.h" + +namespace paddle { +namespace operators { + +using Tensor = framework::Tensor; + +template +HOSTDEVICE inline T sigmoid(T x) { + return 1.0 / (1.0 + std::exp(-x)); +} + +template +HOSTDEVICE inline void GetYoloBox(T* box, const T* x, const int* anchors, int i, + int j, int an_idx, int grid_size, + int input_size, int index, int stride, + int img_height, int img_width) { + box[0] = (i + sigmoid(x[index])) * img_width / grid_size; + box[1] = (j + sigmoid(x[index + stride])) * img_height / grid_size; + box[2] = std::exp(x[index + 2 * stride]) * anchors[2 * an_idx] * img_width / + input_size; + box[3] = std::exp(x[index + 3 * stride]) * anchors[2 * an_idx + 1] * + img_height / input_size; +} + +HOSTDEVICE inline int GetEntryIndex(int batch, int an_idx, int hw_idx, + int an_num, int an_stride, int stride, + int entry) { + return (batch * an_num + an_idx) * an_stride + entry * stride + hw_idx; +} + +template +HOSTDEVICE inline void CalcDetectionBox(T* boxes, T* box, const int box_idx, + const int img_height, + const int img_width) { + boxes[box_idx] = box[0] - box[2] / 2; + boxes[box_idx + 1] = box[1] - box[3] / 2; + boxes[box_idx + 2] = box[0] + box[2] / 2; + boxes[box_idx + 3] = box[1] + box[3] / 2; + + boxes[box_idx] = boxes[box_idx] > 0 ? boxes[box_idx] : static_cast(0); + boxes[box_idx + 1] = + boxes[box_idx + 1] > 0 ? boxes[box_idx + 1] : static_cast(0); + boxes[box_idx + 2] = boxes[box_idx + 2] < img_width - 1 + ? boxes[box_idx + 2] + : static_cast(img_width - 1); + boxes[box_idx + 3] = boxes[box_idx + 3] < img_height - 1 + ? boxes[box_idx + 3] + : static_cast(img_height - 1); +} + +template +HOSTDEVICE inline void CalcLabelScore(T* scores, const T* input, + const int label_idx, const int score_idx, + const int class_num, const T conf, + const int stride) { + for (int i = 0; i < class_num; i++) { + scores[score_idx + i] = conf * sigmoid(input[label_idx + i * stride]); + } +} + +template +class YoloBoxKernel : public framework::OpKernel { + public: + void Compute(const framework::ExecutionContext& ctx) const override { + auto* input = ctx.Input("X"); + auto* imgsize = ctx.Input("ImgSize"); + auto* boxes = ctx.Output("Boxes"); + auto* scores = ctx.Output("Scores"); + auto anchors = ctx.Attr>("anchors"); + int class_num = ctx.Attr("class_num"); + float conf_thresh = ctx.Attr("conf_thresh"); + int downsample_ratio = ctx.Attr("downsample_ratio"); + + const int n = input->dims()[0]; + const int h = input->dims()[2]; + const int w = input->dims()[3]; + const int box_num = boxes->dims()[1]; + const int an_num = anchors.size() / 2; + int input_size = downsample_ratio * h; + + const int stride = h * w; + const int an_stride = (class_num + 5) * stride; + + Tensor anchors_; + auto anchors_data = + anchors_.mutable_data({an_num * 2}, ctx.GetPlace()); + std::copy(anchors.begin(), anchors.end(), anchors_data); + + const T* input_data = input->data(); + const int* imgsize_data = imgsize->data(); + T* boxes_data = boxes->mutable_data({n, box_num, 4}, ctx.GetPlace()); + memset(boxes_data, 0, boxes->numel() * sizeof(T)); + T* scores_data = + scores->mutable_data({n, box_num, class_num}, ctx.GetPlace()); + memset(scores_data, 0, scores->numel() * sizeof(T)); + + T box[4]; + for (int i = 0; i < n; i++) { + int img_height = imgsize_data[2 * i]; + int img_width = imgsize_data[2 * i + 1]; + + for (int j = 0; j < an_num; j++) { + for (int k = 0; k < h; k++) { + for (int l = 0; l < w; l++) { + int obj_idx = + GetEntryIndex(i, j, k * w + l, an_num, an_stride, stride, 4); + T conf = sigmoid(input_data[obj_idx]); + if (conf < conf_thresh) { + continue; + } + + int box_idx = + GetEntryIndex(i, j, k * w + l, an_num, an_stride, stride, 0); + GetYoloBox(box, input_data, anchors_data, l, k, j, h, input_size, + box_idx, stride, img_height, img_width); + box_idx = (i * box_num + j * stride + k * w + l) * 4; + CalcDetectionBox(boxes_data, box, box_idx, img_height, + img_width); + + int label_idx = + GetEntryIndex(i, j, k * w + l, an_num, an_stride, stride, 5); + int score_idx = (i * box_num + j * stride + k * w + l) * class_num; + CalcLabelScore(scores_data, input_data, label_idx, score_idx, + class_num, conf, stride); + } + } + } + } + } +}; + +} // namespace operators +} // namespace paddle diff --git a/paddle/fluid/operators/distributed_ops/fake_init_op.cc b/paddle/fluid/operators/distributed_ops/fake_init_op.cc index 28ebdcb03e..5ee35e0458 100644 --- a/paddle/fluid/operators/distributed_ops/fake_init_op.cc +++ b/paddle/fluid/operators/distributed_ops/fake_init_op.cc @@ -56,8 +56,7 @@ class FakeInitOp : public framework::OperatorBase { class FakeInitOpVarTypeInference : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc &op_desc, - framework::BlockDesc *block) const override {} + void operator()(framework::InferVarTypeContext *ctx) const override {} }; class FakeInitOpMaker : public framework::OpProtoAndCheckerMaker { diff --git a/paddle/fluid/operators/distributed_ops/merge_ids_op.cc b/paddle/fluid/operators/distributed_ops/merge_ids_op.cc index da0185b8c4..1b0b4dd316 100644 --- a/paddle/fluid/operators/distributed_ops/merge_ids_op.cc +++ b/paddle/fluid/operators/distributed_ops/merge_ids_op.cc @@ -114,11 +114,10 @@ class MergeIdsOp : public framework::OperatorWithKernel { class MergeIdsOpInferVarType : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc &op_desc, - framework::BlockDesc *block) const override { - auto *input_var = block->Var(op_desc.Input("Ids")[0]); - for (auto &out_var : op_desc.Output("Out")) { - block->Var(out_var)->SetType(input_var->GetType()); + void operator()(framework::InferVarTypeContext *ctx) const override { + auto input_type = ctx->GetType(ctx->Input("Ids")[0]); + for (auto &out_var : ctx->Output("Out")) { + ctx->SetType(out_var, input_type); } } }; diff --git a/paddle/fluid/operators/distributed_ops/split_ids_op.cc b/paddle/fluid/operators/distributed_ops/split_ids_op.cc index f61d387fbe..191ca1efe8 100644 --- a/paddle/fluid/operators/distributed_ops/split_ids_op.cc +++ b/paddle/fluid/operators/distributed_ops/split_ids_op.cc @@ -14,6 +14,8 @@ limitations under the License. */ #include "paddle/fluid/operators/distributed_ops/split_ids_op.h" +#include + namespace paddle { namespace operators { @@ -71,11 +73,10 @@ class SplitIdsOp : public framework::OperatorWithKernel { class SplitIdsOpInferVarType : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc &op_desc, - framework::BlockDesc *block) const override { - auto *input_var = block->Var(op_desc.Input("Ids")[0]); - for (auto &out_var : op_desc.Output("Out")) { - block->Var(out_var)->SetType(input_var->GetType()); + void operator()(framework::InferVarTypeContext *ctx) const override { + auto input_type = ctx->GetType(ctx->Input("Ids")[0]); + for (auto &out_var : ctx->Output("Out")) { + ctx->SetType(out_var, input_type); } } }; diff --git a/paddle/fluid/operators/fc_op.cc b/paddle/fluid/operators/fc_op.cc index eb4617a935..242f5390b8 100644 --- a/paddle/fluid/operators/fc_op.cc +++ b/paddle/fluid/operators/fc_op.cc @@ -55,17 +55,8 @@ void FCOp::InferShape(framework::InferShapeContext* ctx) const { "The input tensor Input's rank of FCOp should be larger than " "in_num_col_dims."); - auto in_mat_dims = framework::flatten_to_2d(in_dims, in_num_col_dims); - PADDLE_ENFORCE_EQ( - in_mat_dims[1], w_dims[0], - "Fully Connected input and weigth size do not match. %s, %s"); - std::vector output_dims; - output_dims.reserve(static_cast(in_num_col_dims + 1)); - for (int i = 0; i < in_num_col_dims; ++i) { - output_dims.push_back(in_dims[i]); - } - output_dims.push_back(w_dims[1]); + FCOutputSize(in_dims, w_dims, output_dims, in_num_col_dims); ctx->SetOutputDim("Out", framework::make_ddim(output_dims)); ctx->ShareLoD("Input", "Out"); @@ -128,6 +119,9 @@ void FCOpMaker::Make() { AddAttr("use_mkldnn", "(bool, default false) Only used in mkldnn kernel") .SetDefault(false); + AddAttr(framework::kAllKernelsMustComputeRuntimeShape, + "Skip calling InferShape() function in the runtime.") + .SetDefault(true); AddComment(R"DOC( Fully Connected Operator. @@ -142,13 +136,20 @@ class FCOpKernel : public framework::OpKernel { void Compute(const paddle::framework::ExecutionContext& ctx) const override { PADDLE_ENFORCE(platform::is_cpu_place(ctx.GetPlace()), "It must use CPUPlace."); - auto input = ctx.Input("Input"); + auto input = ctx.Input("Input"); auto w = ctx.Input("W"); auto bias = ctx.Input("Bias"); - auto output = ctx.Output("Out"); + auto output = ctx.Output("Out"); + int in_num_col_dims = ctx.Attr("in_num_col_dims"); auto w_dims = w->dims(); + + std::vector output_dims; + FCOutputSize(input->dims(), w_dims, output_dims, in_num_col_dims); + output->Resize(framework::make_ddim(output_dims)); + output->set_lod(input->lod()); + auto out_dims = output->dims(); - int M = framework::product(out_dims) / out_dims[out_dims.size() - 1]; + int M = framework::product(out_dims) / w_dims[1]; const T* input_data = input->data(); const T* w_data = w->data(); diff --git a/paddle/fluid/operators/fc_op.h b/paddle/fluid/operators/fc_op.h index e1b780fc0c..b82a63cd83 100644 --- a/paddle/fluid/operators/fc_op.h +++ b/paddle/fluid/operators/fc_op.h @@ -48,5 +48,21 @@ class FCOpMaker : public framework::OpProtoAndCheckerMaker { void Make() override; }; +inline void FCOutputSize(const framework::DDim& in_dims, + const framework::DDim& w_dims, + std::vector& out_dims, // NOLINT + int in_num_col_dims) { + auto in_mat_dims = framework::flatten_to_2d(in_dims, in_num_col_dims); + PADDLE_ENFORCE_EQ( + in_mat_dims[1], w_dims[0], + "Fully Connected input and weigth size do not match. %s, %s"); + + out_dims.reserve(static_cast(in_num_col_dims + 1)); + for (int i = 0; i < in_num_col_dims; ++i) { + out_dims.push_back(in_dims[i]); + } + out_dims.push_back(w_dims[1]); +} + } // namespace operators } // namespace paddle diff --git a/paddle/fluid/operators/fill_constant_op.cc b/paddle/fluid/operators/fill_constant_op.cc index c86430524e..cf2f4776cf 100644 --- a/paddle/fluid/operators/fill_constant_op.cc +++ b/paddle/fluid/operators/fill_constant_op.cc @@ -39,12 +39,11 @@ class FillConstantOp : public framework::OperatorWithKernel { class FillConstantOpVarTypeInference : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc& op_desc, - framework::BlockDesc* block) const override { + void operator()(framework::InferVarTypeContext* ctx) const override { auto data_type = static_cast( - boost::get(op_desc.GetAttr("dtype"))); - auto& out_var_name = op_desc.Output("Out").front(); - block->Var(out_var_name)->SetDataType(data_type); + boost::get(ctx->GetAttr("dtype"))); + auto& out_var_name = ctx->Output("Out").front(); + ctx->SetDataType(out_var_name, data_type); } }; diff --git a/paddle/fluid/operators/fused/fused_embedding_seq_pool_op.cc b/paddle/fluid/operators/fused/fused_embedding_seq_pool_op.cc index a0026427e2..9cc94ab88d 100644 --- a/paddle/fluid/operators/fused/fused_embedding_seq_pool_op.cc +++ b/paddle/fluid/operators/fused/fused_embedding_seq_pool_op.cc @@ -88,7 +88,8 @@ class FusedEmbeddingSeqPoolOpMaker : public framework::OpProtoAndCheckerMaker { "(boolean, default false) " "Sparse update.") .SetDefault(false); - AddAttr(framework::kAllKernelsMustComputeRuntimeShape, "") + AddAttr(framework::kAllKernelsMustComputeRuntimeShape, + "Skip calling InferShape() function in the runtime.") .SetDefault(true); AddComment(R"DOC( FusedEmbeddingSeqPool Operator. @@ -137,22 +138,20 @@ class FusedEmbeddingSeqPoolOpGrad : public framework::OperatorWithKernel { class FusedEmbeddingSeqPoolOpGradVarTypeInference : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc& op_desc, - framework::BlockDesc* block) const override { - auto out_var_name = op_desc.Output(framework::GradVarName("W")).front(); - auto attr = op_desc.GetAttr("is_sparse"); + void operator()(framework::InferVarTypeContext* ctx) const override { + auto out_var_name = ctx->Output(framework::GradVarName("W")).front(); + auto attr = ctx->GetAttr("is_sparse"); bool is_sparse = boost::get(attr); if (is_sparse) { VLOG(3) << "fused_embedding_seq_pool_grad op " << framework::GradVarName("W") << " is set to SelectedRows"; - block->Var(out_var_name) - ->SetType(framework::proto::VarType::SELECTED_ROWS); + ctx->SetType(out_var_name, framework::proto::VarType::SELECTED_ROWS); } else { VLOG(3) << "fused_embedding_seq_pool_grad op " << framework::GradVarName("W") << " is set to LoDTensor"; - block->Var(out_var_name)->SetType(framework::proto::VarType::LOD_TENSOR); + ctx->SetType(out_var_name, framework::proto::VarType::LOD_TENSOR); } - block->Var(out_var_name)->SetDataType(block->Var("W")->GetDataType()); + ctx->SetDataType(out_var_name, ctx->GetDataType(ctx->Input("W")[0])); } }; diff --git a/paddle/fluid/operators/get_tensor_from_selected_rows_op.cc b/paddle/fluid/operators/get_tensor_from_selected_rows_op.cc index a4ae19d9c1..c0893359af 100644 --- a/paddle/fluid/operators/get_tensor_from_selected_rows_op.cc +++ b/paddle/fluid/operators/get_tensor_from_selected_rows_op.cc @@ -81,15 +81,12 @@ GetTensorFromSelectedRows is used to get the tensor from SelectedRows. class GetTensorFromSelectedRowsOpVarTypeInference : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc &op_desc, - framework::BlockDesc *block) const final { - auto out_var_name = op_desc.Output("Out").front(); - auto in_var_name = op_desc.Input("X").front(); - - auto out_var = block->FindRecursiveOrCreateVar(out_var_name); - auto in_var = block->FindRecursiveOrCreateVar(in_var_name); - out_var.SetType(framework::proto::VarType::LOD_TENSOR); - out_var.SetDataType(in_var.GetDataType()); + void operator()(framework::InferVarTypeContext *ctx) const { // NOLINT + auto out_var_name = ctx->Output("Out").front(); + auto in_var_name = ctx->Input("X").front(); + + ctx->SetType(out_var_name, framework::proto::VarType::LOD_TENSOR); + ctx->SetDataType(out_var_name, ctx->GetDataType(in_var_name)); } }; diff --git a/paddle/fluid/operators/hash_op.cc b/paddle/fluid/operators/hash_op.cc index f6395fb32f..82222d0a7e 100644 --- a/paddle/fluid/operators/hash_op.cc +++ b/paddle/fluid/operators/hash_op.cc @@ -54,7 +54,8 @@ $$Out = scale * X$$ )DOC"); AddAttr("num_hash", "").SetDefault(1); AddAttr("mod_by", "").SetDefault(100000); - AddAttr(framework::kAllKernelsMustComputeRuntimeShape, "") + AddAttr(framework::kAllKernelsMustComputeRuntimeShape, + "Skip calling InferShape() function in the runtime.") .SetDefault(true); } }; diff --git a/paddle/fluid/operators/hierarchical_sigmoid_op.cc b/paddle/fluid/operators/hierarchical_sigmoid_op.cc index 6ca6f0bc04..d0e1057c43 100644 --- a/paddle/fluid/operators/hierarchical_sigmoid_op.cc +++ b/paddle/fluid/operators/hierarchical_sigmoid_op.cc @@ -197,38 +197,32 @@ class HierarchicalSigmoidGradOp : public framework::OperatorWithKernel { class HierarchicalSigmoidGradOpGradVarTypeInference : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc& op_desc, - framework::BlockDesc* block) const override { - auto w_grad_var_name = op_desc.Output(framework::GradVarName("W")).front(); - auto bias_grad_var_name_vec = - op_desc.Output(framework::GradVarName("Bias")); + void operator()(framework::InferVarTypeContext* ctx) const override { + auto w_grad_var_name = ctx->Output(framework::GradVarName("W")).front(); + auto bias_grad_var_name_vec = ctx->Output(framework::GradVarName("Bias")); std::string bias_grad_var_name; bool hasBias = false; if (bias_grad_var_name_vec.size()) { hasBias = true; - bias_grad_var_name = - op_desc.Output(framework::GradVarName("Bias")).front(); + bias_grad_var_name = ctx->Output(framework::GradVarName("Bias")).front(); } - auto attr = op_desc.GetAttr("is_sparse"); + auto attr = ctx->GetAttr("is_sparse"); bool is_sparse = boost::get(attr); if (is_sparse) { VLOG(30) << "hierarchical_sigmoid_grad op " << framework::GradVarName("W") << " is set to SelectedRows"; - block->Var(w_grad_var_name) - ->SetType(framework::proto::VarType::SELECTED_ROWS); + ctx->SetType(w_grad_var_name, framework::proto::VarType::SELECTED_ROWS); } else { VLOG(30) << "hierarchical_sigmoid_grad op " << framework::GradVarName("W") << " is set to LoDTensor"; - block->Var(w_grad_var_name) - ->SetType(framework::proto::VarType::LOD_TENSOR); + ctx->SetType(w_grad_var_name, framework::proto::VarType::LOD_TENSOR); } if (hasBias) { VLOG(30) << "hierarchical_sigmoid_grad op " << framework::GradVarName("Bias") << " is set to LoDTensor"; - block->Var(bias_grad_var_name) - ->SetType(framework::proto::VarType::LOD_TENSOR); + ctx->SetType(bias_grad_var_name, framework::proto::VarType::LOD_TENSOR); } - block->Var(w_grad_var_name)->SetDataType(block->Var("W")->GetDataType()); + ctx->SetDataType(w_grad_var_name, ctx->GetDataType(ctx->Input("W")[0])); } }; diff --git a/paddle/fluid/operators/lod_rank_table_op.cc b/paddle/fluid/operators/lod_rank_table_op.cc index 166952fe23..0a43ac0c52 100644 --- a/paddle/fluid/operators/lod_rank_table_op.cc +++ b/paddle/fluid/operators/lod_rank_table_op.cc @@ -64,11 +64,9 @@ class LoDRankTableInferShape : public framework::InferShapeBase { class LoDRankTableInferVarType : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc &op_desc, - framework::BlockDesc *block) const override { - for (auto &o : op_desc.Output("Out")) { - block->FindRecursiveOrCreateVar(o).SetType( - framework::proto::VarType::LOD_RANK_TABLE); + void operator()(framework::InferVarTypeContext *ctx) const override { + for (auto &o : ctx->Output("Out")) { + ctx->SetType(o, framework::proto::VarType::LOD_RANK_TABLE); } } }; diff --git a/paddle/fluid/operators/lod_tensor_to_array_op.cc b/paddle/fluid/operators/lod_tensor_to_array_op.cc index 9b91cf5260..61e3427370 100644 --- a/paddle/fluid/operators/lod_tensor_to_array_op.cc +++ b/paddle/fluid/operators/lod_tensor_to_array_op.cc @@ -201,10 +201,9 @@ class LoDTensorToArrayInferShape : public framework::InferShapeBase { class LoDTensorToArrayInferVarType : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc &op_desc, - framework::BlockDesc *block) const override { - for (auto &out_var : op_desc.Output("Out")) { - block->Var(out_var)->SetType(framework::proto::VarType::LOD_TENSOR_ARRAY); + void operator()(framework::InferVarTypeContext *ctx) const override { + for (auto &out_var : ctx->Output("Out")) { + ctx->SetType(out_var, framework::proto::VarType::LOD_TENSOR_ARRAY); } } }; diff --git a/paddle/fluid/operators/lookup_table_op.cc b/paddle/fluid/operators/lookup_table_op.cc index 0029932bc0..8d1ebe6b1c 100644 --- a/paddle/fluid/operators/lookup_table_op.cc +++ b/paddle/fluid/operators/lookup_table_op.cc @@ -147,22 +147,20 @@ class LookupTableOpGrad : public framework::OperatorWithKernel { class LookupTableOpGradVarTypeInference : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc& op_desc, - framework::BlockDesc* block) const override { - auto out_var_name = op_desc.Output(framework::GradVarName("W")).front(); - auto attr = op_desc.GetAttr("is_sparse"); + void operator()(framework::InferVarTypeContext* ctx) const override { + auto out_var_name = ctx->Output(framework::GradVarName("W")).front(); + auto attr = ctx->GetAttr("is_sparse"); bool is_sparse = boost::get(attr); if (is_sparse) { VLOG(3) << "lookup_table_grad op " << framework::GradVarName("W") << " is set to SelectedRows"; - block->Var(out_var_name) - ->SetType(framework::proto::VarType::SELECTED_ROWS); + ctx->SetType(out_var_name, framework::proto::VarType::SELECTED_ROWS); } else { VLOG(3) << "lookup_table_grad op " << framework::GradVarName("W") << " is set to LoDTensor"; - block->Var(out_var_name)->SetType(framework::proto::VarType::LOD_TENSOR); + ctx->SetType(out_var_name, framework::proto::VarType::LOD_TENSOR); } - block->Var(out_var_name)->SetDataType(block->Var("W")->GetDataType()); + ctx->SetDataType(out_var_name, ctx->GetDataType(ctx->Input("W")[0])); } }; diff --git a/paddle/fluid/operators/mkldnn/conv_mkldnn_op.cc b/paddle/fluid/operators/mkldnn/conv_mkldnn_op.cc index 14ca3e8073..8d96ae7e42 100644 --- a/paddle/fluid/operators/mkldnn/conv_mkldnn_op.cc +++ b/paddle/fluid/operators/mkldnn/conv_mkldnn_op.cc @@ -592,6 +592,7 @@ class ConvMKLDNNOpKernel : public paddle::framework::OpKernel { platform::SetDstMemoryHandler(ctx, output, handler, &dst_memory_p); } else { + need_s8_to_u8 = fuse_relu; platform::SetDstMemoryHandler(ctx, output, handler, &dst_memory_p); } diff --git a/paddle/fluid/operators/mkldnn/fc_mkldnn_op.cc b/paddle/fluid/operators/mkldnn/fc_mkldnn_op.cc index 3a926a716f..69c0486eb6 100644 --- a/paddle/fluid/operators/mkldnn/fc_mkldnn_op.cc +++ b/paddle/fluid/operators/mkldnn/fc_mkldnn_op.cc @@ -123,7 +123,7 @@ class FCMKLDNNOpKernel : public paddle::framework::OpKernel { auto& dev_ctx = ctx.template device_context(); const auto& mkldnn_engine = dev_ctx.GetEngine(); - auto input = ctx.Input("Input"); + auto input = ctx.Input("Input"); auto w = ctx.Input("W"); auto bias = ctx.Input("Bias"); @@ -151,7 +151,13 @@ class FCMKLDNNOpKernel : public paddle::framework::OpKernel { const T* input_data = input->data(); const T* w_data = w->data(); - auto output = ctx.Output("Out"); + auto output = ctx.Output("Out"); + int in_num_col_dims = ctx.Attr("in_num_col_dims"); + std::vector output_dims; + FCOutputSize(input->dims(), w->dims(), output_dims, in_num_col_dims); + output->Resize(framework::make_ddim(output_dims)); + output->set_lod(input->lod()); + T* output_data = output->mutable_data(ctx.GetPlace()); auto dst_memory = mem.dst(output_data); @@ -204,19 +210,21 @@ class FCMKLDNNGradOpKernel : public paddle::framework::OpKernel { Tensor* input_grad = ctx.Output(framework::GradVarName("Input")); Tensor* w_grad = ctx.Output(framework::GradVarName("W")); + const Tensor* input = ctx.Input("Input"); + const T* input_data = input->data(); + + const Tensor* w = ctx.Input("W"); + const T* w_data = w->data(); + if (input_grad) { + input_grad->Resize(input->dims()); input_grad_data = input_grad->mutable_data(ctx.GetPlace()); } if (w_grad) { + w_grad->Resize(w->dims()); w_grad_data = w_grad->mutable_data(ctx.GetPlace()); } - const Tensor* input = ctx.Input("Input"); - const T* input_data = input->data(); - - const Tensor* w = ctx.Input("W"); - const T* w_data = w->data(); - const Tensor* out_grad = ctx.Input(framework::GradVarName("Out")); const T* out_grad_data = out_grad->data(); diff --git a/paddle/fluid/operators/mkldnn/transpose_mkldnn_op.cc b/paddle/fluid/operators/mkldnn/transpose_mkldnn_op.cc index e41bfb80df..4debc7ca5e 100644 --- a/paddle/fluid/operators/mkldnn/transpose_mkldnn_op.cc +++ b/paddle/fluid/operators/mkldnn/transpose_mkldnn_op.cc @@ -73,6 +73,29 @@ class TransposeMKLDNNOpKernel : public paddle::framework::OpKernel { } }; +template +class TransposeINT8MKLDNNOpKernel : public paddle::framework::OpKernel { + public: + void Compute(const paddle::framework::ExecutionContext& ctx) const override { + std::vector axis = ctx.Attr>("axis"); + std::vector axis_int8 = {0, 2, 3, 1}; + if (axis.size() != 1) { + PADDLE_ENFORCE_EQ(axis.size(), axis_int8.size()); + for (size_t i = 0; i < axis.size(); i++) { + PADDLE_ENFORCE_EQ(axis[i], axis_int8[i], + "Current INT8 MKLDNN Transpose kernel only surpport " + "axis with [0, 2, 3, 1] due to MKL-DNN kernel " + "implementation."); + } + } + auto* input = ctx.Input("X"); + auto* output = ctx.Output("Out"); + output->ShareDataWith(*input); + output->set_layout(DataLayout::kMKLDNN); + output->set_format(input->format()); + } +}; + template class TransposeMKLDNNGradOpKernel : public paddle::framework::OpKernel { public: @@ -140,7 +163,10 @@ class TransposeMKLDNNGradOpKernel : public paddle::framework::OpKernel { namespace ops = paddle::operators; REGISTER_OP_KERNEL(transpose2, MKLDNN, ::paddle::platform::CPUPlace, - ops::TransposeMKLDNNOpKernel); + ops::TransposeMKLDNNOpKernel, + ops::TransposeINT8MKLDNNOpKernel, + ops::TransposeINT8MKLDNNOpKernel); + REGISTER_OP_KERNEL(transpose, MKLDNN, ::paddle::platform::CPUPlace, ops::TransposeMKLDNNOpKernel); diff --git a/paddle/fluid/operators/nccl/nccl_op.cc b/paddle/fluid/operators/nccl/nccl_op.cc index 0018139cb0..6a0ae0dede 100644 --- a/paddle/fluid/operators/nccl/nccl_op.cc +++ b/paddle/fluid/operators/nccl/nccl_op.cc @@ -60,12 +60,9 @@ class NCCLInitOp : public framework::OperatorBase { class NCCLInitOpVarTypeInference : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc &op_desc, - framework::BlockDesc *block) const override { - auto out_var_name = op_desc.Output("Communicator").front(); - auto &out_var = block->FindRecursiveOrCreateVar(out_var_name); - auto var_type = framework::proto::VarType::RAW; - out_var.SetType(var_type); + void operator()(framework::InferVarTypeContext *ctx) const override { + auto out_var_name = ctx->Output("Communicator").front(); + ctx->SetType(out_var_name, framework::proto::VarType::RAW); } }; diff --git a/paddle/fluid/operators/nce_op.cc b/paddle/fluid/operators/nce_op.cc index 256da34912..fa7cc58c08 100644 --- a/paddle/fluid/operators/nce_op.cc +++ b/paddle/fluid/operators/nce_op.cc @@ -237,23 +237,21 @@ class NCEOpGrad : public framework::OperatorWithKernel { class NCEOpGradVarTypeInference : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc &op_desc, - framework::BlockDesc *block) const override { - auto weight_grad = op_desc.Output(framework::GradVarName("Weight")).front(); + void operator()(framework::InferVarTypeContext *ctx) const override { + auto weight_grad = ctx->Output(framework::GradVarName("Weight")).front(); - auto attr = op_desc.GetAttr("is_sparse"); + auto attr = ctx->GetAttr("is_sparse"); bool is_sparse = boost::get(attr); if (is_sparse) { VLOG(3) << "nce_op_grad op " << weight_grad << " and " << " is set to SelectedRows"; - block->Var(weight_grad) - ->SetType(framework::proto::VarType::SELECTED_ROWS); + ctx->SetType(weight_grad, framework::proto::VarType::SELECTED_ROWS); } else { VLOG(3) << "nce_op_grad op " << weight_grad << " and " << " is set to LoDTensor"; - block->Var(weight_grad)->SetType(framework::proto::VarType::LOD_TENSOR); + ctx->SetType(weight_grad, framework::proto::VarType::LOD_TENSOR); } - block->Var(weight_grad)->SetDataType(block->Var("Input")->GetDataType()); + ctx->SetDataType(weight_grad, ctx->GetDataType(ctx->Input("Input")[0])); } }; diff --git a/paddle/fluid/operators/ngraph/ngraph_engine_op.cc b/paddle/fluid/operators/ngraph/ngraph_engine_op.cc index f941f917c8..479c95ba08 100644 --- a/paddle/fluid/operators/ngraph/ngraph_engine_op.cc +++ b/paddle/fluid/operators/ngraph/ngraph_engine_op.cc @@ -37,8 +37,7 @@ class NgraphEngineOpMaker : public framework::OpProtoAndCheckerMaker { class NgraphEngineInferVarType : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc &op_desc, - framework::BlockDesc *block) const override {} + void operator()(framework::InferVarTypeContext *ctx) const override {} }; } // namespace operators diff --git a/paddle/fluid/operators/optimizers/adam_op.h b/paddle/fluid/operators/optimizers/adam_op.h index 09255f60e6..6262ef0c2d 100644 --- a/paddle/fluid/operators/optimizers/adam_op.h +++ b/paddle/fluid/operators/optimizers/adam_op.h @@ -15,6 +15,7 @@ limitations under the License. */ #pragma once #include // for sqrt in CPU and CUDA #include +#include #include #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/framework/threadpool.h" @@ -311,17 +312,17 @@ struct SparseAdamFunctor { T beta1_pow = *beta1_pow_; T beta2_pow = *beta2_pow_; lr *= sqrt(1 - beta2_pow) / (1 - beta1_pow); - size_t row_count = numel / row_numel_; + int64_t row_count = static_cast(numel / row_numel_); - for (size_t i = 0U, j = 0U; i != row_count; ++i) { + for (int64_t i = 0, j = 0; i != row_count; ++i) { if (i == *(rows_ + j)) { - for (size_t k = 0U; k != row_numel_; ++k) { + for (int64_t k = 0; k != row_numel_; ++k) { T g = grad_[j * row_numel_ + k]; adam_update(i * row_numel_ + k, g); } ++j; } else { - for (size_t k = 0U; k != row_numel_; ++k) { + for (int64_t k = 0; k != row_numel_; ++k) { T mom1 = moment1_[i * row_numel_ + k]; T mom2 = moment2_[i * row_numel_ + k]; T p = param_[i * row_numel_ + k]; @@ -427,43 +428,23 @@ class AdamOpKernel : public framework::OpKernel { } } - framework::SelectedRows cpu_grad_merge; + framework::SelectedRows tmp_grad_merge; const framework::SelectedRows* grad_merge_ptr; if (is_strict_sorted) { grad_merge_ptr = &grad; } else { // merge duplicated rows if any. // The rows of grad_merge have been sorted inside MergeAdd functor - framework::SelectedRows* grad_merge_var; scatter::MergeAdd merge_func; - if (platform::is_cpu_place(ctx.GetPlace())) { - grad_merge_var = &cpu_grad_merge; - } else { - // FIXME(qiao): GPU also need to fix this - grad_merge_var = const_cast(ctx.scope()) - .Var() - ->GetMutable(); - } merge_func(ctx.template device_context(), grad, - grad_merge_var, true); - grad_merge_ptr = grad_merge_var; + &tmp_grad_merge, true); + grad_merge_ptr = &tmp_grad_merge; } auto& grad_merge = *grad_merge_ptr; auto& grad_tensor = grad_merge.value(); const T* grad_data = grad_tensor.template data(); - const int64_t* rows = nullptr; -// When compiled without CUDA, the CUDAData() interface should not be -// provided. -#if defined(PADDLE_WITH_CUDA) - if (platform::is_gpu_place(ctx.GetPlace())) { - rows = grad_merge.rows().CUDAData(ctx.GetPlace()); - } else { -#endif - rows = grad_merge.rows().data(); -#if defined(PADDLE_WITH_CUDA) - } -#endif + const int64_t* rows = grad_merge.rows().Data(ctx.GetPlace()); auto row_numel = grad_tensor.numel() / grad_merge.rows().size(); if (platform::is_cpu_place(ctx.GetPlace())) { @@ -488,7 +469,7 @@ class AdamOpKernel : public framework::OpKernel { } } #ifndef _WIN32 - else if (FLAGS_inner_op_parallelism > 1 && + else if (FLAGS_inner_op_parallelism > 1 && // NOLINT min_row_size_to_use_multithread > 0 && param.dims()[0] > min_row_size_to_use_multithread) { VLOG(3) << "use multi thread, inner_op_parallelism=" @@ -516,11 +497,11 @@ class AdamOpKernel : public framework::OpKernel { for (int i = 0; i < FLAGS_inner_op_parallelism; ++i) { int64_t start = i * line_in_each_thread; int64_t end = (i + 1) * line_in_each_thread; - if (start >= param_row_count) { + if (start >= static_cast(param_row_count)) { break; } - if (end > param_row_count) { - end = param_row_count; + if (end > static_cast(param_row_count)) { + end = static_cast(param_row_count); } fs.push_back( framework::Async([&functor, &row_id_to_grad_row_offset, @@ -545,8 +526,8 @@ class AdamOpKernel : public framework::OpKernel { } for (size_t i = 0; i < fs.size(); ++i) fs[i].wait(); } -#endif // !_WIN32 - else { +#endif // !_WIN32 + else { // NOLINT functor(param.numel()); } } else if (platform::is_gpu_place(ctx.GetPlace())) { diff --git a/paddle/fluid/operators/optimizers/lars_momentum_op.cc b/paddle/fluid/operators/optimizers/lars_momentum_op.cc index 574a03680b..126b665dd4 100644 --- a/paddle/fluid/operators/optimizers/lars_momentum_op.cc +++ b/paddle/fluid/operators/optimizers/lars_momentum_op.cc @@ -56,9 +56,9 @@ This optimizer use LARS (https://arxiv.org/abs/1708.03888) to optimize each weight using a local learning rate: $$ -local\_lr = \eta * +local\_lr = \eta * \frac{\left \| param \right \|}{\left \| grad \right \| + \beta *\left \| param \right \|} \\ -velocity = mu * velocity + +velocity = mu * velocity + local\_lr * (grad + \beta * param) \\ param = param - velocity. \\ $$ @@ -72,8 +72,7 @@ use L2 regularizers in case of using LARS. class LarsMomentumOpVarTypeInference : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc &op_desc, - framework::BlockDesc *block) const override {} + void operator()(framework::InferVarTypeContext* ctx) const override {} }; } // namespace operators } // namespace paddle diff --git a/paddle/fluid/operators/optimizers/momentum_op.cc b/paddle/fluid/operators/optimizers/momentum_op.cc index cde238c076..7cf218c20f 100644 --- a/paddle/fluid/operators/optimizers/momentum_op.cc +++ b/paddle/fluid/operators/optimizers/momentum_op.cc @@ -21,18 +21,14 @@ using Tensor = framework::Tensor; class MomentumOpInferVarType : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc& op_desc, - framework::BlockDesc* block) const override { - auto input_var = op_desc.Input("Param")[0]; - for (auto& out_var : op_desc.Output("ParamOut")) { - if (block->FindRecursiveOrCreateVar(input_var).GetType() == - framework::proto::VarType::SELECTED_ROWS) { - block->FindRecursiveOrCreateVar(out_var).SetType( - framework::proto::VarType::SELECTED_ROWS); - } else if (block->FindRecursiveOrCreateVar(input_var).GetType() == + void operator()(framework::InferVarTypeContext* ctx) const override { + auto& input_var = ctx->Input("Param")[0]; + for (auto& out_var : ctx->Output("ParamOut")) { + if (ctx->GetType(input_var) == framework::proto::VarType::SELECTED_ROWS) { + ctx->SetType(out_var, framework::proto::VarType::SELECTED_ROWS); + } else if (ctx->GetType(input_var) == framework::proto::VarType::LOD_TENSOR) { - block->FindRecursiveOrCreateVar(out_var).SetType( - framework::proto::VarType::LOD_TENSOR); + ctx->SetType(out_var, framework::proto::VarType::LOD_TENSOR); } else { PADDLE_THROW( "Only support LodTensor and SelectedRows, Unexpected Input Type."); diff --git a/paddle/fluid/operators/optimizers/momentum_op.h b/paddle/fluid/operators/optimizers/momentum_op.h index 3ed1bff5ff..29a2ae6755 100644 --- a/paddle/fluid/operators/optimizers/momentum_op.h +++ b/paddle/fluid/operators/optimizers/momentum_op.h @@ -13,6 +13,7 @@ See the License for the specific language governing permissions and limitations under the License. */ #pragma once +#include #include #include "paddle/fluid/framework/eigen.h" #include "paddle/fluid/framework/op_registry.h" @@ -69,6 +70,7 @@ class MomentumOp : public framework::OperatorWithKernel { ctx->SetOutputDim("ParamOut", param_dim); ctx->SetOutputDim("VelocityOut", param_dim); } + framework::OpKernelType GetExpectedKernelType( const framework::ExecutionContext& ctx) const override { auto input_data_type = framework::GetDataTypeOfVar(ctx.InputVar("Param")); @@ -351,23 +353,14 @@ class MomentumOpKernel : public framework::OpKernel { VLOG(3) << "Grad SelectedRows contains no data!"; return; } - auto* merged_grad = const_cast(ctx.scope()) - .Var() - ->GetMutable(); + + framework::SelectedRows tmp_merged_grad; + framework::SelectedRows* merged_grad = &tmp_merged_grad; math::scatter::MergeAdd merge_func; merge_func(ctx.template device_context(), *grad, merged_grad); - const int64_t* rows = nullptr; -#ifdef PADDLE_WITH_CUDA - if (platform::is_gpu_place(ctx.GetPlace())) { - rows = merged_grad->rows().CUDAData(ctx.GetPlace()); - } else { -#endif - rows = merged_grad->rows().data(); -#ifdef PADDLE_WITH_CUDA - } -#endif + const int64_t* rows = merged_grad->rows().Data(ctx.GetPlace()); int64_t row_numel = merged_grad->value().numel() / merged_grad->rows().size(); platform::ForRange for_range( diff --git a/paddle/fluid/operators/optimizers/rmsprop_op.h b/paddle/fluid/operators/optimizers/rmsprop_op.h index 389c84d246..4550052b2d 100644 --- a/paddle/fluid/operators/optimizers/rmsprop_op.h +++ b/paddle/fluid/operators/optimizers/rmsprop_op.h @@ -216,24 +216,14 @@ class RmspropOpKernel : public framework::OpKernel { } } else if (grad_var->IsType()) { auto &grad = grad_var->Get(); - auto *merged_grad = const_cast(ctx.scope()) - .Var() - ->GetMutable(); - + framework::SelectedRows tmp_merged_grad; + framework::SelectedRows *merged_grad = &tmp_merged_grad; math::scatter::MergeAdd merge_func; merge_func(dev_ctx, grad, merged_grad); platform::ForRange for_range(dev_ctx, limit); - const int64_t *rows; -#ifdef PADDLE_WITH_CUDA - if (platform::is_gpu_place(ctx.GetPlace())) { - rows = merged_grad->rows().CUDAData(ctx.GetPlace()); - } else { -#endif - rows = merged_grad->rows().data(); -#ifdef PADDLE_WITH_CUDA - } -#endif + const int64_t *rows = merged_grad->rows().Data(ctx.GetPlace()); + auto &merged_tensor = merged_grad->value(); int64_t row_count = merged_grad->rows().size(); int64_t row_numel = merged_tensor.numel() / row_count; diff --git a/paddle/fluid/operators/optimizers/sgd_op.cc b/paddle/fluid/operators/optimizers/sgd_op.cc index 690381a67f..34e99a14ff 100644 --- a/paddle/fluid/operators/optimizers/sgd_op.cc +++ b/paddle/fluid/operators/optimizers/sgd_op.cc @@ -50,20 +50,18 @@ class SGDOp : public framework::OperatorWithKernel { class SGDOpInferVarType : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc &op_desc, - framework::BlockDesc *block) const override { - auto input_var_n = op_desc.Input("Param")[0]; - auto in_var_type = block->FindRecursiveOrCreateVar(input_var_n).GetType(); + void operator()(framework::InferVarTypeContext *ctx) const override { + auto &input_var_n = ctx->Input("Param")[0]; + auto in_var_type = ctx->GetType(input_var_n); PADDLE_ENFORCE(in_var_type == framework::proto::VarType::SELECTED_ROWS || in_var_type == framework::proto::VarType::LOD_TENSOR, "The input Var's type should be LoDtensor or SelectedRows," " but the received var(%s)'s type is %s", input_var_n, in_var_type); - for (auto &out_var_n : op_desc.Output("ParamOut")) { - auto &out_var = block->FindRecursiveOrCreateVar(out_var_n); - if (out_var.GetType() != in_var_type) { - out_var.SetType(in_var_type); + for (auto &out_var_n : ctx->Output("ParamOut")) { + if (ctx->GetType(out_var_n) != in_var_type) { + ctx->SetType(out_var_n, in_var_type); } } } diff --git a/paddle/fluid/operators/pool_op.cc b/paddle/fluid/operators/pool_op.cc index 0a0ece162c..7963c27a01 100644 --- a/paddle/fluid/operators/pool_op.cc +++ b/paddle/fluid/operators/pool_op.cc @@ -13,6 +13,7 @@ See the License for the specific language governing permissions and limitations under the License. */ #include "paddle/fluid/operators/pool_op.h" +#include #ifdef PADDLE_WITH_CUDA #include "paddle/fluid/platform/cudnn_helper.h" #endif @@ -212,6 +213,12 @@ void Pool2dOpMaker::Make() { AddAttr("use_mkldnn", "(bool, default false) Only used in mkldnn kernel") .SetDefault(false); + AddAttr("use_quantizer", + "(bool, default false) " + "Set to true for operators that should be quantized and use " + "int8 kernel. " + "Only used on CPU.") + .SetDefault(false); AddAttr( "data_format", "(string, default NCHW) Only used in " diff --git a/paddle/fluid/operators/py_func_op.cc b/paddle/fluid/operators/py_func_op.cc index 53eff2de3e..5300e80747 100644 --- a/paddle/fluid/operators/py_func_op.cc +++ b/paddle/fluid/operators/py_func_op.cc @@ -14,8 +14,11 @@ #include "paddle/fluid/operators/py_func_op.h" +#include #include #include +#include +#include #include #include "paddle/fluid/framework/op_registry.h" @@ -91,15 +94,12 @@ static void CallPythonFunc(py::object *callable, } } -class PyFuncOpVarTypInference : public framework::VarTypeInference { +class PyFuncOpVarTypeInference : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc &op, - framework::BlockDesc *block) const override { - auto &outs = op.Outputs(); - bool has_out = (outs.count("Out") > 0 && !outs.at("Out").empty()); + void operator()(framework::InferVarTypeContext *ctx) const override { + bool has_out = (ctx->HasOutput("Out") && !ctx->Output("Out").empty()); - auto &ins = op.Inputs(); - bool has_in = (ins.count("X") > 0 && !ins.at("X").empty()); + bool has_in = (ctx->HasInput("X") && !ctx->Input("X").empty()); /** * X or Out can be empty, so that py_func can be more flexible @@ -107,8 +107,8 @@ class PyFuncOpVarTypInference : public framework::VarTypeInference { */ PADDLE_ENFORCE(has_in || has_out, "Input(X) or Output(Out) must exist"); - PADDLE_ENFORCE_GE(boost::get(op.GetAttr(kForwardPythonCallableId)), 0, - "Function id cannot be less than 0"); + PADDLE_ENFORCE_GE(boost::get(ctx->GetAttr(kForwardPythonCallableId)), + 0, "Function id cannot be less than 0"); if (!has_out) return; @@ -118,7 +118,7 @@ class PyFuncOpVarTypInference : public framework::VarTypeInference { * the corresponding forward variable */ const std::string kGradVarSuffix = framework::kGradVarSuffix; - auto &out_var_names = outs.at("Out"); + auto &out_var_names = ctx->Output("Out"); for (auto &out_var_name : out_var_names) { if (out_var_name == framework::kEmptyVarName || out_var_name.size() < kGradVarSuffix.size()) { @@ -128,18 +128,17 @@ class PyFuncOpVarTypInference : public framework::VarTypeInference { size_t len = out_var_name.size() - kGradVarSuffix.size(); if (out_var_name.substr(len) == kGradVarSuffix) { auto fwd_var_name = out_var_name.substr(0, len); - auto *out_var_desc = block->FindVarRecursive(out_var_name); - auto *fwd_var_desc = block->FindVarRecursive(fwd_var_name); - PADDLE_ENFORCE_NOT_NULL(out_var_desc, "Backward variable %s not found", - out_var_name); - PADDLE_ENFORCE_NOT_NULL(fwd_var_desc, "Forward variable %s not found", - fwd_var_name); + PADDLE_ENFORCE(ctx->HasVar(out_var_name), + "Backward variable %s not found", out_var_name); + PADDLE_ENFORCE(ctx->HasVar(fwd_var_name), + "Backward variable %s not found", fwd_var_name); VLOG(10) << "Infer var_desc of Output(" << out_var_name << ") as Input(" << fwd_var_name << ")"; - out_var_desc->SetShape(fwd_var_desc->GetShape()); - out_var_desc->SetDataType(fwd_var_desc->GetDataType()); - out_var_desc->SetLoDLevel(fwd_var_desc->GetLoDLevel()); - out_var_desc->SetType(fwd_var_desc->GetType()); + + ctx->SetShape(out_var_name, ctx->GetShape(fwd_var_name)); + ctx->SetDataType(out_var_name, ctx->GetDataType(fwd_var_name)); + ctx->SetLoDLevel(out_var_name, ctx->GetLoDLevel(fwd_var_name)); + ctx->SetType(out_var_name, ctx->GetType(fwd_var_name)); } } } @@ -309,5 +308,5 @@ class PyFuncOp : public framework::OperatorBase { namespace ops = paddle::operators; REGISTER_OPERATOR(py_func, ops::PyFuncOp, ops::PyFuncOpMaker, - ops::PyFuncOpVarTypInference, ops::PyFuncOpShapeInference, + ops::PyFuncOpVarTypeInference, ops::PyFuncOpShapeInference, ops::PyFuncOpGradDescMaker); diff --git a/paddle/fluid/operators/reader/create_custom_reader_op.cc b/paddle/fluid/operators/reader/create_custom_reader_op.cc index 85394b336f..fdc7b0f6a0 100644 --- a/paddle/fluid/operators/reader/create_custom_reader_op.cc +++ b/paddle/fluid/operators/reader/create_custom_reader_op.cc @@ -85,10 +85,10 @@ class CreateCustomReaderOpMaker : public DecoratedReaderMakerBase { AddComment(R"DOC( CreateCustomReader Operator - A custom reader can be used for input data preprocessing. - A custom reader holds its own sub-block, which will be executed in CPU - in its 'ReadNext()' function. Users can configurate their own - preprocessing pipelines by inserting operators into custom reader's + A custom reader can be used for input data preprocessing. + A custom reader holds its own sub-block, which will be executed in CPU + in its 'ReadNext()' function. Users can configurate their own + preprocessing pipelines by inserting operators into custom reader's sub-block. )DOC"); } @@ -123,23 +123,22 @@ class CustomReaderInferShape : public framework::InferShapeBase { class CustomReaderInferVarType : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc& op_desc, - framework::BlockDesc* block) const override { - framework::VarDesc* out_reader = block->FindVar(op_desc.Output("Out")[0]); - PADDLE_ENFORCE_NOT_NULL(out_reader); - out_reader->SetType(framework::proto::VarType::READER); + void operator()(framework::InferVarTypeContext* ctx) const override { + auto& out_var_name = ctx->Output("Out")[0]; + PADDLE_ENFORCE(ctx->HasVar(out_var_name)); + ctx->SetType(out_var_name, framework::proto::VarType::READER); auto sink_var_names = - boost::get>(op_desc.GetAttr("sink_var_names")); + boost::get>(ctx->GetAttr("sink_var_names")); const auto* sub_block = - boost::get(op_desc.GetAttr("sub_block")); + boost::get(ctx->GetAttr("sub_block")); std::vector res_data_types; for (const std::string& var_name : sink_var_names) { framework::VarDesc* var = sub_block->FindVar(var_name); PADDLE_ENFORCE_NOT_NULL(var); res_data_types.emplace_back(var->GetDataType()); } - out_reader->SetDataTypes(res_data_types); + ctx->SetDataTypes(out_var_name, res_data_types); } }; diff --git a/paddle/fluid/operators/reader/read_op.cc b/paddle/fluid/operators/reader/read_op.cc index 846b2ed77e..33a69ad5fe 100644 --- a/paddle/fluid/operators/reader/read_op.cc +++ b/paddle/fluid/operators/reader/read_op.cc @@ -51,19 +51,16 @@ class ReadInferShape : public framework::InferShapeBase { class ReadInferVarType : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc& op_desc, - framework::BlockDesc* block) const override { - bool infer_out = boost::get(op_desc.GetAttr("infer_out")); + void operator()(framework::InferVarTypeContext* ctx) const override { + bool infer_out = boost::get(ctx->GetAttr("infer_out")); if (infer_out) { - std::string reader_name = op_desc.Input("Reader")[0]; - std::vector out_names = op_desc.Output("Out"); - framework::VarDesc* reader = block->FindVarRecursive(reader_name); - auto dtypes = reader->GetDataTypes(); + std::string reader_name = ctx->Input("Reader")[0]; + std::vector out_names = ctx->Output("Out"); + auto dtypes = ctx->GetDataTypes(reader_name); PADDLE_ENFORCE_EQ(dtypes.size(), out_names.size()); for (size_t i = 0; i < dtypes.size(); ++i) { - framework::VarDesc& out = block->FindRecursiveOrCreateVar(out_names[i]); - out.SetType(framework::proto::VarType::LOD_TENSOR); - out.SetDataType(dtypes[i]); + ctx->SetType(out_names[i], framework::proto::VarType::LOD_TENSOR); + ctx->SetDataType(out_names[i], dtypes[i]); } } } diff --git a/paddle/fluid/operators/reader/reader_op_registry.cc b/paddle/fluid/operators/reader/reader_op_registry.cc index 3921eedf94..64a1f6b687 100644 --- a/paddle/fluid/operators/reader/reader_op_registry.cc +++ b/paddle/fluid/operators/reader/reader_op_registry.cc @@ -98,11 +98,10 @@ void FileReaderInferShape::operator()(framework::InferShapeContext* ctx) const { } } -void FileReaderInferVarType::operator()(const framework::OpDesc& op_desc, - framework::BlockDesc* block) const { - std::string reader_name = op_desc.Output("Out")[0]; - framework::VarDesc* reader = block->FindVarRecursive(reader_name); - reader->SetType(framework::proto::VarType::READER); +void FileReaderInferVarType::operator()( + framework::InferVarTypeContext* ctx) const { + std::string reader_name = ctx->Output("Out")[0]; + ctx->SetType(reader_name, framework::proto::VarType::READER); } void DecoratedReaderInferShape::operator()( @@ -125,13 +124,11 @@ void DecoratedReaderInferShape::operator()( } void DecoratedReaderInferVarType::operator()( - const framework::OpDesc& op_desc, framework::BlockDesc* block) const { - std::string in_reader_name = op_desc.Input("UnderlyingReader")[0]; - framework::VarDesc* in_reader = block->FindVarRecursive(in_reader_name); - std::string out_reader_name = op_desc.Output("Out")[0]; - framework::VarDesc* out_reader = block->FindVarRecursive(out_reader_name); - out_reader->SetType(framework::proto::VarType::READER); - out_reader->SetDataTypes(in_reader->GetDataTypes()); + framework::InferVarTypeContext* ctx) const { + const std::string& in_reader_name = ctx->Input("UnderlyingReader")[0]; + const std::string& out_reader_name = ctx->Output("Out")[0]; + ctx->SetType(out_reader_name, framework::proto::VarType::READER); + ctx->SetDataTypes(out_reader_name, ctx->GetDataTypes(in_reader_name)); } void DecoratedReaderMakerBase::Make() { diff --git a/paddle/fluid/operators/reader/reader_op_registry.h b/paddle/fluid/operators/reader/reader_op_registry.h index 25c3e7d77b..795a580605 100644 --- a/paddle/fluid/operators/reader/reader_op_registry.h +++ b/paddle/fluid/operators/reader/reader_op_registry.h @@ -14,7 +14,9 @@ #pragma once +#include #include +#include #include #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/framework/reader.h" @@ -59,8 +61,7 @@ class FileReaderInferShape : public framework::InferShapeBase { class FileReaderInferVarType : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc& op_desc, - framework::BlockDesc* block) const override; + void operator()(framework::InferVarTypeContext* ctx) const override; }; // general infershape for decorated reader @@ -72,8 +73,7 @@ class DecoratedReaderInferShape : public framework::InferShapeBase { // general var type inference for decorated reader class DecoratedReaderInferVarType : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc& op_desc, - framework::BlockDesc* block) const override; + void operator()(framework::InferVarTypeContext* ctx) const override; }; class DecoratedReaderMakerBase : public framework::OpProtoAndCheckerMaker { diff --git a/paddle/fluid/operators/save_op.cc b/paddle/fluid/operators/save_op.cc index fcc598f4f1..b02c098099 100644 --- a/paddle/fluid/operators/save_op.cc +++ b/paddle/fluid/operators/save_op.cc @@ -159,12 +159,9 @@ This operator will serialize and write LoDTensor / SelectedRows variable to file class SaveOpVarTypeInference : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc &op_desc, - framework::BlockDesc *block) const override { - auto out_var_name = op_desc.Output(LOOKUP_TABLE_PATH).front(); - auto &out_var = block->FindRecursiveOrCreateVar(out_var_name); - auto var_type = framework::proto::VarType::RAW; - out_var.SetType(var_type); + void operator()(framework::InferVarTypeContext *ctx) const override { + auto out_var_name = ctx->Output(LOOKUP_TABLE_PATH).front(); + ctx->SetType(out_var_name, framework::proto::VarType::RAW); } }; diff --git a/paddle/fluid/operators/scale_op.cc b/paddle/fluid/operators/scale_op.cc index 4ea77ed30d..4e4a015e18 100644 --- a/paddle/fluid/operators/scale_op.cc +++ b/paddle/fluid/operators/scale_op.cc @@ -14,6 +14,7 @@ limitations under the License. */ #include "paddle/fluid/operators/scale_op.h" +#include #include #include "paddle/fluid/operators/detail/safe_ref.h" @@ -69,17 +70,13 @@ $$Out = scale*(X + bias)$$ class ScaleOpVarTypeInference : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc &op_desc, - framework::BlockDesc *block) const override { - auto &in_var_name = op_desc.Input("X").front(); - auto &in_var = detail::Ref(block->FindVarRecursive(in_var_name)); - - auto out_var_name = op_desc.Output("Out").front(); - auto *out_var = block->FindVarRecursive(out_var_name); + void operator()(framework::InferVarTypeContext *ctx) const override { + auto &in_var_name = ctx->Input("X").front(); + auto out_var_name = ctx->Output("Out").front(); if (in_var_name != out_var_name) { - out_var->SetType(in_var.GetType()); - out_var->SetDataType(in_var.GetDataType()); + ctx->SetType(out_var_name, ctx->GetType(in_var_name)); + ctx->SetDataType(out_var_name, ctx->GetDataType(in_var_name)); } } }; diff --git a/paddle/fluid/operators/sequence_ops/sequence_enumerate_op.cc b/paddle/fluid/operators/sequence_ops/sequence_enumerate_op.cc index f357c9c08d..cc4eedbf4d 100644 --- a/paddle/fluid/operators/sequence_ops/sequence_enumerate_op.cc +++ b/paddle/fluid/operators/sequence_ops/sequence_enumerate_op.cc @@ -30,13 +30,6 @@ class SequenceEnumerateOp : public framework::OperatorWithKernel { "Output(X) of SequenceEnumerate operator should not be null."); const auto x_dims = ctx->GetInputDim("X"); - PADDLE_ENFORCE_EQ( - x_dims.size(), 2, - "Input(X) of SequenceEnumerate operator's rank should be 2."); - PADDLE_ENFORCE_EQ(x_dims[1], 1, - "Input(X) of SequenceEnumerate operator's 2nd " - "dimension should be 1."); - const auto win_size = ctx->Attrs().Get("win_size"); ctx->SetOutputDim("Out", {x_dims[0], win_size}); ctx->ShareLoD("X", "Out"); @@ -59,7 +52,8 @@ class SequenceEnumerateOpMaker : public framework::OpProtoAndCheckerMaker { }); AddAttr("pad_value", "(int) The enumerate sequence padding value.") .SetDefault(0); - AddAttr(framework::kAllKernelsMustComputeRuntimeShape, "") + AddAttr(framework::kAllKernelsMustComputeRuntimeShape, + "Skip calling InferShape() function in the runtime.") .SetDefault(true); AddComment(R"DOC( Sequence Enumerate Operator. diff --git a/paddle/fluid/operators/sequence_ops/sequence_enumerate_op.h b/paddle/fluid/operators/sequence_ops/sequence_enumerate_op.h index 18da69993b..6a1eb6e625 100644 --- a/paddle/fluid/operators/sequence_ops/sequence_enumerate_op.h +++ b/paddle/fluid/operators/sequence_ops/sequence_enumerate_op.h @@ -27,30 +27,47 @@ class SequenceEnumerateKernel : public framework::OpKernel { auto* in = context.Input("X"); auto* out = context.Output("Out"); int win_size = context.Attr("win_size"); - int pad_value = context.Attr("pad_value"); + auto pad_value = static_cast(context.Attr("pad_value")); auto in_dims = in->dims(); - auto in_lod = in->lod(); - + auto lod0 = in->lod()[0]; PADDLE_ENFORCE_EQ( - static_cast(in_dims[0]), in_lod[0].back(), + static_cast(in_dims[0]), lod0.back(), "The actual input data's size mismatched with LoD information."); + PADDLE_ENFORCE_EQ( + in_dims.size(), 2UL, + "Input(X) of SequenceEnumerate operator's rank should be 2."); + PADDLE_ENFORCE_EQ(in_dims[1], 1, + "Input(X) of SequenceEnumerate operator's 2nd " + "dimension should be 1."); // Generate enumerate sequence set - auto lod0 = in_lod[0]; auto in_data = in->data(); out->Resize({in_dims[0], win_size}); + out->set_lod(in->lod()); auto out_data = out->mutable_data(context.GetPlace()); for (size_t i = 0; i < lod0.size() - 1; ++i) { - for (size_t idx = lod0[i]; idx < lod0[i + 1]; ++idx) { - for (int word_idx = 0; word_idx < win_size; ++word_idx) { - size_t word_pos = idx + word_idx; - out_data[win_size * idx + word_idx] = - word_pos < lod0[i + 1] ? in_data[word_pos] : pad_value; + int start = lod0[i]; + int end = lod0[i + 1]; + int copy_size = win_size < end - start + 1 ? win_size : end - start + 1; + int mid = end + 1 - copy_size; + int pad_num = win_size - copy_size; + copy_size *= sizeof(T); + for (int idx = start; idx < mid; ++idx) { + std::memcpy(out_data, in_data + idx, copy_size); + out_data += win_size; + } + for (int idx = mid; idx < end; ++idx) { + copy_size -= sizeof(T); + pad_num++; + std::memcpy(out_data, in_data + idx, copy_size); + T* pdata = out_data + copy_size / sizeof(T); + for (int i = 0; i < pad_num; ++i) { + pdata[i] = pad_value; } + out_data += win_size; } } - out->set_lod(in->lod()); } }; diff --git a/paddle/fluid/operators/slice_op.cu b/paddle/fluid/operators/slice_op.cu index 1af57b89a3..24a564f9ef 100644 --- a/paddle/fluid/operators/slice_op.cu +++ b/paddle/fluid/operators/slice_op.cu @@ -31,18 +31,18 @@ __global__ void Padding(const paddle::platform::float16* d_out, paddle::platform::float16* d_in) { int64_t out_idx = threadIdx.x + blockDim.x * blockIdx.x; if (out_idx < n) { + int64_t out_idx_tmp = out_idx; int coords[D] = {0}; for (int i = D - 1; i >= 0; --i) { - coords[i] = out_idx % out_dims[i]; - out_idx /= out_dims[i]; + coords[i] = out_idx_tmp % out_dims[i]; + out_idx_tmp /= out_dims[i]; coords[i] += offsets[i]; } int64_t in_idx = 0; - for (int i = 0; i < D - 1; ++i) { - in_idx += coords[i] * in_dims[i + 1]; + for (int i = 0; i < D; ++i) { + in_idx = in_idx * in_dims[i] + coords[i]; } - in_idx += coords[D - 1]; d_in[in_idx] = d_out[out_idx]; } @@ -80,8 +80,8 @@ class SliceGradKernel(0)); int64_t numel = d_out->numel(); - dim3 blocks((numel - 1) / PADDLE_CUDA_NUM_THREADS + 1, 1, 1); - dim3 threads(PADDLE_CUDA_NUM_THREADS, 1, 1); + dim3 blocks((numel - 1) / PADDLE_CUDA_NUM_THREADS + 1); + dim3 threads(PADDLE_CUDA_NUM_THREADS); auto stream = ctx.cuda_device_context().stream(); auto out_shape = framework::vectorize2int(out_dims); diff --git a/paddle/fluid/operators/softmax_with_cross_entropy_op.cu b/paddle/fluid/operators/softmax_with_cross_entropy_op.cu index 52b8dcc681..89aaac4cbe 100644 --- a/paddle/fluid/operators/softmax_with_cross_entropy_op.cu +++ b/paddle/fluid/operators/softmax_with_cross_entropy_op.cu @@ -439,7 +439,8 @@ class SoftmaxWithCrossEntropyGradCUDAKernel : public framework::OpKernel { context.Input(framework::GradVarName("Loss"))->data(); Tensor* logit_grad = context.Output(framework::GradVarName("Logits")); - logit_grad->ShareDataWith(*context.Input("Softmax")); + framework::TensorCopy(*context.Input("Softmax"), context.GetPlace(), + context.device_context(), logit_grad); T* logit_grad_data = logit_grad->data(); const int batch_size = logit_grad->dims()[0]; diff --git a/paddle/fluid/operators/split_selected_rows_op.cc b/paddle/fluid/operators/split_selected_rows_op.cc index 0e7b1463d1..88dfebc0cf 100644 --- a/paddle/fluid/operators/split_selected_rows_op.cc +++ b/paddle/fluid/operators/split_selected_rows_op.cc @@ -14,6 +14,8 @@ limitations under the License. */ #include "paddle/fluid/operators/split_selected_rows_op.h" +#include + namespace paddle { namespace operators { @@ -60,10 +62,9 @@ class SplitSelectedRowsOp : public framework::OperatorWithKernel { class SplitSelectedRowsOpInferVarType : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc &op_desc, - framework::BlockDesc *block) const override { - for (auto &out_var : op_desc.Output("Out")) { - block->Var(out_var)->SetType(framework::proto::VarType::SELECTED_ROWS); + void operator()(framework::InferVarTypeContext *ctx) const override { + for (auto &out_var : ctx->Output("Out")) { + ctx->SetType(out_var, framework::proto::VarType::SELECTED_ROWS); } } }; diff --git a/paddle/fluid/operators/squeeze_op.cc b/paddle/fluid/operators/squeeze_op.cc index e389c6a65e..ecfb4e8956 100644 --- a/paddle/fluid/operators/squeeze_op.cc +++ b/paddle/fluid/operators/squeeze_op.cc @@ -94,6 +94,7 @@ class SqueezeOpInferShape : public framework::InferShapeBase { } }; +// TODO(paddle-dev): Should use OpKernel. class SqueezeOp : public framework::OperatorBase { public: using OperatorBase::OperatorBase; diff --git a/paddle/fluid/operators/sum_op.cc b/paddle/fluid/operators/sum_op.cc index 7abfbbd3cb..1391148ccf 100644 --- a/paddle/fluid/operators/sum_op.cc +++ b/paddle/fluid/operators/sum_op.cc @@ -12,6 +12,7 @@ limitations under the License. */ #include "paddle/fluid/operators/sum_op.h" #include +#include #include #include @@ -159,24 +160,20 @@ the LoD information with the first input. class SumOpVarTypeInference : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc& op_desc, - framework::BlockDesc* block) const override { - auto& inputs = op_desc.Input("X"); + void operator()(framework::InferVarTypeContext* ctx) const override { + auto& inputs = ctx->Input("X"); auto var_type = framework::proto::VarType::SELECTED_ROWS; - for (auto& name : op_desc.Input("X")) { - VLOG(10) << name << " " - << block->FindRecursiveOrCreateVar(name).GetType(); + for (auto& name : ctx->Input("X")) { + VLOG(10) << name << " " << ctx->GetType(name); } bool any_input_is_lod_tensor = std::any_of( - inputs.begin(), inputs.end(), [block](const std::string& name) { - return block->FindRecursiveOrCreateVar(name).GetType() == - framework::proto::VarType::LOD_TENSOR; + inputs.begin(), inputs.end(), [ctx](const std::string& name) { + return ctx->GetType(name) == framework::proto::VarType::LOD_TENSOR; }); - auto is_tensor_array = [block](const std::string& name) { - return block->FindRecursiveOrCreateVar(name).GetType() == - framework::proto::VarType::LOD_TENSOR_ARRAY; + auto is_tensor_array = [ctx](const std::string& name) { + return ctx->GetType(name) == framework::proto::VarType::LOD_TENSOR_ARRAY; }; bool any_input_is_tensor_array = @@ -188,8 +185,7 @@ class SumOpVarTypeInference : public framework::VarTypeInference { if (!all_inputs_are_tensor_array) { std::ostringstream os; for (auto& each : inputs) { - os << " " << each << " type is " - << block->FindRecursiveOrCreateVar(each).GetType() << "\n"; + os << " " << each << " type is " << ctx->GetType(each) << "\n"; } PADDLE_ENFORCE(all_inputs_are_tensor_array, "Not all inputs are tensor array:\n%s", os.str()); @@ -199,11 +195,9 @@ class SumOpVarTypeInference : public framework::VarTypeInference { var_type = framework::proto::VarType::LOD_TENSOR; } - auto out_var_name = op_desc.Output("Out").front(); - auto& out_var = block->FindRecursiveOrCreateVar(out_var_name); - out_var.SetType(var_type); - auto& in_var = detail::Ref(block->FindVarRecursive(inputs.front())); - out_var.SetDataType(in_var.GetDataType()); + auto out_var_name = ctx->Output("Out").front(); + ctx->SetType(out_var_name, var_type); + ctx->SetDataType(out_var_name, ctx->GetDataType(inputs.front())); } }; diff --git a/paddle/fluid/operators/tensor_array_to_tensor_op.cc b/paddle/fluid/operators/tensor_array_to_tensor_op.cc index 58a74ec2c1..2b83c42f20 100644 --- a/paddle/fluid/operators/tensor_array_to_tensor_op.cc +++ b/paddle/fluid/operators/tensor_array_to_tensor_op.cc @@ -177,10 +177,9 @@ class LoDTensorArray2TensorGradInferShape : public framework::InferShapeBase { class LoDTensorArray2TensorGradInferVarType : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc &op_desc, - framework::BlockDesc *block) const override { - for (auto &out_var : op_desc.Output(framework::GradVarName("X"))) { - block->Var(out_var)->SetType(framework::proto::VarType::LOD_TENSOR_ARRAY); + void operator()(framework::InferVarTypeContext *ctx) const override { + for (auto &out_var : ctx->Output(framework::GradVarName("X"))) { + ctx->SetType(out_var, framework::proto::VarType::LOD_TENSOR_ARRAY); } } }; diff --git a/paddle/fluid/operators/tensorrt/tensorrt_engine_op.cc b/paddle/fluid/operators/tensorrt/tensorrt_engine_op.cc index a8c86de9f9..6cf3e65e00 100644 --- a/paddle/fluid/operators/tensorrt/tensorrt_engine_op.cc +++ b/paddle/fluid/operators/tensorrt/tensorrt_engine_op.cc @@ -46,8 +46,7 @@ class TensorRTEngineOpMaker : public framework::OpProtoAndCheckerMaker { class TensorRTEngineInferVarType : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc &op_desc, - framework::BlockDesc *block) const override {} + void operator()(framework::InferVarTypeContext *ctx) const override {} }; } // namespace operators diff --git a/paddle/fluid/operators/uniform_random_op.cc b/paddle/fluid/operators/uniform_random_op.cc index e3132ae76f..bb6a1c5b16 100644 --- a/paddle/fluid/operators/uniform_random_op.cc +++ b/paddle/fluid/operators/uniform_random_op.cc @@ -112,17 +112,16 @@ uniform distribution. The random result is in set [min, max]. class UniformRandomOpVarTypeInference : public framework::VarTypeInference { public: - void operator()(const framework::OpDesc &op_desc, - framework::BlockDesc *block) const override { - auto out_var_name = op_desc.Output("Out").front(); + void operator()(framework::InferVarTypeContext *ctx) const override { + auto out_var_name = ctx->Output("Out").front(); auto var_data_type = static_cast( - boost::get(op_desc.GetAttr("dtype"))); + boost::get(ctx->GetAttr("dtype"))); - auto out_var = block->FindRecursiveOrCreateVar(out_var_name); - if (out_var.GetType() != framework::proto::VarType::SELECTED_ROWS) { - out_var.SetType(framework::proto::VarType::LOD_TENSOR); + if (ctx->GetType(out_var_name) != + framework::proto::VarType::SELECTED_ROWS) { + ctx->SetType(out_var_name, framework::proto::VarType::LOD_TENSOR); } - out_var.SetDataType(var_data_type); + ctx->SetDataType(out_var_name, var_data_type); } }; diff --git a/paddle/fluid/platform/device_context.cc b/paddle/fluid/platform/device_context.cc index ada9a19736..d54a3e8670 100644 --- a/paddle/fluid/platform/device_context.cc +++ b/paddle/fluid/platform/device_context.cc @@ -316,7 +316,9 @@ CUDADeviceContext::~CUDADeviceContext() { eigen_stream_.reset(); eigen_device_.reset(); PADDLE_ENFORCE(cudaStreamDestroy(stream_)); +#if !defined(_WIN32) PADDLE_ENFORCE(dynload::ncclCommDestroy(nccl_comm_)); +#endif } Place CUDADeviceContext::GetPlace() const { return place_; } diff --git a/paddle/fluid/platform/device_context.h b/paddle/fluid/platform/device_context.h index 3f7ce3d944..1eb8d9691a 100644 --- a/paddle/fluid/platform/device_context.h +++ b/paddle/fluid/platform/device_context.h @@ -265,11 +265,13 @@ class CUDADeviceContext : public DeviceContext { /*! \brief Return cuda stream in the device context. */ cudaStream_t stream() const; +#if !defined(_WIN32) /*! \brief Return nccl communicators. */ ncclComm_t nccl_comm() const { return nccl_comm_; } /*! \brief Set nccl communicators. */ void set_nccl_comm(ncclComm_t comm) { nccl_comm_ = comm; } +#endif template void RecordEvent(cudaEvent_t ev, Callback callback) { @@ -295,12 +297,14 @@ class CUDADeviceContext : public DeviceContext { std::unique_ptr cublas_handle_; std::unique_ptr cublas_tensor_core_handle_; +#if !defined(_WIN32) // NCCL communicator (single process version) for NCCL collective operations. // NCCL collective operations provides fast collectives over multiple GPUs // both within and across nodes. // But, this collectives is used for collectives over multiple GPUs within // nodes. ncclComm_t nccl_comm_{nullptr}; +#endif int compute_capability_; int runtime_version_; diff --git a/paddle/fluid/platform/gpu_info.cc b/paddle/fluid/platform/gpu_info.cc index 400a6d7bfa..ca858f6024 100644 --- a/paddle/fluid/platform/gpu_info.cc +++ b/paddle/fluid/platform/gpu_info.cc @@ -13,7 +13,6 @@ See the License for the specific language governing permissions and limitations under the License. */ #include "paddle/fluid/platform/gpu_info.h" - #include #include #include @@ -31,6 +30,8 @@ constexpr static float fraction_of_gpu_memory_to_use = 0.92f; constexpr static float fraction_of_gpu_memory_to_use = 0.5f; #endif +constexpr static float fraction_reserve_gpu_memory = 0.05f; + DEFINE_double(fraction_of_gpu_memory_to_use, fraction_of_gpu_memory_to_use, "Allocate a trunk of gpu memory that is this fraction of the " "total gpu memory size. Future memory usage will be allocated " @@ -38,6 +39,23 @@ DEFINE_double(fraction_of_gpu_memory_to_use, fraction_of_gpu_memory_to_use, "additional trunks of the same size will be requested from gpu " "until the gpu has no memory left for another trunk."); +DEFINE_uint64(gpu_init_memory_in_mb, 0ul, + "Allocate a trunk of gpu memory whose byte size is specified by " + "the flag. Future memory usage will be allocated from the " + "truck. If the trunk doesn't have enough gpu memory, additional " + "trunks of the gpu memory will be requested from gpu with size " + "speified by FLAGS_gpu_reallocate_memory_in_mb until the gpu has " + "no memory left for the additional trunk. Note: if you set this " + "flag, the memory size set by " + "FLAGS_fraction_of_gpu_memory_to_use will be overrided by this " + "flag. If you don't set this flag, PaddlePaddle will use " + "FLAGS_fraction_of_gpu_memory_to_use to allocate gpu memory"); + +DEFINE_uint64(gpu_reallocate_memory_in_mb, 0ul, + "If this flag is set, Paddle will reallocate the gpu memory with " + "size specified by this flag. Else Paddle will reallocate by " + "FLAGS_fraction_of_gpu_memory_to_use"); + DEFINE_bool( enable_cublas_tensor_op_math, false, "The enable_cublas_tensor_op_math indicate whether to use Tensor Core, " @@ -180,13 +198,43 @@ void GpuMemoryUsage(size_t *available, size_t *total) { } size_t GpuMaxAllocSize() { + return std::max(GpuInitAllocSize(), GpuReallocSize()); +} + +size_t GpuInitAllocSize() { + if (FLAGS_gpu_init_memory_in_mb > 0ul) { + // Initial memory will be allocated by FLAGS_gpu_init_memory_in_mb + return static_cast(FLAGS_gpu_init_memory_in_mb << 20); + } + + // FLAGS_gpu_init_memory_in_mb is 0, initial memory will be allocated by + // fraction size_t total = 0; size_t available = 0; GpuMemoryUsage(&available, &total); + size_t reserving = static_cast(fraction_reserve_gpu_memory * total); - // Reserve the rest for page tables, etc. - return static_cast(total * FLAGS_fraction_of_gpu_memory_to_use); + return static_cast((total - reserving) * + FLAGS_fraction_of_gpu_memory_to_use); +} + +size_t GpuReallocSize() { + if (FLAGS_gpu_reallocate_memory_in_mb > 0ul) { + // Additional memory will be allocated by FLAGS_gpu_reallocate_memory_in_mb + return static_cast(FLAGS_gpu_reallocate_memory_in_mb << 20); + } + + // FLAGS_gpu_reallocate_memory_in_mb is 0, additional memory will be allocated + // by fraction + size_t total = 0; + size_t available = 0; + + GpuMemoryUsage(&available, &total); + size_t reserving = static_cast(fraction_reserve_gpu_memory * total); + + return static_cast((total - reserving) * + FLAGS_fraction_of_gpu_memory_to_use); } size_t GpuMinChunkSize() { @@ -201,16 +249,13 @@ size_t GpuMaxChunkSize() { GpuMemoryUsage(&available, &total); VLOG(10) << "GPU Usage " << available / 1024 / 1024 << "M/" << total / 1024 / 1024 << "M"; - size_t reserving = static_cast(0.05 * total); + size_t reserving = static_cast(fraction_reserve_gpu_memory * total); // If available less than minimum chunk size, no usable memory exists. available = std::min(std::max(available, GpuMinChunkSize()) - GpuMinChunkSize(), total - reserving); - // Reserving the rest memory for page tables, etc. - - size_t allocating = static_cast(FLAGS_fraction_of_gpu_memory_to_use * - (total - reserving)); + size_t allocating = GpuMaxAllocSize(); PADDLE_ENFORCE_LE(allocating, available, "Insufficient GPU memory to allocation."); diff --git a/paddle/fluid/platform/gpu_info.h b/paddle/fluid/platform/gpu_info.h index 1e1ab2503f..d4be7ac97b 100644 --- a/paddle/fluid/platform/gpu_info.h +++ b/paddle/fluid/platform/gpu_info.h @@ -60,6 +60,12 @@ void GpuMemoryUsage(size_t *available, size_t *total); //! Get the maximum allocation size of current GPU device. size_t GpuMaxAllocSize(); +//! Get the initial allocation size of current GPU device. +size_t GpuInitAllocSize(); + +//! Get the re-allocation size of current GPU device. +size_t GpuReallocSize(); + //! Get the minimum chunk size for GPU buddy allocator. size_t GpuMinChunkSize(); diff --git a/paddle/fluid/pybind/CMakeLists.txt b/paddle/fluid/pybind/CMakeLists.txt index 4ac5b83c56..f1385f5718 100644 --- a/paddle/fluid/pybind/CMakeLists.txt +++ b/paddle/fluid/pybind/CMakeLists.txt @@ -1,6 +1,6 @@ set(PYBIND_DEPS pybind python proto_desc memory executor async_executor prune feed_fetch_method pass_builder parallel_executor profiler layer scope_pool - tracer analysis_predictor) + tracer analysis_predictor imperative_profiler) if(WITH_PYTHON) list(APPEND PYBIND_DEPS py_func_op) diff --git a/paddle/fluid/pybind/imperative.cc b/paddle/fluid/pybind/imperative.cc index 6bbda69297..e9ed4e1644 100644 --- a/paddle/fluid/pybind/imperative.cc +++ b/paddle/fluid/pybind/imperative.cc @@ -38,20 +38,22 @@ void BindTracer(pybind11::module* m) { .def("trace", [](imperative::Tracer& self, imperative::OpBase* op, const imperative::VarBasePtrMap& inputs, - const imperative::VarBasePtrMap& outputs, + imperative::VarBasePtrMap* outputs, framework::AttributeMap attrs_map, const platform::CPUPlace expected_place, const bool stop_gradient = false) { + pybind11::gil_scoped_release release; return self.Trace(op, inputs, outputs, attrs_map, expected_place, stop_gradient); }) .def("trace", [](imperative::Tracer& self, imperative::OpBase* op, const imperative::VarBasePtrMap& inputs, - const imperative::VarBasePtrMap& outputs, + imperative::VarBasePtrMap* outputs, framework::AttributeMap attrs_map, const platform::CUDAPlace expected_place, const bool stop_gradient = false) { + pybind11::gil_scoped_release release; return self.Trace(op, inputs, outputs, attrs_map, expected_place, stop_gradient); }) diff --git a/paddle/fluid/pybind/inference_api.cc b/paddle/fluid/pybind/inference_api.cc index 236afc77f7..11e9725aea 100644 --- a/paddle/fluid/pybind/inference_api.cc +++ b/paddle/fluid/pybind/inference_api.cc @@ -242,6 +242,10 @@ void BindAnalysisConfig(py::module *m) { .def("set_mkldnn_op", &AnalysisConfig::SetMKLDNNOp) .def("set_model_buffer", &AnalysisConfig::SetModelBuffer) .def("model_from_memory", &AnalysisConfig::model_from_memory) + .def("runtime_context_cache_enabled", + &AnalysisConfig::runtime_context_cache_enabled) + .def("switch_runtime_context_cache", + &AnalysisConfig::SwitchRuntimeContextCache, py::arg("x") = true) .def("pass_builder", &AnalysisConfig::pass_builder, py::return_value_policy::reference); } diff --git a/paddle/fluid/pybind/pybind.cc b/paddle/fluid/pybind/pybind.cc index 5a753d0a78..691b437ab0 100644 --- a/paddle/fluid/pybind/pybind.cc +++ b/paddle/fluid/pybind/pybind.cc @@ -36,6 +36,7 @@ limitations under the License. */ #include "paddle/fluid/framework/selected_rows.h" #include "paddle/fluid/framework/version.h" #include "paddle/fluid/imperative/layer.h" +#include "paddle/fluid/imperative/profiler.h" #include "paddle/fluid/memory/allocation/allocator_strategy.h" #include "paddle/fluid/memory/allocation/legacy_allocator.h" #include "paddle/fluid/operators/activation_op.h" @@ -156,6 +157,11 @@ PYBIND11_MODULE(core, m) { m.def("print_mem_usage", []() { return memory::allocation::GPUMemMonitor.PrintMemUsage(); }); + m.def("start_imperative_gperf_profiler", + []() { imperative::StartProfile(); }); + + m.def("stop_imperative_gperf_profiler", []() { imperative::StopProfile(); }); + py::class_(m, "VarBase", R"DOC()DOC") .def( py::init 2 and len(x_shape) > 2: for i, dim_x in enumerate(x_shape[:-2]): @@ -6367,6 +6400,8 @@ def squeeze(input, axes, name=None): x = layers.data(name='x', shape=[5, 1, 10]) y = layers.sequeeze(input=x, axes=[1]) """ + assert not _in_imperative_mode(), ( + "squeeze layer is not supported in imperative mode yet.") helper = LayerHelper("squeeze", **locals()) out = helper.create_variable_for_type_inference(dtype=input.dtype) x_shape = helper.create_variable_for_type_inference(dtype=input.dtype) @@ -9104,6 +9139,10 @@ def _elementwise_op(helper): op_type = helper.layer_type x = helper.kwargs.get('x', None) y = helper.kwargs.get('y', None) + if _in_imperative_mode(): + x = base.to_variable(x) + y = base.to_variable(y) + assert x is not None, 'x cannot be None in {}'.format(op_type) assert y is not None, 'y cannot be None in {}'.format(op_type) axis = helper.kwargs.get('axis', -1) diff --git a/python/paddle/fluid/tests/test_detection.py b/python/paddle/fluid/tests/test_detection.py index b756c532ca..7d1b869cf5 100644 --- a/python/paddle/fluid/tests/test_detection.py +++ b/python/paddle/fluid/tests/test_detection.py @@ -489,6 +489,16 @@ class TestYoloDetection(unittest.TestCase): self.assertIsNotNone(loss) + def test_yolo_box(self): + program = Program() + with program_guard(program): + x = layers.data(name='x', shape=[30, 7, 7], dtype='float32') + img_size = layers.data(name='img_size', shape=[2], dtype='int32') + boxes, scores = layers.yolo_box(x, img_size, [10, 13, 30, 13], 10, + 0.01, 32) + self.assertIsNotNone(boxes) + self.assertIsNotNone(scores) + class TestBoxClip(unittest.TestCase): def test_box_clip(self): diff --git a/python/paddle/fluid/tests/unittests/mkldnn/test_transpose_int8_mkldnn_op.py b/python/paddle/fluid/tests/unittests/mkldnn/test_transpose_int8_mkldnn_op.py new file mode 100644 index 0000000000..a8127bcc78 --- /dev/null +++ b/python/paddle/fluid/tests/unittests/mkldnn/test_transpose_int8_mkldnn_op.py @@ -0,0 +1,78 @@ +# 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. + +from __future__ import print_function + +import unittest +import numpy as np +from paddle.fluid.tests.unittests.op_test import OpTest +from mkldnn_op_test import format_reorder + + +class TestTransposeOp(OpTest): + def setUp(self): + self.init_op_type() + self.initTestCase() + self.initInputData() + self.use_mkldnn = True + self.axis = (0, 2, 3, 1) + + self.inputs = { + 'X': format_reorder(self.input_data, self.shape) + } #transform data format to 'NHWC' for INT8 transpose specially. + + self.attrs = { + 'axis': list(self.axis), + 'use_mkldnn': self.use_mkldnn, + } + + self.outputs = { + 'XShape': np.random.random(self.shape).astype('int8'), + 'Out': self.inputs['X'].transpose(self.axis) + } + + def init_op_type(self): + self.op_type = "transpose2" + + def test_check_output(self): + self.check_output(no_check_set=['XShape']) + + def initTestCase(self): + self.shape = (2, 3, 4, 5) + + def initInputData(self): + self.input_data = ( + np.random.randint(0, 100, self.shape) - 50).astype('int8') + + +class TestINT8Case(TestTransposeOp): + def initTestCase(self): + self.shape = (2, 4, 6, 8) + + def initInputData(self): + self.input_data = ( + np.random.randint(0, 100, self.shape) - 50).astype('int8') + + +class TestUINT8Case(TestTransposeOp): + def initTestCase(self): + self.shape = (1, 3, 5, 7) + + def initDataType(self): + self.input_data = (np.random.randint(0, 100, + self.shape)).astype('uint8') + + +if __name__ == '__main__': + unittest.main() diff --git a/python/paddle/fluid/tests/unittests/test_imperative_gnn.py b/python/paddle/fluid/tests/unittests/test_imperative_gnn.py new file mode 100644 index 0000000000..2086fab5c8 --- /dev/null +++ b/python/paddle/fluid/tests/unittests/test_imperative_gnn.py @@ -0,0 +1,144 @@ +# 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 contextlib +import unittest +import numpy as np +import six +import sys + +import paddle +import paddle.fluid as fluid +import paddle.fluid.core as core +from paddle.fluid.optimizer import AdamOptimizer +from paddle.fluid.imperative.nn import Conv2D, Pool2D, FC +from test_imperative_base import new_program_scope +from paddle.fluid.imperative.base import to_variable + + +def gen_data(): + pass + + +class GraphConv(fluid.imperative.Layer): + def __init__(self, name_scope, in_features, out_features): + super(GraphConv, self).__init__(name_scope) + + self._in_features = in_features + self._out_features = out_features + self.weight = self.create_parameter( + attr=None, + dtype='float32', + shape=[self._in_features, self._out_features]) + self.bias = self.create_parameter( + attr=None, dtype='float32', shape=[self._out_features]) + + def forward(self, features, adj): + support = fluid.layers.matmul(features, self.weight) + # TODO(panyx0718): sparse matmul? + return fluid.layers.matmul(adj, support) + self.bias + + +class GCN(fluid.imperative.Layer): + def __init__(self, name_scope, num_hidden): + super(GCN, self).__init__(name_scope) + self.gc = GraphConv(self.full_name(), num_hidden, 32) + self.gc2 = GraphConv(self.full_name(), 32, 10) + + def forward(self, x, adj): + x = fluid.layers.relu(self.gc(x, adj)) + return self.gc2(x, adj) + + +class TestImperativeGNN(unittest.TestCase): + def test_gnn_float32(self): + seed = 90 + + startup = fluid.Program() + startup.random_seed = seed + main = fluid.Program() + main.random_seed = seed + + scope = fluid.core.Scope() + with new_program_scope(main=main, startup=startup, scope=scope): + features = fluid.layers.data( + name='features', + shape=[1, 100, 50], + dtype='float32', + append_batch_size=False) + # Use selected rows when it's supported. + adj = fluid.layers.data( + name='adj', + shape=[1, 100, 100], + dtype='float32', + append_batch_size=False) + labels = fluid.layers.data( + name='labels', + shape=[100, 1], + dtype='int64', + append_batch_size=False) + + model = GCN('test_gcn', 50) + logits = model(features, adj) + logits = fluid.layers.reshape(logits, logits.shape[1:]) + # In other example, it's nll with log_softmax. However, paddle's + # log_loss only supports binary classification now. + loss = fluid.layers.softmax_with_cross_entropy(logits, labels) + loss = fluid.layers.reduce_sum(loss) + + adam = AdamOptimizer(learning_rate=1e-3) + adam.minimize(loss) + exe = fluid.Executor(fluid.CPUPlace( + ) if not core.is_compiled_with_cuda() else fluid.CUDAPlace(0)) + exe.run(startup) + static_loss = exe.run(feed={ + 'features': np.zeros( + [1, 100, 50], dtype=np.float32), + 'adj': np.zeros( + [1, 100, 100], dtype=np.float32), + 'labels': np.zeros( + [100, 1], dtype=np.int64) + }, + fetch_list=[loss])[0] + + static_weight = np.array( + scope.find_var(model.gc.weight.name).get_tensor()) + + with fluid.imperative.guard(): + fluid.default_startup_program().random_seed = seed + fluid.default_main_program().random_seed = seed + + features = np.zeros([1, 100, 50], dtype=np.float32) + # Use selected rows when it's supported. + adj = np.zeros([1, 100, 100], dtype=np.float32) + labels = np.zeros([100, 1], dtype=np.int64) + + model = GCN('test_gcn', 50) + logits = model(to_variable(features), to_variable(adj)) + logits = fluid.layers.reshape(logits, logits.shape[1:]) + # In other example, it's nll with log_softmax. However, paddle's + # log_loss only supports binary classification now. + loss = fluid.layers.softmax_with_cross_entropy(logits, + to_variable(labels)) + loss = fluid.layers.reduce_sum(loss) + adam = AdamOptimizer(learning_rate=1e-3) + adam.minimize(loss) + self.assertEqual(static_loss, loss._numpy()) + self.assertTrue( + np.allclose(static_weight, model.gc.weight._numpy())) + sys.stderr.write('%s %s\n' % (static_loss, loss._numpy())) + + +if __name__ == '__main__': + unittest.main() diff --git a/python/paddle/fluid/tests/unittests/test_layers.py b/python/paddle/fluid/tests/unittests/test_layers.py index 5b186ae038..885ee170e8 100644 --- a/python/paddle/fluid/tests/unittests/test_layers.py +++ b/python/paddle/fluid/tests/unittests/test_layers.py @@ -84,6 +84,27 @@ class TestLayer(LayerTest): self.assertTrue(np.allclose(static_ret, dy_ret._numpy())) + def test_matmul(self): + with self.static_graph(): + t = layers.data(name='t', shape=[3, 3], dtype='float32') + t2 = layers.data(name='t2', shape=[3, 3], dtype='float32') + ret = layers.matmul(t, t2) + static_ret = self.get_static_graph_result( + feed={ + 't': np.ones( + [3, 3], dtype='float32'), + 't2': np.ones( + [3, 3], dtype='float32') + }, + fetch_list=[ret])[0] + + with self.dynamic_graph(): + t = np.ones([3, 3], dtype='float32') + t2 = np.ones([3, 3], dtype='float32') + dy_ret = layers.matmul(base.to_variable(t), base.to_variable(t2)) + + self.assertTrue(np.allclose(static_ret, dy_ret._numpy())) + def test_conv2d(self): with self.static_graph(): images = layers.data(name='pixel', shape=[3, 5, 5], dtype='float32') @@ -153,6 +174,60 @@ class TestLayer(LayerTest): self.assertTrue(np.allclose(static_ret[i], static_ret2[i])) self.assertTrue(np.allclose(static_ret[i], dy_ret[i]._numpy())) + def test_elementwise_math(self): + n = np.ones([3, 3], dtype='float32') + n2 = np.ones([3, 3], dtype='float32') * 1.1 + n3 = np.ones([3, 3], dtype='float32') * 2 + n4 = np.ones([3, 3], dtype='float32') * 3 + n5 = np.ones([3, 3], dtype='float32') * 4 + n6 = np.ones([3, 3], dtype='float32') * 5 + + with self.static_graph(): + t = layers.data(name='t', shape=[3, 3], dtype='float32') + t2 = layers.data(name='t2', shape=[3, 3], dtype='float32') + t3 = layers.data(name='t3', shape=[3, 3], dtype='float32') + t4 = layers.data(name='t4', shape=[3, 3], dtype='float32') + t5 = layers.data(name='t5', shape=[3, 3], dtype='float32') + t6 = layers.data(name='t6', shape=[3, 3], dtype='float32') + + ret = layers.elementwise_add(t, t2) + ret = layers.elementwise_pow(ret, t3) + ret = layers.elementwise_div(ret, t4) + ret = layers.elementwise_sub(ret, t5) + ret = layers.elementwise_mul(ret, t6) + + static_ret = self.get_static_graph_result( + feed={ + 't': n, + 't2': n2, + 't3': n3, + 't4': n4, + 't5': n5, + 't6': n6 + }, + fetch_list=[ret])[0] + + with self.dynamic_graph(): + ret = layers.elementwise_add(n, n2) + ret = layers.elementwise_pow(ret, n3) + ret = layers.elementwise_div(ret, n4) + ret = layers.elementwise_sub(ret, n5) + dy_ret = layers.elementwise_mul(ret, n6) + self.assertTrue( + np.allclose(static_ret, dy_ret._numpy()), + '%s vs %s' % (static_ret, dy_ret._numpy())) + + def test_elementwise_minmax(self): + n = np.ones([3, 3], dtype='float32') + n2 = np.ones([3, 3], dtype='float32') * 2 + + with self.dynamic_graph(): + min_ret = layers.elementwise_min(n, n2) + max_ret = layers.elementwise_max(n, n2) + + self.assertTrue(np.allclose(n, min_ret._numpy())) + self.assertTrue(np.allclose(n2, max_ret._numpy())) + class TestBook(unittest.TestCase): def test_fit_a_line(self): diff --git a/python/paddle/fluid/tests/unittests/test_slice_op.py b/python/paddle/fluid/tests/unittests/test_slice_op.py index 5fdabbabed..aefd8cb6d3 100644 --- a/python/paddle/fluid/tests/unittests/test_slice_op.py +++ b/python/paddle/fluid/tests/unittests/test_slice_op.py @@ -87,5 +87,31 @@ class TestFP16(TestSliceOp): place, ['Input'], 'Out', max_relative_error=0.006) +@unittest.skipIf(not core.is_compiled_with_cuda(), + "core is not compiled with CUDA") +class TestFP16_2(TestSliceOp): + def config(self): + self.dtype = "float16" + self.input = np.random.random([3, 4, 5]).astype(self.dtype) + self.starts = [0] + self.ends = [1] + self.axes = [1] + self.out = self.input[:, 0:1, :] + + def test_check_output(self): + place = core.CUDAPlace(0) + if core.is_float16_supported(place): + self.check_output_with_place(place, atol=1e-5) + + def test_check_grad_normal(self): + place = core.CUDAPlace(0) + if core.is_float16_supported(place): + self.check_grad_with_place( + place, ['Input'], + 'Out', + max_relative_error=0.006, + numeric_grad_delta=0.5) + + if __name__ == '__main__': unittest.main() diff --git a/python/paddle/fluid/tests/unittests/test_yolo_box_op.py b/python/paddle/fluid/tests/unittests/test_yolo_box_op.py new file mode 100644 index 0000000000..416e6ea9f4 --- /dev/null +++ b/python/paddle/fluid/tests/unittests/test_yolo_box_op.py @@ -0,0 +1,117 @@ +# 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. + +from __future__ import division + +import unittest +import numpy as np +from op_test import OpTest + +from paddle.fluid import core + + +def sigmoid(x): + return 1.0 / (1.0 + np.exp(-1.0 * x)) + + +def YoloBox(x, img_size, attrs): + n, c, h, w = x.shape + anchors = attrs['anchors'] + an_num = int(len(anchors) // 2) + class_num = attrs['class_num'] + conf_thresh = attrs['conf_thresh'] + downsample = attrs['downsample'] + input_size = downsample * h + + x = x.reshape((n, an_num, 5 + class_num, h, w)).transpose((0, 1, 3, 4, 2)) + + pred_box = x[:, :, :, :, :4].copy() + grid_x = np.tile(np.arange(w).reshape((1, w)), (h, 1)) + grid_y = np.tile(np.arange(h).reshape((h, 1)), (1, w)) + pred_box[:, :, :, :, 0] = (grid_x + sigmoid(pred_box[:, :, :, :, 0])) / w + pred_box[:, :, :, :, 1] = (grid_y + sigmoid(pred_box[:, :, :, :, 1])) / h + + anchors = [(anchors[i], anchors[i + 1]) for i in range(0, len(anchors), 2)] + anchors_s = np.array( + [(an_w / input_size, an_h / input_size) for an_w, an_h in anchors]) + anchor_w = anchors_s[:, 0:1].reshape((1, an_num, 1, 1)) + anchor_h = anchors_s[:, 1:2].reshape((1, an_num, 1, 1)) + pred_box[:, :, :, :, 2] = np.exp(pred_box[:, :, :, :, 2]) * anchor_w + pred_box[:, :, :, :, 3] = np.exp(pred_box[:, :, :, :, 3]) * anchor_h + + pred_conf = sigmoid(x[:, :, :, :, 4:5]) + pred_conf[pred_conf < conf_thresh] = 0. + pred_score = sigmoid(x[:, :, :, :, 5:]) * pred_conf + pred_box = pred_box * (pred_conf > 0.).astype('float32') + + pred_box = pred_box.reshape((n, -1, 4)) + pred_box[:, :, :2], pred_box[:, :, 2:4] = \ + pred_box[:, :, :2] - pred_box[:, :, 2:4] / 2., \ + pred_box[:, :, :2] + pred_box[:, :, 2:4] / 2.0 + pred_box[:, :, 0] = pred_box[:, :, 0] * img_size[:, 1][:, np.newaxis] + pred_box[:, :, 1] = pred_box[:, :, 1] * img_size[:, 0][:, np.newaxis] + pred_box[:, :, 2] = pred_box[:, :, 2] * img_size[:, 1][:, np.newaxis] + pred_box[:, :, 3] = pred_box[:, :, 3] * img_size[:, 0][:, np.newaxis] + + for i in range(len(pred_box)): + pred_box[i, :, 0] = np.clip(pred_box[i, :, 0], 0, np.inf) + pred_box[i, :, 1] = np.clip(pred_box[i, :, 1], 0, np.inf) + pred_box[i, :, 2] = np.clip(pred_box[i, :, 2], -np.inf, + img_size[i, 1] - 1) + pred_box[i, :, 3] = np.clip(pred_box[i, :, 3], -np.inf, + img_size[i, 0] - 1) + + return pred_box, pred_score.reshape((n, -1, class_num)) + + +class TestYoloBoxOp(OpTest): + def setUp(self): + self.initTestCase() + self.op_type = 'yolo_box' + x = np.random.random(self.x_shape).astype('float32') + img_size = np.random.randint(10, 20, self.imgsize_shape).astype('int32') + + self.attrs = { + "anchors": self.anchors, + "class_num": self.class_num, + "conf_thresh": self.conf_thresh, + "downsample": self.downsample, + } + + self.inputs = { + 'X': x, + 'ImgSize': img_size, + } + boxes, scores = YoloBox(x, img_size, self.attrs) + self.outputs = { + "Boxes": boxes, + "Scores": scores, + } + + def test_check_output(self): + self.check_output() + + def initTestCase(self): + self.anchors = [10, 13, 16, 30, 33, 23] + an_num = int(len(self.anchors) // 2) + self.batch_size = 32 + self.class_num = 2 + self.conf_thresh = 0.5 + self.downsample = 32 + self.x_shape = (self.batch_size, an_num * (5 + self.class_num), 13, 13) + self.imgsize_shape = (self.batch_size, 2) + + +if __name__ == "__main__": + unittest.main() diff --git a/python/paddle/reader/__init__.py b/python/paddle/reader/__init__.py index 678026cf95..b55a6298f6 100644 --- a/python/paddle/reader/__init__.py +++ b/python/paddle/reader/__init__.py @@ -38,9 +38,8 @@ items. It can be any function with no parameter that creates a iterable Element produced from the iterable should be a **single** entry of data, **not** a mini batch. That entry of data could be a single item, or a tuple of items. -Item should be of `supported type `_ (e.g., numpy 1d -array of float32, int, list of int) +Item should be of supported type (e.g., numpy array or list/tuple of float +or int). An example implementation for single item data reader creator: @@ -62,8 +61,6 @@ An example implementation for multiple item data reader creator: yield numpy.random.uniform(-1, 1, size=width*height), label return reader - -TODO(yuyang18): Should we add whole design doc here? """ import paddle.reader.decorator diff --git a/python/paddle/reader/creator.py b/python/paddle/reader/creator.py index c861020225..353aca92f4 100644 --- a/python/paddle/reader/creator.py +++ b/python/paddle/reader/creator.py @@ -44,8 +44,11 @@ def text_file(path): Creates a data reader that outputs text line by line from given text file. Trailing new line ('\\\\n') of each line will be removed. - :path: path of the text file. - :returns: data reader of text file + Args: + path (str): path of the text file. + + Returns: + callable: data reader of text file. """ def reader(): @@ -59,10 +62,15 @@ def text_file(path): def recordio(paths, buf_size=100): """ - Creates a data reader from given RecordIO file paths separated by ",", - glob pattern is supported. - :path: path of recordio files, can be a string or a string list. - :returns: data reader of recordio files. + Creates a data reader from given RecordIO file paths separated + by ",", glob pattern is supported. + + Args: + paths (str|list(str)): path of recordio files. + buf_size (int): prefetched buffer size. + + Returns: + callable: data reader of recordio files. """ import recordio as rec diff --git a/python/paddle/reader/decorator.py b/python/paddle/reader/decorator.py index b2ef9f7580..685d08b9e0 100644 --- a/python/paddle/reader/decorator.py +++ b/python/paddle/reader/decorator.py @@ -242,20 +242,18 @@ class XmapEndSignal(): def xmap_readers(mapper, reader, process_num, buffer_size, order=False): """ - Use multiprocess to map samples from reader by a mapper defined by user. - And this function contains a buffered decorator. - :param mapper: a function to map sample. - :type mapper: callable - :param reader: the data reader to read from - :type reader: callable - :param process_num: process number to handle original sample - :type process_num: int - :param buffer_size: max buffer size - :type buffer_size: int - :param order: keep the order of reader - :type order: bool - :return: the decarated reader - :rtype: callable + Use multi-threads to map samples from reader by a mapper defined by user. + + Args: + mapper (callable): a function to map the data from reader. + reader (callable): a data reader which yields the data. + process_num (int): thread number to handle original sample. + buffer_size (int): size of the queue to read data in. + order (bool): whether to keep the data order from original reader. + Default False. + + Returns: + callable: a decorated reader with data mapping. """ end = XmapEndSignal() @@ -477,7 +475,7 @@ class PipeReader: """ :param cut_lines: cut buffer to lines :type cut_lines: bool - :param line_break: line break of the file, like \n or \r + :param line_break: line break of the file, like '\\\\n' or '\\\\r' :type line_break: string :return: one line or a buffer of bytes diff --git a/tools/manylinux1/build_scripts/build.sh b/tools/manylinux1/build_scripts/build.sh index 1b0059a8c6..3be94a42d5 100644 --- a/tools/manylinux1/build_scripts/build.sh +++ b/tools/manylinux1/build_scripts/build.sh @@ -153,3 +153,9 @@ done # Restore LD_LIBRARY_PATH LD_LIBRARY_PATH="${ORIGINAL_LD_LIBRARY_PATH}" + +# According to ar issues: https://lists.gnu.org/archive/html/bug-binutils/2016-05/msg00211.html +# we should install new version ar with 64-bit supported here +wget https://ftp.gnu.org/gnu/binutils/binutils-2.27.tar.gz +tar xzf binutils-2.27.tar.gz && cd binutils-2.27 +./configure --prefix=/opt/rh/devtoolset-2/root/usr/ --enable-64-bit-archive && make -j `nproc` && make install From ec88b6cc5aa403ebeca600adbe5f2be99e40f064 Mon Sep 17 00:00:00 2001 From: Zhen Wang Date: Tue, 19 Mar 2019 22:42:26 +0800 Subject: [PATCH 13/44] add channel wise quantization in ir pass. --- paddle/fluid/operators/fake_dequantize_op.h | 46 +++- paddle/fluid/operators/fake_quantize_op.cc | 8 +- paddle/fluid/operators/fake_quantize_op.h | 8 +- .../slim/quantization/quantization_pass.py | 223 +++++++++++++++--- .../slim/tests/test_quantization_pass.py | 28 ++- .../unittests/test_fake_dequantize_op.py | 45 +++- .../tests/unittests/test_fake_quantize_op.py | 2 +- 7 files changed, 290 insertions(+), 70 deletions(-) diff --git a/paddle/fluid/operators/fake_dequantize_op.h b/paddle/fluid/operators/fake_dequantize_op.h index d05f203853..1a504bf035 100644 --- a/paddle/fluid/operators/fake_dequantize_op.h +++ b/paddle/fluid/operators/fake_dequantize_op.h @@ -15,6 +15,7 @@ limitations under the License. */ #pragma once #include +#include "paddle/fluid/framework/ddim.h" #include "paddle/fluid/framework/eigen.h" #include "paddle/fluid/framework/op_registry.h" @@ -54,10 +55,6 @@ class FakeChannelWiseDequantizeMaxAbsKernel : public framework::OpKernel { auto scales = ctx.MultiInput("Scales"); auto* out = ctx.Output("Out"); - PADDLE_ENFORCE_EQ(scales[0]->numel(), in->dims()[0], - "The number of first scale values must be the same with " - "first dimension value of Input(X)."); - auto quant_bits = ctx.Attr>("quant_bits"); int max_range = std::pow(2, quant_bits[0] - 1) - 1; @@ -65,15 +62,38 @@ class FakeChannelWiseDequantizeMaxAbsKernel : public framework::OpKernel { out->mutable_data(dev_ctx.GetPlace()); auto dequant = DequantizeFunctor(); - for (int64_t i = 0; i < in->dims()[0]; i++) { - framework::Tensor one_channel_in = in->Slice(i, i + 1); - framework::Tensor one_channel_out = out->Slice(i, i + 1); - framework::Tensor one_channel_scale = scales[0]->Slice(i, i + 1); - dequant(dev_ctx, &one_channel_in, &one_channel_scale, - static_cast(max_range), &one_channel_out); - } - - if (scales.size() == 2) { + if (scales.size() == 1) { + PADDLE_ENFORCE_EQ( + scales[0]->numel(), in->dims()[0], + "The number of first scale values must be the same with " + "first dimension value of Input(X) when the `Scales` has only one " + "element."); + for (int64_t i = 0; i < in->dims()[0]; i++) { + framework::Tensor one_channel_in = in->Slice(i, i + 1); + framework::Tensor one_channel_out = out->Slice(i, i + 1); + framework::Tensor one_channel_scale = scales[0]->Slice(i, i + 1); + dequant(dev_ctx, &one_channel_in, &one_channel_scale, + static_cast(max_range), &one_channel_out); + } + } else if (scales.size() == 2) { + PADDLE_ENFORCE_EQ( + scales[0]->numel(), in->dims()[1], + "The number of first scale values must be the same with " + "second dimension value of Input(X) when the `Scales` has two " + "elements."); + for (int64_t i = 0; i < in->dims()[0]; i++) { + framework::Tensor one_batch_in = in->Slice(i, i + 1).Resize( + framework::slice_ddim(in->dims(), 1, in->dims().size())); + framework::Tensor one_batch_out = out->Slice(i, i + 1).Resize( + framework::slice_ddim(out->dims(), 1, out->dims().size())); + for (int64_t j = 0; j < in->dims()[1]; j++) { + framework::Tensor one_channel_in = one_batch_in.Slice(j, j + 1); + framework::Tensor one_channel_out = one_batch_out.Slice(j, j + 1); + framework::Tensor one_channel_scale = scales[0]->Slice(j, j + 1); + dequant(dev_ctx, &one_channel_in, &one_channel_scale, + static_cast(max_range), &one_channel_out); + } + } PADDLE_ENFORCE_EQ( scales[1]->numel(), 1, "The second scale tensor should only have one value at now."); diff --git a/paddle/fluid/operators/fake_quantize_op.cc b/paddle/fluid/operators/fake_quantize_op.cc index d51d51b495..f9f28f60c0 100644 --- a/paddle/fluid/operators/fake_quantize_op.cc +++ b/paddle/fluid/operators/fake_quantize_op.cc @@ -169,10 +169,10 @@ class FakeChannelWiseQuantizeAbsMaxOp : public framework::OperatorWithKernel { ctx->HasOutput("Out"), "Output(Out) of FakeChannelWiseQuantizeOp should not be null."); PADDLE_ENFORCE( - ctx->HasOutput("OutScales"), - "Output(Scales) of FakeChannelWiseQuantizeOp should not be null."); + ctx->HasOutput("OutScale"), + "Output(Scale) of FakeChannelWiseQuantizeOp should not be null."); ctx->SetOutputDim("Out", ctx->GetInputDim("X")); - ctx->SetOutputDim("OutScales", {ctx->GetInputDim("X")[0]}); + ctx->SetOutputDim("OutScale", {ctx->GetInputDim("X")[0]}); ctx->ShareLoD("X", /*->*/ "Out"); } @@ -192,7 +192,7 @@ class FakeChannelWiseQuantizeAbsMaxOpMaker AddOutput("Out", "(Tensor) Output of quantized low level tensor, " "but also saved as float data type."); - AddOutput("OutScales", "(Tensor) Current channel wise scale"); + AddOutput("OutScale", "(Tensor) Current channel wise scale"); AddAttr("bit_length", "(int, default 8)") .SetDefault(8) .AddCustomChecker([](const int& bit_length) { diff --git a/paddle/fluid/operators/fake_quantize_op.h b/paddle/fluid/operators/fake_quantize_op.h index ec667e89e7..2616cd996f 100644 --- a/paddle/fluid/operators/fake_quantize_op.h +++ b/paddle/fluid/operators/fake_quantize_op.h @@ -78,8 +78,8 @@ class FakeChannelWiseQuantizeAbsMaxKernel : public framework::OpKernel { auto* in = context.Input("X"); auto* out = context.Output("Out"); - auto* out_scales = context.Output("OutScales"); - T* out_scales_data = out_scales->mutable_data(context.GetPlace()); + auto* out_scale = context.Output("OutScale"); + T* out_scale_data = out_scale->mutable_data(context.GetPlace()); out->mutable_data(context.GetPlace()); int bit_length = context.Attr("bit_length"); @@ -91,13 +91,13 @@ class FakeChannelWiseQuantizeAbsMaxKernel : public framework::OpKernel { framework::Tensor one_channel = in->Slice(i, i + 1); const T* one_channel_data = one_channel.data(); find_abs_max(dev_ctx, one_channel_data, one_channel.numel(), - &out_scales_data[i]); + &out_scale_data[i]); } auto clip_quant = ClipAndFakeQuantFunctor(); for (int64_t i = 0; i < in->dims()[0]; i++) { framework::Tensor one_channel_in = in->Slice(i, i + 1); framework::Tensor one_channel_out = out->Slice(i, i + 1); - framework::Tensor one_channel_scale = out_scales->Slice(i, i + 1); + framework::Tensor one_channel_scale = out_scale->Slice(i, i + 1); clip_quant(dev_ctx, one_channel_in, one_channel_scale, bin_cnt, &one_channel_out); } diff --git a/python/paddle/fluid/contrib/slim/quantization/quantization_pass.py b/python/paddle/fluid/contrib/slim/quantization/quantization_pass.py index 919db4c78e..03ffd2795d 100644 --- a/python/paddle/fluid/contrib/slim/quantization/quantization_pass.py +++ b/python/paddle/fluid/contrib/slim/quantization/quantization_pass.py @@ -22,6 +22,7 @@ from ....framework import IrGraph from ....framework import IrNode from ....framework import Program from ....initializer import Constant +from ....initializer import NumpyArrayInitializer from .... import unique_name __all__ = [ @@ -54,14 +55,15 @@ class QuantizationTransformPass(object): the bias is not quantized. activation_bits (int): quantization bit number for activation. activation_quantize_type (str): quantization type for activation, - now support 'abs_max', 'range_abs_max'. If use 'abs_max' mode, - the quantization scale will be calculated dynamically each step - in both training and testing period. If use 'range_abs_max', - a static quantization scale will be calculated during training - and used in inference. + now support 'abs_max', 'range_abs_max' and 'moving_average_abs_max'. + If use 'abs_max' mode, the quantization scale will be calculated + dynamically each step in both training and testing period. If use + 'range_abs_max', a static quantization scale will be calculated + during training and used in inference. weight_quantize_type (str): quantization type for weights, - support 'abs_max'. The 'range_abs_max' usually is not used for - weight, since weights are fixed once the model is well trained. + support 'abs_max' and 'channel_wise_abs_max'. The 'range_abs_max' + usually is not used for weight, since weights are fixed once the + model is well trained. window_size (int): the window size for 'range_abs_max' quantization. Examples: @@ -84,7 +86,11 @@ class QuantizationTransformPass(object): self._weight_bits = weight_bits self._activation_bits = activation_bits - quant_type = ['abs_max', 'range_abs_max', 'moving_average_abs_max'] + quant_type = [ + 'abs_max', 'channel_wise_abs_max', 'range_abs_max', + 'moving_average_abs_max' + ] + assert activation_quantize_type != 'channel_wise_abs_max', "The activation quantization type does not support 'channel_wise_abs_max'." if activation_quantize_type not in quant_type: raise ValueError( "Unknown activation_quantize_type : '%s'. It can only be ", @@ -93,7 +99,7 @@ class QuantizationTransformPass(object): if weight_quantize_type not in quant_type: raise ValueError( "Unknown weight_quantize_type: '%s'. It can only be ", - "'abs_max' or 'range_abs_max' or 'moving_average_abs_max'.", + "'abs_max' or 'channel_wise_abs_max' or 'range_abs_max' or 'moving_average_abs_max'.", str(weight_quantize_type)) self._activation_quantize_type = activation_quantize_type @@ -103,6 +109,7 @@ class QuantizationTransformPass(object): self._need_initialized = collections.OrderedDict() self._quantizable_ops = ['conv2d', 'depthwise_conv2d', 'mul'] + self._conv_ops = ['conv2d', 'depthwise_conv2d'] self._quantizable_grad_ops = [ '%s_grad' % (op) for op in self._quantizable_ops ] @@ -135,10 +142,26 @@ class QuantizationTransformPass(object): else self._activation_bits quant_type = self._weight_quantize_type if var_node.name() \ in persistable_vars else self._activation_quantize_type - quant_var_node, scale_var_node = self._insert_quant_op( - graph, var_node, quant_bits, quant_type) - dequant_var_node = self._insert_dequant_op( - graph, quant_var_node, scale_var_node, quant_bits) + if quant_type == 'channel_wise_abs_max': + assert var_node.name( + ) in persistable_vars, "'channel_wise_abs_max' can only be applied on weights." + if op.name() in self._conv_ops: + quant_var_node, scale_var_node = self._insert_channel_quant_op( + graph, var_node, quant_bits) + dequant_var_node = self._insert_channel_dequant_op( + graph, quant_var_node, [scale_var_node], + [quant_bits]) + else: + quant_var_node, scale_var_node = self._insert_quant_op( + graph, var_node, quant_bits, 'abs_max') + dequant_var_node = self._insert_dequant_op( + graph, quant_var_node, scale_var_node, + quant_bits) + else: + quant_var_node, scale_var_node = self._insert_quant_op( + graph, var_node, quant_bits, quant_type) + dequant_var_node = self._insert_dequant_op( + graph, quant_var_node, scale_var_node, quant_bits) dequantized_vars[var_node.name()] = dequant_var_node graph.update_input_link(var_node, dequant_var_node, op) @@ -244,7 +267,7 @@ class QuantizationTransformPass(object): scale_var_node = graph.create_var_node( name=self._quantized_scale_name(var_node.name()), var_type=var_node.type(), - shape=var_node.shape(), + shape=[1], var_dtype=var_node.dtype()) quant_op_node = graph.create_op_node( op_type='fake_quantize_abs_max', @@ -384,6 +407,36 @@ class QuantizationTransformPass(object): return quant_var_node, scale_out_node + def _insert_channel_quant_op(self, graph, var_node, quant_bits): + """ + Insert fake_channel_wise_quantize_abs_max op in the graph. + """ + assert var_node.is_var(), '{} is not a var'.format(var_node.name()) + + quant_var_node = graph.create_var_node( + name=self._quantized_var_name(var_node.name()), + var_type=var_node.type(), + shape=var_node.shape(), + var_dtype=var_node.dtype()) + scale_var_node = graph.create_var_node( + name=self._quantized_scale_name(var_node.name()), + var_type=var_node.type(), + shape=[var_node.shape()[0]], + var_dtype=var_node.dtype()) + quant_op_node = graph.create_op_node( + op_type='fake_channel_wise_quantize_abs_max', + attrs={ + 'bit_length': quant_bits, + 'op_role': core.op_proto_and_checker_maker.OpRole.Forward + }, + inputs={'X': var_node}, + outputs={'Out': quant_var_node, + 'OutScale': scale_var_node}) + graph.link_to(var_node, quant_op_node) + graph.link_to(quant_op_node, quant_var_node) + graph.link_to(quant_op_node, scale_var_node) + return quant_var_node, scale_var_node + def _insert_dequant_op(self, graph, var_node, scale_var_node, quant_bits): """ Insert fake_dequantize_op in the graph. @@ -410,6 +463,33 @@ class QuantizationTransformPass(object): graph.link_to(dequant_op_node, dequant_var_node) return dequant_var_node + def _insert_channel_dequant_op(self, graph, var_node, scale_var_nodes, + quant_bits): + """ + Insert fake_channel_wise_dequantize_max_abs in the graph. + """ + assert var_node.is_var(), '{} is not a var'.format(var_node.name()) + + dequant_var_node = graph.create_var_node( + name=self._dequantized_var_name(var_node.name()), + var_type=var_node.type(), + shape=var_node.shape(), + var_dtype=var_node.dtype()) + dequant_op_node = graph.create_op_node( + op_type='fake_channel_wise_dequantize_max_abs', + attrs={ + 'quant_bits': quant_bits, + 'op_role': core.op_proto_and_checker_maker.OpRole.Forward + }, + inputs={'X': var_node, + 'Scales': scale_var_nodes}, + outputs={'Out': dequant_var_node}) + graph.link_to(var_node, dequant_op_node) + for scale_n in scale_var_nodes: + graph.link_to(scale_n, dequant_op_node) + graph.link_to(dequant_op_node, dequant_var_node) + return dequant_var_node + def _quantized_var_name(self, var_name): """ Return quantized variable name for the input `var_name`. @@ -442,7 +522,7 @@ class QuantizationFreezePass(object): place(fluid.CPUPlace|fluid.CUDAPlace): place is used to restore the weight tensors. weight_bits (int): quantization bit number for weights. activation_bits (int): quantization bit number for activation. - weight_quantize_type (str): quantization type for weights, support 'abs_max'. + weight_quantize_type (str): quantization type for weights, support 'abs_max' and 'channel_wise_abs_max'. The 'range_abs_max' usually is not used for weight, since weights are fixed once the model is well trained. """ @@ -463,11 +543,15 @@ class QuantizationFreezePass(object): self._activation_bits = activation_bits self._weight_quantize_type = weight_quantize_type self._quantizable_ops = ['conv2d', 'depthwise_conv2d', 'mul'] + self._conv_ops = ['conv2d', 'depthwise_conv2d'] self._fake_quant_op_names = [ 'fake_quantize_abs_max', 'fake_quantize_range_abs_max', - 'fake_quantize_moving_average_abs_max' + 'fake_quantize_moving_average_abs_max', + 'fake_channel_wise_quantize_abs_max' + ] + self._fake_dequant_op_names = [ + 'fake_dequantize_max_abs', 'fake_channel_wise_dequantize_max_abs' ] - self._fake_dequant_op_names = ['fake_dequantize_max_abs'] self._op_input_rename_map = collections.OrderedDict() self._op_output_rename_map = collections.OrderedDict() self._var_scale_map = collections.OrderedDict() @@ -489,20 +573,29 @@ class QuantizationFreezePass(object): if self._weight_quantize_type == 'abs_max': param = self._load_var(input_arg_name) scale_v = np.max(np.abs(param)) + elif self._weight_quantize_type == 'channel_wise_abs_max': + param = self._load_var(input_arg_name) + if len(param.shape) == 4: # conv2d or depthwise_conv2d + print('DEBUG**************************: %s' % + input_arg_name) + scale_v = [] + for i in range(param.shape[0]): + scale_v.append(np.max(np.abs(param[i]))) + else: + scale_v = np.max(np.abs(param)) else: scale_v = self._load_var( op_node.output('OutScale')[0])[0] self._var_scale_map[input_arg_name] = scale_v - else: - scale_v = graph.var_node(op_node.output('OutScale')[0]) - self._var_scale_map[input_arg_name] = scale_v - if input_arg_name in persistable_vars: self._remove_fake_quant_and_dequant_op(graph, op_node) # quantize weight and restore param_v = self._load_var(input_arg_name) quantized_param_v = self._quant(param_v, scale_v, self._weight_bits) self._restore_var(input_arg_name, quantized_param_v) + else: + scale_v = graph.var_node(op_node.output('OutScale')[0]) + self._var_scale_map[input_arg_name] = scale_v ops = graph.all_op_nodes() for op_node in ops: @@ -514,7 +607,10 @@ class QuantizationFreezePass(object): for op_node in ops: op_name = op_node.name() if op_name in self._quantizable_ops: - self._insert_post_dequant_op(graph, op_node) + if self._weight_quantize_type == 'channel_wise_abs_max' and op_name in self._conv_ops: + self._insert_post_channel_dequant_op(graph, op_node) + else: + self._insert_post_dequant_op(graph, op_node) for op_node in ops: # insert dequant_op after fc/conv, need to rename inputs of the followed ops @@ -538,9 +634,73 @@ class QuantizationFreezePass(object): self._op_input_rename_map[k] = self._op_input_rename_map[v] graph.safe_remove_nodes(op_node) + def _insert_post_channel_dequant_op(self, graph, op_node): + persistable_vars = [p.name() for p in graph.all_persistable_nodes()] + for var_node in op_node.inputs: + name = var_node.name() + if name in self._op_input_rename_map: + old_in = graph.var_node(name) + new_in = graph.var_node(self._op_input_rename_map[name]) + new_in.clear_outputs() + graph.update_input_link(old_in, new_in, op_node) + original_var_name = self._original_var_name(name) + scale_v = self._var_scale_map[original_var_name] + if original_var_name in persistable_vars: + assert isinstance( + scale_v, + list), 'The scale of parameter %s is not a list.' % ( + original_var_name) + channel_scale = np.array(scale_v) + else: + assert isinstance(scale_v, IrNode) + scale_var_node = self._var_scale_map[original_var_name] + + if len(op_node.outputs) != 1: + raise ValueError("Only support one output, but op %s has" + " more than one output." % (op_node.name())) + + output_var_node = op_node.outputs[0] + weight_scale_node = graph.create_persistable_node( + name=unique_name.generate('channel_scale'), + var_type=core.VarDesc.VarType.LOD_TENSOR, + shape=[channel_scale.shape[0]], + var_dtype=output_var_node.dtype()) + init_program = Program() + weight_scale_var = init_program.global_block().create_var( + name=weight_scale_node.name(), + shape=weight_scale_node.shape(), + dtype=weight_scale_node.dtype(), + type=weight_scale_node.type(), + lod_level=weight_scale_node.var().lod_level(), + persistable=weight_scale_node.persistable()) + initializer = NumpyArrayInitializer(value=channel_scale) + initializer(weight_scale_var, init_program.global_block()) + exe = Executor(self._place) + exe.run(program=init_program, scope=self._scope) + dequant_var_node = graph.create_var_node( + name=self._dequantized_var_name(output_var_node.name()), + var_type=output_var_node.type(), + shape=output_var_node.shape(), + var_dtype=output_var_node.dtype()) + dequant_op_node = graph.create_op_node( + op_type='fake_channel_wise_dequantize_max_abs', + attrs={ + 'quant_bits': [self._weight_bits, self._activation_bits], + 'op_role': core.op_proto_and_checker_maker.OpRole.Forward + }, + inputs={ + 'X': output_var_node, + 'Scales': [weight_scale_node, scale_var_node] + }, + outputs={'Out': dequant_var_node}) + graph.link_to(output_var_node, dequant_op_node) + graph.link_to(scale_var_node, dequant_op_node) + graph.link_to(weight_scale_node, dequant_op_node) + graph.link_to(dequant_op_node, dequant_var_node) + self._op_output_rename_map[output_var_node.name()] = dequant_var_node + return dequant_var_node + def _insert_post_dequant_op(self, graph, op_node): - max_range = None - scale_var_node = None persistable_vars = [p.name() for p in graph.all_persistable_nodes()] for var_node in op_node.inputs: name = var_node.name() @@ -637,7 +797,12 @@ class QuantizationFreezePass(object): or isinstance(v, np.float64) def _quant(self, x, scale, num_bits): - return np.round(x / scale * ((1 << (num_bits - 1)) - 1)) + if isinstance(scale, list): + for i, s in enumerate(scale): + x[i] = np.round(x[i] / s * ((1 << (num_bits - 1)) - 1)) + return x + else: + return np.round(x / scale * ((1 << (num_bits - 1)) - 1)) class ConvertToInt8Pass(object): @@ -731,9 +896,13 @@ class TransformForMobilePass(object): def __init__(self): self._fake_quant_op_names = [ - 'fake_quantize_abs_max', 'fake_quantize_range_abs_max' + 'fake_quantize_abs_max', 'fake_quantize_range_abs_max', + 'fake_quantize_moving_average_abs_max', + 'fake_channel_wise_quantize_abs_max' + ] + self._fake_dequant_op_names = [ + 'fake_dequantize_max_abs', 'fake_channel_wise_dequantize_max_abs' ] - self._fake_dequant_op_names = ['fake_dequantize_max_abs'] def apply(self, graph): """ diff --git a/python/paddle/fluid/contrib/slim/tests/test_quantization_pass.py b/python/paddle/fluid/contrib/slim/tests/test_quantization_pass.py index 0b4b2a285f..cda7ecbd82 100644 --- a/python/paddle/fluid/contrib/slim/tests/test_quantization_pass.py +++ b/python/paddle/fluid/contrib/slim/tests/test_quantization_pass.py @@ -243,7 +243,12 @@ class TestQuantizationFreezePass(unittest.TestCase): with fluid.scope_guard(scope): exe.run(startup) transform_pass = QuantizationTransformPass( - scope=scope, place=place, activation_quantize_type=quant_type) + scope=scope, + place=place, + activation_quantize_type=quant_type, + weight_quantize_type='channel_wise_abs_max') + #transform_pass = QuantizationTransformPass( + # scope=scope, place=place, activation_quantize_type=quant_type) transform_pass.apply(main_graph) transform_pass.apply(test_graph) dev_name = '_gpu_' if use_cuda else '_cpu_' @@ -296,7 +301,11 @@ class TestQuantizationFreezePass(unittest.TestCase): fetch_list=[loss, w_var]) # Freeze graph for inference, but the weight of fc/conv is still float type. - freeze_pass = QuantizationFreezePass(scope=scope, place=place) + freeze_pass = QuantizationFreezePass( + scope=scope, + place=place, + weight_quantize_type='channel_wise_abs_max') + #freeze_pass = QuantizationFreezePass(scope=scope, place=place) freeze_pass.apply(test_graph) if not for_ci: marked_nodes = set() @@ -375,29 +384,32 @@ class TestQuantizationFreezePass(unittest.TestCase): if fluid.core.is_compiled_with_cuda(): with fluid.unique_name.guard(): self.freeze_graph( - True, seed=1, quant_type='abs_max', for_ci=True) + True, seed=1, quant_type='abs_max', for_ci=False) def test_freeze_graph_cpu_dynamic(self): with fluid.unique_name.guard(): - self.freeze_graph(False, seed=2, quant_type='abs_max', for_ci=True) + self.freeze_graph(False, seed=2, quant_type='abs_max', for_ci=False) def test_freeze_graph_cuda_static(self): if fluid.core.is_compiled_with_cuda(): with fluid.unique_name.guard(): self.freeze_graph( - True, seed=1, quant_type='range_abs_max', for_ci=True) + True, seed=1, quant_type='range_abs_max', for_ci=False) self.freeze_graph( True, seed=1, quant_type='moving_average_abs_max', - for_ci=True) + for_ci=False) def test_freeze_graph_cpu_static(self): with fluid.unique_name.guard(): self.freeze_graph( - False, seed=2, quant_type='range_abs_max', for_ci=True) + False, seed=2, quant_type='range_abs_max', for_ci=False) self.freeze_graph( - False, seed=2, quant_type='moving_average_abs_max', for_ci=True) + False, + seed=2, + quant_type='moving_average_abs_max', + for_ci=False) if __name__ == '__main__': diff --git a/python/paddle/fluid/tests/unittests/test_fake_dequantize_op.py b/python/paddle/fluid/tests/unittests/test_fake_dequantize_op.py index 32cb23cbfa..0812b02b47 100644 --- a/python/paddle/fluid/tests/unittests/test_fake_dequantize_op.py +++ b/python/paddle/fluid/tests/unittests/test_fake_dequantize_op.py @@ -31,15 +31,27 @@ def dequantize_max_abs(x, scale, max_range): return y -def channel_wise_quantize_max_abs(x, quant_bit=8): +def channel_wise_quantize_max_abs(x, quant_bit=8, use_second_dim=False): scales = [] - for i in range(x.shape[0]): - scales.append(np.max(np.abs(x[i])).astype("float32")) - - y = x.copy() - max_range = math.pow(2, quant_bit - 1) - 1 - for i, scale in enumerate(scales): - y[i] = np.round(y[i] / scale * max_range) + if not use_second_dim: + for i in range(x.shape[0]): + scales.append(np.max(np.abs(x[i])).astype("float32")) + y = x.copy() + max_range = math.pow(2, quant_bit - 1) - 1 + for i, scale in enumerate(scales): + y[i] = np.round(x[i] / scale * max_range) + else: + for i in range(x.shape[0]): + s = [] + for j in range(x.shape[1]): + s.append(np.max(np.abs(x[i][j])).astype("float32")) + scales.append(s) + scales = np.amax(np.array(scales), axis=0) + y = x.copy() + max_range = math.pow(2, quant_bit - 1) - 1 + for i in range(x.shape[0]): + for j, scale in enumerate(scales): + y[i][j] = np.round(x[i][j] / scale * max_range) return y, scales @@ -47,10 +59,16 @@ def channel_wise_dequantize_max_abs(x, scales, quant_bits, activation_scale=None): - y = x.copy() - for i in range(x.shape[0]): - y[i] = (scales[i] / (math.pow(2, quant_bits[0] - 1) - 1)) * y[i] - if activation_scale is not None: + if activation_scale is None: + y = x.copy() + for i in range(x.shape[0]): + y[i] = (scales[i] / (math.pow(2, quant_bits[0] - 1) - 1)) * x[i] + else: + y = x.copy() + for i in range(x.shape[0]): + for j in range(x.shape[1]): + y[i][j] = (scales[j] / + (math.pow(2, quant_bits[0] - 1) - 1)) * x[i][j] y *= activation_scale / (math.pow(2, quant_bits[1] - 1) - 1) return y @@ -65,7 +83,8 @@ class TestFakeChannelWiseDequantizeMaxAbsOpTwoScales(OpTest): self.set_args() self.op_type = "fake_channel_wise_dequantize_max_abs" x = np.random.randn(4, 3, 64, 64).astype(self.data_type) - yq, scales = channel_wise_quantize_max_abs(x, self.quant_bits[0]) + yq, scales = channel_wise_quantize_max_abs( + x, self.quant_bits[0], use_second_dim=True) ydq = channel_wise_dequantize_max_abs(yq, scales, self.quant_bits, self.activation_scale) diff --git a/python/paddle/fluid/tests/unittests/test_fake_quantize_op.py b/python/paddle/fluid/tests/unittests/test_fake_quantize_op.py index cf8f01edb9..07038b0441 100644 --- a/python/paddle/fluid/tests/unittests/test_fake_quantize_op.py +++ b/python/paddle/fluid/tests/unittests/test_fake_quantize_op.py @@ -53,7 +53,7 @@ class TestFakeChannelWiseQuantizeOp(OpTest): self.outputs = { 'Out': outputs, - 'OutScales': np.array(scales).astype("float32"), + 'OutScale': np.array(scales).astype("float32"), } def test_check_output(self): From bed0ecf3d23567b9102cde9bf9338e666c2e4b1a Mon Sep 17 00:00:00 2001 From: lujun Date: Wed, 20 Mar 2019 02:05:35 +0800 Subject: [PATCH 14/44] checkpoint pr be moved here, test=develop --- paddle/fluid/operators/load_combine_op.cc | 146 +++++++------ paddle/fluid/operators/load_op.cc | 94 +++++---- paddle/fluid/operators/save_combine_op.cc | 109 +++++----- .../operators/save_load_combine_op_test.cc | 4 +- paddle/fluid/operators/save_load_op_test.cc | 4 +- paddle/fluid/operators/save_op.cc | 164 +++++++++------ python/paddle/fluid/framework.py | 7 +- python/paddle/fluid/imperative/__init__.py | 4 + python/paddle/fluid/imperative/checkpoint.py | 194 ++++++++++++++++++ python/paddle/fluid/imperative/layers.py | 28 +++ .../unittests/test_imperative_checkpoint.py | 163 +++++++++++++++ 11 files changed, 691 insertions(+), 226 deletions(-) create mode 100644 python/paddle/fluid/imperative/checkpoint.py create mode 100644 python/paddle/fluid/tests/unittests/test_imperative_checkpoint.py diff --git a/paddle/fluid/operators/load_combine_op.cc b/paddle/fluid/operators/load_combine_op.cc index f5c802986e..3180f16576 100644 --- a/paddle/fluid/operators/load_combine_op.cc +++ b/paddle/fluid/operators/load_combine_op.cc @@ -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 limitations under the License. */ #include +#include "paddle/fluid/framework/data_type.h" #include "paddle/fluid/framework/data_type_transform.h" #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/platform/device_context.h" @@ -19,21 +20,71 @@ limitations under the License. */ namespace paddle { namespace operators { -class LoadCombineOp : public framework::OperatorBase { +class LoadCombineOp : public framework::OperatorWithKernel { public: - LoadCombineOp(const std::string &type, - const framework::VariableNameMap &inputs, - const framework::VariableNameMap &outputs, - const framework::AttributeMap &attrs) - : OperatorBase(type, inputs, outputs, attrs) {} - - private: - void RunImpl(const framework::Scope &scope, - const platform::Place &place) const override { - auto filename = Attr("file_path"); - auto load_as_fp16 = Attr("load_as_fp16"); - auto model_from_memory = Attr("model_from_memory"); - auto out_var_names = Outputs("Out"); + using framework::OperatorWithKernel::OperatorWithKernel; + + void InferShape(framework::InferShapeContext *ctx) const override {} + + protected: + framework::OpKernelType GetExpectedKernelType( + const framework::ExecutionContext &ctx) const override { + framework::OpKernelType kt = framework::OpKernelType( + framework::proto::VarType::FP32, platform::CPUPlace()); + return kt; + } +}; + +class LoadCombineOpProtoMaker : public framework::OpProtoAndCheckerMaker { + public: + void Make() override { + AddOutput( + "Out", + "(vector) The output LoDTensors that will be read from the input file.") + .AsDuplicable(); + AddAttr( + "load_as_fp16", + "(boolean, default false)" + "If true, the tensor will be first loaded and then " + "converted to float16 data type. Otherwise, the tensor will be " + "directly loaded without data type conversion.") + .SetDefault(false); + AddAttr("file_path", + "(string) " + "LoDTensors will be loaded from \"file_path\".") + .AddCustomChecker( + [](const std::string &path) { return !path.empty(); }); + AddAttr("model_from_memory", + "(boolean, default false)" + "If true, file_path is in memory, and LoDTensors will be " + "loaded directly from memory") + .SetDefault(false); + AddComment(R"DOC( +LoadCombine Operator. + +LoadCombine operator loads LoDTensor variables from a file, which could be +loaded in memory already. The file should contain one or more LoDTensors +serialized using the SaveCombine operator. The +LoadCombine operator applies a deserialization strategy to appropriately load +the LodTensors, and this strategy complements the serialization strategy used +in the SaveCombine operator. Hence, the LoadCombine operator is tightly coupled +with the SaveCombine operator, and can only deserialize one or more LoDTensors +that were saved using the SaveCombine operator. + +)DOC"); + } +}; + +template +class LoadCombineOpKernel : public framework::OpKernel { + public: + void Compute(const framework::ExecutionContext &ctx) const override { + auto place = ctx.GetPlace(); + auto filename = ctx.Attr("file_path"); + auto load_as_fp16 = ctx.Attr("load_as_fp16"); + auto model_from_memory = ctx.Attr("model_from_memory"); + auto &out_var_names = ctx.Outputs("Out"); + PADDLE_ENFORCE_GT( static_cast(out_var_names.size()), 0, "The number of output variables should be greater than 0."); @@ -41,27 +92,27 @@ class LoadCombineOp : public framework::OperatorBase { std::ifstream fin(filename, std::ios::binary); PADDLE_ENFORCE(static_cast(fin), "Cannot open file %s for load_combine op", filename); - LoadParamsFromBuffer(scope, place, &fin, load_as_fp16, out_var_names); + LoadParamsFromBuffer(ctx, place, &fin, load_as_fp16, out_var_names); } else { PADDLE_ENFORCE(!filename.empty(), "Cannot load file from memory"); std::stringstream fin(filename, std::ios::in | std::ios::binary); - LoadParamsFromBuffer(scope, place, &fin, load_as_fp16, out_var_names); + LoadParamsFromBuffer(ctx, place, &fin, load_as_fp16, out_var_names); } } + void LoadParamsFromBuffer( - const framework::Scope &scope, const platform::Place &place, + const framework::ExecutionContext &context, const platform::Place &place, std::istream *buffer, bool load_as_fp16, const std::vector &out_var_names) const { platform::DeviceContextPool &pool = platform::DeviceContextPool::Instance(); auto &dev_ctx = *pool.Get(place); + auto out_vars = context.MultiOutputVar("Out"); for (size_t i = 0; i < out_var_names.size(); i++) { - auto *out_var = scope.FindVar(out_var_names[i]); + PADDLE_ENFORCE(out_vars[i] != nullptr, + "Output variable %s cannot be found", out_var_names[i]); - PADDLE_ENFORCE(out_var != nullptr, "Output variable %s cannot be found", - out_var_names[i]); - - auto *tensor = out_var->GetMutable(); + auto *tensor = out_vars[i]->GetMutable(); // Error checking PADDLE_ENFORCE(static_cast(*buffer), "Cannot read more"); @@ -84,8 +135,8 @@ class LoadCombineOp : public framework::OperatorBase { &fp16_tensor); // reset output tensor - out_var->Clear(); - tensor = out_var->GetMutable(); + out_vars[i]->Clear(); + tensor = out_vars[i]->GetMutable(); tensor->set_lod(fp16_tensor.lod()); tensor->ShareDataWith(fp16_tensor); } @@ -97,48 +148,17 @@ class LoadCombineOp : public framework::OperatorBase { } }; -class LoadCombineOpProtoMaker : public framework::OpProtoAndCheckerMaker { - public: - void Make() override { - AddOutput( - "Out", - "(vector) The output LoDTensors that will be read from the input file.") - .AsDuplicable(); - AddAttr( - "load_as_fp16", - "(boolean, default false)" - "If true, the tensor will be first loaded and then " - "converted to float16 data type. Otherwise, the tensor will be " - "directly loaded without data type conversion.") - .SetDefault(false); - AddAttr("file_path", - "(string) " - "LoDTensors will be loaded from \"file_path\".") - .AddCustomChecker( - [](const std::string &path) { return !path.empty(); }); - AddAttr("model_from_memory", - "(boolean, default false)" - "If true, file_path is in memory, and LoDTensors will be " - "loaded directly from memory") - .SetDefault(false); - AddComment(R"DOC( -LoadCombine Operator. - -LoadCombine operator loads LoDTensor variables from a file, which could be -loaded in memory already. The file should contain one or more LoDTensors -serialized using the SaveCombine operator. The -LoadCombine operator applies a deserialization strategy to appropriately load -the LodTensors, and this strategy complements the serialization strategy used -in the SaveCombine operator. Hence, the LoadCombine operator is tightly coupled -with the SaveCombine operator, and can only deserialize one or more LoDTensors -that were saved using the SaveCombine operator. - -)DOC"); - } -}; } // namespace operators } // namespace paddle + namespace ops = paddle::operators; REGISTER_OPERATOR(load_combine, ops::LoadCombineOp, ops::LoadCombineOpProtoMaker); + +REGISTER_OP_CPU_KERNEL( + load_combine, + ops::LoadCombineOpKernel, + ops::LoadCombineOpKernel, + ops::LoadCombineOpKernel, + ops::LoadCombineOpKernel); diff --git a/paddle/fluid/operators/load_op.cc b/paddle/fluid/operators/load_op.cc index 4bce4eba22..befe0ea018 100644 --- a/paddle/fluid/operators/load_op.cc +++ b/paddle/fluid/operators/load_op.cc @@ -21,31 +21,63 @@ limitations under the License. */ namespace paddle { namespace operators { -class LoadOp : public framework::OperatorBase { +class LoadOp : public framework::OperatorWithKernel { public: - LoadOp(const std::string &type, const framework::VariableNameMap &inputs, - const framework::VariableNameMap &outputs, - const framework::AttributeMap &attrs) - : OperatorBase(type, inputs, outputs, attrs) {} - - private: - void RunImpl(const framework::Scope &scope, - const platform::Place &place) const override { + using framework::OperatorWithKernel::OperatorWithKernel; + + void InferShape(framework::InferShapeContext *ctx) const override {} + + protected: + framework::OpKernelType GetExpectedKernelType( + const framework::ExecutionContext &ctx) const override { + framework::OpKernelType kt = framework::OpKernelType( + framework::proto::VarType::FP32, platform::CPUPlace()); + return kt; + } +}; + +class LoadOpProtoMaker : public framework::OpProtoAndCheckerMaker { + public: + void Make() override { + AddOutput("Out", "The LoDTensor / SelectedRows need to be loaded"); + AddAttr( + "load_as_fp16", + "If true, the tensor will be first loaded and then " + "converted to float16 data type. Otherwise, the tensor will be " + "directly loaded without data type conversion. Default is false.") + .SetDefault(false); + AddAttr("file_path", + R"(Variable will be loaded from "file_path")") + .AddCustomChecker( + [](const std::string &path) { return !path.empty(); }); + AddComment( + "Load operator will load a LoDTensor / SelectedRows variable from disk " + "file."); + } +}; + +template +class LoadOpKernel : public framework::OpKernel { + public: + void Compute(const framework::ExecutionContext &ctx) const override { + auto place = ctx.GetPlace(); // FIXME(yuyang18): We save variable to local file now, but we should change // it to save an output stream. - auto filename = Attr("file_path"); + auto filename = ctx.Attr("file_path"); std::ifstream fin(filename, std::ios::binary); PADDLE_ENFORCE(static_cast(fin), "Cannot open file %s for load op", filename); - auto out_var_name = Output("Out"); - auto *out_var = scope.FindVar(out_var_name); - PADDLE_ENFORCE(out_var != nullptr, - "Output variable %s cannot be found in scope %p", - out_var_name, &scope); + auto out_var_name = ctx.Outputs("Out").data(); + auto *out_var = ctx.OutputVar("Out"); + + PADDLE_ENFORCE(out_var != nullptr, "Output variable %s cannot be found ", + out_var_name); + + PADDLE_ENFORCE(out_var != nullptr, "Output variable cannot be found "); if (out_var->IsType()) { - LoadLodTensor(fin, place, out_var); + LoadLodTensor(fin, place, out_var, ctx); } else if (out_var->IsType()) { LoadSelectedRows(fin, place, out_var); } else { @@ -57,14 +89,15 @@ class LoadOp : public framework::OperatorBase { } void LoadLodTensor(std::istream &fin, const platform::Place &place, - framework::Variable *var) const { + framework::Variable *var, + const framework::ExecutionContext &ctx) const { // get device context from pool platform::DeviceContextPool &pool = platform::DeviceContextPool::Instance(); auto &dev_ctx = *pool.Get(place); auto *tensor = var->GetMutable(); DeserializeFromStream(fin, tensor, dev_ctx); - auto load_as_fp16 = Attr("load_as_fp16"); + auto load_as_fp16 = ctx.Attr("load_as_fp16"); auto in_dtype = tensor->type(); auto out_dtype = load_as_fp16 ? framework::proto::VarType::FP16 : in_dtype; @@ -97,27 +130,14 @@ class LoadOp : public framework::OperatorBase { } }; -class LoadOpProtoMaker : public framework::OpProtoAndCheckerMaker { - public: - void Make() override { - AddOutput("Out", "The LoDTensor / SelectedRows need to be loaded"); - AddAttr( - "load_as_fp16", - "If true, the tensor will be first loaded and then " - "converted to float16 data type. Otherwise, the tensor will be " - "directly loaded without data type conversion. Default is false.") - .SetDefault(false); - AddAttr("file_path", - R"(Variable will be loaded from "file_path")") - .AddCustomChecker( - [](const std::string &path) { return !path.empty(); }); - AddComment( - "Load operator will load a LoDTensor / SelectedRows variable from disk " - "file."); - } -}; } // namespace operators } // namespace paddle namespace ops = paddle::operators; REGISTER_OPERATOR(load, ops::LoadOp, ops::LoadOpProtoMaker); + +REGISTER_OP_CPU_KERNEL( + load, ops::LoadOpKernel, + ops::LoadOpKernel, + ops::LoadOpKernel, + ops::LoadOpKernel); diff --git a/paddle/fluid/operators/save_combine_op.cc b/paddle/fluid/operators/save_combine_op.cc index d0edcc170f..7110925565 100644 --- a/paddle/fluid/operators/save_combine_op.cc +++ b/paddle/fluid/operators/save_combine_op.cc @@ -27,20 +27,53 @@ limitations under the License. */ namespace paddle { namespace operators { -class SaveCombineOp : public framework::OperatorBase { +class SaveCombineOp : public framework::OperatorWithKernel { public: - SaveCombineOp(const std::string &type, - const framework::VariableNameMap &inputs, - const framework::VariableNameMap &outputs, - const framework::AttributeMap &attrs) - : OperatorBase(type, inputs, outputs, attrs) {} - - private: - void RunImpl(const framework::Scope &scope, - const platform::Place &place) const override { - auto filename = Attr("file_path"); - auto overwrite = Attr("overwrite"); - auto save_as_fp16 = Attr("save_as_fp16"); + using framework::OperatorWithKernel::OperatorWithKernel; + + void InferShape(framework::InferShapeContext *ctx) const override {} +}; + +class SaveCombineOpProtoMaker : public framework::OpProtoAndCheckerMaker { + public: + void Make() override { + AddInput( + "X", + "(vector) Input LoDTensors that need to be saved together in a file.") + .AsDuplicable(); + AddComment(R"DOC( +SaveCombine operator + +This operator will serialize and write a list of input LoDTensor variables +to a file on disk. +)DOC"); + AddAttr("overwrite", + "(boolean, default true)" + "Overwrite the output file if it exists.") + .SetDefault(true); + AddAttr("save_as_fp16", + "(boolean, default false)" + "If true, the tensor will be converted to float16 data " + "type and then saved. Otherwise, the tensor will be " + "directly saved without data type conversion.") + .SetDefault(false); + AddAttr( + "file_path", + "(string)" + "The \"file_path\" where the LoDTensor variables will be saved.") + .AddCustomChecker( + [](const std::string &path) { return !path.empty(); }); + } +}; + +template +class SaveCombineOpKernel : public framework::OpKernel { + public: + void Compute(const framework::ExecutionContext &ctx) const override { + auto place = ctx.GetPlace(); + auto filename = ctx.Attr("file_path"); + auto overwrite = ctx.Attr("overwrite"); + auto save_as_fp16 = ctx.Attr("save_as_fp16"); bool is_present = FileExists(filename); if (is_present && !overwrite) { @@ -53,7 +86,8 @@ class SaveCombineOp : public framework::OperatorBase { PADDLE_ENFORCE(static_cast(fout), "Cannot open %s to write", filename); - auto inp_var_names = Inputs("X"); + auto &inp_var_names = ctx.Inputs("X"); + auto &inp_vars = ctx.MultiInputVar("X"); PADDLE_ENFORCE_GT(static_cast(inp_var_names.size()), 0, "The number of input variables should be greater than 0"); @@ -62,16 +96,14 @@ class SaveCombineOp : public framework::OperatorBase { auto &dev_ctx = *pool.Get(place); for (size_t i = 0; i < inp_var_names.size(); i++) { - auto *var = scope.FindVar(inp_var_names[i]); - - PADDLE_ENFORCE(var != nullptr, + PADDLE_ENFORCE(inp_vars[i] != nullptr, "Cannot find variable %s for save_combine_op", inp_var_names[i]); - PADDLE_ENFORCE(var->IsType(), + PADDLE_ENFORCE(inp_vars[i]->IsType(), "SaveCombineOp only supports LoDTensor, %s has wrong type", inp_var_names[i]); - auto &tensor = var->Get(); + auto &tensor = inp_vars[i]->Get(); // Serialize tensors one by one // Check types to see if a fp16 transformation is required @@ -95,38 +127,6 @@ class SaveCombineOp : public framework::OperatorBase { } }; -class SaveCombineOpProtoMaker : public framework::OpProtoAndCheckerMaker { - public: - void Make() override { - AddInput( - "X", - "(vector) Input LoDTensors that need to be saved together in a file.") - .AsDuplicable(); - AddComment(R"DOC( -SaveCombine operator - -This operator will serialize and write a list of input LoDTensor variables -to a file on disk. -)DOC"); - AddAttr("overwrite", - "(boolean, default true)" - "Overwrite the output file if it exists.") - .SetDefault(true); - AddAttr("save_as_fp16", - "(boolean, default false)" - "If true, the tensor will be converted to float16 data " - "type and then saved. Otherwise, the tensor will be " - "directly saved without data type conversion.") - .SetDefault(false); - AddAttr( - "file_path", - "(string)" - "The \"file_path\" where the LoDTensor variables will be saved.") - .AddCustomChecker( - [](const std::string &path) { return !path.empty(); }); - } -}; - } // namespace operators } // namespace paddle @@ -134,3 +134,10 @@ namespace ops = paddle::operators; REGISTER_OPERATOR(save_combine, ops::SaveCombineOp, ops::SaveCombineOpProtoMaker); + +REGISTER_OP_CPU_KERNEL( + save_combine, + ops::SaveCombineOpKernel, + ops::SaveCombineOpKernel, + ops::SaveCombineOpKernel, + ops::SaveCombineOpKernel); diff --git a/paddle/fluid/operators/save_load_combine_op_test.cc b/paddle/fluid/operators/save_load_combine_op_test.cc index 4743e0d949..5594de16b6 100644 --- a/paddle/fluid/operators/save_load_combine_op_test.cc +++ b/paddle/fluid/operators/save_load_combine_op_test.cc @@ -19,8 +19,8 @@ limitations under the License. */ #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/platform/float16.h" -USE_NO_KERNEL_OP(save_combine); -USE_NO_KERNEL_OP(load_combine); +USE_CPU_ONLY_OP(save_combine); +USE_CPU_ONLY_OP(load_combine); template T* CreateForSaveCombineOp(int x, int y, const std::vector& lod_info, diff --git a/paddle/fluid/operators/save_load_op_test.cc b/paddle/fluid/operators/save_load_op_test.cc index ccaea0eef2..d277198a2f 100644 --- a/paddle/fluid/operators/save_load_op_test.cc +++ b/paddle/fluid/operators/save_load_op_test.cc @@ -16,8 +16,8 @@ limitations under the License. */ #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/platform/float16.h" -USE_NO_KERNEL_OP(save); -USE_NO_KERNEL_OP(load); +USE_CPU_ONLY_OP(save); +USE_CPU_ONLY_OP(load); TEST(SaveLoadOp, CPU) { paddle::framework::Scope scope; diff --git a/paddle/fluid/operators/save_op.cc b/paddle/fluid/operators/save_op.cc index b02c098099..8f61525436 100644 --- a/paddle/fluid/operators/save_op.cc +++ b/paddle/fluid/operators/save_op.cc @@ -15,6 +15,7 @@ limitations under the License. */ #include #include #include +#include #include "paddle/fluid/framework/data_type.h" #include "paddle/fluid/framework/data_type_transform.h" @@ -29,29 +30,88 @@ limitations under the License. */ namespace paddle { namespace operators { +using Tensor = framework::Tensor; +using LoDTensor = framework::LoDTensor; + // define LOOKUP_TABLE_PATH for checkpoint notify to save lookup table variables // to directory specified. constexpr char LOOKUP_TABLE_PATH[] = "kLookupTablePath"; -class SaveOp : public framework::OperatorBase { +class SaveOp : public framework::OperatorWithKernel { public: - SaveOp(const std::string &type, const framework::VariableNameMap &inputs, - const framework::VariableNameMap &outputs, - const framework::AttributeMap &attrs) - : OperatorBase(type, inputs, outputs, attrs) {} - - private: - void RunImpl(const framework::Scope &scope, - const platform::Place &place) const override { - auto iname = Input("X"); - auto *var = scope.FindVar(iname); - PADDLE_ENFORCE(var != nullptr, "Cannot find variable %s for save_op", + using framework::OperatorWithKernel::OperatorWithKernel; + + void InferShape(framework::InferShapeContext *ctx) const override {} + + protected: + framework::OpKernelType GetExpectedKernelType( + const framework::ExecutionContext &ctx) const override { + return framework::OpKernelType(ctx.Input("X")->type(), + ctx.GetPlace()); + } +}; + +class SaveOpProtoMaker : public framework::OpProtoAndCheckerMaker { + public: + void Make() override { + AddInput("X", "(Tensor ) Input LoDTensor and SelectedRows to be saved"); + AddComment(R"DOC( +Save operator + +This operator will serialize and write LoDTensor / SelectedRows variable to file on disk. +)DOC"); + AddAttr("overwrite", + "(boolean, default true)" + "Overwrite the output file if exist") + .SetDefault(true); + AddAttr("save_as_fp16", + "(boolean, default false)" + "If true, the tensor will be converted to float16 data " + "type and then saved. Otherwise, the tensor will be " + "directly saved without data type conversion.") + .SetDefault(false); + AddAttr("file_path", + "(string)" + "The \"file_path\" where the variable will be saved.") + .AddCustomChecker( + [](const std::string &path) { return !path.empty(); }); + AddOutput(LOOKUP_TABLE_PATH, + "(string)" + "for pserver: The \"kLookupTablePath\" where checkpoint notify " + "to save lookup table variables" + " to directory specified.") + .AsDispensable(); + } +}; + +class SaveOpVarTypeInference : public framework::VarTypeInference { + public: + void operator()(framework::InferVarTypeContext *ctx) const override { + auto var_type = framework::proto::VarType::RAW; + ctx->SetType(LOOKUP_TABLE_PATH, var_type); + } +}; + +class SaveOpShapeInference : public framework::InferShapeBase { + public: + void operator()(framework::InferShapeContext *ctx) const override {} +}; + +template +class SaveOpKernel : public framework::OpKernel { + public: + void Compute(const framework::ExecutionContext &ctx) const override { + auto place = ctx.GetPlace(); + + auto *input_var = ctx.InputVar("X"); + auto iname = ctx.Inputs("X").data(); + PADDLE_ENFORCE(input_var != nullptr, "Cannot find variable %s for save_op", iname); - if (var->IsType()) { - SaveLodTensor(place, var); - } else if (var->IsType()) { - SaveSelectedRows(scope, place, var); + if (input_var->IsType()) { + SaveLodTensor(ctx, place, input_var); + } else if (input_var->IsType()) { + SaveSelectedRows(ctx, place, input_var); } else { PADDLE_ENFORCE( false, @@ -60,10 +120,11 @@ class SaveOp : public framework::OperatorBase { } } - void SaveLodTensor(const platform::Place &place, - framework::Variable *var) const { - auto filename = Attr("file_path"); - auto overwrite = Attr("overwrite"); + void SaveLodTensor(const framework::ExecutionContext &ctx, + const platform::Place &place, + const framework::Variable *var) const { + auto filename = ctx.Attr("file_path"); + auto overwrite = ctx.Attr("overwrite"); if (FileExists(filename) && !overwrite) { PADDLE_THROW("%s is existed, cannot save to it when overwrite=false", @@ -84,7 +145,7 @@ class SaveOp : public framework::OperatorBase { PADDLE_ENFORCE(static_cast(fout), "Cannot open %s to write", filename); - auto save_as_fp16 = Attr("save_as_fp16"); + auto save_as_fp16 = ctx.Attr("save_as_fp16"); auto in_dtype = tensor.type(); auto out_dtype = save_as_fp16 ? framework::proto::VarType::FP16 : in_dtype; @@ -102,13 +163,15 @@ class SaveOp : public framework::OperatorBase { fout.close(); } - void SaveSelectedRows(const framework::Scope &scope, + void SaveSelectedRows(const framework::ExecutionContext &ctx, const platform::Place &place, - framework::Variable *var) const { - auto *lt_var = scope.FindVar(LOOKUP_TABLE_PATH)->GetMutable(); + const framework::Variable *var) const { + framework::Variable *out_put_var = ctx.OutputVar(LOOKUP_TABLE_PATH); PADDLE_ENFORCE( - lt_var != nullptr, + out_put_var != nullptr, "Can not find variable kLookupTablePath for SaveSelectedRows"); + auto *lt_var = out_put_var->GetMutable(); + std::string filename = lt_var->data(); VLOG(4) << "SaveSelectedRows get File name: " << filename; @@ -130,50 +193,17 @@ class SaveOp : public framework::OperatorBase { } }; -class SaveOpProtoMaker : public framework::OpProtoAndCheckerMaker { - public: - void Make() override { - AddInput("X", "(Tensor ) Input LoDTensor and SelectedRows to be saved"); - AddComment(R"DOC( -Save operator - -This operator will serialize and write LoDTensor / SelectedRows variable to file on disk. -)DOC"); - AddAttr("overwrite", - "(boolean, default true)" - "Overwrite the output file if exist") - .SetDefault(true); - AddAttr("save_as_fp16", - "(boolean, default false)" - "If true, the tensor will be converted to float16 data " - "type and then saved. Otherwise, the tensor will be " - "directly saved without data type conversion.") - .SetDefault(false); - AddAttr("file_path", - "(string)" - "The \"file_path\" where the variable will be saved.") - .AddCustomChecker( - [](const std::string &path) { return !path.empty(); }); - } -}; - -class SaveOpVarTypeInference : public framework::VarTypeInference { - public: - void operator()(framework::InferVarTypeContext *ctx) const override { - auto out_var_name = ctx->Output(LOOKUP_TABLE_PATH).front(); - ctx->SetType(out_var_name, framework::proto::VarType::RAW); - } -}; - -class SaveOpShapeInference : public framework::InferShapeBase { - public: - void operator()(framework::InferShapeContext *ctx) const override {} -}; } // namespace operators } // namespace paddle namespace ops = paddle::operators; -REGISTER_OPERATOR(save, ops::SaveOp, paddle::framework::EmptyGradOpMaker, - ops::SaveOpProtoMaker, ops::SaveOpVarTypeInference, - ops::SaveOpShapeInference); +REGISTER_OPERATOR(save, ops::SaveOp, ops::SaveOpProtoMaker, + ops::SaveOpVarTypeInference, ops::SaveOpShapeInference); + +REGISTER_OP_CPU_KERNEL( + save, ops::SaveOpKernel, + ops::SaveOpKernel, + ops::SaveOpKernel, + ops::SaveOpKernel, + ops::SaveOpKernel); diff --git a/python/paddle/fluid/framework.py b/python/paddle/fluid/framework.py index 556ce71ee5..e4169c247f 100644 --- a/python/paddle/fluid/framework.py +++ b/python/paddle/fluid/framework.py @@ -644,10 +644,9 @@ class Operator(object): outputs={"Out": [var1]}) """ OP_WITHOUT_KERNEL_SET = { - 'feed', 'fetch', 'save', 'load', 'recurrent', 'go', - 'rnn_memory_helper_grad', 'conditional_block', 'while', 'send', 'recv', - 'listen_and_serv', 'save_combine', 'load_combine', 'ncclInit', 'select', - 'checkpoint_notify', 'gen_nccl_id' + 'feed', 'fetch', 'recurrent', 'go', 'rnn_memory_helper_grad', + 'conditional_block', 'while', 'send', 'recv', 'listen_and_serv', + 'ncclInit', 'select', 'checkpoint_notify', 'gen_nccl_id' } def __init__(self, diff --git a/python/paddle/fluid/imperative/__init__.py b/python/paddle/fluid/imperative/__init__.py index 7f31ca1b9b..7281b3ea4b 100644 --- a/python/paddle/fluid/imperative/__init__.py +++ b/python/paddle/fluid/imperative/__init__.py @@ -29,9 +29,13 @@ from .tracer import * from . import profiler from .profiler import * +from . import checkpoint +from .checkpoint import * + __all__ = [] __all__ += layers.__all__ __all__ += base.__all__ __all__ += nn.__all__ __all__ += tracer.__all__ __all__ += profiler.__all__ +__all__ += checkpoint.__all__ diff --git a/python/paddle/fluid/imperative/checkpoint.py b/python/paddle/fluid/imperative/checkpoint.py new file mode 100644 index 0000000000..97bad77179 --- /dev/null +++ b/python/paddle/fluid/imperative/checkpoint.py @@ -0,0 +1,194 @@ +# 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. + +from __future__ import print_function + +import os +import collections +from .. import core +from ..framework import Variable, Parameter, default_main_program +from .layers import Layer + +__all__ = ['save_persistables', 'load_persistables'] + + +def save_persistables(obj, dirname, filename=None): + """ + This function filters out all variables in layer.parameters from the + give `layer` and then trys to load these variables from the folder + `dirname` or the file `filename`. + + Use the `dirname` to specify the folder where persistable variables were + saved. If variables were saved in separate files, set `filename` None; + if all variables were saved in a single file, use `filename` to specify + the file name. + + Args: + var_list(dict of Parameters|Layer): The parameters will + be saved. If it is None, nothing + will be deal. + dirname(str): The directory path. + filename(str|None): The file which saved all variables. If variables were + saved in differnet files, set it to None. + Default: None + + Returns: + + Examples: + .. code-block:: python + ptb_model = PtbModel( + hidden_size=hidden_size, + vocab_size=vocab_size, + num_layers=num_layers, + num_steps=num_steps, + init_scale=init_scale) + + x_data = np.arange(12).reshape(4, 3).astype('int64') + y_data = np.arange(1, 13).reshape(4, 3).astype('int64') + x_data = x_data.reshape((-1, num_steps, 1)) + y_data = y_data.reshape((-1, 1)) + init_hidden_data = np.zeros( + (num_layers, batch_size, hidden_size), dtype='float32') + init_cell_data = np.zeros( + (num_layers, batch_size, hidden_size), dtype='float32') + x = to_variable(x_data) + y = to_variable(y_data) + init_hidden = to_variable(init_hidden_data) + init_cell = to_variable(init_cell_data) + dy_loss, last_hidden, last_cell = ptb_model(x, y, init_hidden, + init_cell) + param_path = "./my_paddle_model" + fluid.imperative.checkpoint.save_persistables(ptb_model.parameters(), dirname=param_path, + layer=ptb_model) + """ + if isinstance(obj, collections.OrderedDict): + _save_var_to_file(obj, dirname, filename) + elif isinstance(obj, Layer): + _save_var_to_file( + obj.state_dict(include_sublayers=True), dirname, filename) + + +def load_persistables(obj, dirname, filename=None): + """ + This function trys to load persistable variables from the folder + `dirname` or the file `filename`. + + Use the `dirname` to specify the folder where persistable variables were + saved. If variables were saved in separate files, set `filename` None; + if all variables were saved in a single file, use `filename` to specify + the file name. + + Args: + obj(dict of Parameters|Layer): The parameters will be loaded. + dirname(str): The directory path. + filename(str|None): The file which saved all variables, this file path should be end with '.npz'. If variables were + saved in differnet files, set it to None. + Default: None + + Returns: + dict: The parameter-dict resumed from file + + Examples: + .. code-block:: python + my_layer = layer(fluid.imperative.Layer) + param_path = "./my_paddle_model" + + param_dict = fluid.imperative.checkpoint.load_persistables(my_layer.parameters(), param_path) + param_1 = param_dict['PtbModel_0.w_1'] + + or: + my_layer = layer(fluid.imperative.Layer) + param_path = "./my_paddle_model" + filename = "model.file" + param_dict = fluid.imperative.checkpoint.load_persistables(my_layer, var_list, param_path, + filename=filename) + param_1 = param_dict['PtbModel_0.w_1'] + + """ + if isinstance(obj, collections.OrderedDict): + return _load_var_from_file(obj, dirname, filename) + elif isinstance(obj, Layer): + return _load_var_from_file( + obj.state_dict(include_sublayers=True), dirname, filename) + + return {} + + +def _save_var_to_file(stat_dict, file_dir, file_name): + save_block = default_main_program().global_block() + save_var_map = {} + for each_var in stat_dict.items(): + save_var_map[each_var.name] = each_var + if file_name is None: + save_block.append_op( + type='save', + inputs={'X': [each_var]}, + outputs={}, + attrs={'file_path': os.path.join(file_dir, each_var.name)}) + + if file_name is not None: + save_var_list = [] + for name in sorted(save_var_map.keys()): + save_var_list.append(save_var_map[name]) + + save_block.append_op( + type='save_combine', + inputs={'X': save_var_list}, + outputs={}, + attrs={'file_path': os.path.join(file_dir, file_name)}) + + +def _load_var_from_file(stat_dict, file_dir, file_name): + load_block = default_main_program().global_block() + load_var_map = {} + + for each_var in stat_dict.items(): + assert isinstance(each_var, Variable) + if each_var.type == core.VarDesc.VarType.RAW: + continue + new_var = _clone_var_in_block_(load_block, each_var) + if file_name is None: + load_block.append_op( + type='load', + inputs={}, + outputs={'Out': [new_var]}, + attrs={'file_path': os.path.join(file_dir, each_var.name)}) + + load_var_map[new_var.name] = new_var + + if file_name is not None: + load_var_list = [] + for name in sorted(load_var_map.keys()): + load_var_list.append(load_var_map[name]) + + load_block.append_op( + type='load_combine', + inputs={}, + outputs={"Out": load_var_list}, + attrs={'file_path': os.path.join(file_dir, file_name)}) + for res_var in load_var_list: + load_var_map[res_var.name] = res_var + + return load_var_map + + +def _clone_var_in_block_(block, var): + assert isinstance(var, Variable) + return block.create_var( + name=var.name, + shape=var.shape, + dtype=var.dtype, + type=var.type, + lod_level=var.lod_level, + persistable=True) diff --git a/python/paddle/fluid/imperative/layers.py b/python/paddle/fluid/imperative/layers.py index 71d169a7dc..a8a1aac8b0 100644 --- a/python/paddle/fluid/imperative/layers.py +++ b/python/paddle/fluid/imperative/layers.py @@ -212,6 +212,34 @@ class Layer(core.Layer): else: object.__delattr__(self, name) + def state_dict(self, destination=None, prefix='', include_sublayers=True): + if destination is None: + destination = collections.OrderedDict() + for name, data in self._parameters.items(): + if data is not None: + destination[prefix + name] = data + + if include_sublayers: + for layer_name, layer_item in self._sub_layers.items(): + if layer_item is not None: + destination_temp = destination.copy() + destination_temp.update( + layer_item.state_dict(destination_temp, prefix + + layer_name + ".", + include_sublayers)) + destination = destination_temp + return destination + + def load_dict(self, stat_dict, include_sublayers=True): + for name, item in self.__dict__.get('_parameters', None).items(): + if item.name in stat_dict: + self.__setattr__(name, stat_dict[item.name]) + + if include_sublayers: + for layer_name, layer_item in self._sub_layers.items(): + if layer_item is not None: + layer_item.load_dict(stat_dict) + class PyLayer(core.PyLayer): """Layers composed of user-defined python codes.""" diff --git a/python/paddle/fluid/tests/unittests/test_imperative_checkpoint.py b/python/paddle/fluid/tests/unittests/test_imperative_checkpoint.py new file mode 100644 index 0000000000..62c25f7345 --- /dev/null +++ b/python/paddle/fluid/tests/unittests/test_imperative_checkpoint.py @@ -0,0 +1,163 @@ +# 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 + +import paddle +import paddle.fluid as fluid +from paddle.fluid.optimizer import SGDOptimizer +from paddle.fluid.imperative.nn import Conv2D, Pool2D, FC +from paddle.fluid.imperative.base import to_variable + + +class SimpleImgConvPool(fluid.imperative.Layer): + def __init__(self, + name_scope, + num_channels, + num_filters, + filter_size, + pool_size, + pool_stride, + pool_padding=0, + pool_type='max', + global_pooling=False, + conv_stride=1, + conv_padding=0, + conv_dilation=1, + conv_groups=1, + act=None, + use_cudnn=False, + param_attr=None, + bias_attr=None): + super(SimpleImgConvPool, self).__init__(name_scope) + + self._conv2d = Conv2D( + self.full_name(), + num_channels=num_channels, + num_filters=num_filters, + filter_size=filter_size, + stride=conv_stride, + padding=conv_padding, + dilation=conv_dilation, + groups=conv_groups, + param_attr=None, + bias_attr=None, + use_cudnn=use_cudnn) + + self._pool2d = Pool2D( + self.full_name(), + pool_size=pool_size, + pool_type=pool_type, + pool_stride=pool_stride, + pool_padding=pool_padding, + global_pooling=global_pooling, + use_cudnn=use_cudnn) + + def forward(self, inputs): + x = self._conv2d(inputs) + x = self._pool2d(x) + return x + + +class MNIST(fluid.imperative.Layer): + def __init__(self, name_scope): + super(MNIST, self).__init__(name_scope) + + self._simple_img_conv_pool_1 = SimpleImgConvPool( + self.full_name(), 1, 20, 5, 2, 2, act="relu") + + self._simple_img_conv_pool_2 = SimpleImgConvPool( + self.full_name(), 20, 50, 5, 2, 2, act="relu") + + pool_2_shape = 50 * 4 * 4 + SIZE = 10 + scale = (2.0 / (pool_2_shape**2 * SIZE))**0.5 + self._fc = FC(self.full_name(), + 10, + param_attr=fluid.param_attr.ParamAttr( + initializer=fluid.initializer.NormalInitializer( + loc=0.0, scale=scale)), + act="softmax") + + def forward(self, inputs): + x = self._simple_img_conv_pool_1(inputs) + x = self._simple_img_conv_pool_2(x) + x = self._fc(x) + return x + + +class TestImperativeCheckpoint(unittest.TestCase): + def save_load_persistables(self): + seed = 90 + epoch_num = 1 + + with fluid.imperative.guard(): + fluid.default_startup_program().random_seed = seed + fluid.default_main_program().random_seed = seed + + mnist = MNIST("mnist") + sgd = SGDOptimizer(learning_rate=1e-3) + train_reader = paddle.batch( + paddle.dataset.mnist.train(), batch_size=128, drop_last=True) + + dy_param_init_value = {} + + step = 0 + for epoch in range(epoch_num): + for batch_id, data in enumerate(train_reader()): + dy_x_data = np.array( + [x[0].reshape(1, 28, 28) + for x in data]).astype('float32') + y_data = np.array( + [x[1] for x in data]).astype('int64').reshape(128, 1) + + img = to_variable(dy_x_data) + label = to_variable(y_data) + label._stop_gradient = True + + cost = mnist(img) + loss = fluid.layers.cross_entropy(cost, label) + avg_loss = fluid.layers.mean(loss) + + dy_out = avg_loss._numpy() + + avg_loss._backward() + sgd.minimize(avg_loss) + fluid.imperative.save_persistables(mnist, "save_dir") + mnist.clear_gradients() + + for param in mnist.parameters(): + dy_param_init_value[param.name] = param._numpy() + + mnist.load_dict( + fluid.imperative.load_persistables(mnist, "save_dir")) + + restore = mnist.parameters() + + self.assertEqual(len(dy_param_init_value), len(restore)) + for value in restore: + self.assertTrue( + np.allclose(value, dy_param_init_value[value.name])) + self.assertTrue(np.isfinite(value.all())) + self.assertFalse(np.isnan(value.any())) + + step += 1 + + if step > 20: + break + + +if __name__ == '__main__': + unittest.main() From bb166a1e1019ff500f04d3a60c59ca8b5f71ea63 Mon Sep 17 00:00:00 2001 From: sneaxiy Date: Tue, 19 Mar 2019 08:12:43 +0000 Subject: [PATCH 15/44] fix API.spec test=develop --- paddle/fluid/API.spec | 8 +- python/paddle/fluid/compiler.py | 10 +-- python/paddle/fluid/reader.py | 143 +++++++++++++++--------------- python/paddle/reader/decorator.py | 3 +- 4 files changed, 81 insertions(+), 83 deletions(-) diff --git a/paddle/fluid/API.spec b/paddle/fluid/API.spec index f168d5608d..188f5037c6 100644 --- a/paddle/fluid/API.spec +++ b/paddle/fluid/API.spec @@ -47,7 +47,7 @@ paddle.fluid.AsyncExecutor.run (ArgSpec(args=['self', 'program', 'data_feed', 'f paddle.fluid.AsyncExecutor.save_model (ArgSpec(args=['self', 'save_path'], varargs=None, keywords=None, defaults=None), ('document', 'c8ac0dfcb3b187aba25d03af7fea56b2')) paddle.fluid.AsyncExecutor.stop (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '5f23d043607bb5d55e466ec3f578e093')) paddle.fluid.CompiledProgram.__init__ (ArgSpec(args=['self', 'program_or_graph'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) -paddle.fluid.CompiledProgram.with_data_parallel (ArgSpec(args=['self', 'loss_name', 'build_strategy', 'exec_strategy', 'share_vars_from', 'places'], varargs=None, keywords=None, defaults=(None, None, None, None, None)), ('document', 'dbf542d1384741650a1238ddb05daa37')) +paddle.fluid.CompiledProgram.with_data_parallel (ArgSpec(args=['self', 'loss_name', 'build_strategy', 'exec_strategy', 'share_vars_from', 'places'], varargs=None, keywords=None, defaults=(None, None, None, None, None)), ('document', '5e8cca4619a5d7c3280fb3cae7021b14')) paddle.fluid.CompiledProgram.with_inference_optimize (ArgSpec(args=['self', 'config'], varargs=None, keywords=None, defaults=None), ('document', '9e5b009d850191a010e859189c127fd8')) paddle.fluid.ExecutionStrategy.__init__ __init__(self: paddle.fluid.core.ParallelExecutor.ExecutionStrategy) -> None paddle.fluid.BuildStrategy.GradientScaleStrategy.__init__ __init__(self: paddle.fluid.core.ParallelExecutor.BuildStrategy.GradientScaleStrategy, arg0: int) -> None @@ -61,8 +61,8 @@ paddle.fluid.io.load_params (ArgSpec(args=['executor', 'dirname', 'main_program' paddle.fluid.io.load_persistables (ArgSpec(args=['executor', 'dirname', 'main_program', 'filename'], varargs=None, keywords=None, defaults=(None, None)), ('document', '28df5bfe26ca7a077f91156abb0fe6d2')) paddle.fluid.io.save_inference_model (ArgSpec(args=['dirname', 'feeded_var_names', 'target_vars', 'executor', 'main_program', 'model_filename', 'params_filename', 'export_for_deployment'], varargs=None, keywords=None, defaults=(None, None, None, True)), ('document', '582d87b8df75a5a639a107db8ff86f9c')) paddle.fluid.io.load_inference_model (ArgSpec(args=['dirname', 'executor', 'model_filename', 'params_filename', 'pserver_endpoints'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '7a5255386075dac3c75b7058254fcdcb')) -paddle.fluid.io.PyReader.__init__ (ArgSpec(args=['self', 'feed_list', 'capacity', 'use_double_buffer', 'iterable'], varargs=None, keywords=None, defaults=(True, False)), ('document', 'b3d72958b2568aae3f90f72abdcb7d1a')) -paddle.fluid.io.PyReader.decorate_batch_generator (ArgSpec(args=['self', 'reader', 'places'], varargs=None, keywords=None, defaults=(None,)), ('document', 'd10224fef1095247063b6976da793021')) +paddle.fluid.io.PyReader.__init__ (ArgSpec(args=['self', 'feed_list', 'capacity', 'use_double_buffer', 'iterable'], varargs=None, keywords=None, defaults=(True, False)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) +paddle.fluid.io.PyReader.decorate_batch_generator (ArgSpec(args=['self', 'reader', 'places'], varargs=None, keywords=None, defaults=(None,)), ('document', 'a3fefec8bacd6ce83f49906a9d05e779')) paddle.fluid.io.PyReader.decorate_sample_generator (ArgSpec(args=['self', 'sample_generator', 'batch_size', 'drop_last', 'places'], varargs=None, keywords=None, defaults=(True, None)), ('document', '7abd9cf7d695bab5bb6cf7ded5903cb2')) paddle.fluid.io.PyReader.decorate_sample_list_generator (ArgSpec(args=['self', 'reader', 'places'], varargs=None, keywords=None, defaults=(None,)), ('document', 'faef298f73e91aedcfaf5d184f3109b7')) paddle.fluid.io.PyReader.reset (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', 'ff1cc1e2beb8824d453656c72c28ddfb')) @@ -521,7 +521,7 @@ paddle.fluid.unique_name.guard (ArgSpec(args=['new_generator'], varargs=None, ke paddle.fluid.recordio_writer.convert_reader_to_recordio_file (ArgSpec(args=['filename', 'reader_creator', 'feeder', 'compressor', 'max_num_records', 'feed_order'], varargs=None, keywords=None, defaults=(Compressor.Snappy, 1000, None)), ('document', '65c7523e86f0c50bb729b01667f36310')) paddle.fluid.recordio_writer.convert_reader_to_recordio_files (ArgSpec(args=['filename', 'batch_per_file', 'reader_creator', 'feeder', 'compressor', 'max_num_records', 'feed_order'], varargs=None, keywords=None, defaults=(Compressor.Snappy, 1000, None)), ('document', 'bc643f0f5f1b9db57ff0d8a57d379bd7')) paddle.fluid.Scope Scope() -> paddle.fluid.core._Scope -paddle.reader.cache (ArgSpec(args=['reader'], varargs=None, keywords=None, defaults=None), ('document', '83b94750674c6a04b5f96599d4bf3105')) +paddle.reader.cache (ArgSpec(args=['reader'], varargs=None, keywords=None, defaults=None), ('document', '1676886070eb607cb608f7ba47be0d3c')) paddle.reader.map_readers (ArgSpec(args=['func'], varargs='readers', keywords=None, defaults=None), ('document', '77cbadb09df588e21e5cc0819b69c87d')) paddle.reader.buffered (ArgSpec(args=['reader', 'size'], varargs=None, keywords=None, defaults=None), ('document', '0d6186f109feceb99f60ec50a0a624cb')) paddle.reader.compose (ArgSpec(args=[], varargs='readers', keywords='kwargs', defaults=None), ('document', '884291104e1c3f37f33aae44b7deeb0d')) diff --git a/python/paddle/fluid/compiler.py b/python/paddle/fluid/compiler.py index 894220d96c..75cbd141d3 100644 --- a/python/paddle/fluid/compiler.py +++ b/python/paddle/fluid/compiler.py @@ -123,7 +123,7 @@ class CompiledProgram(object): will share variables from `share_vars_from`. `share_vars_from` must be run by the executor before this CompiledProgram so that vars are ready. - places(list(CUDAPlace)|list(CPUPlace)|None): If provide, only compile + places(list(CUDAPlace)|list(CPUPlace)|None): If provided, only compile program in the given places. Otherwise, the places used when compiled is determined by the Executor, and the places used are controlled by environment variables: FLAGS_selected_gpus or CUDA_VISIBLE_DEVICES @@ -148,7 +148,7 @@ class CompiledProgram(object): if places is not None: if not isinstance(places, (list, tuple)): places = [places] - self._places = [_place_obj(p) for p in places] + self._places = places else: self._places = None self._build_strategy.is_distribution = _is_pserver_mode(self._program) @@ -195,14 +195,12 @@ class CompiledProgram(object): self._exec_strategy.use_cuda = use_cuda has_set_place = (self._places is not None) if has_set_place: - desire_place = _place_obj(self._place) for p in self._places: - assert p._type() == desire_place._type(), \ + assert p._type() == self._place._type(), \ "Place type not match. You may set the wrong type of places" else: - places = cuda_places( + self._places = cuda_places( ) if self._exec_strategy.use_cuda else cpu_places() - self._places = [_place_obj(p) for p in places] assert self._places, "no place for execution" if self._exec_strategy.num_threads == 0: diff --git a/python/paddle/fluid/reader.py b/python/paddle/fluid/reader.py index 5893798069..74ee2828de 100644 --- a/python/paddle/fluid/reader.py +++ b/python/paddle/fluid/reader.py @@ -40,6 +40,77 @@ def _convert_places(places): class PyReader(object): + """ + Create a reader object for data feeding in Python. + Data would be prefetched using Python thread and be pushed + into a queue asynchronously. Data in the queue would be extracted + automatically when `Executor.run(...)` is called. + + Args: + feed_list (list(Variable)|tuple(Variable)): feed variable list. + The variables should be created by :code:`fluid.layers.data()`. + capacity (int): capacity of the queue maintained in PyReader object. + use_double_buffer (bool): whether to use double_buffer_reader to + speed up data feeding. + iterable (bool): whether the created reader object is iterable. + + Returns: + reader (Reader): the created reader object. + + Examples: + 1. If iterable = False, the created PyReader object is almost the + same as :code:`fluid.layers.py_reader()`. Operators would be + inserted into the program. User should call :code:`start()` + before each epoch and catch :code:`fluid.core.EOFException` + thrown by :code:`Executor.run()` when epoch ends. Once the + exception is caught, user should call :code:`reset()` to reset + the reader manually. + + .. code-block:: python + + image = fluid.layers.data( + name='image', shape=[784], dtype='float32') + label = fluid.layers.data( + name='label', shape=[1], dtype='int64') + + reader = fluid.io.PyReader(feed_list=[image, label], + capacity=4, iterable=False) + reader.decorate_sample_list_generator(user_defined_reader) + ... # definition of network is omitted + executor.run(fluid.default_main_program()) + for _ in range(EPOCH_NUM): + reader.start() + while True: + try: + executor.run(feed=None, ...) + except fluid.core.EOFException: + reader.reset() + break + + 2. If iterable=True, the created PyReader object is decoupled with + the program. No operator would be inserted into the program. + In this case, the created reader is a Python generator, which + is iterable. User should feed the data yielded from PyReader + object into :code:`Executor.run(feed=...)`. + + .. code-block:: python + + image = fluid.layers.data( + name='image', shape=[784], dtype='float32') + label = fluid.layers.data( + name='label', shape=[1], dtype='int64') + + reader = fluid.io.PyReader(feed_list=[image, label], + capacity=4, iterable=True) + reader.decorate_sample_list_generator(user_defined_reader, + places=fluid.cuda_places()) + ... # definition of network is omitted + executor.run(fluid.default_main_program()) + for _ in range(EPOCH_NUM): + for data in reader(): + executor.run(feed=data, ...) + """ + unique_name_generator = UniqueNameGenerator() def __init__(self, @@ -47,76 +118,6 @@ class PyReader(object): capacity, use_double_buffer=True, iterable=False): - """ - Create a reader object for data feeding in Python. - Data would be prefetched using Python thread and be pushed - into a queue asynchronously. Data in the queue would be extracted - automatically when `Executor.run(...)` is called. - - Args: - feed_list (list(Variable)|tuple(Variable)): feed variable list. - The variables should be created by :code:`fluid.layers.data()`. - capacity (int): capacity of the queue maintained in PyReader object. - use_double_buffer (bool): whether to use double_buffer_reader to - speed up data feeding. - iterable (bool): whether the created reader object is iterable. - - Returns: - reader (Reader): the created reader object. - - Examples: - 1. If iterable = False, the created PyReader object is almost the - same as :code:`fluid.layers.py_reader()`. Operators would be - inserted into the program. User should call :code:`start()` - before each epoch and catch :code:`fluid.core.EOFException` - thrown by :code:`Executor.run()` when epoch ends. Once the - exception is caught, user should call :code:`reset()` to reset - the reader manually. - - .. code-block:: python - - image = fluid.layers.data( - name='image', shape=[784], dtype='float32') - label = fluid.layers.data( - name='label', shape=[1], dtype='int64') - - reader = fluid.io.PyReader(feed_list=[image, label], - capacity=4, iterable=False) - reader.decorate_sample_list_generator(user_defined_reader) - ... # definition of network is omitted - executor.run(fluid.default_main_program()) - for _ in range(EPOCH_NUM): - reader.start() - while True: - try: - executor.run(feed=None, ...) - except fluid.core.EOFException: - reader.reset() - break - - 2. If iterable=True, the created PyReader object is decoupled with - the program. No operator would be inserted into the program. - In this case, the created reader is a Python generator, which - is iterable. User should feed the data yielded from PyReader - object into :code:`Executor.run(feed=...)`. - - .. code-block:: python - - image = fluid.layers.data( - name='image', shape=[784], dtype='float32') - label = fluid.layers.data( - name='label', shape=[1], dtype='int64') - - reader = fluid.io.PyReader(feed_list=[image, label], - capacity=4, iterable=True) - reader.decorate_sample_list_generator(user_defined_reader, - places=fluid.cuda_places()) - ... # definition of network is omitted - executor.run(fluid.default_main_program()) - for _ in range(EPOCH_NUM): - for data in reader(): - executor.run(feed=data, ...) - """ self._tensor_reader = None self._thread = None self._iterable = iterable @@ -361,7 +362,7 @@ class PyReader(object): Args: reader (generator): Python generator that yields LoDTensor-typed batched data. - places (None|list(CUDAPlace)|list(CPUPlace)): place list. Must + places (None|list(CUDAPlace)|list(CPUPlace)): place list. Must be provided when PyReader is iterable. ''' assert self._tensor_reader is None, \ diff --git a/python/paddle/reader/decorator.py b/python/paddle/reader/decorator.py index 7443a6bb19..168a525b58 100644 --- a/python/paddle/reader/decorator.py +++ b/python/paddle/reader/decorator.py @@ -46,8 +46,7 @@ def cache(reader): data each time. Returns: - reader (generator): a decorated reader object - which yields data from cached memory. + generator: a decorated reader object which yields data from cached memory. """ all_data = tuple(reader()) From 622fe6a56b4946cf3ad5e21c2fb46f6c9fbfcb3f Mon Sep 17 00:00:00 2001 From: lujun Date: Wed, 20 Mar 2019 19:24:09 +0800 Subject: [PATCH 16/44] checkpoint pr be moved here, test=develop --- paddle/fluid/operators/load_combine_op.cc | 85 +------------- paddle/fluid/operators/load_combine_op.cu | 24 ++++ paddle/fluid/operators/load_combine_op.h | 102 +++++++++++++++++ paddle/fluid/operators/load_op.cc | 82 +------------ paddle/fluid/operators/load_op.cu | 23 ++++ paddle/fluid/operators/load_op.h | 102 +++++++++++++++++ paddle/fluid/operators/save_combine_op.cc | 75 +----------- paddle/fluid/operators/save_combine_op.cu | 24 ++++ paddle/fluid/operators/save_combine_op.h | 95 ++++++++++++++++ paddle/fluid/operators/save_op.cc | 115 +------------------ paddle/fluid/operators/save_op.cu | 26 +++++ paddle/fluid/operators/save_op.h | 133 ++++++++++++++++++++++ 12 files changed, 543 insertions(+), 343 deletions(-) create mode 100644 paddle/fluid/operators/load_combine_op.cu create mode 100644 paddle/fluid/operators/load_combine_op.h create mode 100644 paddle/fluid/operators/load_op.cu create mode 100644 paddle/fluid/operators/load_op.h create mode 100644 paddle/fluid/operators/save_combine_op.cu create mode 100644 paddle/fluid/operators/save_combine_op.h create mode 100644 paddle/fluid/operators/save_op.cu create mode 100644 paddle/fluid/operators/save_op.h diff --git a/paddle/fluid/operators/load_combine_op.cc b/paddle/fluid/operators/load_combine_op.cc index 3180f16576..2948cf71a9 100644 --- a/paddle/fluid/operators/load_combine_op.cc +++ b/paddle/fluid/operators/load_combine_op.cc @@ -11,11 +11,11 @@ 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 -#include "paddle/fluid/framework/data_type.h" -#include "paddle/fluid/framework/data_type_transform.h" -#include "paddle/fluid/framework/op_registry.h" -#include "paddle/fluid/platform/device_context.h" + +#include +#include + +#include "paddle/fluid/operators/load_combine_op.h" namespace paddle { namespace operators { @@ -30,7 +30,7 @@ class LoadCombineOp : public framework::OperatorWithKernel { framework::OpKernelType GetExpectedKernelType( const framework::ExecutionContext &ctx) const override { framework::OpKernelType kt = framework::OpKernelType( - framework::proto::VarType::FP32, platform::CPUPlace()); + framework::proto::VarType::FP32, ctx.GetPlace()); return kt; } }; @@ -75,79 +75,6 @@ that were saved using the SaveCombine operator. } }; -template -class LoadCombineOpKernel : public framework::OpKernel { - public: - void Compute(const framework::ExecutionContext &ctx) const override { - auto place = ctx.GetPlace(); - auto filename = ctx.Attr("file_path"); - auto load_as_fp16 = ctx.Attr("load_as_fp16"); - auto model_from_memory = ctx.Attr("model_from_memory"); - auto &out_var_names = ctx.Outputs("Out"); - - PADDLE_ENFORCE_GT( - static_cast(out_var_names.size()), 0, - "The number of output variables should be greater than 0."); - if (!model_from_memory) { - std::ifstream fin(filename, std::ios::binary); - PADDLE_ENFORCE(static_cast(fin), - "Cannot open file %s for load_combine op", filename); - LoadParamsFromBuffer(ctx, place, &fin, load_as_fp16, out_var_names); - } else { - PADDLE_ENFORCE(!filename.empty(), "Cannot load file from memory"); - std::stringstream fin(filename, std::ios::in | std::ios::binary); - LoadParamsFromBuffer(ctx, place, &fin, load_as_fp16, out_var_names); - } - } - - void LoadParamsFromBuffer( - const framework::ExecutionContext &context, const platform::Place &place, - std::istream *buffer, bool load_as_fp16, - const std::vector &out_var_names) const { - platform::DeviceContextPool &pool = platform::DeviceContextPool::Instance(); - auto &dev_ctx = *pool.Get(place); - auto out_vars = context.MultiOutputVar("Out"); - - for (size_t i = 0; i < out_var_names.size(); i++) { - PADDLE_ENFORCE(out_vars[i] != nullptr, - "Output variable %s cannot be found", out_var_names[i]); - - auto *tensor = out_vars[i]->GetMutable(); - - // Error checking - PADDLE_ENFORCE(static_cast(*buffer), "Cannot read more"); - - // Get data from fin to tensor - DeserializeFromStream(*buffer, tensor, dev_ctx); - - auto in_dtype = tensor->type(); - auto out_dtype = - load_as_fp16 ? framework::proto::VarType::FP16 : in_dtype; - - if (in_dtype != out_dtype) { - // convert to float16 tensor - auto in_kernel_type = framework::OpKernelType(in_dtype, place); - auto out_kernel_type = framework::OpKernelType(out_dtype, place); - framework::LoDTensor fp16_tensor; - // copy LoD info to the new tensor - fp16_tensor.set_lod(tensor->lod()); - framework::TransDataType(in_kernel_type, out_kernel_type, *tensor, - &fp16_tensor); - - // reset output tensor - out_vars[i]->Clear(); - tensor = out_vars[i]->GetMutable(); - tensor->set_lod(fp16_tensor.lod()); - tensor->ShareDataWith(fp16_tensor); - } - } - buffer->peek(); - PADDLE_ENFORCE(buffer->eof(), - "You are not allowed to load partial data via " - "load_combine_op, use load_op instead."); - } -}; - } // namespace operators } // namespace paddle diff --git a/paddle/fluid/operators/load_combine_op.cu b/paddle/fluid/operators/load_combine_op.cu new file mode 100644 index 0000000000..f23f4ea5aa --- /dev/null +++ b/paddle/fluid/operators/load_combine_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/load_combine_op.h" + +namespace ops = paddle::operators; + +REGISTER_OP_CUDA_KERNEL( + load_combine, + ops::LoadCombineOpKernel, + ops::LoadCombineOpKernel, + ops::LoadCombineOpKernel, + ops::LoadCombineOpKernel); diff --git a/paddle/fluid/operators/load_combine_op.h b/paddle/fluid/operators/load_combine_op.h new file mode 100644 index 0000000000..8f620ba7d2 --- /dev/null +++ b/paddle/fluid/operators/load_combine_op.h @@ -0,0 +1,102 @@ +/* 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/data_type.h" +#include "paddle/fluid/framework/data_type_transform.h" +#include "paddle/fluid/framework/op_registry.h" +#include "paddle/fluid/platform/device_context.h" + +namespace paddle { +namespace operators { +template +class LoadCombineOpKernel : public framework::OpKernel { + public: + void Compute(const framework::ExecutionContext &ctx) const override { + auto place = ctx.GetPlace(); + auto filename = ctx.Attr("file_path"); + auto load_as_fp16 = ctx.Attr("load_as_fp16"); + auto model_from_memory = ctx.Attr("model_from_memory"); + auto &out_var_names = ctx.Outputs("Out"); + + PADDLE_ENFORCE_GT( + static_cast(out_var_names.size()), 0, + "The number of output variables should be greater than 0."); + if (!model_from_memory) { + std::ifstream fin(filename, std::ios::binary); + PADDLE_ENFORCE(static_cast(fin), + "Cannot open file %s for load_combine op", filename); + LoadParamsFromBuffer(ctx, place, &fin, load_as_fp16, out_var_names); + } else { + PADDLE_ENFORCE(!filename.empty(), "Cannot load file from memory"); + std::stringstream fin(filename, std::ios::in | std::ios::binary); + LoadParamsFromBuffer(ctx, place, &fin, load_as_fp16, out_var_names); + } + } + + void LoadParamsFromBuffer( + const framework::ExecutionContext &context, const platform::Place &place, + std::istream *buffer, bool load_as_fp16, + const std::vector &out_var_names) const { + platform::DeviceContextPool &pool = platform::DeviceContextPool::Instance(); + auto &dev_ctx = *pool.Get(place); + auto out_vars = context.MultiOutputVar("Out"); + + for (size_t i = 0; i < out_var_names.size(); i++) { + PADDLE_ENFORCE(out_vars[i] != nullptr, + "Output variable %s cannot be found", out_var_names[i]); + + auto *tensor = out_vars[i]->GetMutable(); + + // Error checking + PADDLE_ENFORCE(static_cast(*buffer), "Cannot read more"); + + // Get data from fin to tensor + DeserializeFromStream(*buffer, tensor, dev_ctx); + + auto in_dtype = tensor->type(); + auto out_dtype = + load_as_fp16 ? framework::proto::VarType::FP16 : in_dtype; + + if (in_dtype != out_dtype) { + // convert to float16 tensor + auto in_kernel_type = framework::OpKernelType(in_dtype, place); + auto out_kernel_type = framework::OpKernelType(out_dtype, place); + framework::LoDTensor fp16_tensor; + // copy LoD info to the new tensor + fp16_tensor.set_lod(tensor->lod()); + framework::TransDataType(in_kernel_type, out_kernel_type, *tensor, + &fp16_tensor); + + // reset output tensor + out_vars[i]->Clear(); + tensor = out_vars[i]->GetMutable(); + tensor->set_lod(fp16_tensor.lod()); + tensor->ShareDataWith(fp16_tensor); + } + } + buffer->peek(); + PADDLE_ENFORCE(buffer->eof(), + "You are not allowed to load partial data via " + "load_combine_op, use load_op instead."); + } +}; + +} // namespace operators +} // namespace paddle diff --git a/paddle/fluid/operators/load_op.cc b/paddle/fluid/operators/load_op.cc index befe0ea018..2d8e6ca854 100644 --- a/paddle/fluid/operators/load_op.cc +++ b/paddle/fluid/operators/load_op.cc @@ -11,12 +11,10 @@ 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 -#include "paddle/fluid/framework/data_type_transform.h" -#include "paddle/fluid/framework/op_registry.h" -#include "paddle/fluid/platform/device_context.h" -#include "paddle/fluid/platform/profiler.h" +#include + +#include "paddle/fluid/operators/load_op.h" namespace paddle { namespace operators { @@ -56,80 +54,6 @@ class LoadOpProtoMaker : public framework::OpProtoAndCheckerMaker { } }; -template -class LoadOpKernel : public framework::OpKernel { - public: - void Compute(const framework::ExecutionContext &ctx) const override { - auto place = ctx.GetPlace(); - // FIXME(yuyang18): We save variable to local file now, but we should change - // it to save an output stream. - auto filename = ctx.Attr("file_path"); - std::ifstream fin(filename, std::ios::binary); - PADDLE_ENFORCE(static_cast(fin), "Cannot open file %s for load op", - filename); - - auto out_var_name = ctx.Outputs("Out").data(); - auto *out_var = ctx.OutputVar("Out"); - - PADDLE_ENFORCE(out_var != nullptr, "Output variable %s cannot be found ", - out_var_name); - - PADDLE_ENFORCE(out_var != nullptr, "Output variable cannot be found "); - - if (out_var->IsType()) { - LoadLodTensor(fin, place, out_var, ctx); - } else if (out_var->IsType()) { - LoadSelectedRows(fin, place, out_var); - } else { - PADDLE_ENFORCE( - false, - "Load only support LoDTensor and SelectedRows, %s has wrong type", - out_var_name); - } - } - - void LoadLodTensor(std::istream &fin, const platform::Place &place, - framework::Variable *var, - const framework::ExecutionContext &ctx) const { - // get device context from pool - platform::DeviceContextPool &pool = platform::DeviceContextPool::Instance(); - auto &dev_ctx = *pool.Get(place); - auto *tensor = var->GetMutable(); - DeserializeFromStream(fin, tensor, dev_ctx); - - auto load_as_fp16 = ctx.Attr("load_as_fp16"); - auto in_dtype = tensor->type(); - auto out_dtype = load_as_fp16 ? framework::proto::VarType::FP16 : in_dtype; - - if (in_dtype != out_dtype) { - // convert to float16 tensor - auto in_kernel_type = framework::OpKernelType(in_dtype, place); - auto out_kernel_type = framework::OpKernelType(out_dtype, place); - framework::LoDTensor fp16_tensor; - // copy LoD info to the new tensor - fp16_tensor.set_lod(tensor->lod()); - framework::TransDataType(in_kernel_type, out_kernel_type, *tensor, - &fp16_tensor); - - // reset output tensor - var->Clear(); - tensor = var->GetMutable(); - tensor->set_lod(fp16_tensor.lod()); - tensor->ShareDataWith(fp16_tensor); - } - } - - void LoadSelectedRows(std::istream &fin, const platform::Place &place, - framework::Variable *var) const { - auto *selectedRows = var->GetMutable(); - // get device context from pool - platform::DeviceContextPool &pool = platform::DeviceContextPool::Instance(); - auto &dev_ctx = *pool.Get(place); - framework::DeserializeFromStream(fin, selectedRows, dev_ctx); - selectedRows->SyncIndex(); - } -}; - } // namespace operators } // namespace paddle namespace ops = paddle::operators; diff --git a/paddle/fluid/operators/load_op.cu b/paddle/fluid/operators/load_op.cu new file mode 100644 index 0000000000..dcbef0b592 --- /dev/null +++ b/paddle/fluid/operators/load_op.cu @@ -0,0 +1,23 @@ +/* Copyright (c) 2016 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/load_op.h" + +namespace ops = paddle::operators; + +REGISTER_OP_CUDA_KERNEL( + load, ops::LoadOpKernel, + ops::LoadOpKernel, + ops::LoadOpKernel, + ops::LoadOpKernel); diff --git a/paddle/fluid/operators/load_op.h b/paddle/fluid/operators/load_op.h new file mode 100644 index 0000000000..3bf3c6bed2 --- /dev/null +++ b/paddle/fluid/operators/load_op.h @@ -0,0 +1,102 @@ +/* Copyright (c) 2016 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/data_type_transform.h" +#include "paddle/fluid/framework/op_registry.h" +#include "paddle/fluid/platform/device_context.h" +#include "paddle/fluid/platform/profiler.h" + +namespace paddle { +namespace operators { +template +class LoadOpKernel : public framework::OpKernel { + public: + void Compute(const framework::ExecutionContext &ctx) const override { + auto place = ctx.GetPlace(); + // FIXME(yuyang18): We save variable to local file now, but we should change + // it to save an output stream. + auto filename = ctx.Attr("file_path"); + std::ifstream fin(filename, std::ios::binary); + PADDLE_ENFORCE(static_cast(fin), "Cannot open file %s for load op", + filename); + + auto out_var_name = ctx.Outputs("Out").data(); + auto *out_var = ctx.OutputVar("Out"); + + PADDLE_ENFORCE(out_var != nullptr, "Output variable %s cannot be found ", + out_var_name); + + PADDLE_ENFORCE(out_var != nullptr, "Output variable cannot be found "); + + if (out_var->IsType()) { + LoadLodTensor(fin, place, out_var, ctx); + } else if (out_var->IsType()) { + LoadSelectedRows(fin, place, out_var); + } else { + PADDLE_ENFORCE( + false, + "Load only support LoDTensor and SelectedRows, %s has wrong type", + out_var_name); + } + } + + void LoadLodTensor(std::istream &fin, const platform::Place &place, + framework::Variable *var, + const framework::ExecutionContext &ctx) const { + // get device context from pool + platform::DeviceContextPool &pool = platform::DeviceContextPool::Instance(); + auto &dev_ctx = *pool.Get(place); + auto *tensor = var->GetMutable(); + DeserializeFromStream(fin, tensor, dev_ctx); + + auto load_as_fp16 = ctx.Attr("load_as_fp16"); + auto in_dtype = tensor->type(); + auto out_dtype = load_as_fp16 ? framework::proto::VarType::FP16 : in_dtype; + + if (in_dtype != out_dtype) { + // convert to float16 tensor + auto in_kernel_type = framework::OpKernelType(in_dtype, place); + auto out_kernel_type = framework::OpKernelType(out_dtype, place); + framework::LoDTensor fp16_tensor; + // copy LoD info to the new tensor + fp16_tensor.set_lod(tensor->lod()); + framework::TransDataType(in_kernel_type, out_kernel_type, *tensor, + &fp16_tensor); + + // reset output tensor + var->Clear(); + tensor = var->GetMutable(); + tensor->set_lod(fp16_tensor.lod()); + tensor->ShareDataWith(fp16_tensor); + } + } + + void LoadSelectedRows(std::istream &fin, const platform::Place &place, + framework::Variable *var) const { + auto *selectedRows = var->GetMutable(); + // get device context from pool + platform::DeviceContextPool &pool = platform::DeviceContextPool::Instance(); + auto &dev_ctx = *pool.Get(place); + framework::DeserializeFromStream(fin, selectedRows, dev_ctx); + selectedRows->SyncIndex(); + } +}; + +} // namespace operators +} // namespace paddle diff --git a/paddle/fluid/operators/save_combine_op.cc b/paddle/fluid/operators/save_combine_op.cc index 7110925565..62b1e09737 100644 --- a/paddle/fluid/operators/save_combine_op.cc +++ b/paddle/fluid/operators/save_combine_op.cc @@ -12,17 +12,9 @@ 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 -#include -#include -#include -#include "paddle/fluid/framework/data_type.h" -#include "paddle/fluid/framework/data_type_transform.h" -#include "paddle/fluid/framework/framework.pb.h" -#include "paddle/fluid/framework/lod_tensor.h" -#include "paddle/fluid/framework/op_registry.h" -#include "paddle/fluid/platform/device_context.h" -#include "paddle/fluid/platform/port.h" +#include + +#include "paddle/fluid/operators/save_combine_op.h" namespace paddle { namespace operators { @@ -66,67 +58,6 @@ to a file on disk. } }; -template -class SaveCombineOpKernel : public framework::OpKernel { - public: - void Compute(const framework::ExecutionContext &ctx) const override { - auto place = ctx.GetPlace(); - auto filename = ctx.Attr("file_path"); - auto overwrite = ctx.Attr("overwrite"); - auto save_as_fp16 = ctx.Attr("save_as_fp16"); - - bool is_present = FileExists(filename); - if (is_present && !overwrite) { - PADDLE_THROW("%s exists!, cannot save_combine to it when overwrite=false", - filename, overwrite); - } - - MkDirRecursively(DirName(filename).c_str()); - std::ofstream fout(filename, std::ios::binary); - PADDLE_ENFORCE(static_cast(fout), "Cannot open %s to write", - filename); - - auto &inp_var_names = ctx.Inputs("X"); - auto &inp_vars = ctx.MultiInputVar("X"); - PADDLE_ENFORCE_GT(static_cast(inp_var_names.size()), 0, - "The number of input variables should be greater than 0"); - - // get device context from pool - platform::DeviceContextPool &pool = platform::DeviceContextPool::Instance(); - auto &dev_ctx = *pool.Get(place); - - for (size_t i = 0; i < inp_var_names.size(); i++) { - PADDLE_ENFORCE(inp_vars[i] != nullptr, - "Cannot find variable %s for save_combine_op", - inp_var_names[i]); - PADDLE_ENFORCE(inp_vars[i]->IsType(), - "SaveCombineOp only supports LoDTensor, %s has wrong type", - inp_var_names[i]); - - auto &tensor = inp_vars[i]->Get(); - // Serialize tensors one by one - - // Check types to see if a fp16 transformation is required - auto in_dtype = tensor.type(); - auto out_dtype = - save_as_fp16 ? framework::proto::VarType::FP16 : in_dtype; - - if (in_dtype != out_dtype) { - auto in_kernel_type = framework::OpKernelType(in_dtype, place); - auto out_kernel_type = framework::OpKernelType(out_dtype, place); - framework::LoDTensor out; - // copy LoD info to the new tensor - out.set_lod(tensor.lod()); - framework::TransDataType(in_kernel_type, out_kernel_type, tensor, &out); - framework::SerializeToStream(fout, out, dev_ctx); - } else { - framework::SerializeToStream(fout, tensor, dev_ctx); - } - } - fout.close(); - } -}; - } // namespace operators } // namespace paddle diff --git a/paddle/fluid/operators/save_combine_op.cu b/paddle/fluid/operators/save_combine_op.cu new file mode 100644 index 0000000000..71476fd802 --- /dev/null +++ b/paddle/fluid/operators/save_combine_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/save_combine_op.h" + +namespace ops = paddle::operators; + +REGISTER_OP_CUDA_KERNEL( + save_combine, + ops::SaveCombineOpKernel, + ops::SaveCombineOpKernel, + ops::SaveCombineOpKernel, + ops::SaveCombineOpKernel); diff --git a/paddle/fluid/operators/save_combine_op.h b/paddle/fluid/operators/save_combine_op.h new file mode 100644 index 0000000000..4ee82e17dd --- /dev/null +++ b/paddle/fluid/operators/save_combine_op.h @@ -0,0 +1,95 @@ +/* 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 +#include + +#include "paddle/fluid/framework/data_type.h" +#include "paddle/fluid/framework/data_type_transform.h" +#include "paddle/fluid/framework/framework.pb.h" +#include "paddle/fluid/framework/lod_tensor.h" +#include "paddle/fluid/framework/op_registry.h" +#include "paddle/fluid/platform/device_context.h" +#include "paddle/fluid/platform/port.h" + +namespace paddle { +namespace operators { +template +class SaveCombineOpKernel : public framework::OpKernel { + public: + void Compute(const framework::ExecutionContext &ctx) const override { + auto place = ctx.GetPlace(); + auto filename = ctx.Attr("file_path"); + auto overwrite = ctx.Attr("overwrite"); + auto save_as_fp16 = ctx.Attr("save_as_fp16"); + + bool is_present = FileExists(filename); + if (is_present && !overwrite) { + PADDLE_THROW("%s exists!, cannot save_combine to it when overwrite=false", + filename, overwrite); + } + + MkDirRecursively(DirName(filename).c_str()); + std::ofstream fout(filename, std::ios::binary); + PADDLE_ENFORCE(static_cast(fout), "Cannot open %s to write", + filename); + + auto &inp_var_names = ctx.Inputs("X"); + auto &inp_vars = ctx.MultiInputVar("X"); + PADDLE_ENFORCE_GT(static_cast(inp_var_names.size()), 0, + "The number of input variables should be greater than 0"); + + // get device context from pool + platform::DeviceContextPool &pool = platform::DeviceContextPool::Instance(); + auto &dev_ctx = *pool.Get(place); + + for (size_t i = 0; i < inp_var_names.size(); i++) { + PADDLE_ENFORCE(inp_vars[i] != nullptr, + "Cannot find variable %s for save_combine_op", + inp_var_names[i]); + PADDLE_ENFORCE(inp_vars[i]->IsType(), + "SaveCombineOp only supports LoDTensor, %s has wrong type", + inp_var_names[i]); + + auto &tensor = inp_vars[i]->Get(); + // Serialize tensors one by one + + // Check types to see if a fp16 transformation is required + auto in_dtype = tensor.type(); + auto out_dtype = + save_as_fp16 ? framework::proto::VarType::FP16 : in_dtype; + + if (in_dtype != out_dtype) { + auto in_kernel_type = framework::OpKernelType(in_dtype, place); + auto out_kernel_type = framework::OpKernelType(out_dtype, place); + framework::LoDTensor out; + // copy LoD info to the new tensor + out.set_lod(tensor.lod()); + framework::TransDataType(in_kernel_type, out_kernel_type, tensor, &out); + framework::SerializeToStream(fout, out, dev_ctx); + } else { + framework::SerializeToStream(fout, tensor, dev_ctx); + } + } + fout.close(); + } +}; + +} // namespace operators +} // namespace paddle diff --git a/paddle/fluid/operators/save_op.cc b/paddle/fluid/operators/save_op.cc index 8f61525436..338e2fbb5d 100644 --- a/paddle/fluid/operators/save_op.cc +++ b/paddle/fluid/operators/save_op.cc @@ -15,28 +15,13 @@ limitations under the License. */ #include #include #include +#include #include -#include "paddle/fluid/framework/data_type.h" -#include "paddle/fluid/framework/data_type_transform.h" -#include "paddle/fluid/framework/framework.pb.h" -#include "paddle/fluid/framework/lod_tensor.h" -#include "paddle/fluid/framework/op_registry.h" -#include "paddle/fluid/framework/selected_rows.h" -#include "paddle/fluid/framework/variable.h" -#include "paddle/fluid/platform/device_context.h" -#include "paddle/fluid/platform/port.h" +#include "paddle/fluid/operators/save_op.h" namespace paddle { namespace operators { - -using Tensor = framework::Tensor; -using LoDTensor = framework::LoDTensor; - -// define LOOKUP_TABLE_PATH for checkpoint notify to save lookup table variables -// to directory specified. -constexpr char LOOKUP_TABLE_PATH[] = "kLookupTablePath"; - class SaveOp : public framework::OperatorWithKernel { public: using framework::OperatorWithKernel::OperatorWithKernel; @@ -97,102 +82,6 @@ class SaveOpShapeInference : public framework::InferShapeBase { void operator()(framework::InferShapeContext *ctx) const override {} }; -template -class SaveOpKernel : public framework::OpKernel { - public: - void Compute(const framework::ExecutionContext &ctx) const override { - auto place = ctx.GetPlace(); - - auto *input_var = ctx.InputVar("X"); - auto iname = ctx.Inputs("X").data(); - PADDLE_ENFORCE(input_var != nullptr, "Cannot find variable %s for save_op", - iname); - - if (input_var->IsType()) { - SaveLodTensor(ctx, place, input_var); - } else if (input_var->IsType()) { - SaveSelectedRows(ctx, place, input_var); - } else { - PADDLE_ENFORCE( - false, - "SaveOp only support LoDTensor and SelectedRows, %s has wrong type", - iname); - } - } - - void SaveLodTensor(const framework::ExecutionContext &ctx, - const platform::Place &place, - const framework::Variable *var) const { - auto filename = ctx.Attr("file_path"); - auto overwrite = ctx.Attr("overwrite"); - - if (FileExists(filename) && !overwrite) { - PADDLE_THROW("%s is existed, cannot save to it when overwrite=false", - filename, overwrite); - } - - MkDirRecursively(DirName(filename).c_str()); - - auto &tensor = var->Get(); - - // get device context from pool - platform::DeviceContextPool &pool = platform::DeviceContextPool::Instance(); - auto &dev_ctx = *pool.Get(place); - - // FIXME(yuyang18): We save variable to local file now, but we should change - // it to save an output stream. - std::ofstream fout(filename, std::ios::binary); - PADDLE_ENFORCE(static_cast(fout), "Cannot open %s to write", - filename); - - auto save_as_fp16 = ctx.Attr("save_as_fp16"); - auto in_dtype = tensor.type(); - auto out_dtype = save_as_fp16 ? framework::proto::VarType::FP16 : in_dtype; - - if (in_dtype != out_dtype) { - auto in_kernel_type = framework::OpKernelType(in_dtype, place); - auto out_kernel_type = framework::OpKernelType(out_dtype, place); - framework::LoDTensor out; - framework::TransDataType(in_kernel_type, out_kernel_type, tensor, &out); - // copy LoD info to the new tensor - out.set_lod(tensor.lod()); - framework::SerializeToStream(fout, out, dev_ctx); - } else { - framework::SerializeToStream(fout, tensor, dev_ctx); - } - fout.close(); - } - - void SaveSelectedRows(const framework::ExecutionContext &ctx, - const platform::Place &place, - const framework::Variable *var) const { - framework::Variable *out_put_var = ctx.OutputVar(LOOKUP_TABLE_PATH); - PADDLE_ENFORCE( - out_put_var != nullptr, - "Can not find variable kLookupTablePath for SaveSelectedRows"); - auto *lt_var = out_put_var->GetMutable(); - - std::string filename = lt_var->data(); - VLOG(4) << "SaveSelectedRows get File name: " << filename; - - MkDirRecursively(DirName(filename).c_str()); - - auto &selectedRows = var->Get(); - - // get device context from pool - platform::DeviceContextPool &pool = platform::DeviceContextPool::Instance(); - auto &dev_ctx = *pool.Get(place); - - // FIXME(yuyang18): We save variable to local file now, but we should change - // it to save an output stream. - std::ofstream fout(filename, std::ios::binary); - PADDLE_ENFORCE(static_cast(fout), "Cannot open %s to write", - filename); - framework::SerializeToStream(fout, selectedRows, dev_ctx); - fout.close(); - } -}; - } // namespace operators } // namespace paddle diff --git a/paddle/fluid/operators/save_op.cu b/paddle/fluid/operators/save_op.cu new file mode 100644 index 0000000000..ebb7e1832c --- /dev/null +++ b/paddle/fluid/operators/save_op.cu @@ -0,0 +1,26 @@ +/* Copyright (c) 2016 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/save_op.h" +#include "paddle/fluid/platform/float16.h" + +namespace ops = paddle::operators; + +REGISTER_OP_CUDA_KERNEL( + save, ops::SaveOpKernel, + ops::SaveOpKernel, + ops::SaveOpKernel, + ops::SaveOpKernel, + ops::SaveOpKernel); diff --git a/paddle/fluid/operators/save_op.h b/paddle/fluid/operators/save_op.h new file mode 100644 index 0000000000..642235aad5 --- /dev/null +++ b/paddle/fluid/operators/save_op.h @@ -0,0 +1,133 @@ +/* Copyright (c) 2016 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 + +#include "paddle/fluid/framework/data_type.h" +#include "paddle/fluid/framework/data_type_transform.h" +#include "paddle/fluid/framework/framework.pb.h" +#include "paddle/fluid/framework/lod_tensor.h" +#include "paddle/fluid/framework/op_registry.h" +#include "paddle/fluid/framework/selected_rows.h" +#include "paddle/fluid/framework/variable.h" + +namespace paddle { +namespace operators { +// define LOOKUP_TABLE_PATH for checkpoint notify to save lookup table variables +// to directory specified. +constexpr char LOOKUP_TABLE_PATH[] = "kLookupTablePath"; +template +class SaveOpKernel : public framework::OpKernel { + public: + void Compute(const framework::ExecutionContext &ctx) const override { + auto place = ctx.GetPlace(); + + auto *input_var = ctx.InputVar("X"); + auto iname = ctx.Inputs("X").data(); + PADDLE_ENFORCE(input_var != nullptr, "Cannot find variable %s for save_op", + iname); + + if (input_var->IsType()) { + SaveLodTensor(ctx, place, input_var); + } else if (input_var->IsType()) { + SaveSelectedRows(ctx, place, input_var); + } else { + PADDLE_ENFORCE( + false, + "SaveOp only support LoDTensor and SelectedRows, %s has wrong type", + iname); + } + } + + void SaveLodTensor(const framework::ExecutionContext &ctx, + const platform::Place &place, + const framework::Variable *var) const { + auto filename = ctx.Attr("file_path"); + auto overwrite = ctx.Attr("overwrite"); + + if (FileExists(filename) && !overwrite) { + PADDLE_THROW("%s is existed, cannot save to it when overwrite=false", + filename, overwrite); + } + + MkDirRecursively(DirName(filename).c_str()); + + auto &tensor = var->Get(); + + // get device context from pool + platform::DeviceContextPool &pool = platform::DeviceContextPool::Instance(); + auto &dev_ctx = *pool.Get(place); + + // FIXME(yuyang18): We save variable to local file now, but we should change + // it to save an output stream. + std::ofstream fout(filename, std::ios::binary); + PADDLE_ENFORCE(static_cast(fout), "Cannot open %s to write", + filename); + + auto save_as_fp16 = ctx.Attr("save_as_fp16"); + auto in_dtype = tensor.type(); + auto out_dtype = save_as_fp16 ? framework::proto::VarType::FP16 : in_dtype; + + if (in_dtype != out_dtype) { + auto in_kernel_type = framework::OpKernelType(in_dtype, place); + auto out_kernel_type = framework::OpKernelType(out_dtype, place); + framework::LoDTensor out; + framework::TransDataType(in_kernel_type, out_kernel_type, tensor, &out); + // copy LoD info to the new tensor + out.set_lod(tensor.lod()); + framework::SerializeToStream(fout, out, dev_ctx); + } else { + framework::SerializeToStream(fout, tensor, dev_ctx); + } + fout.close(); + } + + void SaveSelectedRows(const framework::ExecutionContext &ctx, + const platform::Place &place, + const framework::Variable *var) const { + framework::Variable *out_put_var = ctx.OutputVar(LOOKUP_TABLE_PATH); + PADDLE_ENFORCE( + out_put_var != nullptr, + "Can not find variable kLookupTablePath for SaveSelectedRows"); + auto *lt_var = out_put_var->GetMutable(); + + std::string filename = lt_var->data(); + VLOG(4) << "SaveSelectedRows get File name: " << filename; + + MkDirRecursively(DirName(filename).c_str()); + + auto &selectedRows = var->Get(); + + // get device context from pool + platform::DeviceContextPool &pool = platform::DeviceContextPool::Instance(); + auto &dev_ctx = *pool.Get(place); + + // FIXME(yuyang18): We save variable to local file now, but we should change + // it to save an output stream. + std::ofstream fout(filename, std::ios::binary); + PADDLE_ENFORCE(static_cast(fout), "Cannot open %s to write", + filename); + framework::SerializeToStream(fout, selectedRows, dev_ctx); + fout.close(); + } +}; + +} // namespace operators +} // namespace paddle From 09442fb27e680857d343dbc6087870cac468fed5 Mon Sep 17 00:00:00 2001 From: lujun Date: Thu, 21 Mar 2019 02:19:48 +0800 Subject: [PATCH 17/44] checkpoint pr be moved here, test=develop --- paddle/fluid/operators/load_combine_op.cu | 1 + paddle/fluid/operators/load_op.cu | 1 + paddle/fluid/operators/save_combine_op.cu | 1 + paddle/fluid/operators/save_op.cu | 1 + 4 files changed, 4 insertions(+) diff --git a/paddle/fluid/operators/load_combine_op.cu b/paddle/fluid/operators/load_combine_op.cu index f23f4ea5aa..2a42c0daa7 100644 --- a/paddle/fluid/operators/load_combine_op.cu +++ b/paddle/fluid/operators/load_combine_op.cu @@ -21,4 +21,5 @@ REGISTER_OP_CUDA_KERNEL( ops::LoadCombineOpKernel, ops::LoadCombineOpKernel, ops::LoadCombineOpKernel, + ops::LoadCombineOpKernel, ops::LoadCombineOpKernel); diff --git a/paddle/fluid/operators/load_op.cu b/paddle/fluid/operators/load_op.cu index dcbef0b592..90f78110f8 100644 --- a/paddle/fluid/operators/load_op.cu +++ b/paddle/fluid/operators/load_op.cu @@ -20,4 +20,5 @@ REGISTER_OP_CUDA_KERNEL( load, ops::LoadOpKernel, ops::LoadOpKernel, ops::LoadOpKernel, + ops::LoadOpKernel, ops::LoadOpKernel); diff --git a/paddle/fluid/operators/save_combine_op.cu b/paddle/fluid/operators/save_combine_op.cu index 71476fd802..bc4478b51b 100644 --- a/paddle/fluid/operators/save_combine_op.cu +++ b/paddle/fluid/operators/save_combine_op.cu @@ -21,4 +21,5 @@ REGISTER_OP_CUDA_KERNEL( ops::SaveCombineOpKernel, ops::SaveCombineOpKernel, ops::SaveCombineOpKernel, + ops::SaveCombineOpKernel, ops::SaveCombineOpKernel); diff --git a/paddle/fluid/operators/save_op.cu b/paddle/fluid/operators/save_op.cu index ebb7e1832c..0a778a694e 100644 --- a/paddle/fluid/operators/save_op.cu +++ b/paddle/fluid/operators/save_op.cu @@ -21,6 +21,7 @@ REGISTER_OP_CUDA_KERNEL( save, ops::SaveOpKernel, ops::SaveOpKernel, ops::SaveOpKernel, + ops::SaveOpKernel, ops::SaveOpKernel, ops::SaveOpKernel); From 8965819fbbd23f5ff159903eb673898c45408aa4 Mon Sep 17 00:00:00 2001 From: Zhen Wang Date: Thu, 21 Mar 2019 12:36:11 +0800 Subject: [PATCH 18/44] rewrite the cuda kernels of channel_wise_quant_op and channe_wise_dequant_op. test=develop --- paddle/fluid/operators/fake_dequantize_op.cc | 43 +++++ paddle/fluid/operators/fake_dequantize_op.cu | 58 ++++++ paddle/fluid/operators/fake_dequantize_op.h | 43 ++--- paddle/fluid/operators/fake_quantize_op.cc | 45 +++++ paddle/fluid/operators/fake_quantize_op.cu | 125 ++++++++++--- paddle/fluid/operators/fake_quantize_op.h | 32 ++-- .../slim/quantization/quantization_pass.py | 2 - .../slim/tests/test_quantization_pass.py | 173 +++++++++++++----- 8 files changed, 405 insertions(+), 116 deletions(-) diff --git a/paddle/fluid/operators/fake_dequantize_op.cc b/paddle/fluid/operators/fake_dequantize_op.cc index 68c7227e5a..4a8937ba1c 100644 --- a/paddle/fluid/operators/fake_dequantize_op.cc +++ b/paddle/fluid/operators/fake_dequantize_op.cc @@ -33,8 +33,51 @@ struct DequantizeFunctor { } }; +template +struct ChannelDequantizeFunctor { + void operator()(const platform::CPUDeviceContext& dev_ctx, + const framework::Tensor* in, const framework::Tensor** scales, + const int scale_num, T max_range, framework::Tensor* out) { + if (scale_num == 1) { + const int channel = in->dims()[0]; + const T* scale_factor = scales[0]->data(); + for (int i = 0; i < channel; i++) { + T s = scale_factor[i]; + framework::Tensor one_channel_in = in->Slice(i, i + 1); + framework::Tensor one_channel_out = out->Slice(i, i + 1); + auto in_e = framework::EigenVector::Flatten(one_channel_in); + auto out_e = framework::EigenVector::Flatten(one_channel_out); + auto& dev = *dev_ctx.eigen_device(); + out_e.device(dev) = (s / max_range) * in_e; + } + } else if (scale_num == 2) { + int batch_size = in->dims()[0]; + int channel = in->dims()[1]; + const T* scale_one = scales[0]->data(); + const T* scale_two = scales[1]->data(); + for (int i = 0; i < batch_size; i++) { + framework::Tensor one_batch_in = in->Slice(i, i + 1).Resize( + framework::slice_ddim(in->dims(), 1, in->dims().size())); + framework::Tensor one_batch_out = out->Slice(i, i + 1).Resize( + framework::slice_ddim(out->dims(), 1, out->dims().size())); + for (int j = 0; j < channel; j++) { + T s = scale_one[j]; + framework::Tensor one_channel_in = one_batch_in.Slice(j, j + 1); + framework::Tensor one_channel_out = one_batch_out.Slice(j, j + 1); + auto in_e = framework::EigenVector::Flatten(one_channel_in); + auto out_e = framework::EigenVector::Flatten(one_channel_out); + auto& dev = *dev_ctx.eigen_device(); + out_e.device(dev) = (s * scale_two[0] / max_range) * in_e; + } + } + } + } +}; + template struct DequantizeFunctor; template struct DequantizeFunctor; +template struct ChannelDequantizeFunctor; +template struct ChannelDequantizeFunctor; class FakeDequantizeMaxAbsOp : public framework::OperatorWithKernel { public: diff --git a/paddle/fluid/operators/fake_dequantize_op.cu b/paddle/fluid/operators/fake_dequantize_op.cu index 35dcc69279..02f9dc827d 100644 --- a/paddle/fluid/operators/fake_dequantize_op.cu +++ b/paddle/fluid/operators/fake_dequantize_op.cu @@ -44,8 +44,66 @@ struct DequantizeFunctor { } }; +template +__global__ void DequantizeOneScale(const T* in, const T* scale, T max_range, + int num, int channel, T* out) { + int tid = threadIdx.x; + int channel_size = num / channel; + const T* in_c = in + blockIdx.x * channel_size; + T* out_c = out + blockIdx.x * channel_size; + for (int i = tid; i < channel_size; i += blockDim.x) { + out_c[i] = in_c[i] * scale[blockIdx.x] / max_range; + } +} + +template +__global__ void DequantizeTwoScale(const T* in, const T* scale_one, + const T* scale_two, T max_range, int num, + int batch_size, int channel, T* out) { + int tid = threadIdx.x; + int channel_size = num / (batch_size * channel); + int scale_index = blockIdx.x % channel; + const T* in_c = in + blockIdx.x * channel_size; + T* out_c = out + blockIdx.x * channel_size; + for (int i = tid; i < channel_size; i += blockDim.x) { + out_c[i] = in_c[i] * scale_one[scale_index] * scale_two[0] / max_range; + } +} + +template +struct ChannelDequantizeFunctor { + void operator()(const platform::CUDADeviceContext& dev_ctx, + const framework::Tensor* in, const framework::Tensor** scales, + const int scale_num, T max_range, framework::Tensor* out) { + const T* in_data = in->data(); + T* out_data = out->mutable_data(dev_ctx.GetPlace()); + if (scale_num == 1) { + int num = in->numel(); + int channel = in->dims()[0]; + const T* scale_factor = scales[0]->data(); + int block = 1024; + int grid = channel; + DequantizeOneScale<<>>( + in_data, scale_factor, max_range, num, channel, out_data); + } else if (scale_num == 2) { + int num = in->numel(); + int batch_size = in->dims()[0]; + int channel = in->dims()[1]; + const T* scale_one = scales[0]->data(); + const T* scale_two = scales[1]->data(); + int block = 1024; + int grid = batch_size * channel; + DequantizeTwoScale<<>>( + in_data, scale_one, scale_two, max_range, num, batch_size, channel, + out_data); + } + } +}; + template struct DequantizeFunctor; template struct DequantizeFunctor; +template struct ChannelDequantizeFunctor; +template struct ChannelDequantizeFunctor; } // namespace operators } // namespace paddle diff --git a/paddle/fluid/operators/fake_dequantize_op.h b/paddle/fluid/operators/fake_dequantize_op.h index 1a504bf035..ed9a0a4d65 100644 --- a/paddle/fluid/operators/fake_dequantize_op.h +++ b/paddle/fluid/operators/fake_dequantize_op.h @@ -29,6 +29,13 @@ struct DequantizeFunctor { framework::Tensor* out); }; +template +struct ChannelDequantizeFunctor { + void operator()(const DeviceContext& dev_ctx, const framework::Tensor* in, + const framework::Tensor** scales, const int scale_num, + T max_range, framework::Tensor* out); +}; + template class FakeDequantizeMaxAbsKernel : public framework::OpKernel { public: @@ -56,50 +63,32 @@ class FakeChannelWiseDequantizeMaxAbsKernel : public framework::OpKernel { auto* out = ctx.Output("Out"); auto quant_bits = ctx.Attr>("quant_bits"); - int max_range = std::pow(2, quant_bits[0] - 1) - 1; + int max_range = 1; auto& dev_ctx = ctx.template device_context(); out->mutable_data(dev_ctx.GetPlace()); - - auto dequant = DequantizeFunctor(); - if (scales.size() == 1) { + int scale_num = scales.size(); + if (scale_num == 1) { PADDLE_ENFORCE_EQ( scales[0]->numel(), in->dims()[0], "The number of first scale values must be the same with " "first dimension value of Input(X) when the `Scales` has only one " "element."); - for (int64_t i = 0; i < in->dims()[0]; i++) { - framework::Tensor one_channel_in = in->Slice(i, i + 1); - framework::Tensor one_channel_out = out->Slice(i, i + 1); - framework::Tensor one_channel_scale = scales[0]->Slice(i, i + 1); - dequant(dev_ctx, &one_channel_in, &one_channel_scale, - static_cast(max_range), &one_channel_out); - } - } else if (scales.size() == 2) { + max_range *= (std::pow(2, quant_bits[0] - 1) - 1); + } else if (scale_num == 2) { PADDLE_ENFORCE_EQ( scales[0]->numel(), in->dims()[1], "The number of first scale values must be the same with " "second dimension value of Input(X) when the `Scales` has two " "elements."); - for (int64_t i = 0; i < in->dims()[0]; i++) { - framework::Tensor one_batch_in = in->Slice(i, i + 1).Resize( - framework::slice_ddim(in->dims(), 1, in->dims().size())); - framework::Tensor one_batch_out = out->Slice(i, i + 1).Resize( - framework::slice_ddim(out->dims(), 1, out->dims().size())); - for (int64_t j = 0; j < in->dims()[1]; j++) { - framework::Tensor one_channel_in = one_batch_in.Slice(j, j + 1); - framework::Tensor one_channel_out = one_batch_out.Slice(j, j + 1); - framework::Tensor one_channel_scale = scales[0]->Slice(j, j + 1); - dequant(dev_ctx, &one_channel_in, &one_channel_scale, - static_cast(max_range), &one_channel_out); - } - } PADDLE_ENFORCE_EQ( scales[1]->numel(), 1, "The second scale tensor should only have one value at now."); - max_range = std::pow(2, quant_bits[1] - 1) - 1; - dequant(dev_ctx, out, scales[1], static_cast(max_range), out); + max_range *= (std::pow(2, quant_bits[0] - 1) - 1) * + (std::pow(2, quant_bits[1] - 1) - 1); } + ChannelDequantizeFunctor()( + dev_ctx, in, scales.data(), scale_num, static_cast(max_range), out); } }; diff --git a/paddle/fluid/operators/fake_quantize_op.cc b/paddle/fluid/operators/fake_quantize_op.cc index f9f28f60c0..054ef4658c 100644 --- a/paddle/fluid/operators/fake_quantize_op.cc +++ b/paddle/fluid/operators/fake_quantize_op.cc @@ -37,6 +37,21 @@ struct FindAbsMaxFunctor { template struct FindAbsMaxFunctor; +template +struct FindChannelAbsMaxFunctor { + void operator()(const platform::CPUDeviceContext& ctx, const T* in, + const int num, const int channel, T* out) { + const int channel_size = num / channel; + for (int i = 0; i < channel; i++) { + auto* start = in + i * channel_size; + auto* end = in + (i + 1) * channel_size; + out[i] = std::abs(*(std::max_element(start, end, Compare()))); + } + } +}; + +template struct FindChannelAbsMaxFunctor; + template struct ClipAndFakeQuantFunctor { void operator()(const platform::CPUDeviceContext& ctx, @@ -53,6 +68,36 @@ struct ClipAndFakeQuantFunctor { template struct ClipAndFakeQuantFunctor; +template +struct ChannelClipAndFakeQuantFunctor { + void operator()(const platform::CPUDeviceContext& ctx, + const framework::Tensor& in, const framework::Tensor& scale, + const int bin_cnt, const int channel, + framework::Tensor* out) { + auto* scale_data = scale.data(); + auto* in_data = in.data(); + auto* out_data = out->mutable_data(ctx.GetPlace()); + const int channel_size = in.numel() / channel; + platform::Transform trans; + for (int i = 0; i < channel; i++) { + T s = scale_data[i]; + auto* start = in_data + i * channel_size; + auto* end = in_data + (i + 1) * channel_size; + trans(ctx, start, end, out_data + i * channel_size, + ClipFunctor(-s, s)); + } + for (int i = 0; i < channel; i++) { + T s = scale_data[i]; + framework::Tensor one_channel_out = out->Slice(i, i + 1); + auto out_e = framework::EigenVector::Flatten(one_channel_out); + out_e.device(*ctx.eigen_device()) = (bin_cnt / s * out_e).round(); + } + } +}; + +template struct ChannelClipAndFakeQuantFunctor; + template struct FindRangeAbsMaxFunctor { void operator()(const platform::CPUDeviceContext& ctx, diff --git a/paddle/fluid/operators/fake_quantize_op.cu b/paddle/fluid/operators/fake_quantize_op.cu index 3707f6772e..33bd275e5c 100644 --- a/paddle/fluid/operators/fake_quantize_op.cu +++ b/paddle/fluid/operators/fake_quantize_op.cu @@ -74,6 +74,45 @@ struct FindAbsMaxFunctor { template struct FindAbsMaxFunctor; +template +__global__ void FindChannelAbsMaxKernel(const T* in, const int n, const int c, + T* out) { + int tid = threadIdx.x; + int channel_size = n / c; + const T* in_c = in + blockIdx.x * channel_size; + extern __shared__ T shared_max_data[]; + shared_max_data[tid] = T(0); + for (int i = tid; i < channel_size; i += blockDim.x) { + T tmp = fabs(in_c[i]); + if (tmp > shared_max_data[tid]) { + shared_max_data[tid] = tmp; + } + } + __syncthreads(); + for (int i = blockDim.x / 2; i > 0; i >>= 1) { + if (tid < i && (shared_max_data[tid] < shared_max_data[tid + i])) { + shared_max_data[tid] = shared_max_data[tid + i]; + } + __syncthreads(); + } + if (tid == 0) { + out[blockIdx.x] = shared_max_data[0]; + } +} + +template +struct FindChannelAbsMaxFunctor { + void operator()(const platform::CUDADeviceContext& ctx, const T* in, + const int num, const int channel, T* out) { + int block = 1024; + int grid = channel; + FindChannelAbsMaxKernel<<>>( + in, num, channel, out); + } +}; + +template struct FindChannelAbsMaxFunctor; + template __global__ void ClipAndQuantKernel(const T* in, const T* scale, const int bin_cnt, const int n, T* out) { @@ -82,14 +121,76 @@ __global__ void ClipAndQuantKernel(const T* in, const T* scale, T s = scale[0]; for (int i = bid; i < n; i += blockDim.x * gridDim.x) { - T x = in[bid]; + T x = in[i]; T v = x > s ? s : x; v = v < -s ? -s : v; v = bin_cnt / s * v; - out[bid] = round(v); + out[i] = round(v); } } +template +struct ClipAndFakeQuantFunctor { + void operator()(const platform::CUDADeviceContext& ctx, + const framework::Tensor& in, const framework::Tensor& scale, + const int bin_cnt, framework::Tensor* out) { + int num = in.numel(); + int block = 1024; + int grid = (block - 1 + num) / block; + + const T* in_data = in.data(); + const T* scale_data = scale.data(); + T* out_data = out->mutable_data(ctx.GetPlace()); + + ClipAndQuantKernel<<>>( + in_data, scale_data, bin_cnt, num, out_data); + } +}; + +template struct ClipAndFakeQuantFunctor; + +template +__global__ void ChannelClipAndQuantKernel(const T* in, const T* scale, + const int bin_cnt, const int n, + const int c, T* out) { + int tid = threadIdx.x; + + int channel_size = n / c; + const T* in_c = in + blockIdx.x * channel_size; + T* out_c = out + blockIdx.x * channel_size; + + T s = scale[blockIdx.x]; + for (int i = tid; i < channel_size; i += blockDim.x) { + T x = in_c[i]; + T v = x > s ? s : x; + v = v < -s ? -s : v; + v = bin_cnt / s * v; + out_c[i] = round(v); + } +} + +template +struct ChannelClipAndFakeQuantFunctor { + void operator()(const platform::CUDADeviceContext& ctx, + const framework::Tensor& in, const framework::Tensor& scale, + const int bin_cnt, const int channel, + framework::Tensor* out) { + int num = in.numel(); + int block = 1024; + int grid = channel; + + const T* in_data = in.data(); + const T* scale_data = scale.data(); + T* out_data = out->mutable_data(ctx.GetPlace()); + + ChannelClipAndQuantKernel<<>>( + in_data, scale_data, bin_cnt, num, channel, out_data); + } +}; + +template struct ChannelClipAndFakeQuantFunctor; + template __global__ void FindRangeAbsMaxAndFillArray(const T* cur_scale, const T* last_scale, @@ -182,26 +283,6 @@ struct FindMovingAverageAbsMaxFunctor { template struct FindMovingAverageAbsMaxFunctor; -template -struct ClipAndFakeQuantFunctor { - void operator()(const platform::CUDADeviceContext& ctx, - const framework::Tensor& in, const framework::Tensor& scale, - const int bin_cnt, framework::Tensor* out) { - int num = in.numel(); - int block = 1024; - int grid = (block - 1 + num) / block; - - const T* in_data = in.data(); - const T* scale_data = scale.data(); - T* out_data = out->mutable_data(ctx.GetPlace()); - - ClipAndQuantKernel<<>>( - in_data, scale_data, bin_cnt, num, out_data); - } -}; - -template struct ClipAndFakeQuantFunctor; - } // namespace operators } // namespace paddle diff --git a/paddle/fluid/operators/fake_quantize_op.h b/paddle/fluid/operators/fake_quantize_op.h index 2616cd996f..5ab38b086d 100644 --- a/paddle/fluid/operators/fake_quantize_op.h +++ b/paddle/fluid/operators/fake_quantize_op.h @@ -42,6 +42,19 @@ struct FindRangeAbsMaxFunctor { framework::Tensor* scales_arr, framework::Tensor* out_scale); }; +template +struct FindChannelAbsMaxFunctor { + void operator()(const DeviceContext& ctx, const T* in, const int num, + const int channel, T* out); +}; + +template +struct ChannelClipAndFakeQuantFunctor { + void operator()(const DeviceContext& ctx, const framework::Tensor& in, + const framework::Tensor& scale, const int bin_cnt, + const int channel, framework::Tensor* out); +}; + template struct FindMovingAverageAbsMaxFunctor { void operator()(const DeviceContext& ctx, const framework::Tensor& in_accum, @@ -86,21 +99,10 @@ class FakeChannelWiseQuantizeAbsMaxKernel : public framework::OpKernel { int bin_cnt = std::pow(2, bit_length - 1) - 1; auto& dev_ctx = context.template device_context(); - auto find_abs_max = FindAbsMaxFunctor(); - for (int64_t i = 0; i < in->dims()[0]; i++) { - framework::Tensor one_channel = in->Slice(i, i + 1); - const T* one_channel_data = one_channel.data(); - find_abs_max(dev_ctx, one_channel_data, one_channel.numel(), - &out_scale_data[i]); - } - auto clip_quant = ClipAndFakeQuantFunctor(); - for (int64_t i = 0; i < in->dims()[0]; i++) { - framework::Tensor one_channel_in = in->Slice(i, i + 1); - framework::Tensor one_channel_out = out->Slice(i, i + 1); - framework::Tensor one_channel_scale = out_scale->Slice(i, i + 1); - clip_quant(dev_ctx, one_channel_in, one_channel_scale, bin_cnt, - &one_channel_out); - } + FindChannelAbsMaxFunctor()( + dev_ctx, in->data(), in->numel(), in->dims()[0], out_scale_data); + ChannelClipAndFakeQuantFunctor()( + dev_ctx, *in, *out_scale, bin_cnt, in->dims()[0], out); } }; diff --git a/python/paddle/fluid/contrib/slim/quantization/quantization_pass.py b/python/paddle/fluid/contrib/slim/quantization/quantization_pass.py index 03ffd2795d..5dcef50671 100644 --- a/python/paddle/fluid/contrib/slim/quantization/quantization_pass.py +++ b/python/paddle/fluid/contrib/slim/quantization/quantization_pass.py @@ -576,8 +576,6 @@ class QuantizationFreezePass(object): elif self._weight_quantize_type == 'channel_wise_abs_max': param = self._load_var(input_arg_name) if len(param.shape) == 4: # conv2d or depthwise_conv2d - print('DEBUG**************************: %s' % - input_arg_name) scale_v = [] for i in range(param.shape[0]): scale_v.append(np.max(np.abs(param[i]))) diff --git a/python/paddle/fluid/contrib/slim/tests/test_quantization_pass.py b/python/paddle/fluid/contrib/slim/tests/test_quantization_pass.py index cda7ecbd82..c7feca0b82 100644 --- a/python/paddle/fluid/contrib/slim/tests/test_quantization_pass.py +++ b/python/paddle/fluid/contrib/slim/tests/test_quantization_pass.py @@ -127,7 +127,7 @@ class TestQuantizationTransformPass(unittest.TestCase): arg_name.endswith('.quantized.dequantized')) self.assertTrue(arg_name in quantized_ops) - def linear_fc_quant(self, quant_type, for_ci=False): + def linear_fc_quant(self, activation_quant_type, for_ci=False): main = fluid.Program() startup = fluid.Program() with fluid.program_guard(main, startup): @@ -140,14 +140,15 @@ class TestQuantizationTransformPass(unittest.TestCase): transform_pass = QuantizationTransformPass( scope=fluid.global_scope(), place=place, - activation_quantize_type=quant_type) + activation_quantize_type=activation_quant_type) transform_pass.apply(graph) if not for_ci: marked_nodes = set() for op in graph.all_op_nodes(): if op.name().find('quantize') > -1: marked_nodes.add(op) - graph.draw('.', 'quantize_fc_' + quant_type, marked_nodes) + graph.draw('.', 'quantize_fc_' + activation_quant_type, + marked_nodes) program = graph.to_program() self.check_program(transform_pass, program) val_graph = IrGraph(core.Graph(program.desc), for_test=False) @@ -156,7 +157,8 @@ class TestQuantizationTransformPass(unittest.TestCase): for op in val_graph.all_op_nodes(): if op.name().find('quantize') > -1: val_marked_nodes.add(op) - val_graph.draw('.', 'val_fc_' + quant_type, val_marked_nodes) + val_graph.draw('.', 'val_fc_' + activation_quant_type, + val_marked_nodes) def test_linear_fc_quant_abs_max(self): self.linear_fc_quant('abs_max', for_ci=True) @@ -167,7 +169,7 @@ class TestQuantizationTransformPass(unittest.TestCase): def test_linear_fc_quant_moving_average_abs_max(self): self.linear_fc_quant('moving_average_abs_max', for_ci=True) - def residual_block_quant(self, quant_type, for_ci=False): + def residual_block_quant(self, activation_quant_type, for_ci=False): main = fluid.Program() startup = fluid.Program() with fluid.program_guard(main, startup): @@ -180,14 +182,15 @@ class TestQuantizationTransformPass(unittest.TestCase): transform_pass = QuantizationTransformPass( scope=fluid.global_scope(), place=place, - activation_quantize_type=quant_type) + activation_quantize_type=activation_quant_type) transform_pass.apply(graph) if not for_ci: marked_nodes = set() for op in graph.all_op_nodes(): if op.name().find('quantize') > -1: marked_nodes.add(op) - graph.draw('.', 'quantize_residual_' + quant_type, marked_nodes) + graph.draw('.', 'quantize_residual_' + activation_quant_type, + marked_nodes) program = graph.to_program() self.check_program(transform_pass, program) val_graph = IrGraph(core.Graph(program.desc), for_test=False) @@ -196,7 +199,8 @@ class TestQuantizationTransformPass(unittest.TestCase): for op in val_graph.all_op_nodes(): if op.name().find('quantize') > -1: val_marked_nodes.add(op) - val_graph.draw('.', 'val_residual_' + quant_type, val_marked_nodes) + val_graph.draw('.', 'val_residual_' + activation_quant_type, + val_marked_nodes) def test_residual_block_abs_max(self): self.residual_block_quant('abs_max', for_ci=True) @@ -209,7 +213,12 @@ class TestQuantizationTransformPass(unittest.TestCase): class TestQuantizationFreezePass(unittest.TestCase): - def freeze_graph(self, use_cuda, seed, quant_type, for_ci=False): + def freeze_graph(self, + use_cuda, + seed, + activation_quant_type, + weight_quant_type='abs_max', + for_ci=False): def build_program(main, startup, is_test): main.random_seed = seed startup.random_seed = seed @@ -245,10 +254,10 @@ class TestQuantizationFreezePass(unittest.TestCase): transform_pass = QuantizationTransformPass( scope=scope, place=place, - activation_quantize_type=quant_type, - weight_quantize_type='channel_wise_abs_max') + activation_quantize_type=activation_quant_type, + weight_quantize_type=weight_quant_type) #transform_pass = QuantizationTransformPass( - # scope=scope, place=place, activation_quantize_type=quant_type) + # scope=scope, place=place, activation_quantize_type=activation_quant_type) transform_pass.apply(main_graph) transform_pass.apply(test_graph) dev_name = '_gpu_' if use_cuda else '_cpu_' @@ -257,12 +266,14 @@ class TestQuantizationFreezePass(unittest.TestCase): for op in main_graph.all_op_nodes(): if op.name().find('quantize') > -1: marked_nodes.add(op) - main_graph.draw('.', 'main' + dev_name + quant_type, marked_nodes) + main_graph.draw('.', 'main' + dev_name + activation_quant_type + '_' + + weight_quant_type, marked_nodes) marked_nodes = set() for op in test_graph.all_op_nodes(): if op.name().find('quantize') > -1: marked_nodes.add(op) - test_graph.draw('.', 'test' + dev_name + quant_type, marked_nodes) + test_graph.draw('.', 'test' + dev_name + activation_quant_type + '_' + + weight_quant_type, marked_nodes) build_strategy = fluid.BuildStrategy() build_strategy.memory_optimize = False @@ -287,8 +298,9 @@ class TestQuantizationFreezePass(unittest.TestCase): feed=feeder.feed(data), fetch_list=[loss]) if not for_ci: - print('{}: {}'.format('loss' + dev_name + quant_type, - loss_v)) + print('{}: {}'.format('loss' + dev_name + + activation_quant_type + '_' + + weight_quant_type, loss_v)) test_data = next(test_reader()) with fluid.program_guard(quantized_test_program): @@ -302,9 +314,7 @@ class TestQuantizationFreezePass(unittest.TestCase): # Freeze graph for inference, but the weight of fc/conv is still float type. freeze_pass = QuantizationFreezePass( - scope=scope, - place=place, - weight_quantize_type='channel_wise_abs_max') + scope=scope, place=place, weight_quantize_type=weight_quant_type) #freeze_pass = QuantizationFreezePass(scope=scope, place=place) freeze_pass.apply(test_graph) if not for_ci: @@ -312,7 +322,8 @@ class TestQuantizationFreezePass(unittest.TestCase): for op in test_graph.all_op_nodes(): if op.name().find('quantize') > -1: marked_nodes.add(op) - test_graph.draw('.', 'test_freeze' + dev_name + quant_type, + test_graph.draw('.', 'test_freeze' + dev_name + + activation_quant_type + '_' + weight_quant_type, marked_nodes) server_program = test_graph.to_program() @@ -322,18 +333,20 @@ class TestQuantizationFreezePass(unittest.TestCase): fetch_list=[loss]) self.assertAlmostEqual(test_loss1, test_loss2, delta=5e-3) if not for_ci: - print('{}: {}'.format('test_loss1' + dev_name + quant_type, - test_loss1)) - print('{}: {}'.format('test_loss2' + dev_name + quant_type, - test_loss2)) + print( + '{}: {}'.format('test_loss1' + dev_name + activation_quant_type + + '_' + weight_quant_type, test_loss1)) + print( + '{}: {}'.format('test_loss2' + dev_name + activation_quant_type + + '_' + weight_quant_type, test_loss2)) w_freeze = np.array(scope.find_var('conv2d_1.w_0').get_tensor()) # Maybe failed, this is due to the calculation precision # self.assertAlmostEqual(np.sum(w_freeze), np.sum(w_quant)) if not for_ci: - print('{}: {}'.format('w_freeze' + dev_name + quant_type, - np.sum(w_freeze))) - print('{}: {}'.format('w_quant' + dev_name + quant_type, - np.sum(w_quant))) + print('{}: {}'.format('w_freeze' + dev_name + activation_quant_type + + '_' + weight_quant_type, np.sum(w_freeze))) + print('{}: {}'.format('w_quant' + dev_name + activation_quant_type + + '_' + weight_quant_type, np.sum(w_quant))) # Convert parameter to 8-bit. convert_int8_pass = ConvertToInt8Pass(scope=scope, place=place) @@ -343,26 +356,28 @@ class TestQuantizationFreezePass(unittest.TestCase): for op in test_graph.all_op_nodes(): if op.name().find('quantize') > -1: marked_nodes.add(op) - test_graph.draw('.', 'test_int8' + dev_name + quant_type, - marked_nodes) + test_graph.draw('.', 'test_int8' + dev_name + activation_quant_type + + '_' + weight_quant_type, marked_nodes) server_program_int8 = test_graph.to_program() # Save the 8-bit parameter and model file. with fluid.scope_guard(scope): - fluid.io.save_inference_model('server_int8' + dev_name + quant_type, - ['image', 'label'], [loss], exe, - server_program_int8) + fluid.io.save_inference_model( + 'server_int8' + dev_name + activation_quant_type + '_' + + weight_quant_type, ['image', 'label'], [loss], exe, + server_program_int8) # Test whether the 8-bit parameter and model file can be loaded successfully. [infer, feed, fetch] = fluid.io.load_inference_model( - 'server_int8' + dev_name + quant_type, exe) + 'server_int8' + dev_name + activation_quant_type + '_' + + weight_quant_type, exe) # Check the loaded 8-bit weight. w_8bit = np.array(scope.find_var('conv2d_1.w_0.int8').get_tensor()) self.assertEqual(w_8bit.dtype, np.int8) self.assertEqual(np.sum(w_8bit), np.sum(w_freeze)) if not for_ci: - print('{}: {}'.format('w_8bit' + dev_name + quant_type, - np.sum(w_8bit))) - print('{}: {}'.format('w_freeze' + dev_name + quant_type, - np.sum(w_freeze))) + print('{}: {}'.format('w_8bit' + dev_name + activation_quant_type + + '_' + weight_quant_type, np.sum(w_8bit))) + print('{}: {}'.format('w_freeze' + dev_name + activation_quant_type + + '_' + weight_quant_type, np.sum(w_freeze))) mobile_pass = TransformForMobilePass() mobile_pass.apply(test_graph) @@ -371,45 +386,103 @@ class TestQuantizationFreezePass(unittest.TestCase): for op in test_graph.all_op_nodes(): if op.name().find('quantize') > -1: marked_nodes.add(op) - test_graph.draw('.', 'test_mobile' + dev_name + quant_type, + test_graph.draw('.', 'test_mobile' + dev_name + + activation_quant_type + '_' + weight_quant_type, marked_nodes) mobile_program = test_graph.to_program() with fluid.scope_guard(scope): - fluid.io.save_inference_model('mobile_int8' + dev_name + quant_type, - ['image', 'label'], [loss], exe, - mobile_program) + fluid.io.save_inference_model( + 'mobile_int8' + dev_name + activation_quant_type + '_' + + weight_quant_type, ['image', 'label'], [loss], exe, + mobile_program) def test_freeze_graph_cuda_dynamic(self): if fluid.core.is_compiled_with_cuda(): with fluid.unique_name.guard(): self.freeze_graph( - True, seed=1, quant_type='abs_max', for_ci=False) + True, + seed=1, + activation_quant_type='abs_max', + weight_quant_type='abs_max', + for_ci=True) + with fluid.unique_name.guard(): + self.freeze_graph( + True, + seed=1, + activation_quant_type='abs_max', + weight_quant_type='channel_wise_abs_max', + for_ci=True) def test_freeze_graph_cpu_dynamic(self): with fluid.unique_name.guard(): - self.freeze_graph(False, seed=2, quant_type='abs_max', for_ci=False) + self.freeze_graph( + False, + seed=2, + activation_quant_type='abs_max', + weight_quant_type='abs_max', + for_ci=True) + self.freeze_graph( + False, + seed=2, + activation_quant_type='abs_max', + weight_quant_type='channel_wise_abs_max', + for_ci=True) def test_freeze_graph_cuda_static(self): if fluid.core.is_compiled_with_cuda(): with fluid.unique_name.guard(): self.freeze_graph( - True, seed=1, quant_type='range_abs_max', for_ci=False) + True, + seed=1, + activation_quant_type='range_abs_max', + weight_quant_type='abs_max', + for_ci=True) + self.freeze_graph( + True, + seed=1, + activation_quant_type='moving_average_abs_max', + weight_quant_type='abs_max', + for_ci=True) self.freeze_graph( True, seed=1, - quant_type='moving_average_abs_max', - for_ci=False) + activation_quant_type='range_abs_max', + weight_quant_type='channel_wise_abs_max', + for_ci=True) + self.freeze_graph( + True, + seed=1, + activation_quant_type='moving_average_abs_max', + weight_quant_type='channel_wise_abs_max', + for_ci=True) def test_freeze_graph_cpu_static(self): with fluid.unique_name.guard(): self.freeze_graph( - False, seed=2, quant_type='range_abs_max', for_ci=False) + False, + seed=2, + activation_quant_type='range_abs_max', + weight_quant_type='abs_max', + for_ci=True) + self.freeze_graph( + False, + seed=2, + activation_quant_type='moving_average_abs_max', + weight_quant_type='abs_max', + for_ci=True) + self.freeze_graph( + False, + seed=2, + activation_quant_type='range_abs_max', + weight_quant_type='channel_wise_abs_max', + for_ci=True) self.freeze_graph( False, seed=2, - quant_type='moving_average_abs_max', - for_ci=False) + activation_quant_type='moving_average_abs_max', + weight_quant_type='channel_wise_abs_max', + for_ci=True) if __name__ == '__main__': From 56c2d384c7dbdd1d87b73cc34ae79e10d7d146d3 Mon Sep 17 00:00:00 2001 From: phlrain Date: Thu, 21 Mar 2019 07:15:44 +0000 Subject: [PATCH 19/44] add elementwise floordiv, mod; test=develop --- .../elementwise/elementwise_floordiv_op.cc | 38 ++++++++++ .../elementwise/elementwise_floordiv_op.cu | 23 +++++++ .../elementwise/elementwise_floordiv_op.h | 55 +++++++++++++++ .../elementwise/elementwise_mod_op.cc | 36 ++++++++++ .../elementwise/elementwise_mod_op.cu | 22 ++++++ .../elementwise/elementwise_mod_op.h | 55 +++++++++++++++ python/paddle/fluid/layers/math_op_patch.py | 2 + python/paddle/fluid/layers/nn.py | 19 ++++- .../unittests/test_elementwise_floordiv_op.py | 69 +++++++++++++++++++ .../unittests/test_elementwise_mod_op.py | 69 +++++++++++++++++++ 10 files changed, 386 insertions(+), 2 deletions(-) create mode 100644 paddle/fluid/operators/elementwise/elementwise_floordiv_op.cc create mode 100644 paddle/fluid/operators/elementwise/elementwise_floordiv_op.cu create mode 100644 paddle/fluid/operators/elementwise/elementwise_floordiv_op.h create mode 100644 paddle/fluid/operators/elementwise/elementwise_mod_op.cc create mode 100644 paddle/fluid/operators/elementwise/elementwise_mod_op.cu create mode 100644 paddle/fluid/operators/elementwise/elementwise_mod_op.h create mode 100644 python/paddle/fluid/tests/unittests/test_elementwise_floordiv_op.py create mode 100644 python/paddle/fluid/tests/unittests/test_elementwise_mod_op.py diff --git a/paddle/fluid/operators/elementwise/elementwise_floordiv_op.cc b/paddle/fluid/operators/elementwise/elementwise_floordiv_op.cc new file mode 100644 index 0000000000..56db8a7c48 --- /dev/null +++ b/paddle/fluid/operators/elementwise/elementwise_floordiv_op.cc @@ -0,0 +1,38 @@ +/* Copyright (c) 2016 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/elementwise/elementwise_floordiv_op.h" +#include +#include "paddle/fluid/operators/elementwise/elementwise_op.h" + +namespace paddle { +namespace operators { +class ElementwiseFloorDivOpMaker : public ElementwiseOpMaker { + protected: + std::string GetName() const override { return "FloorDiv"; } + std::string GetEquation() const override { return "Out = X % Y"; } +}; +} // namespace operators +} // namespace paddle + +namespace ops = paddle::operators; + +REGISTER_OP_WITHOUT_GRADIENT(elementwise_floordiv, ops::ElementwiseOp, + ops::ElementwiseFloorDivOpMaker); + +REGISTER_OP_CPU_KERNEL( + elementwise_floordiv, + ops::ElementwiseFloorDivKernel, + ops::ElementwiseFloorDivKernel); diff --git a/paddle/fluid/operators/elementwise/elementwise_floordiv_op.cu b/paddle/fluid/operators/elementwise/elementwise_floordiv_op.cu new file mode 100644 index 0000000000..a22175044e --- /dev/null +++ b/paddle/fluid/operators/elementwise/elementwise_floordiv_op.cu @@ -0,0 +1,23 @@ +/* Copyright (c) 2016 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/elementwise/elementwise_floordiv_op.h" +#include "paddle/fluid/platform/float16.h" + +namespace ops = paddle::operators; +namespace plat = paddle::platform; + +REGISTER_OP_CUDA_KERNEL( + elementwise_floordiv, + ops::ElementwiseFloorDivKernel, + ops::ElementwiseFloorDivKernel); diff --git a/paddle/fluid/operators/elementwise/elementwise_floordiv_op.h b/paddle/fluid/operators/elementwise/elementwise_floordiv_op.h new file mode 100644 index 0000000000..4781f9fe16 --- /dev/null +++ b/paddle/fluid/operators/elementwise/elementwise_floordiv_op.h @@ -0,0 +1,55 @@ +/* Copyright (c) 2016 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 "paddle/fluid/framework/eigen.h" +#include "paddle/fluid/operators/elementwise/elementwise_op.h" +#include "paddle/fluid/operators/elementwise/elementwise_op_function.h" +#include "paddle/fluid/operators/math/blas.h" + +namespace paddle { +namespace operators { + +template +struct FloorDivFunctor { + inline HOSTDEVICE T operator()(T a, T b) const { return a / b; } +}; + +template +void elementwise_floor_div(const framework::ExecutionContext &ctx, + const framework::Tensor *x, + const framework::Tensor *y, framework::Tensor *z) { + int axis = ctx.Attr("axis"); + ElementwiseComputeEx, DeviceContext, T>( + ctx, x, y, axis, FloorDivFunctor(), z); +} + +template +class ElementwiseFloorDivKernel : public framework::OpKernel { + public: + void Compute(const framework::ExecutionContext &ctx) const override { + auto *x = ctx.Input("X"); + auto *y = ctx.Input("Y"); + auto *z = ctx.Output("Out"); + + z->mutable_data(ctx.GetPlace()); + + // dtype of x and y is int64 or int32 + elementwise_floor_div(ctx, x, y, z); + } +}; + +} // namespace operators +} // namespace paddle diff --git a/paddle/fluid/operators/elementwise/elementwise_mod_op.cc b/paddle/fluid/operators/elementwise/elementwise_mod_op.cc new file mode 100644 index 0000000000..426b14dcd9 --- /dev/null +++ b/paddle/fluid/operators/elementwise/elementwise_mod_op.cc @@ -0,0 +1,36 @@ +/* Copyright (c) 2016 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/elementwise/elementwise_mod_op.h" +#include +#include "paddle/fluid/operators/elementwise/elementwise_op.h" + +namespace paddle { +namespace operators { +class ElementwiseModOpMaker : public ElementwiseOpMaker { + protected: + std::string GetName() const override { return "Mod"; } + std::string GetEquation() const override { return "Out = X % Y"; } +}; +} // namespace operators +} // namespace paddle + +namespace ops = paddle::operators; +REGISTER_OP_WITHOUT_GRADIENT(elementwise_mod, ops::ElementwiseOp, + ops::ElementwiseModOpMaker); + +REGISTER_OP_CPU_KERNEL( + elementwise_mod, + ops::ElementwiseModKernel, + ops::ElementwiseModKernel); diff --git a/paddle/fluid/operators/elementwise/elementwise_mod_op.cu b/paddle/fluid/operators/elementwise/elementwise_mod_op.cu new file mode 100644 index 0000000000..3c8c62e564 --- /dev/null +++ b/paddle/fluid/operators/elementwise/elementwise_mod_op.cu @@ -0,0 +1,22 @@ +/* Copyright (c) 2016 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/elementwise/elementwise_mod_op.h" +#include "paddle/fluid/platform/float16.h" + +namespace ops = paddle::operators; +namespace plat = paddle::platform; + +REGISTER_OP_CUDA_KERNEL( + elementwise_mod, ops::ElementwiseModKernel, + ops::ElementwiseModKernel); diff --git a/paddle/fluid/operators/elementwise/elementwise_mod_op.h b/paddle/fluid/operators/elementwise/elementwise_mod_op.h new file mode 100644 index 0000000000..790807c79b --- /dev/null +++ b/paddle/fluid/operators/elementwise/elementwise_mod_op.h @@ -0,0 +1,55 @@ +/* Copyright (c) 2016 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 "paddle/fluid/framework/eigen.h" +#include "paddle/fluid/operators/elementwise/elementwise_op.h" +#include "paddle/fluid/operators/elementwise/elementwise_op_function.h" +#include "paddle/fluid/operators/math/blas.h" + +namespace paddle { +namespace operators { + +template +struct ModFunctor { + inline HOSTDEVICE T operator()(T a, T b) const { return a % b; } +}; + +template +void elementwise_mod(const framework::ExecutionContext &ctx, + const framework::Tensor *x, const framework::Tensor *y, + framework::Tensor *z) { + int axis = ctx.Attr("axis"); + ElementwiseComputeEx, DeviceContext, T>(ctx, x, y, axis, + ModFunctor(), z); +} + +template +class ElementwiseModKernel : public framework::OpKernel { + public: + void Compute(const framework::ExecutionContext &ctx) const override { + auto *x = ctx.Input("X"); + auto *y = ctx.Input("Y"); + auto *z = ctx.Output("Out"); + + z->mutable_data(ctx.GetPlace()); + + // dtype of x and y is int64 or int32 + elementwise_mod(ctx, x, y, z); + } +}; + +} // namespace operators +} // namespace paddle diff --git a/python/paddle/fluid/layers/math_op_patch.py b/python/paddle/fluid/layers/math_op_patch.py index a458cebfb1..734383655c 100644 --- a/python/paddle/fluid/layers/math_op_patch.py +++ b/python/paddle/fluid/layers/math_op_patch.py @@ -174,6 +174,8 @@ def monkey_patch_variable(): ("__rtruediv__", "elementwise_div", True), ("__pow__", "elementwise_pow", False), ("__rpow__", "elementwise_pow", True), + ("__floordiv__", "elementwise_floordiv", False), + ("__mod__", "elementwise_mod", False), # for logical compare ("__eq__", "equal", False), ("__ne__", "not_equal", False), diff --git a/python/paddle/fluid/layers/nn.py b/python/paddle/fluid/layers/nn.py index 28655314d3..b814062622 100644 --- a/python/paddle/fluid/layers/nn.py +++ b/python/paddle/fluid/layers/nn.py @@ -9228,9 +9228,24 @@ def elementwise_pow(x, y, axis=-1, act=None, name=None): return _elementwise_op(LayerHelper('elementwise_pow', **locals())) +def elementwise_mod(x, y, axis=-1, act=None, name=None): + return _elementwise_op(LayerHelper('elementwise_mod', **locals())) + + +def elementwise_floordiv(x, y, axis=-1, act=None, name=None): + return _elementwise_op(LayerHelper('elementwise_floordiv', **locals())) + + for func in [ - elementwise_add, elementwise_div, elementwise_sub, elementwise_mul, - elementwise_max, elementwise_min, elementwise_pow + elementwise_add, + elementwise_div, + elementwise_sub, + elementwise_mul, + elementwise_max, + elementwise_min, + elementwise_pow, + elementwise_mod, + elementwise_floordiv, ]: op_proto = OpProtoHolder.instance().get_op_proto(func.__name__) func.__doc__ = _generate_doc_string_( diff --git a/python/paddle/fluid/tests/unittests/test_elementwise_floordiv_op.py b/python/paddle/fluid/tests/unittests/test_elementwise_floordiv_op.py new file mode 100644 index 0000000000..49b5e19592 --- /dev/null +++ b/python/paddle/fluid/tests/unittests/test_elementwise_floordiv_op.py @@ -0,0 +1,69 @@ +# 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. + +from __future__ import print_function +import unittest +import numpy as np +import paddle.fluid.core as core +from op_test import OpTest + +import random + + +class TestElementwiseModOp(OpTest): + def init_kernel_type(self): + self.use_mkldnn = False + + def setUp(self): + self.op_type = "elementwise_floordiv" + self.dtype = np.int32 + self.axis = -1 + self.init_dtype() + self.init_input_output() + self.init_kernel_type() + self.init_axis() + + self.inputs = { + 'X': OpTest.np_dtype_to_fluid_dtype(self.x), + 'Y': OpTest.np_dtype_to_fluid_dtype(self.y) + } + self.attrs = {'axis': self.axis, 'use_mkldnn': self.use_mkldnn} + self.outputs = {'Out': self.out} + + def test_check_output(self): + self.check_output() + + def init_input_output(self): + self.x = np.random.uniform(0, 10000, [10, 10]).astype(self.dtype) + self.y = np.random.uniform(0, 1000, [10, 10]).astype(self.dtype) + self.out = np.floor_divide(self.x, self.y) + + def init_dtype(self): + pass + + def init_axis(self): + pass + + +class TestElementwiseModOp_scalar(TestElementwiseModOp): + def init_input_output(self): + scale_x = random.randint(0, 100000000) + scale_y = random.randint(1, 100000000) + self.x = (np.random.rand(2, 3, 4) * scale_x).astype(self.dtype) + self.y = (np.random.rand(1) * scale_y + 1).astype(self.dtype) + self.out = np.floor_divide(self.x, self.y) + + +if __name__ == '__main__': + unittest.main() diff --git a/python/paddle/fluid/tests/unittests/test_elementwise_mod_op.py b/python/paddle/fluid/tests/unittests/test_elementwise_mod_op.py new file mode 100644 index 0000000000..98496252b2 --- /dev/null +++ b/python/paddle/fluid/tests/unittests/test_elementwise_mod_op.py @@ -0,0 +1,69 @@ +# 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. + +from __future__ import print_function +import unittest +import numpy as np +import paddle.fluid.core as core +from op_test import OpTest + +import random + + +class TestElementwiseModOp(OpTest): + def init_kernel_type(self): + self.use_mkldnn = False + + def setUp(self): + self.op_type = "elementwise_mod" + self.dtype = np.int32 + self.axis = -1 + self.init_dtype() + self.init_input_output() + self.init_kernel_type() + self.init_axis() + + self.inputs = { + 'X': OpTest.np_dtype_to_fluid_dtype(self.x), + 'Y': OpTest.np_dtype_to_fluid_dtype(self.y) + } + self.attrs = {'axis': self.axis, 'use_mkldnn': self.use_mkldnn} + self.outputs = {'Out': self.out} + + def test_check_output(self): + self.check_output() + + def init_input_output(self): + self.x = np.random.uniform(0, 10000, [10, 10]).astype(self.dtype) + self.y = np.random.uniform(0, 1000, [10, 10]).astype(self.dtype) + self.out = np.mod(self.x, self.y) + + def init_dtype(self): + pass + + def init_axis(self): + pass + + +class TestElementwiseModOp_scalar(TestElementwiseModOp): + def init_input_output(self): + scale_x = random.randint(0, 100000000) + scale_y = random.randint(1, 100000000) + self.x = (np.random.rand(2, 3, 4) * scale_x).astype(self.dtype) + self.y = (np.random.rand(1) * scale_y + 1).astype(self.dtype) + self.out = np.mod(self.x, self.y) + + +if __name__ == '__main__': + unittest.main() From 249546bf45d9e71651efb4ab3ab452adc2b4c05c Mon Sep 17 00:00:00 2001 From: phlrain Date: Thu, 21 Mar 2019 09:36:05 +0000 Subject: [PATCH 20/44] fix concat shape check; test=develop --- paddle/fluid/operators/concat_op.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/paddle/fluid/operators/concat_op.cc b/paddle/fluid/operators/concat_op.cc index 5d5ad9e711..71985ea9d4 100644 --- a/paddle/fluid/operators/concat_op.cc +++ b/paddle/fluid/operators/concat_op.cc @@ -57,7 +57,7 @@ class ConcatOp : public framework::OperatorWithKernel { "elements except the specify axis."); } else { // not check -1 with other in compile time - if (out_dims[j] != -1 && ins[i][j] != -1) { + if (out_dims[j] != > 0 && ins[i][j] > 0) { PADDLE_ENFORCE_EQ(out_dims[j], ins[i][j], "Input tensors should have the same " "elements except the specify axis."); From ac32bf6f77d45275418e45b0d6579f96877dfaf1 Mon Sep 17 00:00:00 2001 From: lujun Date: Thu, 21 Mar 2019 19:32:48 +0800 Subject: [PATCH 21/44] update input params type, test=develop --- python/paddle/fluid/imperative/checkpoint.py | 29 ++++++++------------ 1 file changed, 11 insertions(+), 18 deletions(-) diff --git a/python/paddle/fluid/imperative/checkpoint.py b/python/paddle/fluid/imperative/checkpoint.py index 97bad77179..37c43f29d2 100644 --- a/python/paddle/fluid/imperative/checkpoint.py +++ b/python/paddle/fluid/imperative/checkpoint.py @@ -17,13 +17,12 @@ from __future__ import print_function import os import collections from .. import core -from ..framework import Variable, Parameter, default_main_program -from .layers import Layer +from ..framework import Variable, default_main_program __all__ = ['save_persistables', 'load_persistables'] -def save_persistables(obj, dirname, filename=None): +def save_persistables(vardict, dirname, filename=None): """ This function filters out all variables in layer.parameters from the give `layer` and then trys to load these variables from the folder @@ -35,7 +34,7 @@ def save_persistables(obj, dirname, filename=None): the file name. Args: - var_list(dict of Parameters|Layer): The parameters will + vardict(dict of Parameters): The parameters will be saved. If it is None, nothing will be deal. dirname(str): The directory path. @@ -69,17 +68,14 @@ def save_persistables(obj, dirname, filename=None): dy_loss, last_hidden, last_cell = ptb_model(x, y, init_hidden, init_cell) param_path = "./my_paddle_model" - fluid.imperative.checkpoint.save_persistables(ptb_model.parameters(), dirname=param_path, + fluid.imperative.checkpoint.save_persistables(ptb_model.state_dict(), dirname=param_path, layer=ptb_model) """ - if isinstance(obj, collections.OrderedDict): - _save_var_to_file(obj, dirname, filename) - elif isinstance(obj, Layer): - _save_var_to_file( - obj.state_dict(include_sublayers=True), dirname, filename) + if isinstance(vardict, collections.OrderedDict): + _save_var_to_file(vardict, dirname, filename) -def load_persistables(obj, dirname, filename=None): +def load_persistables(vardict, dirname, filename=None): """ This function trys to load persistable variables from the folder `dirname` or the file `filename`. @@ -90,7 +86,7 @@ def load_persistables(obj, dirname, filename=None): the file name. Args: - obj(dict of Parameters|Layer): The parameters will be loaded. + vardict(dict of Parameters): The parameters will be loaded. dirname(str): The directory path. filename(str|None): The file which saved all variables, this file path should be end with '.npz'. If variables were saved in differnet files, set it to None. @@ -111,16 +107,13 @@ def load_persistables(obj, dirname, filename=None): my_layer = layer(fluid.imperative.Layer) param_path = "./my_paddle_model" filename = "model.file" - param_dict = fluid.imperative.checkpoint.load_persistables(my_layer, var_list, param_path, + param_dict = fluid.imperative.checkpoint.load_persistables(my_layer.state_dict(), param_path, filename=filename) param_1 = param_dict['PtbModel_0.w_1'] """ - if isinstance(obj, collections.OrderedDict): - return _load_var_from_file(obj, dirname, filename) - elif isinstance(obj, Layer): - return _load_var_from_file( - obj.state_dict(include_sublayers=True), dirname, filename) + if isinstance(vardict, collections.OrderedDict): + return _load_var_from_file(vardict, dirname, filename) return {} From 8274d9d733d697c8ba647f9d85edb81dbfdcb13d Mon Sep 17 00:00:00 2001 From: phlrain Date: Thu, 21 Mar 2019 12:21:07 +0000 Subject: [PATCH 22/44] fix concat shape check; test=develop --- paddle/fluid/operators/concat_op.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/paddle/fluid/operators/concat_op.cc b/paddle/fluid/operators/concat_op.cc index 71985ea9d4..6e3c9f2864 100644 --- a/paddle/fluid/operators/concat_op.cc +++ b/paddle/fluid/operators/concat_op.cc @@ -57,7 +57,7 @@ class ConcatOp : public framework::OperatorWithKernel { "elements except the specify axis."); } else { // not check -1 with other in compile time - if (out_dims[j] != > 0 && ins[i][j] > 0) { + if (out_dims[j] > 0 && ins[i][j] > 0) { PADDLE_ENFORCE_EQ(out_dims[j], ins[i][j], "Input tensors should have the same " "elements except the specify axis."); From 18d107c27aa4da0524ec974246af05fadbccde4e Mon Sep 17 00:00:00 2001 From: phlrain Date: Thu, 21 Mar 2019 12:25:06 +0000 Subject: [PATCH 23/44] add floordiv and mod op; test=develop --- paddle/fluid/operators/elementwise/elementwise_floordiv_op.cc | 4 ++-- paddle/fluid/operators/elementwise/elementwise_mod_op.cc | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/paddle/fluid/operators/elementwise/elementwise_floordiv_op.cc b/paddle/fluid/operators/elementwise/elementwise_floordiv_op.cc index 56db8a7c48..66c56da417 100644 --- a/paddle/fluid/operators/elementwise/elementwise_floordiv_op.cc +++ b/paddle/fluid/operators/elementwise/elementwise_floordiv_op.cc @@ -1,4 +1,4 @@ -/* Copyright (c) 2016 PaddlePaddle Authors. All Rights Reserved. +/* 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. @@ -21,7 +21,7 @@ namespace operators { class ElementwiseFloorDivOpMaker : public ElementwiseOpMaker { protected: std::string GetName() const override { return "FloorDiv"; } - std::string GetEquation() const override { return "Out = X % Y"; } + std::string GetEquation() const override { return "Out = X // Y"; } }; } // namespace operators } // namespace paddle diff --git a/paddle/fluid/operators/elementwise/elementwise_mod_op.cc b/paddle/fluid/operators/elementwise/elementwise_mod_op.cc index 426b14dcd9..d63a7df03d 100644 --- a/paddle/fluid/operators/elementwise/elementwise_mod_op.cc +++ b/paddle/fluid/operators/elementwise/elementwise_mod_op.cc @@ -1,4 +1,4 @@ -/* Copyright (c) 2016 PaddlePaddle Authors. All Rights Reserved. +/* 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. From 5dc9b5199478a2068952d7e7b086ff2fe1267cf7 Mon Sep 17 00:00:00 2001 From: phlrain Date: Thu, 21 Mar 2019 12:50:47 +0000 Subject: [PATCH 24/44] fix time; test=develop --- paddle/fluid/operators/elementwise/elementwise_floordiv_op.cu | 2 +- paddle/fluid/operators/elementwise/elementwise_floordiv_op.h | 2 +- paddle/fluid/operators/elementwise/elementwise_mod_op.cu | 2 +- paddle/fluid/operators/elementwise/elementwise_mod_op.h | 2 +- .../fluid/tests/unittests/test_elementwise_floordiv_op.py | 2 +- python/paddle/fluid/tests/unittests/test_elementwise_mod_op.py | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/paddle/fluid/operators/elementwise/elementwise_floordiv_op.cu b/paddle/fluid/operators/elementwise/elementwise_floordiv_op.cu index a22175044e..60846d1e8f 100644 --- a/paddle/fluid/operators/elementwise/elementwise_floordiv_op.cu +++ b/paddle/fluid/operators/elementwise/elementwise_floordiv_op.cu @@ -1,4 +1,4 @@ -/* Copyright (c) 2016 PaddlePaddle Authors. All Rights Reserved. +/* 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. diff --git a/paddle/fluid/operators/elementwise/elementwise_floordiv_op.h b/paddle/fluid/operators/elementwise/elementwise_floordiv_op.h index 4781f9fe16..2d24e394d5 100644 --- a/paddle/fluid/operators/elementwise/elementwise_floordiv_op.h +++ b/paddle/fluid/operators/elementwise/elementwise_floordiv_op.h @@ -1,4 +1,4 @@ -/* Copyright (c) 2016 PaddlePaddle Authors. All Rights Reserved. +/* 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. diff --git a/paddle/fluid/operators/elementwise/elementwise_mod_op.cu b/paddle/fluid/operators/elementwise/elementwise_mod_op.cu index 3c8c62e564..da3304a839 100644 --- a/paddle/fluid/operators/elementwise/elementwise_mod_op.cu +++ b/paddle/fluid/operators/elementwise/elementwise_mod_op.cu @@ -1,4 +1,4 @@ -/* Copyright (c) 2016 PaddlePaddle Authors. All Rights Reserved. +/* 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. diff --git a/paddle/fluid/operators/elementwise/elementwise_mod_op.h b/paddle/fluid/operators/elementwise/elementwise_mod_op.h index 790807c79b..5b139fd4b3 100644 --- a/paddle/fluid/operators/elementwise/elementwise_mod_op.h +++ b/paddle/fluid/operators/elementwise/elementwise_mod_op.h @@ -1,4 +1,4 @@ -/* Copyright (c) 2016 PaddlePaddle Authors. All Rights Reserved. +/* 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. diff --git a/python/paddle/fluid/tests/unittests/test_elementwise_floordiv_op.py b/python/paddle/fluid/tests/unittests/test_elementwise_floordiv_op.py index 49b5e19592..104e896b6e 100644 --- a/python/paddle/fluid/tests/unittests/test_elementwise_floordiv_op.py +++ b/python/paddle/fluid/tests/unittests/test_elementwise_floordiv_op.py @@ -1,4 +1,4 @@ -# Copyright (c) 2018 PaddlePaddle Authors. All Rights Reserved. +# 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. diff --git a/python/paddle/fluid/tests/unittests/test_elementwise_mod_op.py b/python/paddle/fluid/tests/unittests/test_elementwise_mod_op.py index 98496252b2..a354ba0177 100644 --- a/python/paddle/fluid/tests/unittests/test_elementwise_mod_op.py +++ b/python/paddle/fluid/tests/unittests/test_elementwise_mod_op.py @@ -1,4 +1,4 @@ -# Copyright (c) 2018 PaddlePaddle Authors. All Rights Reserved. +# 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. From 090d25f724f34e92e74c95a6a0c95d17759b9eeb Mon Sep 17 00:00:00 2001 From: tianshuo78520a <707759223@qq.com> Date: Fri, 22 Mar 2019 12:22:29 +0800 Subject: [PATCH 25/44] test=develop;fix docker build failed --- tools/manylinux1/Dockerfile.x64 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/manylinux1/Dockerfile.x64 b/tools/manylinux1/Dockerfile.x64 index c2fd743f62..c37a9a92e6 100644 --- a/tools/manylinux1/Dockerfile.x64 +++ b/tools/manylinux1/Dockerfile.x64 @@ -52,7 +52,7 @@ RUN LD_LIBRARY_PATH=/opt/_internal/cpython-2.7.11-ucs4/lib:${LD_LIBRARY_PATH} /o LD_LIBRARY_PATH=/opt/_internal/cpython-3.6.0/lib/:${LD_LIBRARY_PATH} /opt/_internal/cpython-3.6.0/bin/pip3 install pre-commit 'ipython==5.3.0' opencv-python && \ LD_LIBRARY_PATH=/opt/_internal/cpython-3.7.0/lib/:${LD_LIBRARY_PATH} /opt/_internal/cpython-3.7.0/bin/pip3 install pre-commit 'ipython==5.3.0' opencv-python -RUN wget -O /opt/swig-2.0.12.tar.gz https://cytranet.dl.sourceforge.net/project/swig/swig/swig-2.0.12/swig-2.0.12.tar.gz && \ +RUN wget -O /opt/swig-2.0.12.tar.gz https://sourceforge.net/projects/swig/files/swig/swig-2.0.12/swig-2.0.12.tar.gz/download && \ cd /opt && tar xzf swig-2.0.12.tar.gz && cd /opt/swig-2.0.12 && ./configure && make && make install && cd /opt && rm swig-2.0.12.tar.gz CMD ["bash", "/paddle/paddle/scripts/docker/build.sh"] From 77a08750e954cd7e995cd76e59b3a8ea8efbd20e Mon Sep 17 00:00:00 2001 From: phlrain Date: Fri, 22 Mar 2019 05:08:59 +0000 Subject: [PATCH 26/44] add var name in optimizer; test=develop --- paddle/fluid/API.spec | 11 +++++++++++ python/paddle/fluid/optimizer.py | 11 ++++++++++- 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/paddle/fluid/API.spec b/paddle/fluid/API.spec index 9b111e09e0..598604ca7a 100644 --- a/paddle/fluid/API.spec +++ b/paddle/fluid/API.spec @@ -432,48 +432,59 @@ paddle.fluid.nets.img_conv_group (ArgSpec(args=['input', 'conv_num_filter', 'poo paddle.fluid.optimizer.SGDOptimizer.__init__ (ArgSpec(args=['self', 'learning_rate', 'regularization', 'name'], varargs=None, keywords=None, defaults=(None, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.optimizer.SGDOptimizer.apply_gradients (ArgSpec(args=['self', 'params_grads'], varargs=None, keywords=None, defaults=None), ('document', 'bfe7305918552aaecfdaa22411dbe871')) paddle.fluid.optimizer.SGDOptimizer.backward (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'ba3a113d0229ff7bc9d39bda0a6d947f')) +paddle.fluid.optimizer.SGDOptimizer.get_opti_var_name_list (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.optimizer.SGDOptimizer.minimize (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '35fd5d3330c97903528c7e0dacc7f6ea')) paddle.fluid.optimizer.MomentumOptimizer.__init__ (ArgSpec(args=['self', 'learning_rate', 'momentum', 'use_nesterov', 'regularization', 'name'], varargs=None, keywords=None, defaults=(False, None, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.optimizer.MomentumOptimizer.apply_gradients (ArgSpec(args=['self', 'params_grads'], varargs=None, keywords=None, defaults=None), ('document', 'bfe7305918552aaecfdaa22411dbe871')) paddle.fluid.optimizer.MomentumOptimizer.backward (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'ba3a113d0229ff7bc9d39bda0a6d947f')) +paddle.fluid.optimizer.MomentumOptimizer.get_opti_var_name_list (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.optimizer.MomentumOptimizer.minimize (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '35fd5d3330c97903528c7e0dacc7f6ea')) paddle.fluid.optimizer.AdagradOptimizer.__init__ (ArgSpec(args=['self', 'learning_rate', 'epsilon', 'regularization', 'name', 'initial_accumulator_value'], varargs=None, keywords=None, defaults=(1e-06, None, None, 0.0)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.optimizer.AdagradOptimizer.apply_gradients (ArgSpec(args=['self', 'params_grads'], varargs=None, keywords=None, defaults=None), ('document', 'bfe7305918552aaecfdaa22411dbe871')) paddle.fluid.optimizer.AdagradOptimizer.backward (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'ba3a113d0229ff7bc9d39bda0a6d947f')) +paddle.fluid.optimizer.AdagradOptimizer.get_opti_var_name_list (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.optimizer.AdagradOptimizer.minimize (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '35fd5d3330c97903528c7e0dacc7f6ea')) paddle.fluid.optimizer.AdamOptimizer.__init__ (ArgSpec(args=['self', 'learning_rate', 'beta1', 'beta2', 'epsilon', 'regularization', 'name', 'lazy_mode'], varargs=None, keywords=None, defaults=(0.001, 0.9, 0.999, 1e-08, None, None, False)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.optimizer.AdamOptimizer.apply_gradients (ArgSpec(args=['self', 'params_grads'], varargs=None, keywords=None, defaults=None), ('document', 'bfe7305918552aaecfdaa22411dbe871')) paddle.fluid.optimizer.AdamOptimizer.backward (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'ba3a113d0229ff7bc9d39bda0a6d947f')) +paddle.fluid.optimizer.AdamOptimizer.get_opti_var_name_list (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.optimizer.AdamOptimizer.minimize (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '35fd5d3330c97903528c7e0dacc7f6ea')) paddle.fluid.optimizer.AdamaxOptimizer.__init__ (ArgSpec(args=['self', 'learning_rate', 'beta1', 'beta2', 'epsilon', 'regularization', 'name'], varargs=None, keywords=None, defaults=(0.001, 0.9, 0.999, 1e-08, None, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.optimizer.AdamaxOptimizer.apply_gradients (ArgSpec(args=['self', 'params_grads'], varargs=None, keywords=None, defaults=None), ('document', 'bfe7305918552aaecfdaa22411dbe871')) paddle.fluid.optimizer.AdamaxOptimizer.backward (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'ba3a113d0229ff7bc9d39bda0a6d947f')) +paddle.fluid.optimizer.AdamaxOptimizer.get_opti_var_name_list (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.optimizer.AdamaxOptimizer.minimize (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '35fd5d3330c97903528c7e0dacc7f6ea')) paddle.fluid.optimizer.DecayedAdagradOptimizer.__init__ (ArgSpec(args=['self', 'learning_rate', 'decay', 'epsilon', 'regularization', 'name'], varargs=None, keywords=None, defaults=(0.95, 1e-06, None, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.optimizer.DecayedAdagradOptimizer.apply_gradients (ArgSpec(args=['self', 'params_grads'], varargs=None, keywords=None, defaults=None), ('document', 'bfe7305918552aaecfdaa22411dbe871')) paddle.fluid.optimizer.DecayedAdagradOptimizer.backward (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'ba3a113d0229ff7bc9d39bda0a6d947f')) +paddle.fluid.optimizer.DecayedAdagradOptimizer.get_opti_var_name_list (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.optimizer.DecayedAdagradOptimizer.minimize (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '35fd5d3330c97903528c7e0dacc7f6ea')) paddle.fluid.optimizer.FtrlOptimizer.__init__ (ArgSpec(args=['self', 'learning_rate', 'l1', 'l2', 'lr_power', 'regularization', 'name'], varargs=None, keywords=None, defaults=(0.0, 0.0, -0.5, None, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.optimizer.FtrlOptimizer.apply_gradients (ArgSpec(args=['self', 'params_grads'], varargs=None, keywords=None, defaults=None), ('document', 'bfe7305918552aaecfdaa22411dbe871')) paddle.fluid.optimizer.FtrlOptimizer.backward (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'ba3a113d0229ff7bc9d39bda0a6d947f')) +paddle.fluid.optimizer.FtrlOptimizer.get_opti_var_name_list (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.optimizer.FtrlOptimizer.minimize (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '35fd5d3330c97903528c7e0dacc7f6ea')) paddle.fluid.optimizer.RMSPropOptimizer.__init__ (ArgSpec(args=['self', 'learning_rate', 'rho', 'epsilon', 'momentum', 'centered', 'regularization', 'name'], varargs=None, keywords=None, defaults=(0.95, 1e-06, 0.0, False, None, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.optimizer.RMSPropOptimizer.apply_gradients (ArgSpec(args=['self', 'params_grads'], varargs=None, keywords=None, defaults=None), ('document', 'bfe7305918552aaecfdaa22411dbe871')) paddle.fluid.optimizer.RMSPropOptimizer.backward (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'ba3a113d0229ff7bc9d39bda0a6d947f')) +paddle.fluid.optimizer.RMSPropOptimizer.get_opti_var_name_list (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.optimizer.RMSPropOptimizer.minimize (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '35fd5d3330c97903528c7e0dacc7f6ea')) paddle.fluid.optimizer.AdadeltaOptimizer.__init__ (ArgSpec(args=['self', 'learning_rate', 'epsilon', 'rho', 'regularization', 'name'], varargs=None, keywords=None, defaults=(1e-06, 0.95, None, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.optimizer.AdadeltaOptimizer.apply_gradients (ArgSpec(args=['self', 'params_grads'], varargs=None, keywords=None, defaults=None), ('document', 'bfe7305918552aaecfdaa22411dbe871')) paddle.fluid.optimizer.AdadeltaOptimizer.backward (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'ba3a113d0229ff7bc9d39bda0a6d947f')) +paddle.fluid.optimizer.AdadeltaOptimizer.get_opti_var_name_list (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.optimizer.AdadeltaOptimizer.minimize (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '35fd5d3330c97903528c7e0dacc7f6ea')) paddle.fluid.optimizer.ModelAverage.__init__ (ArgSpec(args=['self', 'average_window_rate', 'min_average_window', 'max_average_window', 'regularization', 'name'], varargs=None, keywords=None, defaults=(10000, 10000, None, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.optimizer.ModelAverage.apply (ArgSpec(args=['self', 'executor', 'need_restore'], varargs=None, keywords=None, defaults=(True,)), ('document', '46234a5470590feb336346f70a3db715')) paddle.fluid.optimizer.ModelAverage.apply_gradients (ArgSpec(args=['self', 'params_grads'], varargs=None, keywords=None, defaults=None), ('document', 'bfe7305918552aaecfdaa22411dbe871')) paddle.fluid.optimizer.ModelAverage.backward (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'ba3a113d0229ff7bc9d39bda0a6d947f')) +paddle.fluid.optimizer.ModelAverage.get_opti_var_name_list (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.optimizer.ModelAverage.minimize (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '35fd5d3330c97903528c7e0dacc7f6ea')) paddle.fluid.optimizer.ModelAverage.restore (ArgSpec(args=['self', 'executor'], varargs=None, keywords=None, defaults=None), ('document', '18db9c70be9c4dd466f9844457b21bfe')) paddle.fluid.optimizer.LarsMomentumOptimizer.__init__ (ArgSpec(args=['self', 'learning_rate', 'momentum', 'lars_coeff', 'lars_weight_decay', 'regularization', 'name'], varargs=None, keywords=None, defaults=(0.001, 0.0005, None, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.optimizer.LarsMomentumOptimizer.apply_gradients (ArgSpec(args=['self', 'params_grads'], varargs=None, keywords=None, defaults=None), ('document', 'bfe7305918552aaecfdaa22411dbe871')) paddle.fluid.optimizer.LarsMomentumOptimizer.backward (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None, None)), ('document', 'ba3a113d0229ff7bc9d39bda0a6d947f')) +paddle.fluid.optimizer.LarsMomentumOptimizer.get_opti_var_name_list (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.optimizer.LarsMomentumOptimizer.minimize (ArgSpec(args=['self', 'loss', 'startup_program', 'parameter_list', 'no_grad_set'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '35fd5d3330c97903528c7e0dacc7f6ea')) paddle.fluid.backward.append_backward (ArgSpec(args=['loss', 'parameter_list', 'no_grad_set', 'callbacks'], varargs=None, keywords=None, defaults=(None, None, None)), ('document', '1a79bd7d10ae54ca763ec81bca36ba24')) paddle.fluid.regularizer.L1DecayRegularizer.__init__ (ArgSpec(args=['self', 'regularization_coeff'], varargs=None, keywords=None, defaults=(0.0,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) diff --git a/python/paddle/fluid/optimizer.py b/python/paddle/fluid/optimizer.py index d501d02bd4..8918886a80 100644 --- a/python/paddle/fluid/optimizer.py +++ b/python/paddle/fluid/optimizer.py @@ -70,6 +70,10 @@ class Optimizer(object): # {accum_name : { paramter_name : accumulator_for_parameter, ...}, ...} self._accumulators = defaultdict(lambda: dict()) self.helper = None + self._opti_name_list = [] + + def get_opti_var_name_list(self): + return self._opti_name_list def _create_global_learning_rate(self): lr = self._global_learning_rate() @@ -166,8 +170,13 @@ class Optimizer(object): if shape == None: shape = param.shape assert isinstance(self.helper, LayerHelper) + + var_name = param.name + "_" + name + var_name = unique_name.generate(var_name) + self._opti_name_list.append(var_name) + var = self.helper.create_global_variable( - name=unique_name.generate(name), + name=var_name, persistable=True, dtype=dtype or param.dtype, type=param.type, From a6a3b2fbbc50fa0cf7f3a5bb0b136a491675afc2 Mon Sep 17 00:00:00 2001 From: chengduo Date: Fri, 22 Mar 2019 02:46:45 -0500 Subject: [PATCH 27/44] [Speed]Refine ParallelExecutor (#16190) * refine parallelExecutor test=develop * Polish op_handle test=develop * Remove unnecessary op_handle test=develop * Fix Travis CI test=develop * Fix fetch bug test=develop * Remove WaitInputVarGenerated * Fix OpHandleBase::Run test=develop * debug test=develop * use origin fetch_op_handle test=develop * Revert op_handle_base.cc test=develop * Polish code test=develop * Fix OpHandleBase::Run test=develop * code refine * test CI and CE test=develop * fix OpHandle::Run test=develop * refine AllReduceOpHandle test=develop * Polish code test=develop --- paddle/fluid/framework/details/CMakeLists.txt | 4 +- .../framework/details/all_reduce_op_handle.cc | 4 +- .../framework/details/broadcast_op_handle.h | 2 +- .../fluid/framework/details/build_strategy.cc | 4 + .../details/data_balance_op_handle.cc | 154 ------------------ .../details/data_balance_op_handle.h | 59 ------- .../framework/details/fetch_op_handle.cc | 2 + .../fluid/framework/details/fetch_op_handle.h | 2 + .../framework/details/fuse_vars_op_handle.cc | 51 ------ .../framework/details/fuse_vars_op_handle.h | 65 -------- .../details/multi_devices_graph_pass.cc | 4 +- .../fluid/framework/details/op_handle_base.cc | 71 +++++++- .../details/threaded_ssa_graph_executor.cc | 152 ++++++++++------- .../details/threaded_ssa_graph_executor.h | 24 ++- paddle/fluid/framework/details/var_handle.h | 27 ++- 15 files changed, 221 insertions(+), 404 deletions(-) delete mode 100644 paddle/fluid/framework/details/data_balance_op_handle.cc delete mode 100644 paddle/fluid/framework/details/data_balance_op_handle.h delete mode 100644 paddle/fluid/framework/details/fuse_vars_op_handle.cc delete mode 100644 paddle/fluid/framework/details/fuse_vars_op_handle.h diff --git a/paddle/fluid/framework/details/CMakeLists.txt b/paddle/fluid/framework/details/CMakeLists.txt index 5f29ebc70f..7a371af510 100644 --- a/paddle/fluid/framework/details/CMakeLists.txt +++ b/paddle/fluid/framework/details/CMakeLists.txt @@ -51,9 +51,7 @@ else() cc_library(fused_broadcast_op_handle SRCS fused_broadcast_op_handle.cc DEPS broadcast_op_handle) endif() -cc_library(data_balance_op_handle SRCS data_balance_op_handle.cc DEPS op_handle_base scope lod_tensor) cc_library(gather_op_handle SRCS gather_op_handle.cc DEPS op_handle_base scope ddim memory variable_visitor) -cc_library(fuse_vars_op_handle SRCS fuse_vars_op_handle.cc DEPS op_handle_base scope) if(WITH_GPU) cc_library(memory_optimize_helper SRCS memory_optimize_helper.cc DEPS graph graph_helper gpu_info) @@ -74,7 +72,7 @@ cc_library(sequential_execution_pass SRCS sequential_execution_pass.cc DEPS grap cc_library(all_reduce_deps_pass SRCS all_reduce_deps_pass.cc DEPS graph graph_helper pass) cc_library(multi_devices_graph_pass SRCS multi_devices_graph_pass.cc DEPS multi_devices_helper computation_op_handle - scale_loss_grad_op_handle rpc_op_handle all_reduce_op_handle reduce_op_handle broadcast_op_handle data_balance_op_handle fused_broadcast_op_handle) + scale_loss_grad_op_handle rpc_op_handle all_reduce_op_handle reduce_op_handle broadcast_op_handle fused_broadcast_op_handle) cc_library(fuse_all_reduce_op_pass SRCS fuse_all_reduce_op_pass.cc DEPS graph graph_helper fused_all_reduce_op_handle) diff --git a/paddle/fluid/framework/details/all_reduce_op_handle.cc b/paddle/fluid/framework/details/all_reduce_op_handle.cc index c1f9c2b60c..fdaff08e53 100644 --- a/paddle/fluid/framework/details/all_reduce_op_handle.cc +++ b/paddle/fluid/framework/details/all_reduce_op_handle.cc @@ -11,9 +11,8 @@ // 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 - #include "paddle/fluid/framework/details/all_reduce_op_handle.h" +#include #include "paddle/fluid/framework/details/container_cast.h" #include "paddle/fluid/framework/details/reduce_and_gather.h" #include "paddle/fluid/framework/details/variable_visitor.h" @@ -56,6 +55,7 @@ void AllReduceOpHandle::RunImpl() { platform::RecordEvent record_event(Name()); WaitInputVarGenerated(); + auto in_var_handles = DynamicCast(this->Inputs()); auto out_var_handles = DynamicCast(this->Outputs()); PADDLE_ENFORCE_EQ( diff --git a/paddle/fluid/framework/details/broadcast_op_handle.h b/paddle/fluid/framework/details/broadcast_op_handle.h index 0c75e05f86..0b4d335135 100644 --- a/paddle/fluid/framework/details/broadcast_op_handle.h +++ b/paddle/fluid/framework/details/broadcast_op_handle.h @@ -57,7 +57,7 @@ struct BroadcastOpHandle : public OpHandleBase { std::string Name() const override; - bool IsMultiDeviceTransfer() override { return false; }; + bool IsMultiDeviceTransfer() override { return true; }; protected: void RunImpl() override; diff --git a/paddle/fluid/framework/details/build_strategy.cc b/paddle/fluid/framework/details/build_strategy.cc index 4184353bcb..5d9db23753 100644 --- a/paddle/fluid/framework/details/build_strategy.cc +++ b/paddle/fluid/framework/details/build_strategy.cc @@ -147,6 +147,10 @@ class ParallelExecutorPassBuilder : public ir::PassBuilder { // Verify that the graph is correct for multi-device executor. AppendPass("multi_devices_check_pass"); + if (VLOG_IS_ON(2)) { + AppendPass("all_reduce_deps_pass"); + } + if (SeqOnlyAllReduceOps(strategy)) { VLOG(10) << "Add all_reduce_deps_pass"; AppendPass("all_reduce_deps_pass"); diff --git a/paddle/fluid/framework/details/data_balance_op_handle.cc b/paddle/fluid/framework/details/data_balance_op_handle.cc deleted file mode 100644 index c9b52b6820..0000000000 --- a/paddle/fluid/framework/details/data_balance_op_handle.cc +++ /dev/null @@ -1,154 +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. - -#include "paddle/fluid/framework/details/data_balance_op_handle.h" -#include -#include "paddle/fluid/framework/details/container_cast.h" - -namespace paddle { -namespace framework { -namespace details { - -#if defined(PADDLE_WITH_CUDA) && !defined(_WIN32) -DataBalanceOpHandle::DataBalanceOpHandle( - ir::Node *node, const std::vector &local_scopes, - const std::vector &places, - const platform::NCCLContextMap *ctxs) - : OpHandleBase(node), local_scopes_(local_scopes), places_(places) { - if (ctxs) { - for (auto &p : places_) { - this->SetDeviceContext(p, ctxs->DevCtx(p)); - } - } -} -#else -DataBalanceOpHandle::DataBalanceOpHandle( - ir::Node *node, const std::vector &local_scopes, - const std::vector &places) - : OpHandleBase(node), local_scopes_(local_scopes), places_(places) {} -#endif - -std::string DataBalanceOpHandle::Name() const { return "data balance"; } - -std::vector> DataBalanceOpHandle::GetBalancePlan( - const std::vector &device_sizes) { - int device_num = device_sizes.size(); - int total_size = 0; - int empty_num = 0; - std::vector> size_device_vec; - size_device_vec.reserve(device_num); - for (int i = 0; i < device_num; ++i) { - if (device_sizes[i] == 0) { - ++empty_num; - } - total_size += device_sizes[i]; - size_device_vec.push_back({{device_sizes[i], i}}); - } - std::vector> res; - if (empty_num == 0) { - // No need to do data balance. - return res; - } - if (total_size < device_num) { - // No enough data. - PADDLE_THROW_EOF(); - } - std::sort(size_device_vec.begin(), size_device_vec.end(), - [](const std::array &a, const std::array &b) { - return a[0] > b[0]; - }); - int expected_device_size = total_size / device_num; - int src_idx = 0; - for (int dst_idx = device_num - empty_num; dst_idx < device_num; ++dst_idx) { - if (size_device_vec[src_idx][0] <= expected_device_size) { - ++src_idx; - PADDLE_ENFORCE_LT( - src_idx, device_num - empty_num, - "In current srategy an empty tensor should not be copy source."); - } - size_device_vec[src_idx][0] -= expected_device_size; - size_device_vec[dst_idx][0] += expected_device_size; - res.push_back({{size_device_vec[src_idx][1], size_device_vec[dst_idx][1], - expected_device_size}}); - } - return res; -} - -void DataBalanceOpHandle::RunImpl() { - PADDLE_ENFORCE_GT(places_.size(), 1UL, - "Data balance can only be enabled when the number of " - "places to run larger than 1."); - auto in_var_handles = DynamicCast(this->Inputs()); - auto out_var_handles = DynamicCast(this->Outputs()); - PADDLE_ENFORCE(in_var_handles.size() % places_.size() == 0); - PADDLE_ENFORCE_EQ( - in_var_handles.size(), out_var_handles.size(), - "The NoDummyInputSize and NoDummyOutputSize should be equal."); - int data_num = in_var_handles.size() / places_.size(); - WaitInputVarGenerated(); - std::vector> lod_tensors(data_num); - std::vector device_sizes; - for (int i = 0; i < static_cast(in_var_handles.size()); ++i) { - PADDLE_ENFORCE_EQ(in_var_handles[i]->name(), out_var_handles[i]->name(), - "The name of input and output should be equal."); - int place_idx = i / data_num; - int data_idx = i % data_num; - auto *local_scope = - local_scopes_[place_idx]->FindVar(kLocalExecScopeName)->Get(); - auto *tensor_var = local_scope->FindVar(in_var_handles[i]->name()); - PADDLE_ENFORCE(tensor_var->IsType()); - auto *tensor = tensor_var->GetMutable(); - lod_tensors[data_idx].push_back(tensor); - int ins_size = - tensor->lod().empty() ? tensor->dims()[0] : tensor->NumElements(); - if (data_idx == 0) { - device_sizes.emplace_back(ins_size); - } else { - PADDLE_ENFORCE_EQ( - ins_size, device_sizes.at(place_idx), - "All data on the same device shall have the same batch size."); - } - } - const auto &balance_plan = GetBalancePlan(device_sizes); - - for (const auto &trans : balance_plan) { - for (int data_idx = 0; data_idx < data_num; ++data_idx) { - LoDTensor *src_tensor = lod_tensors[data_idx][trans[0]]; - LoDTensor *dst_tensor = lod_tensors[data_idx][trans[1]]; - int trans_ins_size = trans[2]; - LoD src_lod = src_tensor->lod(); - int src_ins_size = - src_lod.empty() ? src_tensor->dims()[0] : src_tensor->NumElements(); - int cut_point = src_ins_size - trans_ins_size; - if (!src_lod.empty()) { - for (auto &level : src_lod) { - cut_point = level[cut_point]; - } - } - TensorCopySync(src_tensor->Slice(cut_point, src_tensor->dims()[0]), - dst_tensor->place(), dst_tensor); - src_tensor->ShareDataWith(src_tensor->Slice(0, cut_point)); - if (!src_lod.empty()) { - dst_tensor->set_lod(SliceInLevel( - src_lod, 0, src_ins_size - trans_ins_size, src_ins_size)); - src_tensor->set_lod( - SliceInLevel(src_lod, 0, 0, src_ins_size - trans_ins_size)); - } - } - } -} - -} // namespace details -} // namespace framework -} // namespace paddle diff --git a/paddle/fluid/framework/details/data_balance_op_handle.h b/paddle/fluid/framework/details/data_balance_op_handle.h deleted file mode 100644 index 2db18a1a72..0000000000 --- a/paddle/fluid/framework/details/data_balance_op_handle.h +++ /dev/null @@ -1,59 +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/op_handle_base.h" -#include "paddle/fluid/framework/lod_tensor.h" -#include "paddle/fluid/framework/scope.h" -#if defined(PADDLE_WITH_CUDA) && !defined(_WIN32) -#include "paddle/fluid/platform/nccl_helper.h" -#endif - -namespace paddle { -namespace framework { -namespace details { - -struct DataBalanceOpHandle : public OpHandleBase { - public: -#if defined(PADDLE_WITH_CUDA) && !defined(_WIN32) - DataBalanceOpHandle(ir::Node *node, const std::vector &local_scopes, - const std::vector &places, - const platform::NCCLContextMap *ctxs); -#else - DataBalanceOpHandle(ir::Node *node, const std::vector &local_scopes, - const std::vector &places); -#endif - - std::string Name() const override; - - bool IsMultiDeviceTransfer() override { return false; }; - - protected: - void RunImpl() override; - - private: - // std::vector<(src_dev_id, dst_dev_id, trans_size)> - std::vector> GetBalancePlan( - const std::vector &batch_size_per_device); - - const std::vector local_scopes_; - const std::vector places_; -}; - -} // namespace details -} // namespace framework -} // namespace paddle diff --git a/paddle/fluid/framework/details/fetch_op_handle.cc b/paddle/fluid/framework/details/fetch_op_handle.cc index bbf81e1b8e..232d82a5da 100644 --- a/paddle/fluid/framework/details/fetch_op_handle.cc +++ b/paddle/fluid/framework/details/fetch_op_handle.cc @@ -82,6 +82,8 @@ void FetchOpHandle::WaitInputVarGenerated(const platform::Place &place) { } } +bool FetchOpHandle::IsMultiDeviceTransfer() { return true; } + std::string FetchOpHandle::Name() const { return "Fetch"; } } // namespace details diff --git a/paddle/fluid/framework/details/fetch_op_handle.h b/paddle/fluid/framework/details/fetch_op_handle.h index 6ce42f92d7..dbb7f4f658 100644 --- a/paddle/fluid/framework/details/fetch_op_handle.h +++ b/paddle/fluid/framework/details/fetch_op_handle.h @@ -39,6 +39,8 @@ struct FetchOpHandle : public OpHandleBase { std::string Name() const override; + bool IsMultiDeviceTransfer() override; + protected: void RunImpl() override; diff --git a/paddle/fluid/framework/details/fuse_vars_op_handle.cc b/paddle/fluid/framework/details/fuse_vars_op_handle.cc deleted file mode 100644 index 14292c0a5d..0000000000 --- a/paddle/fluid/framework/details/fuse_vars_op_handle.cc +++ /dev/null @@ -1,51 +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. - -#include "paddle/fluid/framework/details/fuse_vars_op_handle.h" - -namespace paddle { -namespace framework { -namespace details { - -void FuseVarsOpHandle::RunImpl() { - WaitInputVarGenerated(place_); - - auto in_var_handles = DynamicCast(this->Inputs()); - auto out_var_handles = DynamicCast(this->Outputs()); - PADDLE_ENFORCE_EQ(in_var_handles.size(), 0UL); - PADDLE_ENFORCE_EQ(out_var_handles.size() - 1, inputs_numel_.size(), ""); - - auto scope = local_scope_->FindVar(kLocalExecScopeName)->Get(); - - auto out_var_handle = out_var_handles[0]; - auto out_var = scope->Var(out_var_handle->name()); - - auto out_tensor = out_var->GetMutable(); - out_tensor->Resize({total_numel_}).mutable_data(this->place_, type_); - - int64_t s = 0; - for (size_t i = 1; i < out_var_handles.size(); ++i) { - auto out_name = out_var_handles[i]->name(); - auto out_t = scope->Var(out_name)->GetMutable(); - auto numel = this->inputs_numel_.at(out_name); - out_t->ShareDataWith(out_tensor->Slice(s, s + numel)); - s += numel; - } - this->RunAndRecordEvent([] {}); -} - -std::string FuseVarsOpHandle::Name() const { return "fuse vars"; } -} // namespace details -} // namespace framework -} // namespace paddle diff --git a/paddle/fluid/framework/details/fuse_vars_op_handle.h b/paddle/fluid/framework/details/fuse_vars_op_handle.h deleted file mode 100644 index b40b01df36..0000000000 --- a/paddle/fluid/framework/details/fuse_vars_op_handle.h +++ /dev/null @@ -1,65 +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 - -#include "paddle/fluid/framework/details/container_cast.h" -#include "paddle/fluid/framework/details/op_handle_base.h" -#include "paddle/fluid/framework/lod_tensor.h" -#include "paddle/fluid/framework/scope.h" -#include "paddle/fluid/platform/device_context.h" - -namespace paddle { -namespace framework { -namespace details { - -struct FuseVarsOpHandle : public OpHandleBase { - public: - FuseVarsOpHandle(ir::Node *node, Scope *local_scope, - const platform::Place &place, - const std::unordered_map &inputs_numel, - const proto::VarType::Type var_type) - : OpHandleBase(node), - local_scope_(local_scope), - place_(place), - inputs_numel_(inputs_numel), - type_(var_type) { - total_numel_ = 0; - for (auto in_numel : inputs_numel) { - PADDLE_ENFORCE_GT(in_numel.second, 0); - total_numel_ += in_numel.second; - } - } - - std::string Name() const override; - - bool IsMultiDeviceTransfer() override { return false; }; - - protected: - void RunImpl() override; - - private: - Scope *local_scope_; - const platform::Place place_; - const std::unordered_map inputs_numel_; - const proto::VarType::Type type_; - int64_t total_numel_; -}; -} // namespace details -} // namespace framework -} // namespace paddle diff --git a/paddle/fluid/framework/details/multi_devices_graph_pass.cc b/paddle/fluid/framework/details/multi_devices_graph_pass.cc index e3cd2340c9..125dbf746c 100644 --- a/paddle/fluid/framework/details/multi_devices_graph_pass.cc +++ b/paddle/fluid/framework/details/multi_devices_graph_pass.cc @@ -14,13 +14,15 @@ #include "paddle/fluid/framework/details/multi_devices_graph_pass.h" #include #include +#include #include +#include +#include #include #include #include "paddle/fluid/framework/details/all_reduce_op_handle.h" #include "paddle/fluid/framework/details/broadcast_op_handle.h" #include "paddle/fluid/framework/details/computation_op_handle.h" -#include "paddle/fluid/framework/details/data_balance_op_handle.h" #include "paddle/fluid/framework/details/fused_broadcast_op_handle.h" #include "paddle/fluid/framework/details/reduce_op_handle.h" #include "paddle/fluid/framework/details/rpc_op_handle.h" diff --git a/paddle/fluid/framework/details/op_handle_base.cc b/paddle/fluid/framework/details/op_handle_base.cc index 4822627ac3..158da6f606 100644 --- a/paddle/fluid/framework/details/op_handle_base.cc +++ b/paddle/fluid/framework/details/op_handle_base.cc @@ -13,6 +13,7 @@ // limitations under the License. #include "paddle/fluid/framework/details/op_handle_base.h" #include +#include namespace paddle { namespace framework { @@ -41,15 +42,42 @@ OpHandleBase::~OpHandleBase() { void OpHandleBase::Run(bool use_cuda) { #ifdef PADDLE_WITH_CUDA - if (events_.empty() && use_cuda) { + if (events_.empty() && use_cuda && dev_ctxes_.size() > 0) { for (auto &p : dev_ctxes_) { int dev_id = boost::get(p.first).device; PADDLE_ENFORCE(cudaSetDevice(dev_id)); PADDLE_ENFORCE( cudaEventCreateWithFlags(&events_[dev_id], cudaEventDisableTiming)); } + if (IsMultiDeviceTransfer() && dev_ctxes_.size() > 0) { + for (auto &out_var : outputs_) { + auto *out_var_handle = dynamic_cast(out_var); + if (out_var_handle) { + int dev_id = + boost::get(out_var_handle->place()).device; + out_var_handle->SetGenerateEvent(events_[dev_id]); + } + } + } else { + PADDLE_ENFORCE_EQ(dev_ctxes_.size(), 1UL, + "%s should have only one dev_ctx.", Name()); + auto &place = dev_ctxes_.begin()->first; + int dev_id = boost::get(place).device; + for (auto &out_var : outputs_) { + auto *out_var_handle = dynamic_cast(out_var); + if (out_var_handle) { + PADDLE_ENFORCE( + platform::is_same_place(place, out_var_handle->place()), + "The place of input(%s) is not consistent with the " + "place of current op(%s).", + out_var_handle->Name(), Name()); + out_var_handle->SetGenerateEvent(events_[dev_id]); + } + } + } } #else + PADDLE_ENFORCE(!use_cuda); #endif @@ -93,17 +121,48 @@ void OpHandleBase::AddOutput(VarHandleBase *out) { void OpHandleBase::WaitInputVarGenerated() { for (auto in_var : inputs_) { if (NeedWait(in_var)) { - for (auto &pair : dev_ctxes_) { - in_var->GeneratedOp()->RecordWaitEventOnCtx(pair.second); + // Dummy Variable is used to represent dependencies between operators, so + // there doesn't add event for it. + auto *in_var_handle = dynamic_cast(in_var); + if (in_var_handle) { + auto &place = in_var_handle->place(); + if (platform::is_gpu_place(place)) { +#ifdef PADDLE_WITH_CUDA + auto stream = + static_cast(dev_ctxes_.at(place)) + ->stream(); + PADDLE_ENFORCE( + cudaStreamWaitEvent(stream, in_var_handle->GetEvent(), 0)); +#else + PADDLE_THROW("Doesn't compile the GPU."); +#endif + } + // There are nothing to do when the place is CPUPlace. } } } } void OpHandleBase::WaitInputVarGenerated(const platform::Place &place) { - for (auto *in : inputs_) { - if (NeedWait(in)) { - in->GeneratedOp()->RecordWaitEventOnCtx(dev_ctxes_.at(place)); + for (auto in_var : inputs_) { + if (NeedWait(in_var)) { + // Dummy Variable is used to represent dependencies between operators, so + // there doesn't add event for it. + auto *in_var_handle = dynamic_cast(in_var); + if (in_var_handle) { + if (platform::is_gpu_place(in_var_handle->place())) { +#ifdef PADDLE_WITH_CUDA + auto stream = static_cast( + dev_ctxes_.at(in_var_handle->place())) + ->stream(); + PADDLE_ENFORCE( + cudaStreamWaitEvent(stream, in_var_handle->GetEvent(), 0)); +#else + PADDLE_THROW("Doesn't compile the GPU."); +#endif + } + // There are nothing to do when the place is CPUPlace. + } } } } diff --git a/paddle/fluid/framework/details/threaded_ssa_graph_executor.cc b/paddle/fluid/framework/details/threaded_ssa_graph_executor.cc index 9ba295a2b0..c4254bbadf 100644 --- a/paddle/fluid/framework/details/threaded_ssa_graph_executor.cc +++ b/paddle/fluid/framework/details/threaded_ssa_graph_executor.cc @@ -14,7 +14,6 @@ #include "paddle/fluid/framework/details/threaded_ssa_graph_executor.h" -#include "paddle/fluid/framework/details/multi_devices_helper.h" #include "paddle/fluid/framework/ir/graph_helper.h" #include "paddle/fluid/platform/profiler.h" @@ -27,62 +26,49 @@ ThreadedSSAGraphExecutor::ThreadedSSAGraphExecutor( : graph_(graph), pool_(strategy.num_threads_ >= 2 ? new ::ThreadPool(strategy.num_threads_) : nullptr), + prepare_pool_(1), local_scopes_(local_scopes), places_(places), fetch_ctxs_(places), - running_ops_(0), - strategy_(strategy) {} + strategy_(strategy) { + PrepareOpDeps(); + CopyOpDeps(); +} FeedFetchList ThreadedSSAGraphExecutor::Run( const std::vector &fetch_tensors) { std::unique_ptr event( new platform::RecordEvent("ThreadedSSAGraphExecutorPrepare")); - std::unordered_map pending_ops; - std::unordered_set pending_vars; - auto ready_vars = std::make_shared>(); - std::unordered_set ready_ops; + std::unique_ptr op_deps = op_deps_futures_.get(); + CopyOpDeps(); + VLOG(10) << "ThreadedSSAGraphExecutor::Run"; + std::shared_ptr> ready_vars( + new BlockingQueue); + auto &pending_ops = op_deps->pending_ops_; + auto &pending_vars = op_deps->pending_vars_; + auto &ready_ops = op_deps->ready_ops_; + // For ops (e.g. nccl_all_reduce) that need to coordinate multiple // streams from multiple GPUs, it's faster to buffer them and schedule // together since we currently cannot overlap computation and memcpy streams. // Should revisit it if overlapping is available. std::unordered_set delayed_ops; - // Transform SSAGraph to pending_ops & pending_vars - for (auto &var_map : graph_->Get(details::kGraphVars)) { - for (auto &name_pair : var_map) { - for (auto &version_pair : name_pair.second) { - InsertPendingVar(&pending_vars, ready_vars.get(), version_pair); - } - } - } - for (auto &var : graph_->Get(details::kGraphDepVars)) { - InsertPendingVar(&pending_vars, ready_vars.get(), var); - } - - for (auto &op : ir::FilterByNodeWrapper(*graph_)) { - if (op->Inputs().empty()) { // Special case, Op has no input. - ready_ops.insert(op); - } else { - InsertPendingOp(&pending_ops, op); - } - } - // Step 2. Insert FetchOps std::vector fetch_ops; std::unordered_set fetch_dependencies; FeedFetchList fetch_data(fetch_tensors.size()); - InsertFetchOps(fetch_tensors, &fetch_ops, &fetch_dependencies, &pending_ops, - &pending_vars, ready_vars.get(), &fetch_data); + InsertFetchOps(fetch_tensors, &fetch_ops, &fetch_dependencies, &ready_ops, + &pending_ops, &pending_vars, &fetch_data); auto run_all_ops = [&](std::unordered_set &set) { for (auto *op : set) { - running_ops_++; RunOp(ready_vars, op); } set.clear(); }; - + auto run_all_op = [&](OpHandleBase *op) { RunOp(ready_vars, op); }; // Clean run context run_op_futures_.clear(); exception_holder_.Clear(); @@ -91,19 +77,11 @@ FeedFetchList ThreadedSSAGraphExecutor::Run( while (!pending_vars.empty()) { // 1. Run All Ready ops // Keep loop until all vars are ready. - // - // NOTE: DelayedOps have a lower priority. It will be scheduled after all - // ready_ops have been performed. - if (ready_ops.empty() && strategy_.allow_op_delay_ && running_ops_ == 0) { - run_all_ops(delayed_ops); - } else { - run_all_ops(ready_ops); - } + run_all_ops(ready_ops); // 2. Find ready variable bool timeout; auto cur_ready_vars = ready_vars->PopAll(1, &timeout); - if (timeout) { if (exception_holder_.IsCaught()) { for (auto &run_op_future : run_op_futures_) { @@ -115,6 +93,7 @@ FeedFetchList ThreadedSSAGraphExecutor::Run( continue; } } + // 3. Remove the dependency of ready_var. // Find the ready_ops after the ready_var. for (auto ready_var : cur_ready_vars) { @@ -123,11 +102,7 @@ FeedFetchList ThreadedSSAGraphExecutor::Run( auto &deps = pending_ops[op]; --deps; if (deps == 0) { - if (op->IsMultiDeviceTransfer() && strategy_.allow_op_delay_) { - delayed_ops.insert(op); - } else { - ready_ops.insert(op); - } + run_all_op(op); } } } @@ -143,16 +118,17 @@ void ThreadedSSAGraphExecutor::InsertFetchOps( const std::vector &fetch_tensors, std::vector *fetch_ops, std::unordered_set *fetch_dependencies, + std::unordered_set *ready_ops, std::unordered_map *pending_ops, std::unordered_set *pending_vars, - BlockingQueue *ready_vars, FeedFetchList *fetch_data) { + FeedFetchList *fetch_data) { std::unordered_map> fetched_vars; - + std::unordered_set local_ready_vars; for (auto &fetch_var_name : fetch_tensors) { for (auto &var_map : graph_->Get(details::kGraphVars)) { auto it = var_map.find(fetch_var_name); if (it != var_map.end()) { - fetched_vars[fetch_var_name].push_back(*it->second.rbegin()); + fetched_vars[fetch_var_name].emplace_back(*it->second.rbegin()); } } } @@ -161,8 +137,9 @@ void ThreadedSSAGraphExecutor::InsertFetchOps( auto &var_name = fetch_tensors[i]; auto fetched_var_it = fetched_vars.find(var_name); PADDLE_ENFORCE(fetched_var_it != fetched_vars.end(), - "Cannot find fetched variable.(Perhaps the main_program " - "is not set to ParallelExecutor)"); + "Cannot find fetched variable(%s).(Perhaps the main_program " + "is not set to ParallelExecutor)", + var_name); auto &vars = fetched_var_it->second; @@ -184,9 +161,23 @@ void ThreadedSSAGraphExecutor::InsertFetchOps( auto *fetch_dummy = new DummyVarHandle(fetch_var); op->AddOutput(fetch_dummy); fetch_dependencies->emplace(fetch_dummy); - this->InsertPendingVar(pending_vars, ready_vars, fetch_dummy); - this->InsertPendingOp(pending_ops, op); + + this->InsertPendingVar(pending_vars, &local_ready_vars, fetch_dummy); + + size_t wait_input_num = 0; + std::unordered_set input_set(vars.begin(), vars.end()); + for (auto *var : input_set) { + if (pending_vars->count(var)) { + ++wait_input_num; + } + } + if (wait_input_num) { + pending_ops->insert({op, wait_input_num}); + } else { + ready_ops->insert(static_cast(op)); + } } + PADDLE_ENFORCE_EQ(local_ready_vars.size(), 0); } void ThreadedSSAGraphExecutor::InsertPendingOp( @@ -197,11 +188,63 @@ void ThreadedSSAGraphExecutor::InsertPendingOp( void ThreadedSSAGraphExecutor::InsertPendingVar( std::unordered_set *pending_vars, - BlockingQueue *ready_vars, VarHandleBase *var) const { + std::unordered_set *ready_vars, VarHandleBase *var) const { pending_vars->insert(var); if (var->GeneratedOp() == nullptr) { - ready_vars->Push(var); + ready_vars->insert(var); + } +} + +void ThreadedSSAGraphExecutor::PrepareOpDeps() { + op_deps_.reset(new OpDependentData()); + std::unordered_map &pending_ops = + op_deps_->pending_ops_; + std::unordered_set &pending_vars = op_deps_->pending_vars_; + std::unordered_set &ready_ops = op_deps_->ready_ops_; + std::unordered_set ready_vars; + + // Transform SSAGraph to pending_ops & pending_vars + for (auto &var_map : graph_->Get(details::kGraphVars)) { + for (auto &name_pair : var_map) { + for (auto &version_pair : name_pair.second) { + InsertPendingVar(&pending_vars, &ready_vars, version_pair); + } + } + } + for (auto &var : graph_->Get(details::kGraphDepVars)) { + InsertPendingVar(&pending_vars, &ready_vars, var); + } + + for (auto &op : ir::FilterByNodeWrapper(*graph_)) { + if (op->Inputs().empty()) { // Special case, Op has no input. + ready_ops.insert(op); + } else { + InsertPendingOp(&pending_ops, op); + } } + for (auto ready_var : ready_vars) { + pending_vars.erase(ready_var); + for (auto *op : ready_var->PendingOps()) { + auto &deps = pending_ops[op]; + --deps; + if (deps == 0) { + ready_ops.insert(op); + } + } + } +} + +void ThreadedSSAGraphExecutor::CopyOpDeps() { + op_deps_futures_ = prepare_pool_.enqueue([&] { + auto *op_deps = new OpDependentData(); + op_deps->pending_ops_.insert(op_deps_->pending_ops_.begin(), + op_deps_->pending_ops_.end()); + op_deps->pending_vars_.insert(op_deps_->pending_vars_.begin(), + op_deps_->pending_vars_.end()); + op_deps->ready_ops_.insert(op_deps_->ready_ops_.begin(), + op_deps_->ready_ops_.end()); + return std::unique_ptr(op_deps); + }); } void ThreadedSSAGraphExecutor::RunOp( @@ -216,7 +259,6 @@ void ThreadedSSAGraphExecutor::RunOp( op->Run(strategy_.use_cuda_); } VLOG(10) << op << " " << op->Name() << " Done "; - running_ops_--; ready_var_q->Extend(op->Outputs()); VLOG(10) << op << " " << op->Name() << " Signal posted"; } catch (...) { diff --git a/paddle/fluid/framework/details/threaded_ssa_graph_executor.h b/paddle/fluid/framework/details/threaded_ssa_graph_executor.h index 0867f62104..b9bccba8fa 100644 --- a/paddle/fluid/framework/details/threaded_ssa_graph_executor.h +++ b/paddle/fluid/framework/details/threaded_ssa_graph_executor.h @@ -15,18 +15,20 @@ #pragma once #include +#include #include +#include #include +#include #include #include #include - -#include #include "ThreadPool.h" // ThreadPool in thrird party #include "paddle/fluid/framework/blocking_queue.h" #include "paddle/fluid/framework/details/exception_holder.h" #include "paddle/fluid/framework/details/execution_strategy.h" #include "paddle/fluid/framework/details/fetch_op_handle.h" +#include "paddle/fluid/framework/details/multi_devices_helper.h" #include "paddle/fluid/framework/details/ssa_graph_executor.h" #include "paddle/fluid/framework/ir/graph.h" @@ -36,6 +38,12 @@ class Scope; namespace details { +struct OpDependentData { + std::unordered_map pending_ops_; + std::unordered_set pending_vars_; + std::unordered_set ready_ops_; +}; + class ThreadedSSAGraphExecutor : public SSAGraphExecutor { public: ThreadedSSAGraphExecutor(const ExecutionStrategy &strategy, @@ -57,29 +65,35 @@ class ThreadedSSAGraphExecutor : public SSAGraphExecutor { private: ir::Graph *graph_; std::unique_ptr<::ThreadPool> pool_; + ::ThreadPool prepare_pool_; std::vector local_scopes_; std::vector places_; platform::DeviceContextPool fetch_ctxs_; ExceptionHolder exception_holder_; - std::atomic running_ops_; void InsertPendingOp(std::unordered_map *pending_ops, OpHandleBase *op_instance) const; void InsertPendingVar(std::unordered_set *pending_vars, - BlockingQueue *ready_vars, + std::unordered_set *ready_vars, VarHandleBase *var) const; void InsertFetchOps(const std::vector &fetch_tensors, std::vector *fetch_ops, std::unordered_set *fetch_dependencies, + std::unordered_set *ready_ops, std::unordered_map *pending_ops, std::unordered_set *pending_vars, - BlockingQueue *ready_vars, FeedFetchList *fetch_data); + void PrepareOpDeps(); + void CopyOpDeps(); + private: + std::future> op_deps_futures_; + ExecutionStrategy strategy_; + std::unique_ptr op_deps_; // use std::list because clear(), push_back, and for_each are O(1) std::list> run_op_futures_; }; diff --git a/paddle/fluid/framework/details/var_handle.h b/paddle/fluid/framework/details/var_handle.h index 8321c32f8b..93060ef259 100644 --- a/paddle/fluid/framework/details/var_handle.h +++ b/paddle/fluid/framework/details/var_handle.h @@ -43,6 +43,7 @@ struct VarHandleBase { virtual ~VarHandleBase(); virtual std::string DebugString() const = 0; + virtual const std::string& Name() const = 0; void AddInput(OpHandleBase* in, ir::Node* node) { node_->inputs.clear(); @@ -95,8 +96,6 @@ struct VarHandleBase { // // NOTE: runtime variables have place. struct VarHandle : public VarHandleBase { - explicit VarHandle(ir::Node* node) : VarHandleBase(node) {} - virtual ~VarHandle(); std::string DebugString() const override; @@ -109,6 +108,20 @@ struct VarHandle : public VarHandleBase { name_(std::move(name)), place_(std::move(place)) {} +#ifdef PADDLE_WITH_CUDA + bool HasEvent() { return has_event_; } + + const cudaEvent_t& GetEvent() { + PADDLE_ENFORCE(HasEvent(), "The event is not set."); + return event_; + } + + void SetGenerateEvent(const cudaEvent_t& event) { + has_event_ = true; + event_ = event; + } +#endif + // version field currently is not used, however, just store the version to // debug easily. private: @@ -116,6 +129,11 @@ struct VarHandle : public VarHandleBase { size_t scope_idx_; std::string name_; platform::Place place_; +#ifdef PADDLE_WITH_CUDA + // Only when this event is triggered, var is generated. + cudaEvent_t event_; + bool has_event_{false}; +#endif public: bool IsTheSameVar(const VarHandle& o) const { @@ -125,6 +143,7 @@ struct VarHandle : public VarHandleBase { size_t version() const { return version_; } size_t scope_idx() const { return scope_idx_; } + const std::string& Name() const override { return name_; } const std::string& name() const { return name_; } const platform::Place& place() const { return place_; } }; @@ -136,6 +155,10 @@ struct DummyVarHandle : public VarHandleBase { virtual ~DummyVarHandle(); std::string DebugString() const override; + + public: + const std::string& Name() const override { return name_; } + std::string name_{"DummyVar"}; }; } // namespace details From e235882c18cf4975efd4a232d8da140e412df154 Mon Sep 17 00:00:00 2001 From: xiaolil1 <39753926+xiaolil1@users.noreply.github.com> Date: Fri, 22 Mar 2019 17:02:54 +0800 Subject: [PATCH 28/44] Enable MKL-DNN INT8 Concat Kernel. (#16156) * Enable INT8 Concat Kernel to improve the performance of MobileNet-SSD. test=develop * Optimize UT format. test=develop * Fix UT file address issue. test=develop * Refine the license year. test=develop * Optimize code for new API. test=develop * Restructure INT8 Concat kernel. test=develop --- .../operators/mkldnn/concat_mkldnn_op.cc | 120 ++++++++++++++--- .../mkldnn/test_concat_int8_mkldnn_op.py | 124 ++++++++++++++++++ 2 files changed, 223 insertions(+), 21 deletions(-) create mode 100644 python/paddle/fluid/tests/unittests/mkldnn/test_concat_int8_mkldnn_op.py diff --git a/paddle/fluid/operators/mkldnn/concat_mkldnn_op.cc b/paddle/fluid/operators/mkldnn/concat_mkldnn_op.cc index 54c6a71111..97387af92f 100644 --- a/paddle/fluid/operators/mkldnn/concat_mkldnn_op.cc +++ b/paddle/fluid/operators/mkldnn/concat_mkldnn_op.cc @@ -15,6 +15,7 @@ limitations under the License. */ #include #include "paddle/fluid/operators/concat_op.h" #include "paddle/fluid/platform/mkldnn_helper.h" +#include "paddle/fluid/platform/mkldnn_reuse.h" namespace paddle { namespace operators { @@ -38,15 +39,20 @@ static void EnforceLayouts(const std::vector inputs) { } static memory::primitive_desc CreateMemPrimDesc(const Tensor& input, - const mkldnn::engine& engine) { - constexpr auto data_type = mkldnn::memory::f32; + const mkldnn::engine& engine, + const memory::data_type& dt) { const auto dims = paddle::framework::vectorize2int(input.dims()); const auto format = input.format(); - auto description = memory::desc(dims, data_type, format); + auto description = memory::desc(dims, dt, format); auto mem_prim_desc = memory::primitive_desc(description, engine); return mem_prim_desc; } +static mkldnn::memory::format GetDstMemFormat( + const concat::primitive_desc& concat_pd) { + return (memory::format)concat_pd.dst_primitive_desc().desc().data.format; +} + static platform::CPUPlace GetCpuPlace( const paddle::framework::ExecutionContext& ctx) { auto place = ctx.GetPlace(); @@ -61,14 +67,30 @@ static const mkldnn::engine& GetMKLDNNEngine( return dev_ctx.GetEngine(); } +std::string CreateKey(const paddle::framework::ExecutionContext& ctx, + const std::vector multi_input, + const int64_t& concat_axis, const memory::data_type& dt) { + std::string key; + key.reserve(platform::MKLDNNHandler::MaxKeyLength); + for (size_t i = 0; i < multi_input.size(); i++) { + platform::MKLDNNHandler::AppendKeyDims( + &key, paddle::framework::vectorize2int(multi_input[i]->dims())); + } + platform::MKLDNNHandler::AppendKey(&key, std::to_string(concat_axis)); + platform::MKLDNNHandler::AppendKey(&key, ctx.op().Output("Out")); + platform::MKLDNNHandler::AppendKey(&key, std::to_string(dt)); + return key; +} + template class ConcatPrimitiveFactory { public: concat::primitive_desc CreateConcatPrimDescriptor( const std::vector multi_input, Tensor* output, - int concat_axis, const mkldnn::engine& mkldnn_engine) { - CreateSourcesDescriptors(multi_input, mkldnn_engine); - auto dst_desc = CreateDstMemDescriptor(output); + int concat_axis, const mkldnn::engine& mkldnn_engine, + const memory::data_type& dt = memory::data_type::f32) { + CreateSourcesDescriptors(multi_input, mkldnn_engine, dt); + auto dst_desc = CreateDstMemDescriptor(output, dt); return concat::primitive_desc(dst_desc, concat_axis, srcs_pd); } @@ -79,23 +101,39 @@ class ConcatPrimitiveFactory { return concat(concat_pd, inputs, dst_mem.get()); } + void SetSrcDataHandleByIndex(const std::vector& srcs, const size_t& i, + void* handler) { + srcs[i].set_data_handle(handler); + } + + void SetDstDataHandle(const memory& dst_mem, void* handler) { + dst_mem.set_data_handle(handler); + } + + std::vector GetSrcs() { return srcs; } + + memory GetDst() { return dst_mem.get(); } + private: - memory::desc CreateDstMemDescriptor(Tensor* output) { + memory::desc CreateDstMemDescriptor(Tensor* output, + const memory::data_type& dt) { auto dst_dims = paddle::framework::vectorize2int(output->dims()); - return memory::desc(dst_dims, platform::MKLDNNGetDataType(), - memory::format::any); + return memory::desc(dst_dims, dt, memory::format::any); } mkldnn::memory CreateDstMemory(const concat::primitive_desc& concat_pd, - Tensor* output, platform::CPUPlace place) { + Tensor* output, + const platform::CPUPlace& place) { return memory(concat_pd.dst_primitive_desc(), output->mutable_data(place)); } void CreateSourcesDescriptors(const std::vector multi_input, - const mkldnn::engine& mkldnn_engine) { + const mkldnn::engine& mkldnn_engine, + const memory::data_type& dt) { for (size_t i = 0; i < multi_input.size(); i++) { - auto mem_prim_desc = CreateMemPrimDesc(*multi_input[i], mkldnn_engine); + auto mem_prim_desc = + CreateMemPrimDesc(*multi_input[i], mkldnn_engine, dt); srcs_pd.push_back(mem_prim_desc); srcs.push_back( memory(mem_prim_desc, to_void_cast(multi_input[i]->data()))); @@ -120,21 +158,59 @@ template class ConcatMKLDNNOpKernel : public paddle::framework::OpKernel { public: void Compute(const paddle::framework::ExecutionContext& ctx) const override { - auto place = GetCpuPlace(ctx); - const auto& mkldnn_engine = GetMKLDNNEngine(ctx); - auto multi_input = ctx.MultiInput("X"); EnforceLayouts(multi_input); Tensor* output = ctx.Output("Out"); int64_t concat_axis = static_cast(ctx.Attr("axis")); + auto& dev_ctx = + ctx.template device_context(); + auto place = GetCpuPlace(ctx); + + memory::data_type dt = + paddle::framework::ToMKLDNNDataType(multi_input[0]->type()); ConcatPrimitiveFactory prim_creator; - auto concat_pd = prim_creator.CreateConcatPrimDescriptor( - multi_input, output, static_cast(concat_axis), mkldnn_engine); - auto concat = prim_creator.CreateConcatPrimitive(concat_pd, output, place); - stream(stream::kind::eager).submit({concat}).wait(); + std::string key = CreateKey(ctx, multi_input, concat_axis, dt); + const std::string key_prim = key + "@concat_p"; + const std::string key_concat_pd = key + "@concat_pd"; + const std::string key_srcs = key + "@concat_srcs"; + const std::string key_dst = key + "@concat_dst"; + + std::shared_ptr concat_pd; + std::shared_ptr> srcs; + std::shared_ptr dst_mem; + auto concat_p = std::static_pointer_cast(dev_ctx.GetBlob(key_prim)); + + if (concat_p == nullptr) { + const auto& mkldnn_engine = dev_ctx.GetEngine(); + concat_pd = std::make_shared( + prim_creator.CreateConcatPrimDescriptor(multi_input, output, + static_cast(concat_axis), + mkldnn_engine, dt)); + concat_p = std::make_shared( + prim_creator.CreateConcatPrimitive(*concat_pd, output, place)); + srcs = std::make_shared>(prim_creator.GetSrcs()); + dst_mem = std::make_shared(prim_creator.GetDst()); + dev_ctx.SetBlob(key_prim, concat_p); + dev_ctx.SetBlob(key_concat_pd, concat_pd); + dev_ctx.SetBlob(key_srcs, srcs); + dev_ctx.SetBlob(key_dst, dst_mem); + } else { + srcs = std::static_pointer_cast>( + dev_ctx.GetBlob(key_srcs)); + dst_mem = std::static_pointer_cast(dev_ctx.GetBlob(key_dst)); + concat_pd = std::static_pointer_cast( + dev_ctx.GetBlob(key_concat_pd)); + for (size_t i = 0; i < multi_input.size(); i++) { + prim_creator.SetSrcDataHandleByIndex( + *srcs, i, to_void_cast(multi_input[i]->data())); + } + prim_creator.SetDstDataHandle(*dst_mem, output->mutable_data(place)); + } + + stream(stream::kind::eager).submit({*concat_p}).wait(); - output->set_mkldnn_prim_desc(concat_pd.dst_primitive_desc()); + output->set_mkldnn_prim_desc(concat_pd->dst_primitive_desc()); } }; } // namespace operators @@ -143,4 +219,6 @@ class ConcatMKLDNNOpKernel : public paddle::framework::OpKernel { namespace ops = paddle::operators; REGISTER_OP_KERNEL(concat, MKLDNN, ::paddle::platform::CPUPlace, - ops::ConcatMKLDNNOpKernel) + ops::ConcatMKLDNNOpKernel, + ops::ConcatMKLDNNOpKernel, + ops::ConcatMKLDNNOpKernel); diff --git a/python/paddle/fluid/tests/unittests/mkldnn/test_concat_int8_mkldnn_op.py b/python/paddle/fluid/tests/unittests/mkldnn/test_concat_int8_mkldnn_op.py new file mode 100644 index 0000000000..0b6556746c --- /dev/null +++ b/python/paddle/fluid/tests/unittests/mkldnn/test_concat_int8_mkldnn_op.py @@ -0,0 +1,124 @@ +# 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. + +from __future__ import print_function + +import unittest +import numpy as np +from paddle.fluid.tests.unittests.op_test import OpTest + + +class TestConcatOp(OpTest): + def setUp(self): + self.op_type = "concat" + self.use_mkldnn = True + self._cpu_only = True + self.init_axis() + self.init_shape() + self.init_test_data() + self.inputs = {'X': [('x0', self.x0), ('x1', self.x1), ('x2', self.x2)]} + self.attrs = {'axis': self.axis, 'use_mkldnn': True} + + self.output = np.concatenate( + (self.x0, self.x1, self.x2), axis=self.axis).astype('int') + + self.outputs = {'Out': self.output} + + def test_check_output(self): + self.check_output() + +#--------------------test concat s8 in with axis 0-------------------- + + def init_test_data(self): + self.x0 = (np.random.randint(0, 100, self.x0_shape) - 50).astype('int8') + self.x1 = (np.random.randint(0, 80, self.x1_shape) - 30).astype('int8') + self.x2 = (np.random.randint(0, 110, self.x2_shape) - 80).astype('int8') + + def init_axis(self): + self.axis = 0 + + def init_shape(self): + self.x0_shape = [2, 2, 1, 2] + self.x1_shape = [1, 2, 1, 2] + self.x2_shape = [3, 2, 1, 2] + + +#--------------------test concat u8 in with axis 0-------------------- + + +class TestConcatOp2(TestConcatOp): + def init_test_data(self): + self.x0 = (np.random.randint(0, 100, self.x0_shape)).astype('uint8') + self.x1 = (np.random.randint(0, 50, self.x1_shape)).astype('uint8') + self.x2 = (np.random.randint(0, 80, self.x2_shape)).astype('uint8') + + def init_axis(self): + self.axis = 0 + + def init_shape(self): + self.x0_shape = [2, 1, 5, 5] + self.x1_shape = [1, 1, 5, 5] + self.x2_shape = [3, 1, 5, 5] + + +def create_test_int8_class(parent): + + #--------------------test concat s8/u8 in with axis 1-------------------- + + class TestAxis1Case(parent): + def init_axis(self): + self.axis = 1 + + def init_shape(self): + self.x0_shape = [1, 1, 5, 5] + self.x1_shape = [1, 2, 5, 5] + self.x2_shape = [1, 3, 5, 5] + +#--------------------test concat s8/u8 in with axis 2-------------------- + + class TestAxis2Case(parent): + def init_axis(self): + self.axis = 2 + + def init_shape(self): + self.x0_shape = [2, 3, 4, 5] + self.x1_shape = [2, 3, 5, 5] + self.x2_shape = [2, 3, 6, 5] + +#--------------------test concat s8/u8 in with axis 3-------------------- + + class TestAxis3Case(parent): + def init_axis(self): + self.axis = 3 + + def init_shape(self): + self.x0_shape = [2, 3, 5, 5] + self.x1_shape = [2, 3, 5, 6] + self.x2_shape = [2, 3, 5, 7] + + cls_name_1 = "{0}_axis_{1}".format(parent.__name__, "1") + cls_name_2 = "{0}_axis_{1}".format(parent.__name__, "2") + cls_name_3 = "{0}_axis_{1}".format(parent.__name__, "3") + TestAxis1Case.__name__ = cls_name_1 + TestAxis2Case.__name__ = cls_name_2 + TestAxis3Case.__name__ = cls_name_3 + globals()[cls_name_1] = TestAxis1Case + globals()[cls_name_2] = TestAxis2Case + globals()[cls_name_3] = TestAxis3Case + +create_test_int8_class(TestConcatOp) +create_test_int8_class(TestConcatOp2) + +if __name__ == '__main__': + unittest.main() From 55a7b98126f8ae08916d9b3740f5fb310b851094 Mon Sep 17 00:00:00 2001 From: Xin Pan Date: Fri, 22 Mar 2019 17:25:40 +0800 Subject: [PATCH 29/44] Add DeepCF model test=develop --- .../tests/unittests/test_imperative_deepcf.py | 196 ++++++++++++++++++ .../tests/unittests/test_imperative_gan.py | 2 +- 2 files changed, 197 insertions(+), 1 deletion(-) create mode 100644 python/paddle/fluid/tests/unittests/test_imperative_deepcf.py diff --git a/python/paddle/fluid/tests/unittests/test_imperative_deepcf.py b/python/paddle/fluid/tests/unittests/test_imperative_deepcf.py new file mode 100644 index 0000000000..af80ca6ce7 --- /dev/null +++ b/python/paddle/fluid/tests/unittests/test_imperative_deepcf.py @@ -0,0 +1,196 @@ +# 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 +import random +import sys + +import paddle +import paddle.fluid as fluid +import paddle.fluid.core as core +from test_imperative_base import new_program_scope +from paddle.fluid.imperative.base import to_variable + +NUM_USERS = 100 +NUM_ITEMS = 1000 + +BATCH_SIZE = 32 +NUM_BATCHES = 2 + + +class MLP(fluid.imperative.Layer): + def __init__(self, name_scope): + super(MLP, self).__init__(name_scope) + self._user_latent = fluid.imperative.FC(self.full_name(), 256) + self._item_latent = fluid.imperative.FC(self.full_name(), 256) + + self._user_layers = [] + self._item_layers = [] + self._hid_sizes = [128, 64] + for i in range(len(self._hid_sizes)): + self._user_layers.append( + self.add_sublayer( + 'user_layer_%d' % i, + fluid.imperative.FC( + self.full_name(), self._hid_sizes[i], act='relu'))) + self._item_layers.append( + self.add_sublayer( + 'item_layer_%d' % i, + fluid.imperative.FC( + self.full_name(), self._hid_sizes[i], act='relu'))) + + def forward(self, users, items): + users = self._user_latent(users) + items = self._item_latent(items) + + for ul, il in zip(self._user_layers, self._item_layers): + users = ul(users) + items = il(items) + return fluid.layers.elementwise_mul(users, items) + + +class DMF(fluid.imperative.Layer): + def __init__(self, name_scope): + super(DMF, self).__init__(name_scope) + self._user_latent = fluid.imperative.FC(self.full_name(), 256) + self._item_latent = fluid.imperative.FC(self.full_name(), 256) + self._match_layers = [] + self._hid_sizes = [128, 64] + for i in range(len(self._hid_sizes)): + self._match_layers.append( + self.add_sublayer( + 'match_layer_%d' % i, + fluid.imperative.FC( + self.full_name(), self._hid_sizes[i], act='relu'))) + self._mat + + def forward(self, users, items): + users = self._user_latent(users) + items = self._item_latent(items) + match_vec = fluid.layers.concat( + [users, items], axis=len(users.shape) - 1) + for l in self._match_layers: + match_vec = l(match_vec) + return match_vec + + +class DeepCF(fluid.imperative.Layer): + def __init__(self, name_scope): + super(DeepCF, self).__init__(name_scope) + + self._user_emb = fluid.imperative.Embedding(self.full_name(), + [NUM_USERS, 256]) + self._item_emb = fluid.imperative.Embedding(self.full_name(), + [NUM_ITEMS, 256]) + + self._mlp = MLP(self.full_name()) + self._dmf = DMF(self.full_name()) + self._match_fc = fluid.imperative.FC(self.full_name(), 1, act='sigmoid') + + def forward(self, users, items): + users_emb = self._user_emb(users) + items_emb = self._item_emb(items) + + mlp_predictive = self._mlp(users_emb, items_emb) + dmf_predictive = self._dmf(users_emb, items_emb) + predictive = fluid.layers.concat( + [mlp_predictive, dmf_predictive], + axis=len(mlp_predictive.shape) - 1) + prediction = self._match_fc(predictive) + return prediction + + +def get_data(): + user_ids = [] + item_ids = [] + labels = [] + for uid in range(NUM_USERS): + for iid in range(NUM_ITEMS): + # 10% positive + label = float(random.randint(1, 10) == 1) + user_ids.append(uid) + item_ids.append(iid) + labels.append(label) + indices = np.arange(NUM_USERS * NUM_ITEMS) + np.random.shuffle(indices) + users_np = np.array(user_ids, dtype=np.int64)[indices] + items_np = np.array(item_ids, dtype=np.int64)[indices] + labels_np = np.array(labels, dtype=np.float32)[indices] + return np.expand_dims(users_np, -1), \ + np.expand_dims(items_np, -1), \ + np.expand_dims(labels_np, -1) + + +class TestImperativeDeepCF(unittest.TestCase): + def test_gan_float32(self): + seed = 90 + users_np, items_np, labels_np = get_data() + + startup = fluid.Program() + startup.random_seed = seed + main = fluid.Program() + main.random_seed = seed + + scope = fluid.core.Scope() + with new_program_scope(main=main, startup=startup, scope=scope): + users = fluid.layers.data('users', [1], dtype='int64') + items = fluid.layers.data('items', [1], dtype='int64') + labels = fluid.layers.data('labels', [1], dtype='float32') + + deepcf = DeepCF('deepcf') + prediction = deepcf(users, items) + loss = fluid.layers.reduce_sum( + fluid.layers.log_loss(prediction, labels)) + adam = fluid.optimizer.AdamOptimizer(0.01) + adam.minimize(loss) + + exe = fluid.Executor(fluid.CPUPlace( + ) if not core.is_compiled_with_cuda() else fluid.CUDAPlace(0)) + exe.run(startup) + for slice in range(0, BATCH_SIZE * NUM_BATCHES, BATCH_SIZE): + static_loss = exe.run( + main, + feed={ + users.name: users_np[slice:slice + BATCH_SIZE], + items.name: items_np[slice:slice + BATCH_SIZE], + labels.name: labels_np[slice:slice + BATCH_SIZE] + }, + fetch_list=[loss])[0] + sys.stderr.write('static loss %s\n' % static_loss) + + with fluid.imperative.guard(): + fluid.default_startup_program().random_seed = seed + fluid.default_main_program().random_seed = seed + + deepcf = DeepCF('deepcf') + for slice in range(0, BATCH_SIZE * NUM_BATCHES, BATCH_SIZE): + prediction = deepcf( + to_variable(users_np[slice:slice + BATCH_SIZE]), + to_variable(items_np[slice:slice + BATCH_SIZE])) + loss = fluid.layers.reduce_sum( + fluid.layers.log_loss(prediction, + to_variable(labels_np[slice:slice + + BATCH_SIZE]))) + loss._backward() + adam = fluid.optimizer.AdamOptimizer(0.01) + adam.minimize(loss) + deepcf.clear_gradients() + dy_loss = loss._numpy() + + self.assertEqual(static_loss, dy_loss) + + +if __name__ == '__main__': + unittest.main() diff --git a/python/paddle/fluid/tests/unittests/test_imperative_gan.py b/python/paddle/fluid/tests/unittests/test_imperative_gan.py index a80202d6dd..6024fb5f81 100644 --- a/python/paddle/fluid/tests/unittests/test_imperative_gan.py +++ b/python/paddle/fluid/tests/unittests/test_imperative_gan.py @@ -51,7 +51,7 @@ class Generator(fluid.imperative.Layer): return self._fc3(x) -class TestImperativeMnist(unittest.TestCase): +class TestImperativeGAN(unittest.TestCase): def test_gan_float32(self): seed = 90 From d11d0e18c27783183867cd2ab6d98adbc44b6df1 Mon Sep 17 00:00:00 2001 From: phlrain Date: Fri, 22 Mar 2019 10:12:21 +0000 Subject: [PATCH 30/44] remove test_dist_transplier; test=develop --- python/paddle/fluid/tests/unittests/CMakeLists.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/python/paddle/fluid/tests/unittests/CMakeLists.txt b/python/paddle/fluid/tests/unittests/CMakeLists.txt index b61ef706ba..ecfbfe3b54 100644 --- a/python/paddle/fluid/tests/unittests/CMakeLists.txt +++ b/python/paddle/fluid/tests/unittests/CMakeLists.txt @@ -105,7 +105,6 @@ if(WITH_DISTRIBUTE) # set_tests_properties(test_dist_transformer PROPERTIES TIMEOUT 1000) set_tests_properties(test_dist_ctr test_dist_mnist test_dist_mnist_batch_merge test_dist_save_load test_dist_se_resnext test_dist_simnet_bow test_dist_text_classification test_dist_train test_dist_word2vec PROPERTIES RUN_SERIAL TRUE) endif(NOT APPLE) - py_test_modules(test_dist_transpiler MODULES test_dist_transpiler) endif() py_test_modules(test_parallel_executor_crf MODULES test_parallel_executor_crf SERIAL) py_test_modules(test_parallel_executor_fetch_feed MODULES test_parallel_executor_fetch_feed SERIAL) From 6b971e1f190faf31e06855bc9207bf31a1610671 Mon Sep 17 00:00:00 2001 From: phlrain Date: Fri, 22 Mar 2019 10:16:55 +0000 Subject: [PATCH 31/44] remove test_dist_transplier; test=develop --- python/paddle/fluid/tests/unittests/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/python/paddle/fluid/tests/unittests/CMakeLists.txt b/python/paddle/fluid/tests/unittests/CMakeLists.txt index ecfbfe3b54..3c6b9daca6 100644 --- a/python/paddle/fluid/tests/unittests/CMakeLists.txt +++ b/python/paddle/fluid/tests/unittests/CMakeLists.txt @@ -105,6 +105,7 @@ if(WITH_DISTRIBUTE) # set_tests_properties(test_dist_transformer PROPERTIES TIMEOUT 1000) set_tests_properties(test_dist_ctr test_dist_mnist test_dist_mnist_batch_merge test_dist_save_load test_dist_se_resnext test_dist_simnet_bow test_dist_text_classification test_dist_train test_dist_word2vec PROPERTIES RUN_SERIAL TRUE) endif(NOT APPLE) + # py_test_modules(test_dist_transpiler MODULES test_dist_transpiler) endif() py_test_modules(test_parallel_executor_crf MODULES test_parallel_executor_crf SERIAL) py_test_modules(test_parallel_executor_fetch_feed MODULES test_parallel_executor_fetch_feed SERIAL) From 5d6737b5cbf3669ac0df06660e2826452dc9eb53 Mon Sep 17 00:00:00 2001 From: qingqing01 Date: Fri, 22 Mar 2019 20:30:10 +0800 Subject: [PATCH 32/44] Fix bug in affine_channel API (#16373) --- python/paddle/fluid/layers/nn.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/paddle/fluid/layers/nn.py b/python/paddle/fluid/layers/nn.py index 45e7a67711..e2c8be613f 100644 --- a/python/paddle/fluid/layers/nn.py +++ b/python/paddle/fluid/layers/nn.py @@ -9768,7 +9768,7 @@ def affine_channel(x, 'Bias': bias}, attrs={"data_layout": data_layout}, outputs={"Out": out}) - return helper.append_activation(pre_activation) + return helper.append_activation(out) def similarity_focus(input, axis, indexes, name=None): From 18779b5b8fb8007d10dc24f38dbbc611848f81e1 Mon Sep 17 00:00:00 2001 From: whs Date: Sat, 23 Mar 2019 07:39:28 +0800 Subject: [PATCH 33/44] [Operator] Add range op. (#15431) * Add range op. test=develop * Add more unitests. test=develop * Fix API.spec test=develop * Fix API.spec test=develop * Fix API.spec test=develop --- paddle/fluid/API.spec | 1 + paddle/fluid/operators/range_op.cc | 69 ++++++++++++++++++ paddle/fluid/operators/range_op.cu | 67 +++++++++++++++++ paddle/fluid/operators/range_op.h | 56 +++++++++++++++ python/paddle/fluid/layers/tensor.py | 71 +++++++++++++++++-- .../fluid/tests/unittests/test_layers.py | 8 +++ .../fluid/tests/unittests/test_range.py | 70 ++++++++++++++++++ 7 files changed, 338 insertions(+), 4 deletions(-) create mode 100644 paddle/fluid/operators/range_op.cc create mode 100644 paddle/fluid/operators/range_op.cu create mode 100644 paddle/fluid/operators/range_op.h create mode 100644 python/paddle/fluid/tests/unittests/test_range.py diff --git a/paddle/fluid/API.spec b/paddle/fluid/API.spec index 598604ca7a..6c61a3d63d 100644 --- a/paddle/fluid/API.spec +++ b/paddle/fluid/API.spec @@ -255,6 +255,7 @@ paddle.fluid.layers.reverse (ArgSpec(args=['x', 'axis'], varargs=None, keywords= paddle.fluid.layers.has_inf (ArgSpec(args=['x'], varargs=None, keywords=None, defaults=None), ('document', '8f8c0306117ea441f20dcbbdba1f0ecc')) paddle.fluid.layers.has_nan (ArgSpec(args=['x'], varargs=None, keywords=None, defaults=None), ('document', '2e53e83127dbfd86e7098bdfe9a549e8')) paddle.fluid.layers.isfinite (ArgSpec(args=['x'], varargs=None, keywords=None, defaults=None), ('document', '0a437011c3906079fd8947ed3e52d292')) +paddle.fluid.layers.range (ArgSpec(args=['start', 'end', 'step', 'dtype'], varargs=None, keywords=None, defaults=None), ('document', '2ec937ede953ded2fdff2675883900bb')) paddle.fluid.layers.While.__init__ (ArgSpec(args=['self', 'cond', 'is_test', 'name'], varargs=None, keywords=None, defaults=(False, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.layers.While.block (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.layers.Switch.__init__ (ArgSpec(args=['self', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) diff --git a/paddle/fluid/operators/range_op.cc b/paddle/fluid/operators/range_op.cc new file mode 100644 index 0000000000..ee8c68fd00 --- /dev/null +++ b/paddle/fluid/operators/range_op.cc @@ -0,0 +1,69 @@ +/* Copyright (c) 2016 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/range_op.h" + +namespace paddle { +namespace operators { + +class RangeOp : public framework::OperatorWithKernel { + public: + using framework::OperatorWithKernel::OperatorWithKernel; + + void InferShape(framework::InferShapeContext *ctx) const override { + if (ctx->HasInput("Start")) { + auto s_dims = ctx->GetInputDim("Start"); + PADDLE_ENFORCE((s_dims.size() == 1) && (s_dims[0] == 1), + "The shape of Input(Start) should be [1]."); + } + if (ctx->HasInput("End")) { + auto e_dims = ctx->GetInputDim("End"); + PADDLE_ENFORCE((e_dims.size() == 1) && (e_dims[0] == 1), + "The shape of Input(End) should be [1]."); + } + if (ctx->HasInput("Step")) { + auto step_dims = ctx->GetInputDim("Step"); + PADDLE_ENFORCE((step_dims.size() == 1) && (step_dims[0] == 1), + "The shape of Input(Step) should be [1]."); + } + ctx->SetOutputDim("Out", {-1}); + } +}; + +class RangeOpMaker : public framework::OpProtoAndCheckerMaker { + public: + void Make() override { + AddInput("Start", + "Start of interval. The interval includes this value. It is a " + "tensor with shape=[1]."); + AddInput("End", + "End of interval. The interval does not include this value, " + "except in some cases where step is not an integer and floating " + "point round-off affects the length of out. It is a tensor with " + "shape=[1]."); + AddInput("Step", "Spacing between values. It is a tensor with shape=[1]."); + AddOutput("Out", "A sequence of numbers."); + AddComment(R"DOC( + Return evenly spaced values within a given interval. Values are generated within the half-open interval [start, stop) (in other words, the interval including start but excluding stop). Like arange function of numpy. +)DOC"); + } +}; +} // namespace operators +} // namespace paddle + +namespace ops = paddle::operators; +REGISTER_OP_WITHOUT_GRADIENT(range, ops::RangeOp, ops::RangeOpMaker); +REGISTER_OP_CPU_KERNEL(range, ops::CPURangeKernel, + ops::CPURangeKernel, ops::CPURangeKernel, + ops::CPURangeKernel); diff --git a/paddle/fluid/operators/range_op.cu b/paddle/fluid/operators/range_op.cu new file mode 100644 index 0000000000..e2c03716d5 --- /dev/null +++ b/paddle/fluid/operators/range_op.cu @@ -0,0 +1,67 @@ +/* Copyright (c) 2016 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/op_registry.h" +#include "paddle/fluid/operators/range_op.h" +#include "paddle/fluid/platform/cuda_primitives.h" + +namespace paddle { +namespace operators { + +#define CUDA_1D_KERNEL_LOOP(i, n) \ + for (int i = blockIdx.x * blockDim.x + threadIdx.x; i < (n); \ + i += blockDim.x * gridDim.x) + +template +__global__ void RangeKernel(T start, T step, int64_t size, T* out) { + CUDA_1D_KERNEL_LOOP(index, size) { out[index] = start + step * index; } +} + +template +class CUDARangeKernel : public framework::OpKernel { + public: + void Compute(const framework::ExecutionContext& context) const override { + auto* start_t = context.Input("Start"); + auto* end_t = context.Input("End"); + auto* step_t = context.Input("Step"); + auto* out = context.Output("Out"); + + framework::Tensor n; + framework::TensorCopy(*start_t, platform::CPUPlace(), &n); + T start = n.data()[0]; + framework::TensorCopy(*end_t, platform::CPUPlace(), &n); + T end = n.data()[0]; + framework::TensorCopy(*step_t, platform::CPUPlace(), &n); + T step = n.data()[0]; + + int64_t size = 0; + GetSize(start, end, step, &size); + out->Resize(framework::make_ddim({size})); + T* out_data = out->mutable_data(context.GetPlace()); + + auto stream = context.cuda_device_context().stream(); + int block = 512; + int grid = (size + block - 1) / block; + RangeKernel<<>>(start, step, size, out_data); + } +}; + +} // namespace operators +} // namespace paddle + +namespace ops = paddle::operators; +REGISTER_OP_CUDA_KERNEL(range, ops::CUDARangeKernel, + ops::CUDARangeKernel, + ops::CUDARangeKernel, + ops::CUDARangeKernel); diff --git a/paddle/fluid/operators/range_op.h b/paddle/fluid/operators/range_op.h new file mode 100644 index 0000000000..fce58b45c9 --- /dev/null +++ b/paddle/fluid/operators/range_op.h @@ -0,0 +1,56 @@ +/* Copyright (c) 2016 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/op_registry.h" +#include "paddle/fluid/operators/math/math_function.h" + +namespace paddle { +namespace operators { + +template +void GetSize(T start, T end, T step, int64_t* size) { + PADDLE_ENFORCE(!std::equal_to()(step, 0), + "The step of range op should not be 0."); + PADDLE_ENFORCE(((start < end) && (step > 0)) || ((start > end) && (step < 0)), + "The step should be greater than 0 while start < end. And the " + "step should be less than 0 while start > end."); + *size = std::is_integral::value + ? ((std::abs(end - start) + std::abs(step) - 1) / std::abs(step)) + : std::ceil(std::abs((end - start) / step)); +} + +template +class CPURangeKernel : public framework::OpKernel { + public: + void Compute(const framework::ExecutionContext& context) const override { + T start = context.Input("Start")->data()[0]; + T end = context.Input("End")->data()[0]; + T step = context.Input("Step")->data()[0]; + auto* out = context.Output("Out"); + int64_t size = 0; + GetSize(start, end, step, &size); + out->Resize(framework::make_ddim({size})); + T* out_data = out->mutable_data(context.GetPlace()); + T value = start; + for (int64_t i = 0; i < size; ++i) { + out_data[i] = value; + value += step; + } + } +}; + +} // namespace operators +} // namespace paddle diff --git a/python/paddle/fluid/layers/tensor.py b/python/paddle/fluid/layers/tensor.py index cb97398698..a18e5b6a9c 100644 --- a/python/paddle/fluid/layers/tensor.py +++ b/python/paddle/fluid/layers/tensor.py @@ -25,10 +25,26 @@ from .layer_function_generator import templatedoc import numpy __all__ = [ - 'create_tensor', 'create_parameter', 'create_global_var', 'cast', - 'tensor_array_to_tensor', 'concat', 'sums', 'assign', - 'fill_constant_batch_size_like', 'fill_constant', 'argmin', 'argmax', - 'argsort', 'ones', 'zeros', 'reverse', 'has_inf', 'has_nan', 'isfinite' + 'create_tensor', + 'create_parameter', + 'create_global_var', + 'cast', + 'tensor_array_to_tensor', + 'concat', + 'sums', + 'assign', + 'fill_constant_batch_size_like', + 'fill_constant', + 'argmin', + 'argmax', + 'argsort', + 'ones', + 'zeros', + 'reverse', + 'has_inf', + 'has_nan', + 'isfinite', + 'range', ] @@ -764,3 +780,50 @@ def isfinite(x): out = helper.create_variable_for_type_inference(dtype=x.dtype) helper.append_op(type="isfinite", inputs={"X": x}, outputs={"Out": out}) return out + + +def range(start, end, step, dtype): + """ + Return evenly spaced values within a given interval. + + Values are generated within the half-open interval [start, stop) (in other words, + the interval including start but excluding stop). + + args: + start(int|float|Variable): Start of interval. The interval includes this value. + end(int|float|Variable): End of interval. The interval does not include this + value, except in some cases where step is not an integer + and floating point round-off affects the length of out. + step(int|float|Variable): Spacing between values. For any output out, this is the + distance between two adjacent values, out[i+1] - out[i]. + The default step size is 1. + dtype(string): 'float32'|'int32'|..., the data type of the output tensor. + + returns: + Evenly spaced values within a given interval. + + examples: + + .. code-block:: python + + data = fluid.layers.range(0, 10, 2, 'int32') + + """ + helper = LayerHelper("range", **locals()) + + if not isinstance(start, Variable): + start = fill_constant([1], dtype, start) + if not isinstance(end, Variable): + end = fill_constant([1], dtype, end) + if not isinstance(step, Variable): + step = fill_constant([1], dtype, step) + + out = helper.create_variable_for_type_inference(dtype=start.dtype) + + helper.append_op( + type='range', + inputs={'Start': start, + 'End': end, + 'Step': step}, + outputs={'Out': [out]}) + return out diff --git a/python/paddle/fluid/tests/unittests/test_layers.py b/python/paddle/fluid/tests/unittests/test_layers.py index 885ee170e8..1672c3600f 100644 --- a/python/paddle/fluid/tests/unittests/test_layers.py +++ b/python/paddle/fluid/tests/unittests/test_layers.py @@ -1240,6 +1240,14 @@ class TestBook(unittest.TestCase): print(str(program)) + def test_range(self): + program = Program() + with program_guard(program): + layers.range(0, 10, 2, 'int32') + layers.range(0.1, 10.0, 0.2, 'float32') + + print(str(program)) + def test_spectral_norm(self): program = Program() with program_guard(program): diff --git a/python/paddle/fluid/tests/unittests/test_range.py b/python/paddle/fluid/tests/unittests/test_range.py new file mode 100644 index 0000000000..f129ae78cb --- /dev/null +++ b/python/paddle/fluid/tests/unittests/test_range.py @@ -0,0 +1,70 @@ +# 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. + +from __future__ import print_function + +import unittest +import numpy as np +from op_test import OpTest + + +class TestRangeOp(OpTest): + def setUp(self): + self.op_type = "range" + self.init_config() + self.inputs = { + 'Start': np.array([self.case[0]]).astype(self.dtype), + 'End': np.array([self.case[1]]).astype(self.dtype), + 'Step': np.array([self.case[2]]).astype(self.dtype) + } + + self.outputs = { + 'Out': np.arange(self.case[0], self.case[1], + self.case[2]).astype(self.dtype) + } + + def init_config(self): + self.dtype = np.float32 + self.case = (0, 1, 0.2) + + def test_check_output(self): + self.check_output() + + +class TestFloatRangeOpCase0(TestRangeOp): + def init_config(self): + self.dtype = np.float32 + self.case = (0, 5, 1) + + +class TestInt32RangeOpCase0(TestRangeOp): + def init_config(self): + self.dtype = np.int32 + self.case = (0, 5, 2) + + +class TestInt32RangeOpCase1(TestRangeOp): + def init_config(self): + self.dtype = np.int32 + self.case = (10, 1, -2) + + +class TestInt32RangeOpCase2(TestRangeOp): + def init_config(self): + self.dtype = np.int32 + self.case = (-1, -10, -2) + + +if __name__ == "__main__": + unittest.main() From 2e5831f0dc4d1c63d738001bdb7f51838488a7d3 Mon Sep 17 00:00:00 2001 From: whs Date: Sat, 23 Mar 2019 11:54:56 +0800 Subject: [PATCH 34/44] [slim] Refine framework of slim and add filter pruning strategy (#16226) * First pr of paddle slim. 1. Add framework of paddle slim 2. Add filter pruning strategy test=develop * Rename unitest to tests. test=develop * Add prettytable into requirements. test=develop * Change in_nodes and out_nodes to odered dict. test=develop * Remove distillation. test=develop * Fix API.spec test=develop * Fix unitest. test=develop * Fix unitest in windows. test=develop * Fix unitest in windows. test=develop * Fix unitest. test=develop * Hide some functions. test=develop * Fix python import in python3.5 test=develop * Fix compress pass. test=develop * Fix unitest of test_dist_ctr. test=develop * Enhence flops. * use os.path.join * Fix pickle for python3 Fix log and comments. test=develop * 1. Remove feed_reader in compress pass 2. Fix cache reader 3. Rename CompressPass to Compressor 4. Add comments for distiller optimizer 5. Remove unused pruner currently 6. Add some comments. 7. Change API.spec test=develop * Fix pruning in python3. test=develop * Fix unitest in python3. test=develop * Fix format in python3. test=develop --- paddle/fluid/API.spec | 20 +- python/paddle/fluid/contrib/slim/__init__.py | 11 +- .../fluid/contrib/slim/core/__init__.py | 8 +- .../fluid/contrib/slim/core/compress_pass.py | 129 --- .../fluid/contrib/slim/core/compressor.py | 481 +++++++++ .../paddle/fluid/contrib/slim/core/config.py | 36 +- .../fluid/contrib/slim/core/pass_builder.py | 39 - .../fluid/contrib/slim/core/strategy.py | 6 +- .../slim/demo/filter_prune/config.yaml | 28 - .../contrib/slim/demo/filter_prune/demo.py | 69 -- .../fluid/contrib/slim/graph/__init__.py | 9 +- .../fluid/contrib/slim/graph/executor.py | 67 +- .../paddle/fluid/contrib/slim/graph/graph.py | 49 - .../fluid/contrib/slim/graph/graph_pass.py | 42 - .../fluid/contrib/slim/graph/graph_wrapper.py | 500 ++++++++++ .../contrib/slim/prune/prune_strategy.py | 935 +++++++++++++++++- .../paddle/fluid/contrib/slim/prune/pruner.py | 104 +- .../contrib/slim/tests/configs/config.yaml | 29 - .../slim/tests/configs/filter_pruning.yaml | 34 + .../contrib/slim/tests/configs/pruners.yaml | 12 - .../contrib/slim/tests/configs/pruners_0.yaml | 12 - .../slim/tests/filter_pruning/__init__.py | 13 + .../slim/tests/filter_pruning/compress.yaml | 34 + .../slim/tests/filter_pruning/mobilenet.py | 210 ++++ .../fluid/contrib/slim/tests/test_factory.py | 28 +- .../contrib/slim/tests/test_filter_pruning.py | 89 ++ .../contrib/slim/tests/test_graph_wrapper.py | 140 +++ python/requirements.txt | 1 + 28 files changed, 2545 insertions(+), 590 deletions(-) delete mode 100644 python/paddle/fluid/contrib/slim/core/compress_pass.py create mode 100644 python/paddle/fluid/contrib/slim/core/compressor.py delete mode 100644 python/paddle/fluid/contrib/slim/core/pass_builder.py delete mode 100644 python/paddle/fluid/contrib/slim/demo/filter_prune/config.yaml delete mode 100644 python/paddle/fluid/contrib/slim/demo/filter_prune/demo.py delete mode 100644 python/paddle/fluid/contrib/slim/graph/graph.py delete mode 100644 python/paddle/fluid/contrib/slim/graph/graph_pass.py create mode 100644 python/paddle/fluid/contrib/slim/graph/graph_wrapper.py delete mode 100644 python/paddle/fluid/contrib/slim/tests/configs/config.yaml create mode 100644 python/paddle/fluid/contrib/slim/tests/configs/filter_pruning.yaml delete mode 100644 python/paddle/fluid/contrib/slim/tests/configs/pruners.yaml delete mode 100644 python/paddle/fluid/contrib/slim/tests/configs/pruners_0.yaml create mode 100644 python/paddle/fluid/contrib/slim/tests/filter_pruning/__init__.py create mode 100644 python/paddle/fluid/contrib/slim/tests/filter_pruning/compress.yaml create mode 100644 python/paddle/fluid/contrib/slim/tests/filter_pruning/mobilenet.py create mode 100644 python/paddle/fluid/contrib/slim/tests/test_filter_pruning.py create mode 100644 python/paddle/fluid/contrib/slim/tests/test_graph_wrapper.py diff --git a/paddle/fluid/API.spec b/paddle/fluid/API.spec index 6c61a3d63d..70a4d7b40b 100644 --- a/paddle/fluid/API.spec +++ b/paddle/fluid/API.spec @@ -377,23 +377,9 @@ paddle.fluid.contrib.Calibrator.__init__ (ArgSpec(args=['self'], varargs='args', paddle.fluid.contrib.Calibrator.sample_data (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '3b8c85ca1e2cf753cc8c90a6c6992958')) paddle.fluid.contrib.Calibrator.save_int8_model (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) paddle.fluid.contrib.reader.ctr_reader.ctr_reader (ArgSpec(args=['feed_dict', 'file_type', 'file_format', 'dense_slot_index', 'sparse_slot_index', 'capacity', 'thread_num', 'batch_size', 'file_list', 'slots', 'name'], varargs=None, keywords=None, defaults=(None,)), ('document', 'b2ebf3de2a6ef1af2c3b88d2db7591ab')) -paddle.fluid.contrib.build_compressor (ArgSpec(args=['place', 'data_reader', 'data_feeder', 'scope', 'metrics', 'epoch', 'config'], varargs=None, keywords=None, defaults=(None, None, None, None, None, None, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) -paddle.fluid.contrib.CompressPass.__init__ (ArgSpec(args=['self', 'place', 'data_reader', 'data_feeder', 'scope', 'metrics', 'epoch', 'program_exe'], varargs=None, keywords=None, defaults=(None, None, None, None, None, None, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) -paddle.fluid.contrib.CompressPass.add_strategy (ArgSpec(args=['self', 'strategy'], varargs=None, keywords=None, defaults=None), ('document', '3bf6010b6f47d3c86df0ec8957be95e0')) -paddle.fluid.contrib.CompressPass.apply (ArgSpec(args=['self', 'graph'], varargs=None, keywords=None, defaults=None), ('document', 'a92bf85d4b59bd4f2ac1706d7c4899a6')) -paddle.fluid.contrib.ImitationGraph.__init__ (ArgSpec(args=['self', 'program'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) -paddle.fluid.contrib.ImitationGraph.all_parameters (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) -paddle.fluid.contrib.SensitivePruneStrategy.__init__ (ArgSpec(args=['self', 'pruner', 'start_epoch', 'end_epoch', 'delta_rate', 'acc_loss_threshold', 'sensitivities'], varargs=None, keywords=None, defaults=(None, 0, 10, 0.2, 0.2, None)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) -paddle.fluid.contrib.SensitivePruneStrategy.on_batch_begin (ArgSpec(args=['self', 'context'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) -paddle.fluid.contrib.SensitivePruneStrategy.on_batch_end (ArgSpec(args=['self', 'context'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) -paddle.fluid.contrib.SensitivePruneStrategy.on_compress_begin (ArgSpec(args=['self', 'context'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) -paddle.fluid.contrib.SensitivePruneStrategy.on_compress_end (ArgSpec(args=['self', 'context'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) -paddle.fluid.contrib.SensitivePruneStrategy.on_epoch_begin (ArgSpec(args=['self', 'context'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) -paddle.fluid.contrib.SensitivePruneStrategy.on_epoch_end (ArgSpec(args=['self', 'context'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) -paddle.fluid.contrib.MagnitudePruner.__init__ (ArgSpec(args=['self', 'threshold'], varargs=None, keywords=None, defaults=None), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) -paddle.fluid.contrib.MagnitudePruner.prune (ArgSpec(args=['self', 'param', 'threshold'], varargs=None, keywords=None, defaults=(None,)), ('document', '6adf97f83acf6453d4a6a4b1070f3754')) -paddle.fluid.contrib.RatioPruner.__init__ (ArgSpec(args=['self', 'ratios'], varargs=None, keywords=None, defaults=(None,)), ('document', 'e7a81a325b296a9ca502ee5adb4fc85d')) -paddle.fluid.contrib.RatioPruner.prune (ArgSpec(args=['self', 'param', 'ratio'], varargs=None, keywords=None, defaults=(None,)), ('document', '358cbf2978c91028fb96a195a9884645')) +paddle.fluid.contrib.Compressor.__init__ (ArgSpec(args=['self', 'place', 'scope', 'train_program', 'train_reader', 'train_feed_list', 'train_fetch_list', 'eval_program', 'eval_reader', 'eval_feed_list', 'eval_fetch_list', 'teacher_programs', 'checkpoint_path', 'train_optimizer', 'distiller_optimizer'], varargs=None, keywords=None, defaults=(None, None, None, None, None, None, None, [], './checkpoints', None, None)), ('document', '31ae143830c9bf6b43547dd546c5ba80')) +paddle.fluid.contrib.Compressor.config (ArgSpec(args=['self', 'config_file'], varargs=None, keywords=None, defaults=None), ('document', '780d9c007276ccbb95b292400d7807b0')) +paddle.fluid.contrib.Compressor.run (ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None), ('document', 'c6e43d6a078d307672283c1f36e04fe9')) paddle.fluid.contrib.load_persistables_for_increment (ArgSpec(args=['dirname', 'executor', 'program', 'lookup_table_var', 'lookup_table_var_path'], varargs=None, keywords=None, defaults=None), ('document', '2ab36d4f7a564f5f65e455807ad06c67')) paddle.fluid.contrib.load_persistables_for_inference (ArgSpec(args=['dirname', 'executor', 'program', 'lookup_table_var_name'], varargs=None, keywords=None, defaults=None), ('document', '59066bac9db0ac6ce414d05780b7333f')) paddle.fluid.contrib.convert_dist_to_sparse_program (ArgSpec(args=['program'], varargs=None, keywords=None, defaults=None), ('document', '74c39c595dc70d6be2f16d8e462d282b')) diff --git a/python/paddle/fluid/contrib/slim/__init__.py b/python/paddle/fluid/contrib/slim/__init__.py index 22dbf7c8b6..4a71fab6d0 100644 --- a/python/paddle/fluid/contrib/slim/__init__.py +++ b/python/paddle/fluid/contrib/slim/__init__.py @@ -13,13 +13,4 @@ # limitations under the License. from .core import * -from .graph import * -from .prune import * -__all__ = [ - 'build_compressor', - 'CompressPass', - 'ImitationGraph', - 'SensitivePruneStrategy', - 'MagnitudePruner', - 'RatioPruner', -] +__all__ = ['Compressor', ] diff --git a/python/paddle/fluid/contrib/slim/core/__init__.py b/python/paddle/fluid/contrib/slim/core/__init__.py index 7826d5830a..831bd70ecc 100644 --- a/python/paddle/fluid/contrib/slim/core/__init__.py +++ b/python/paddle/fluid/contrib/slim/core/__init__.py @@ -14,11 +14,9 @@ from . import config from .config import * -from . import compress_pass -from .compress_pass import * +from . import compressor +from .compressor import * from . import strategy from .strategy import * -from . import pass_builder -from .pass_builder import * -__all__ = config.__all__ + compress_pass.__all__ + strategy.__all__ + pass_builder.__all__ +__all__ = config.__all__ + compressor.__all__ + strategy.__all__ diff --git a/python/paddle/fluid/contrib/slim/core/compress_pass.py b/python/paddle/fluid/contrib/slim/core/compress_pass.py deleted file mode 100644 index c4c348b878..0000000000 --- a/python/paddle/fluid/contrib/slim/core/compress_pass.py +++ /dev/null @@ -1,129 +0,0 @@ -# Copyright (c) 2018 PaddlePaddle Authors. All Rights Reserve. -# -# 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 ....core import CPUPlace -from ..graph import get_executor - -__all__ = ['Context', 'CompressPass'] - - -class Context(object): - """ - The context in the process of compression. - Args: - exe: The executor used to execute graph. - graph: The graph to be compressed. - scope: The scope used to execute graph. - program_exe: The program_exe is used to execute the program - created for modifying the variables in scope. - """ - - def __init__(self, exe, graph, scope, program_exe=None): - # The total number of epoches to be trained. - self.epoch = 0 - # Current epoch - self.epoch_id = 0 - # Current batch - self.batch_id = 0 - self.exe = exe - self.graph = graph - self.scope = scope - self.program_exe = program_exe - - -class CompressPass(object): - """ - The pass used to compress model. - Args: - place: The device used in compression. - data_reader: The data_reader used to run graph. - data_feeder: The data_feeder used to run graph. - scope: The scope used to run graph. - metrics: The metrics for evaluating model. - epoch: The total epoches of trainning in compression. - program_exe: The program_exe is used to execute the program - created for modifying the variables in scope. - """ - - def __init__(self, - place=None, - data_reader=None, - data_feeder=None, - scope=None, - metrics=None, - epoch=None, - program_exe=None): - self.strategies = [] - self.place = CPUPlace() if place is None else place - self.data_reader = data_reader - self.data_feeder = data_feeder - self.scope = scope - self.metrics = metrics - self.epoch = epoch - self.program_exe = program_exe - - def add_strategy(self, strategy): - """ - Add a strategy to current compress pass. - Args: - strategy: The strategy to be added into current compress pass. - """ - self.strategies.append(strategy) - self.epoch = max(strategy.end_epoch, self.epoch) - - def apply(self, graph): - """ - Compress a model. - Args: - graph: The target graph to be compressed. - """ - self.executor = get_executor(graph, self.place) - context = Context( - self.executor, graph, self.scope, program_exe=self.program_exe) - - for strategy in self.strategies: - strategy.on_compress_begin(context) - - for epoch in range(self.epoch): - - for strategy in self.strategies: - strategy.on_epoch_begin(context) - - for data in self.data_reader(): - - for strategy in self.strategies: - strategy.on_batch_begin(context) - fetches = None - if self.metrics: - fetches = self.metrics.values() - feed = None - if self.data_feeder: - feed = self.data_feeder.feed(data) - results = self.executor.run(graph, - fetches=fetches, - scope=self.scope, - feed=feed) - if results: - print("results: {}".format( - zip(self.metrics.keys(), results))) - for strategy in self.strategies: - strategy.on_batch_end(context) - context.batch_id += 1 - - for strategy in self.strategies: - strategy.on_epoch_end(context) - context.epoch_id += 1 - - for strategy in self.strategies: - strategy.on_compress_end(context) diff --git a/python/paddle/fluid/contrib/slim/core/compressor.py b/python/paddle/fluid/contrib/slim/core/compressor.py new file mode 100644 index 0000000000..832ade497c --- /dev/null +++ b/python/paddle/fluid/contrib/slim/core/compressor.py @@ -0,0 +1,481 @@ +# Copyright (c) 2018 PaddlePaddle Authors. All Rights Reserve. +# +# 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 ....core import CPUPlace +from .... import compiler +from .... import io +from .... import profiler +from .... import scope_guard +from ....data_feeder import DataFeeder +from ..graph import * +from .config import ConfigFactory +import numpy as np +from collections import Iterable +import time +import os +import logging +import sys +import pickle +import functools + +__all__ = ['Context', 'Compressor'] + +logging.basicConfig(format='%(asctime)s-%(levelname)s: %(message)s') +_logger = logging.getLogger(__name__) +_logger.setLevel(logging.INFO) + + +def cached_reader(reader, sampled_rate, cache_path, cached_id): + """ + Sample partial data from reader and cache them into local file system. + Args: + reader: Iterative data source. + sampled_rate(float): The sampled rate used to sample partial data for evaluation. None means using all data in eval_reader. default: None. + cache_path(str): The path to cache the sampled data. + cached_id(int): The id of dataset sampled. Evaluations with same cached_id use the same sampled dataset. default: 0. + """ + np.random.seed(cached_id) + cache_path = os.path.join(cache_path, str(cached_id)) + _logger.debug('read data from: {}'.format(cache_path)) + + def s_reader(): + if os.path.isdir(cache_path): + for file_name in open(os.path.join(cache_path, "list")): + yield np.load(os.path.join(cache_path, file_name.strip())) + else: + os.makedirs(cache_path) + list_file = open(os.path.join(cache_path, "list"), 'w') + batch = 0 + dtype = None + for data in reader(): + if batch == 0 or (np.random.uniform() < sampled_rate): + np.save( + os.path.join(cache_path, 'batch' + str(batch)), data) + list_file.write('batch' + str(batch) + '.npy\n') + batch += 1 + yield data + + return s_reader + + +class Context(object): + """ + The context in the process of compression. + """ + + def __init__(self, + place, + scope, + train_graph=None, + train_reader=None, + eval_graph=None, + eval_reader=None, + teacher_graphs=None, + train_optimizer=None, + distiller_optimizer=None): + """ + Args: + place: The device place where the compression job running. + scope: The scope used in compression job. + train_graph: The graph with loss as output node. + eval_graph: The graph used for evaluation. + eval_reader: The data reader used for evaluation. + teacher_graphs: The teacher graphs used in distillation strategies. + train_optimizer: The optimizer used to append backward ops and + optimization ops into train_graph. + distiller_optimizer: The optimizer used by distillation strategies. + """ + # The total number of epoches to be trained. + self.epoch = 0 + # Current epoch + self.epoch_id = 0 + # Current batch + self.batch_id = 0 + + self.k_v = {} + + self.place = place + self.scope = scope + self.train_graph = train_graph + self.train_reader = train_reader + self.eval_graph = eval_graph + self.eval_reader = eval_reader + self.executor = None + self.teacher_graphs = teacher_graphs + self.train_optimizer = train_optimizer + self.distiller_optimizer = distiller_optimizer + self.optimize_graph = None + self.cache_path = './eval_cache' + self.eval_results = {} + + def to_file(self, file_name): + """ + Save the context into file. + """ + data = {} + data['epoch_id'] = self.epoch_id + data['eval_results'] = self.eval_results + with open(file_name, 'wb') as context_file: + pickle.dump(data, context_file) + + def from_file(self, file_name): + """ + Load the context from file. + """ + with open(file_name) as context_file: + if sys.version_info < (3, 0): + data = pickle.load(context_file) + else: + data = pickle.load(context_file, encoding='bytes') + self.epoch_id = data['epoch_id'] + self.eval_results = data['eval_results'] + + def eval_converged(self, metric_name, delta=0.001): + """ + Check whether the training has been converged. + Args: + metric_name(str): The metric used to check convergence. + delta(float): '(metric[k] - metric[k-1] / metric[k-1]) < delta' + means that the training has been converged. + Returns: + bool: True means the training has been converged. + """ + # TODO(wanghaoshuang@baidu.com): enhence this method. + if (metric_name not in self.eval_results + ) or len(self.eval_results[metric_name]) < 2: + return False + results = self.eval_results[metric_name][-2:] + _logger.info('Latest evaluations: {}'.format(results)) + return abs(results[1] - results[0]) / results[0] < delta + + def run_eval_graph(self, sampled_rate=None, cached_id=0): + """ + Evaluate the current mode in context. + Args: + sampled_rate(float): The sampled rate used to sample partial data + for evaluation. None means using all data in eval_reader. default: None. + cached_id(int): The id of dataset sampled. Evaluations with same + cached_id use the same sampled dataset. default: 0. + """ + _logger.info('Running evaluation') + assert self.eval_graph is not None + assert self.eval_reader is not None + eval_graph = self.eval_graph.clone(for_test=True) + + executor = SlimGraphExecutor(self.place) + results = [] + batch_id = 0 + s_time = time.time() + reader = self.eval_reader + if sampled_rate: + reader = cached_reader(reader, sampled_rate, self.cache_path, + cached_id) + for data in reader(): + result = executor.run(eval_graph, self.scope, data=data) + result = [np.mean(r) for r in result] + results.append(result) + if batch_id % 20 == 0: + _logger.info("batch-{}; {}={}".format( + batch_id, eval_graph.out_nodes.keys(), result)) + batch_id += 1 + result = np.mean(np.array(results), axis=0) + _logger.info("Final eval result: {}={}".format( + eval_graph.out_nodes.keys(), result)) + if not isinstance(result, Iterable): + result = [result] + _logger.info('Finish evaluation') + return result, eval_graph.out_nodes.keys() + + def put(self, key, value): + self.k_v[key] = value + + def get(self, key): + return self.k_v.get(key) + + +class Compressor(object): + """ + The pass used to compress model. + """ + + def __init__(self, + place, + scope, + train_program, + train_reader=None, + train_feed_list=None, + train_fetch_list=None, + eval_program=None, + eval_reader=None, + eval_feed_list=None, + eval_fetch_list=None, + teacher_programs=[], + checkpoint_path='./checkpoints', + train_optimizer=None, + distiller_optimizer=None): + """ + Args: + place(fluid.Place): The device place where the compression job running. + scope(fluid.core.Scope): The scope used to run graph. + train_program(Program): The main program to be compressed. It must have loss op. + train_reader: The data reader used for training. + train_feed_list(dict): A dict to indicate the input variable of the training program. + The key is user-defined and human-readable name. + The value is the name of Variable. + train_fetch_list(dict): A dict to indicate the output variable of the training program. + The key is user-defined and human-readable name. + The value is the name of Variable. + eval_program(Program): The program used for evaluation. + eval_reader: The data reader used for evaluation. + eval_feed_list(dict): A dict to indicate the input variable of the evaluation program. + The key is user-defined and human-readable name. + The value is the name of Variable. + eval_fetch_list(dict): A dict to indicate the output variable of the evaluation program. + The key is user-defined and human-readable name. + The value is the name of Variable. + teacher_programs: The teacher graphs used in distillation strategies. + train_optimizer: The optimizer used to append backward ops and + optimization ops into train_graph. + distiller_optimizer: The optimizer used by distillation strategies. In distillation strategy, + this optimizer is used to minimize the combined loss of student-net and + teacher-net while train_optimizer is used to minimize loss of + student-net in fine-tune stage. + + """ + assert isinstance( + train_feed_list, list + ), "train_feed_list should be a list of tuple, such as [('image', image.name), ('label', gt.name)]" + assert isinstance( + eval_feed_list, list + ), "eval_feed_list should be a list of tuple, such as [('image', image.name), ('label', gt.name)]" + self.strategies = [] + self.epoch = 0 + self.place = CPUPlace() if place is None else place + self.scope = scope + self.train_graph = GraphWrapper( + train_program, in_nodes=train_feed_list, out_nodes=train_fetch_list) + self.eval_graph = GraphWrapper( + eval_program, in_nodes=eval_feed_list, out_nodes=eval_fetch_list) + self.train_reader = train_reader + self.eval_reader = eval_reader + self.teacher_graphs = [] + for teacher in teacher_programs: + self.teacher_graphs.append(ImitationGraph(teacher, scope=scope)) + + self.checkpoint = None + self.checkpoint_path = checkpoint_path + self.eval_epoch = 1 + + self.train_optimizer = train_optimizer + self.distiller_optimizer = distiller_optimizer + self.init_model = None + + def _add_strategy(self, strategy): + """ + Add a strategy to current compress pass. + Args: + strategy: The strategy to be added into current compress pass. + """ + self.strategies.append(strategy) + self.epoch = max(strategy.end_epoch, self.epoch) + + def config(self, config_file): + """ + Configure the compress pass from file with yaml format. + Args: + config_file(str): The config file in local file system. + """ + factory = ConfigFactory(config_file) + self.epoch = factory.compressor['epoch'] + for strategy in factory.compressor['strategies']: + self._add_strategy(strategy) + if 'checkpoint_path' in factory.compressor: + self.checkpoint_path = factory.compressor['checkpoint_path'] + + if 'init_model' in factory.compressor: + self.init_model = factory.compressor['init_model'] + + def _init_model(self, context): + """ + Load model that has been compressed. + """ + if self.init_model and os.path.exists(self.init_model): + exe = SlimGraphExecutor(context.place) + with scope_guard(context.scope): + context.train_graph.load_persistables(self.init_model, exe) + flops = context.eval_graph.flops() + conv_flops = context.eval_graph.flops(only_conv=True) + context.eval_graph.update_param_shape(context.scope) + context.eval_graph.update_groups_of_conv() + _logger.info("conv flops: -{}".format(1 - float( + context.eval_graph.flops(only_conv=True)) / conv_flops)) + _logger.info("total flops: -{}".format(1 - float( + context.eval_graph.flops()) / flops)) + context.train_graph.update_param_shape(context.scope) + context.train_graph.update_groups_of_conv() + context.train_graph.infer_shape() + _logger.info("Init model from: {}".format(self.init_model)) + + def _load_checkpoint(self, context): + """ + Load checkpoints from file. + """ + _logger.debug('_load_checkpoint') + strategies = self.strategies + if self.checkpoint_path: + if not os.path.exists(self.checkpoint_path): + _logger.warning("Checkpints path doesn't exist: [{}]".format( + self.checkpoint_path)) + return context, strategies + checkpoints = [ + dir for dir in os.listdir(self.checkpoint_path) + if os.path.isdir(os.path.join(self.checkpoint_path, dir)) + ] + _logger.debug('self.checkpoint_path: {}'.format( + self.checkpoint_path)) + _logger.info('checkpoints: {}'.format(checkpoints)) + if len(checkpoints) > 0: + latest = max([int(ck) for ck in checkpoints]) + latest_ck_path = os.path.join(self.checkpoint_path, str(latest)) + + model_path = os.path.join(latest_ck_path, 'model') + context_path = os.path.join(latest_ck_path, 'context') + strategy_path = os.path.join(latest_ck_path, 'strategies') + if os.path.exists(context_path): + context.from_file(context_path) + context.epoch_id += 1 + if os.path.exists(strategy_path): + with open(strategy_path, 'rb') as strategy_file: + if sys.version_info < (3, 0): + strategies = pickle.load(strategy_file) + else: + strategies = pickle.load( + strategy_file, encoding='bytes') + + if os.path.exists(model_path): + exe = SlimGraphExecutor(context.place) + with scope_guard(context.scope): + context.optimize_graph.load_persistables(model_path, + exe) + context.optimize_graph.update_param_shape(context.scope) + context.optimize_graph.update_groups_of_conv() + context.eval_graph.update_param_shape(context.scope) + context.eval_graph.update_groups_of_conv() + _logger.info("Loaded params from: {}".format(model_path)) + return context, strategies + + def _save_checkpoint(self, context): + """ + Save checkpoints to file. + """ + if context.epoch_id % 1 == 0 and self.checkpoint_path: + checkpoint_path = os.path.join(self.checkpoint_path, + str(context.epoch_id)) + model_path = os.path.join(checkpoint_path, 'model') + context_path = os.path.join(checkpoint_path, 'context') + strategy_path = os.path.join(checkpoint_path, 'strategies') + if not os.path.isdir(model_path): + os.makedirs(model_path) + exe = SlimGraphExecutor(context.place) + with scope_guard(context.scope): + context.optimize_graph.save_persistables(model_path, exe) + context.to_file(context_path) + with open(strategy_path, 'wb') as strategy_file: + pickle.dump(self.strategies, strategy_file) + _logger.info('Saved checkpoint to: {}'.format(checkpoint_path)) + + def _train_one_epoch(self, context): + """ + Train one epoch. + """ + + executor = SlimGraphExecutor(self.place) + + if context.optimize_graph.compiled_graph is None: + context.optimize_graph.compiled_graph = compiler.CompiledProgram( + context.optimize_graph.program).with_data_parallel( + loss_name=context.optimize_graph.out_nodes['loss']) + + for data in context.train_reader(): + for strategy in self.strategies: + strategy.on_batch_begin(context) + results = executor.run(context.optimize_graph, + context.scope, + data=data) + results = [float(np.mean(result)) for result in results] + if context.batch_id % 20 == 0: + _logger.info("epoch:{}; batch_id:{}; {} = {}".format( + context.epoch_id, context.batch_id, + context.optimize_graph.out_nodes.keys( + ), [round(r, 3) for r in results])) + for strategy in self.strategies: + strategy.on_batch_end(context) + context.batch_id += 1 + context.batch_id = 0 + + def _eval(self, context): + """ + Runing evaluation. + """ + results, names = context.run_eval_graph() + for name, result in zip(names, results): + if name not in context.eval_results: + context.eval_results[name] = [] + context.eval_results[name].append(result) + + def run(self): + """ + Execute compressiong pass. + """ + context = Context( + place=self.place, + scope=self.scope, + train_graph=self.train_graph, + train_reader=self.train_reader, + eval_graph=self.eval_graph, + eval_reader=self.eval_reader, + teacher_graphs=self.teacher_graphs, + train_optimizer=self.train_optimizer, + distiller_optimizer=self.distiller_optimizer) + self.context = context + if self.teacher_graphs: + context.put('teachers', self.teacher_graphs) + self._init_model(context) + if not context.optimize_graph: + if context.train_optimizer: + context.train_optimizer._name = 'train_opt' + context.optimize_graph = context.train_graph.get_optimize_graph( + context.train_optimizer, context.place, context.scope) + else: + context.optimize_graph = context.train_graph + + context, self.strategies = self._load_checkpoint(context) + + for strategy in self.strategies: + strategy.on_compression_begin(context) + start = context.epoch_id + self._eval(context) + for epoch in range(start, self.epoch): + context.epoch_id = epoch + for strategy in self.strategies: + strategy.on_epoch_begin(context) + self._train_one_epoch(context) + for strategy in self.strategies: + strategy.on_epoch_end(context) + if self.eval_epoch and epoch % self.eval_epoch == 0: + self._eval(context) + self._save_checkpoint(context) + for strategy in self.strategies: + strategy.on_compression_end(context) + return context.eval_graph diff --git a/python/paddle/fluid/contrib/slim/core/config.py b/python/paddle/fluid/contrib/slim/core/config.py index 811c457003..12df9fcd1b 100644 --- a/python/paddle/fluid/contrib/slim/core/config.py +++ b/python/paddle/fluid/contrib/slim/core/config.py @@ -17,7 +17,7 @@ import funcsigs import yaml from collections import OrderedDict from ..prune import * -from .compress_pass import * +from ..quantization import * from .strategy import * __all__ = ['ConfigFactory'] @@ -29,15 +29,10 @@ class ConfigFactory(object): def __init__(self, config): """Init a factory from configure file.""" self.instances = {} + self.compressor = {} self.version = None self._parse_config(config) - def get_compress_pass(self): - """ - Get compress pass from factory. - """ - return self.instance('compress_pass') - def instance(self, name): """ Get instance from factory. @@ -59,8 +54,16 @@ class ConfigFactory(object): args = {} for key in keys: value = attrs[key] + if isinstance(value, str) and value.lower() == 'none': + value = None if isinstance(value, str) and value in self.instances: value = self.instances[value] + if isinstance(value, list): + for i in range(len(value)): + if isinstance(value[i], + str) and value[i] in self.instances: + value[i] = self.instances[value[i]] + args[key] = value self.instances[name] = class_(**args) return self.instances.get(name) @@ -76,16 +79,23 @@ class ConfigFactory(object): assert self.version == int(key_values['version']) # parse pruners - if key == 'pruners' or key == 'strategies': + if key == 'distillers' or key == 'pruners' or key == 'quantizers' or key == 'strategies': instances = key_values[key] for name in instances: self._new_instance(name, instances[name]) - if key == 'compress_pass': - compress_pass = self._new_instance(key, key_values[key]) - for name in key_values[key]['strategies']: - strategy = self.instance(name) - compress_pass.add_strategy(strategy) + if key == 'compressor': + self.compressor['strategies'] = [] + self.compressor['epoch'] = key_values[key]['epoch'] + if 'init_model' in key_values[key]: + self.compressor['init_model'] = key_values[key][ + 'init_model'] + self.compressor['checkpoint_path'] = key_values[key][ + 'checkpoint_path'] + if 'strategies' in key_values[key]: + for name in key_values[key]['strategies']: + strategy = self.instance(name) + self.compressor['strategies'].append(strategy) if key == 'include': for config_file in key_values[key]: diff --git a/python/paddle/fluid/contrib/slim/core/pass_builder.py b/python/paddle/fluid/contrib/slim/core/pass_builder.py deleted file mode 100644 index fc1ddc94e0..0000000000 --- a/python/paddle/fluid/contrib/slim/core/pass_builder.py +++ /dev/null @@ -1,39 +0,0 @@ -# Copyright (c) 2018 PaddlePaddle Authors. All Rights Reserve. -# -# 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 .compress_pass import CompressPass -from .config import ConfigFactory - -__all__ = ['build_compressor'] - - -def build_compressor(place=None, - data_reader=None, - data_feeder=None, - scope=None, - metrics=None, - epoch=None, - config=None): - if config is not None: - factory = ConfigFactory(config) - comp_pass = factory.get_compress_pass() - else: - comp_pass = CompressPass() - comp_pass.place = place - comp_pass.data_reader = data_reader - comp_pass.data_feeder = data_feeder - comp_pass.scope = scope - comp_pass.metrics = metrics - comp_pass.epoch = epoch - return comp_pass diff --git a/python/paddle/fluid/contrib/slim/core/strategy.py b/python/paddle/fluid/contrib/slim/core/strategy.py index 74d98e98b0..28bf24f4e3 100644 --- a/python/paddle/fluid/contrib/slim/core/strategy.py +++ b/python/paddle/fluid/contrib/slim/core/strategy.py @@ -20,7 +20,7 @@ class Strategy(object): Base class for all strategies. """ - def __init__(self, start_epoch=0, end_epoch=10): + def __init__(self, start_epoch=0, end_epoch=0): """ Args: start_epoch: The first epoch to apply the strategy. @@ -29,7 +29,7 @@ class Strategy(object): self.start_epoch = start_epoch self.end_epoch = end_epoch - def on_compress_begin(self, context): + def on_compression_begin(self, context): pass def on_epoch_begin(self, context): @@ -44,5 +44,5 @@ class Strategy(object): def on_batch_end(self, context): pass - def on_compress_end(self, context): + def on_compression_end(self, context): pass diff --git a/python/paddle/fluid/contrib/slim/demo/filter_prune/config.yaml b/python/paddle/fluid/contrib/slim/demo/filter_prune/config.yaml deleted file mode 100644 index ea888fa2c7..0000000000 --- a/python/paddle/fluid/contrib/slim/demo/filter_prune/config.yaml +++ /dev/null @@ -1,28 +0,0 @@ -version: 1.0 -pruners: - pruner_1: - class: 'RatioPruner' - ratios: - 'conv1_1.w': 0.3 - 'conv1_2.w': 0.4 - '*': 0.9 - group_dims: - '*': [1, 2, 3] - criterions: - '*': 'l1-norm' -strategies: - strategy_1: - class: 'SensitivePruneStrategy' - pruner: 'pruner_1' - start_epoch: 0 - end_epoch: 10 - delta_rate: 0.20 - acc_loss_threshold: 0.2 - sensitivities: - 'conv1_1.w': 0.4 - -compress_pass: - class: 'CompressPass' - epoch: 100 - strategies: - - strategy_1 diff --git a/python/paddle/fluid/contrib/slim/demo/filter_prune/demo.py b/python/paddle/fluid/contrib/slim/demo/filter_prune/demo.py deleted file mode 100644 index 21c59c0c9d..0000000000 --- a/python/paddle/fluid/contrib/slim/demo/filter_prune/demo.py +++ /dev/null @@ -1,69 +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. - -import paddle.fluid as fluid -import paddle -import os -import sys -from paddle.fluid.contrib.slim import CompressPass -from paddle.fluid.contrib.slim import build_compressor -from paddle.fluid.contrib.slim import ImitationGraph - - -class LinearModel(object): - def __init__(slef): - pass - - def train(self): - train_program = fluid.Program() - startup_program = fluid.Program() - startup_program.random_seed = 10 - with fluid.program_guard(train_program, startup_program): - x = fluid.layers.data(name='x', shape=[13], dtype='float32') - y = fluid.layers.data(name='y', shape=[1], dtype='float32') - predict = fluid.layers.fc(input=x, size=1, act=None) - cost = fluid.layers.square_error_cost(input=predict, label=y) - avg_cost = fluid.layers.mean(cost) - eval_program = train_program.clone() - sgd_optimizer = fluid.optimizer.SGD(learning_rate=0.001) - sgd_optimizer.minimize(avg_cost) - - train_reader = paddle.batch( - paddle.dataset.uci_housing.train(), batch_size=1) - eval_reader = paddle.batch( - paddle.dataset.uci_housing.test(), batch_size=1) - place = fluid.CPUPlace() - train_feeder = fluid.DataFeeder(place=place, feed_list=[x, y]) - eval_feeder = fluid.DataFeeder(place=place, feed_list=[x, y]) - exe = fluid.Executor(place) - exe.run(startup_program) - train_metrics = {"loss": avg_cost.name} - eval_metrics = {"loss": avg_cost.name} - - graph = ImitationGraph(train_program) - config = './config.yaml' - comp_pass = build_compressor( - place, - data_reader=train_reader, - data_feeder=train_feeder, - scope=fluid.global_scope(), - metrics=train_metrics, - epoch=1, - config=config) - comp_pass.apply(graph) - - -if __name__ == "__main__": - model = LinearModel() - model.train() diff --git a/python/paddle/fluid/contrib/slim/graph/__init__.py b/python/paddle/fluid/contrib/slim/graph/__init__.py index d65472d193..c5d1c4dbdf 100644 --- a/python/paddle/fluid/contrib/slim/graph/__init__.py +++ b/python/paddle/fluid/contrib/slim/graph/__init__.py @@ -14,10 +14,7 @@ from . import executor from .executor import * -from . import graph -from .graph import * -from . import graph_pass -from .graph_pass import * +from . import graph_wrapper +from .graph_wrapper import * __all__ = executor.__all__ -__all__ += graph.__all__ -__all__ += graph_pass.__all__ +__all__ += graph_wrapper.__all__ diff --git a/python/paddle/fluid/contrib/slim/graph/executor.py b/python/paddle/fluid/contrib/slim/graph/executor.py index c02c3af820..70438a90eb 100644 --- a/python/paddle/fluid/contrib/slim/graph/executor.py +++ b/python/paddle/fluid/contrib/slim/graph/executor.py @@ -12,51 +12,46 @@ # See the License for the specific language governing permissions and # limitations under the License. -import abc -from abc import abstractmethod +from ....compiler import CompiledProgram +from ....data_feeder import DataFeeder from .... import executor -from .graph import IRGraph, ImitationGraph +from .graph_wrapper import GraphWrapper -__all__ = ['get_executor'] +__all__ = ['SlimGraphExecutor'] -class GraphExecutor(object): - __metaclass__ = abc.ABCMeta +class SlimGraphExecutor(object): + """ + Wrapper of executor used to run GraphWrapper. + """ def __init__(self, place): - self.place = place - - @abstractmethod - def run(self, graph, feches=None, feed=None): - pass - - -class IRGraphExecutor(GraphExecutor): - def run(self, grah, fetches, feed=None): - pass - - -class ImitationGraphExecutor(GraphExecutor): - def __init__(self, place): - super(ImitationGraphExecutor, self).__init__(place) self.exe = executor.Executor(place) + self.place = place - def run(self, graph, scope=None, fetches=None, feed=None): - assert isinstance(graph, ImitationGraph) - fetch_list = None - if fetches: - fetch_list = [ - graph.program.global_block().var(name) for name in fetches - ] - results = self.exe.run(graph.program, + def run(self, graph, scope, data=None): + """ + Runing a graph with a batch of data. + Args: + graph(GraphWrapper): The graph to be executed. + scope(fluid.core.Scope): The scope to be used. + data(list): A batch of data. Each tuple in this list is a sample. + It will feed the items of tuple to the in_nodes of graph. + Returns: + results(list): A list of result with the same order indicated by graph.out_nodes. + """ + assert isinstance(graph, GraphWrapper) + if data is not None: + feeder = DataFeeder( + feed_list=graph.in_nodes.values(), + place=self.place, + program=graph.program) + feed = feeder.feed(data) + + fetch_list = graph.out_nodes.values() + program = graph.compiled_graph if graph.compiled_graph else graph.program + results = self.exe.run(program, scope=scope, fetch_list=fetch_list, feed=feed) return results - - -def get_executor(graph, place): - if isinstance(graph, ImitationGraph): - return ImitationGraphExecutor(place) - if isinstance(graph, IRGraph): - return IRGraphExecutor(place) diff --git a/python/paddle/fluid/contrib/slim/graph/graph.py b/python/paddle/fluid/contrib/slim/graph/graph.py deleted file mode 100644 index f38d978341..0000000000 --- a/python/paddle/fluid/contrib/slim/graph/graph.py +++ /dev/null @@ -1,49 +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. -from __future__ import print_function -import os -import subprocess -from ....framework import Program -from ....framework import Block -from .... import core - -__all__ = ['Graph', 'ImitationGraph', 'IRGraph'] - - -class Graph(object): - """ - Base class for all graph. - """ - - def __init__(self): - pass - - def all_parameters(self): - """ - Return all the parameters in current graph. - """ - pass - - -class ImitationGraph(Graph): - def __init__(self, program=None): - super(ImitationGraph, self).__init__() - self.program = Program() if program is None else program - - def all_parameters(self): - return self.program.global_block().all_parameters() - - -class IRGraph(Graph): - pass diff --git a/python/paddle/fluid/contrib/slim/graph/graph_pass.py b/python/paddle/fluid/contrib/slim/graph/graph_pass.py deleted file mode 100644 index 1db6c4f110..0000000000 --- a/python/paddle/fluid/contrib/slim/graph/graph_pass.py +++ /dev/null @@ -1,42 +0,0 @@ -# Copyright (c) 2018 PaddlePaddle Authors. All Rights Reserve. -# -# 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. - -__all__ = ['GraphPass', 'PruneParameterPass'] - - -class GraphPass(object): - """ - Base class for all graph pass. - """ - - def __init__(self): - pass - - def apply(self, graph): - pass - - -class PruneParameterPass(GraphPass): - """ - Generate a graph for pruning parameters from target graph. - """ - - def __init__(self, pruned_params, thresholds): - super(PruneParameterPass, self).__init__() - self.pruned_params = pruned_params - self.thresholds = thresholds - self.default_threshold = thresholds['*'] - - def apply(self, graph): - pass diff --git a/python/paddle/fluid/contrib/slim/graph/graph_wrapper.py b/python/paddle/fluid/contrib/slim/graph/graph_wrapper.py new file mode 100644 index 0000000000..8694be7827 --- /dev/null +++ b/python/paddle/fluid/contrib/slim/graph/graph_wrapper.py @@ -0,0 +1,500 @@ +# 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. + +from collections import OrderedDict +from .... import io +from .... import compiler +from ....framework import Program +from ....framework import program_guard +from ....framework import Parameter +from ....framework import Variable +from ....executor import Executor +import copy +from collections import Iterable +from ....io import save_inference_model, load_inference_model, save_persistables +import numpy as np +import pickle +import os + +__all__ = ['GraphWrapper', 'VarWrapper', 'OpWrapper'] + +OPTIMIZER_OPS = [ + 'momentum', + 'lars_momentum', + 'adagrad', + 'adam', + 'adamax', + 'decayed_adagrad', + 'adadelta', + 'rmsprop', +] + + +class VarWrapper(object): + def __init__(self, var, graph): + assert isinstance(var, Variable) + assert isinstance(graph, GraphWrapper) + self._var = var + self._graph = graph + + def __eq__(self, v): + """ + Overwrite this function for ...in... syntax in python. + """ + return self._var.name == v._var.name + + def name(self): + """ + Get the name of the variable. + """ + return self._var.name + + def shape(self): + """ + Get the shape of the varibale. + """ + return self._var.shape + + def set_shape(self, shape): + """ + Set the shape of the variable. + """ + self._var.desc.set_shape(shape) + + def inputs(self): + """ + Get all the operators that use this variable as output. + Returns: + list: A list of operators. + """ + ops = [] + for op in self._graph.ops(): + if self in op.all_inputs(): + ops.append(op) + return ops + + def outputs(self): + """ + Get all the operators that use this variable as input. + Returns: + list: A list of operators. + """ + ops = [] + for op in self._graph.ops(): + if self in op.all_outputs(): + ops.append(op) + return ops + + +class OpWrapper(object): + def __init__(self, op, graph): + assert isinstance(graph, GraphWrapper) + self._op = op + self._graph = graph + + def __eq__(self, op): + """ + Overwrite this function for ...in... syntax in python. + """ + return self.idx() == op.idx() + + def all_inputs(self): + """ + Get all the input variables of this operator. + """ + return [ + self._graph.var(var_name) for var_name in self._op.input_arg_names + ] + + def all_outputs(self): + """ + Get all the output variables of this operator. + """ + return [ + self._graph.var(var_name) for var_name in self._op.output_arg_names + ] + + def idx(self): + """ + Get the id of this operator. + """ + return self._op.idx + + def type(self): + """ + Get the type of this operator. + """ + return self._op.type + + def is_bwd_op(self): + """ + Whether this operator is backward op. + """ + return self.type().endswith('_grad') + + def is_opt_op(self): + """ + Whether this operator is optimizer op. + """ + return self.type() in OPTIMIZER_OPS + + def inputs(self, name): + """ + Get all the varibales by the input name. + """ + return [self._graph.var(var_name) for var_name in self._op.input(name)] + + def outputs(self, name): + """ + Get all the varibales by the output name. + """ + return [self._graph.var(var_name) for var_name in self._op.output(name)] + + def set_attr(self, key, value): + """ + Set the value of attribute by attribute's name. + + Args: + key(str): the attribute name. + value(bool|int|str|float|list): the value of the attribute. + """ + self._op._set_attr(key, value) + + def attr(self, name): + """ + Get the attribute by name. + + Args: + name(str): the attribute name. + + Returns: + bool|int|str|float|list: The attribute value. The return value + can be any valid attribute type. + """ + return self._op.attr(name) + + +class GraphWrapper(object): + """ + It is a wrapper of paddle.fluid.framework.IrGraph with some special functions + for paddle slim framework. + """ + + def __init__(self, program=None, in_nodes=[], out_nodes=[]): + """ + Args: + program(framework.Program): A program with + in_nodes(dict): A dict to indicate the input nodes of the graph. + The key is user-defined and human-readable name. + The value is the name of Variable. + out_nodes(dict): A dict to indicate the input nodes of the graph. + The key is user-defined and human-readable name. + The value is the name of Variable. + """ + super(GraphWrapper, self).__init__() + self.program = Program() if program is None else program + self.compiled_graph = None + self.in_nodes = OrderedDict(in_nodes) + self.out_nodes = OrderedDict(out_nodes) + self._attrs = OrderedDict() + + def all_parameters(self): + """ + Get all the parameters in this graph. + Returns: + list: A list of VarWrapper instances. + """ + params = [] + for block in self.program.blocks: + for param in block.all_parameters(): + params.append(VarWrapper(param, self)) + return params + + def is_parameter(self, var): + """ + Whether the given variable is parameter. + Args: + var(VarWrapper): The given varibale. + """ + return isinstance(var._var, Parameter) + + def is_persistable(self, var): + """ + Whether the given variable is persistable. + Args: + var(VarWrapper): The given varibale. + """ + return var._var.persistable + + def compile(self, for_parallel=True, for_test=False): + """ + Compile the program in this wrapper to framework.CompiledProgram for next running. + This function must be called if the program is modified. + Args: + for_parallel(bool): Whether the program to run in data parallel way. default: True. + for_test(bool): Whether the compiled program is used for test. + """ + target = self.program + if for_test: + loss = None + else: + loss = self.out_nodes['loss'] + if for_parallel: + # disable memory optimize for stable training + build_strategy = compiler.BuildStrategy() + build_strategy.enable_inplace = False + build_strategy.memory_optimize = False + self.compiled_graph = compiler.CompiledProgram( + target).with_data_parallel( + loss_name=loss, build_strategy=build_strategy) + else: + self.compiled_graph = compiler.CompiledProgram(target) + + def ops(self): + """ + Return all operator nodes included in the graph as a set. + """ + ops = [] + for block in self.program.blocks: + for op in block.ops: + ops.append(OpWrapper(op, self)) + return ops + + def vars(self): + """ + Get all the variables. + """ + return [VarWrapper(var, self) for var in self.program.list_vars()] + + def var(self, name): + """ + Get the variable by variable name. + """ + return VarWrapper(self.program.global_block().var(name), self) + + def clone(self, for_test=False): + """ + Clone a new graph from current graph. + Returns: + (GraphWrapper): The wrapper of a new graph. + """ + return GraphWrapper( + self.program.clone(for_test), + copy.deepcopy(self.in_nodes), copy.deepcopy(self.out_nodes)) + + def merge(self, graph): + """ + Merge a graph into current graph. + Args: + graph(GraphWrapper): The graph to be merged by current graph. + """ + for var in graph.program.list_vars(): + self.program.global_block()._clone_variable(var) + # TODO: parameters should be cloned + for op in graph.ops(): + op = op._op + inputs = {} + outputs = {} + attrs = {} + for input_name in op.input_names: + inputs[input_name] = [ + self.var(in_var_name) + for in_var_name in op.inputs(input_name) + ] + for output_name in op.output_names: + outputs[output_name] = [ + self.var(out_var_name) + for out_var_name in op.output(output_name) + ] + for attr_name in op.attr_names: + attrs[attr_name] = op.attr(attr_name) + self.program.global_block().append_op( + type=op.type, inputs=inputs, outputs=outputs, attrs=attrs) + + def program(self): + """ + Get the program in current wrapper. + """ + return self.program + + def pre_ops(self, op): + """ + Get all the previous operators of target operator. + Args: + op(OpWrapper): Target operator.. + Returns: + list: A list of operators. + """ + ops = [] + for p in self.ops(): + for in_var in op.all_inputs(): + if in_var in p.all_outputs(): + ops.append(p) + return ops + + def next_ops(self, op): + """ + Get all the next operators of target operator. + Args: + op(OpWrapper): Target operator.. + Returns: + list: A list of operators. + """ + ops = [] + for p in self.ops(): + for out_var in op.all_outputs(): + if out_var in p.all_inputs(): + ops.append(p) + return ops + + def get_param_by_op(self, op): + """ + Get the parameters used by target operator. + """ + assert isinstance(op, OpWrapper) + params = [] + for var in op.all_inputs(): + if isinstance(var._var, Parameter): + params.append(var) + assert len(params) > 0 + return params + + def numel_params(self): + """ + Get the number of elements in all parameters. + """ + ret = 0 + for param in self.all_parameters(): + ret += np.product(param.shape()) + return ret + + def get_optimize_graph(self, optimizer, place, scope, no_grad_var_names=[]): + """ + Get a new graph for training by appending some backward operators and optimization operators. + Args: + optimizer: The optimzier used to generate training graph. + place: The place to run the graph. + scope: The scope used to run the graph. Some new variable will be added into this scope. + no_grad_var_names(list): Names of variables that should be ignored while computing gradients. default: []. + Returns: + (GraphWrapper): The wrapper of new graph with backward ops and optimization ops. + """ + graph = self.clone() + startup_program = Program() + with program_guard( + main_program=graph.program, startup_program=startup_program): + target_name = None + if 'loss' in graph.out_nodes: + target_name = graph.out_nodes['loss'] + elif 'cost' in graph.out_nodes: + target_name = graph.out_nodes['cost'] + target = graph.var(target_name)._var + optimizer.minimize(target, no_grad_set=no_grad_var_names) + + exe = Executor(place) + exe.run(program=startup_program, scope=scope) + return graph + + def flops(self, only_conv=False): + """ + Get the flops of current graph. + Args: + only_conv: Only calculating the conv layers. default: False. + Returns: + int: The flops of current graph. + """ + flops = 0 + for op in self.ops(): + if op.type() in ['conv2d', 'depthwise_conv2d']: + filter_shape = op.inputs("Filter")[0].shape() + input_shape = op.inputs("Input")[0].shape() + output_shape = op.outputs("Output")[0].shape() + c_out, c_in, k_h, k_w = filter_shape + _, _, h_out, w_out = output_shape + groups = op.attr("groups") + kernel_ops = k_h * k_w * (c_in / groups) + if len(op.inputs("Bias")) > 0: + with_bias = 1 + else: + with_bias = 0 + flops += 2 * h_out * w_out * c_out * (kernel_ops + with_bias) + elif op.type() == 'pool2d' and not only_conv: + input_shape = op.inputs("X")[0].shape() + output_shape = op.outputs("Out")[0].shape() + _, c_out, h_out, w_out = output_shape + k_size = op.attr("ksize") + flops += h_out * w_out * c_out * (k_size[0]**2) + + elif op.type() == 'mul' and not only_conv: + x_shape = list(op.inputs("X")[0].shape()) + y_shape = op.inputs("Y")[0].shape() + if x_shape[0] == -1: + x_shape[0] = 1 + flops += 2 * x_shape[0] * x_shape[1] * y_shape[1] + + elif op.type() in ['relu', 'sigmoid', 'batch_norm' + ] and not only_conv: + input_shape = list(op.inputs("X")[0].shape()) + if input_shape[0] == -1: + input_shape[0] = 1 + flops += np.product(input_shape) + + return flops + + def save_persistables(self, path, exe): + """ + Save all the persistable variables into file. + Args: + path(str): The path to save the persistables. + exe(framework.Executor): The executor used to save the persistables. + """ + io.save_persistables(exe.exe, path, main_program=self.program) + + def load_persistables(self, path, exe): + """ + Load the persistable variables from file. + Args: + path(str): The path to load the persistables. + exe(framework.Executor): The executor used to load the persistables. + """ + + def if_exist(var): + return os.path.exists(os.path.join(path, var.name)) + + io.load_vars( + exe.exe, path, main_program=self.program, predicate=if_exist) + + def update_param_shape(self, scope): + """ + Update the shape of parameters in the graph according to tensors in scope. + It is used after loading pruned parameters from file. + """ + for param in self.all_parameters(): + tensor_shape = np.array(scope.find_var(param.name()).get_tensor( + )).shape + param.set_shape(tensor_shape) + + def infer_shape(self): + """ + Update the groups of convolution layer according to current filters. + It is used after loading pruned parameters from file. + """ + for op in self.ops(): + if op.type() != 'conditional_block': + op._op.desc.infer_shape(op._op.block.desc) + + def update_groups_of_conv(self): + for op in self.ops(): + if op.type() == 'depthwise_conv2d': + op.set_attr('groups', op.inputs('Filter')[0].shape()[0]) diff --git a/python/paddle/fluid/contrib/slim/prune/prune_strategy.py b/python/paddle/fluid/contrib/slim/prune/prune_strategy.py index 34c5107daa..7a25c3a61e 100644 --- a/python/paddle/fluid/contrib/slim/prune/prune_strategy.py +++ b/python/paddle/fluid/contrib/slim/prune/prune_strategy.py @@ -13,54 +13,919 @@ # limitations under the License. from ..core.strategy import Strategy -from ....framework import Program, program_guard +from ..graph import VarWrapper, OpWrapper, GraphWrapper +from ....framework import Program, program_guard, Parameter from .... import layers +import prettytable as pt import numpy as np +from scipy.optimize import leastsq +import copy +import re +import os +import pickle +import logging +import sys -__all__ = ['SensitivePruneStrategy', 'PruneStrategy'] +__all__ = ['SensitivePruneStrategy', 'UniformPruneStrategy'] +logging.basicConfig(format='%(asctime)s-%(levelname)s: %(message)s') +_logger = logging.getLogger(__name__) +_logger.setLevel(logging.INFO) + + +class PruneStrategy(Strategy): + """ + The base class of all pruning strategies. + """ -class SensitivePruneStrategy(Strategy): def __init__(self, pruner=None, start_epoch=0, - end_epoch=10, - delta_rate=0.20, - acc_loss_threshold=0.2, - sensitivities=None): - super(SensitivePruneStrategy, self).__init__(start_epoch, end_epoch) + end_epoch=0, + target_ratio=0.5, + metric_name=None, + pruned_params='conv.*_weights'): + """ + Args: + pruner(slim.Pruner): The pruner used to prune the parameters. + start_epoch(int): The 'on_epoch_begin' function will be called in start_epoch. default: 0 + end_epoch(int): The 'on_epoch_end' function will be called in end_epoch. default: 0 + target_ratio(float): The flops ratio to be pruned from current model. + metric_name(str): The metric used to evaluate the model. + It should be one of keys in out_nodes of graph wrapper. + pruned_params(str): The pattern str to match the parameter names to be pruned. + """ + super(PruneStrategy, self).__init__(start_epoch, end_epoch) self.pruner = pruner - self.delta_rate = delta_rate - self.acc_loss_threshold = acc_loss_threshold - self.sensitivities = sensitivities + self.target_ratio = target_ratio + self.metric_name = metric_name + self.pruned_params = pruned_params + self.pruned_list = [] + self.backup = {} + self.param_shape_backup = {} + def _eval_graph(self, context, sampled_rate=None, cached_id=0): + """ + Evaluate the current mode in context. + Args: + context(slim.core.Context): The context storing all information used to evaluate the current model. + sampled_rate(float): The sampled rate used to sample partial data for evaluation. None means using all data in eval_reader. default: None. + cached_id(int): The id of dataset sampled. Evaluations with same cached_id use the same sampled dataset. default: 0. + """ + results, names = context.run_eval_graph(sampled_rate, cached_id) + metric = np.mean(results[list(names).index(self.metric_name)]) + return metric -class PruneStrategy(Strategy): + def _prune_filters_by_ratio(self, + scope, + params, + ratio, + place, + lazy=False, + only_graph=False): + """ + Pruning filters by given ratio. + Args: + scope(fluid.core.Scope): The scope used to pruning filters. + params(list): A list of filter parameters. + ratio(float): The ratio to be pruned. + place(fluid.Place): The device place of filter parameters. + lazy(bool): True means setting the pruned elements to zero. + False means cutting down the pruned elements. + only_graph(bool): True means only modifying the graph. + False means modifying graph and variables in scope. + """ + if params[0].name() in self.pruned_list[0]: + return + param_t = scope.find_var(params[0].name()).get_tensor() + pruned_idx = self.pruner.cal_pruned_idx( + params[0].name(), np.array(param_t), ratio, axis=0) + for param in params: + assert isinstance(param, VarWrapper) + param_t = scope.find_var(param.name()).get_tensor() + if lazy: + self.backup[param.name()] = copy.deepcopy(np.array(param_t)) + pruned_param = self.pruner.prune_tensor( + np.array(param_t), pruned_idx, pruned_axis=0, lazy=lazy) + if not only_graph: + param_t.set(pruned_param, place) + ori_shape = param.shape() + if param.name() not in self.param_shape_backup: + self.param_shape_backup[param.name()] = copy.deepcopy( + param.shape()) + new_shape = list(param.shape()) + new_shape[0] = pruned_param.shape[0] + param.set_shape(new_shape) + _logger.debug( + '|----------------------------------------+----+------------------------------+------------------------------|' + ) + _logger.debug('|{:^40}|{:^4}|{:^30}|{:^30}|'.format( + str(param.name()), str(0), str(ori_shape), str(param.shape()))) + self.pruned_list[0].append(param.name()) + return pruned_idx + + def _prune_parameter_by_idx(self, + scope, + params, + pruned_idx, + pruned_axis, + place, + lazy=False, + only_graph=False): + """ + Pruning parameters in given axis. + Args: + scope(fluid.core.Scope): The scope storing paramaters to be pruned. + params(VarWrapper): The parameter to be pruned. + pruned_idx(list): The index of elements to be pruned. + pruned_axis(int): The pruning axis. + place(fluid.Place): The device place of filter parameters. + lazy(bool): True means setting the pruned elements to zero. + False means cutting down the pruned elements. + only_graph(bool): True means only modifying the graph. + False means modifying graph and variables in scope. + """ + if params[0].name() in self.pruned_list[pruned_axis]: + return + for param in params: + assert isinstance(param, VarWrapper) + param_t = scope.find_var(param.name()).get_tensor() + if lazy: + self.backup[param.name()] = copy.deepcopy(np.array(param_t)) + pruned_param = self.pruner.prune_tensor( + np.array(param_t), pruned_idx, pruned_axis, lazy=lazy) + if not only_graph: + param_t.set(pruned_param, place) + ori_shape = param.shape() + if param.name() not in self.param_shape_backup: + self.param_shape_backup[param.name()] = copy.deepcopy( + param.shape()) + new_shape = list(param.shape()) + new_shape[pruned_axis] = pruned_param.shape[pruned_axis] + param.set_shape(new_shape) + _logger.debug( + '|----------------------------------------+----+------------------------------+------------------------------|' + ) + _logger.debug('|{:^40}|{:^4}|{:^30}|{:^30}|'.format( + str(param.name()), + str(pruned_axis), str(ori_shape), str(param.shape()))) + self.pruned_list[pruned_axis].append(param.name()) + + def _forward_search_related_op(self, graph, param): + """ + Forward search operators that will be affected by pruning of param. + Args: + graph(GraphWrapper): The graph to be searched. + param(VarWrapper): The current pruned parameter. + Returns: + list: A list of operators. + """ + assert isinstance(param, VarWrapper) + visited = {} + for op in graph.ops(): + visited[op.idx()] = False + stack = [] + for op in graph.ops(): + if (not op.is_bwd_op()) and (param in op.all_inputs()): + stack.append(op) + visit_path = [] + while len(stack) > 0: + top_op = stack[len(stack) - 1] + if visited[top_op.idx()] == False: + visit_path.append(top_op) + visited[top_op.idx()] = True + next_ops = None + if top_op.type() == "conv2d" and param not in top_op.all_inputs(): + next_ops = None + elif top_op.type() == "mul": + next_ops = None + else: + next_ops = self._get_next_unvisited_op(graph, visited, top_op) + if next_ops == None: + stack.pop() + else: + stack += next_ops + return visit_path + + def _get_next_unvisited_op(self, graph, visited, top_op): + """ + Get next unvisited adjacent operators of given operators. + Args: + graph(GraphWrapper): The graph used to search. + visited(list): The ids of operators that has been visited. + top_op: The given operator. + Returns: + list: A list of operators. + """ + assert isinstance(top_op, OpWrapper) + next_ops = [] + for op in graph.next_ops(top_op): + if (visited[op.idx()] == False) and (not op.is_bwd_op()): + next_ops.append(op) + return next_ops if len(next_ops) > 0 else None + + def _get_accumulator(self, graph, param): + """ + Get accumulators of given parameter. The accumulator was created by optimizer. + Args: + graph(GraphWrapper): The graph used to search. + param(VarWrapper): The given parameter. + Returns: + list: A list of accumulators which are variables. + """ + assert isinstance(param, VarWrapper) + params = [] + for op in param.outputs(): + if op.is_opt_op(): + for out_var in op.all_outputs(): + if graph.is_persistable(out_var) and out_var.name( + ) != param.name(): + params.append(out_var) + return params + + def _forward_pruning_ralated_params(self, + graph, + scope, + param, + place, + ratio=None, + pruned_idxs=None, + lazy=False, + only_graph=False): + """ + Pruning all the parameters affected by the pruning of given parameter. + Args: + graph(GraphWrapper): The graph to be searched. + scope(fluid.core.Scope): The scope storing paramaters to be pruned. + param(VarWrapper): The given parameter. + place(fluid.Place): The device place of filter parameters. + ratio(float): The target ratio to be pruned. + pruned_idx(list): The index of elements to be pruned. + lazy(bool): True means setting the pruned elements to zero. + False means cutting down the pruned elements. + only_graph(bool): True means only modifying the graph. + False means modifying graph and variables in scope. + """ + assert isinstance( + graph, + GraphWrapper), "graph must be instance of slim.core.GraphWrapper" + assert isinstance( + param, VarWrapper), "param must be instance of slim.core.VarWrapper" + + if param.name() in self.pruned_list[0]: + return + related_ops = self._forward_search_related_op(graph, param) + + if ratio is None: + assert pruned_idxs is not None + self._prune_parameter_by_idx( + scope, [param] + self._get_accumulator(graph, param), + pruned_idxs, + pruned_axis=0, + place=place, + lazy=lazy, + only_graph=only_graph) + + else: + pruned_idxs = self._prune_filters_by_ratio( + scope, [param] + self._get_accumulator(graph, param), + ratio, + place, + lazy=lazy, + only_graph=only_graph) + corrected_idxs = pruned_idxs[:] + + for idx, op in enumerate(related_ops): + if op.type() == "conv2d" and (param not in op.all_inputs()): + for in_var in op.all_inputs(): + if graph.is_parameter(in_var): + conv_param = in_var + self._prune_parameter_by_idx( + scope, [conv_param] + self._get_accumulator( + graph, conv_param), + corrected_idxs, + pruned_axis=1, + place=place, + lazy=lazy, + only_graph=only_graph) + if op.type() == "depthwise_conv2d": + for in_var in op.all_inputs(): + if graph.is_parameter(in_var): + conv_param = in_var + self._prune_parameter_by_idx( + scope, [conv_param] + self._get_accumulator( + graph, conv_param), + corrected_idxs, + pruned_axis=0, + place=place, + lazy=lazy, + only_graph=only_graph) + elif op.type() == "elementwise_add": + # pruning bias + for in_var in op.all_inputs(): + if graph.is_parameter(in_var): + bias_param = in_var + self._prune_parameter_by_idx( + scope, [bias_param] + self._get_accumulator( + graph, bias_param), + pruned_idxs, + pruned_axis=0, + place=place, + lazy=lazy, + only_graph=only_graph) + elif op.type() == "mul": # pruning fc layer + fc_input = None + fc_param = None + for in_var in op.all_inputs(): + if graph.is_parameter(in_var): + fc_param = in_var + else: + fc_input = in_var + + idx = [] + feature_map_size = fc_input.shape()[2] * fc_input.shape()[3] + range_idx = np.array(range(feature_map_size)) + for i in corrected_idxs: + idx += list(range_idx + i * feature_map_size) + corrected_idxs = idx + self._prune_parameter_by_idx( + scope, [fc_param] + self._get_accumulator(graph, fc_param), + corrected_idxs, + pruned_axis=0, + place=place, + lazy=lazy, + only_graph=only_graph) + + elif op.type() == "concat": + concat_inputs = op.all_inputs() + last_op = related_ops[idx - 1] + for out_var in last_op.all_outputs(): + if out_var in concat_inputs: + concat_idx = concat_inputs.index(out_var) + offset = 0 + for ci in range(concat_idx): + offset += concat_inputs[ci].shape()[1] + corrected_idxs = [x + offset for x in pruned_idxs] + elif op.type() == "batch_norm": + bn_inputs = op.all_inputs() + mean = bn_inputs[2] + variance = bn_inputs[3] + alpha = bn_inputs[0] + beta = bn_inputs[1] + self._prune_parameter_by_idx( + scope, [mean] + self._get_accumulator(graph, mean), + corrected_idxs, + pruned_axis=0, + place=place, + lazy=lazy, + only_graph=only_graph) + self._prune_parameter_by_idx( + scope, [variance] + self._get_accumulator(graph, variance), + corrected_idxs, + pruned_axis=0, + place=place, + lazy=lazy, + only_graph=only_graph) + self._prune_parameter_by_idx( + scope, [alpha] + self._get_accumulator(graph, alpha), + corrected_idxs, + pruned_axis=0, + place=place, + lazy=lazy, + only_graph=only_graph) + self._prune_parameter_by_idx( + scope, [beta] + self._get_accumulator(graph, beta), + corrected_idxs, + pruned_axis=0, + place=place, + lazy=lazy, + only_graph=only_graph) + + def _prune_parameters(self, + graph, + scope, + params, + ratios, + place, + lazy=False, + only_graph=False): + """ + Pruning the given parameters. + Args: + graph(GraphWrapper): The graph to be searched. + scope(fluid.core.Scope): The scope storing paramaters to be pruned. + params(list): A list of parameter names to be pruned. + ratios(list): A list of ratios to be used to pruning parameters. + place(fluid.Place): The device place of filter parameters. + pruned_idx(list): The index of elements to be pruned. + lazy(bool): True means setting the pruned elements to zero. + False means cutting down the pruned elements. + only_graph(bool): True means only modifying the graph. + False means modifying graph and variables in scope. + + """ + _logger.debug('\n################################') + _logger.debug('# pruning parameters #') + _logger.debug('################################\n') + _logger.debug( + '|----------------------------------------+----+------------------------------+------------------------------|' + ) + _logger.debug('|{:^40}|{:^4}|{:^30}|{:^30}|'.format('parameter', 'axis', + 'from', 'to')) + assert len(params) == len(ratios) + self.pruned_list = [[], []] + for param, ratio in zip(params, ratios): + assert isinstance(param, str) or isinstance(param, unicode) + param = graph.var(param) + self._forward_pruning_ralated_params( + graph, + scope, + param, + place, + ratio=ratio, + lazy=lazy, + only_graph=only_graph) + ops = param.outputs() + for op in ops: + if op.type() == 'conv2d': + brother_ops = self._search_brother_ops(graph, op) + for broher in brother_ops: + for p in graph.get_param_by_op(broher): + self._forward_pruning_ralated_params( + graph, + scope, + p, + place, + ratio=ratio, + lazy=lazy, + only_graph=only_graph) + _logger.debug( + '|----------------------------------------+----+------------------------------+------------------------------|' + ) + + def _search_brother_ops(self, graph, op_node): + """ + Search brother operators that was affected by pruning of given operator. + Args: + graph(GraphWrapper): The graph to be searched. + op_node(OpWrapper): The start node for searching. + Returns: + list: A list of operators. + """ + visited = [op_node.idx()] + stack = [] + brothers = [] + for op in graph.next_ops(op_node): + if (op.type() != 'conv2d') and (op.type() != 'fc') and ( + not op._is_bwd_op()): + stack.append(op) + visited.append(op.idx()) + while len(stack) > 0: + top_op = stack.pop() + for parent in graph.pre_ops(top_op): + if parent.idx() not in visited and (not parent._is_bwd_op()): + if ((parent.type == 'conv2d') or (parent.type == 'fc')): + brothers.append(parent) + else: + stack.append(parent) + visited.append(parent.idx()) + + for child in graph.next_ops(top_op): + if (child.type != 'conv2d') and (child.type != 'fc') and ( + child.idx() not in visited) and ( + not child._is_bwd_op()): + stack.append(child) + visited.append(child.idx()) + return brothers + + def _prune_graph(self, graph, target_graph): + """ + Pruning parameters of graph according to target graph. + Args: + graph(GraphWrapper): The graph to be pruned. + target_graph(GraphWrapper): The reference graph. + Return: None + """ + count = 1 + _logger.debug( + '|----+----------------------------------------+------------------------------+------------------------------|' + ) + _logger.debug('|{:^4}|{:^40}|{:^30}|{:^30}|'.format('id', 'parammeter', + 'from', 'to')) + for param in target_graph.all_parameters(): + var = graph.var(param.name()) + ori_shape = var.shape() + var.set_shape(param.shape()) + _logger.debug( + '|----+----------------------------------------+------------------------------+------------------------------|' + ) + _logger.debug('|{:^4}|{:^40}|{:^30}|{:^30}|'.format( + str(count), + str(param.name()), str(ori_shape), str(param.shape()))) + count += 1 + _logger.debug( + '|----+----------------------------------------+------------------------------+------------------------------|' + ) + + +class UniformPruneStrategy(PruneStrategy): """ - The strategy that pruning weights by threshold or ratio iteratively. + The uniform pruning strategy. The parameters will be pruned by uniform ratio. """ def __init__(self, - pruner, - mini_batch_pruning_frequency=1, + pruner=None, start_epoch=0, - end_epoch=10): - super(PruneStrategy, self).__init__(start_epoch, end_epoch) - self.pruner = pruner - self.mini_batch_pruning_frequency = mini_batch_pruning_frequency - - def _triger(self, context): - return (context.batch_id % self.mini_batch_pruning_frequency == 0 and - self.start_epoch <= context.epoch_id < self.end_epoch) - - def on_batch_end(self, context): - if self._triger(context): - prune_program = Program() - with program_guard(prune_program): - for param in context.graph.all_parameters(): - prune_program.global_block().clone_variable(param) - p = prune_program.global_block().var(param.name) - zeros_mask = self.pruner.prune(p) - pruned_param = p * zeros_mask - layers.assign(input=pruned_param, output=param) - context.program_exe.run(prune_program, scope=context.scope) + end_epoch=0, + target_ratio=0.5, + metric_name=None, + pruned_params='conv.*_weights'): + """ + Args: + pruner(slim.Pruner): The pruner used to prune the parameters. + start_epoch(int): The 'on_epoch_begin' function will be called in start_epoch. default: 0 + end_epoch(int): The 'on_epoch_end' function will be called in end_epoch. default: 0 + target_ratio(float): The flops ratio to be pruned from current model. + metric_name(str): The metric used to evaluate the model. + It should be one of keys in out_nodes of graph wrapper. + pruned_params(str): The pattern str to match the parameter names to be pruned. + """ + super(UniformPruneStrategy, self).__init__(pruner, start_epoch, + end_epoch, target_ratio, + metric_name, pruned_params) + + def _get_best_ratios(self, context): + """ + Search a group of ratios for pruning target flops. + """ + _logger.info('_get_best_ratios') + pruned_params = [] + for param in context.eval_graph.all_parameters(): + if re.match(self.pruned_params, param.name()): + pruned_params.append(param.name()) + + min_ratio = 0. + max_ratio = 1. + + flops = context.eval_graph.flops() + model_size = context.eval_graph.numel_params() + + while min_ratio < max_ratio: + ratio = (max_ratio + min_ratio) / 2 + _logger.debug( + '-----------Try pruning ratio: {:.2f}-----------'.format(ratio)) + ratios = [ratio] * len(pruned_params) + self._prune_parameters( + context.eval_graph, + context.scope, + pruned_params, + ratios, + context.place, + only_graph=True) + + pruned_flops = 1 - (float(context.eval_graph.flops()) / flops) + pruned_size = 1 - (float(context.eval_graph.numel_params()) / + model_size) + _logger.debug('Pruned flops: {:.2f}'.format(pruned_flops)) + _logger.debug('Pruned model size: {:.2f}'.format(pruned_size)) + for param in self.param_shape_backup.keys(): + context.eval_graph.var(param).set_shape(self.param_shape_backup[ + param]) + self.param_shape_backup = {} + + if abs(pruned_flops - self.target_ratio) < 1e-2: + break + if pruned_flops > self.target_ratio: + max_ratio = ratio + else: + min_ratio = ratio + _logger.info('Get ratios: {}'.format([round(r, 2) for r in ratios])) + return pruned_params, ratios + + def on_epoch_begin(self, context): + if context.epoch_id == self.start_epoch: + params, ratios = self._get_best_ratios(context) + + self._prune_parameters(context.optimize_graph, context.scope, + params, ratios, context.place) + + model_size = context.eval_graph.numel_params() + flops = context.eval_graph.flops() + _logger.debug('\n################################') + _logger.debug('# pruning eval graph #') + _logger.debug('################################\n') + self._prune_graph(context.eval_graph, context.optimize_graph) + context.optimize_graph.update_groups_of_conv() + context.eval_graph.update_groups_of_conv() + + _logger.info( + '------------------finish pruning--------------------------------' + ) + _logger.info('Pruned size: {:.2f}'.format(1 - (float( + context.eval_graph.numel_params()) / model_size))) + _logger.info('Pruned flops: {:.2f}'.format(1 - (float( + context.eval_graph.flops()) / flops))) + # metric = self._eval_graph(context) + # _logger.info('Metric after pruning: {:.2f}'.format(metric)) + _logger.info( + '------------------UniformPruneStrategy.on_compression_begin finish--------------------------------' + ) + + +class SensitivePruneStrategy(PruneStrategy): + """ + Sensitive pruning strategy. Different pruned ratio was applied on each layer. + """ + + def __init__(self, + pruner=None, + start_epoch=0, + end_epoch=0, + delta_rate=0.20, + target_ratio=0.5, + metric_name='top1_acc', + pruned_params='conv.*_weights', + sensitivities_file='./sensitivities.data', + sensitivities={}, + num_steps=1, + eval_rate=None): + """ + Args: + pruner(slim.Pruner): The pruner used to prune the parameters. + start_epoch(int): The 'on_epoch_begin' function will be called in start_epoch. default: 0. + end_epoch(int): The 'on_epoch_end' function will be called in end_epoch. default: 10. + delta_rate(float): The delta used to generate ratios when calculating sensitivities. default: 0.2 + target_ratio(float): The flops ratio to be pruned from current model. default: 0.5 + metric_name(str): The metric used to evaluate the model. + It should be one of keys in out_nodes of graph wrapper. default: 'top1_acc' + pruned_params(str): The pattern str to match the parameter names to be pruned. default: 'conv.*_weights'. + sensitivities_file(str): The sensitivities file. default: './sensitivities.data' + sensitivities(dict): The user-defined sensitivities. default: {}. + num_steps(int): The number of pruning steps. default: 1. + eval_rate(float): The rate of sampled data used to calculate sensitivities. + None means using all the data. default: None. + """ + super(SensitivePruneStrategy, self).__init__(pruner, start_epoch, + end_epoch, target_ratio, + metric_name, pruned_params) + self.delta_rate = delta_rate + self.pruned_list = [] + self.sensitivities = sensitivities + self.sensitivities_file = sensitivities_file + self.backup = {} + self.param_shape_backup = {} + self.num_steps = num_steps + self.eval_rate = eval_rate + self.pruning_step = 1 - pow((1 - target_ratio), 1.0 / self.num_steps) + + def _save_sensitivities(self, sensitivities, sensitivities_file): + """ + Save sensitivities into file. + """ + with open(sensitivities_file, 'wb') as f: + pickle.dump(sensitivities, f) + + def _load_sensitivities(self, sensitivities_file): + """ + Load sensitivities from file. + """ + sensitivities = {} + if sensitivities_file and os.path.exists(sensitivities_file): + with open(sensitivities_file, 'rb') as f: + if sys.version_info < (3, 0): + sensitivities = pickle.load(f) + else: + sensitivities = pickle.load(f, encoding='bytes') + + for param in sensitivities: + sensitivities[param]['pruned_percent'] = [ + round(p, 2) for p in sensitivities[param]['pruned_percent'] + ] + self._format_sensitivities(sensitivities) + return sensitivities + + def _format_sensitivities(self, sensitivities): + """ + Print formated sensitivities in debug log level. + """ + tb = pt.PrettyTable() + tb.field_names = ["parameter", "size"] + [ + str(round(i, 2)) + for i in np.arange(self.delta_rate, 1, self.delta_rate) + ] + for param in sensitivities: + if len(sensitivities[param]['loss']) == (len(tb.field_names) - 2): + tb.add_row([param, sensitivities[param]['size']] + [ + round(loss, 2) for loss in sensitivities[param]['loss'] + ]) + _logger.debug('\n################################') + _logger.debug('# sensitivities table #') + _logger.debug('################################\n') + _logger.debug(tb) + + def _compute_sensitivities(self, context): + """ + Computing the sensitivities of all parameters. + """ + _logger.info("calling _compute_sensitivities.") + self.param_shape_backup = {} + self.backup = {} + cached_id = np.random.randint(1000) + if self.start_epoch == context.epoch_id: + sensitivities_file = self.sensitivities_file + else: + sensitivities_file = self.sensitivities_file + ".epoch" + str( + context.epoch_id) + sensitivities = self._load_sensitivities(sensitivities_file) + + for param in context.eval_graph.all_parameters(): + if not re.match(self.pruned_params, param.name()): + continue + if param.name() not in sensitivities: + sensitivities[param.name()] = { + 'pruned_percent': [], + 'loss': [], + 'size': param.shape()[0] + } + + metric = None + + for param in sensitivities.keys(): + ratio = self.delta_rate + while ratio < 1: + ratio = round(ratio, 2) + if ratio in sensitivities[param]['pruned_percent']: + _logger.debug('{}, {} has computed.'.format(param, ratio)) + ratio += self.delta_rate + continue + if metric is None: + metric = self._eval_graph(context, self.eval_rate, + cached_id) + # prune parameter by ratio + self._prune_parameters( + context.eval_graph, + context.scope, [param], [ratio], + context.place, + lazy=True) + self.pruned_list[0] + # get accuracy after pruning and update self.sensitivities + pruned_metric = self._eval_graph(context, self.eval_rate, + cached_id) + loss = metric - pruned_metric + _logger.info("pruned param: {}; {}; loss={}".format( + param, ratio, loss)) + for brother in self.pruned_list[0]: + if re.match(self.pruned_params, brother): + if brother not in sensitivities: + sensitivities[brother] = { + 'pruned_percent': [], + 'loss': [] + } + sensitivities[brother]['pruned_percent'].append(ratio) + sensitivities[brother]['loss'].append(loss) + + self._save_sensitivities(sensitivities, sensitivities_file) + + # restore pruned parameters + for param_name in self.backup.keys(): + param_t = context.scope.find_var(param_name).get_tensor() + param_t.set(self.backup[param_name], context.place) + +# pruned_metric = self._eval_graph(context) + self.backup = {} + + ratio += self.delta_rate + return sensitivities + + def _get_best_ratios(self, context, sensitivities, target_ratio): + """ + Search a group of ratios for pruning target flops. + """ + _logger.info('_get_best_ratios for pruning ratie: {}'.format( + target_ratio)) + self.param_shape_backup = {} + self.backup = {} + + def func(params, x): + a, b, c, d = params + return a * x * x * x + b * x * x + c * x + d + + def error(params, x, y): + return func(params, x) - y + + def slove_coefficient(x, y): + init_coefficient = [10, 10, 10, 10] + coefficient, loss = leastsq(error, init_coefficient, args=(x, y)) + return coefficient + + min_loss = 0. + max_loss = 0. + + # step 1: fit curve by sensitivities + coefficients = {} + for param in sensitivities: + losses = np.array([0] * 5 + sensitivities[param]['loss']) + precents = np.array([0] * 5 + sensitivities[param][ + 'pruned_percent']) + coefficients[param] = slove_coefficient(precents, losses) + loss = np.max(losses) + max_loss = np.max([max_loss, loss]) + + # step 2: Find a group of ratios by binary searching. + flops = context.eval_graph.flops() + model_size = context.eval_graph.numel_params() + ratios = [] + while min_loss < max_loss: + loss = (max_loss + min_loss) / 2 + _logger.info( + '-----------Try pruned ratios while acc loss={:.4f}-----------'. + format(loss)) + ratios = [] + # step 2.1: Get ratios according to current loss + for param in sensitivities: + coefficient = copy.deepcopy(coefficients[param]) + coefficient[-1] = coefficient[-1] - loss + roots = np.roots(coefficient) + for root in roots: + min_root = 1 + if np.isreal(root) and root > 0 and root < 1: + selected_root = min(root.real, min_root) + ratios.append(selected_root) + _logger.info('Pruned ratios={}'.format( + [round(ratio, 3) for ratio in ratios])) + # step 2.2: Pruning by current ratios + self._prune_parameters( + context.eval_graph, + context.scope, + sensitivities.keys(), + ratios, + context.place, + only_graph=True) + + pruned_flops = 1 - (float(context.eval_graph.flops()) / flops) + pruned_size = 1 - (float(context.eval_graph.numel_params()) / + model_size) + _logger.info('Pruned flops: {:.4f}'.format(pruned_flops)) + _logger.info('Pruned model size: {:.4f}'.format(pruned_size)) + for param in self.param_shape_backup.keys(): + context.eval_graph.var(param).set_shape(self.param_shape_backup[ + param]) + self.param_shape_backup = {} + + # step 2.3: Check whether current ratios is enough + if abs(pruned_flops - target_ratio) < 0.015: + break + if pruned_flops > target_ratio: + max_loss = loss + else: + min_loss = loss + return sensitivities.keys(), ratios + + def _current_pruning_target(self, context): + ''' + Get the target pruning rate in current epoch. + ''' + _logger.info('Left number of pruning steps: {}'.format(self.num_steps)) + if self.num_steps <= 0: + return None + if (self.start_epoch == context.epoch_id) or context.eval_converged( + self.metric_name, 0.005): + self.num_steps -= 1 + return self.pruning_step + + def on_epoch_begin(self, context): + current_ratio = self._current_pruning_target(context) + if current_ratio is not None: + sensitivities = self._compute_sensitivities(context) + params, ratios = self._get_best_ratios(context, sensitivities, + current_ratio) + self._prune_parameters(context.optimize_graph, context.scope, + params, ratios, context.place) + + self.param_shape_backup = {} + self.backup = {} + + model_size = context.eval_graph.numel_params() + flops = context.eval_graph.flops() + _logger.debug('################################') + _logger.debug('# pruning eval graph #') + _logger.debug('################################') + self._prune_graph(context.eval_graph, context.optimize_graph) + context.optimize_graph.update_groups_of_conv() + context.eval_graph.update_groups_of_conv() + context.optimize_graph.compile() # to update the compiled program + context.eval_graph.compile( + for_parallel=False, + for_test=True) # to update the compiled program + _logger.info( + '------------------finish pruning--------------------------------' + ) + _logger.info('Pruned size: {:.3f}'.format(1 - (float( + context.eval_graph.numel_params()) / model_size))) + _logger.info('Pruned flops: {:.3f}'.format(1 - (float( + context.eval_graph.flops()) / flops))) + metric = self._eval_graph(context) + _logger.info('Metric after pruning: {:.2f}'.format(metric)) + _logger.info( + '------------------SensitivePruneStrategy.on_epoch_begin finish--------------------------------' + ) diff --git a/python/paddle/fluid/contrib/slim/prune/pruner.py b/python/paddle/fluid/contrib/slim/prune/pruner.py index ca72bcb6f6..506b8fbe1d 100644 --- a/python/paddle/fluid/contrib/slim/prune/pruner.py +++ b/python/paddle/fluid/contrib/slim/prune/pruner.py @@ -13,9 +13,10 @@ # limitations under the License. import numpy as np +import collections from .... import layers -__all__ = ['Pruner', 'MagnitudePruner', 'RatioPruner'] +__all__ = ['Pruner', 'StructurePruner'] class Pruner(object): @@ -30,54 +31,77 @@ class Pruner(object): pass -class MagnitudePruner(Pruner): +class StructurePruner(Pruner): """ - Pruner used to pruning a parameter by threshold. + Pruner used to pruning parameters by groups. """ - def __init__(self, threshold): - self.threshold = threshold - - def prune(self, param, threshold=None): - if threshold is None: - thres = layers.fill_constant( - shape=[1], dtype='float32', value=self.threshold) - else: - thres = threshold - zeros_mask = layers.less_than(x=param, y=thres) - return zeros_mask - - -class RatioPruner(Pruner): - """ - Pruner used to pruning a parameter by ratio. - """ + def __init__(self, pruning_axis, criterions): + """ + Args: + pruning_axis(dict): The key is the name of parameter to be pruned, + '*' means all the parameters. + The value is the axis to be used. Given a parameter + with shape [3, 4], the result of pruning 50% on aixs 1 + is a parameter with shape [3, 2]. + criterions(dict): The key is the name of parameter to be pruned, + '*' means all the parameters. + The value is the criterion used to sort groups for pruning. + It only supports 'l1_norm' currently. + """ + self.pruning_axis = pruning_axis + self.criterions = criterions - def __init__(self, ratios=None): + def cal_pruned_idx(self, name, param, ratio, axis=None): """ + Calculate the index to be pruned on axis by given pruning ratio. Args: - ratios: dict with pair (paramer_name, pruned_ratio). + name(str): The name of parameter to be pruned. + param(np.array): The data of parameter to be pruned. + ratio(float): The ratio to be pruned. + axis(int): The axis to be used for pruning given parameter. + If it is None, the value in self.pruning_axis will be used. + default: None. + Returns: + list: The indexes to be pruned on axis. """ - self.ratios = ratios + criterion = self.criterions[ + name] if name in self.criterions else self.criterions['*'] + if axis is None: + assert self.pruning_axis is not None, "pruning_axis should set if axis is None." + axis = self.pruning_axis[ + name] if name in self.pruning_axis else self.pruning_axis['*'] + prune_num = int(round(param.shape[axis] * ratio)) + reduce_dims = [i for i in range(len(param.shape)) if i != axis] + if criterion == 'l1_norm': + criterions = np.sum(np.abs(param), axis=tuple(reduce_dims)) + pruned_idx = criterions.argsort()[:prune_num] + return pruned_idx - def prune(self, param, ratio=None): + def prune_tensor(self, tensor, pruned_idx, pruned_axis, lazy=False): """ + Pruning a array by indexes on given axis. Args: - ratio: `ratio=40%` means pruning (1 - 40%) weights to zero. + tensor(numpy.array): The target array to be pruned. + pruned_idx(list): The indexes to be pruned. + pruned_axis(int): The axis of given array to be pruned on. + lazy(bool): True means setting the pruned elements to zero. + False means remove the pruned elements from memory. + default: False. + Returns: + numpy.array: The pruned array. """ - if ratio is None: - rat = self.ratios[ - param.name] if param.name in self.ratios else self.ratios['*'] - else: - rat = ratio - if rat < 1.0: - k = max(int(rat * np.prod(param.shape)), 1) - param_vec = layers.reshape(x=param, shape=[1, -1]) - param_topk, _ = layers.topk(param_vec, k=k) - threshold = layers.slice( - param_topk, axes=[1], starts=[-1], ends=[k]) - threshold = layers.reshape(x=threshold, shape=[1]) - zeros_mask = layers.less_than(x=param, y=threshold) + mask = np.zeros(tensor.shape[pruned_axis], dtype=bool) + mask[pruned_idx] = True + + def func(data): + return data[~mask] + + def lazy_func(data): + data[mask] = 0 + return data + + if lazy: + return np.apply_along_axis(lazy_func, pruned_axis, tensor) else: - zeros_mask = layers.ones(param.shape) - return zeros_mask + return np.apply_along_axis(func, pruned_axis, tensor) diff --git a/python/paddle/fluid/contrib/slim/tests/configs/config.yaml b/python/paddle/fluid/contrib/slim/tests/configs/config.yaml deleted file mode 100644 index d9b49029d3..0000000000 --- a/python/paddle/fluid/contrib/slim/tests/configs/config.yaml +++ /dev/null @@ -1,29 +0,0 @@ -version: 1.0 -include: ["./configs/pruners.yaml", "./configs/pruners_0.yaml"] -pruners: - pruner_1: - class: 'RatioPruner' - ratios: - 'conv1_1.w': 0.3 - 'conv1_2.w': 0.4 - '*': 0.9 - group_dims: - '*': [1, 2, 3] - criterions: - '*': 'l1-norm' -strategies: - strategy_1: - class: 'SensitivePruneStrategy' - pruner: 'pruner_2' - start_epoch: 0 - end_epoch: 10 - delta_rate: 0.20 - acc_loss_threshold: 0.2 - sensitivities: - 'conv1_1.w': 0.4 - -compress_pass: - class: 'CompressPass' - epoch: 100 - strategies: - - strategy_1 diff --git a/python/paddle/fluid/contrib/slim/tests/configs/filter_pruning.yaml b/python/paddle/fluid/contrib/slim/tests/configs/filter_pruning.yaml new file mode 100644 index 0000000000..570c60026d --- /dev/null +++ b/python/paddle/fluid/contrib/slim/tests/configs/filter_pruning.yaml @@ -0,0 +1,34 @@ +#start_epoch: The 'on_epoch_begin' function will be called in start_epoch. default: 0. +#end_epoch: The 'on_epoch_end' function will be called in end_epoch. default: 10. +#delta_rate: The delta used to generate ratios when calculating sensitivities. +#target_ratio: The flops ratio to be pruned from current model. +#metric_name: The metric used to evaluate the model. +#pruned_params: The pattern str to match the parameter names to be pruned. +#sensitivities_file: The sensitivities file. +#num_steps: The number of pruning steps. +#eval_rate: The rate of sampled data used to calculate sensitivities. +version: 1.0 +pruners: + pruner_1: + class: 'StructurePruner' + pruning_axis: + '*': 0 + criterions: + '*': 'l1_norm' +strategies: + sensitive_pruning_strategy: + class: 'SensitivePruneStrategy' + pruner: 'pruner_1' + start_epoch: 0 + delta_rate: 0.1 + target_ratio: 0.3 + num_steps: 1 + eval_rate: 0.5 + pruned_params: '.*_sep_weights' + sensitivities_file: 'mobilenet_acc_top1_sensitive.data' + metric_name: 'acc_top1' +compressor: + epoch: 120 + checkpoint_path: './checkpoints/' + strategies: + - sensitive_pruning_strategy diff --git a/python/paddle/fluid/contrib/slim/tests/configs/pruners.yaml b/python/paddle/fluid/contrib/slim/tests/configs/pruners.yaml deleted file mode 100644 index 235092c595..0000000000 --- a/python/paddle/fluid/contrib/slim/tests/configs/pruners.yaml +++ /dev/null @@ -1,12 +0,0 @@ -version: 1.0 -pruners: - pruner_2: - class: 'RatioPruner' - ratios: - 'conv1_1.w': 0.5 - 'conv1_2.w': 0.2 - '*': 0.7 - group_dims: - '*': [1, 2, 3] - criterions: - '*': 'l1-norm' diff --git a/python/paddle/fluid/contrib/slim/tests/configs/pruners_0.yaml b/python/paddle/fluid/contrib/slim/tests/configs/pruners_0.yaml deleted file mode 100644 index cd2ef9eb56..0000000000 --- a/python/paddle/fluid/contrib/slim/tests/configs/pruners_0.yaml +++ /dev/null @@ -1,12 +0,0 @@ -version: 1.0 -pruners: - pruner_3: - class: 'RatioPruner' - ratios: - 'conv1_1.w': 0.5 - 'conv1_2.w': 0.2 - '*': 0.7 - group_dims: - '*': [1, 2, 3] - criterions: - '*': 'l1-norm' diff --git a/python/paddle/fluid/contrib/slim/tests/filter_pruning/__init__.py b/python/paddle/fluid/contrib/slim/tests/filter_pruning/__init__.py new file mode 100644 index 0000000000..d0c32e2609 --- /dev/null +++ b/python/paddle/fluid/contrib/slim/tests/filter_pruning/__init__.py @@ -0,0 +1,13 @@ +# 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. diff --git a/python/paddle/fluid/contrib/slim/tests/filter_pruning/compress.yaml b/python/paddle/fluid/contrib/slim/tests/filter_pruning/compress.yaml new file mode 100644 index 0000000000..232276feac --- /dev/null +++ b/python/paddle/fluid/contrib/slim/tests/filter_pruning/compress.yaml @@ -0,0 +1,34 @@ +#start_epoch: The 'on_epoch_begin' function will be called in start_epoch. default: 0. +#end_epoch: The 'on_epoch_end' function will be called in end_epoch. default: 10. +#delta_rate: The delta used to generate ratios when calculating sensitivities. +#target_ratio: The flops ratio to be pruned from current model. +#metric_name: The metric used to evaluate the model. +#pruned_params: The pattern str to match the parameter names to be pruned. +#sensitivities_file: The sensitivities file. +#num_steps: The number of pruning steps. +#eval_rate: The rate of sampled data used to calculate sensitivities. +version: 1.0 +pruners: + pruner_1: + class: 'StructurePruner' + pruning_axis: + '*': 0 + criterions: + '*': 'l1_norm' +strategies: + sensitive_pruning_strategy: + class: 'SensitivePruneStrategy' + pruner: 'pruner_1' + start_epoch: 1 + delta_rate: 0.2 + target_ratio: 0.08 + num_steps: 1 + eval_rate: 0.5 + pruned_params: 'conv6_sep_weights' + sensitivities_file: 'mobilenet_acc_top1_sensitive.data' + metric_name: 'acc_top1' +compressor: + epoch: 2 + checkpoint_path: './checkpoints/' + strategies: + - sensitive_pruning_strategy diff --git a/python/paddle/fluid/contrib/slim/tests/filter_pruning/mobilenet.py b/python/paddle/fluid/contrib/slim/tests/filter_pruning/mobilenet.py new file mode 100644 index 0000000000..0148325a64 --- /dev/null +++ b/python/paddle/fluid/contrib/slim/tests/filter_pruning/mobilenet.py @@ -0,0 +1,210 @@ +# 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. + +from __future__ import absolute_import +from __future__ import division +from __future__ import print_function +import paddle.fluid as fluid +from paddle.fluid.initializer import MSRA +from paddle.fluid.param_attr import ParamAttr + +__all__ = ['MobileNet'] + +train_parameters = { + "input_size": [3, 224, 224], + "input_mean": [0.485, 0.456, 0.406], + "input_std": [0.229, 0.224, 0.225], + "learning_strategy": { + "name": "piecewise_decay", + "batch_size": 256, + "epochs": [30, 60, 90], + "steps": [0.1, 0.01, 0.001, 0.0001] + } +} + + +class MobileNet(): + def __init__(self): + self.params = train_parameters + + def net(self, input, class_dim=1000, scale=1.0): + # conv1: 112x112 + input = self.conv_bn_layer( + input, + filter_size=3, + channels=3, + num_filters=int(32 * scale), + stride=2, + padding=1, + name="conv1") + + # 56x56 + input = self.depthwise_separable( + input, + num_filters1=32, + num_filters2=64, + num_groups=32, + stride=1, + scale=scale, + name="conv2_1") + + input = self.depthwise_separable( + input, + num_filters1=64, + num_filters2=128, + num_groups=64, + stride=2, + scale=scale, + name="conv2_2") + + # 28x28 + input = self.depthwise_separable( + input, + num_filters1=128, + num_filters2=128, + num_groups=128, + stride=1, + scale=scale, + name="conv3_1") + + input = self.depthwise_separable( + input, + num_filters1=128, + num_filters2=256, + num_groups=128, + stride=2, + scale=scale, + name="conv3_2") + + # 14x14 + input = self.depthwise_separable( + input, + num_filters1=256, + num_filters2=256, + num_groups=256, + stride=1, + scale=scale, + name="conv4_1") + + input = self.depthwise_separable( + input, + num_filters1=256, + num_filters2=512, + num_groups=256, + stride=2, + scale=scale, + name="conv4_2") + + # 14x14 + for i in range(5): + input = self.depthwise_separable( + input, + num_filters1=512, + num_filters2=512, + num_groups=512, + stride=1, + scale=scale, + name="conv5" + "_" + str(i + 1)) + # 7x7 + input = self.depthwise_separable( + input, + num_filters1=512, + num_filters2=1024, + num_groups=512, + stride=2, + scale=scale, + name="conv5_6") + + input = self.depthwise_separable( + input, + num_filters1=1024, + num_filters2=1024, + num_groups=1024, + stride=1, + scale=scale, + name="conv6") + + input = fluid.layers.pool2d( + input=input, + pool_size=0, + pool_stride=1, + pool_type='avg', + global_pooling=True) + + output = fluid.layers.fc(input=input, + size=class_dim, + act='softmax', + param_attr=ParamAttr( + initializer=MSRA(), name="fc7_weights"), + bias_attr=ParamAttr(name="fc7_offset")) + return output + + def conv_bn_layer(self, + input, + filter_size, + num_filters, + stride, + padding, + channels=None, + num_groups=1, + act='relu', + use_cudnn=True, + name=None): + conv = fluid.layers.conv2d( + input=input, + num_filters=num_filters, + filter_size=filter_size, + stride=stride, + padding=padding, + groups=num_groups, + act=None, + use_cudnn=use_cudnn, + param_attr=ParamAttr( + initializer=MSRA(), name=name + "_weights"), + bias_attr=False) + bn_name = name + "_bn" + return fluid.layers.batch_norm( + input=conv, + act=act, + param_attr=ParamAttr(name=bn_name + "_scale"), + bias_attr=ParamAttr(name=bn_name + "_offset"), + moving_mean_name=bn_name + '_mean', + moving_variance_name=bn_name + '_variance') + + def depthwise_separable(self, + input, + num_filters1, + num_filters2, + num_groups, + stride, + scale, + name=None): + depthwise_conv = self.conv_bn_layer( + input=input, + filter_size=3, + num_filters=int(num_filters1 * scale), + stride=stride, + padding=1, + num_groups=int(num_groups * scale), + use_cudnn=False, + name=name + "_dw") + + pointwise_conv = self.conv_bn_layer( + input=depthwise_conv, + filter_size=1, + num_filters=int(num_filters2 * scale), + stride=1, + padding=0, + name=name + "_sep") + return pointwise_conv diff --git a/python/paddle/fluid/contrib/slim/tests/test_factory.py b/python/paddle/fluid/contrib/slim/tests/test_factory.py index 2fc72b6475..90eb8bd4b3 100644 --- a/python/paddle/fluid/contrib/slim/tests/test_factory.py +++ b/python/paddle/fluid/contrib/slim/tests/test_factory.py @@ -12,29 +12,25 @@ # See the License for the specific language governing permissions and # limitations under the License. -from paddle.fluid.contrib.slim import ConfigFactory +from paddle.fluid.contrib.slim.core import ConfigFactory import unittest class TestFactory(unittest.TestCase): - def test_parse(self): - factory = ConfigFactory('./configs/config.yaml') + def test_parse_pruning(self): + factory = ConfigFactory('./configs/filter_pruning.yaml') - pruner = factory.instance('pruner_1') - self.assertEquals(pruner.ratios['conv1_1.w'], 0.3) + pruner_1 = factory.instance('pruner_1') + self.assertEquals(pruner_1.pruning_axis['*'], 0) + self.assertEquals(pruner_1.criterions['*'], 'l1_norm') - pruner = factory.instance('pruner_2') - self.assertEquals(pruner.ratios['*'], 0.7) + strategy = factory.instance('sensitive_pruning_strategy') + pruner_1 = strategy.pruner + self.assertEquals(pruner_1.criterions['*'], 'l1_norm') - strategy = factory.instance('strategy_1') - pruner = strategy.pruner - self.assertEquals(pruner.ratios['*'], 0.7) - - compress_pass = factory.get_compress_pass() - self.assertEquals(compress_pass.epoch, 100) - - strategy = compress_pass.strategies[0] - self.assertEquals(strategy.delta_rate, 0.2) + self.assertEquals(strategy.start_epoch, 0) + self.assertEquals(strategy.sensitivities_file, + 'mobilenet_acc_top1_sensitive.data') if __name__ == '__main__': diff --git a/python/paddle/fluid/contrib/slim/tests/test_filter_pruning.py b/python/paddle/fluid/contrib/slim/tests/test_filter_pruning.py new file mode 100644 index 0000000000..d73ee27779 --- /dev/null +++ b/python/paddle/fluid/contrib/slim/tests/test_filter_pruning.py @@ -0,0 +1,89 @@ +# 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 paddle +import unittest +import paddle.fluid as fluid +from filter_pruning.mobilenet import MobileNet +from paddle.fluid.contrib.slim.core import Compressor +from paddle.fluid.contrib.slim.graph import GraphWrapper + + +class TestFilterPruning(unittest.TestCase): + def test_compression(self): + """ + Model: mobilenet_v1 + data: mnist + step1: Training one epoch + step2: pruning flops + step3: fine-tune one epoch + step4: check top1_acc. + """ + if not fluid.core.is_compiled_with_cuda(): + return + class_dim = 10 + image_shape = [1, 28, 28] + image = fluid.layers.data( + name='image', shape=image_shape, dtype='float32') + image.stop_gradient = False + label = fluid.layers.data(name='label', shape=[1], dtype='int64') + out = MobileNet().net(input=image, class_dim=class_dim) + acc_top1 = fluid.layers.accuracy(input=out, label=label, k=1) + acc_top5 = fluid.layers.accuracy(input=out, label=label, k=5) + val_program = fluid.default_main_program().clone(for_test=False) + + cost = fluid.layers.cross_entropy(input=out, label=label) + avg_cost = fluid.layers.mean(x=cost) + + optimizer = fluid.optimizer.Momentum( + momentum=0.9, + learning_rate=0.01, + regularization=fluid.regularizer.L2Decay(4e-5)) + + place = fluid.CUDAPlace(0) + exe = fluid.Executor(place) + exe.run(fluid.default_startup_program()) + + val_reader = paddle.batch(paddle.dataset.mnist.test(), batch_size=128) + + val_feed_list = [('img', image.name), ('label', label.name)] + val_fetch_list = [('acc_top1', acc_top1.name), ('acc_top5', + acc_top5.name)] + + train_reader = paddle.batch( + paddle.dataset.mnist.train(), batch_size=128) + train_feed_list = [('img', image.name), ('label', label.name)] + train_fetch_list = [('loss', avg_cost.name)] + + com_pass = Compressor( + place, + fluid.global_scope(), + fluid.default_main_program(), + train_reader=train_reader, + train_feed_list=train_feed_list, + train_fetch_list=train_fetch_list, + eval_program=val_program, + eval_reader=val_reader, + eval_feed_list=val_feed_list, + eval_fetch_list=val_fetch_list, + train_optimizer=optimizer) + com_pass.config('./filter_pruning/compress.yaml') + eval_graph = com_pass.run() + self.assertTrue( + abs((com_pass.context.eval_results['acc_top1'][-1] - 0.969) / 0.969) + < 0.02) + + +if __name__ == '__main__': + unittest.main() diff --git a/python/paddle/fluid/contrib/slim/tests/test_graph_wrapper.py b/python/paddle/fluid/contrib/slim/tests/test_graph_wrapper.py new file mode 100644 index 0000000000..ad82aa9411 --- /dev/null +++ b/python/paddle/fluid/contrib/slim/tests/test_graph_wrapper.py @@ -0,0 +1,140 @@ +# 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. + +from __future__ import print_function +import unittest +import paddle.fluid as fluid +import six +import numpy as np +from paddle.fluid.contrib.slim.graph import GraphWrapper +from paddle.fluid import core + + +def residual_block(num): + def conv_bn_layer(input, + ch_out, + filter_size, + stride, + padding, + act='relu', + bias_attr=False): + tmp = fluid.layers.conv2d( + input=input, + filter_size=filter_size, + num_filters=ch_out, + stride=stride, + padding=padding, + act=None, + bias_attr=bias_attr) + return fluid.layers.batch_norm(input=tmp, act=act) + + data = fluid.layers.data(name='image', shape=[1, 8, 8], dtype='float32') + label = fluid.layers.data(name='label', shape=[1], dtype='int64') + data.stop_gradinet = False + hidden = data + for _ in six.moves.xrange(num): + conv = conv_bn_layer(hidden, 16, 3, 1, 1, act=None, bias_attr=True) + short = conv_bn_layer(hidden, 16, 1, 1, 0, act=None) + hidden = fluid.layers.elementwise_add(x=conv, y=short, act='relu') + fc = fluid.layers.fc(input=hidden, size=10) + + loss = fluid.layers.cross_entropy(input=fc, label=label) + loss = fluid.layers.mean(loss) + return data, label, loss + + +class TestGraphWrapper(unittest.TestCase): + def build_program(self): + place = fluid.CPUPlace() + if fluid.core.is_compiled_with_cuda(): + place = fluid.CUDAPlace(0) + main = fluid.Program() + startup = fluid.Program() + with fluid.program_guard(main, startup): + image, label, self.loss = residual_block(2) + eval_program = main.clone() + opt = fluid.optimizer.SGD(learning_rate=0.001) + opt.minimize(self.loss) + self.scope = core.Scope() + exe = fluid.Executor(place) + exe.run(startup, scope=self.scope) + self.eval_graph = GraphWrapper( + program=eval_program, + in_nodes={'image': image.name, + 'label': label.name}, + out_nodes={'loss': self.loss.name}) + self.train_graph = GraphWrapper( + program=main, + in_nodes={'image': image.name, + 'label': label.name}, + out_nodes={'loss': self.loss.name}) + + def test_all_parameters(self): + self.build_program() + self.assertEquals(len(self.train_graph.all_parameters()), 24) + + def test_all_vars(self): + self.build_program() + self.assertEquals(len(self.train_graph.vars()), 90) + + def test_numel_params(self): + self.build_program() + self.assertEquals(self.train_graph.numel_params(), 13258) + + def test_compile(self): + self.build_program() + place = fluid.CPUPlace() + if fluid.core.is_compiled_with_cuda(): + place = fluid.CUDAPlace(0) + exe = fluid.Executor(place) + self.train_graph.compile() + exe.run(self.train_graph.compiled_graph, + scope=self.scope, + feed={ + 'image': + np.random.randint(0, 40, [16, 1, 8, 8]).astype('float32'), + 'label': np.random.randint(0, 10, [16, 1]).astype('int64') + }) + + def test_pre_and_next_ops(self): + self.build_program() + for op in self.train_graph.ops(): + for next_op in self.train_graph.next_ops(op): + self.assertTrue(op in self.train_graph.pre_ops(next_op)) + + def test_get_optimize_graph(self): + self.build_program() + place = fluid.CPUPlace() + if fluid.core.is_compiled_with_cuda(): + place = fluid.CUDAPlace(0) + opt = fluid.optimizer.SGD(learning_rate=0.001) + train_graph = self.eval_graph.get_optimize_graph( + opt, place, self.scope, no_grad_var_names=['image']) + self.assertEquals(len(self.train_graph.ops()), len(train_graph.ops())) + exe = fluid.Executor(place) + train_graph.compile() + image = np.random.randint(0, 225, [16, 1, 8, 8]).astype('float32') + label = np.random.randint(0, 10, [16, 1]).astype('int64') + exe.run(train_graph.compiled_graph, + scope=self.scope, + feed={'image': image, + 'label': label}) + + def test_flops(self): + self.build_program() + self.assertEquals(self.train_graph.flops(), 354624) + + +if __name__ == '__main__': + unittest.main() diff --git a/python/requirements.txt b/python/requirements.txt index 36bd5d4261..ce56462fac 100644 --- a/python/requirements.txt +++ b/python/requirements.txt @@ -12,3 +12,4 @@ six funcsigs pyyaml decorator +prettytable From 431068c9cac6292189170834b5eea3855f745b8e Mon Sep 17 00:00:00 2001 From: chuanqiw Date: Sat, 23 Mar 2019 20:25:26 +0800 Subject: [PATCH 35/44] Enhance test calibration script on accuracy assert test=develop --- python/paddle/fluid/contrib/tests/test_calibration.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/paddle/fluid/contrib/tests/test_calibration.py b/python/paddle/fluid/contrib/tests/test_calibration.py index b9f938bebe..1a046a7941 100644 --- a/python/paddle/fluid/contrib/tests/test_calibration.py +++ b/python/paddle/fluid/contrib/tests/test_calibration.py @@ -290,7 +290,7 @@ class TestCalibrationForResnet50(unittest.TestCase): self.model, self.infer_iterations) (int8_throughput, int8_latency, int8_acc1) = self.run_program("calibration_out") - delta_value = np.abs(fp32_acc1 - int8_acc1) + delta_value = fp32_acc1 - int8_acc1 self.assertLess(delta_value, 0.01) print( "FP32 {0}: batch_size {1}, throughput {2} images/second, latency {3} second, accuracy {4}". From c917c13af150c578593e00cc6f25017c40e9dacd Mon Sep 17 00:00:00 2001 From: chengduo Date: Sat, 23 Mar 2019 09:10:00 -0500 Subject: [PATCH 36/44] increase the time limite (#16405) test=develop --- python/paddle/fluid/tests/unittests/CMakeLists.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/paddle/fluid/tests/unittests/CMakeLists.txt b/python/paddle/fluid/tests/unittests/CMakeLists.txt index 3c6b9daca6..cefa2b4919 100644 --- a/python/paddle/fluid/tests/unittests/CMakeLists.txt +++ b/python/paddle/fluid/tests/unittests/CMakeLists.txt @@ -118,8 +118,8 @@ if(NOT APPLE) py_test_modules(test_image_classification_resnet MODULES test_image_classification_resnet SERIAL) endif() if(CMAKE_BUILD_TYPE STREQUAL "Debug") - # change the timeout from 600 to 1200, because in debug mode, this test need more time. - set_tests_properties(test_parallel_executor_seresnext PROPERTIES TIMEOUT 1200) + # change the timeout from 600 to 2200, because in debug mode, this test need more time. + set_tests_properties(test_parallel_executor_seresnext PROPERTIES TIMEOUT 2200) endif() if (WITH_NGRAPH) From 46677fb080136bf5f70e108888cff2cdffcf8e78 Mon Sep 17 00:00:00 2001 From: Wojciech Uss Date: Mon, 25 Mar 2019 03:54:52 +0100 Subject: [PATCH 37/44] Move cpu_quantize_* passes into mkldnn subfolder test=develop --- paddle/fluid/framework/ir/CMakeLists.txt | 12 ++++++------ .../framework/ir/{ => mkldnn}/cpu_quantize_pass.cc | 2 +- .../framework/ir/{ => mkldnn}/cpu_quantize_pass.h | 0 .../ir/{ => mkldnn}/cpu_quantize_pass_tester.cc | 2 +- .../ir/{ => mkldnn}/cpu_quantize_placement_pass.cc | 2 +- .../ir/{ => mkldnn}/cpu_quantize_placement_pass.h | 0 .../cpu_quantize_placement_pass_tester.cc | 2 +- .../ir/{ => mkldnn}/cpu_quantize_squash_pass.cc | 2 +- .../ir/{ => mkldnn}/cpu_quantize_squash_pass.h | 0 .../{ => mkldnn}/cpu_quantize_squash_pass_tester.cc | 2 +- paddle/fluid/inference/analysis/argument.h | 5 +++++ paddle/fluid/inference/analysis/ir_pass_manager.cc | 2 ++ 12 files changed, 19 insertions(+), 12 deletions(-) rename paddle/fluid/framework/ir/{ => mkldnn}/cpu_quantize_pass.cc (99%) rename paddle/fluid/framework/ir/{ => mkldnn}/cpu_quantize_pass.h (100%) rename paddle/fluid/framework/ir/{ => mkldnn}/cpu_quantize_pass_tester.cc (99%) rename paddle/fluid/framework/ir/{ => mkldnn}/cpu_quantize_placement_pass.cc (96%) rename paddle/fluid/framework/ir/{ => mkldnn}/cpu_quantize_placement_pass.h (100%) rename paddle/fluid/framework/ir/{ => mkldnn}/cpu_quantize_placement_pass_tester.cc (98%) rename paddle/fluid/framework/ir/{ => mkldnn}/cpu_quantize_squash_pass.cc (98%) rename paddle/fluid/framework/ir/{ => mkldnn}/cpu_quantize_squash_pass.h (100%) rename paddle/fluid/framework/ir/{ => mkldnn}/cpu_quantize_squash_pass_tester.cc (98%) diff --git a/paddle/fluid/framework/ir/CMakeLists.txt b/paddle/fluid/framework/ir/CMakeLists.txt index a79a53867d..8c7d4cf0e3 100644 --- a/paddle/fluid/framework/ir/CMakeLists.txt +++ b/paddle/fluid/framework/ir/CMakeLists.txt @@ -46,9 +46,6 @@ cc_library(fuse_pass_base SRCS fuse_pass_base.cc DEPS pass) pass_library(graph_to_program_pass base) pass_library(graph_viz_pass base) pass_library(lock_free_optimize_pass base) -pass_library(cpu_quantize_placement_pass base) -pass_library(cpu_quantize_pass inference) -pass_library(cpu_quantize_squash_pass inference) pass_library(fc_fuse_pass inference) pass_library(attention_lstm_fuse_pass inference) pass_library(infer_clean_graph_pass inference) @@ -87,6 +84,9 @@ if(WITH_MKLDNN) pass_library(conv_bias_mkldnn_fuse_pass inference mkldnn) pass_library(conv_relu_mkldnn_fuse_pass inference mkldnn) pass_library(conv_elementwise_add_mkldnn_fuse_pass inference mkldnn) + pass_library(cpu_quantize_placement_pass base mkldnn) + pass_library(cpu_quantize_pass inference mkldnn) + pass_library(cpu_quantize_squash_pass inference mkldnn) endif() cc_library(fuse_elewise_add_act_pass SRCS fuse_elewise_add_act_pass.cc DEPS pass graph_pattern_detector ) @@ -105,9 +105,6 @@ cc_test(test_graph_pattern_detector SRCS graph_pattern_detector_tester.cc DEPS g cc_test(test_fc_fuse_pass SRCS fc_fuse_pass_tester.cc DEPS fc_fuse_pass framework_proto) cc_test(test_seqpool_concat_fuse_pass SRCS seqpool_concat_fuse_pass_tester.cc DEPS seqpool_concat_fuse_pass framework_proto) cc_test(test_is_test_pass SRCS is_test_pass_tester.cc DEPS is_test_pass) -cc_test(test_cpu_quantize_placement_pass SRCS cpu_quantize_placement_pass_tester.cc DEPS cpu_quantize_placement_pass) -cc_test(test_cpu_quantize_pass SRCS cpu_quantize_pass_tester.cc DEPS cpu_quantize_pass naive_executor) -cc_test(test_cpu_quantize_squash_pass SRCS cpu_quantize_squash_pass_tester.cc DEPS cpu_quantize_squash_pass naive_executor) if(NOT WIN32) cc_test(test_sync_batch_norm_pass SRCS sync_batch_norm_pass_tester.cc DEPS sync_batch_norm_pass) endif() @@ -117,4 +114,7 @@ if (WITH_MKLDNN) cc_test(test_conv_relu_mkldnn_fuse_pass SRCS mkldnn/conv_relu_mkldnn_fuse_pass_tester.cc DEPS conv_relu_mkldnn_fuse_pass) cc_test(test_conv_elementwise_add_mkldnn_fuse_pass SRCS mkldnn/conv_elementwise_add_mkldnn_fuse_pass_tester.cc DEPS conv_elementwise_add_mkldnn_fuse_pass) cc_test(test_mkldnn_placement_pass SRCS mkldnn/mkldnn_placement_pass_tester.cc DEPS mkldnn_placement_pass) + cc_test(test_cpu_quantize_placement_pass SRCS mkldnn/cpu_quantize_placement_pass_tester.cc DEPS cpu_quantize_placement_pass) + cc_test(test_cpu_quantize_pass SRCS mkldnn/cpu_quantize_pass_tester.cc DEPS cpu_quantize_pass naive_executor) + cc_test(test_cpu_quantize_squash_pass SRCS mkldnn/cpu_quantize_squash_pass_tester.cc DEPS cpu_quantize_squash_pass naive_executor) endif () diff --git a/paddle/fluid/framework/ir/cpu_quantize_pass.cc b/paddle/fluid/framework/ir/mkldnn/cpu_quantize_pass.cc similarity index 99% rename from paddle/fluid/framework/ir/cpu_quantize_pass.cc rename to paddle/fluid/framework/ir/mkldnn/cpu_quantize_pass.cc index ed80f9cae3..b3a8c20891 100644 --- a/paddle/fluid/framework/ir/cpu_quantize_pass.cc +++ b/paddle/fluid/framework/ir/mkldnn/cpu_quantize_pass.cc @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include "paddle/fluid/framework/ir/cpu_quantize_pass.h" +#include "paddle/fluid/framework/ir/mkldnn/cpu_quantize_pass.h" #include #include #include "paddle/fluid/framework/eigen.h" diff --git a/paddle/fluid/framework/ir/cpu_quantize_pass.h b/paddle/fluid/framework/ir/mkldnn/cpu_quantize_pass.h similarity index 100% rename from paddle/fluid/framework/ir/cpu_quantize_pass.h rename to paddle/fluid/framework/ir/mkldnn/cpu_quantize_pass.h diff --git a/paddle/fluid/framework/ir/cpu_quantize_pass_tester.cc b/paddle/fluid/framework/ir/mkldnn/cpu_quantize_pass_tester.cc similarity index 99% rename from paddle/fluid/framework/ir/cpu_quantize_pass_tester.cc rename to paddle/fluid/framework/ir/mkldnn/cpu_quantize_pass_tester.cc index 89601be7d1..0d0ed98901 100644 --- a/paddle/fluid/framework/ir/cpu_quantize_pass_tester.cc +++ b/paddle/fluid/framework/ir/mkldnn/cpu_quantize_pass_tester.cc @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include "paddle/fluid/framework/ir/cpu_quantize_pass.h" +#include "paddle/fluid/framework/ir/mkldnn/cpu_quantize_pass.h" #include #include "paddle/fluid/framework/naive_executor.h" #include "paddle/fluid/platform/place.h" diff --git a/paddle/fluid/framework/ir/cpu_quantize_placement_pass.cc b/paddle/fluid/framework/ir/mkldnn/cpu_quantize_placement_pass.cc similarity index 96% rename from paddle/fluid/framework/ir/cpu_quantize_placement_pass.cc rename to paddle/fluid/framework/ir/mkldnn/cpu_quantize_placement_pass.cc index 50bbe4915b..511003dce5 100644 --- a/paddle/fluid/framework/ir/cpu_quantize_placement_pass.cc +++ b/paddle/fluid/framework/ir/mkldnn/cpu_quantize_placement_pass.cc @@ -12,7 +12,7 @@ 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/ir/cpu_quantize_placement_pass.h" +#include "paddle/fluid/framework/ir/mkldnn/cpu_quantize_placement_pass.h" #include #include diff --git a/paddle/fluid/framework/ir/cpu_quantize_placement_pass.h b/paddle/fluid/framework/ir/mkldnn/cpu_quantize_placement_pass.h similarity index 100% rename from paddle/fluid/framework/ir/cpu_quantize_placement_pass.h rename to paddle/fluid/framework/ir/mkldnn/cpu_quantize_placement_pass.h diff --git a/paddle/fluid/framework/ir/cpu_quantize_placement_pass_tester.cc b/paddle/fluid/framework/ir/mkldnn/cpu_quantize_placement_pass_tester.cc similarity index 98% rename from paddle/fluid/framework/ir/cpu_quantize_placement_pass_tester.cc rename to paddle/fluid/framework/ir/mkldnn/cpu_quantize_placement_pass_tester.cc index 5a4d622645..11d72a56bd 100644 --- a/paddle/fluid/framework/ir/cpu_quantize_placement_pass_tester.cc +++ b/paddle/fluid/framework/ir/mkldnn/cpu_quantize_placement_pass_tester.cc @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include "paddle/fluid/framework/ir/cpu_quantize_placement_pass.h" +#include "paddle/fluid/framework/ir/mkldnn/cpu_quantize_placement_pass.h" #include #include diff --git a/paddle/fluid/framework/ir/cpu_quantize_squash_pass.cc b/paddle/fluid/framework/ir/mkldnn/cpu_quantize_squash_pass.cc similarity index 98% rename from paddle/fluid/framework/ir/cpu_quantize_squash_pass.cc rename to paddle/fluid/framework/ir/mkldnn/cpu_quantize_squash_pass.cc index de62a69de4..6e74cc7787 100644 --- a/paddle/fluid/framework/ir/cpu_quantize_squash_pass.cc +++ b/paddle/fluid/framework/ir/mkldnn/cpu_quantize_squash_pass.cc @@ -13,7 +13,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include "paddle/fluid/framework/ir/cpu_quantize_squash_pass.h" +#include "paddle/fluid/framework/ir/mkldnn/cpu_quantize_squash_pass.h" #include #include #include "paddle/fluid/platform/enforce.h" diff --git a/paddle/fluid/framework/ir/cpu_quantize_squash_pass.h b/paddle/fluid/framework/ir/mkldnn/cpu_quantize_squash_pass.h similarity index 100% rename from paddle/fluid/framework/ir/cpu_quantize_squash_pass.h rename to paddle/fluid/framework/ir/mkldnn/cpu_quantize_squash_pass.h diff --git a/paddle/fluid/framework/ir/cpu_quantize_squash_pass_tester.cc b/paddle/fluid/framework/ir/mkldnn/cpu_quantize_squash_pass_tester.cc similarity index 98% rename from paddle/fluid/framework/ir/cpu_quantize_squash_pass_tester.cc rename to paddle/fluid/framework/ir/mkldnn/cpu_quantize_squash_pass_tester.cc index 3a3eb53f79..3cf51d97aa 100644 --- a/paddle/fluid/framework/ir/cpu_quantize_squash_pass_tester.cc +++ b/paddle/fluid/framework/ir/mkldnn/cpu_quantize_squash_pass_tester.cc @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include "paddle/fluid/framework/ir/cpu_quantize_squash_pass.h" +#include "paddle/fluid/framework/ir/mkldnn/cpu_quantize_squash_pass.h" #include #include "paddle/fluid/framework/naive_executor.h" #include "paddle/fluid/platform/place.h" diff --git a/paddle/fluid/inference/analysis/argument.h b/paddle/fluid/inference/analysis/argument.h index 997f3575f4..78cc1e9688 100644 --- a/paddle/fluid/inference/analysis/argument.h +++ b/paddle/fluid/inference/analysis/argument.h @@ -41,8 +41,11 @@ namespace inference { namespace analysis { using framework::ir::Graph; + +#ifdef PADDLE_WITH_MKLDNN using VarQuantScale = std::unordered_map>; +#endif /* * The argument definition of both Pass and PassManagers. @@ -132,6 +135,7 @@ struct Argument { DECL_ARGUMENT_FIELD(mkldnn_enabled_op_types, MKLDNNEnabledOpTypes, std::unordered_set); +#ifdef PADDLE_WITH_MKLDNN // A set of op types to enable their quantized kernels DECL_ARGUMENT_FIELD(quantize_enabled_op_types, QuantizeEnabledOpTypes, std::unordered_set); @@ -142,6 +146,7 @@ struct Argument { // Scales for variables to be quantized DECL_ARGUMENT_FIELD(quant_var_scales, QuantVarScales, VarQuantScale); +#endif // Passed from config. DECL_ARGUMENT_FIELD(use_gpu, UseGPU, bool); diff --git a/paddle/fluid/inference/analysis/ir_pass_manager.cc b/paddle/fluid/inference/analysis/ir_pass_manager.cc index 1556caa464..c796000466 100644 --- a/paddle/fluid/inference/analysis/ir_pass_manager.cc +++ b/paddle/fluid/inference/analysis/ir_pass_manager.cc @@ -61,6 +61,7 @@ void IRPassManager::CreatePasses(Argument *argument, pass->Set("mkldnn_enabled_op_types", new std::unordered_set( argument->mkldnn_enabled_op_types())); +#ifdef PADDLE_WITH_MKLDNN } else if (pass_name == "cpu_quantize_placement_pass") { pass->Set("quantize_enabled_op_types", new std::unordered_set( @@ -71,6 +72,7 @@ void IRPassManager::CreatePasses(Argument *argument, } else if (pass_name == "cpu_quantize_pass") { pass->Set("quant_var_scales", new VarQuantScale(argument->quant_var_scales())); +#endif } else if (pass_name == "tensorrt_subgraph_pass") { pass->Set("workspace_size", new int(argument->tensorrt_workspace_size())); pass->Set("max_batch_size", new int(argument->tensorrt_max_batch_size())); From de3b70a101dabd48e0659ddb4df4651b882aeafa Mon Sep 17 00:00:00 2001 From: liuwei1031 <46661762+liuwei1031@users.noreply.github.com> Date: Mon, 25 Mar 2019 21:08:58 +0800 Subject: [PATCH 38/44] fix cdn issue, test=develop (#16423) * fix cdn issue, test=develop * fix cdn issue, test=develop --- cmake/external/boost.cmake | 2 +- cmake/external/grpc.cmake | 2 +- cmake/external/mklml.cmake | 4 ++-- paddle/fluid/inference/api/demo_ci/run.sh | 2 +- paddle/fluid/inference/tests/api/CMakeLists.txt | 4 ++-- paddle/fluid/inference/tests/test.cmake | 2 +- python/paddle/dataset/flowers.py | 6 +++--- python/paddle/fluid/contrib/tests/test_calibration.py | 6 +++--- python/paddle/fluid/tests/unittests/dist_ctr_reader.py | 2 +- .../paddle/fluid/tests/unittests/test_dist_transformer.py | 2 +- 10 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cmake/external/boost.cmake b/cmake/external/boost.cmake index fc204dc919..ba8b5fc6c8 100644 --- a/cmake/external/boost.cmake +++ b/cmake/external/boost.cmake @@ -24,7 +24,7 @@ set(BOOST_PROJECT "extern_boost") # So we use 1.41.0 here. set(BOOST_VER "1.41.0") set(BOOST_TAR "boost_1_41_0" CACHE STRING "" FORCE) -set(BOOST_URL "http://paddlepaddledeps.cdn.bcebos.com/${BOOST_TAR}.tar.gz" CACHE STRING "" FORCE) +set(BOOST_URL "http://paddlepaddledeps.bj.bcebos.com/${BOOST_TAR}.tar.gz" CACHE STRING "" FORCE) MESSAGE(STATUS "BOOST_TAR: ${BOOST_TAR}, BOOST_URL: ${BOOST_URL}") diff --git a/cmake/external/grpc.cmake b/cmake/external/grpc.cmake index c5754da59b..d96da470b3 100644 --- a/cmake/external/grpc.cmake +++ b/cmake/external/grpc.cmake @@ -44,7 +44,7 @@ ExternalProject_Add( # 3. keep only zlib, cares, protobuf, boringssl under "third_party", # checkout and clean other dirs under third_party # 4. remove .git, and package the directory. - URL "http://paddlepaddledeps.cdn.bcebos.com/grpc-v1.10.x.tar.gz" + URL "http://paddlepaddledeps.bj.bcebos.com/grpc-v1.10.x.tar.gz" URL_MD5 "1f268a2aff6759839dccd256adcc91cf" PREFIX ${GRPC_SOURCES_DIR} UPDATE_COMMAND "" diff --git a/cmake/external/mklml.cmake b/cmake/external/mklml.cmake index ae2679db4a..142fce816d 100644 --- a/cmake/external/mklml.cmake +++ b/cmake/external/mklml.cmake @@ -34,7 +34,7 @@ SET(CMAKE_INSTALL_RPATH "${CMAKE_INSTALL_RPATH}" "${MKLML_ROOT}/lib") SET(TIME_VERSION "2019.0.1.20181227") IF(WIN32) SET(MKLML_VER "mklml_win_${TIME_VERSION}" CACHE STRING "" FORCE) - SET(MKLML_URL "https://paddlepaddledeps.cdn.bcebos.com/${MKLML_VER}.zip" CACHE STRING "" FORCE) + SET(MKLML_URL "https://paddlepaddledeps.bj.bcebos.com/${MKLML_VER}.zip" CACHE STRING "" FORCE) SET(MKLML_LIB ${MKLML_LIB_DIR}/mklml.lib) SET(MKLML_IOMP_LIB ${MKLML_LIB_DIR}/libiomp5md.lib) SET(MKLML_SHARED_LIB ${MKLML_LIB_DIR}/mklml.dll) @@ -43,7 +43,7 @@ ELSE() #TODO(intel-huying): # Now enable Erf function in mklml library temporarily, it will be updated as offical version later. SET(MKLML_VER "Glibc225_vsErf_mklml_lnx_${TIME_VERSION}" CACHE STRING "" FORCE) - SET(MKLML_URL "http://paddlepaddledeps.cdn.bcebos.com/${MKLML_VER}.tgz" CACHE STRING "" FORCE) + SET(MKLML_URL "http://paddlepaddledeps.bj.bcebos.com/${MKLML_VER}.tgz" CACHE STRING "" FORCE) SET(MKLML_LIB ${MKLML_LIB_DIR}/libmklml_intel.so) SET(MKLML_IOMP_LIB ${MKLML_LIB_DIR}/libiomp5.so) SET(MKLML_SHARED_LIB ${MKLML_LIB_DIR}/libmklml_intel.so) diff --git a/paddle/fluid/inference/api/demo_ci/run.sh b/paddle/fluid/inference/api/demo_ci/run.sh index 963986f245..bf2e3593c2 100755 --- a/paddle/fluid/inference/api/demo_ci/run.sh +++ b/paddle/fluid/inference/api/demo_ci/run.sh @@ -27,7 +27,7 @@ if [ -d "$TENSORRT_INCLUDE_DIR" -a -d "$TENSORRT_LIB_DIR" ]; then fi PREFIX=inference-vis-demos%2F -URL_ROOT=http://paddlemodels.cdn.bcebos.com/${PREFIX} +URL_ROOT=http://paddlemodels.bj.bcebos.com/${PREFIX} # download vis_demo data function download() { diff --git a/paddle/fluid/inference/tests/api/CMakeLists.txt b/paddle/fluid/inference/tests/api/CMakeLists.txt index d9ac73b063..2f17a44e0c 100644 --- a/paddle/fluid/inference/tests/api/CMakeLists.txt +++ b/paddle/fluid/inference/tests/api/CMakeLists.txt @@ -115,14 +115,14 @@ inference_analysis_test(test_analyzer_transformer SRCS analyzer_transformer_test # ocr set(OCR_INSTALL_DIR "${INFERENCE_DEMO_INSTALL_DIR}/ocr") if (NOT EXISTS ${OCR_INSTALL_DIR}) - inference_download_and_uncompress(${OCR_INSTALL_DIR} "http://paddlemodels.cdn.bcebos.com/" "inference-vis-demos%2Focr.tar.gz") + inference_download_and_uncompress(${OCR_INSTALL_DIR} "http://paddlemodels.bj.bcebos.com/" "inference-vis-demos%2Focr.tar.gz") endif() inference_analysis_api_test_with_refer_result(test_analyzer_ocr ${OCR_INSTALL_DIR} analyzer_vis_tester.cc SERIAL) # mobilenet with transpose op set(MOBILENET_INSTALL_DIR "${INFERENCE_DEMO_INSTALL_DIR}/mobilenet") if (NOT EXISTS ${MOBILENET_INSTALL_DIR}) - inference_download_and_uncompress(${MOBILENET_INSTALL_DIR} "http://paddlemodels.cdn.bcebos.com/" "inference-vis-demos%2Fmobilenet.tar.gz") + inference_download_and_uncompress(${MOBILENET_INSTALL_DIR} "http://paddlemodels.bj.bcebos.com/" "inference-vis-demos%2Fmobilenet.tar.gz") endif() inference_analysis_api_test_with_refer_result(test_analyzer_mobilenet_transpose ${MOBILENET_INSTALL_DIR} analyzer_vis_tester.cc SERIAL) diff --git a/paddle/fluid/inference/tests/test.cmake b/paddle/fluid/inference/tests/test.cmake index f551b322fe..df7af71d9b 100644 --- a/paddle/fluid/inference/tests/test.cmake +++ b/paddle/fluid/inference/tests/test.cmake @@ -1,5 +1,5 @@ include(ExternalProject) -set(INFERENCE_URL "http://paddle-inference-dist.cdn.bcebos.com" CACHE STRING "inference download url") +set(INFERENCE_URL "http://paddle-inference-dist.bj.bcebos.com" CACHE STRING "inference download url") set(INFERENCE_DEMO_INSTALL_DIR "${THIRD_PARTY_PATH}/inference_demo" CACHE STRING "A path setting inference demo download directories.") diff --git a/python/paddle/dataset/flowers.py b/python/paddle/dataset/flowers.py index 57c5e83c82..5728a37fc3 100644 --- a/python/paddle/dataset/flowers.py +++ b/python/paddle/dataset/flowers.py @@ -46,9 +46,9 @@ import six from six.moves import cPickle as pickle __all__ = ['train', 'test', 'valid'] -DATA_URL = 'http://paddlemodels.cdn.bcebos.com/flowers/102flowers.tgz' -LABEL_URL = 'http://paddlemodels.cdn.bcebos.com/flowers/imagelabels.mat' -SETID_URL = 'http://paddlemodels.cdn.bcebos.com/flowers/setid.mat' +DATA_URL = 'http://paddlemodels.bj.bcebos.com/flowers/102flowers.tgz' +LABEL_URL = 'http://paddlemodels.bj.bcebos.com/flowers/imagelabels.mat' +SETID_URL = 'http://paddlemodels.bj.bcebos.com/flowers/setid.mat' DATA_MD5 = '52808999861908f626f3c1f4e79d11fa' LABEL_MD5 = 'e0620be6f572b9609742df49c70aed4d' SETID_MD5 = 'a5357ecc9cb78c4bef273ce3793fc85c' diff --git a/python/paddle/fluid/contrib/tests/test_calibration.py b/python/paddle/fluid/contrib/tests/test_calibration.py index 1a046a7941..00885eb5d6 100644 --- a/python/paddle/fluid/contrib/tests/test_calibration.py +++ b/python/paddle/fluid/contrib/tests/test_calibration.py @@ -136,7 +136,7 @@ class TestCalibrationForResnet50(unittest.TestCase): "full_data", False) else: data_urls.append( - 'http://paddle-inference-dist.cdn.bcebos.com/int8/calibration_test_data.tar.gz' + 'http://paddle-inference-dist.bj.bcebos.com/int8/calibration_test_data.tar.gz' ) data_md5s.append('1b6c1c434172cca1bf9ba1e4d7a3157d') self.data_cache_folder = self.download_data(data_urls, data_md5s, @@ -189,7 +189,7 @@ class TestCalibrationForResnet50(unittest.TestCase): def download_model(self): # resnet50 fp32 data data_urls = [ - 'http://paddle-inference-dist.cdn.bcebos.com/int8/resnet50_int8_model.tar.gz' + 'http://paddle-inference-dist.bj.bcebos.com/int8/resnet50_int8_model.tar.gz' ] data_md5s = ['4a5194524823d9b76da6e738e1367881'] self.model_cache_folder = self.download_data(data_urls, data_md5s, @@ -307,7 +307,7 @@ class TestCalibrationForMobilenetv1(TestCalibrationForResnet50): def download_model(self): # mobilenetv1 fp32 data data_urls = [ - 'http://paddle-inference-dist.cdn.bcebos.com/int8/mobilenetv1_int8_model.tar.gz' + 'http://paddle-inference-dist.bj.bcebos.com/int8/mobilenetv1_int8_model.tar.gz' ] data_md5s = ['13892b0716d26443a8cdea15b3c6438b'] self.model_cache_folder = self.download_data(data_urls, data_md5s, diff --git a/python/paddle/fluid/tests/unittests/dist_ctr_reader.py b/python/paddle/fluid/tests/unittests/dist_ctr_reader.py index 95e39d891f..48a4768782 100644 --- a/python/paddle/fluid/tests/unittests/dist_ctr_reader.py +++ b/python/paddle/fluid/tests/unittests/dist_ctr_reader.py @@ -20,7 +20,7 @@ logging.basicConfig() logger = logging.getLogger("paddle") logger.setLevel(logging.INFO) -DATA_URL = "http://paddle-ctr-data.cdn.bcebos.com/avazu_ctr_data.tgz" +DATA_URL = "http://paddle-ctr-data.bj.bcebos.com/avazu_ctr_data.tgz" DATA_MD5 = "c11df99fbd14e53cd4bfa6567344b26e" """ avazu_ctr_data/train.txt diff --git a/python/paddle/fluid/tests/unittests/test_dist_transformer.py b/python/paddle/fluid/tests/unittests/test_dist_transformer.py index 25dcccc28d..3307caa8b2 100644 --- a/python/paddle/fluid/tests/unittests/test_dist_transformer.py +++ b/python/paddle/fluid/tests/unittests/test_dist_transformer.py @@ -21,7 +21,7 @@ from test_dist_base import TestDistBase def download_files(): - url_prefix = 'http://paddle-unittest-data.cdn.bcebos.com/dist_transformer/' + url_prefix = 'http://paddle-unittest-data.bj.bcebos.com/dist_transformer/' vocab_url = url_prefix + 'vocab.bpe.32000' vocab_md5 = 'a86d345ca6e27f6591d0dccb1b9be853' paddle.dataset.common.download(vocab_url, 'test_dist_transformer', From e9bec9369b1a690386c20091a9e4a4c5f65326f4 Mon Sep 17 00:00:00 2001 From: whs Date: Tue, 26 Mar 2019 02:48:57 +0800 Subject: [PATCH 39/44] [slim] Add quantization strategy and distillation strategy. (#16408) * Add fsp operator. 1 Add unitest. 2. Add python API. 3. Add layer test. * Add quantization strategy. 1. Add API. 2. Add unitest. * Add distillatoin strategy. * Add unitest config file for quantization * Fix Copyright test=develop * Fix setup.py * Fix document of layers.py. test=develop * Fix unitest in python3. test=develop * Fix documents. test=develop * 1. refine fsp op by batched gemm 2. remove unused import test=develop * Fix test_dist_se_resnext. 1. disable test distillation. 2. reset framework.py test=develop * Enable unitest of distillation after fixing Block._clone_variable test=develop * Fix cdn issue. test=develop --- paddle/fluid/API.spec | 1 + paddle/fluid/operators/fsp_op.cc | 128 +++++++++++ paddle/fluid/operators/fsp_op.cu | 24 ++ paddle/fluid/operators/fsp_op.h | 136 ++++++++++++ .../fluid/contrib/slim/core/compressor.py | 2 +- .../paddle/fluid/contrib/slim/core/config.py | 1 + .../__init__.py | 10 +- .../distillation/distillation_strategy.py | 94 ++++++++ .../contrib/slim/distillation/distiller.py | 188 ++++++++++++++++ .../fluid/contrib/slim/graph/graph_wrapper.py | 10 +- .../contrib/slim/quantization/__init__.py | 4 +- .../quantization/quantization_strategy.py | 209 ++++++++++++++++++ .../slim/tests/distillation/compress.yaml | 46 ++++ .../slim/tests/filter_pruning/compress.yaml | 2 +- .../tests/{filter_pruning => }/mobilenet.py | 39 ++-- .../slim/tests/quantization/compress.yaml | 48 ++++ .../slim/tests/test_distillation_strategy.py | 94 ++++++++ .../contrib/slim/tests/test_filter_pruning.py | 2 +- .../slim/tests/test_quantization_strategy.py | 82 +++++++ python/paddle/fluid/framework.py | 9 +- python/paddle/fluid/layers/nn.py | 44 ++++ .../fluid/tests/unittests/test_fsp_op.py | 60 +++++ .../fluid/tests/unittests/test_layers.py | 9 + python/setup.py.in | 1 + 24 files changed, 1214 insertions(+), 29 deletions(-) create mode 100644 paddle/fluid/operators/fsp_op.cc create mode 100644 paddle/fluid/operators/fsp_op.cu create mode 100644 paddle/fluid/operators/fsp_op.h rename python/paddle/fluid/contrib/slim/{tests/filter_pruning => distillation}/__init__.py (68%) create mode 100644 python/paddle/fluid/contrib/slim/distillation/distillation_strategy.py create mode 100644 python/paddle/fluid/contrib/slim/distillation/distiller.py create mode 100644 python/paddle/fluid/contrib/slim/quantization/quantization_strategy.py create mode 100644 python/paddle/fluid/contrib/slim/tests/distillation/compress.yaml rename python/paddle/fluid/contrib/slim/tests/{filter_pruning => }/mobilenet.py (86%) create mode 100644 python/paddle/fluid/contrib/slim/tests/quantization/compress.yaml create mode 100644 python/paddle/fluid/contrib/slim/tests/test_distillation_strategy.py create mode 100644 python/paddle/fluid/contrib/slim/tests/test_quantization_strategy.py create mode 100644 python/paddle/fluid/tests/unittests/test_fsp_op.py diff --git a/paddle/fluid/API.spec b/paddle/fluid/API.spec index 70a4d7b40b..cfe6730e0c 100644 --- a/paddle/fluid/API.spec +++ b/paddle/fluid/API.spec @@ -222,6 +222,7 @@ paddle.fluid.layers.teacher_student_sigmoid_loss (ArgSpec(args=['input', 'label' paddle.fluid.layers.huber_loss (ArgSpec(args=['input', 'label', 'delta'], varargs=None, keywords=None, defaults=None), ('document', '431a4301c35032166ec029f7432c80a7')) paddle.fluid.layers.tree_conv (ArgSpec(args=['nodes_vector', 'edge_set', 'output_size', 'num_filters', 'max_depth', 'act', 'param_attr', 'bias_attr', 'name'], varargs=None, keywords=None, defaults=(1, 2, 'tanh', None, None, None)), ('document', '34ea12ac9f10a65dccbc50100d12e607')) paddle.fluid.layers.npair_loss (ArgSpec(args=['anchor', 'positive', 'labels', 'l2_reg'], varargs=None, keywords=None, defaults=(0.002,)), ('document', '46994d10276dd4cb803b4062b5d14329')) +paddle.fluid.layers.fsp_matrix (ArgSpec(args=['x', 'y'], varargs=None, keywords=None, defaults=None), ('document', 'b76ccca3735bea4a58a0dbf0d77c5393')) paddle.fluid.layers.data (ArgSpec(args=['name', 'shape', 'append_batch_size', 'dtype', 'lod_level', 'type', 'stop_gradient'], varargs=None, keywords=None, defaults=(True, 'float32', 0, VarType.LOD_TENSOR, True)), ('document', '33bbd42027d872b3818b3d64ec52e139')) paddle.fluid.layers.open_files (ArgSpec(args=['filenames', 'shapes', 'lod_levels', 'dtypes', 'thread_num', 'buffer_size', 'pass_num', 'is_test'], varargs=None, keywords=None, defaults=(None, None, 1, None)), ('document', 'b1ae2e1cc0750e58726374061ea90ecc')) paddle.fluid.layers.read_file (ArgSpec(args=['reader'], varargs=None, keywords=None, defaults=None), ('document', 'b0a1c2fc51c27a106da28f3308c41f5e')) diff --git a/paddle/fluid/operators/fsp_op.cc b/paddle/fluid/operators/fsp_op.cc new file mode 100644 index 0000000000..fbe8e56a61 --- /dev/null +++ b/paddle/fluid/operators/fsp_op.cc @@ -0,0 +1,128 @@ +/* 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/operators/fsp_op.h" + +namespace paddle { +namespace operators { + +class FSPOp : public framework::OperatorWithKernel { + public: + using framework::OperatorWithKernel::OperatorWithKernel; + + void InferShape(framework::InferShapeContext* ctx) const override { + PADDLE_ENFORCE(ctx->HasInput("X"), "Input(X) of FSPOp should not be null."); + PADDLE_ENFORCE(ctx->HasInput("Y"), "Input(Y) of FSPOp should not be null."); + PADDLE_ENFORCE(ctx->HasOutput("Out"), + "Output(Out) of FSPOp should not be null."); + + auto x_dims = ctx->GetInputDim("X"); + auto y_dims = ctx->GetInputDim("Y"); + + PADDLE_ENFORCE( + x_dims.size() == 4, + "The Input(X) must have shape [batch_size, channel, height, width]."); + PADDLE_ENFORCE( + y_dims.size() == 4, + "The Input(Y) must have shape [batch_size, channel, height, width]."); + PADDLE_ENFORCE( + (x_dims[2] == y_dims[2]) && (x_dims[3] == y_dims[3]), + "The Input(X) and Input(Y) should have the same height and width."); + + ctx->SetOutputDim("Out", {x_dims[0], x_dims[1], y_dims[1]}); + ctx->ShareLoD("X", "Out"); + } + + protected: + framework::OpKernelType GetExpectedKernelType( + const framework::ExecutionContext& ctx) const override { + framework::LibraryType library_{framework::LibraryType::kPlain}; + framework::DataLayout layout_ = framework::DataLayout::kAnyLayout; + return framework::OpKernelType(ctx.Input("X")->type(), + ctx.device_context(), layout_, library_); + } +}; + +class FSPOpMaker : public framework::OpProtoAndCheckerMaker { + public: + void Make() override { + AddInput("X", + "(Tensor) The input of FSP op with shape [batch_size, x_channel, " + "height, width]"); + AddInput("Y", + "(Tensor) The input of FSP op with shape" + "[batch_size, y_channel, height, width]." + "The y_channel can be different with the x_channel of Input(X)" + " while the other dimensions must be the same with Input(X)'s."); + AddOutput( + "Out", + "(Tensor) The output of FSP op with shape " + "[batch_size, x_channel, y_channel]. The x_channel is the channel " + "of Input(X) and the y_channel is the channel of Input(Y)."); + AddComment(R"DOC( + This op is used to calculate the flow of solution procedure (FSP) matrix of two feature maps. + Given feature map x with shape [x_channel, h, w] and feature map y with shape + [y_channel, h, w], we can get the fsp matrix of x and y in two steps: + + step 1: reshape x into matrix with shape [x_channel, h * w] and reshape and + transpose y into matrix with shape [h * w, y_channel] + step 2: multiply x and y to get fsp matrix with shape [x_channel, y_channel] + + The output is a batch of fsp matrices. + )DOC"); + } +}; + +class FSPOpGrad : 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->HasInput("Y"), "Input(Y) should not be null"); + PADDLE_ENFORCE(ctx->HasInput(framework::GradVarName("Out")), + "Input(Out@GRAD) should not be null"); + auto x_dims = ctx->GetInputDim("X"); + auto y_dims = ctx->GetInputDim("Y"); + auto x_grad_name = framework::GradVarName("X"); + auto y_grad_name = framework::GradVarName("Y"); + if (ctx->HasOutput(x_grad_name)) { + ctx->SetOutputDim(x_grad_name, x_dims); + } + if (ctx->HasOutput(y_grad_name)) { + ctx->SetOutputDim(y_grad_name, y_dims); + } + } + + framework::OpKernelType GetExpectedKernelType( + const framework::ExecutionContext& ctx) const override { + return framework::OpKernelType( + ctx.Input(framework::GradVarName("Out"))->type(), + ctx.device_context()); + } +}; + +} // namespace operators +} // namespace paddle + +namespace ops = paddle::operators; +REGISTER_OPERATOR(fsp, ops::FSPOp, ops::FSPOpMaker, + paddle::framework::DefaultGradOpDescMaker); +REGISTER_OPERATOR(fsp_grad, ops::FSPOpGrad); +REGISTER_OP_CPU_KERNEL( + fsp, ops::FSPOpKernel, + ops::FSPOpKernel); +REGISTER_OP_CPU_KERNEL( + fsp_grad, ops::FSPGradOpKernel, + ops::FSPGradOpKernel); diff --git a/paddle/fluid/operators/fsp_op.cu b/paddle/fluid/operators/fsp_op.cu new file mode 100644 index 0000000000..4fd7ba04ff --- /dev/null +++ b/paddle/fluid/operators/fsp_op.cu @@ -0,0 +1,24 @@ +/* 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/op_registry.h" +#include "paddle/fluid/operators/fsp_op.h" + +namespace ops = paddle::operators; +namespace plat = paddle::platform; +REGISTER_OP_CUDA_KERNEL(fsp, ops::FSPOpKernel, + ops::FSPOpKernel); +REGISTER_OP_CUDA_KERNEL(fsp_grad, + ops::FSPGradOpKernel, + ops::FSPGradOpKernel); diff --git a/paddle/fluid/operators/fsp_op.h b/paddle/fluid/operators/fsp_op.h new file mode 100644 index 0000000000..544af2b7d9 --- /dev/null +++ b/paddle/fluid/operators/fsp_op.h @@ -0,0 +1,136 @@ +/* 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 "paddle/fluid/framework/op_registry.h" +#include "paddle/fluid/operators/math/blas.h" +#include "paddle/fluid/operators/math/math_function.h" + +namespace paddle { +namespace operators { + +using Tensor = framework::Tensor; + +template +class FSPOpKernel : public framework::OpKernel { + public: + void Compute(const framework::ExecutionContext& context) const override { + auto* x = context.Input("X"); + auto* y = context.Input("Y"); + auto* output = context.Output("Out"); + output->mutable_data(context.GetPlace()); + auto x_dims = x->dims(); + auto y_dims = y->dims(); + + auto batch_size = x_dims[0]; + auto x_channel = x_dims[1]; + auto y_channel = y_dims[1]; + auto height = x_dims[2]; + auto width = x_dims[3]; + + auto blas = math::GetBlas(context); + + math::MatDescriptor x_mat_desc; + x_mat_desc.height_ = x_channel; + x_mat_desc.width_ = height * width; + x_mat_desc.batch_size_ = batch_size; + x_mat_desc.stride_ = x_channel * height * width; + + math::MatDescriptor y_mat_desc; + y_mat_desc.height_ = height * width; + y_mat_desc.width_ = y_channel; + y_mat_desc.batch_size_ = batch_size; + y_mat_desc.stride_ = y_channel * height * width; + y_mat_desc.trans_ = true; + + blas.MatMul(*x, x_mat_desc, *y, y_mat_desc, + static_cast(1.0 / (height * width)), output, + static_cast(0.0)); + } +}; + +template +class FSPGradOpKernel : public framework::OpKernel { + public: + void Compute(const framework::ExecutionContext& context) const override { + auto* d_x = context.Output(framework::GradVarName("X")); + auto* d_y = context.Output(framework::GradVarName("Y")); + if (d_x == nullptr && d_y == nullptr) { + return; + } + auto* d_out = context.Input(framework::GradVarName("Out")); + auto d_out_dims = d_out->dims(); + auto batch_size = d_out_dims[0]; + auto x_channel = d_out_dims[1]; + auto y_channel = d_out_dims[2]; + int64_t h = 0; + int64_t w = 0; + + auto blas = math::GetBlas(context); + math::SetConstant set_zero; + if (d_x != nullptr) { + d_x->mutable_data(context.GetPlace()); + set_zero(context.template device_context(), d_x, + static_cast(0)); + auto* y = context.Input("Y"); + auto y_dims = y->dims(); + h = y_dims[2]; + w = y_dims[3]; + + math::MatDescriptor d_out_mat_desc; + d_out_mat_desc.height_ = x_channel; + d_out_mat_desc.width_ = y_channel; + d_out_mat_desc.batch_size_ = batch_size; + d_out_mat_desc.stride_ = x_channel * y_channel; + + math::MatDescriptor y_mat_desc; + y_mat_desc.height_ = y_channel; + y_mat_desc.width_ = h * w; + y_mat_desc.batch_size_ = batch_size; + y_mat_desc.stride_ = y_channel * h * w; + + blas.MatMul(*d_out, d_out_mat_desc, *y, y_mat_desc, + static_cast(1.0 / (h * w)), d_x, static_cast(0.0)); + } + + if (d_y != nullptr) { + d_y->mutable_data(context.GetPlace()); + set_zero(context.template device_context(), d_y, + static_cast(0)); + auto* x = context.Input("X"); + auto x_dims = x->dims(); + h = x_dims[2]; + w = x_dims[3]; + + math::MatDescriptor d_out_mat_desc; + d_out_mat_desc.height_ = y_channel; + d_out_mat_desc.width_ = x_channel; + d_out_mat_desc.batch_size_ = batch_size; + d_out_mat_desc.stride_ = x_channel * y_channel; + d_out_mat_desc.trans_ = true; + + math::MatDescriptor x_mat_desc; + x_mat_desc.height_ = x_channel; + x_mat_desc.width_ = h * w; + x_mat_desc.batch_size_ = batch_size; + x_mat_desc.stride_ = x_channel * h * w; + + blas.MatMul(*d_out, d_out_mat_desc, *x, x_mat_desc, + static_cast(1.0 / (h * w)), d_y, static_cast(0.0)); + } + } +}; + +} // namespace operators +} // namespace paddle diff --git a/python/paddle/fluid/contrib/slim/core/compressor.py b/python/paddle/fluid/contrib/slim/core/compressor.py index 832ade497c..1547b6abbe 100644 --- a/python/paddle/fluid/contrib/slim/core/compressor.py +++ b/python/paddle/fluid/contrib/slim/core/compressor.py @@ -271,7 +271,7 @@ class Compressor(object): self.eval_reader = eval_reader self.teacher_graphs = [] for teacher in teacher_programs: - self.teacher_graphs.append(ImitationGraph(teacher, scope=scope)) + self.teacher_graphs.append(GraphWrapper(teacher)) self.checkpoint = None self.checkpoint_path = checkpoint_path diff --git a/python/paddle/fluid/contrib/slim/core/config.py b/python/paddle/fluid/contrib/slim/core/config.py index 12df9fcd1b..9bb395aee9 100644 --- a/python/paddle/fluid/contrib/slim/core/config.py +++ b/python/paddle/fluid/contrib/slim/core/config.py @@ -19,6 +19,7 @@ from collections import OrderedDict from ..prune import * from ..quantization import * from .strategy import * +from ..distillation import * __all__ = ['ConfigFactory'] """This factory is used to create instances by loading and parsing configure file with yaml format. diff --git a/python/paddle/fluid/contrib/slim/tests/filter_pruning/__init__.py b/python/paddle/fluid/contrib/slim/distillation/__init__.py similarity index 68% rename from python/paddle/fluid/contrib/slim/tests/filter_pruning/__init__.py rename to python/paddle/fluid/contrib/slim/distillation/__init__.py index d0c32e2609..455c7c5633 100644 --- a/python/paddle/fluid/contrib/slim/tests/filter_pruning/__init__.py +++ b/python/paddle/fluid/contrib/slim/distillation/__init__.py @@ -1,4 +1,4 @@ -# Copyright (c) 2019 PaddlePaddle Authors. All Rights Reserved. +# Copyright (c) 2019 PaddlePaddle Authors. All Rights Reserve. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -11,3 +11,11 @@ # 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 . import distiller +from .distiller import * +from . import distillation_strategy +from .distillation_strategy import * + +__all__ = distiller.__all__ +__all__ += distillation_strategy.__all__ diff --git a/python/paddle/fluid/contrib/slim/distillation/distillation_strategy.py b/python/paddle/fluid/contrib/slim/distillation/distillation_strategy.py new file mode 100644 index 0000000000..1f11f07a51 --- /dev/null +++ b/python/paddle/fluid/contrib/slim/distillation/distillation_strategy.py @@ -0,0 +1,94 @@ +# 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. + +from ..core.strategy import Strategy +from ....framework import Program, program_guard +from .... import Executor +import logging + +__all__ = ['DistillationStrategy'] + +logging.basicConfig(format='%(asctime)s-%(levelname)s: %(message)s') +_logger = logging.getLogger(__name__) +_logger.setLevel(logging.INFO) + + +class DistillationStrategy(Strategy): + def __init__(self, distillers=None, start_epoch=0, end_epoch=0): + """ + Args: + distillers(list): A list of distiller used to combine student graph and teacher graph + by adding some loss. + start_epoch(int): The epoch when to merge student graph and teacher graph for + distillation training. default: 0 + end_epoch(int): The epoch when to finish distillation training. default: 0 + + """ + super(DistillationStrategy, self).__init__(start_epoch, end_epoch) + self.distillers = distillers + + def on_compression_begin(self, context): + # load from checkpoint + if context.epoch_id > 0: + if context.epoch_id > self.start_epoch and context.epoch_id < self.end_epoch: + _logger.info('Restore DistillationStrategy') + self._create_distillation_graph(context) + _logger.info('Restore DistillationStrategy finish.') + + def on_epoch_begin(self, context): + if self.start_epoch == context.epoch_id: + _logger.info('DistillationStrategy::on_epoch_begin.') + self._create_distillation_graph(context) + _logger.info('DistillationStrategy set optimize_graph.') + + def _create_distillation_graph(self, context): + """ + step 1: Merge student graph and teacher graph into distillation graph. + step 2: Add loss into distillation graph by distillers. + step 3: Append backward ops and optimize ops into distillation graph for training. + """ + # step 1 + teacher = context.teacher_graphs[0] + for var in teacher.program.list_vars(): + var.stop_gradient = True + graph = context.train_graph.clone() + graph.merge(teacher) + graph.out_nodes['student_loss'] = graph.out_nodes['loss'] + + # step 2 + for distiller in self.distillers: + graph = distiller.distiller_loss(graph) + + # step 3 + startup_program = Program() + with program_guard(graph.program, startup_program): + context.distiller_optimizer._name = 'distillation_optimizer' + context.distiller_optimizer.minimize( + graph.var(graph.out_nodes['loss'])._var) + exe = Executor(context.place) + exe.run(startup_program, scope=context.scope) + + # backup graph for fine-tune after distillation + context.put('distillation_backup_optimize_graph', + context.optimize_graph) + context.optimize_graph = graph + + def on_epoch_end(self, context): + if context.epoch_id == (self.end_epoch - 1): + _logger.info('DistillationStrategy::on_epoch_end.') + # restore optimize_graph for fine-tune or other strategy in next stage. + context.optimize_graph = context.get( + 'distillation_backup_optimize_graph') + _logger.info( + 'DistillationStrategy set context.optimize_graph to None.') diff --git a/python/paddle/fluid/contrib/slim/distillation/distiller.py b/python/paddle/fluid/contrib/slim/distillation/distiller.py new file mode 100644 index 0000000000..13bb35a8be --- /dev/null +++ b/python/paddle/fluid/contrib/slim/distillation/distiller.py @@ -0,0 +1,188 @@ +# 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. + +from .... import layers +from .... import optimizer +from .... import Executor +from .... import Program +from .... import program_guard +from .... import regularizer + +__all__ = ['FSPDistiller', 'L2Distiller'] + + +class L2Distiller(object): + """ + Combine two layers from student net and teacher net by l2-loss. + And add the loss into the total loss using for distillation training. + """ + + def __init__(self, + student_feature_map, + teacher_feature_map, + distillation_loss_weight=1): + """ + Args: + student_feature_map(str): The name of feature map from student network. + teacher_feature_map(str): The name of feature map from teacher network. + It's shape should be the same with student network. + distillation_loss_weight(float): The weight of the l2-loss. + """ + self.student_feature_map = student_feature_map + self.teacher_feature_map = teacher_feature_map + self.distillation_loss_weight = distillation_loss_weight + + def distiller_loss(self, graph): + """ + Modify graph inplace to add l2-loss. + Args: + graph(GraphWrapper): The graph to be modified. + Returns: + GraphWrapper: The modified graph. + """ + distiller_pass = L2DistillerPass(self.student_feature_map, + self.teacher_feature_map, + self.distillation_loss_weight) + dis_graph = distiller_pass.apply(graph) + return dis_graph + + +class L2DistillerPass(object): + """ + The pass used to add l2-loss. + """ + + def __init__(self, + student_feature_map, + teacher_feature_map, + distillation_loss_weight=1): + """ + Args: + student_feature_map(str): The name of feature map from student network. + teacher_feature_map(str): The name of feature map from teacher network. + It's shape should be the same with student network. + distillation_loss_weight(float): The weight of the l2-loss. + """ + self.student_feature_map = student_feature_map + self.teacher_feature_map = teacher_feature_map + self.distillation_loss_weight = distillation_loss_weight + + def apply(self, graph): + ret_graph = graph + with program_guard(ret_graph.program): + + student_feature_map = ret_graph.var(self.student_feature_map)._var + teacher_feature_map = ret_graph.var(self.teacher_feature_map)._var + l2loss = layers.reduce_mean( + layers.square(student_feature_map - teacher_feature_map)) + + distillation_loss = l2loss * self.distillation_loss_weight + student_loss = ret_graph.var(ret_graph.out_nodes['loss'])._var + loss = distillation_loss + student_loss + + ret_graph.out_nodes[ + 'l2loss_' + self.student_feature_map + "_" + + self.teacher_feature_map] = distillation_loss.name + ret_graph.out_nodes['loss'] = loss.name + return ret_graph + + +class FSPDistiller(object): + """ + Combine layers from student net and teacher net by fsp-loss. + """ + + def __init__(self, student_pairs, teacher_pairs, + distillation_loss_weight=1): + """ + Args: + student_pairs(list): Each tuple, with two variable names, in student_pairs indicates + a section in student network. The variables in a tuple should + have the same feature map size. + teacher_pairs(list): Each tuple, with two variable names, in teacher_pairs indicates + a section in teacher network. The variables in a tuple should + have the same feature map size. Varibale named teacher_pairs[i][j] + should has the save channel number with that of variable named + student_pairs[i][j]. + + distillation_loss_weight(float): The weight of the fsp-loss. default: 1. + """ + self.student_pairs = student_pairs + self.teacher_pairs = teacher_pairs + self.distillation_loss_weight = distillation_loss_weight + + def distiller_loss(self, graph): + """ + Modify graph inplace to add fsp-loss. + Args: + graph(GraphWrapper): The graph to be modified. + Returns: + GraphWrapper: The modified graph. + """ + distiller_pass = FSPDistillerPass(self.student_pairs, + self.teacher_pairs, + self.distillation_loss_weight) + dis_graph = distiller_pass.apply(graph) + return dis_graph + + +class FSPDistillerPass(object): + ''' + Combine layers from student net and teacher net by fsp-loss. + ''' + + def __init__(self, s_pairs, t_pairs, distillation_loss_weight=1): + """ + Args: + s_pairs(list): Each tuple, with two variable names, in student_pairs indicates + a section in student network. The variables in a tuple should + have the same feature map size. + t_pairs(list): Each tuple, with two variable names, in teacher_pairs indicates + a section in teacher network. The variables in a tuple should + have the same feature map size. Varibale named teacher_pairs[i][j] + should has the save channel number with that of variable named + student_pairs[i][j]. + + distillation_loss_weight(float): The weight of the fsp-loss. default: 1. + """ + self.s_pairs = s_pairs + self.t_pairs = t_pairs + self.distillation_loss_weight = distillation_loss_weight + + def apply(self, graph): + ret_graph = graph + with program_guard(ret_graph.program): + losses = [] + for s_pair, t_pair in zip(self.s_pairs, self.t_pairs): + s_pair_start = ret_graph.var(s_pair[0])._var + s_pair_end = ret_graph.var(s_pair[1])._var + s_fsp_matrix = self._fsp_matrix(s_pair_start, s_pair_end) + t_pair_start = ret_graph.var(t_pair[0])._var + t_pair_end = ret_graph.var(t_pair[1])._var + t_fsp_matrix = self._fsp_matrix(t_pair_start, t_pair_end) + l2_loss = layers.reduce_mean( + layers.square(s_fsp_matrix - t_fsp_matrix)) + losses.append(l2_loss) + distillation_loss = layers.sum( + losses) * self.distillation_loss_weight + student_loss = ret_graph.var(ret_graph.out_nodes['loss'])._var + loss = distillation_loss + student_loss + + ret_graph.out_nodes[ + 'fsp_distillation_loss'] = distillation_loss.name + ret_graph.out_nodes['loss'] = loss.name + return ret_graph + + def _fsp_matrix(self, fea_map_0, fea_map_1): + return layers.fsp_matrix(fea_map_0, fea_map_1) diff --git a/python/paddle/fluid/contrib/slim/graph/graph_wrapper.py b/python/paddle/fluid/contrib/slim/graph/graph_wrapper.py index 8694be7827..c208553fd8 100644 --- a/python/paddle/fluid/contrib/slim/graph/graph_wrapper.py +++ b/python/paddle/fluid/contrib/slim/graph/graph_wrapper.py @@ -300,7 +300,9 @@ class GraphWrapper(object): graph(GraphWrapper): The graph to be merged by current graph. """ for var in graph.program.list_vars(): - self.program.global_block()._clone_variable(var) + new_var = self.program.global_block()._clone_variable( + var, force_persistable=False) + new_var.stop_gradient = var.stop_gradient # TODO: parameters should be cloned for op in graph.ops(): op = op._op @@ -309,12 +311,12 @@ class GraphWrapper(object): attrs = {} for input_name in op.input_names: inputs[input_name] = [ - self.var(in_var_name) - for in_var_name in op.inputs(input_name) + self.var(in_var_name)._var + for in_var_name in op.input(input_name) ] for output_name in op.output_names: outputs[output_name] = [ - self.var(out_var_name) + self.var(out_var_name)._var for out_var_name in op.output(output_name) ] for attr_name in op.attr_names: diff --git a/python/paddle/fluid/contrib/slim/quantization/__init__.py b/python/paddle/fluid/contrib/slim/quantization/__init__.py index 6c26475f48..1c51aa1537 100644 --- a/python/paddle/fluid/contrib/slim/quantization/__init__.py +++ b/python/paddle/fluid/contrib/slim/quantization/__init__.py @@ -16,5 +16,7 @@ from __future__ import print_function from . import quantization_pass from .quantization_pass import * +from . import quantization_strategy +from .quantization_strategy import * -__all__ = quantization_pass.__all__ +__all__ = quantization_pass.__all__ + quantization_strategy.__all__ diff --git a/python/paddle/fluid/contrib/slim/quantization/quantization_strategy.py b/python/paddle/fluid/contrib/slim/quantization/quantization_strategy.py new file mode 100644 index 0000000000..6812b4c633 --- /dev/null +++ b/python/paddle/fluid/contrib/slim/quantization/quantization_strategy.py @@ -0,0 +1,209 @@ +# 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 logging +import sys +import numpy as np +from .... import Executor +from .... import io +from .... import core +from ....compiler import CompiledProgram +from ....compiler import BuildStrategy +from ....framework import IrGraph +from ..core.strategy import Strategy +from .quantization_pass import * + +__all__ = ['QuantizationStrategy'] + +logging.basicConfig(format='%(asctime)s-%(levelname)s: %(message)s') +_logger = logging.getLogger(__name__) +_logger.setLevel(logging.INFO) + + +class QuantizationStrategy(Strategy): + """ + The strategy for Quantization. + """ + + def __init__(self, + start_epoch=0, + end_epoch=0, + float_model_save_path=None, + mobile_model_save_path=None, + int8_model_save_path=None, + activation_bits=8, + weight_bits=8, + activation_quantize_type='abs_max', + save_in_nodes=None, + save_out_nodes=None): + """ + Args: + start_epoch(int): The 'on_epoch_begin' function will be called in start_epoch. default: 0 + end_epoch(int): The 'on_epoch_end' function will be called in end_epoch. default: 0 + float_model_save_path(str): The path to save model with float weights. + None means it doesn't save float model. defalut: None. + mobile_model_save_path(str): The path to save model for paddle-mobile execution. + None means it doesn't save mobile model. defalut: None. + int8_model_save_path(str): The path to save model with int8_t weight. + None means it doesn't save int8 model. defalut: None. + activation_bits(int): quantization bit number for activation. default: 8. + weight_bits(int): quantization bit number for weights. The bias is not quantized. + default: 8. + activation_quantize_type(str): quantization type for activation, + now support 'abs_max', 'range_abs_max' and 'moving_average_abs_max'. + If use 'abs_max' mode, the quantization scale will be calculated + dynamically each step in both training and testing period. If use + 'range_abs_max', a static quantization scale will be calculated + during training and used in inference. + save_in_nodes(list): A list of variable names used to prune graph + for saving inference model. + save_out_nodes(list): A list of variable names used to prune graph + for saving inference model. + + """ + super(QuantizationStrategy, self).__init__(start_epoch, end_epoch) + self.start_epoch = start_epoch + self.end_epoch = end_epoch + self.float_model_save_path = float_model_save_path + self.mobile_model_save_path = mobile_model_save_path + self.int8_model_save_path = int8_model_save_path + self.activation_bits = activation_bits + self.weight_bits = weight_bits + self.activation_quantize_type = activation_quantize_type + self.save_out_nodes = save_out_nodes + self.save_in_nodes = save_in_nodes + + def on_epoch_begin(self, context): + """ + Insert fake_quantize_op and fake_dequantize_op before trainging and testing. + """ + super(QuantizationStrategy, self).on_compression_begin(context) + if self.start_epoch == context.epoch_id: + _logger.info('QuantizationStrategy::on_epoch_begin') + train_ir_graph = IrGraph( + core.Graph(context.optimize_graph.program.desc), for_test=False) + test_ir_graph = IrGraph( + core.Graph(context.eval_graph.program.desc), for_test=True) + transform_pass = QuantizationTransformPass( + scope=context.scope, + place=context.place, + weight_bits=self.weight_bits, + activation_bits=self.activation_bits, + activation_quantize_type=self.activation_quantize_type) + transform_pass.apply(train_ir_graph) + transform_pass.apply(test_ir_graph) + + build_strategy = BuildStrategy() + build_strategy.enable_inplace = False + build_strategy.memory_optimize = False + # for quantization training + context.optimize_graph.compiled_graph = CompiledProgram( + train_ir_graph.graph).with_data_parallel( + loss_name=context.optimize_graph.out_nodes['loss'], + build_strategy=build_strategy) + # for evaluation. And program compiled from ir graph must be with data parallel. + context.eval_graph.compiled_graph = CompiledProgram( + test_ir_graph.graph).with_data_parallel( + build_strategy=build_strategy) + # for saving inference model after training + context.put('quantization_test_ir_graph_backup', test_ir_graph) + _logger.info('Finish QuantizationStrategy::on_epoch_begin') + + def on_epoch_end(self, context): + """ + Free and save inference model. + """ + super(QuantizationStrategy, self).on_compression_end(context) + + if context.epoch_id == self.end_epoch: + _logger.info('QuantizationStrategy::on_epoch_end') + test_ir_graph = context.get('quantization_test_ir_graph_backup') + # freeze the graph after training + freeze_pass = QuantizationFreezePass( + scope=context.scope, + place=context.place, + weight_bits=self.weight_bits, + activation_bits=self.activation_bits) + freeze_pass.apply(test_ir_graph) + + # for other strategies + context.eval_graph.program = test_ir_graph.to_program() + + if self.save_out_nodes == None: + out_vars = [ + context.eval_graph.var(var_name)._var + for var_name in context.eval_graph.out_nodes.values() + ] + else: + out_vars = [ + context.eval_graph.var(var_name)._var + for var_name in self.save_out_nodes + ] + + if self.save_in_nodes == None: + in_vars = list(context.eval_graph.out_nodes.values()) + else: + in_vars = self.save_in_nodes + + # save float model + if self.float_model_save_path: + executor = Executor(context.place) + io.save_inference_model( + self.float_model_save_path, + in_vars, + out_vars, + executor, + main_program=test_ir_graph.to_program(), + model_filename='model', + params_filename='weights', + export_for_deployment=True) + + # save int8 model + if self.int8_model_save_path: + convert_int8_pass = ConvertToInt8Pass( + scope=context.scope, place=context.place) + convert_int8_pass.apply(test_ir_graph) + + executor = Executor(context.place) + io.save_inference_model( + self.int8_model_save_path, + in_vars, + out_vars, + executor, + main_program=test_ir_graph.to_program(), + model_filename='model', + params_filename='weights', + export_for_deployment=True) + + # save mobile model + if self.mobile_model_save_path: + if not self.int8_model_save_path: + # convert the weights as int8_t type + convert_int8_pass = ConvertToInt8Pass( + scope=context.scope, place=context.place) + convert_int8_pass.apply(test_ir_graph) + # make some changes on the graph for the mobile inference + mobile_pass = TransformForMobilePass() + mobile_pass.apply(test_ir_graph) + executor = Executor(context.place) + io.save_inference_model( + self.mobile_model_save_path, + in_vars, + out_vars, + executor, + main_program=test_ir_graph.to_program(), + model_filename='model', + params_filename='weights', + export_for_deployment=True) + _logger.info('Finish QuantizationStrategy::on_epoch_end') diff --git a/python/paddle/fluid/contrib/slim/tests/distillation/compress.yaml b/python/paddle/fluid/contrib/slim/tests/distillation/compress.yaml new file mode 100644 index 0000000000..ef89dfb780 --- /dev/null +++ b/python/paddle/fluid/contrib/slim/tests/distillation/compress.yaml @@ -0,0 +1,46 @@ +#start_epoch(int): The epoch when to merge student graph and teacher graph for +# distillation training. default: 0 +# +#end_epoch(int): The epoch when to finish distillation training. default: 0 +# +#student_feature_map(str): The name of feature map from student network. +# +#teacher_feature_map(str): The name of feature map from teacher network. +# It's shape should be the same with student network. +# +#student_pairs(list): Each tuple, with two variable names, in student_pairs indicates +# a section in student network. The variables in a tuple should +# have the same feature map size. +# +#teacher_pairs(list): Each tuple, with two variable names, in teacher_pairs indicates +# a section in teacher network. The variables in a tuple should +# have the same feature map size. Varibale named teacher_pairs[i][j] +# should has the save channel number with that of variable named +# student_pairs[i][j]. +# +#distillation_loss_weight(float): The weight of the loss. +version: 1.0 +distillers: + fsp_distiller: + class: 'FSPDistiller' +# teacher_pairs: [['teacher_depthwise_conv2d_1.tmp_0', 'teacher_conv2d_3.tmp_0']] +# student_pairs: [['student_depthwise_conv2d_1.tmp_0', 'student_conv2d_3.tmp_0']] + teacher_pairs: [['teacher_conv2_1_dw.tmp_0', 'teacher_conv1.tmp_0']] + student_pairs: [['student_conv2_1_dw.tmp_0', 'student_conv1.tmp_0']] + distillation_loss_weight: 1 + l2_distiller: + class: 'L2Distiller' + teacher_feature_map: 'teacher.tmp_2' + student_feature_map: 'student.tmp_2' + distillation_loss_weight: 1 +strategies: + distillation_strategy: + class: 'DistillationStrategy' + distillers: ['fsp_distiller', 'l2_distiller'] + start_epoch: 0 + end_epoch: 1 +compressor: + epoch: 1 + checkpoint_path: './distillation_checkpoints/' + strategies: + - distillation_strategy diff --git a/python/paddle/fluid/contrib/slim/tests/filter_pruning/compress.yaml b/python/paddle/fluid/contrib/slim/tests/filter_pruning/compress.yaml index 232276feac..5f747a049e 100644 --- a/python/paddle/fluid/contrib/slim/tests/filter_pruning/compress.yaml +++ b/python/paddle/fluid/contrib/slim/tests/filter_pruning/compress.yaml @@ -29,6 +29,6 @@ strategies: metric_name: 'acc_top1' compressor: epoch: 2 - checkpoint_path: './checkpoints/' + checkpoint_path: './checkpoints_pruning/' strategies: - sensitive_pruning_strategy diff --git a/python/paddle/fluid/contrib/slim/tests/filter_pruning/mobilenet.py b/python/paddle/fluid/contrib/slim/tests/mobilenet.py similarity index 86% rename from python/paddle/fluid/contrib/slim/tests/filter_pruning/mobilenet.py rename to python/paddle/fluid/contrib/slim/tests/mobilenet.py index 0148325a64..f5dbef17e8 100644 --- a/python/paddle/fluid/contrib/slim/tests/filter_pruning/mobilenet.py +++ b/python/paddle/fluid/contrib/slim/tests/mobilenet.py @@ -35,8 +35,9 @@ train_parameters = { class MobileNet(): - def __init__(self): + def __init__(self, name=""): self.params = train_parameters + self.name = name def net(self, input, class_dim=1000, scale=1.0): # conv1: 112x112 @@ -47,7 +48,7 @@ class MobileNet(): num_filters=int(32 * scale), stride=2, padding=1, - name="conv1") + name=self.name + "_conv1") # 56x56 input = self.depthwise_separable( @@ -57,7 +58,7 @@ class MobileNet(): num_groups=32, stride=1, scale=scale, - name="conv2_1") + name=self.name + "_conv2_1") input = self.depthwise_separable( input, @@ -66,7 +67,7 @@ class MobileNet(): num_groups=64, stride=2, scale=scale, - name="conv2_2") + name=self.name + "_conv2_2") # 28x28 input = self.depthwise_separable( @@ -76,7 +77,7 @@ class MobileNet(): num_groups=128, stride=1, scale=scale, - name="conv3_1") + name=self.name + "_conv3_1") input = self.depthwise_separable( input, @@ -85,7 +86,7 @@ class MobileNet(): num_groups=128, stride=2, scale=scale, - name="conv3_2") + name=self.name + "_conv3_2") # 14x14 input = self.depthwise_separable( @@ -95,7 +96,7 @@ class MobileNet(): num_groups=256, stride=1, scale=scale, - name="conv4_1") + name=self.name + "_conv4_1") input = self.depthwise_separable( input, @@ -104,7 +105,7 @@ class MobileNet(): num_groups=256, stride=2, scale=scale, - name="conv4_2") + name=self.name + "_conv4_2") # 14x14 for i in range(5): @@ -115,7 +116,7 @@ class MobileNet(): num_groups=512, stride=1, scale=scale, - name="conv5" + "_" + str(i + 1)) + name=self.name + "_conv5" + "_" + str(i + 1)) # 7x7 input = self.depthwise_separable( input, @@ -124,7 +125,7 @@ class MobileNet(): num_groups=512, stride=2, scale=scale, - name="conv5_6") + name=self.name + "_conv5_6") input = self.depthwise_separable( input, @@ -133,7 +134,7 @@ class MobileNet(): num_groups=1024, stride=1, scale=scale, - name="conv6") + name=self.name + "_conv6") input = fluid.layers.pool2d( input=input, @@ -142,12 +143,14 @@ class MobileNet(): pool_type='avg', global_pooling=True) - output = fluid.layers.fc(input=input, - size=class_dim, - act='softmax', - param_attr=ParamAttr( - initializer=MSRA(), name="fc7_weights"), - bias_attr=ParamAttr(name="fc7_offset")) + output = fluid.layers.fc( + input=input, + size=class_dim, + act='softmax', + param_attr=ParamAttr( + initializer=MSRA(), name=self.name + "_fc7_weights"), + bias_attr=ParamAttr(name=self.name + "_fc7_offset"), + name=self.name) return output def conv_bn_layer(self, @@ -172,11 +175,13 @@ class MobileNet(): use_cudnn=use_cudnn, param_attr=ParamAttr( initializer=MSRA(), name=name + "_weights"), + name=name, bias_attr=False) bn_name = name + "_bn" return fluid.layers.batch_norm( input=conv, act=act, + name=name, param_attr=ParamAttr(name=bn_name + "_scale"), bias_attr=ParamAttr(name=bn_name + "_offset"), moving_mean_name=bn_name + '_mean', diff --git a/python/paddle/fluid/contrib/slim/tests/quantization/compress.yaml b/python/paddle/fluid/contrib/slim/tests/quantization/compress.yaml new file mode 100644 index 0000000000..f29eb53f88 --- /dev/null +++ b/python/paddle/fluid/contrib/slim/tests/quantization/compress.yaml @@ -0,0 +1,48 @@ +#start_epoch(int): The epoch to insert quantization operators. default: 0 +# +#end_epoch(int): The epoch to save inferecne model. default: 0 +# +#float_model_save_path(str): The path to save model with float weights. +# None means it doesn't save float model. defalut: None. +# +#mobile_model_save_path(str): The path to save model for paddle-mobile execution. +# None means it doesn't save mobile model. defalut: None. +# +#int8_model_save_path(str): The path to save model with int8_t weight. +# None means it doesn't save int8 model. defalut: None. +# +#activation_bits(int): quantization bit number for activation. default: 8. +# +#weight_bits(int): quantization bit number for weights. The bias is not quantized. +# default: 8. +# +#activation_quantize_type(str): quantization type for activation, +# now support 'abs_max', 'range_abs_max' and 'moving_average_abs_max'. +# If use 'abs_max' mode, the quantization scale will be calculated +# dynamically each step in both training and testing period. If use +# 'range_abs_max', a static quantization scale will be calculated +# during training and used in inference. +# +#save_in_nodes(list): A list of variable names used to prune graph +# for saving inference model. +# +#save_out_nodes(list): A list of variable names used to prune graph +# for saving inference model. +version: 1.0 +strategies: + quantization_strategy: + class: 'QuantizationStrategy' + start_epoch: 0 + end_epoch: 0 + float_model_save_path: './output/float' + weight_bits: 8 + activation_bits: 8 + weight_quantize_type: 'abs_max' + activation_quantize_type: 'abs_max' + save_in_nodes: ['image'] + save_out_nodes: ['quan.tmp_2'] +compressor: + epoch: 1 + checkpoint_path: './checkpoints_quan/' + strategies: + - quantization_strategy diff --git a/python/paddle/fluid/contrib/slim/tests/test_distillation_strategy.py b/python/paddle/fluid/contrib/slim/tests/test_distillation_strategy.py new file mode 100644 index 0000000000..9b967c0ac7 --- /dev/null +++ b/python/paddle/fluid/contrib/slim/tests/test_distillation_strategy.py @@ -0,0 +1,94 @@ +# 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 paddle +import unittest +import paddle.fluid as fluid +from mobilenet import MobileNet +from paddle.fluid.contrib.slim.core import Compressor +from paddle.fluid.contrib.slim.graph import GraphWrapper + + +class TestDistillationStrategy(unittest.TestCase): + """ + Test API of distillation strategy. + """ + + def test_compression(self): + if not fluid.core.is_compiled_with_cuda(): + return + class_dim = 10 + image_shape = [1, 28, 28] + image = fluid.layers.data( + name='image', shape=image_shape, dtype='float32') + image.stop_gradient = False + label = fluid.layers.data(name='label', shape=[1], dtype='int64') + out = MobileNet(name="student").net(input=image, class_dim=class_dim) + acc_top1 = fluid.layers.accuracy(input=out, label=label, k=1) + acc_top5 = fluid.layers.accuracy(input=out, label=label, k=5) + val_program = fluid.default_main_program().clone(for_test=False) + + cost = fluid.layers.cross_entropy(input=out, label=label) + avg_cost = fluid.layers.mean(x=cost) + optimizer = fluid.optimizer.Momentum( + momentum=0.9, + learning_rate=0.01, + regularization=fluid.regularizer.L2Decay(4e-5)) + + place = fluid.CUDAPlace(0) + exe = fluid.Executor(place) + exe.run(fluid.default_startup_program()) + + val_reader = paddle.batch(paddle.dataset.mnist.test(), batch_size=128) + + val_feed_list = [('img', image.name), ('label', label.name)] + val_fetch_list = [('acc_top1', acc_top1.name), ('acc_top5', + acc_top5.name)] + + train_reader = paddle.batch( + paddle.dataset.mnist.train(), batch_size=128) + train_feed_list = [('img', image.name), ('label', label.name)] + train_fetch_list = [('loss', avg_cost.name)] + + # define teacher program + teacher_program = fluid.Program() + startup_program = fluid.Program() + with fluid.program_guard(teacher_program, startup_program): + img = teacher_program.global_block()._clone_variable( + image, force_persistable=False) + predict = MobileNet(name="teacher").net(input=img, + class_dim=class_dim) + + exe.run(startup_program) + + com_pass = Compressor( + place, + fluid.global_scope(), + fluid.default_main_program(), + train_reader=train_reader, + train_feed_list=train_feed_list, + train_fetch_list=train_fetch_list, + eval_program=val_program, + eval_reader=val_reader, + eval_feed_list=val_feed_list, + eval_fetch_list=val_fetch_list, + teacher_programs=[teacher_program.clone(for_test=True)], + train_optimizer=optimizer, + distiller_optimizer=optimizer) + com_pass.config('./distillation/compress.yaml') + eval_graph = com_pass.run() + + +if __name__ == '__main__': + unittest.main() diff --git a/python/paddle/fluid/contrib/slim/tests/test_filter_pruning.py b/python/paddle/fluid/contrib/slim/tests/test_filter_pruning.py index d73ee27779..e1763039b3 100644 --- a/python/paddle/fluid/contrib/slim/tests/test_filter_pruning.py +++ b/python/paddle/fluid/contrib/slim/tests/test_filter_pruning.py @@ -15,7 +15,7 @@ import paddle import unittest import paddle.fluid as fluid -from filter_pruning.mobilenet import MobileNet +from mobilenet import MobileNet from paddle.fluid.contrib.slim.core import Compressor from paddle.fluid.contrib.slim.graph import GraphWrapper diff --git a/python/paddle/fluid/contrib/slim/tests/test_quantization_strategy.py b/python/paddle/fluid/contrib/slim/tests/test_quantization_strategy.py new file mode 100644 index 0000000000..92afd892af --- /dev/null +++ b/python/paddle/fluid/contrib/slim/tests/test_quantization_strategy.py @@ -0,0 +1,82 @@ +# 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 paddle +import unittest +import paddle.fluid as fluid +from mobilenet import MobileNet +from paddle.fluid.contrib.slim.core import Compressor +from paddle.fluid.contrib.slim.graph import GraphWrapper + + +class TestQuantizationStrategy(unittest.TestCase): + """ + Test API of quantization strategy. + """ + + def test_compression(self): + if not fluid.core.is_compiled_with_cuda(): + return + class_dim = 10 + image_shape = [1, 28, 28] + image = fluid.layers.data( + name='image', shape=image_shape, dtype='float32') + image.stop_gradient = False + label = fluid.layers.data(name='label', shape=[1], dtype='int64') + out = MobileNet(name='quan').net(input=image, class_dim=class_dim) + acc_top1 = fluid.layers.accuracy(input=out, label=label, k=1) + acc_top5 = fluid.layers.accuracy(input=out, label=label, k=5) + val_program = fluid.default_main_program().clone(for_test=False) + + cost = fluid.layers.cross_entropy(input=out, label=label) + avg_cost = fluid.layers.mean(x=cost) + + optimizer = fluid.optimizer.Momentum( + momentum=0.9, + learning_rate=0.01, + regularization=fluid.regularizer.L2Decay(4e-5)) + + place = fluid.CUDAPlace(0) + exe = fluid.Executor(place) + exe.run(fluid.default_startup_program()) + + val_reader = paddle.batch(paddle.dataset.mnist.test(), batch_size=128) + + val_feed_list = [('img', image.name), ('label', label.name)] + val_fetch_list = [('acc_top1', acc_top1.name), ('acc_top5', + acc_top5.name)] + + train_reader = paddle.batch( + paddle.dataset.mnist.train(), batch_size=128) + train_feed_list = [('img', image.name), ('label', label.name)] + train_fetch_list = [('loss', avg_cost.name)] + + com_pass = Compressor( + place, + fluid.global_scope(), + fluid.default_main_program(), + train_reader=train_reader, + train_feed_list=train_feed_list, + train_fetch_list=train_fetch_list, + eval_program=val_program, + eval_reader=val_reader, + eval_feed_list=val_feed_list, + eval_fetch_list=val_fetch_list, + train_optimizer=optimizer) + com_pass.config('./quantization/compress.yaml') + eval_graph = com_pass.run() + + +if __name__ == '__main__': + unittest.main() diff --git a/python/paddle/fluid/framework.py b/python/paddle/fluid/framework.py index e4169c247f..f3d876f141 100644 --- a/python/paddle/fluid/framework.py +++ b/python/paddle/fluid/framework.py @@ -1559,12 +1559,15 @@ class Block(object): name=v.name) self.vars[new_p.name] = new_p - def _clone_variable(self, var): + def _clone_variable(self, var, force_persistable=True): """ Clone a variable into current block. Args: var: the variable to be cloned. + force_persistable(bool): True means setting the result variable to being persistable. + False means setting the persistable the same with that of input var. + default: True. Returns: Variable: the new variable cloned from 'var' in current block. @@ -1584,7 +1587,7 @@ class Block(object): shape=var.shape, dtype=var.dtype, type=var.type, - persistable=True, + persistable=True if force_persistable else var.persistable, is_data=var.is_data) else: ret_var = self.create_var( @@ -1593,7 +1596,7 @@ class Block(object): dtype=var.dtype, type=var.type, lod_level=var.lod_level, - persistable=True, + persistable=True if force_persistable else var.persistable, is_data=var.is_data) return ret_var diff --git a/python/paddle/fluid/layers/nn.py b/python/paddle/fluid/layers/nn.py index e2c8be613f..c4e6053fec 100644 --- a/python/paddle/fluid/layers/nn.py +++ b/python/paddle/fluid/layers/nn.py @@ -189,6 +189,7 @@ __all__ = [ 'huber_loss', 'tree_conv', 'npair_loss', + 'fsp_matrix', ] kIgnoreIndex = -100 @@ -10790,3 +10791,46 @@ def npair_loss(anchor, positive, labels, l2_reg=0.002): celoss = reduce_mean(cross_entropy) return l2loss + celoss + + +def fsp_matrix(x, y): + """ + + **FSP matrix op** + + This op is used to calculate the flow of solution procedure (FSP) matrix of two feature maps. + Given feature map x with shape [x_channel, h, w] and feature map y with shape + [y_channel, h, w], we can get the fsp matrix of x and y in two steps: + + 1. reshape x into matrix with shape [x_channel, h * w] and reshape and + transpose y into matrix with shape [h * w, y_channel]. + 2. multiply x and y to get fsp matrix with shape [x_channel, y_channel]. + + The output is a batch of fsp matrices. + + Args: + + x (Variable): A feature map with shape [batch_size, x_channel, height, width]. + y (Variable): A feature map with shape [batch_size, y_channel, height, width]. + The y_channel can be different with the x_channel of Input(X) + while the other dimensions must be the same with Input(X)'s. + + Returns: + + fsp matrix (Variable): The output of FSP op with shape [batch_size, x_channel, y_channel]. + The x_channel is the channel of x and the y_channel is the channel of y. + + Examples: + + .. code-block:: python + + feature_map_0 = fluid.layers.conv2d(x) + feature_map_1 = fluid.layers.conv2d(feature_map_0) + loss = fluid.layers.fsp_matrix(feature_map_0, feature_map_1) + + """ + helper = LayerHelper('fsp_matrix', **locals()) + out = helper.create_variable_for_type_inference(dtype=helper.input_dtype( + input_param_name='x')) + helper.append_op(type='fsp', inputs={'X': x, 'Y': y}, outputs={'Out': out}) + return out diff --git a/python/paddle/fluid/tests/unittests/test_fsp_op.py b/python/paddle/fluid/tests/unittests/test_fsp_op.py new file mode 100644 index 0000000000..6ad7418447 --- /dev/null +++ b/python/paddle/fluid/tests/unittests/test_fsp_op.py @@ -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. + +import unittest +import numpy as np +from op_test import OpTest + + +def fsp_matrix(a, b): + batch = a.shape[0] + a_channel = a.shape[1] + b_channel = b.shape[1] + h = a.shape[2] + w = a.shape[3] + a_t = a.transpose([0, 2, 3, 1]) + a_t = a_t.reshape([batch, h * w, a_channel]) + b_t = b.transpose([0, 2, 3, 1]).reshape([batch, h * w, b_channel]) + a_r = a_t.repeat( + b_channel, axis=1).reshape( + [batch, h * w, b_channel, a_channel]).transpose([0, 1, 3, 2]) + b_r = b_t.repeat( + a_channel, axis=1).reshape([batch, h * w, a_channel, b_channel]) + return np.mean(a_r * b_r, axis=1) + + +class TestFSPOp(OpTest): + def setUp(self): + self.op_type = "fsp" + self.initTestCase() + + feature_map_0 = np.random.uniform(0, 10, self.a_shape).astype('float32') + feature_map_1 = np.random.uniform(0, 10, self.b_shape).astype('float32') + + self.inputs = {'X': feature_map_0, 'Y': feature_map_1} + self.outputs = {'Out': fsp_matrix(feature_map_0, feature_map_1)} + + def initTestCase(self): + self.a_shape = (2, 16, 32, 31) + self.b_shape = (2, 28, 32, 31) + + def test_check_output(self): + self.check_output() + + def test_check_grad_normal(self): + self.check_grad(['X', 'Y'], 'Out', max_relative_error=0.05) + + +if __name__ == '__main__': + unittest.main() diff --git a/python/paddle/fluid/tests/unittests/test_layers.py b/python/paddle/fluid/tests/unittests/test_layers.py index 1672c3600f..f343ed4e87 100644 --- a/python/paddle/fluid/tests/unittests/test_layers.py +++ b/python/paddle/fluid/tests/unittests/test_layers.py @@ -1269,6 +1269,15 @@ class TestBook(unittest.TestCase): self.assertIsNotNone(out) print(str(program)) + def test_fsp(self): + program = Program() + with program_guard(program): + x = layers.data(name="X", shape=[16, 4, 4], dtype="float32") + y = layers.data(name="Y", shape=[8, 4, 4], dtype="float32") + out = layers.fsp_matrix(x, y) + self.assertIsNotNone(out) + print(str(program)) + if __name__ == '__main__': unittest.main() diff --git a/python/setup.py.in b/python/setup.py.in index a7c1e91f9c..9f87f5644f 100644 --- a/python/setup.py.in +++ b/python/setup.py.in @@ -117,6 +117,7 @@ packages=['paddle', 'paddle.fluid.contrib.slim.graph', 'paddle.fluid.contrib.slim.prune', 'paddle.fluid.contrib.slim.quantization', + 'paddle.fluid.contrib.slim.distillation', 'paddle.fluid.contrib.utils', 'paddle.fluid.transpiler', 'paddle.fluid.transpiler.details'] From 85e1cc1e02b0fe0df86cbc7d78245ba892f8fdb3 Mon Sep 17 00:00:00 2001 From: "Wang, Chuanqi" Date: Tue, 26 Mar 2019 09:53:42 +0800 Subject: [PATCH 40/44] Update Readme with new accuracy and performance data measured on 6271 (#16437) * Update Readme with new accuracy and performance data measured on 6271 test=develop * Modify tables and example command test=develop --- .../fluid/contrib/int8_inference/README.md | 33 +++++++++++++------ 1 file changed, 23 insertions(+), 10 deletions(-) diff --git a/python/paddle/fluid/contrib/int8_inference/README.md b/python/paddle/fluid/contrib/int8_inference/README.md index 460ae393f1..55a21ed1c5 100644 --- a/python/paddle/fluid/contrib/int8_inference/README.md +++ b/python/paddle/fluid/contrib/int8_inference/README.md @@ -45,28 +45,41 @@ You can load INT8 model by load_inference_model [API](https://github.com/PaddleP ``` ## 3. Result -We provide the results of accuracy measurd on [Intel® Xeon® Platinum Gold Processor](https://ark.intel.com/products/120489/Intel-Xeon-Gold-6148-Processor-27-5M-Cache-2-40-GHz- "Intel® Xeon® Gold 6148 Processor") (also known as Intel® Xeon® Skylake6148). +We provide the results of accuracy and performance measured on Intel(R) Xeon(R) Gold 6271 (single core). + +**I. Top-1 Accuracy on Intel(R) Xeon(R) Gold 6271** | Model | Dataset | FP32 Accuracy | INT8 Accuracy | Accuracy Diff | -| ------------ | ------------ | ------------ | ------------ | ------------ | -| ResNet-50 | Small | 72.00% | 72.00% | 0.00% | -| MobileNet-V1 | Small | 62.00% | 62.00% | 0.00% | -| ResNet-50 | Full ImageNet Val | 76.63% | 76.17% | 0.46% | -| MobileNet-V1 | Full ImageNet Val | 70.78% | 70.49% | 0.29% | +| :------------: | :------------: | :------------: | :------------: | :------------: | +| ResNet-50 | Full ImageNet Val | 76.63% | 76.23% | 0.40% | +| MobileNet-V1 | Full ImageNet Val | 70.78% | 70.47% | 0.31% | + +**II. Throughput on Intel(R) Xeon(R) Gold 6271 (batch size 1 on single core)** + +| Model | Dataset | FP32 Throughput | INT8 Throughput | Ratio(INT8/FP32) | +| :------------: | :------------: | :------------: | :------------: | :------------: | +| ResNet-50 | Full ImageNet Val | 11.54 images/s | 32.2 images/s | 2.79 | +| MobileNet-V1 | Full ImageNet Val | 49.21 images/s | 108.37 images/s | 2.2 | -Please note that [Small](http://paddle-inference-dist.cdn.bcebos.com/int8/calibration_test_data.tar.gz "Small") is a subset of [full ImageNet validation dataset](http://www.image-net.org/challenges/LSVRC/2012/nnoupb/ILSVRC2012_img_val.tar "full ImageNet validation dataset"). +Please note that [full ImageNet validation dataset](http://www.image-net.org/challenges/LSVRC/2012/nnoupb/ILSVRC2012_img_val.tar "full ImageNet validation dataset") can be downloaded by script `test_calibration.py` with `DATASET=full`. Notes: * The accuracy measurement requires the model with `label`. -* The INT8 theoretical speedup is ~1.33X on Intel® Xeon® Skylake Server (please refer to `This allows for 4x more input at the cost of 3x more instructions or 33.33% more compute` in [Reference](https://software.intel.com/en-us/articles/lower-numerical-precision-deep-learning-inference-and-training "Reference")). +* The INT8 theoretical speedup is 4X on Intel® Xeon® Cascadelake Server (please refer to `providing a theoretical peak compute gain of 4x int8 OPS over fp32 OPS` in [Reference](https://software.intel.com/en-us/articles/lower-numerical-precision-deep-learning-inference-and-training "Reference")). However, the actual test results at the model level will be less than 4X, and in general the average is about 2X. In addition, the calculation library optimization of batch size 1 is not as good as the large batch size. ## 4. How to reproduce the results -* Small dataset +* Small dataset (Single core) ```bash FLAGS_use_mkldnn=true python python/paddle/fluid/contrib/tests/test_calibration.py ``` -* Full dataset +* Full dataset (Single core) ```bash FLAGS_use_mkldnn=true DATASET=full python python/paddle/fluid/contrib/tests/test_calibration.py ``` + +* Full dataset (Multi-core) +```bash +FLAGS_use_mkldnn=true OMP_NUM_THREADS=20 DATASET=full python python/paddle/fluid/contrib/tests/test_calibration.py +``` +> Notes: This is an example command with 20 cores by using set `OMP_NUM_THREADS` value. From becf7994318b1adb7de3e4aea109bd41b580b712 Mon Sep 17 00:00:00 2001 From: Xin Pan Date: Tue, 26 Mar 2019 10:12:12 +0800 Subject: [PATCH 41/44] fix test=develop --- python/paddle/fluid/framework.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/paddle/fluid/framework.py b/python/paddle/fluid/framework.py index 556ce71ee5..db66b1990b 100644 --- a/python/paddle/fluid/framework.py +++ b/python/paddle/fluid/framework.py @@ -318,8 +318,8 @@ class Variable(object): self._ivar = core.VarBase( name, dtype if dtype else core.VarDesc.VarType.FP32, list(shape) if shape else [], - _current_expected_place(), True - if persistable else False, stop_gradient) + _current_expected_place(), stop_gradient, True + if persistable else False) if persistable: _imperative_tracer().trace_var(name, self) else: From f735102eab15d39ba23c91cc440cf857fca4282c Mon Sep 17 00:00:00 2001 From: Jiabin Yang Date: Tue, 26 Mar 2019 13:51:00 +0800 Subject: [PATCH 42/44] add layer norm to Layers, add transformer test in imperative mode (#16092) * add layer norm to Layers, add transformer prepare encoding * little change * finish encoder part * add decoder part * finish model part * add test case and part of data feed * add transformer test * add to_parameter, add remove in set_attr * test=develop, fix pos encoding bug, create_parameter with stantard name * test=develop, rm dropout test in imperative * test=develop, fix cpu error * test=develop, fix minize bug * test=develop, fix one hot not stop gradient * test=develop, fix one hot not stop gradient * test=develop, refine parameter name * test=develop, fix transformer test in imperative mode * test=develop, fix transformer test in imperative mode * test=develop, fix boost and mkl download error * test=develop, fix boost and mkl download error * test=develop, fix ci and refine code * test=develop, fix ci and refine code --- paddle/fluid/imperative/layer.cc | 3 + paddle/fluid/imperative/tracer.cc | 1 + python/paddle/dataset/wmt14.py | 2 +- python/paddle/fluid/imperative/base.py | 4 +- .../fluid/imperative/layer_object_helper.py | 1 + python/paddle/fluid/imperative/layers.py | 7 + python/paddle/fluid/imperative/nn.py | 132 +- python/paddle/fluid/layer_helper_base.py | 4 +- python/paddle/fluid/layers/nn.py | 3 +- python/paddle/fluid/optimizer.py | 15 +- .../unittests/test_imperative_transformer.py | 1096 +++++++++++++++++ .../fluid/tests/unittests/test_layers.py | 28 + 12 files changed, 1279 insertions(+), 17 deletions(-) create mode 100644 python/paddle/fluid/tests/unittests/test_imperative_transformer.py diff --git a/paddle/fluid/imperative/layer.cc b/paddle/fluid/imperative/layer.cc index 3d1de95f58..036d2a50a4 100644 --- a/paddle/fluid/imperative/layer.cc +++ b/paddle/fluid/imperative/layer.cc @@ -315,6 +315,9 @@ std::map> OpBase::ApplyGrad() { for (size_t i = 0; i < outputs.size(); ++i) { framework::Variable* grad = outputs[i]->var_; framework::Variable* orig_grad = origin_outputs[i]->var_; + VLOG(3) << "AddTo Called with orig_grad is: " + << origin_outputs[i]->name_ << " Grad to be added is " + << outputs[i]->name_; AddTo(grad, orig_grad, place_); delete grad; } diff --git a/paddle/fluid/imperative/tracer.cc b/paddle/fluid/imperative/tracer.cc index 0cfdea030e..7c9d0af3ec 100644 --- a/paddle/fluid/imperative/tracer.cc +++ b/paddle/fluid/imperative/tracer.cc @@ -277,6 +277,7 @@ std::set Tracer::Trace(OpBase* op, const VarBasePtrMap& inputs, VarBase* var = current_vars_map[var_it->second]; InitGrad(var, prepared_op.GetDeviceContext()); grad_out_vars.push_back(var->grads_); + VLOG(3) << "grads output var name: " << var->name_; } } } diff --git a/python/paddle/dataset/wmt14.py b/python/paddle/dataset/wmt14.py index adc0c1aac8..450f159f9d 100644 --- a/python/paddle/dataset/wmt14.py +++ b/python/paddle/dataset/wmt14.py @@ -15,7 +15,7 @@ WMT14 dataset. The original WMT14 dataset is too large and a small set of data for set is provided. This module will download dataset from -http://paddlepaddle.cdn.bcebos.com/demo/wmt_shrinked_data/wmt14.tgz and +http://paddlepaddle.bj.bcebos.com/demo/wmt_shrinked_data/wmt14.tgz and parse training set and test set into paddle reader creators. """ diff --git a/python/paddle/fluid/imperative/base.py b/python/paddle/fluid/imperative/base.py index 174f138bfa..d619c09b1b 100644 --- a/python/paddle/fluid/imperative/base.py +++ b/python/paddle/fluid/imperative/base.py @@ -44,7 +44,7 @@ def guard(place=None): yield -def to_variable(value, block=None): +def to_variable(value, block=None, name=None): if isinstance(value, np.ndarray): assert enabled(), "to_variable could only be called in imperative mode" @@ -53,7 +53,7 @@ def to_variable(value, block=None): py_var = framework.Variable( block, type=core.VarDesc.VarType.LOD_TENSOR, - name=None, + name=name, shape=value.shape, dtype=value.dtype) var = py_var._ivar.value() diff --git a/python/paddle/fluid/imperative/layer_object_helper.py b/python/paddle/fluid/imperative/layer_object_helper.py index 6afffe3636..0dac99a491 100644 --- a/python/paddle/fluid/imperative/layer_object_helper.py +++ b/python/paddle/fluid/imperative/layer_object_helper.py @@ -105,6 +105,7 @@ class LayerObjectHelper(LayerHelperBase): Returns dtype of the input """ + inputs_in = inputs_in if (inputs_in is not None) else [] inputs = self._multiple_input(inputs_in) dtype = None for each in inputs: diff --git a/python/paddle/fluid/imperative/layers.py b/python/paddle/fluid/imperative/layers.py index a8a1aac8b0..e64667f7f4 100644 --- a/python/paddle/fluid/imperative/layers.py +++ b/python/paddle/fluid/imperative/layers.py @@ -17,10 +17,12 @@ import contextlib import sys import numpy as np import collections +import six from .. import unique_name from paddle.fluid import core from .layer_object_helper import LayerObjectHelper from paddle.fluid import framework +from ..param_attr import ParamAttr __all__ = ['Layer', 'PyLayer'] @@ -72,6 +74,10 @@ class Layer(core.Layer): Returns created parameter Variable. """ + if isinstance(attr, ParamAttr) and (attr.name is not None): + attr.name = ".".join([self._full_name, attr.name]) + elif isinstance(attr, six.string_types): + attr = ".".join([self._full_name, attr]) return self._helper.create_parameter(attr, shape, dtype, is_bias, default_initializer) @@ -164,6 +170,7 @@ class Layer(core.Layer): the sublayer passed in. """ assert isinstance(sublayer, core.Layer) + self._sub_layers[name] = sublayer return sublayer diff --git a/python/paddle/fluid/imperative/nn.py b/python/paddle/fluid/imperative/nn.py index 604ff75349..9cbfc3e3eb 100644 --- a/python/paddle/fluid/imperative/nn.py +++ b/python/paddle/fluid/imperative/nn.py @@ -20,10 +20,12 @@ from .. import core from ..layers import utils from . import layers from ..framework import Variable, OpProtoHolder +from ..layers import layer_function_generator from ..param_attr import ParamAttr from ..initializer import Normal, Constant - -__all__ = ['Conv2D', 'Pool2D', 'FC', 'BatchNorm', 'Embedding', 'GRUUnit'] +__all__ = [ + 'Conv2D', 'Pool2D', 'FC', 'BatchNorm', 'Embedding', 'GRUUnit', 'LayerNorm' +] class Conv2D(layers.Layer): @@ -438,7 +440,6 @@ class Embedding(layers.Layer): self._size = size self._is_sparse = is_sparse self._is_distributed = is_distributed - self._padding_idx = -1 if padding_idx is None else padding_idx if padding_idx >= 0 else ( size[0] + padding_idx) @@ -471,6 +472,131 @@ class Embedding(layers.Layer): return out +class LayerNorm(layers.Layer): + def __init__(self, + name_scope, + scale=True, + shift=True, + begin_norm_axis=1, + epsilon=1e-05, + param_attr=None, + bias_attr=None, + act=None): + """ + ${comment} + + The formula is as follows: + + .. math:: + + \\mu & = \\frac{1}{H}\\sum_{i=1}^{H} a_i + + \\sigma & = \\sqrt{\\frac{1}{H}\sum_{i=1}^{H}(a_i - \\mu)^2} + + h & = f(\\frac{g}{\\sigma}(a - \\mu) + b) + + * :math:`a`: the vector representation of the summed inputs to the neurons + in that layer. + + * :math:`H`: the number of hidden units in a layers + + * :math:`g`: the trainable scale parameter. + + * :math:`b`: the trainable bias parameter. + + Args: + input(Variable): The input tensor variable. + scale(bool): Whether to learn the adaptive gain :math:`g` after + normalization. Default True. + shift(bool): Whether to learn the adaptive bias :math:`b` after + normalization. Default True. + begin_norm_axis(int): The normalization will be performed along + dimensions from :attr:`begin_norm_axis` to :attr:`rank(input)`. + Default 1. + epsilon(float): The small value added to the variance to prevent + division by zero. Default 1e-05. + param_attr(ParamAttr|None): The parameter attribute for the learnable + gain :math:`g`. If :attr:`scale` is False, :attr:`param_attr` is + omitted. If :attr:`scale` is True and :attr:`param_attr` is None, + a default :code:`ParamAttr` would be added as scale. The + :attr:`param_attr` is initialized as 1 if it is added. Default None. + bias_attr(ParamAttr|None): The parameter attribute for the learnable + bias :math:`b`. If :attr:`shift` is False, :attr:`bias_attr` is + omitted. If :attr:`shift` is True and :attr:`param_attr` is None, + a default :code:`ParamAttr` would be added as bias. The + :attr:`bias_attr` is initialized as 0 if it is added. Default None. + act(str): Activation to be applied to the output of layer normalizaiton. + Default None. + Returns: + ${y_comment} + + Examples: + + >>> data = fluid.layers.data(name='data', shape=[3, 32, 32], + >>> dtype='float32') + >>> x = fluid.layers.layer_norm(input=data, begin_norm_axis=1) + """ + + super(LayerNorm, self).__init__(name_scope) + self._scale = scale + self._shift = shift + self._begin_norm_axis = begin_norm_axis + self._epsilon = epsilon + self._param_attr = param_attr + self._bias_attr = bias_attr + self._act = act + + def _build_once(self, input): + self._dtype = self._helper.input_dtype(input) + input_shape = input.shape + param_shape = [ + reduce(lambda x, y: x * y, input_shape[self._begin_norm_axis:]) + ] + if self._scale: + self._scale_w = self.create_parameter( + attr=self._param_attr, + shape=param_shape, + dtype=self._dtype, + default_initializer=Constant(1.0)) + if self._shift: + assert self._bias_attr is not False + self._bias_w = self.create_parameter( + attr=self._bias_attr, + shape=param_shape, + dtype=self._dtype, + is_bias=True) + + def forward(self, input): + inputs = dict() + inputs['X'] = input + if self._scale: + inputs['Scale'] = self._scale_w + if self._shift: + inputs['Bias'] = self._bias_w + # create output + mean_out = self._helper.create_variable_for_type_inference( + dtype=self._dtype, stop_gradient=True) + variance_out = self._helper.create_variable_for_type_inference( + dtype=self._dtype, stop_gradient=True) + layer_norm_out = self._helper.create_variable_for_type_inference( + self._dtype) + + self._helper.append_op( + type="layer_norm", + inputs=inputs, + outputs={ + "Y": layer_norm_out, + "Mean": mean_out, + "Variance": variance_out, + }, + attrs={ + "epsilon": self._epsilon, + "begin_norm_axis": self._begin_norm_axis + }) + + return self._helper.append_activation(layer_norm_out) + + class GRUUnit(layers.Layer): """ **GRU unit layer** diff --git a/python/paddle/fluid/layer_helper_base.py b/python/paddle/fluid/layer_helper_base.py index 3504cb7935..a68160d797 100644 --- a/python/paddle/fluid/layer_helper_base.py +++ b/python/paddle/fluid/layer_helper_base.py @@ -268,11 +268,9 @@ class LayerHelperBase(object): """ # Deepcopy the attr so that parameters can be shared in program attr = copy.deepcopy(attr) - if attr is None: - attr = ParamAttr._to_attr(attr) + attr = ParamAttr._to_attr(attr) if not attr: return None - assert isinstance(attr, ParamAttr) suffix = 'b' if is_bias else 'w' if attr.name is None: diff --git a/python/paddle/fluid/layers/nn.py b/python/paddle/fluid/layers/nn.py index c4e6053fec..713aedb81e 100644 --- a/python/paddle/fluid/layers/nn.py +++ b/python/paddle/fluid/layers/nn.py @@ -6206,7 +6206,8 @@ def one_hot(input, depth): type="one_hot", inputs={'X': input}, attrs={'depth': depth}, - outputs={'Out': one_hot_out}) + outputs={'Out': one_hot_out}, + stop_gradient=True) return one_hot_out diff --git a/python/paddle/fluid/optimizer.py b/python/paddle/fluid/optimizer.py index 8918886a80..505d9572a6 100644 --- a/python/paddle/fluid/optimizer.py +++ b/python/paddle/fluid/optimizer.py @@ -397,13 +397,14 @@ class Optimizer(object): for param in parameters: if not param.trainable: continue - # create gradient variable - grad_var = Variable( - block=loss.block, - name=param._ivar._grad_name(), - stop_gradient=True, - ivar=param._ivar._grad_ivar()) - params_grads.append((param, grad_var)) + if param._ivar._grad_ivar() is not None: + # create gradient variable + grad_var = Variable( + block=loss.block, + name=param._ivar._grad_name(), + stop_gradient=True, + ivar=param._ivar._grad_ivar()) + params_grads.append((param, grad_var)) with program_guard(framework.default_main_program(), framework.default_startup_program()): optimize_ops = self._create_optimization_pass(params_grads) diff --git a/python/paddle/fluid/tests/unittests/test_imperative_transformer.py b/python/paddle/fluid/tests/unittests/test_imperative_transformer.py new file mode 100644 index 0000000000..b06d3e8894 --- /dev/null +++ b/python/paddle/fluid/tests/unittests/test_imperative_transformer.py @@ -0,0 +1,1096 @@ +# 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. + +from __future__ import print_function + +import unittest +import paddle.fluid as fluid +from paddle.fluid.imperative import Embedding, LayerNorm, FC, to_variable, Layer, guard +from test_imperative_base import new_program_scope +from paddle.fluid import core +import numpy as np +import six +np.set_printoptions(suppress=True) + + +# Copy from models +class TrainTaskConfig(object): + # support both CPU and GPU now. + use_gpu = True + # the epoch number to train. + pass_num = 30 + # the number of sequences contained in a mini-batch. + # deprecated, set batch_size in args. + batch_size = 32 + # the hyper parameters for Adam optimizer. + # This static learning_rate will be multiplied to the LearningRateScheduler + # derived learning rate the to get the final learning rate. + learning_rate = 2.0 + beta1 = 0.9 + beta2 = 0.997 + eps = 1e-9 + # the parameters for learning rate scheduling. + warmup_steps = 8000 + # the weight used to mix up the ground-truth distribution and the fixed + # uniform distribution in label smoothing when training. + # Set this as zero if label smoothing is not wanted. + label_smooth_eps = 0.1 + # the directory for saving trained models. + model_dir = "trained_models" + # the directory for saving checkpoints. + ckpt_dir = "trained_ckpts" + # the directory for loading checkpoint. + # If provided, continue training from the checkpoint. + ckpt_path = None + # the parameter to initialize the learning rate scheduler. + # It should be provided if use checkpoints, since the checkpoint doesn't + # include the training step counter currently. + start_step = 0 + # the frequency to save trained models. + save_freq = 10000 + + +class InferTaskConfig(object): + use_gpu = True + # the number of examples in one run for sequence generation. + batch_size = 10 + # the parameters for beam search. + beam_size = 5 + max_out_len = 256 + # the number of decoded sentences to output. + n_best = 1 + # the flags indicating whether to output the special tokens. + output_bos = False + output_eos = False + output_unk = True + # the directory for loading the trained model. + model_path = "trained_models/pass_1.infer.model" + + +class ModelHyperParams(object): + # These following five vocabularies related configurations will be set + # automatically according to the passed vocabulary path and special tokens. + # size of source word dictionary. + src_vocab_size = 10000 + # size of target word dictionay + trg_vocab_size = 10000 + # index for token + bos_idx = 0 + # index for token + eos_idx = 1 + # index for token + unk_idx = 2 + # max length of sequences deciding the size of position encoding table. + max_length = 4 + # the dimension for word embeddings, which is also the last dimension of + # the input and output of multi-head attention, position-wise feed-forward + # networks, encoder and decoder. + d_model = 512 + # size of the hidden layer in position-wise feed-forward networks. + d_inner_hid = 2048 + # the dimension that keys are projected to for dot-product attention. + d_key = 64 + # the dimension that values are projected to for dot-product attention. + d_value = 64 + # number of head used in multi-head attention. + n_head = 8 + # number of sub-layers to be stacked in the encoder and decoder. + n_layer = 6 + # dropout rates of different modules. + prepostprocess_dropout = 0.1 + attention_dropout = 0.1 + relu_dropout = 0.1 + # to process before each sub-layer + preprocess_cmd = "n" # layer normalization + # to process after each sub-layer + postprocess_cmd = "da" # dropout + residual connection + # random seed used in dropout for CE. + dropout_seed = 1 + # the flag indicating whether to share embedding and softmax weights. + # vocabularies in source and target should be same for weight sharing. + weight_sharing = True + + +def merge_cfg_from_list(cfg_list, g_cfgs): + """ + Set the above global configurations using the cfg_list. + """ + assert len(cfg_list) % 2 == 0 + for key, value in zip(cfg_list[0::2], cfg_list[1::2]): + for g_cfg in g_cfgs: + if hasattr(g_cfg, key): + try: + value = eval(value) + except Exception: # for file path + pass + setattr(g_cfg, key, value) + break + + +def position_encoding_init(n_position, d_pos_vec): + """ + Generate the initial values for the sinusoid position encoding table. + """ + channels = d_pos_vec + position = np.arange(n_position) + num_timescales = channels // 2 + log_timescale_increment = (np.log(float(1e4) / float(1)) / + (num_timescales - 1)) + inv_timescales = np.exp(np.arange( + num_timescales)) * -log_timescale_increment + scaled_time = np.expand_dims(position, 1) * np.expand_dims(inv_timescales, + 0) + signal = np.concatenate([np.sin(scaled_time), np.cos(scaled_time)], axis=1) + signal = np.pad(signal, [[0, 0], [0, np.mod(channels, 2)]], 'constant') + position_enc = signal + return position_enc.astype("float32") + + +def create_data(is_static=False): + if is_static: + return [ + src_word_np, src_pos_np, src_slf_attn_bias_np, trg_word_np, + trg_pos_np, trg_slf_attn_bias_np, trg_src_attn_bias_np, lbl_word_np, + lbl_weight_np + ] + else: + enc_inputs = [ + to_variable(src_word_np), to_variable(src_pos_np), + to_variable(src_slf_attn_bias_np) + ] + dec_inputs = [ + to_variable(trg_word_np), to_variable(trg_pos_np), + to_variable(trg_slf_attn_bias_np), to_variable(trg_src_attn_bias_np) + ] + label = to_variable(lbl_word_np) + weight = to_variable(lbl_weight_np) + return enc_inputs, dec_inputs, label, weight + + +def create_feed_dict_list(data, init=False): + if init: + data_input_names = encoder_data_input_fields + \ + decoder_data_input_fields[:-1] + label_data_input_fields + pos_enc_param_names + else: + data_input_names = encoder_data_input_fields + \ + decoder_data_input_fields[:-1] + label_data_input_fields + feed_dict_list = dict() + for i in range(len(data_input_names)): + feed_dict_list[data_input_names[i]] = data[i] + return feed_dict_list + + +def make_all_inputs(input_fields): + """ + Define the input data layers for the transformer model. + """ + inputs = [] + for input_field in input_fields: + input_var = fluid.layers.data( + name=input_field, + shape=input_descs[input_field][0], + dtype=input_descs[input_field][1], + lod_level=input_descs[input_field][2] + if len(input_descs[input_field]) == 3 else 0, + append_batch_size=False) + inputs.append(input_var) + return inputs + + +# The placeholder for batch_size in compile time. Must be -1 currently to be +# consistent with some ops' infer-shape output in compile time, such as the +# sequence_expand op used in beamsearch decoder. +batch_size = 32 +# The placeholder for squence length in compile time. +seq_len = ModelHyperParams.max_length +# Here list the data shapes and data types of all inputs. +# The shapes here act as placeholder and are set to pass the infer-shape in +# compile time. +input_descs = { + # The actual data shape of src_word is: + # [batch_size, max_src_len_in_batch, 1] + "src_word": [(batch_size, seq_len, 1), "int64", 2], + # The actual data shape of src_pos is: + # [batch_size, max_src_len_in_batch, 1] + "src_pos": [(batch_size, seq_len, 1), "int64"], + # This input is used to remove attention weights on paddings in the + # encoder. + # The actual data shape of src_slf_attn_bias is: + # [batch_size, n_head, max_src_len_in_batch, max_src_len_in_batch] + "src_slf_attn_bias": [(batch_size, ModelHyperParams.n_head, seq_len, + seq_len), "float32"], + # The actual data shape of trg_word is: + # [batch_size, max_trg_len_in_batch, 1] + "trg_word": [(batch_size, seq_len, 1), "int64", + 2], # lod_level is only used in fast decoder. + # The actual data shape of trg_pos is: + # [batch_size, max_trg_len_in_batch, 1] + "trg_pos": [(batch_size, seq_len, 1), "int64"], + # This input is used to remove attention weights on paddings and + # subsequent words in the decoder. + # The actual data shape of trg_slf_attn_bias is: + # [batch_size, n_head, max_trg_len_in_batch, max_trg_len_in_batch] + "trg_slf_attn_bias": [(batch_size, ModelHyperParams.n_head, seq_len, + seq_len), "float32"], + # This input is used to remove attention weights on paddings of the source + # input in the encoder-decoder attention. + # The actual data shape of trg_src_attn_bias is: + # [batch_size, n_head, max_trg_len_in_batch, max_src_len_in_batch] + "trg_src_attn_bias": [(batch_size, ModelHyperParams.n_head, seq_len, + seq_len), "float32"], + # This input is used in independent decoder program for inference. + # The actual data shape of enc_output is: + # [batch_size, max_src_len_in_batch, d_model] + "enc_output": [(batch_size, seq_len, ModelHyperParams.d_model), "float32"], + # The actual data shape of label_word is: + # [batch_size * max_trg_len_in_batch, 1] + "lbl_word": [(batch_size * seq_len, 1), "int64"], + # This input is used to mask out the loss of paddding tokens. + # The actual data shape of label_weight is: + # [batch_size * max_trg_len_in_batch, 1] + "lbl_weight": [(batch_size * seq_len, 1), "float32"], + # This input is used in beam-search decoder. + "init_score": [(batch_size, 1), "float32", 2], + # This input is used in beam-search decoder for the first gather + # (cell states updation) + "init_idx": [(batch_size, ), "int32"], +} + +# Names of word embedding table which might be reused for weight sharing. +word_emb_param_names = ( + "src_word_emb_table", + "trg_word_emb_table", ) +# Names of position encoding table which will be initialized externally. +pos_enc_param_names = ( + "src_pos_enc_table", + "trg_pos_enc_table", ) +# separated inputs for different usages. +encoder_data_input_fields = ( + "src_word", + "src_pos", + "src_slf_attn_bias", ) +decoder_data_input_fields = ( + "trg_word", + "trg_pos", + "trg_slf_attn_bias", + "trg_src_attn_bias", + "enc_output", ) +label_data_input_fields = ( + "lbl_word", + "lbl_weight", ) +# In fast decoder, trg_pos (only containing the current time step) is generated +# by ops and trg_slf_attn_bias is not needed. +fast_decoder_data_input_fields = ( + "trg_word", + "init_score", + "init_idx", + "trg_src_attn_bias", ) +# if we use py_reader +use_py_reader = False + +# if we run sync mode +sync = False + +# how many batches we use +batch_num = 2 + +np.random.seed = 1 +src_word_np = np.random.randint( + 1, + ModelHyperParams.src_vocab_size - 1, + size=(batch_size, seq_len, 1), + dtype='int64') +src_pos_np = np.random.randint( + 1, seq_len, size=(batch_size, seq_len, 1), dtype='int64') +src_slf_attn_bias_np = np.random.randn(batch_size, ModelHyperParams.n_head, + seq_len, seq_len).astype('float32') + +trg_word_np = np.random.randint( + 1, + ModelHyperParams.src_vocab_size - 1, + size=(batch_size, seq_len, 1), + dtype='int64') +trg_pos_np = np.random.randint( + 1, seq_len, size=(batch_size, seq_len, 1), dtype='int64') +trg_slf_attn_bias_np = np.random.randn(batch_size, ModelHyperParams.n_head, + seq_len, seq_len).astype('float32') +trg_src_attn_bias_np = np.random.randn(batch_size, ModelHyperParams.n_head, + seq_len, seq_len).astype('float32') + +lbl_word_np = np.random.randint( + 1, + ModelHyperParams.src_vocab_size - 1, + size=(batch_size * seq_len, 1), + dtype='int64') +lbl_weight_np = np.random.randn(batch_size * seq_len, 1).astype('float32') + +# np.random.seed = 1 +# src_word_np = np.arange(0, 10).reshape([batch_size, seq_len, 1]).astype('int64') +# src_pos_np = np.random.randint( +# 1, seq_len, size=(batch_size, seq_len, 1), dtype='int64') +# src_slf_attn_bias_np = np.random.randn(batch_size, ModelHyperParams.n_head, +# seq_len, seq_len).astype('float32') +# +# trg_word_np = np.arange(0, 10).reshape([batch_size, seq_len, 1]).astype('int64') +# trg_pos_np = np.random.randint( +# 1, seq_len, size=(batch_size, seq_len, 1), dtype='int64') +# trg_slf_attn_bias_np = np.random.randn(batch_size, ModelHyperParams.n_head, +# seq_len, seq_len).astype('float32') +# trg_src_attn_bias_np = np.random.randn(batch_size, ModelHyperParams.n_head, +# seq_len, seq_len).astype('float32') +# +# lbl_word_np = np.arange(0, 10).reshape([batch_size * seq_len, 1]).astype('int64') +# lbl_weight_np = np.random.randn(batch_size * seq_len, 1).astype('float32') +# +pos_inp1 = position_encoding_init(ModelHyperParams.max_length, + ModelHyperParams.d_model) +pos_inp2 = position_encoding_init(ModelHyperParams.max_length, + ModelHyperParams.d_model) + + +class PrePostProcessLayer(Layer): + def __init__(self, name_scope, process_cmd, shape_len=None): + super(PrePostProcessLayer, self).__init__(name_scope) + for cmd in process_cmd: + if cmd == "n": + self._layer_norm = LayerNorm( + name_scope=self.full_name(), + begin_norm_axis=shape_len - 1, + param_attr=fluid.ParamAttr( + initializer=fluid.initializer.Constant(1.)), + bias_attr=fluid.ParamAttr( + initializer=fluid.initializer.Constant(0.))) + + def forward(self, prev_out, out, process_cmd, dropout_rate=0.): + for cmd in process_cmd: + if cmd == "a": # add residual connection + out = out + prev_out if prev_out else out + elif cmd == "n": # add layer normalization + out = self._layer_norm(out) + elif cmd == "d": # add dropout + if dropout_rate: + out = fluid.layers.dropout( + out, + dropout_prob=dropout_rate, + seed=ModelHyperParams.dropout_seed, + is_test=False) + return out + + +class PositionwiseFeedForwardLayer(Layer): + def __init__(self, name_scope, d_inner_hid, d_hid, dropout_rate): + super(PositionwiseFeedForwardLayer, self).__init__(name_scope) + self._i2h = FC(name_scope=self.full_name(), + size=d_inner_hid, + num_flatten_dims=2, + act="relu") + self._h2o = FC(name_scope=self.full_name(), + size=d_hid, + num_flatten_dims=2) + self._dropout_rate = dropout_rate + + def forward(self, x): + hidden = self._i2h(x) + if self._dropout_rate: + hidden = fluid.layers.dropout( + hidden, + dropout_prob=self._dropout_rate, + seed=ModelHyperParams.dropout_seed, + is_test=False) + out = self._h2o(hidden) + return out + + +class MultiHeadAttentionLayer(Layer): + def __init__(self, + name_scope, + d_key, + d_value, + d_model, + n_head=1, + dropout_rate=0., + cache=None, + gather_idx=None, + static_kv=False): + super(MultiHeadAttentionLayer, self).__init__(name_scope) + self._n_head = n_head + self._d_key = d_key + self._d_value = d_value + self._d_model = d_model + self._dropout_rate = dropout_rate + self._q_fc = FC(name_scope=self.full_name(), + size=d_key * n_head, + bias_attr=False, + num_flatten_dims=2) + self._k_fc = FC(name_scope=self.full_name(), + size=d_key * n_head, + bias_attr=False, + num_flatten_dims=2) + self._v_fc = FC(name_scope=self.full_name(), + size=d_value * n_head, + bias_attr=False, + num_flatten_dims=2) + self._proj_fc = FC(name_scope=self.full_name(), + size=self._d_model, + bias_attr=False, + num_flatten_dims=2) + + def forward(self, queries, keys, values, attn_bias): + # compute q ,k ,v + keys = queries if keys is None else keys + values = keys if values is None else values + + q = self._q_fc(queries) + k = self._k_fc(keys) + v = self._v_fc(values) + + # split head + reshaped_q = fluid.layers.reshape( + x=q, shape=[0, 0, self._n_head, self._d_key], inplace=False) + transpose_q = fluid.layers.transpose(x=reshaped_q, perm=[0, 2, 1, 3]) + reshaped_k = fluid.layers.reshape( + x=k, shape=[0, 0, self._n_head, self._d_key], inplace=False) + transpose_k = fluid.layers.transpose(x=reshaped_k, perm=[0, 2, 1, 3]) + reshaped_v = fluid.layers.reshape( + x=v, shape=[0, 0, self._n_head, self._d_value], inplace=False) + transpose_v = fluid.layers.transpose(x=reshaped_v, perm=[0, 2, 1, 3]) + + #scale dot product attention + product = fluid.layers.matmul( + x=transpose_q, + y=transpose_k, + transpose_y=True, + alpha=self._d_model**-0.5) + if attn_bias: + product += attn_bias + weights = fluid.layers.softmax(product) + if self._dropout_rate: + weights_droped = fluid.layers.dropout( + weights, + dropout_prob=self._dropout_rate, + seed=ModelHyperParams.dropout_seed, + is_test=False) + out = fluid.layers.matmul(weights_droped, transpose_v) + else: + out = fluid.layers.matmul(weights, transpose_v) + + # combine heads + if len(out.shape) != 4: + raise ValueError("Input(x) should be a 4-D Tensor.") + trans_x = fluid.layers.transpose(out, perm=[0, 2, 1, 3]) + final_out = fluid.layers.reshape( + x=trans_x, + shape=[0, 0, trans_x.shape[2] * trans_x.shape[3]], + inplace=False) + + # fc to output + proj_out = self._proj_fc(final_out) + return proj_out + + +class EncoderSubLayer(Layer): + def __init__(self, + name_scope, + n_head, + d_key, + d_value, + d_model, + d_inner_hid, + prepostprocess_dropout, + attention_dropout, + relu_dropout, + preprocess_cmd="n", + postprocess_cmd="da"): + + super(EncoderSubLayer, self).__init__(name_scope) + self._preprocess_cmd = preprocess_cmd + self._postprocess_cmd = postprocess_cmd + self._prepostprocess_dropout = prepostprocess_dropout + + self._preprocess_layer = PrePostProcessLayer(self.full_name(), + self._preprocess_cmd, 3) + self._multihead_attention_layer = MultiHeadAttentionLayer( + self.full_name(), d_key, d_value, d_model, n_head, + attention_dropout) + self._postprocess_layer = PrePostProcessLayer( + self.full_name(), self._postprocess_cmd, None) + self._preprocess_layer2 = PrePostProcessLayer(self.full_name(), + self._preprocess_cmd, 3) + self._positionwise_feed_forward = PositionwiseFeedForwardLayer( + self.full_name(), d_inner_hid, d_model, relu_dropout) + self._postprocess_layer2 = PrePostProcessLayer( + self.full_name(), self._postprocess_cmd, None) + + def forward(self, enc_input, attn_bias): + pre_process_multihead = self._preprocess_layer( + None, enc_input, self._preprocess_cmd, self._prepostprocess_dropout) + attn_output = self._multihead_attention_layer(pre_process_multihead, + None, None, attn_bias) + attn_output = self._postprocess_layer(enc_input, attn_output, + self._postprocess_cmd, + self._prepostprocess_dropout) + pre_process2_output = self._preprocess_layer2( + None, attn_output, self._preprocess_cmd, + self._prepostprocess_dropout) + ffd_output = self._positionwise_feed_forward(pre_process2_output) + return self._postprocess_layer2(attn_output, ffd_output, + self._postprocess_cmd, + self._prepostprocess_dropout) + + +class EncoderLayer(Layer): + def __init__(self, + name_scope, + n_layer, + n_head, + d_key, + d_value, + d_model, + d_inner_hid, + prepostprocess_dropout, + attention_dropout, + relu_dropout, + preprocess_cmd="n", + postprocess_cmd="da"): + + super(EncoderLayer, self).__init__(name_scope) + self._preprocess_cmd = preprocess_cmd + self._encoder_sublayers = list() + self._prepostprocess_dropout = prepostprocess_dropout + self._n_layer = n_layer + self._preprocess_layer = PrePostProcessLayer(self.full_name(), + self._preprocess_cmd, 3) + for i in range(n_layer): + self._encoder_sublayers.append( + self.add_sublayer( + 'esl_%d' % i, + EncoderSubLayer( + self.full_name(), n_head, d_key, d_value, d_model, + d_inner_hid, prepostprocess_dropout, attention_dropout, + relu_dropout, preprocess_cmd, postprocess_cmd))) + + def forward(self, enc_input, attn_bias): + for i in range(self._n_layer): + enc_output = self._encoder_sublayers[i](enc_input, attn_bias) + enc_input = enc_output + + return self._preprocess_layer(None, enc_output, self._preprocess_cmd, + self._prepostprocess_dropout) + + +class PrepareEncoderDecoderLayer(Layer): + def __init__(self, + name_scope, + src_vocab_size, + src_emb_dim, + src_max_len, + dropout_rate, + word_emb_param_name=None, + pos_enc_param_name=None): + super(PrepareEncoderDecoderLayer, self).__init__(name_scope) + self._src_max_len = src_max_len + self._src_emb_dim = src_emb_dim + self._src_vocab_size = src_vocab_size + self._dropout_rate = dropout_rate + self._input_emb = Embedding( + name_scope=self.full_name(), + size=[src_vocab_size, src_emb_dim], + padding_idx=0, + param_attr=fluid.ParamAttr( + name=word_emb_param_name, + initializer=fluid.initializer.Normal(0., src_emb_dim**-0.5))) + + if pos_enc_param_name is pos_enc_param_names[0]: + pos_inp = pos_inp1 + else: + pos_inp = pos_inp2 + self._pos_emb = Embedding( + name_scope=self.full_name(), + size=[self._src_max_len, src_emb_dim], + param_attr=fluid.ParamAttr( + name=pos_enc_param_name, + initializer=fluid.initializer.NumpyArrayInitializer(pos_inp), + trainable=False)) + + # use in imperative_mode to fit different length batch + # self._pos_emb._w = to_variable( + # position_encoding_init(self._src_max_len, self._src_emb_dim)) + + def forward(self, src_word, src_pos): + src_word_emb = self._input_emb(src_word) + src_word_emb = fluid.layers.scale( + x=src_word_emb, scale=self._src_emb_dim**0.5) + # # TODO change this to fit dynamic length input + src_pos_emb = self._pos_emb(src_pos) + src_pos_emb.stop_gradient = True + enc_input = src_word_emb + src_pos_emb + return fluid.layers.dropout( + enc_input, + dropout_prob=self._dropout_rate, + seed=ModelHyperParams.dropout_seed, + is_test=False) if self._dropout_rate else enc_input + + +class WrapEncoderLayer(Layer): + def __init__(self, name_cope, src_vocab_size, max_length, n_layer, n_head, + d_key, d_value, d_model, d_inner_hid, prepostprocess_dropout, + attention_dropout, relu_dropout, preprocess_cmd, + postprocess_cmd, weight_sharing): + """ + The wrapper assembles together all needed layers for the encoder. + """ + super(WrapEncoderLayer, self).__init__(name_cope) + + self._prepare_encoder_layer = PrepareEncoderDecoderLayer( + self.full_name(), + src_vocab_size, + d_model, + max_length, + prepostprocess_dropout, + word_emb_param_name=word_emb_param_names[0], + pos_enc_param_name=pos_enc_param_names[0]) + self._encoder = EncoderLayer( + self.full_name(), n_layer, n_head, d_key, d_value, d_model, + d_inner_hid, prepostprocess_dropout, attention_dropout, + relu_dropout, preprocess_cmd, postprocess_cmd) + + def forward(self, enc_inputs): + src_word, src_pos, src_slf_attn_bias = enc_inputs + enc_input = self._prepare_encoder_layer(src_word, src_pos) + enc_output = self._encoder(enc_input, src_slf_attn_bias) + return enc_output + + +class DecoderSubLayer(Layer): + def __init__(self, + name_scope, + n_head, + d_key, + d_value, + d_model, + d_inner_hid, + prepostprocess_dropout, + attention_dropout, + relu_dropout, + preprocess_cmd, + postprocess_cmd, + cache=None, + gather_idx=None): + super(DecoderSubLayer, self).__init__(name_scope) + self._postprocess_cmd = postprocess_cmd + self._preprocess_cmd = preprocess_cmd + self._prepostprcess_dropout = prepostprocess_dropout + self._pre_process_layer = PrePostProcessLayer(self.full_name(), + preprocess_cmd, 3) + self._multihead_attention_layer = MultiHeadAttentionLayer( + self.full_name(), + d_key, + d_value, + d_model, + n_head, + attention_dropout, + cache=cache, + gather_idx=gather_idx) + self._post_process_layer = PrePostProcessLayer(self.full_name(), + postprocess_cmd, None) + self._pre_process_layer2 = PrePostProcessLayer(self.full_name(), + preprocess_cmd, 3) + self._multihead_attention_layer2 = MultiHeadAttentionLayer( + self.full_name(), + d_key, + d_value, + d_model, + n_head, + attention_dropout, + cache=cache, + gather_idx=gather_idx, + static_kv=True) + self._post_process_layer2 = PrePostProcessLayer(self.full_name(), + postprocess_cmd, None) + self._pre_process_layer3 = PrePostProcessLayer(self.full_name(), + preprocess_cmd, 3) + self._positionwise_feed_forward_layer = PositionwiseFeedForwardLayer( + self.full_name(), d_inner_hid, d_model, relu_dropout) + self._post_process_layer3 = PrePostProcessLayer(self.full_name(), + postprocess_cmd, None) + + def forward(self, dec_input, enc_output, slf_attn_bias, dec_enc_attn_bias): + pre_process_rlt = self._pre_process_layer( + None, dec_input, self._preprocess_cmd, self._prepostprcess_dropout) + slf_attn_output = self._multihead_attention_layer(pre_process_rlt, None, + None, slf_attn_bias) + slf_attn_output_pp = self._post_process_layer( + dec_input, slf_attn_output, self._postprocess_cmd, + self._prepostprcess_dropout) + pre_process_rlt2 = self._pre_process_layer2(None, slf_attn_output_pp, + self._preprocess_cmd, + self._prepostprcess_dropout) + enc_attn_output_pp = self._multihead_attention_layer2( + pre_process_rlt2, enc_output, enc_output, dec_enc_attn_bias) + enc_attn_output = self._post_process_layer2( + slf_attn_output, enc_attn_output_pp, self._postprocess_cmd, + self._prepostprcess_dropout) + pre_process_rlt3 = self._pre_process_layer3(None, enc_attn_output, + self._preprocess_cmd, + self._prepostprcess_dropout) + ffd_output = self._positionwise_feed_forward_layer(pre_process_rlt3) + dec_output = self._post_process_layer3(enc_attn_output, ffd_output, + self._postprocess_cmd, + self._prepostprcess_dropout) + return dec_output + + +class DecoderLayer(Layer): + def __init__(self, + name_scope, + n_layer, + n_head, + d_key, + d_value, + d_model, + d_inner_hid, + prepostprocess_dropout, + attention_dropout, + relu_dropout, + preprocess_cmd, + postprocess_cmd, + caches=None, + gather_idx=None): + super(DecoderLayer, self).__init__(name_scope) + self._pre_process_layer = PrePostProcessLayer(self.full_name(), + preprocess_cmd, 3) + self._decoder_sub_layers = list() + self._n_layer = n_layer + self._preprocess_cmd = preprocess_cmd + self._prepostprocess_dropout = prepostprocess_dropout + for i in range(n_layer): + self._decoder_sub_layers.append( + self.add_sublayer( + 'dsl_%d' % i, + DecoderSubLayer( + self.full_name(), + n_head, + d_key, + d_value, + d_model, + d_inner_hid, + prepostprocess_dropout, + attention_dropout, + relu_dropout, + preprocess_cmd, + postprocess_cmd, + cache=None if caches is None else caches[i], + gather_idx=gather_idx))) + + def forward(self, dec_input, enc_output, dec_slf_attn_bias, + dec_enc_attn_bias): + for i in range(self._n_layer): + tmp_dec_output = self._decoder_sub_layers[i]( + dec_input, enc_output, dec_slf_attn_bias, dec_enc_attn_bias) + dec_input = tmp_dec_output + + dec_output = self._pre_process_layer(None, tmp_dec_output, + self._preprocess_cmd, + self._prepostprocess_dropout) + return dec_output + + +class WrapDecoderLayer(Layer): + def __init__(self, + name_scope, + trg_vocab_size, + max_length, + n_layer, + n_head, + d_key, + d_value, + d_model, + d_inner_hid, + prepostprocess_dropout, + attention_dropout, + relu_dropout, + preprocess_cmd, + postprocess_cmd, + weight_sharing, + caches=None, + gather_idx=None): + """ + The wrapper assembles together all needed layers for the encoder. + """ + super(WrapDecoderLayer, self).__init__(name_scope) + + self._prepare_decoder_layer = PrepareEncoderDecoderLayer( + self.full_name(), + trg_vocab_size, + d_model, + max_length, + prepostprocess_dropout, + word_emb_param_name=word_emb_param_names[1], + pos_enc_param_name=pos_enc_param_names[1]) + self._decoder_layer = DecoderLayer( + self.full_name(), + n_layer, + n_head, + d_key, + d_value, + d_model, + d_inner_hid, + prepostprocess_dropout, + attention_dropout, + relu_dropout, + preprocess_cmd, + postprocess_cmd, + caches=caches, + gather_idx=gather_idx) + self._weight_sharing = weight_sharing + if not weight_sharing: + self._fc = FC(self.full_name(), + size=trg_vocab_size, + bias_attr=False) + + def forward(self, dec_inputs=None, enc_output=None): + trg_word, trg_pos, trg_slf_attn_bias, trg_src_attn_bias = dec_inputs + dec_input = self._prepare_decoder_layer(trg_word, trg_pos) + dec_output = self._decoder_layer(dec_input, enc_output, + trg_slf_attn_bias, trg_src_attn_bias) + + dec_output_reshape = fluid.layers.reshape( + dec_output, shape=[-1, dec_output.shape[-1]], inplace=False) + + if self._weight_sharing: + predict = fluid.layers.matmul( + x=dec_output_reshape, + y=self._prepare_decoder_layer._input_emb._w, + transpose_y=True) + else: + predict = self._fc(dec_output_reshape) + + if dec_inputs is None: + # Return probs for independent decoder program. + predict_out = fluid.layers.softmax(predict) + return predict_out + return predict + + +class TransFormer(Layer): + def __init__(self, + name_scope, + src_vocab_size, + trg_vocab_size, + max_length, + n_layer, + n_head, + d_key, + d_value, + d_model, + d_inner_hid, + prepostprocess_dropout, + attention_dropout, + relu_dropout, + preprocess_cmd, + postprocess_cmd, + weight_sharing, + label_smooth_eps, + use_py_reader=False, + is_test=False): + super(TransFormer, self).__init__(name_scope) + self._label_smooth_eps = label_smooth_eps + self._trg_vocab_size = trg_vocab_size + if weight_sharing: + assert src_vocab_size == trg_vocab_size, ( + "Vocabularies in source and target should be same for weight sharing." + ) + self._wrap_encoder_layer = WrapEncoderLayer( + self.full_name(), src_vocab_size, max_length, n_layer, n_head, + d_key, d_value, d_model, d_inner_hid, prepostprocess_dropout, + attention_dropout, relu_dropout, preprocess_cmd, postprocess_cmd, + weight_sharing) + self._wrap_decoder_layer = WrapDecoderLayer( + self.full_name(), trg_vocab_size, max_length, n_layer, n_head, + d_key, d_value, d_model, d_inner_hid, prepostprocess_dropout, + attention_dropout, relu_dropout, preprocess_cmd, postprocess_cmd, + weight_sharing) + + if weight_sharing: + self._wrap_decoder_layer._prepare_decoder_layer._input_emb._w = self._wrap_encoder_layer._prepare_encoder_layer._input_emb._w + + def forward(self, enc_inputs, dec_inputs, label, weights): + enc_output = self._wrap_encoder_layer(enc_inputs) + predict = self._wrap_decoder_layer(dec_inputs, enc_output) + if self._label_smooth_eps: + label_out = fluid.layers.label_smooth( + label=fluid.layers.one_hot( + input=label, depth=self._trg_vocab_size), + epsilon=self._label_smooth_eps) + + cost = fluid.layers.softmax_with_cross_entropy( + logits=predict, + label=label_out, + soft_label=True if self._label_smooth_eps else False) + weighted_cost = cost * weights + sum_cost = fluid.layers.reduce_sum(weighted_cost) + token_num = fluid.layers.reduce_sum(weights) + token_num.stop_gradient = True + avg_cost = sum_cost / token_num + return sum_cost, avg_cost, predict, token_num + + +class TestImperativeTransformer(unittest.TestCase): + def test_transformer_float32(self): + seed = 90 + with guard(): + fluid.default_startup_program().random_seed = seed + fluid.default_main_program().random_seed = seed + transformer = TransFormer( + 'transformer', + ModelHyperParams.src_vocab_size, + ModelHyperParams.trg_vocab_size, + ModelHyperParams.max_length + 1, + ModelHyperParams.n_layer, + ModelHyperParams.n_head, + ModelHyperParams.d_key, + ModelHyperParams.d_value, + ModelHyperParams.d_model, + ModelHyperParams.d_inner_hid, + ModelHyperParams.prepostprocess_dropout, + ModelHyperParams.attention_dropout, + ModelHyperParams.relu_dropout, + ModelHyperParams.preprocess_cmd, + ModelHyperParams.postprocess_cmd, + ModelHyperParams.weight_sharing, + TrainTaskConfig.label_smooth_eps, + use_py_reader=use_py_reader, + is_test=False) + if sync: + lr_decay = fluid.layers.learning_rate_scheduler.noam_decay( + ModelHyperParams.d_model, TrainTaskConfig.warmup_steps) + with fluid.default_main_program()._lr_schedule_guard(): + learning_rate = lr_decay * TrainTaskConfig.learning_rate + optimizer = fluid.optimizer.Adam( + learning_rate=learning_rate, + beta1=TrainTaskConfig.beta1, + beta2=TrainTaskConfig.beta2, + epsilon=TrainTaskConfig.eps) + else: + optimizer = fluid.optimizer.SGD(learning_rate=0.003) + dy_param_init = dict() + dy_param_updated = dict() + for i in range(batch_num): + enc_inputs, dec_inputs, label, weights = create_data() + dy_sum_cost, dy_avg_cost, dy_predict, dy_token_num = transformer( + enc_inputs, dec_inputs, label, weights) + if i == 0: + for param in transformer.parameters(): + dy_param_init[param.name] = param._numpy() + + dy_avg_cost._backward() + optimizer.minimize(dy_avg_cost) + transformer.clear_gradients() + if i == batch_num - 1: + for param in transformer.parameters(): + dy_param_updated[param.name] = param._numpy() + + with new_program_scope(): + fluid.default_startup_program().random_seed = seed + fluid.default_main_program().random_seed = seed + transformer = TransFormer( + 'transformer', + ModelHyperParams.src_vocab_size, + ModelHyperParams.trg_vocab_size, + ModelHyperParams.max_length + 1, + ModelHyperParams.n_layer, + ModelHyperParams.n_head, + ModelHyperParams.d_key, + ModelHyperParams.d_value, + ModelHyperParams.d_model, + ModelHyperParams.d_inner_hid, + ModelHyperParams.prepostprocess_dropout, + ModelHyperParams.attention_dropout, + ModelHyperParams.relu_dropout, + ModelHyperParams.preprocess_cmd, + ModelHyperParams.postprocess_cmd, + ModelHyperParams.weight_sharing, + TrainTaskConfig.label_smooth_eps, + use_py_reader=use_py_reader, + is_test=False) + exe = fluid.Executor(fluid.CPUPlace( + ) if not core.is_compiled_with_cuda() else fluid.CUDAPlace(0)) + optimizer = fluid.optimizer.SGD(learning_rate=0.003) + + data_input_names = encoder_data_input_fields + decoder_data_input_fields[: + -1] + label_data_input_fields + all_inputs = make_all_inputs(data_input_names) + enc_inputs_len = len(encoder_data_input_fields) + dec_inputs_len = len(decoder_data_input_fields[:-1]) + enc_inputs = all_inputs[0:enc_inputs_len] + dec_inputs = all_inputs[enc_inputs_len:enc_inputs_len + + dec_inputs_len] + label = all_inputs[-2] + weights = all_inputs[-1] + static_param_updated = dict() + static_param_init = dict() + static_param_name_list = list() + static_sum_cost, static_avg_cost, static_predict, static_token_num = transformer( + enc_inputs, dec_inputs, label, weights) + + optimizer.minimize(static_avg_cost) + for param in transformer.parameters(): + static_param_name_list.append(param.name) + out = exe.run(fluid.default_startup_program(), + fetch_list=static_param_name_list) + for i in range(len(static_param_name_list)): + static_param_init[static_param_name_list[i]] = out[i] + static_sum_cost_value = None + static_avg_cost_value = None + static_predict_value = None + static_token_num_value = None + for i in range(batch_num): + feed_dict = create_feed_dict_list(create_data(True)) + fetch_list = [ + static_sum_cost, static_avg_cost, static_predict, + static_token_num + ] + fetch_list.extend(static_param_name_list) + + out = exe.run(fluid.default_main_program(), + feed=feed_dict, + fetch_list=fetch_list) + static_sum_cost_value = out[0] + static_avg_cost_value = out[1] + static_predict_value = out[2] + static_token_num_value = out[3] + if i == batch_num - 1: + for k in range(4, len(out)): + static_param_updated[static_param_name_list[k - + 4]] = out[k] + + self.assertTrue( + np.allclose(static_avg_cost_value, dy_avg_cost._numpy())) + self.assertTrue( + np.allclose(static_sum_cost_value, dy_sum_cost._numpy())) + self.assertTrue( + np.allclose( + static_predict_value, dy_predict._numpy(), atol=1e-5)) + self.assertTrue( + np.allclose(static_token_num_value, dy_token_num._numpy())) + for key, value in six.iteritems(static_param_init): + self.assertTrue(np.allclose(value, dy_param_init[key])) + for key, value in six.iteritems(static_param_updated): + self.assertTrue( + np.allclose( + value, dy_param_updated[key], atol=1e-4)) + + +if __name__ == '__main__': + unittest.main() diff --git a/python/paddle/fluid/tests/unittests/test_layers.py b/python/paddle/fluid/tests/unittests/test_layers.py index f343ed4e87..1429d24107 100644 --- a/python/paddle/fluid/tests/unittests/test_layers.py +++ b/python/paddle/fluid/tests/unittests/test_layers.py @@ -70,6 +70,34 @@ class LayerTest(unittest.TestCase): class TestLayer(LayerTest): + def test_layer_norm(self): + inp = np.ones([3, 32, 32], dtype='float32') + with self.static_graph(): + t = layers.data( + name='data', + shape=[3, 32, 32], + dtype='float32', + append_batch_size=False) + ret = layers.layer_norm(t) + static_ret = self.get_static_graph_result( + feed={'data': inp}, fetch_list=[ret])[0] + with self.static_graph(): + t = layers.data( + name='data', + shape=[3, 32, 32], + dtype='float32', + append_batch_size=False) + lm = nn.LayerNorm('layer_norm') + ret = lm(t) + static_ret2 = self.get_static_graph_result( + feed={'data': inp}, fetch_list=[ret])[0] + with self.dynamic_graph(): + lm = nn.LayerNorm('layer_norm') + dy_ret = lm(base.to_variable(inp)) + + self.assertTrue(np.allclose(static_ret, static_ret2)) + self.assertTrue(np.allclose(dy_ret._numpy(), static_ret2)) + def test_relu(self): with self.static_graph(): t = layers.data(name='t', shape=[3, 3], dtype='float32') From 7c5319ba121a6d73aeba0f06ce158680b160dcc2 Mon Sep 17 00:00:00 2001 From: Jiabin Yang Date: Tue, 26 Mar 2019 13:52:22 +0800 Subject: [PATCH 43/44] Fix/test imperative ptb rnn (#16433) * test=develop, fix ptb rnn * test=develop, change cdn to bj to pass ci * test=develop, fix ci --- .../unittests/test_imperative_ptb_rnn.py | 49 ++++++++++--------- 1 file changed, 27 insertions(+), 22 deletions(-) diff --git a/python/paddle/fluid/tests/unittests/test_imperative_ptb_rnn.py b/python/paddle/fluid/tests/unittests/test_imperative_ptb_rnn.py index 3b602303ae..460ba65a48 100644 --- a/python/paddle/fluid/tests/unittests/test_imperative_ptb_rnn.py +++ b/python/paddle/fluid/tests/unittests/test_imperative_ptb_rnn.py @@ -59,7 +59,7 @@ class SimpleLSTMRNN(fluid.imperative.Layer): dtype="float32", default_initializer=fluid.initializer.UniformInitializer( low=-self._init_scale, high=self._init_scale)) - self.weight_1_arr.append(weight_1) + self.weight_1_arr.append(self.add_parameter('w_%d' % i, weight_1)) bias_1 = self.create_parameter( attr=fluid.ParamAttr( initializer=fluid.initializer.UniformInitializer( @@ -67,7 +67,7 @@ class SimpleLSTMRNN(fluid.imperative.Layer): shape=[self._hidden_size * 4], dtype="float32", default_initializer=fluid.initializer.Constant(0.0)) - self.bias_arr.append(bias_1) + self.bias_arr.append(self.add_parameter('b_%d' % i, bias_1)) def forward(self, input_embedding, init_hidden=None, init_cell=None): self.cell_array = [] @@ -242,7 +242,7 @@ class TestImperativePtbRnn(unittest.TestCase): dy_loss = None last_hidden = None last_cell = None - batch_num = 50 + batch_num = 200 for i in range(batch_num): x_data = np.arange(12).reshape(4, 3).astype('int64') @@ -264,8 +264,10 @@ class TestImperativePtbRnn(unittest.TestCase): dy_param_init[param.name] = param._numpy() dy_loss._backward() sgd.minimize(dy_loss) - for param in ptb_model.parameters(): - dy_param_updated[param.name] = param._numpy() + ptb_model.clear_gradients() + if i == batch_num - 1: + for param in ptb_model.parameters(): + dy_param_updated[param.name] = param._numpy() with new_program_scope(): fluid.default_startup_program().random_seed = seed @@ -323,25 +325,28 @@ class TestImperativePtbRnn(unittest.TestCase): }, fetch_list=fetch_list) static_loss_value = out[0] - static_last_cell_value = out[1] - static_last_hidden_value = out[2] - for k in range(3, len(out)): - static_param_updated[static_param_name_list[k - 3]] = out[k] + static_last_hidden_value = out[1] + static_last_cell_value = out[2] + if i == batch_num - 1: + for k in range(3, len(out)): + static_param_updated[static_param_name_list[k - + 3]] = out[k] + + self.assertTrue(np.allclose(static_loss_value, dy_loss._numpy())) + self.assertTrue(np.allclose(static_last_cell_value, last_cell._numpy())) + self.assertTrue( + np.allclose(static_last_hidden_value, last_hidden._numpy())) + for key, value in six.iteritems(static_param_init): + # print("static_init name: {}, value {}".format(key, value)) + # print("dy_init name: {}, value {}".format(key, dy_param_init[key])) + self.assertTrue(np.allclose(value, dy_param_init[key], atol=1e-5)) + for key, value in six.iteritems(static_param_updated): + # print("static name: {}, value {}".format(key, value)) + # print("dy name: {}, value {}".format(key, dy_param_updated[key])) self.assertTrue( - np.allclose(static_loss_value.all(), dy_loss._numpy().all())) - self.assertTrue( - np.allclose(static_last_cell_value.all(), - last_cell._numpy().all())) - self.assertTrue( - np.allclose(static_last_hidden_value.all(), - last_hidden._numpy().all())) - for key, value in six.iteritems(static_param_init): - self.assertTrue( - np.allclose(value.all(), dy_param_init[key].all())) - for key, value in six.iteritems(static_param_updated): - self.assertTrue( - np.allclose(value.all(), dy_param_updated[key].all())) + np.allclose( + value, dy_param_updated[key], atol=1e-5)) if __name__ == '__main__': From ec6519e806add60ed1838572887e018b8f478c13 Mon Sep 17 00:00:00 2001 From: gongweibao Date: Tue, 26 Mar 2019 17:48:42 +0800 Subject: [PATCH 44/44] Fix allreducedep bug (#16443) --- .../framework/details/all_reduce_deps_pass.cc | 26 +++++++++++++++---- 1 file changed, 21 insertions(+), 5 deletions(-) diff --git a/paddle/fluid/framework/details/all_reduce_deps_pass.cc b/paddle/fluid/framework/details/all_reduce_deps_pass.cc index ff223e616f..c084410864 100644 --- a/paddle/fluid/framework/details/all_reduce_deps_pass.cc +++ b/paddle/fluid/framework/details/all_reduce_deps_pass.cc @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include #include @@ -52,13 +53,28 @@ std::unique_ptr AllReduceDepsPass::ApplyImpl( // Note that must assert topology sort is stable auto& ops = graph->Get>(kStaleProgramOpDescs); for (auto* op_desc : ops) { - auto outputs = op_desc->Outputs(); - for (auto& o_it : outputs) { - for (auto& v : o_it.second) { // values - vars[v] = order; + try { + bool is_bk_op = + static_cast(boost::get(op_desc->GetAttr( + OpProtoAndCheckerMaker::OpRoleAttrName())) & + static_cast(OpRole::kBackward)); + if (!is_bk_op) continue; + + auto backward_vars = + boost::get>(op_desc->GetNullableAttr( + OpProtoAndCheckerMaker::OpRoleVarAttrName())); + PADDLE_ENFORCE_EQ(backward_vars.size() % 2, 0); + + auto outputs = op_desc->Outputs(); + for (auto& o_it : outputs) { + for (auto& v : o_it.second) { // values + vars[v] = order; + VLOG(1) << "in all_reduce_deps_pass:" << v; + } } + order++; + } catch (boost::bad_get e) { } - order++; } std::vector dist_ops;