diff --git a/paddle/fluid/API.spec b/paddle/fluid/API.spec index 6b5ed1024449bdd5e73d69b3d6531e1d4e86126f..8f6797429c9dc21b4848961fcc9814f39b503308 100644 --- a/paddle/fluid/API.spec +++ b/paddle/fluid/API.spec @@ -32,6 +32,13 @@ paddle.fluid.BuildStrategy.ReduceStrategy.__init__ __init__(self: paddle.fluid.c paddle.fluid.BuildStrategy.__init__ __init__(self: paddle.fluid.core.ParallelExecutor.BuildStrategy) -> None paddle.fluid.create_lod_tensor ArgSpec(args=['data', 'recursive_seq_lens', 'place'], varargs=None, keywords=None, defaults=None) paddle.fluid.create_random_int_lodtensor ArgSpec(args=['recursive_seq_lens', 'base_shape', 'place', 'low', 'high'], varargs=None, keywords=None, defaults=None) +paddle.fluid.DataFeedDesc.__init__ ArgSpec(args=['self', 'proto_file'], varargs=None, keywords=None, defaults=None) +paddle.fluid.DataFeedDesc.desc ArgSpec(args=['self'], varargs=None, keywords=None, defaults=None) +paddle.fluid.DataFeedDesc.set_batch_size ArgSpec(args=['self', 'batch_size'], varargs=None, keywords=None, defaults=None) +paddle.fluid.DataFeedDesc.set_dense_slots ArgSpec(args=['self', 'dense_slots_name'], varargs=None, keywords=None, defaults=None) +paddle.fluid.DataFeedDesc.set_use_slots ArgSpec(args=['self', 'use_slots_name'], varargs=None, keywords=None, defaults=None) +paddle.fluid.AsyncExecutor.__init__ ArgSpec(args=['self', 'place'], varargs=None, keywords=None, defaults=(None,)) +paddle.fluid.AsyncExecutor.run ArgSpec(args=['self', 'program', 'data_feed', 'filelist', 'thread_num', 'fetch', 'debug'], varargs=None, keywords=None, defaults=(False,)) paddle.fluid.io.save_vars ArgSpec(args=['executor', 'dirname', 'main_program', 'vars', 'predicate', 'filename'], varargs=None, keywords=None, defaults=(None, None, None, None)) paddle.fluid.io.save_params ArgSpec(args=['executor', 'dirname', 'main_program', 'filename'], varargs=None, keywords=None, defaults=(None, None)) paddle.fluid.io.save_persistables ArgSpec(args=['executor', 'dirname', 'main_program', 'filename'], varargs=None, keywords=None, defaults=(None, None)) @@ -187,6 +194,7 @@ paddle.fluid.layers.grid_sampler ArgSpec(args=['x', 'grid', 'name'], varargs=Non paddle.fluid.layers.log_loss ArgSpec(args=['input', 'label', 'epsilon', 'name'], varargs=None, keywords=None, defaults=(0.0001, None)) paddle.fluid.layers.add_position_encoding ArgSpec(args=['input', 'alpha', 'beta', 'name'], varargs=None, keywords=None, defaults=(None,)) 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)) +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)) 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)) 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)) paddle.fluid.layers.read_file ArgSpec(args=['reader'], varargs=None, keywords=None, defaults=None) diff --git a/paddle/fluid/framework/CMakeLists.txt b/paddle/fluid/framework/CMakeLists.txt index 52946c7f11f90490b1af1347f20db236a8fe24af..9f5631b87cba62aa984f27b13418d61e12e86c8a 100644 --- a/paddle/fluid/framework/CMakeLists.txt +++ b/paddle/fluid/framework/CMakeLists.txt @@ -34,6 +34,7 @@ add_subdirectory(ir) add_subdirectory(details) # ddim lib proto_library(framework_proto SRCS framework.proto) +proto_library(async_executor_proto SRCS data_feed.proto) cc_library(ddim SRCS ddim.cc DEPS eigen3 boost) cc_test(ddim_test SRCS ddim_test.cc DEPS ddim) @@ -135,7 +136,7 @@ endif(NOT WIN32) cc_library(op_registry SRCS op_registry.cc DEPS op_proto_maker op_info operator glog proto_desc) nv_test(op_registry_test SRCS op_registry_test.cc DEPS op_registry) -py_proto_compile(framework_py_proto SRCS framework.proto) +py_proto_compile(framework_py_proto SRCS framework.proto data_feed.proto) # Generate an empty __init__.py to make framework_py_proto as a valid python module. add_custom_target(framework_py_proto_init ALL COMMAND ${CMAKE_COMMAND} -E touch __init__.py) add_dependencies(framework_py_proto framework_py_proto_init) @@ -157,18 +158,19 @@ endif(NOT WIN32) cc_library(lod_rank_table SRCS lod_rank_table.cc DEPS lod_tensor) cc_library(feed_fetch_method SRCS feed_fetch_method.cc DEPS lod_tensor scope glog) +cc_library(variable_helper SRCS variable_helper.cc DEPS lod_tensor) -cc_library(naive_executor SRCS naive_executor.cc DEPS op_registry device_context scope framework_proto glog lod_rank_table feed_fetch_method graph_to_program_pass) +cc_library(naive_executor SRCS naive_executor.cc DEPS op_registry device_context scope framework_proto glog lod_rank_table feed_fetch_method graph_to_program_pass variable_helper) if(WITH_DISTRIBUTE) - cc_library(executor SRCS executor.cc DEPS op_registry device_context scope framework_proto glog lod_rank_table feed_fetch_method sendrecvop_grpc cares grpc++_unsecure grpc_unsecure gpr graph_to_program_pass) + cc_library(executor SRCS executor.cc DEPS op_registry device_context scope framework_proto glog lod_rank_table feed_fetch_method sendrecvop_grpc cares grpc++_unsecure grpc_unsecure gpr graph_to_program_pass variable_helper) set(DISTRIBUTE_COMPILE_FLAGS "-Wno-non-virtual-dtor -Wno-error=non-virtual-dtor -Wno-error=delete-non-virtual-dtor") set_source_files_properties(executor.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) else() if(NOT WIN32) - cc_library(executor SRCS executor.cc DEPS op_registry device_context scope framework_proto glog lod_rank_table feed_fetch_method graph_to_program_pass ngraph_operator) + cc_library(executor SRCS executor.cc DEPS op_registry device_context scope framework_proto glog lod_rank_table feed_fetch_method graph_to_program_pass ngraph_operator variable_helper) else(NOT WIN32) - cc_library(executor SRCS executor.cc DEPS op_registry device_context scope framework_proto glog lod_rank_table feed_fetch_method graph_to_program_pass) + cc_library(executor SRCS executor.cc DEPS op_registry device_context scope framework_proto glog lod_rank_table feed_fetch_method graph_to_program_pass variable_helper) endif(NOT WIN32) cc_test(test_naive_executor SRCS naive_executor_test.cc DEPS naive_executor elementwise_add_op) endif() @@ -176,8 +178,11 @@ endif() cc_library(parallel_executor SRCS parallel_executor.cc DEPS threaded_ssa_graph_executor scope_buffered_ssa_graph_executor graph build_strategy - fast_threaded_ssa_graph_executor) + fast_threaded_ssa_graph_executor variable_helper) +cc_library(async_executor SRCS async_executor.cc data_feed.cc data_feed_factory.cc executor_thread_worker.cc DEPS op_registry device_context scope framework_proto glog lod_rank_table feed_fetch_method graph_to_program_pass async_executor_proto variable_helper) + +cc_test(data_feed_test SRCS data_feed_test.cc DEPS async_executor) cc_library(prune SRCS prune.cc DEPS framework_proto) cc_test(prune_test SRCS prune_test.cc DEPS op_info prune recurrent_op device_context) cc_test(var_type_inference_test SRCS var_type_inference_test.cc DEPS op_registry diff --git a/paddle/fluid/framework/async_executor.cc b/paddle/fluid/framework/async_executor.cc new file mode 100644 index 0000000000000000000000000000000000000000..afb2dd2f064384da39904f6aceead4fa915a80f2 --- /dev/null +++ b/paddle/fluid/framework/async_executor.cc @@ -0,0 +1,138 @@ +/* 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/async_executor.h" +#include "google/protobuf/io/zero_copy_stream_impl.h" +#include "google/protobuf/message.h" +#include "google/protobuf/text_format.h" + +#include "gflags/gflags.h" +#include "paddle/fluid/framework/data_feed_factory.h" +#include "paddle/fluid/framework/executor_thread_worker.h" +#include "paddle/fluid/framework/feed_fetch_method.h" +#include "paddle/fluid/framework/feed_fetch_type.h" +#include "paddle/fluid/framework/lod_rank_table.h" +#include "paddle/fluid/framework/lod_tensor_array.h" +#include "paddle/fluid/framework/op_registry.h" +#include "paddle/fluid/framework/reader.h" +#include "paddle/fluid/inference/io.h" +#include "paddle/fluid/platform/place.h" +#include "paddle/fluid/pybind/pybind.h" + +namespace paddle { +namespace framework { +AsyncExecutor::AsyncExecutor(Scope* scope, const platform::Place& place) + : root_scope_(scope), place_(place) {} + +void AsyncExecutor::CreateThreads( + ExecutorThreadWorker* worker, const ProgramDesc& main_program, + const std::shared_ptr& reader, + const std::vector& fetch_var_names, Scope* root_scope, + const int thread_index, const bool debug) { + worker->SetThreadId(thread_index); + worker->SetDebug(debug); + worker->SetRootScope(root_scope); + worker->CreateThreadResource(main_program, place_); + worker->SetDataFeed(reader); + worker->SetFetchVarNames(fetch_var_names); + worker->BindingDataFeedMemory(); +} + +void PrepareReaders(std::vector>& readers, // NOLINT + const int thread_num, const DataFeedDesc& data_feed_desc, + const std::vector& filelist) { + readers.resize(thread_num); + for (size_t i = 0; i < readers.size(); ++i) { + readers[i] = DataFeedFactory::CreateDataFeed(data_feed_desc.name()); + readers[i]->Init(data_feed_desc); // set batch_size and queue_size here + } + readers[0]->SetFileList(filelist); +} + +void AsyncExecutor::RunFromFile(const ProgramDesc& main_program, + const std::string& data_feed_desc_str, + const std::vector& filelist, + const int thread_num, + const std::vector& fetch_var_names, + const bool debug) { + std::vector threads; + + auto& block = main_program.Block(0); + for (auto var_name : fetch_var_names) { + auto var_desc = block.FindVar(var_name); + auto shapes = var_desc->GetShape(); + PADDLE_ENFORCE(shapes[shapes.size() - 1] == 1, + "var %s: Fetched var has wrong shape, " + "only variables with the last dimension size 1 supported", + var_name); + } + + DataFeedDesc data_feed_desc; + google::protobuf::TextFormat::ParseFromString(data_feed_desc_str, + &data_feed_desc); + + int actual_thread_num = thread_num; + int file_cnt = filelist.size(); + PADDLE_ENFORCE(file_cnt > 0, "File list cannot be empty"); + + if (actual_thread_num > file_cnt) { + VLOG(1) << "Thread num = " << thread_num << ", file num = " << file_cnt + << ". Changing thread_num = " << file_cnt; + actual_thread_num = file_cnt; + } + + /* + readerDesc: protobuf description for reader initlization + argument: class_name, batch_size, use_slot, queue_size, buffer_size, + padding_index + + reader: + 1) each thread has a reader, reader will read input data and + put it into input queue + 2) each reader has a Next() iterface, that can fetch an instance + from the input queue + */ + // todo: should be factory method for creating datafeed + std::vector> readers; + PrepareReaders(readers, actual_thread_num, data_feed_desc, filelist); + + std::vector> workers; + workers.resize(actual_thread_num); + for (auto& worker : workers) { + worker.reset(new ExecutorThreadWorker); + } + + // prepare thread resource here + for (int thidx = 0; thidx < actual_thread_num; ++thidx) { + CreateThreads(workers[thidx].get(), main_program, readers[thidx], + fetch_var_names, root_scope_, thidx, debug); + } + + // start executing ops in multiple threads + for (int thidx = 0; thidx < actual_thread_num; ++thidx) { + threads.push_back( + std::thread(&ExecutorThreadWorker::TrainFiles, workers[thidx].get())); + } + + for (auto& th : threads) { + th.join(); + } + + root_scope_->DropKids(); + + return; +} + +} // einit_modelnd namespace framework +} // end namespace paddle diff --git a/paddle/fluid/framework/async_executor.h b/paddle/fluid/framework/async_executor.h new file mode 100644 index 0000000000000000000000000000000000000000..f4d2a79ac592e02f49ec0b988c824dc98883fbf6 --- /dev/null +++ b/paddle/fluid/framework/async_executor.h @@ -0,0 +1,58 @@ +/* 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 // NOLINT +#include +#include +#include // NOLINT +#include +#include +#include "paddle/fluid/framework/data_feed.pb.h" +#include "paddle/fluid/framework/executor.h" +#include "paddle/fluid/framework/executor_thread_worker.h" +#include "paddle/fluid/framework/program_desc.h" +#include "paddle/fluid/framework/scope.h" + +namespace paddle { +namespace framework { +class AsyncExecutor { + public: + AsyncExecutor(Scope* scope, const platform::Place& place); + virtual ~AsyncExecutor() {} + void RunFromFile(const ProgramDesc& main_program, + const std::string& data_feed_desc_str, + const std::vector& filelist, + const int thread_num, + const std::vector& fetch_names, + const bool debug = false); + + private: + void CreateThreads(ExecutorThreadWorker* worker, + const ProgramDesc& main_program, + const std::shared_ptr& reader, + const std::vector& fetch_var_names, + Scope* root_scope, const int thread_index, + const bool debug); + + public: + Scope* root_scope_; + platform::Place place_; +}; + +} // namespace framework +} // namespace paddle diff --git a/paddle/fluid/framework/data_feed.cc b/paddle/fluid/framework/data_feed.cc new file mode 100644 index 0000000000000000000000000000000000000000..291d8ffc3c3334c2836e1651a8997984bba084e1 --- /dev/null +++ b/paddle/fluid/framework/data_feed.cc @@ -0,0 +1,386 @@ +/* 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 "google/protobuf/io/zero_copy_stream_impl.h" +#include "google/protobuf/message.h" +#include "google/protobuf/text_format.h" + +#include "gflags/gflags.h" +#include "paddle/fluid/framework/data_feed.h" +#include "paddle/fluid/framework/feed_fetch_method.h" +#include "paddle/fluid/framework/feed_fetch_type.h" + +namespace paddle { +namespace framework { + +std::vector DataFeed::filelist_; +size_t DataFeed::file_idx_; +std::mutex DataFeed::mutex_for_pick_file_; +bool DataFeed::finish_set_filelist_; + +void DataFeed::AddFeedVar(Variable* var, const std::string& name) { + CheckInit(); + for (size_t i = 0; i < use_slots_.size(); ++i) { + if (name == use_slots_[i]) { + if (use_slots_is_dense_[i]) { + feed_vec_[i] = MixTensor(var->GetMutable()); + } else { + feed_vec_[i] = MixTensor(var->GetMutable()); + } + } + } +} + +bool DataFeed::SetFileList(const std::vector& files) { + std::unique_lock lock(mutex_for_pick_file_); + CheckInit(); + if (finish_set_filelist_) { + VLOG(3) << "info: you have set the filelist."; + return false; + } + PADDLE_ENFORCE(files.size(), "You have set an empty filelist."); + filelist_.assign(files.begin(), files.end()); + file_idx_ = 0; + + finish_set_filelist_ = true; + return true; +} + +void DataFeed::SetBatchSize(int batch_size) { + PADDLE_ENFORCE(batch_size > 0, "Illegal batch size: %d.", batch_size); + default_batch_size_ = batch_size; +} + +bool DataFeed::PickOneFile(std::string* filename) { + std::unique_lock lock(mutex_for_pick_file_); + if (file_idx_ == filelist_.size()) { + return false; + } + *filename = filelist_[file_idx_++]; + return true; +} + +void DataFeed::CheckInit() { + PADDLE_ENFORCE(finish_init_, "Initialization did not succeed."); +} + +void DataFeed::CheckSetFileList() { + PADDLE_ENFORCE(finish_set_filelist_, "Set filelist did not succeed."); +} + +void DataFeed::CheckStart() { + PADDLE_ENFORCE(finish_start_, "Datafeed has not started running yet."); +} + +template +void PrivateQueueDataFeed::SetQueueSize(int queue_size) { + PADDLE_ENFORCE(queue_size > 0, "Illegal queue size: %d.", queue_size); + queue_size_ = queue_size; + queue_ = std::unique_ptr>( + new paddle::operators::reader::BlockingQueue(queue_size_)); +} + +template +bool PrivateQueueDataFeed::Start() { + CheckSetFileList(); + read_thread_ = std::thread(&PrivateQueueDataFeed::ReadThread, this); + read_thread_.detach(); + + finish_start_ = true; + return true; +} + +template +void PrivateQueueDataFeed::ReadThread() { + std::string filename; + while (PickOneFile(&filename)) { + file_.open(filename.c_str()); // is_text_feed + PADDLE_ENFORCE(file_.good(), "Open file<%s> fail.", filename.c_str()); + T instance; + while (ParseOneInstance(&instance)) { + queue_->Send(instance); + } + file_.close(); + } + queue_->Close(); +} + +template +int PrivateQueueDataFeed::Next() { + CheckStart(); + int index = 0; + T instance; + T ins_vec; + while (index < default_batch_size_) { + if (!queue_->Receive(&instance)) { + break; + } + AddInstanceToInsVec(&ins_vec, instance, index++); + } + batch_size_ = index; + if (batch_size_ != 0) { + PutToFeedVec(ins_vec); + } + return batch_size_; +} + +#ifdef _WIN32 +template class PrivateQueueDataFeed>; +#endif + +void MultiSlotDataFeed::Init( + const paddle::framework::DataFeedDesc& data_feed_desc) { + finish_init_ = false; + finish_set_filelist_ = false; + finish_start_ = false; + + PADDLE_ENFORCE(data_feed_desc.has_multi_slot_desc(), + "Multi_slot_desc has not been set."); + paddle::framework::MultiSlotDesc multi_slot_desc = + data_feed_desc.multi_slot_desc(); + SetBatchSize(data_feed_desc.batch_size()); + SetQueueSize(data_feed_desc.batch_size()); + size_t all_slot_num = multi_slot_desc.slots_size(); + all_slots_.resize(all_slot_num); + all_slots_type_.resize(all_slot_num); + use_slots_index_.resize(all_slot_num); + use_slots_.clear(); + use_slots_is_dense_.clear(); + for (size_t i = 0; i < all_slot_num; ++i) { + const auto& slot = multi_slot_desc.slots(i); + all_slots_[i] = slot.name(); + all_slots_type_[i] = slot.type(); + use_slots_index_[i] = slot.is_used() ? use_slots_.size() : -1; + if (slot.is_used()) { + use_slots_.push_back(all_slots_[i]); + use_slots_is_dense_.push_back(slot.is_dense()); + } + } + feed_vec_.resize(use_slots_.size()); + finish_init_ = true; +} + +bool MultiSlotDataFeed::CheckFile(const char* filename) { + CheckInit(); // get info of slots + std::ifstream fin(filename); + if (!fin.good()) { + VLOG(1) << "error: open file<" << filename << "> fail"; + return false; + } + std::string line; + int instance_cout = 0; + std::string all_slots_alias = ""; + for (const auto& alias : all_slots_) { + all_slots_alias += alias + " "; + } + std::string use_slots_alias = ""; + for (const auto& alias : use_slots_) { + use_slots_alias += alias + " "; + } + VLOG(3) << "total slots num: " << all_slots_.size(); + VLOG(3) << "total slots alias: " << all_slots_alias; + VLOG(3) << "used slots num: " << use_slots_.size(); + VLOG(3) << "used slots alias: " << use_slots_alias; + while (getline(fin, line)) { + ++instance_cout; + const char* str = line.c_str(); + char* endptr = const_cast(str); + int len = line.length(); + for (size_t i = 0; i < all_slots_.size(); ++i) { + int num = strtol(endptr, &endptr, 10); + if (num < 0) { + VLOG(0) << "error: the number of ids is a negative number: " << num; + VLOG(0) << "please check line<" << instance_cout << "> in file<" + << filename << ">"; + return false; + } else if (num == 0) { + VLOG(0) + << "error: the number of ids can not be zero, you need " + "padding it in data generator; or if there is something wrong" + " with the data, please check if the data contains unresolvable " + "characters."; + VLOG(0) << "please check line<" << instance_cout << "> in file<" + << filename << ">"; + return false; + } else if (errno == ERANGE || num > INT_MAX) { + VLOG(0) << "error: the number of ids greater than INT_MAX"; + VLOG(0) << "please check line<" << instance_cout << "> in file<" + << filename << ">"; + return false; + } + if (all_slots_type_[i] == "float") { + for (int i = 0; i < num; ++i) { + strtof(endptr, &endptr); + if (errno == ERANGE) { + VLOG(0) << "error: the value is out of the range of " + "representable values for float"; + VLOG(0) << "please check line<" << instance_cout << "> in file<" + << filename << ">"; + return false; + } + if (i + 1 != num && endptr - str == len) { + VLOG(0) << "error: there is a wrong with the number of ids."; + VLOG(0) << "please check line<" << instance_cout << "> in file<" + << filename << ">"; + return false; + } + } + } else if (all_slots_type_[i] == "uint64") { + for (int i = 0; i < num; ++i) { + strtoull(endptr, &endptr, 10); + if (errno == ERANGE) { + VLOG(0) << "error: the value is out of the range of " + "representable values for uint64_t"; + VLOG(0) << "please check line<" << instance_cout << "> in file<" + << filename << ">"; + return false; + } + if (i + 1 != num && endptr - str == len) { + VLOG(0) << "error: there is a wrong with the number of ids."; + VLOG(0) << "please check line<" << instance_cout << "> in file<" + << filename << ">"; + return false; + } + } + } else { + VLOG(0) << "error: this type<" << all_slots_type_[i] + << "> is not supported"; + return false; + } + } + // It may be added '\t' character to the end of the output of reduce + // task when processes data by Hadoop(when the output of the reduce + // task of Hadoop has only one field, it will add a '\t' at the end + // of the line by default, and you can use this option to avoid it: + // `-D mapred.textoutputformat.ignoreseparator=true`), which does + // not affect the correctness of the data. Therefore, it should be + // judged that the data is not normal when the end of each line of + // data contains characters which are not spaces. + while (endptr - str != len) { + if (!isspace(*(endptr++))) { + VLOG(0) + << "error: there is some extra characters at the end of the line."; + VLOG(0) << "please check line<" << instance_cout << "> in file<" + << filename << ">"; + return false; + } + } + } + VLOG(3) << "instances cout: " << instance_cout; + VLOG(3) << "The file format is correct"; + return true; +} + +bool MultiSlotDataFeed::ParseOneInstance(std::vector* instance) { + std::string line; + if (getline(file_, line)) { + int use_slots_num = use_slots_.size(); + instance->resize(use_slots_num); + // parse line + const char* str = line.c_str(); + char* endptr = const_cast(str); + int pos = 0; + for (size_t i = 0; i < use_slots_index_.size(); ++i) { + int idx = use_slots_index_[i]; + int num = strtol(&str[pos], &endptr, 10); + PADDLE_ENFORCE( + num, + "The number of ids can not be zero, you need padding " + "it in data generator; or if there is something wrong with " + "the data, please check if the data contains unresolvable " + "characters.\nplease check this error line: %s", + str); + if (idx != -1) { + (*instance)[idx].Init(all_slots_type_[i]); + if ((*instance)[idx].GetType()[0] == 'f') { // float + for (int j = 0; j < num; ++j) { + float feasign = strtof(endptr, &endptr); + (*instance)[idx].AddValue(feasign); + } + } else if ((*instance)[idx].GetType()[0] == 'u') { // uint64 + for (int j = 0; j < num; ++j) { + uint64_t feasign = (uint64_t)strtoull(endptr, &endptr, 10); + (*instance)[idx].AddValue(feasign); + } + } + pos = endptr - str; + } else { + for (int j = 0; j <= num; ++j) { + pos = line.find_first_of(' ', pos + 1); + } + } + } + } else { + return false; + } + return true; +} + +void MultiSlotDataFeed::AddInstanceToInsVec( + std::vector* ins_vec, + const std::vector& instance, int index) { + if (index == 0) { + ins_vec->resize(instance.size()); + for (size_t i = 0; i < instance.size(); ++i) { + (*ins_vec)[i].Init(instance[i].GetType()); + (*ins_vec)[i].InitOffset(); + } + } + for (size_t i = 0; i < instance.size(); ++i) { + (*ins_vec)[i].AddIns(instance[i]); + } +} + +void MultiSlotDataFeed::PutToFeedVec( + const std::vector& ins_vec) { + for (size_t i = 0; i < use_slots_.size(); ++i) { + const auto& type = ins_vec[i].GetType(); + const auto& offset = ins_vec[i].GetOffset(); + int total_instance = static_cast(offset.back()); + if (type[0] == 'f') { // float + const auto& feasign = ins_vec[i].GetFloatData(); + if (feed_vec_[i].IsDense()) { + int size_in_each_batch = total_instance / batch_size_; + float* tensor_ptr = feed_vec_[i].GetTensor()->mutable_data( + {batch_size_, size_in_each_batch}, platform::CPUPlace()); + memcpy(tensor_ptr, &feasign[0], total_instance * sizeof(float)); + } else { + float* tensor_ptr = feed_vec_[i].GetLoDTensor()->mutable_data( + {total_instance, 1}, platform::CPUPlace()); + memcpy(tensor_ptr, &feasign[0], total_instance * sizeof(float)); + LoD data_lod{offset}; + feed_vec_[i].GetLoDTensor()->set_lod(data_lod); + } + } else if (type[0] == 'u') { // uint64 + // no uint64_t type in paddlepaddle + const auto& feasign = ins_vec[i].GetUint64Data(); + if (feed_vec_[i].IsDense()) { + int size_in_each_batch = total_instance / batch_size_; + int64_t* tensor_ptr = feed_vec_[i].GetTensor()->mutable_data( + {batch_size_, size_in_each_batch}, platform::CPUPlace()); + memcpy(tensor_ptr, &feasign[0], total_instance * sizeof(int64_t)); + } else { + int64_t* tensor_ptr = + feed_vec_[i].GetLoDTensor()->mutable_data( + {total_instance, 1}, platform::CPUPlace()); + memcpy(tensor_ptr, &feasign[0], total_instance * sizeof(int64_t)); + LoD data_lod{offset}; + feed_vec_[i].GetLoDTensor()->set_lod(data_lod); + } + } + } +} + +} // namespace framework +} // namespace paddle diff --git a/paddle/fluid/framework/data_feed.h b/paddle/fluid/framework/data_feed.h new file mode 100644 index 0000000000000000000000000000000000000000..a7f8d1d31752af200145bc7934e7880910338e9d --- /dev/null +++ b/paddle/fluid/framework/data_feed.h @@ -0,0 +1,269 @@ +/* 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 // NOLINT +#include +#include // NOLINT +#include + +#include "paddle/fluid/framework/data_feed.pb.h" +#include "paddle/fluid/framework/lod_tensor.h" +#include "paddle/fluid/framework/reader.h" +#include "paddle/fluid/framework/variable.h" +#include "paddle/fluid/operators/reader/blocking_queue.h" + +namespace paddle { +namespace framework { + +// Pack Tensor type and LoDTensor type into MixTensor type, in order +// to record either Tensor or LoDTensor information at the same time. +class MixTensor { + public: + MixTensor() {} + explicit MixTensor(LoDTensor* lodtensor) { + is_dense_ = false; + lodtensor_ = lodtensor; + } + explicit MixTensor(Tensor* tensor) { + is_dense_ = true; + tensor_ = tensor; + } + bool IsDense() { return is_dense_; } + LoDTensor* GetLoDTensor() { + PADDLE_ENFORCE(!is_dense_, "Let a dense var return a LoDTensor ptr."); + return lodtensor_; + } + Tensor* GetTensor() { + PADDLE_ENFORCE(is_dense_, "Let a sparse var return a Tensor ptr."); + return tensor_; + } + + private: + bool is_dense_; + LoDTensor* lodtensor_; + Tensor* tensor_; +}; + +// DataFeed is the base virtual class for all ohther DataFeeds. +// It is used to read files and parse the data for subsequent trainer. +// Example: +// DataFeed* reader = +// paddle::framework::DataFeedFactory::CreateDataFeed(data_feed_name); +// reader->Init(data_feed_desc); // data_feed_desc is a protobuf object +// reader->SetFileList(filelist); +// const std::vector & use_slot_alias = +// reader->GetUseSlotAlias(); +// for (auto name: use_slot_alias){ // for binding memory +// reader->AddFeedVar(scope->Var(name), name); +// } +// reader->Start(); +// while (reader->Next()) { +// // trainer do something +// } +class DataFeed { + public: + DataFeed() {} + virtual ~DataFeed() {} + virtual void Init(const paddle::framework::DataFeedDesc& data_feed_desc) = 0; + virtual bool CheckFile(const char* filename) { + PADDLE_THROW("This function(CheckFile) is not implemented."); + } + // Set filelist for DataFeed. + // Pay attention that it must init all readers before call this function. + // Otherwise, Init() function will init finish_set_filelist_ flag. + virtual bool SetFileList(const std::vector& files); + virtual bool Start() = 0; + // The trainer calls the Next() function, and the DataFeed will load a new + // batch to the feed_vec. The return value of this function is the batch + // size of the current batch. + virtual int Next() = 0; + // Get all slots' alias which defined in protofile + virtual const std::vector& GetAllSlotAlias() { + return all_slots_; + } + // Get used slots' alias which defined in protofile + virtual const std::vector& GetUseSlotAlias() { + return use_slots_; + } + // This function is used for binding feed_vec memory + virtual void AddFeedVar(Variable* var, const std::string& name); + + protected: + // The following three functions are used to check if it is executed in this + // order: + // Init() -> SetFileList() -> Start() -> Next() + virtual void CheckInit(); + virtual void CheckSetFileList(); + virtual void CheckStart(); + virtual void SetBatchSize( + int batch); // batch size will be set in Init() function + // This function is used to pick one file from the global filelist(thread + // safe). + virtual bool PickOneFile(std::string* filename); + + static std::vector filelist_; + static size_t file_idx_; + static std::mutex mutex_for_pick_file_; + + // the alias of used slots, and its order is determined by + // data_feed_desc(proto object) + std::vector use_slots_; + std::vector use_slots_is_dense_; + + // the alias of all slots, and its order is determined by data_feed_desc(proto + // object) + std::vector all_slots_; + std::vector all_slots_type_; + std::vector + use_slots_index_; // -1: not used; >=0: the index of use_slots_ + + // The data read by DataFeed will be stored here + std::vector feed_vec_; + + // the batch size defined by user + int default_batch_size_; + // current batch size + int batch_size_; + + bool finish_init_; + static bool finish_set_filelist_; + bool finish_start_; +}; + +// PrivateQueueDataFeed is the base virtual class for ohther DataFeeds. +// It use a read-thread to read file and parse data to a private-queue +// (thread level), and get data from this queue when trainer call Next(). +template +class PrivateQueueDataFeed : public DataFeed { + public: + PrivateQueueDataFeed() {} + virtual ~PrivateQueueDataFeed() {} + virtual void Init(const paddle::framework::DataFeedDesc& data_feed_desc) = 0; + virtual bool Start(); + virtual int Next(); + + protected: + // The thread implementation function for reading file and parse. + virtual void ReadThread(); + // This function is used to set private-queue size, and the most + // efficient when the queue size is close to the batch size. + virtual void SetQueueSize(int queue_size); + // The reading and parsing method called in the ReadThread. + virtual bool ParseOneInstance(T* instance) = 0; + // This function is used to put instance to vec_ins + virtual void AddInstanceToInsVec(T* vec_ins, const T& instance, + int index) = 0; + // This function is used to put ins_vec to feed_vec + virtual void PutToFeedVec(const T& ins_vec) = 0; + + // The thread for read files + std::thread read_thread_; + // using ifstream one line and one line parse is faster + // than using fread one buffer and one buffer parse. + // for a 601M real data: + // ifstream one line and one line parse: 6034 ms + // fread one buffer and one buffer parse: 7097 ms + std::ifstream file_; + size_t queue_size_; + // The queue for store parsed data + std::unique_ptr> queue_; +}; + +// This class define the data type of instance(ins_vec) in MultiSlotDataFeed +class MultiSlotType { + public: + MultiSlotType() {} + ~MultiSlotType() {} + void Init(const std::string& type) { + CheckType(type); + if (type_[0] == 'f') { + float_feasign_.clear(); + } else if (type_[0] == 'u') { + uint64_feasign_.clear(); + } + type_ = type; + } + void InitOffset() { + offset_.resize(1); + // LoDTensor' lod is counted from 0, the size of lod + // is one size larger than the size of data. + offset_[0] = 0; + } + const std::vector& GetOffset() const { return offset_; } + void AddValue(const float v) { + CheckFloat(); + float_feasign_.push_back(v); + } + void AddValue(const uint64_t v) { + CheckUint64(); + uint64_feasign_.push_back(v); + } + void AddIns(const MultiSlotType& ins) { + if (ins.GetType()[0] == 'f') { // float + CheckFloat(); + auto& vec = ins.GetFloatData(); + offset_.push_back(offset_.back() + vec.size()); + float_feasign_.insert(float_feasign_.end(), vec.begin(), vec.end()); + } else if (ins.GetType()[0] == 'u') { // uint64 + CheckUint64(); + auto& vec = ins.GetUint64Data(); + offset_.push_back(offset_.back() + vec.size()); + uint64_feasign_.insert(uint64_feasign_.end(), vec.begin(), vec.end()); + } + } + const std::vector& GetFloatData() const { return float_feasign_; } + const std::vector& GetUint64Data() const { return uint64_feasign_; } + const std::string& GetType() const { return type_; } + + private: + void CheckType(const std::string& type) const { + PADDLE_ENFORCE((type == "uint64") || (type == "float"), + "There is no this type<%s>.", type); + } + void CheckFloat() const { + PADDLE_ENFORCE(type_[0] == 'f', "Add %s value to float slot.", type_); + } + void CheckUint64() const { + PADDLE_ENFORCE(type_[0] == 'u', "Add %s value to uint64 slot.", type_); + } + std::vector float_feasign_; + std::vector uint64_feasign_; + std::string type_; + std::vector offset_; +}; + +// This DataFeed is used to feed multi-slot type data. +// The format of multi-slot type data: +// [n feasign_0 feasign_1 ... feasign_n]* +class MultiSlotDataFeed + : public PrivateQueueDataFeed> { + public: + MultiSlotDataFeed() {} + virtual ~MultiSlotDataFeed() {} + virtual void Init(const paddle::framework::DataFeedDesc& data_feed_desc); + virtual bool CheckFile(const char* filename); + + protected: + virtual void AddInstanceToInsVec(std::vector* vec_ins, + const std::vector& instance, + int index); + virtual bool ParseOneInstance(std::vector* instance); + virtual void PutToFeedVec(const std::vector& ins_vec); +}; +} // namespace framework +} // namespace paddle diff --git a/paddle/fluid/framework/data_feed.proto b/paddle/fluid/framework/data_feed.proto new file mode 100644 index 0000000000000000000000000000000000000000..489fec08d86ccf61ece29bbba6d0204f25530b0f --- /dev/null +++ b/paddle/fluid/framework/data_feed.proto @@ -0,0 +1,30 @@ +/* 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. */ +syntax = "proto2"; +package paddle.framework; + +message Slot { + required string name = 1; + required string type = 2; + optional bool is_dense = 3 [ default = false ]; + optional bool is_used = 4 [ default = false ]; +} + +message MultiSlotDesc { repeated Slot slots = 1; } + +message DataFeedDesc { + optional string name = 1; + optional int32 batch_size = 2 [ default = 32 ]; + optional MultiSlotDesc multi_slot_desc = 3; +} diff --git a/paddle/fluid/framework/data_feed_factory.cc b/paddle/fluid/framework/data_feed_factory.cc new file mode 100644 index 0000000000000000000000000000000000000000..72148b9f7d343e19d60bb2be44d8270ad78d1412 --- /dev/null +++ b/paddle/fluid/framework/data_feed_factory.cc @@ -0,0 +1,64 @@ +/* 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/data_feed_factory.h" +#include +#include +#include + +#include "paddle/fluid/framework/data_feed.h" + +namespace paddle { +namespace framework { +typedef std::shared_ptr (*Createdata_feedFunction)(); +typedef std::unordered_map data_feedMap; +data_feedMap g_data_feed_map; + +#define REGISTER_DATAFEED_CLASS(data_feed_class) \ + namespace { \ + std::shared_ptr Creator_##data_feed_class() { \ + return std::shared_ptr(new data_feed_class); \ + } \ + class __Registerer_##data_feed_class { \ + public: \ + __Registerer_##data_feed_class() { \ + g_data_feed_map[#data_feed_class] = &Creator_##data_feed_class; \ + } \ + }; \ + __Registerer_##data_feed_class g_registerer_##data_feed_class; \ + } // namespace + +std::string DataFeedFactory::DataFeedTypeList() { + std::string data_feed_types; + for (auto iter = g_data_feed_map.begin(); iter != g_data_feed_map.end(); + ++iter) { + if (iter != g_data_feed_map.begin()) { + data_feed_types += ", "; + } + data_feed_types += iter->first; + } + return data_feed_types; +} + +std::shared_ptr DataFeedFactory::CreateDataFeed( + std::string data_feed_class) { + if (g_data_feed_map.count(data_feed_class) < 1) { + exit(-1); + } + return g_data_feed_map[data_feed_class](); +} + +REGISTER_DATAFEED_CLASS(MultiSlotDataFeed); +} // namespace framework +} // namespace paddle diff --git a/paddle/fluid/framework/data_feed_factory.h b/paddle/fluid/framework/data_feed_factory.h new file mode 100644 index 0000000000000000000000000000000000000000..13678edb0b8d084a0b3016d93f6e1bc32ce0169a --- /dev/null +++ b/paddle/fluid/framework/data_feed_factory.h @@ -0,0 +1,29 @@ +/* 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/data_feed.h" + +namespace paddle { +namespace framework { +class DataFeedFactory { + public: + static std::string DataFeedTypeList(); + static std::shared_ptr CreateDataFeed(std::string data_feed_class); +}; +} // namespace framework +} // namespace paddle diff --git a/paddle/fluid/framework/data_feed_test.cc b/paddle/fluid/framework/data_feed_test.cc new file mode 100644 index 0000000000000000000000000000000000000000..3974f8dbadf332801a822618d77f140db440b29d --- /dev/null +++ b/paddle/fluid/framework/data_feed_test.cc @@ -0,0 +1,337 @@ +// 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/data_feed.h" +#include +#include // NOLINT +#include +#include +#include +#include // NOLINT +#include +#include // NOLINT +#include +#include +#include "google/protobuf/io/zero_copy_stream_impl.h" +#include "google/protobuf/text_format.h" +#include "gtest/gtest.h" +#include "paddle/fluid/framework/data_feed_factory.h" +#include "paddle/fluid/framework/lod_tensor.h" +#include "paddle/fluid/framework/scope.h" + +paddle::framework::DataFeedDesc load_datafeed_param_from_file( + const char* filename) { + paddle::framework::DataFeedDesc data_feed_desc; + int file_descriptor = open(filename, O_RDONLY); + PADDLE_ENFORCE(file_descriptor != -1, "Can not open %s.", filename); + google::protobuf::io::FileInputStream fileInput(file_descriptor); + google::protobuf::TextFormat::Parse(&fileInput, &data_feed_desc); + close(file_descriptor); + return data_feed_desc; +} + +const std::vector load_filelist_from_file(const char* filename) { + std::vector filelist; + std::ifstream fin(filename); + PADDLE_ENFORCE(fin.good(), "Can not open %s.", filename); + std::string line; + while (getline(fin, line)) { + filelist.push_back(line); + } + fin.close(); + return filelist; +} + +void GenerateFileForTest(const char* protofile, const char* filelist) { + std::ofstream w_protofile(protofile); + w_protofile << "name: \"MultiSlotDataFeed\"\n" + "batch_size: 2\n" + "multi_slot_desc {\n" + " slots {\n" + " name: \"uint64_sparse_slot\"\n" + " type: \"uint64\"\n" + " is_dense: false\n" + " is_used: true\n" + " }\n" + " slots {\n" + " name: \"float_sparse_slot\"\n" + " type: \"float\"\n" + " is_dense: false\n" + " is_used: true\n" + " }\n" + " slots {\n" + " name: \"uint64_dense_slot\"\n" + " type: \"uint64\"\n" + " is_dense: true\n" + " is_used: true\n" + " }\n" + " slots {\n" + " name: \"float_dense_slot\"\n" + " type: \"float\"\n" + " is_dense: true\n" + " is_used: true\n" + " }\n" + " slots {\n" + " name: \"not_used_slot\"\n" + " type: \"uint64\"\n" + " is_dense: false\n" + " is_used: false\n" + " }\n" + "}"; + w_protofile.close(); + std::ofstream w_filelist(filelist); + int total_file = 4; + for (int i = 0; i < total_file; ++i) { + std::string filename = "TestMultiSlotDataFeed.data." + std::to_string(i); + w_filelist << filename; + if (i + 1 != total_file) { + w_filelist << std::endl; + } + std::ofstream w_datafile(filename.c_str()); + w_datafile << "3 3978 620 82 1 1926.08 1 1926 1 6.02 1 1996\n" + "2 1300 2983353 1 985.211 1 8 1 0.618 1 12\n" + "1 19260827 2 3.14 2.718 1 27 1 2.236 1 28\n"; + w_datafile.close(); + } + w_filelist.close(); +} + +class MultiTypeSet { + public: + MultiTypeSet() { + uint64_set_.clear(); + float_set_.clear(); + } + ~MultiTypeSet() {} + void AddValue(uint64_t v) { uint64_set_.insert(v); } + void AddValue(float v) { float_set_.insert(v); } + const std::set& GetUint64Set() const { return uint64_set_; } + const std::set& GetFloatSet() const { return float_set_; } + + private: + std::set uint64_set_; + std::set float_set_; +}; + +void GetElemSetFromReader(std::vector* reader_elem_set, + const paddle::framework::DataFeedDesc& data_feed_desc, + const std::vector& filelist, + const int thread_num) { + int used_slot_num = 0; + for (auto i = 0; i < data_feed_desc.multi_slot_desc().slots_size(); ++i) { + if (data_feed_desc.multi_slot_desc().slots(i).is_used()) { + ++used_slot_num; + } + } + reader_elem_set->resize(used_slot_num); + std::vector threads; + std::vector> readers; + readers.resize(thread_num); + for (int i = 0; i < thread_num; ++i) { + readers[i] = paddle::framework::DataFeedFactory::CreateDataFeed( + data_feed_desc.name()); + readers[i]->Init(data_feed_desc); + } + readers[0]->SetFileList(filelist); + std::mutex mu; + for (int idx = 0; idx < thread_num; ++idx) { + threads.emplace_back(std::thread([&, idx] { + std::unique_ptr scope( + new paddle::framework::Scope()); + const auto& multi_slot_desc = data_feed_desc.multi_slot_desc(); + std::map + lodtensor_targets; + std::map tensor_targets; + for (int i = 0; i < multi_slot_desc.slots_size(); ++i) { + const auto& slot = multi_slot_desc.slots(i); + if (slot.is_used()) { + const auto& name = slot.name(); + readers[idx]->AddFeedVar(scope->Var(name), name); + if (slot.is_dense()) { + tensor_targets[name] = + &scope->FindVar(name)->Get(); + } else { + lodtensor_targets[name] = + &scope->FindVar(name)->Get(); + } + } + } + readers[idx]->Start(); + while (readers[idx]->Next()) { + int index = 0; + for (int k = 0; k < multi_slot_desc.slots_size(); ++k) { + const auto& slot = multi_slot_desc.slots(k); + if (!slot.is_used()) { + continue; + } + if (slot.is_dense()) { // dense branch + const paddle::framework::Tensor* tens = tensor_targets[slot.name()]; + if (slot.type() == "uint64") { + const int64_t* data = tens->data(); + int batch_size = tens->dims()[0]; + int dim = tens->dims()[1]; + for (int i = 0; i < batch_size; ++i) { + for (int j = 0; j < dim; ++j) { + std::lock_guard lock(mu); + (*reader_elem_set)[index].AddValue( + (uint64_t)data[i * dim + j]); + } + } + } else if (slot.type() == "float") { + const float* data = tens->data(); + int batch_size = tens->dims()[0]; + int dim = tens->dims()[1]; + for (int i = 0; i < batch_size; ++i) { + for (int j = 0; j < dim; ++j) { + std::lock_guard lock(mu); + (*reader_elem_set)[index].AddValue(data[i * dim + j]); + } + } + } else { + PADDLE_THROW("Error type in proto file."); + } + } else { // sparse branch + const paddle::framework::LoDTensor* tens = + lodtensor_targets[slot.name()]; + if (slot.type() == "uint64") { + const int64_t* data = tens->data(); + for (size_t i = 0; i < tens->NumElements(); ++i) { + std::pair element = tens->lod_element(0, i); + for (size_t j = element.first; j < element.second; ++j) { + std::lock_guard lock(mu); + (*reader_elem_set)[index].AddValue((uint64_t)data[j]); + } + } + } else if (slot.type() == "float") { + const float* data = tens->data(); + for (size_t i = 0; i < tens->NumElements(); ++i) { + std::pair element = tens->lod_element(0, i); + for (size_t j = element.first; j < element.second; ++j) { + std::lock_guard lock(mu); + (*reader_elem_set)[index].AddValue(data[j]); + } + } + } else { + PADDLE_THROW("Error type in proto file."); + } + } // end sparse branch + ++index; + } // end slots loop + } // end while Next() + })); // end anonymous function + } + for (auto& th : threads) { + th.join(); + } +} + +void CheckIsUnorderedSame(const std::vector& s1, + const std::vector& s2) { + EXPECT_EQ(s1.size(), s2.size()); + for (size_t i = 0; i < s1.size(); ++i) { + // check for uint64 + const std::set& uint64_s1 = s1[i].GetUint64Set(); + const std::set& uint64_s2 = s2[i].GetUint64Set(); + EXPECT_EQ(uint64_s1.size(), uint64_s2.size()); + auto uint64_it1 = uint64_s1.begin(); + auto uint64_it2 = uint64_s2.begin(); + while (uint64_it1 != uint64_s1.end()) { + EXPECT_EQ(*uint64_it1, *uint64_it2); + ++uint64_it1; + ++uint64_it2; + } + // check for float + const std::set& float_s1 = s1[i].GetFloatSet(); + const std::set& float_s2 = s2[i].GetFloatSet(); + EXPECT_EQ(float_s1.size(), float_s2.size()); + auto float_it1 = float_s1.begin(); + auto float_it2 = float_s2.begin(); + while (float_it1 != float_s1.end()) { + EXPECT_EQ(*float_it1, *float_it2); + ++float_it1; + ++float_it2; + } + } +} + +void GetElemSetFromFile(std::vector* file_elem_set, + const paddle::framework::DataFeedDesc& data_feed_desc, + const std::vector& filelist) { + int used_slot_num = 0; + for (auto i = 0; i < data_feed_desc.multi_slot_desc().slots_size(); ++i) { + if (data_feed_desc.multi_slot_desc().slots(i).is_used()) { + ++used_slot_num; + } + } + file_elem_set->resize(used_slot_num); + for (const auto& file : filelist) { + std::ifstream fin(file.c_str()); + PADDLE_ENFORCE(fin.good(), "Can not open %s.", file.c_str()); + while (1) { + bool end_flag = false; + int index = 0; + for (auto i = 0; i < data_feed_desc.multi_slot_desc().slots_size(); ++i) { + int num; + if (fin >> num) { + auto slot = data_feed_desc.multi_slot_desc().slots(i); + auto type = slot.type(); + if (type == "uint64") { + while (num--) { + uint64_t feasign; + fin >> feasign; + if (slot.is_used()) { + (*file_elem_set)[index].AddValue(feasign); + } + } + } else if (type == "float") { + while (num--) { + float feasign; + fin >> feasign; + if (slot.is_used()) { + (*file_elem_set)[index].AddValue(feasign); + } + } + } else { + PADDLE_THROW("Error type in proto file."); + } + if (slot.is_used()) { + ++index; + } + } else { + end_flag = true; + break; + } + } + if (end_flag) { + break; + } + } + fin.close(); + } +} + +TEST(DataFeed, MultiSlotUnitTest) { + const char* protofile = "data_feed_desc.prototxt"; + const char* filelist_name = "filelist.txt"; + GenerateFileForTest(protofile, filelist_name); + const std::vector filelist = + load_filelist_from_file(filelist_name); + paddle::framework::DataFeedDesc data_feed_desc = + load_datafeed_param_from_file(protofile); + std::vector reader_elem_set; + std::vector file_elem_set; + GetElemSetFromReader(&reader_elem_set, data_feed_desc, filelist, 4); + GetElemSetFromFile(&file_elem_set, data_feed_desc, filelist); + CheckIsUnorderedSame(reader_elem_set, file_elem_set); +} diff --git a/paddle/fluid/framework/details/op_registry.h b/paddle/fluid/framework/details/op_registry.h index eea7e712f8f6e187cdceedce77cc76d1d4ca2101..1ce18c3d6b26c541beed668a113b7a4de7f0e79e 100644 --- a/paddle/fluid/framework/details/op_registry.h +++ b/paddle/fluid/framework/details/op_registry.h @@ -32,7 +32,9 @@ enum OpInfoFillType { kOpProtoAndCheckerMaker = 1, kGradOpDescMaker = 2, kVarTypeInference = 3, - kShapeInference = 4 + kShapeInference = 4, + kEstimateFlops = 5, + kUnknown = -1 }; template @@ -48,8 +50,10 @@ struct OpInfoFillTypeID { ? kVarTypeInference : (std::is_base_of::value ? kShapeInference - : static_cast( - -1))))); + : (std::is_base_of::value + ? kEstimateFlops + : kUnknown))))); } }; @@ -139,6 +143,16 @@ struct OpInfoFiller { } }; +template +struct OpInfoFiller { + void operator()(const char* op_tpe, OpInfo* info) const { + info->estimate_flops_ = [](InferShapeContext* ctx) { + T estimate_flops; + return estimate_flops(ctx); + }; + } +}; + } // namespace details } // namespace framework diff --git a/paddle/fluid/framework/details/scope_buffered_ssa_graph_executor.cc b/paddle/fluid/framework/details/scope_buffered_ssa_graph_executor.cc index e5b1eaa7318aecde1dbf89de8fe242a3008db97c..499246a9856bb3ba67a155c6f00c3ad06af50edf 100644 --- a/paddle/fluid/framework/details/scope_buffered_ssa_graph_executor.cc +++ b/paddle/fluid/framework/details/scope_buffered_ssa_graph_executor.cc @@ -16,7 +16,7 @@ #include #include #include -#include "paddle/fluid/framework/executor.h" +#include "paddle/fluid/framework/variable_helper.h" #include "paddle/fluid/platform/profiler.h" #ifdef PADDLE_WITH_CUDA #include "paddle/fluid/framework/details/reference_count_op_handle.h" diff --git a/paddle/fluid/framework/executor.cc b/paddle/fluid/framework/executor.cc index 96132a2c18233ca10d7bad4e26dfabadd39d84db..73cec21e20f2fd26e144872f1f7b5bb7065adb74 100644 --- a/paddle/fluid/framework/executor.cc +++ b/paddle/fluid/framework/executor.cc @@ -21,6 +21,7 @@ limitations under the License. */ #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/framework/reader.h" #include "paddle/fluid/framework/transfer_scope_cache.h" +#include "paddle/fluid/framework/variable_helper.h" #include "paddle/fluid/operators/detail/macros.h" #include "paddle/fluid/platform/place.h" #include "paddle/fluid/platform/profiler.h" @@ -114,36 +115,6 @@ void Executor::Close() { #endif } -void InitializeVariable(Variable* var, proto::VarType::Type var_type) { - if (var_type == proto::VarType::LOD_TENSOR) { - var->GetMutable(); - } else if (var_type == proto::VarType::SELECTED_ROWS) { - var->GetMutable(); - } else if (var_type == proto::VarType::FEED_MINIBATCH) { - var->GetMutable(); - } else if (var_type == proto::VarType::FETCH_LIST) { - var->GetMutable(); - } else if (var_type == proto::VarType::STEP_SCOPES) { - var->GetMutable>(); - } else if (var_type == proto::VarType::LOD_RANK_TABLE) { - var->GetMutable(); - } else if (var_type == proto::VarType::LOD_TENSOR_ARRAY) { - var->GetMutable(); - } else if (var_type == proto::VarType::PLACE_LIST) { - var->GetMutable(); - } else if (var_type == proto::VarType::READER) { - var->GetMutable(); - } else if (var_type == proto::VarType::RAW) { - // GetMutable will be called in operator - } else { - PADDLE_THROW( - "Variable type %d is not in " - "[LOD_TENSOR, SELECTED_ROWS, FEED_MINIBATCH, FETCH_LIST, " - "LOD_RANK_TABLE, PLACE_LIST, READER, RAW]", - var_type); - } -} - void Executor::CreateVariables(const ProgramDesc& pdesc, Scope* scope, int block_id) { auto& global_block = pdesc.Block(block_id); diff --git a/paddle/fluid/framework/executor.h b/paddle/fluid/framework/executor.h index 36b36d49c2728dbef93042158dffa26d8f56d529..2d47903ffbd8d821b7c31386b225fe5e65ca2720 100644 --- a/paddle/fluid/framework/executor.h +++ b/paddle/fluid/framework/executor.h @@ -26,7 +26,6 @@ limitations under the License. */ namespace paddle { namespace framework { -extern void InitializeVariable(Variable* var, proto::VarType::Type var_type); template std::unordered_map GetNonPersistableReferenceCount( diff --git a/paddle/fluid/framework/executor_thread_worker.cc b/paddle/fluid/framework/executor_thread_worker.cc new file mode 100644 index 0000000000000000000000000000000000000000..4e4001e979fdd0774779fa288402c7847af90637 --- /dev/null +++ b/paddle/fluid/framework/executor_thread_worker.cc @@ -0,0 +1,223 @@ +/* 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/executor_thread_worker.h" +#include "google/protobuf/io/zero_copy_stream_impl.h" +#include "google/protobuf/message.h" +#include "google/protobuf/text_format.h" + +#include "gflags/gflags.h" +#include "paddle/fluid/framework/feed_fetch_method.h" +#include "paddle/fluid/framework/feed_fetch_type.h" +#include "paddle/fluid/framework/lod_rank_table.h" +#include "paddle/fluid/framework/lod_tensor_array.h" +#include "paddle/fluid/framework/op_registry.h" +#include "paddle/fluid/framework/reader.h" +#include "paddle/fluid/framework/variable_helper.h" +#include "paddle/fluid/inference/io.h" +#include "paddle/fluid/platform/place.h" +#include "paddle/fluid/pybind/pybind.h" +namespace paddle { +namespace framework { + +void ExecutorThreadWorker::CreateThreadOperators(const ProgramDesc& program) { + auto& block = program.Block(0); + op_names_.clear(); + for (auto& op_desc : block.AllOps()) { + std::unique_ptr local_op = OpRegistry::CreateOp(*op_desc); + op_names_.push_back(op_desc->Type()); + OperatorBase* local_op_ptr = local_op.release(); + ops_.push_back(local_op_ptr); + continue; + } +} + +void ExecutorThreadWorker::CreateThreadResource( + const framework::ProgramDesc& program, + const paddle::platform::Place& place) { + CreateThreadScope(program); + CreateThreadOperators(program); + SetMainProgram(program); + SetPlace(place); +} + +void ExecutorThreadWorker::CreateThreadScope(const ProgramDesc& program) { + auto& block = program.Block(0); + + PADDLE_ENFORCE_NOT_NULL( + root_scope_, "root_scope should be set before creating thread scope"); + + thread_scope_ = &root_scope_->NewScope(); + for (auto& var : block.AllVars()) { + if (var->Persistable()) { + auto* ptr = root_scope_->Var(var->Name()); + InitializeVariable(ptr, var->GetType()); + } else { + auto* ptr = thread_scope_->Var(var->Name()); + InitializeVariable(ptr, var->GetType()); + } + } +} + +void ExecutorThreadWorker::SetDataFeed( + const std::shared_ptr& datafeed) { + thread_reader_ = datafeed; +} + +void ExecutorThreadWorker::BindingDataFeedMemory() { + const std::vector& input_feed = + thread_reader_->GetUseSlotAlias(); + for (auto name : input_feed) { + thread_reader_->AddFeedVar(thread_scope_->Var(name), name); + } +} + +void ExecutorThreadWorker::SetFetchVarNames( + const std::vector& fetch_var_names) { + fetch_var_names_.clear(); + fetch_var_names_.insert(fetch_var_names_.end(), fetch_var_names.begin(), + fetch_var_names.end()); +} + +void ExecutorThreadWorker::SetDevice() { +#if defined _WIN32 || defined __APPLE__ + return; +#else + static unsigned concurrency_cap = std::thread::hardware_concurrency(); + int thread_id = this->thread_id_; + + if (thread_id < concurrency_cap) { + unsigned proc = thread_id; + + cpu_set_t mask; + CPU_ZERO(&mask); + CPU_SET(proc, &mask); + + if (-1 == sched_setaffinity(0, sizeof(mask), &mask)) { + VLOG(1) << "WARNING: Failed to set thread affinity for thread " + << thread_id; + } else { + CPU_ZERO(&mask); + if ((0 != sched_getaffinity(0, sizeof(mask), &mask)) || + (CPU_ISSET(proc, &mask) == 0)) { + VLOG(3) << "WARNING: Failed to set thread affinity for thread " + << thread_id; + } + } + } else { + VLOG(1) << "WARNING: Failed to set thread affinity for thread " + << thread_id; + } +#endif +} + +template +void print_lod_tensor(std::string var_name, const LoDTensor& lod_tensor) { + auto inspect = lod_tensor.data(); + auto element_num = lod_tensor.numel(); + + std::ostringstream sstream; + sstream << var_name << " (element num " << element_num << "): ["; + sstream << inspect[0]; + for (int j = 1; j < element_num; ++j) { + sstream << " " << inspect[j]; + } + sstream << "]"; + + std::cout << sstream.str() << std::endl; +} + +void print_fetch_var(Scope* scope, std::string var_name) { + const LoDTensor& tensor = scope->FindVar(var_name)->Get(); + + if (std::type_index(tensor.type()) == + std::type_index(typeid(platform::float16))) { + print_lod_tensor(var_name, tensor); + } else if (std::type_index(tensor.type()) == std::type_index(typeid(float))) { + print_lod_tensor(var_name, tensor); + } else if (std::type_index(tensor.type()) == + std::type_index(typeid(double))) { + print_lod_tensor(var_name, tensor); + } else if (std::type_index(tensor.type()) == std::type_index(typeid(int))) { + print_lod_tensor(var_name, tensor); + } else if (std::type_index(tensor.type()) == + std::type_index(typeid(int64_t))) { + print_lod_tensor(var_name, tensor); + } else if (std::type_index(tensor.type()) == std::type_index(typeid(bool))) { + print_lod_tensor(var_name, tensor); + } else if (std::type_index(tensor.type()) == + std::type_index(typeid(uint8_t))) { + print_lod_tensor(var_name, tensor); + } else if (std::type_index(tensor.type()) == + std::type_index(typeid(int16_t))) { + print_lod_tensor(var_name, tensor); + } else if (std::type_index(tensor.type()) == + std::type_index(typeid(int8_t))) { + print_lod_tensor(var_name, tensor); + } else { + VLOG(1) << "print_fetch_var: unrecognized data type:" + << tensor.type().name(); + } + + return; +} + +void ExecutorThreadWorker::TrainFiles() { + // todo: configurable + SetDevice(); + + int fetch_var_num = fetch_var_names_.size(); + fetch_values_.clear(); + fetch_values_.resize(fetch_var_num); + + thread_reader_->Start(); + + int cur_batch; + int batch_cnt = 0; + while ((cur_batch = thread_reader_->Next()) > 0) { + // executor run here + for (auto& op : ops_) { + op->Run(*thread_scope_, place_); + } + + ++batch_cnt; + thread_scope_->DropKids(); + + if (debug_ == false || thread_id_ != 0) { + continue; + } + + for (int i = 0; i < fetch_var_num; ++i) { + print_fetch_var(thread_scope_, fetch_var_names_[i]); + } // end for (int i = 0...) + } // end while () +} + +void ExecutorThreadWorker::SetThreadId(int tid) { thread_id_ = tid; } + +void ExecutorThreadWorker::SetPlace(const platform::Place& place) { + place_ = place; +} + +void ExecutorThreadWorker::SetMainProgram( + const ProgramDesc& main_program_desc) { + main_program_.reset(new ProgramDesc(main_program_desc)); +} + +void ExecutorThreadWorker::SetRootScope(Scope* g_scope) { + root_scope_ = g_scope; +} + +} // einit_modelnd namespace framework +} // end namespace paddle diff --git a/paddle/fluid/framework/executor_thread_worker.h b/paddle/fluid/framework/executor_thread_worker.h new file mode 100644 index 0000000000000000000000000000000000000000..13ec2442c46459116320236bf98f23c91340f389 --- /dev/null +++ b/paddle/fluid/framework/executor_thread_worker.h @@ -0,0 +1,88 @@ +/* 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 // NOLINT +#include +#include +#include // NOLINT +#include +#include "paddle/fluid/framework/data_feed.h" +#include "paddle/fluid/framework/executor.h" +#include "paddle/fluid/framework/program_desc.h" +#include "paddle/fluid/framework/scope.h" + +namespace paddle { +namespace framework { +void CreateTensor(Variable* var, proto::VarType::Type var_type); + +class ExecutorThreadWorker { + public: + ExecutorThreadWorker() + : thread_id_(-1), root_scope_(NULL), thread_scope_(NULL), debug_(false) {} + ~ExecutorThreadWorker() {} + + void CreateThreadResource(const framework::ProgramDesc& program, + const paddle::platform::Place& place); + void SetThreadId(int tid); + void SetDebug(const bool debug) { debug_ = debug; } + void SetRootScope(Scope* g_scope); + // set cpu device in this function + // cpu binding is used by default + void SetDevice(); + // since we read data into memory that can not be accessed by program + // we need to bind memory of data with corresponding variables in program + // this function should be called after data feed is set + void BindingDataFeedMemory(); + // set data feed declared in executor + void SetDataFeed(const std::shared_ptr& datafeed); + // A multi-thread training function + void TrainFiles(); + // set fetch variable names from python interface assigned by users + void SetFetchVarNames(const std::vector& fetch_var_names); + + private: + void CreateThreadScope(const framework::ProgramDesc& program); + void CreateThreadOperators(const framework::ProgramDesc& program); + void SetMainProgram(const ProgramDesc& main_program_desc); + void SetPlace(const paddle::platform::Place& place); + + protected: + // thread index + std::shared_ptr thread_reader_; // shared queue, thread buffer + int thread_id_; + // operator name + std::vector op_names_; + // thread level, local operators for forward and backward + std::vector ops_; + // main program for training + std::unique_ptr main_program_; + // execution place + platform::Place place_; + // root scope for model parameters + Scope* root_scope_; + // a thread scope, father scope is global score which is shared + Scope* thread_scope_; + + private: + std::vector fetch_var_names_; + std::vector> fetch_values_; + bool debug_; +}; + +} // namespace framework +} // namespace paddle diff --git a/paddle/fluid/framework/naive_executor.cc b/paddle/fluid/framework/naive_executor.cc index e8295639520b5838dce3c9c9e443cc846bd9c1ec..f1642bc0d2b10f97295e80ee201db8f83bfd06ef 100644 --- a/paddle/fluid/framework/naive_executor.cc +++ b/paddle/fluid/framework/naive_executor.cc @@ -21,42 +21,11 @@ #include "paddle/fluid/framework/naive_executor.h" #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/framework/reader.h" +#include "paddle/fluid/framework/variable_helper.h" #include "paddle/fluid/string/pretty_log.h" namespace paddle { namespace framework { - -// These code can be shared with Executor. -static void InitializeVariable(Variable *var, proto::VarType::Type var_type) { - if (var_type == proto::VarType::LOD_TENSOR) { - var->GetMutable(); - } else if (var_type == proto::VarType::SELECTED_ROWS) { - var->GetMutable(); - } else if (var_type == proto::VarType::FEED_MINIBATCH) { - var->GetMutable(); - } else if (var_type == proto::VarType::FETCH_LIST) { - var->GetMutable(); - } else if (var_type == proto::VarType::STEP_SCOPES) { - var->GetMutable>(); - } else if (var_type == proto::VarType::LOD_RANK_TABLE) { - var->GetMutable(); - } else if (var_type == proto::VarType::LOD_TENSOR_ARRAY) { - var->GetMutable(); - } else if (var_type == proto::VarType::PLACE_LIST) { - var->GetMutable(); - } else if (var_type == proto::VarType::READER) { - var->GetMutable(); - } else if (var_type == proto::VarType::RAW) { - // GetMutable will be called in operator - } else { - PADDLE_THROW( - "Variable type %d is not in " - "[LOD_TENSOR, SELECTED_ROWS, FEED_MINIBATCH, FETCH_LIST, " - "LOD_RANK_TABLE, PLACE_LIST, READER, CHANNEL, RAW]", - var_type); - } -} - void NaiveExecutor::Prepare(Scope *scope, const ProgramDesc &program_desc, int block_id, bool with_feed_fetch_ops) { if (!scope) { diff --git a/paddle/fluid/framework/op_info.h b/paddle/fluid/framework/op_info.h index 19e5c2c73eac74dee030a4f7820531800f737e4e..e0bf5ed999f580f217af285bf97d0bc0232f1ded 100644 --- a/paddle/fluid/framework/op_info.h +++ b/paddle/fluid/framework/op_info.h @@ -31,6 +31,12 @@ class InferShapeBase { virtual void operator()(InferShapeContext*) const = 0; }; +class EstimateFlopsBase { + public: + virtual ~EstimateFlopsBase() = default; + virtual size_t operator()(InferShapeContext*) const = 0; +}; + struct OpInfo { OpCreator creator_; GradOpMakerFN grad_op_maker_; @@ -38,6 +44,7 @@ struct OpInfo { OpAttrChecker* checker_{nullptr}; InferVarTypeFN infer_var_type_; InferShapeFN infer_shape_; + EstimateFlopsFN estimate_flops_; bool HasOpProtoAndChecker() const { return proto_ != nullptr && checker_ != nullptr; diff --git a/paddle/fluid/framework/type_defs.h b/paddle/fluid/framework/type_defs.h index 2de6233a9e0d320ec9a06d547db3575eb61925c0..cdc5fa6862e3b2a2784151302f15540a0e9db8ff 100644 --- a/paddle/fluid/framework/type_defs.h +++ b/paddle/fluid/framework/type_defs.h @@ -54,5 +54,7 @@ using InferVarTypeFN = using InferShapeFN = std::function; +using EstimateFlopsFN = std::function; + } // namespace framework } // namespace paddle diff --git a/paddle/fluid/framework/variable_helper.cc b/paddle/fluid/framework/variable_helper.cc new file mode 100644 index 0000000000000000000000000000000000000000..fc4525549caeebb06dea766ccb123b5ebc6d5b13 --- /dev/null +++ b/paddle/fluid/framework/variable_helper.cc @@ -0,0 +1,60 @@ +/* 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/variable_helper.h" + +#include + +#include "paddle/fluid/framework/feed_fetch_type.h" +#include "paddle/fluid/framework/lod_rank_table.h" +#include "paddle/fluid/framework/lod_tensor.h" +#include "paddle/fluid/framework/lod_tensor_array.h" +#include "paddle/fluid/framework/reader.h" +#include "paddle/fluid/framework/scope.h" +#include "paddle/fluid/framework/selected_rows.h" +#include "paddle/fluid/platform/place.h" + +namespace paddle { +namespace framework { +void InitializeVariable(Variable* var, proto::VarType::Type var_type) { + if (var_type == proto::VarType::LOD_TENSOR) { + var->GetMutable(); + } else if (var_type == proto::VarType::SELECTED_ROWS) { + var->GetMutable(); + } else if (var_type == proto::VarType::FEED_MINIBATCH) { + var->GetMutable(); + } else if (var_type == proto::VarType::FETCH_LIST) { + var->GetMutable(); + } else if (var_type == proto::VarType::STEP_SCOPES) { + var->GetMutable>(); + } else if (var_type == proto::VarType::LOD_RANK_TABLE) { + var->GetMutable(); + } else if (var_type == proto::VarType::LOD_TENSOR_ARRAY) { + var->GetMutable(); + } else if (var_type == proto::VarType::PLACE_LIST) { + var->GetMutable(); + } else if (var_type == proto::VarType::READER) { + var->GetMutable(); + } else if (var_type == proto::VarType::RAW) { + // GetMutable will be called in operator + } else { + PADDLE_THROW( + "Variable type %d is not in " + "[LOD_TENSOR, SELECTED_ROWS, FEED_MINIBATCH, FETCH_LIST, " + "LOD_RANK_TABLE, PLACE_LIST, READER, RAW]", + var_type); + } +} +} // namespace framework +} // namespace paddle diff --git a/paddle/fluid/framework/variable_helper.h b/paddle/fluid/framework/variable_helper.h new file mode 100644 index 0000000000000000000000000000000000000000..0e0c72c3621dce0a6b372f9a9110a63fbc0a1d71 --- /dev/null +++ b/paddle/fluid/framework/variable_helper.h @@ -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. */ +#pragma once + +#include "paddle/fluid/framework/framework.pb.h" +#include "paddle/fluid/framework/variable.h" +namespace paddle { +namespace framework { +void InitializeVariable(Variable *var, proto::VarType::Type var_type); +} +} diff --git a/paddle/fluid/inference/analysis/analysis_pass.h b/paddle/fluid/inference/analysis/analysis_pass.h index 299f235a74ae0ffb663be61079607d8ac1105a97..d5a972fab3beae4d4e2e512d1ccda3f0b8356682 100644 --- a/paddle/fluid/inference/analysis/analysis_pass.h +++ b/paddle/fluid/inference/analysis/analysis_pass.h @@ -46,8 +46,6 @@ class AnalysisPass { protected: // User should implement these. virtual void RunImpl(Argument* argument) = 0; - - Argument* argument_{nullptr}; }; } // namespace analysis diff --git a/paddle/fluid/inference/api/analysis_predictor.cc b/paddle/fluid/inference/api/analysis_predictor.cc index 1862f61f0f4b94c9fa9636e876e943113d9aebd4..391330a7c0f2dda731fe8455fdab81b276e3f272 100644 --- a/paddle/fluid/inference/api/analysis_predictor.cc +++ b/paddle/fluid/inference/api/analysis_predictor.cc @@ -190,9 +190,13 @@ bool AnalysisPredictor::Run(const std::vector &inputs, } VLOG(3) << "predict cost: " << timer.toc() << "ms"; - // Fix TensorArray reuse not cleaned bug. - tensor_array_batch_cleaner_.CollectTensorArrays(scope_.get()); - tensor_array_batch_cleaner_.ResetTensorArray(); + // All the containers in the scope will be hold in inference, but the + // operators assume that the container will be reset after each batch. + // Here is a bugfix, collect all the container variables, and reset then to a + // bool; the next time, the operator will call MutableData and construct a new + // container again, so that the container will be empty for each batch. + tensor_array_batch_cleaner_.CollectNoTensorVars(sub_scope_); + tensor_array_batch_cleaner_.ResetNoTensorVars(); return true; } @@ -417,7 +421,7 @@ std::unique_ptr AnalysisPredictor::GetOutputTensor( bool AnalysisPredictor::ZeroCopyRun() { executor_->Run(); // Fix TensorArray reuse not cleaned bug. - tensor_array_batch_cleaner_.CollectTensorArrays(scope_.get()); + tensor_array_batch_cleaner_.CollectTensorArrays(sub_scope_); tensor_array_batch_cleaner_.ResetTensorArray(); return true; } diff --git a/paddle/fluid/inference/api/api_impl.cc b/paddle/fluid/inference/api/api_impl.cc index 74369e886692fef3172d24c637b03a5bcf81a6c2..4c5b412a2c1717b8edbb17c238caaa11aeccebd3 100644 --- a/paddle/fluid/inference/api/api_impl.cc +++ b/paddle/fluid/inference/api/api_impl.cc @@ -154,9 +154,9 @@ bool NativePaddlePredictor::Run(const std::vector &inputs, } VLOG(3) << "predict cost: " << timer.toc() << "ms"; - // Fix TensorArray reuse not cleaned bug. - tensor_array_batch_cleaner_.CollectTensorArrays(scope_.get()); - tensor_array_batch_cleaner_.ResetTensorArray(); + // For some other vector like containers not cleaned after each batch. + tensor_array_batch_cleaner_.CollectNoTensorVars(scope_.get()); + tensor_array_batch_cleaner_.ResetNoTensorVars(); return true; } diff --git a/paddle/fluid/inference/api/details/reset_tensor_array.cc b/paddle/fluid/inference/api/details/reset_tensor_array.cc index 4ae6c6dc9f44650c1c62f5be5448864d817513b1..569a487328e2f1febe2ca5014b232dbd51d28079 100644 --- a/paddle/fluid/inference/api/details/reset_tensor_array.cc +++ b/paddle/fluid/inference/api/details/reset_tensor_array.cc @@ -46,5 +46,28 @@ void TensorArrayBatchCleaner::ResetTensorArray() { } } +void TensorArrayBatchCleaner::CollectNoTensorVars(framework::Scope *scope) { + if (no_tensor_flag_) { + for (auto &var_name : scope->LocalVarNames()) { + auto *var = scope->FindVar(var_name); + if (!var->IsInitialized()) continue; + if (!valid_types_.count(var->Type())) { + no_tensor_vars_.insert(var); + } + } + + for (auto *kid : scope->kids()) { + CollectTensorArrays(kid); + } + no_tensor_flag_ = false; // Only collect one time. + } +} + +void TensorArrayBatchCleaner::ResetNoTensorVars() { + for (auto *var : no_tensor_vars_) { + var->Clear(); + } +} + } // namespace details } // namespace paddle diff --git a/paddle/fluid/inference/api/details/reset_tensor_array.h b/paddle/fluid/inference/api/details/reset_tensor_array.h index a39449ff0e67786815dfb8d2d30d79dcdba757d7..6a5ea64de66fcac44117d0d8f7798e8875703ec6 100644 --- a/paddle/fluid/inference/api/details/reset_tensor_array.h +++ b/paddle/fluid/inference/api/details/reset_tensor_array.h @@ -14,9 +14,11 @@ #pragma once +#include #include #include "paddle/fluid/framework/lod_tensor_array.h" #include "paddle/fluid/framework/scope.h" +#include "paddle/fluid/framework/variable.h" namespace paddle { namespace details { @@ -24,13 +26,28 @@ namespace details { // Clean the TensorArray each batch to make the behavior the same with the // training phase. struct TensorArrayBatchCleaner { + TensorArrayBatchCleaner() { + valid_types_.insert(typeid(framework::Tensor)); + valid_types_.insert(typeid(framework::LoDTensor)); + } + // Collect the variables that are not Tensor or LoDTensor, and reset them to a + // bool(trick), because some of them are containers, and some operators just + // keep inserting new items without clearing the containers first; So the + // memory grow larger and larger in inference service deployed online. + void CollectNoTensorVars(framework::Scope *scope); + void ResetNoTensorVars(); + // Fix the tensor array not clear in the inference scenarios. void CollectTensorArrays(framework::Scope *scope); void ResetTensorArray(); private: bool flag_{true}; + bool no_tensor_flag_{true}; std::vector arrays_; + + std::unordered_set valid_types_; + std::unordered_set no_tensor_vars_; }; } // namespace details diff --git a/paddle/fluid/inference/tests/api/CMakeLists.txt b/paddle/fluid/inference/tests/api/CMakeLists.txt index 7dc88d9dd052c59aaa59b7802ee5a38ea9d89bc6..a07626a10315a6206f8c1ebc9a19df90663a88ee 100644 --- a/paddle/fluid/inference/tests/api/CMakeLists.txt +++ b/paddle/fluid/inference/tests/api/CMakeLists.txt @@ -46,11 +46,18 @@ set(RNN2_INSTALL_DIR "${INFERENCE_DEMO_INSTALL_DIR}/rnn2") download_model_and_data(${RNN2_INSTALL_DIR} "rnn2_model.tar.gz" "rnn2_data.txt.tar.gz") inference_analysis_api_test(test_analyzer_rnn2 ${RNN2_INSTALL_DIR} analyzer_rnn2_tester.cc) -# DAM +# normal DAM set(DAM_INSTALL_DIR "${INFERENCE_DEMO_INSTALL_DIR}/dam") download_model_and_data(${DAM_INSTALL_DIR} "DAM_model.tar.gz" "DAM_data.txt.tar.gz") inference_analysis_api_test(test_analyzer_dam ${DAM_INSTALL_DIR} analyzer_dam_tester.cc) +# small DAM +set(DAM_SMALL_INSTALL_DIR "${INFERENCE_DEMO_INSTALL_DIR}/small_dam") +download_model_and_data(${DAM_SMALL_INSTALL_DIR} "dam_small_model.tar.gz" "dam_small_data.txt.tar.gz") +inference_analysis_test(test_analyzer_small_dam SRCS analyzer_dam_tester.cc + EXTRA_DEPS ${INFERENCE_EXTRA_DEPS} + ARGS --infer_model=${DAM_SMALL_INSTALL_DIR}/model --infer_data=${DAM_SMALL_INSTALL_DIR}/data.txt --max_turn_num=1) + # chinese_ner set(CHINESE_NER_INSTALL_DIR "${INFERENCE_DEMO_INSTALL_DIR}/chinese_ner") download_model_and_data(${CHINESE_NER_INSTALL_DIR} "chinese_ner_model.tar.gz" "chinese_ner-data.txt.tar.gz") diff --git a/paddle/fluid/inference/tests/api/analyzer_dam_tester.cc b/paddle/fluid/inference/tests/api/analyzer_dam_tester.cc index b369cba5c8b3f8aadd1123d6b7345fad6e47bd0f..a3a6130db7cfe75ef558dc901883c29a20088b3f 100644 --- a/paddle/fluid/inference/tests/api/analyzer_dam_tester.cc +++ b/paddle/fluid/inference/tests/api/analyzer_dam_tester.cc @@ -14,38 +14,54 @@ #include "paddle/fluid/inference/tests/api/tester_helper.h" +DEFINE_int32(max_turn_num, 9, + "The max turn number: 1 for the small and 9 for the normal."); + namespace paddle { namespace inference { using contrib::AnalysisConfig; -#define MAX_TURN_NUM 9 -#define MAX_TURN_LEN 50 + +constexpr int32_t kMaxTurnLen = 50; + static std::vector result_data; struct DataRecord { - std::vector> - turns[MAX_TURN_NUM]; // turns data : MAX_TURN_NUM - std::vector> - turns_mask[MAX_TURN_NUM]; // turns mask data : MAX_TURN_NUM - std::vector> response; // response data : 1 + std::vector> *turns; + std::vector> *turns_mask; + std::vector> response; // response data : 1 std::vector> response_mask; // response mask data : 1 size_t batch_iter{0}; size_t batch_size{1}; size_t num_samples; // total number of samples - DataRecord() = default; + + DataRecord() { + turns = new std::vector>[FLAGS_max_turn_num]; // turns data : FLAGS_max_turn_num + turns_mask = new std::vector>[FLAGS_max_turn_num]; // turns mask data : FLAGS_max_turn_num + } + explicit DataRecord(const std::string &path, int batch_size = 1) - : batch_size(batch_size) { + : DataRecord() { + this->batch_size = batch_size; Load(path); } + + ~DataRecord() { + delete[] turns; + delete[] turns_mask; + } + DataRecord NextBatch() { DataRecord data; size_t batch_end = batch_iter + batch_size; // NOTE skip the final batch, if no enough data is provided. if (batch_end <= response.size()) { - for (int i = 0; i < MAX_TURN_NUM; ++i) { + for (int i = 0; i < FLAGS_max_turn_num; ++i) { data.turns[i].assign(turns[i].begin() + batch_iter, turns[i].begin() + batch_end); } - for (int i = 0; i < MAX_TURN_NUM; ++i) { + for (int i = 0; i < FLAGS_max_turn_num; ++i) { data.turns_mask[i].assign(turns_mask[i].begin() + batch_iter, turns_mask[i].begin() + batch_end); } @@ -60,6 +76,7 @@ struct DataRecord { batch_iter += batch_size; return data; } + void Load(const std::string &path) { std::ifstream file(path); std::string line; @@ -69,30 +86,30 @@ struct DataRecord { num_lines++; std::vector data; split(line, ',', &data); - CHECK_EQ(data.size(), (size_t)(2 * MAX_TURN_NUM + 3)); + CHECK_EQ(data.size(), (size_t)(2 * FLAGS_max_turn_num + 3)); // load turn data - std::vector turns_tmp[MAX_TURN_NUM]; - for (int i = 0; i < MAX_TURN_NUM; ++i) { + std::vector turns_tmp[FLAGS_max_turn_num]; + for (int i = 0; i < FLAGS_max_turn_num; ++i) { split_to_int64(data[i], ' ', &turns_tmp[i]); turns[i].push_back(std::move(turns_tmp[i])); } // load turn_mask data - std::vector turns_mask_tmp[MAX_TURN_NUM]; - for (int i = 0; i < MAX_TURN_NUM; ++i) { - split_to_float(data[MAX_TURN_NUM + i], ' ', &turns_mask_tmp[i]); + std::vector turns_mask_tmp[FLAGS_max_turn_num]; + for (int i = 0; i < FLAGS_max_turn_num; ++i) { + split_to_float(data[FLAGS_max_turn_num + i], ' ', &turns_mask_tmp[i]); turns_mask[i].push_back(std::move(turns_mask_tmp[i])); } // load response data std::vector response_tmp; - split_to_int64(data[2 * MAX_TURN_NUM], ' ', &response_tmp); + split_to_int64(data[2 * FLAGS_max_turn_num], ' ', &response_tmp); response.push_back(std::move(response_tmp)); // load response_mask data std::vector response_mask_tmp; - split_to_float(data[2 * MAX_TURN_NUM + 1], ' ', &response_mask_tmp); + split_to_float(data[2 * FLAGS_max_turn_num + 1], ' ', &response_mask_tmp); response_mask.push_back(std::move(response_mask_tmp)); // load result data float result_tmp; - result_tmp = std::stof(data[2 * MAX_TURN_NUM + 2]); + result_tmp = std::stof(data[2 * FLAGS_max_turn_num + 2]); result_data.push_back(result_tmp); } num_samples = num_lines; @@ -101,8 +118,8 @@ struct DataRecord { void PrepareInputs(std::vector *input_slots, DataRecord *data, int batch_size) { - PaddleTensor turns_tensor[MAX_TURN_NUM]; - PaddleTensor turns_mask_tensor[MAX_TURN_NUM]; + PaddleTensor turns_tensor[FLAGS_max_turn_num]; + PaddleTensor turns_mask_tensor[FLAGS_max_turn_num]; PaddleTensor response_tensor; PaddleTensor response_mask_tensor; std::string turn_pre = "turn_"; @@ -110,16 +127,16 @@ void PrepareInputs(std::vector *input_slots, DataRecord *data, auto one_batch = data->NextBatch(); int size = one_batch.response[0].size(); - CHECK_EQ(size, MAX_TURN_LEN); + CHECK_EQ(size, kMaxTurnLen); // turn tensor assignment - for (int i = 0; i < MAX_TURN_NUM; ++i) { + for (int i = 0; i < FLAGS_max_turn_num; ++i) { turns_tensor[i].name = turn_pre + std::to_string(i); turns_tensor[i].shape.assign({batch_size, size, 1}); turns_tensor[i].dtype = PaddleDType::INT64; TensorAssignData(&turns_tensor[i], one_batch.turns[i]); } // turn mask tensor assignment - for (int i = 0; i < MAX_TURN_NUM; ++i) { + for (int i = 0; i < FLAGS_max_turn_num; ++i) { turns_mask_tensor[i].name = turn_mask_pre + std::to_string(i); turns_mask_tensor[i].shape.assign({batch_size, size, 1}); turns_mask_tensor[i].dtype = PaddleDType::FLOAT32; @@ -137,10 +154,10 @@ void PrepareInputs(std::vector *input_slots, DataRecord *data, TensorAssignData(&response_mask_tensor, one_batch.response_mask); // Set inputs. - for (int i = 0; i < MAX_TURN_NUM; ++i) { + for (int i = 0; i < FLAGS_max_turn_num; ++i) { input_slots->push_back(std::move(turns_tensor[i])); } - for (int i = 0; i < MAX_TURN_NUM; ++i) { + for (int i = 0; i < FLAGS_max_turn_num; ++i) { input_slots->push_back(std::move(turns_mask_tensor[i])); } input_slots->push_back(std::move(response_tensor)); @@ -202,8 +219,6 @@ TEST(Analyzer_dam, fuse_statis) { auto fuse_statis = GetFuseStatis( static_cast(predictor.get()), &num_ops); ASSERT_TRUE(fuse_statis.count("fc_fuse")); - EXPECT_EQ(fuse_statis.at("fc_fuse"), 317); - EXPECT_EQ(num_ops, 2020); } // Compare result of NativeConfig and AnalysisConfig diff --git a/paddle/fluid/operators/cudnn_lstm_op.cc b/paddle/fluid/operators/cudnn_lstm_op.cc new file mode 100644 index 0000000000000000000000000000000000000000..e63d57be57a66e8e02f7ef88acd01246302bc53c --- /dev/null +++ b/paddle/fluid/operators/cudnn_lstm_op.cc @@ -0,0 +1,218 @@ +/* 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 +#include "paddle/fluid/framework/op_registry.h" + +namespace paddle { +namespace operators { + +class CudnnLSTMOp : public framework::OperatorWithKernel { + public: + using framework::OperatorWithKernel::OperatorWithKernel; + + void InferShape(framework::InferShapeContext* ctx) const override { + PADDLE_ENFORCE(ctx->HasInput("Input"), + "Input(Input) of LSTM should not be null."); + PADDLE_ENFORCE(ctx->HasInput("W"), + "Input(Weight) of LSTM should not be null."); + + PADDLE_ENFORCE(ctx->HasInput("InitH"), + "Input(init_h) of LSTM should not be null."); + PADDLE_ENFORCE(ctx->HasInput("InitC"), + "Input(init_c) of LSTM should not be null."); + PADDLE_ENFORCE(ctx->HasInput("Cache"), + "Input(Cache) of LSTM should not be null."); + PADDLE_ENFORCE(ctx->HasOutput("Out"), + "Output(Out) of LSTM should not be null."); + PADDLE_ENFORCE(ctx->HasOutput("last_h"), + "Output(last_h) of LSTM should not be null."); + PADDLE_ENFORCE(ctx->HasOutput("last_c"), + "Output(last_c) of LSTM should not be null."); + + auto in_dims = ctx->GetInputDim("Input"); + PADDLE_ENFORCE_EQ(in_dims.size(), 3, "Input(X)'s rank must be 3."); + + ctx->SetOutputDim("Out", ctx->GetInputDim("Input")); + ctx->SetOutputDim("last_h", ctx->GetInputDim("InitH")); + ctx->SetOutputDim("last_c", ctx->GetInputDim("InitC")); + } +}; + +class CudnnLSTMOpMaker : public framework::OpProtoAndCheckerMaker { + public: + void Make() override { + AddInput( + "Input", + "(Tensor) RNN input tensor, which support variable-time length input " + "sequence." + "The shape of the Tensor MUST be ( seq_len * batch_size * input_size)" + "seq_len is the total time step in this mini-batch (CAN be change in " + "different batch)" + "batch_size is the instance number of this batch" + "input_size is the hidden size of the input." + "input_hidden_size and the hidden_size in the next may not be same"); + AddInput("InitH", + "(Tensor) the initial hidden state of the LSTM" + "input. This is a tensor with shape (num_layers x batch_size x " + "hidden_size)" + "and When is_bidirec is True, the shape will be (num_layers*2 x " + "batch_size x hidden_size)"); + AddInput("InitC", + "(Tensor) the initial cell state of the LSTm " + "input. This is a tensor with shape (num_layers x batch_size x " + "hidden_size)" + "and When is_bidirec is True, the shape will be (num_layers*2 x " + "batch_size x hidden_size)"); + AddInput("W", + "(Tensor) the learnable hidden-hidden weights." + " The shape is (N), where N is total weight size of the LSTM. " + " cudnn concatenate all the weight to one Tensor"); + AddInput("Cache", + "The cache of dropout op, a RAW type variable including random " + "number generator states and some descriptors, which is used in " + "cudnn kernel.") + .AsDispensable(); + AddOutput("Out", + "(Tensor) the hidden state of LSTM operator. " + "The shape is ( seq_len x batch_size x hidden_size) if " + "is_bidirec is False" + "and When is_bidirec is True, the shape will be ( seq_len x " + "batch_size x hidden_size * 2) "); + AddOutput("last_h", + "(Tensor) the hidden state of the last step. " + "The shape is ( num_layers x batch_size x hidden_size) if " + "is_bidirec is False" + "and When is_bidirec is True, the shape will be (num_layers*2 x " + "batch_size x hidden_size)"); + AddOutput("last_c", + "(Tensor) the cell state of the last step" + "The shape is ( num_layers x batch_size x hidden_size) if " + "is_bidirec is False" + "and When is_bidirect is True, the shape will be (num_layers*2 x " + "batch_size x hidden_size*2)"); + AddAttr("max_len", + "max length of the LSTM op" + "the first dim of the Input can NOT be greater than max_len") + .SetDefault(20); + AddAttr( + "dropout_prob", + "dropout prob of the dropout op" + "the dropout ONLY work between lstm layers, not between time steps" + "There is no dropout work on the Out tensor") + .SetDefault(0.0); + AddAttr("is_bidirec", + "is_bidirec" + "if it is bidirection rnn" + "The will affect the shape of the Out, last_h, and last_c") + .SetDefault(false); + AddAttr("input_size", "input size ot the Input Tensor").SetDefault(10); + AddAttr("hidden_size", "hidden size of the LSTM").SetDefault(100); + AddAttr("num_layers", "the total layer number of the LSTM") + .SetDefault(1); + AddAttr("is_test", "True if in test phase.").SetDefault(false); + AddAttr("seed", "seed to used if fix_seed is True").SetDefault(-1); + AddComment(R"DOC( +CUDNN LSTM implementation + +A four-gate Long Short-Term Memory network with no peephole connections. +In the forward pass the output ht and cell output ct for a given iteration can be computed from the recurrent input ht-1, +the cell input ct-1 and the previous layer input xt given matrices W, R and biases bW, bR from the following equations: + +$$ i_t = sigmoid(W_{ix}x_{t} + W_{ih}h_{t-1} + bx_i + bh_i) $$ + +$$ f_t = sigmoid(W_{fx}x_{t} + W_{fh}h_{t-1} + bx_f + bh_f) $$ + +$$ o_t = sigmoid(W_{ox}x_{t} + W_{oh}h_{t-1} + bx_o + bh_o) $$ + +$$ \\tilde{c_t} = tanh(W_{cx}x_t + W_{ch}h_{t-1} + bx_c + bh_c) $$ + +$$ c_t = f_t \\odot c_{t-1} + i_t \\odot \\tilde{c_t} $$ + +$$ h_t = o_t \\odot tanh(c_t) $$ + +- W terms denote weight matrices (e.g. $W_{ix}$ is the matrix + of weights from the input gate to the input) +- The b terms denote bias vectors ($bx_i$ and $bh_i$ are the input gate bias vector). +- sigmoid is the logistic sigmoid function. +- $i, f, o$ and $c$ are the input gate, forget gate, output gate, + and cell activation vectors, respectively, all of which have the same size as + the cell output activation vector $h$. +- The $\odot$ is the element-wise product of the vectors. +- `tanh` is the activation functions. +- $\tilde{c_t}$ is also called candidate hidden state, + which is computed based on the current input and the previous hidden state. + +Where sigmoid is the sigmoid operator: sigmoid(x) = 1 / (1 + e^-x), * represents a point-wise multiplication, +X represensts a matrix multiplication + + +)DOC"); + } +}; + +class CudnnLSTMGradOp : public framework::OperatorWithKernel { + public: + using framework::OperatorWithKernel::OperatorWithKernel; + + void InferShape(framework::InferShapeContext* ctx) const override { + PADDLE_ENFORCE(ctx->HasInput("Input"), + "Input(Input) of LSTM should not be null."); + PADDLE_ENFORCE(ctx->HasInput("W"), "Input(W) of LSTM should not be null."); + PADDLE_ENFORCE(ctx->HasInput("last_h"), + "Input(last_h) of LSTM should not be null."); + PADDLE_ENFORCE(ctx->HasInput("last_c"), + "Input(last_c) of LSTM should not be null."); + + PADDLE_ENFORCE(ctx->HasInput("Cache"), + "Input(last_c) of LSTM should not be null."); + PADDLE_ENFORCE(ctx->HasInput("InitH"), + "Input(init_h) of LSTM should not be null."); + + PADDLE_ENFORCE(ctx->HasInput("InitC"), + "Input(init_c) of LSTM should not be null."); + + auto SetOutGradDim = [&ctx](const std::string& name) { + auto g_name = framework::GradVarName(name); + if (ctx->HasOutput(g_name)) { + ctx->SetOutputDim(g_name, ctx->GetInputDim(name)); + } + }; + + SetOutGradDim("Input"); + SetOutGradDim("W"); + SetOutGradDim("InitH"); + SetOutGradDim("InitC"); + } +}; + +template +class NotImpleKernel : public framework::OpKernel { + public: + void Compute(const framework::ExecutionContext& ctx) const override { + PADDLE_THROW( + "CPU is not support for this kernel now. Will be add in the future"); + } +}; + +} // namespace operators +} // namespace paddle + +namespace ops = paddle::operators; +REGISTER_OPERATOR(cudnn_lstm, ops::CudnnLSTMOp, ops::CudnnLSTMOpMaker, + paddle::framework::DefaultGradOpDescMaker); +REGISTER_OPERATOR(cudnn_lstm_grad, ops::CudnnLSTMGradOp); + +REGISTER_OP_CPU_KERNEL(cudnn_lstm, ops::NotImpleKernel); +REGISTER_OP_CPU_KERNEL(cudnn_lstm_grad, ops::NotImpleKernel); diff --git a/paddle/fluid/operators/cudnn_lstm_op.cu.cc b/paddle/fluid/operators/cudnn_lstm_op.cu.cc new file mode 100644 index 0000000000000000000000000000000000000000..e01070c7b8ed4374cf8a61cfde4de940b4ea38b2 --- /dev/null +++ b/paddle/fluid/operators/cudnn_lstm_op.cu.cc @@ -0,0 +1,485 @@ +/* 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/op_registry.h" +#include "paddle/fluid/operators/math/math_function.h" +#include "paddle/fluid/platform/cudnn_helper.h" + +namespace paddle { +namespace operators { + +using LoDTensor = framework::LoDTensor; +using Tensor = framework::Tensor; + +struct CudnnRNNCache { + CudnnRNNCache() { + x_desc_ = NULL; + y_desc_ = NULL; + dx_desc_ = NULL; + dy_desc_ = NULL; + } + ~CudnnRNNCache() { release(); } + + cudnnRNNDescriptor_t rnn_desc_; + cudnnTensorDescriptor_t *x_desc_; + cudnnTensorDescriptor_t *y_desc_; + cudnnTensorDescriptor_t *dx_desc_; + cudnnTensorDescriptor_t *dy_desc_; + + cudnnTensorDescriptor_t hx_desc_; + cudnnTensorDescriptor_t cx_desc_; + cudnnTensorDescriptor_t hy_desc_; + cudnnTensorDescriptor_t cy_desc_; + + cudnnTensorDescriptor_t dhx_desc_; + cudnnTensorDescriptor_t dcx_desc_; + cudnnTensorDescriptor_t dhy_desc_; + cudnnTensorDescriptor_t dcy_desc_; + + cudnnTensorDescriptor_t output_x_desc_; + cudnnTensorDescriptor_t output_y_desc_; + + cudnnDropoutDescriptor_t dropout_desc_; + + size_t weights_size_; + cudnnFilterDescriptor_t w_desc_; + cudnnFilterDescriptor_t dw_desc_; + + size_t workspace_size_; + size_t reserve_size_; + Tensor reserve_data_; + Tensor workspace_data_; + + Tensor dropout_state_; + + size_t max_length_; + + float dropout_prob_; + bool is_bidirec_; + + int batch_size_; + int input_size_; + int hidden_size_; + int num_layers_; + int seed_; + + void init(cudnnHandle_t handle, const framework::ExecutionContext &ctx, + size_t max_len, int batch_size, int input_size, int hidden_size, + int num_layers, float dropout_prob, bool is_bidirec, int seed, + int weight_numel) { + max_length_ = max_len; + batch_size_ = batch_size; + input_size_ = input_size; + hidden_size_ = hidden_size; + num_layers_ = num_layers; + dropout_prob_ = dropout_prob; + is_bidirec_ = is_bidirec; + seed_ = seed; + + x_desc_ = new cudnnTensorDescriptor_t[max_length_]; + y_desc_ = new cudnnTensorDescriptor_t[max_length_]; + dx_desc_ = new cudnnTensorDescriptor_t[max_length_]; + dy_desc_ = new cudnnTensorDescriptor_t[max_length_]; + int dim_a[3]; + int stride_a[3]; + + for (size_t i = 0; i < max_length_; ++i) { + CUDNN_ENFORCE( + platform::dynload::cudnnCreateTensorDescriptor(&x_desc_[i])); + CUDNN_ENFORCE( + platform::dynload::cudnnCreateTensorDescriptor(&y_desc_[i])); + CUDNN_ENFORCE( + platform::dynload::cudnnCreateTensorDescriptor(&dx_desc_[i])); + CUDNN_ENFORCE( + platform::dynload::cudnnCreateTensorDescriptor(&dy_desc_[i])); + dim_a[0] = batch_size_; + dim_a[1] = input_size_; + dim_a[2] = 1; + + stride_a[0] = dim_a[2] * dim_a[1]; + stride_a[1] = dim_a[2]; + stride_a[2] = 1; + CUDNN_ENFORCE(platform::dynload::cudnnSetTensorNdDescriptor( + x_desc_[i], CUDNN_DATA_FLOAT, 3, dim_a, stride_a)); + CUDNN_ENFORCE(platform::dynload::cudnnSetTensorNdDescriptor( + dx_desc_[i], CUDNN_DATA_FLOAT, 3, dim_a, stride_a)); + + dim_a[0] = batch_size_; + dim_a[1] = is_bidirec_ ? hidden_size_ * 2 : hidden_size_; + dim_a[2] = 1; + + stride_a[0] = dim_a[2] * dim_a[1]; + stride_a[1] = dim_a[2]; + stride_a[2] = 1; + + CUDNN_ENFORCE(platform::dynload::cudnnSetTensorNdDescriptor( + y_desc_[i], CUDNN_DATA_FLOAT, 3, dim_a, stride_a)); + CUDNN_ENFORCE(platform::dynload::cudnnSetTensorNdDescriptor( + dy_desc_[i], CUDNN_DATA_FLOAT, 3, dim_a, stride_a)); + } + + dim_a[0] = num_layers_ * (is_bidirec_ ? 2 : 1); + dim_a[1] = batch_size_; + dim_a[2] = hidden_size_; + + stride_a[0] = dim_a[2] * dim_a[1]; + stride_a[1] = dim_a[2]; + stride_a[2] = 1; + + CUDNN_ENFORCE(platform::dynload::cudnnCreateTensorDescriptor(&hx_desc_)); + CUDNN_ENFORCE(platform::dynload::cudnnCreateTensorDescriptor(&cx_desc_)); + CUDNN_ENFORCE(platform::dynload::cudnnCreateTensorDescriptor(&hy_desc_)); + CUDNN_ENFORCE(platform::dynload::cudnnCreateTensorDescriptor(&cy_desc_)); + CUDNN_ENFORCE(platform::dynload::cudnnCreateTensorDescriptor(&dhx_desc_)); + CUDNN_ENFORCE(platform::dynload::cudnnCreateTensorDescriptor(&dcx_desc_)); + CUDNN_ENFORCE(platform::dynload::cudnnCreateTensorDescriptor(&dhy_desc_)); + CUDNN_ENFORCE(platform::dynload::cudnnCreateTensorDescriptor(&dcy_desc_)); + + CUDNN_ENFORCE(platform::dynload::cudnnSetTensorNdDescriptor( + hx_desc_, CUDNN_DATA_FLOAT, 3, dim_a, stride_a)); + CUDNN_ENFORCE(platform::dynload::cudnnSetTensorNdDescriptor( + cx_desc_, CUDNN_DATA_FLOAT, 3, dim_a, stride_a)); + CUDNN_ENFORCE(platform::dynload::cudnnSetTensorNdDescriptor( + hy_desc_, CUDNN_DATA_FLOAT, 3, dim_a, stride_a)); + CUDNN_ENFORCE(platform::dynload::cudnnSetTensorNdDescriptor( + cy_desc_, CUDNN_DATA_FLOAT, 3, dim_a, stride_a)); + CUDNN_ENFORCE(platform::dynload::cudnnSetTensorNdDescriptor( + dhx_desc_, CUDNN_DATA_FLOAT, 3, dim_a, stride_a)); + CUDNN_ENFORCE(platform::dynload::cudnnSetTensorNdDescriptor( + dcx_desc_, CUDNN_DATA_FLOAT, 3, dim_a, stride_a)); + CUDNN_ENFORCE(platform::dynload::cudnnSetTensorNdDescriptor( + dhy_desc_, CUDNN_DATA_FLOAT, 3, dim_a, stride_a)); + CUDNN_ENFORCE(platform::dynload::cudnnSetTensorNdDescriptor( + dcy_desc_, CUDNN_DATA_FLOAT, 3, dim_a, stride_a)); + + CUDNN_ENFORCE( + platform::dynload::cudnnCreateDropoutDescriptor(&dropout_desc_)); + + size_t state_size; + CUDNN_ENFORCE( + platform::dynload::cudnnDropoutGetStatesSize(handle, &state_size); + dropout_state_.Resize({static_cast(state_size)})); + auto *dropout_state_data = + dropout_state_.mutable_data(ctx.GetPlace()); + CUDNN_ENFORCE(platform::dynload::cudnnSetDropoutDescriptor( + dropout_desc_, handle, dropout_prob_, dropout_state_data, state_size, + seed_)); + + CUDNN_ENFORCE(platform::dynload::cudnnCreateRNNDescriptor(&rnn_desc_)); + CUDNN_ENFORCE(platform::dynload::cudnnSetRNNDescriptor_v6( + handle, rnn_desc_, hidden_size_, num_layers_, dropout_desc_, + CUDNN_LINEAR_INPUT, + is_bidirec_ ? CUDNN_BIDIRECTIONAL : CUDNN_UNIDIRECTIONAL, CUDNN_LSTM, + CUDNN_RNN_ALGO_STANDARD, CUDNN_DATA_FLOAT)); + + CUDNN_ENFORCE(platform::dynload::cudnnCreateFilterDescriptor(&w_desc_)); + CUDNN_ENFORCE(platform::dynload::cudnnCreateFilterDescriptor(&dw_desc_)); + + CUDNN_ENFORCE(platform::dynload::cudnnGetRNNParamsSize( + handle, rnn_desc_, x_desc_[0], &weights_size_, CUDNN_DATA_FLOAT)); + + PADDLE_ENFORCE_EQ(weights_size_, sizeof(float) * weight_numel, + "cudnn lstm weight size should be SAME"); + int dim_w[3]; + dim_w[0] = weights_size_ / sizeof(float); + dim_w[1] = 1; + dim_w[2] = 1; + CUDNN_ENFORCE(platform::dynload::cudnnSetFilterNdDescriptor( + w_desc_, CUDNN_DATA_FLOAT, CUDNN_TENSOR_NCHW, 3, dim_w)); + CUDNN_ENFORCE(platform::dynload::cudnnSetFilterNdDescriptor( + dw_desc_, CUDNN_DATA_FLOAT, CUDNN_TENSOR_NCHW, 3, dim_w)); + + CUDNN_ENFORCE(platform::dynload::cudnnGetRNNWorkspaceSize( + handle, rnn_desc_, max_length_, x_desc_, &workspace_size_)); + CUDNN_ENFORCE(platform::dynload::cudnnGetRNNTrainingReserveSize( + handle, rnn_desc_, max_length_, x_desc_, &reserve_size_)); + + reserve_data_.Resize({static_cast(reserve_size_)}); + reserve_data_.mutable_data(ctx.GetPlace()); + + workspace_data_.Resize({static_cast(workspace_size_)}); + workspace_data_.mutable_data(ctx.GetPlace()); + } + + void release() { + for (size_t i = 0; i < max_length_; ++i) { + CUDNN_ENFORCE( + platform::dynload::cudnnDestroyTensorDescriptor(x_desc_[i])); + CUDNN_ENFORCE( + platform::dynload::cudnnDestroyTensorDescriptor(y_desc_[i])); + CUDNN_ENFORCE( + platform::dynload::cudnnDestroyTensorDescriptor(dx_desc_[i])); + CUDNN_ENFORCE( + platform::dynload::cudnnDestroyTensorDescriptor(dy_desc_[i])); + } + + delete[] x_desc_; + delete[] y_desc_; + delete[] dx_desc_; + delete[] dy_desc_; + + CUDNN_ENFORCE(platform::dynload::cudnnDestroyTensorDescriptor(hx_desc_)); + CUDNN_ENFORCE(platform::dynload::cudnnDestroyTensorDescriptor(cx_desc_)); + CUDNN_ENFORCE(platform::dynload::cudnnDestroyTensorDescriptor(hy_desc_)); + CUDNN_ENFORCE(platform::dynload::cudnnDestroyTensorDescriptor(cy_desc_)); + CUDNN_ENFORCE(platform::dynload::cudnnDestroyTensorDescriptor(dhx_desc_)); + CUDNN_ENFORCE(platform::dynload::cudnnDestroyTensorDescriptor(dcx_desc_)); + CUDNN_ENFORCE(platform::dynload::cudnnDestroyTensorDescriptor(dhy_desc_)); + CUDNN_ENFORCE(platform::dynload::cudnnDestroyTensorDescriptor(dcy_desc_)); + + CUDNN_ENFORCE( + platform::dynload::cudnnDestroyDropoutDescriptor(dropout_desc_)); + CUDNN_ENFORCE(platform::dynload::cudnnDestroyRNNDescriptor(rnn_desc_)); + + CUDNN_ENFORCE(platform::dynload::cudnnDestroyFilterDescriptor(w_desc_)); + CUDNN_ENFORCE(platform::dynload::cudnnDestroyFilterDescriptor(dw_desc_)); + } +}; + +template +class CudnnLSTMGPUKernel : public framework::OpKernel { + public: + void Compute(const framework::ExecutionContext &ctx) const override { + const Tensor *x = ctx.Input("Input"); + const Tensor *init_h = ctx.Input("InitH"); + const Tensor *init_c = ctx.Input("InitC"); + + auto w = ctx.Input("W"); + + Tensor *out = ctx.Output("Out"); + Tensor *last_h = ctx.Output("last_h"); + Tensor *last_c = ctx.Output("last_c"); + + const T *x_data = x->data(); + const T *init_h_data = init_h->data(); + const T *init_c_data = init_c->data(); + + const T *w_data = w->data(); + + T *out_data = out->mutable_data(ctx.GetPlace()); + T *last_h_data = last_h->mutable_data(ctx.GetPlace()); + T *last_c_data = last_c->mutable_data(ctx.GetPlace()); + + size_t max_len = ctx.Attr("max_len"); + float dropout_prob = ctx.Attr("dropout_prob"); + bool is_bidirec = ctx.Attr("is_bidirec"); + int input_size = ctx.Attr("input_size"); + int hidden_size = ctx.Attr("hidden_size"); + int num_layers = ctx.Attr("num_layers"); + bool is_test = ctx.Attr("is_test"); + + auto &dev_ctx = ctx.template device_context(); + auto handle = dev_ctx.cudnn_handle(); + auto *cache_var = ctx.InputVar("Cache"); + if (!cache_var) { + // The RAW type cache variable wouldn't be created and broadcasted on + // multi-devices before the first running. + // use parent scope to make cache persistable + auto *scope = const_cast(ctx.scope().parent()); + auto cache_var_name = ctx.Inputs("Cache")[0]; + cache_var = scope->Var(cache_var_name); + } + CudnnRNNCache *cudnn_rnn_cache = nullptr; + if (cache_var->IsInitialized()) { + cudnn_rnn_cache = const_cast(cache_var) + ->GetMutable(); + } else { + cudnn_rnn_cache = const_cast(cache_var) + ->GetMutable(); + std::random_device rnd; + int seed = ctx.Attr("seed"); + if (seed == -1) { + seed = rnd(); + } + + auto input_w_numel = w->numel(); + auto batch_size = x->dims()[1]; + cudnn_rnn_cache->init(handle, ctx, max_len, batch_size, input_size, + hidden_size, num_layers, dropout_prob, is_bidirec, + seed, input_w_numel); + } + + auto run_seq_len = x->dims()[0]; + + if (is_test) { + // for inference + CUDNN_ENFORCE(platform::dynload::cudnnRNNForwardInference( + handle, cudnn_rnn_cache->rnn_desc_, run_seq_len, + cudnn_rnn_cache->x_desc_, x_data, cudnn_rnn_cache->hx_desc_, + init_h_data, cudnn_rnn_cache->cx_desc_, init_c_data, + cudnn_rnn_cache->w_desc_, w_data, cudnn_rnn_cache->y_desc_, out_data, + cudnn_rnn_cache->hy_desc_, last_h_data, cudnn_rnn_cache->cy_desc_, + last_c_data, cudnn_rnn_cache->workspace_data_.data(), + cudnn_rnn_cache->workspace_size_)); + } else { + // for train + CUDNN_ENFORCE(platform::dynload::cudnnRNNForwardTraining( + handle, cudnn_rnn_cache->rnn_desc_, run_seq_len, + cudnn_rnn_cache->x_desc_, x_data, cudnn_rnn_cache->hx_desc_, + init_h_data, cudnn_rnn_cache->cx_desc_, init_c_data, + cudnn_rnn_cache->w_desc_, w_data, cudnn_rnn_cache->y_desc_, out_data, + cudnn_rnn_cache->hy_desc_, last_h_data, cudnn_rnn_cache->cy_desc_, + last_c_data, cudnn_rnn_cache->workspace_data_.data(), + cudnn_rnn_cache->workspace_size_, + cudnn_rnn_cache->reserve_data_.data(), + cudnn_rnn_cache->reserve_size_)); + } + } +}; + +template +class CudnnLSTMGPUGradKernel : public framework::OpKernel { + public: + void Compute(const framework::ExecutionContext &ctx) const override { + auto *input = ctx.Input("Input"); + auto *weight = ctx.Input("W"); + auto *init_h = ctx.Input("InitH"); + auto *init_c = ctx.Input("InitC"); + // auto * last_h = ctx.Input("last_h"); + // auto * last_c = ctx.Input("last_c"); + auto *out = ctx.Input("Out"); + auto *out_grad = ctx.Input(framework::GradVarName("Out")); + auto *last_h_grad = ctx.Input(framework::GradVarName("last_h")); + auto *last_c_grad = ctx.Input(framework::GradVarName("last_c")); + + // auto* init_h = ctx.Input("init_h"); + // auto* init_c = ctx.Input("init_c"); + + auto *in_grad = ctx.Output(framework::GradVarName("Input")); + auto *weight_grad = ctx.Output(framework::GradVarName("W")); + auto *init_h_grad = ctx.Output(framework::GradVarName("InitH")); + auto *init_c_grad = ctx.Output(framework::GradVarName("InitC")); + + auto &dev_ctx = ctx.template device_context(); + auto handle = dev_ctx.cudnn_handle(); + auto *cache_var = ctx.InputVar("Cache"); + PADDLE_ENFORCE(cache_var->IsInitialized()); + CudnnRNNCache *cudnn_rnn_cache = + const_cast(cache_var) + ->GetMutable(); + + auto input_dims = input->dims(); + auto weight_dims = weight->dims(); + auto init_h_dims = init_h->dims(); + auto init_c_dims = init_c->dims(); + in_grad->mutable_data(ctx.GetPlace()); + weight_grad->mutable_data(ctx.GetPlace()); + math::SetConstant zero; + zero(dev_ctx, in_grad, static_cast(0.0)); + zero(dev_ctx, weight_grad, static_cast(0.0)); + + T *init_h_grad_data = NULL; + if (init_h_grad == nullptr) { + Tensor init_h_grad_temp; + init_h_grad_temp.mutable_data(init_h_dims, ctx.GetPlace()); + zero(dev_ctx, &init_h_grad_temp, static_cast(0.0)); + + init_h_grad_data = init_h_grad_temp.data(); + } else { + init_h_grad->mutable_data(init_h_dims, ctx.GetPlace()); + zero(dev_ctx, init_h_grad, static_cast(0.0)); + init_h_grad_data = init_h_grad->data(); + } + + T *init_c_grad_data = NULL; + if (init_c_grad == nullptr) { + Tensor init_c_grad_temp; + init_c_grad_temp.mutable_data(init_c_dims, ctx.GetPlace()); + zero(dev_ctx, &init_c_grad_temp, static_cast(0.0)); + + init_c_grad_data = init_c_grad_temp.data(); + } else { + init_c_grad->mutable_data(init_c_dims, ctx.GetPlace()); + zero(dev_ctx, init_c_grad, static_cast(0.0)); + init_c_grad_data = init_c_grad->data(); + } + + const T *last_h_grad_data = NULL; + if (last_h_grad == nullptr) { + Tensor last_h_grad_temp; + last_h_grad_temp.mutable_data(init_h_dims, ctx.GetPlace()); + zero(dev_ctx, &last_h_grad_temp, static_cast(0.0)); + + last_h_grad_data = (const T *)last_h_grad_temp.data(); + } else { + last_h_grad_data = last_h_grad->data(); + } + + const T *last_c_grad_data = NULL; + if (last_c_grad == nullptr) { + Tensor last_c_grad_temp; + last_c_grad_temp.mutable_data(init_c_dims, ctx.GetPlace()); + zero(dev_ctx, &last_c_grad_temp, static_cast(0.0)); + + last_c_grad_data = (const T *)last_c_grad_temp.data(); + } else { + last_c_grad_data = last_c_grad->data(); + } + + const T *out_grad_data = NULL; + if (out_grad == nullptr) { + Tensor out_grad_temp; + out_grad_temp.mutable_data(out->dims(), ctx.GetPlace()); + zero(dev_ctx, &out_grad_temp, static_cast(0.0)); + + out_grad_data = (const T *)out_grad_temp.data(); + } else { + out_grad_data = out_grad->data(); + } + + // zero( dev_ctx, last_h_grad, static_cast(0.0)); + // zero( dev_ctx, last_c_grad, static_cast(0.0)); + + auto out_data = out->data(); + // auto out_grad_data = out_grad->data(); + auto weight_data = weight->data(); + auto init_h_data = init_h->data(); + auto init_c_data = init_c->data(); + auto in_grad_data = in_grad->data(); + + auto work_data = cudnn_rnn_cache->workspace_data_.data(); + auto reserve_data = cudnn_rnn_cache->reserve_data_.data(); + + auto run_seq_len = input_dims[0]; + PADDLE_ENFORCE_LE((size_t)run_seq_len, cudnn_rnn_cache->max_length_, + "cudnn running seq_len CAN not greater max_lengh"); + CUDNN_ENFORCE(platform::dynload::cudnnRNNBackwardData( + handle, cudnn_rnn_cache->rnn_desc_, run_seq_len, + cudnn_rnn_cache->y_desc_, out_data, cudnn_rnn_cache->dy_desc_, + out_grad_data, cudnn_rnn_cache->dhy_desc_, last_h_grad_data, + cudnn_rnn_cache->dcy_desc_, last_c_grad_data, cudnn_rnn_cache->w_desc_, + weight_data, cudnn_rnn_cache->hx_desc_, init_h_data, + cudnn_rnn_cache->cx_desc_, init_c_data, cudnn_rnn_cache->dx_desc_, + in_grad_data, cudnn_rnn_cache->dhx_desc_, init_h_grad_data, + cudnn_rnn_cache->dcx_desc_, init_c_grad_data, work_data, + cudnn_rnn_cache->workspace_size_, reserve_data, + cudnn_rnn_cache->reserve_size_)); + + CUDNN_ENFORCE(platform::dynload::cudnnRNNBackwardWeights( + handle, cudnn_rnn_cache->rnn_desc_, run_seq_len, + cudnn_rnn_cache->x_desc_, input->data(), cudnn_rnn_cache->hx_desc_, + init_h->data(), cudnn_rnn_cache->y_desc_, out->data(), + cudnn_rnn_cache->workspace_data_.data(), + cudnn_rnn_cache->workspace_size_, cudnn_rnn_cache->dw_desc_, + weight_grad->data(), cudnn_rnn_cache->reserve_data_.data(), + cudnn_rnn_cache->reserve_size_)); + } +}; + +} // namespace operators +} // namespace paddle + +namespace ops = paddle::operators; +REGISTER_OP_CUDA_KERNEL(cudnn_lstm, ops::CudnnLSTMGPUKernel); +REGISTER_OP_CUDA_KERNEL(cudnn_lstm_grad, ops::CudnnLSTMGPUGradKernel); diff --git a/paddle/fluid/operators/detection/box_coder_op.h b/paddle/fluid/operators/detection/box_coder_op.h index 5ed8520acddfa8fe2105a7c1615bcb3243cb130f..b2a2bcdce932032a761a1fc064fe622f7629f9bf 100644 --- a/paddle/fluid/operators/detection/box_coder_op.h +++ b/paddle/fluid/operators/detection/box_coder_op.h @@ -43,6 +43,9 @@ class BoxCoderKernel : public framework::OpKernel { const T* prior_box_var_data = nullptr; if (prior_box_var) prior_box_var_data = prior_box_var->data(); +#ifdef PADDLE_WITH_MKLML +#pragma omp parallel for collapse(2) +#endif for (int64_t i = 0; i < row; ++i) { for (int64_t j = 0; j < col; ++j) { T prior_box_width = prior_box_data[j * len + 2] - @@ -96,6 +99,9 @@ class BoxCoderKernel : public framework::OpKernel { const T* prior_box_var_data = nullptr; if (prior_box_var) prior_box_var_data = prior_box_var->data(); +#ifdef PADDLE_WITH_MKLML +#pragma omp parallel for collapse(2) +#endif for (int64_t i = 0; i < row; ++i) { for (int64_t j = 0; j < col; ++j) { size_t offset = i * col * len + j * len; diff --git a/paddle/fluid/operators/distributed/request_handler_impl.cc b/paddle/fluid/operators/distributed/request_handler_impl.cc index 0258f8f2384669c8a7466fd3c60f9b55a0fde9fd..9722f8c96e91d2dfbe929dcc11645a40c44afb4e 100644 --- a/paddle/fluid/operators/distributed/request_handler_impl.cc +++ b/paddle/fluid/operators/distributed/request_handler_impl.cc @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include "paddle/fluid/operators/distributed/request_handler_impl.h" #include #include #include @@ -20,7 +21,7 @@ #include "paddle/fluid/framework/lod_tensor.h" #include "paddle/fluid/framework/scope.h" #include "paddle/fluid/framework/selected_rows.h" -#include "paddle/fluid/operators/distributed/request_handler_impl.h" +#include "paddle/fluid/framework/variable_helper.h" #include "paddle/fluid/operators/distributed/rpc_server.h" #include "paddle/fluid/string/printf.h" diff --git a/paddle/fluid/operators/lookup_table_op.cu b/paddle/fluid/operators/lookup_table_op.cu index 36156a1f6174631dd084c8dc63dc432f5275008e..6a0d6bad512fe7cc15e60ed25028bc3cbbbca2ab 100644 --- a/paddle/fluid/operators/lookup_table_op.cu +++ b/paddle/fluid/operators/lookup_table_op.cu @@ -31,8 +31,8 @@ __global__ void LookupTable(T *output, const T *table, const int64_t *ids, while (idy < K) { int64_t id = ids[idy]; - PADDLE_ASSERT(id >= 0); - PADDLE_ASSERT(id < N); + PADDLE_ASSERT_MSG_CODE(id >= 0, "received id:", id); + PADDLE_ASSERT_MSG_CODE(id < N, "received id:", id); T *out = output + idy * D; const T *tab = table + id * D; for (int i = idx; i < D; i += BlockDimX) { @@ -57,9 +57,9 @@ __global__ void LookupTableGrad(T *table, const T *output, const int64_t *ids, int idy = blockIdx.x + threadIdx.y * GridDimX; while (idy < K) { - int id = ids[idy]; - PADDLE_ASSERT(id >= 0); - PADDLE_ASSERT(id < N); + int64_t id = ids[idy]; + PADDLE_ASSERT_MSG_CODE(id >= 0, "received id:", id); + PADDLE_ASSERT_MSG_CODE(id < N, "received id:", id); const T *out = output + idy * D; T *tab = table + id * D; for (int i = idx; i < D; i += BlockDimX) { diff --git a/paddle/fluid/operators/metrics/auc_op.h b/paddle/fluid/operators/metrics/auc_op.h index fb370842d1942c3b3eebecb1fe5e8ffb845cb34b..4ab5cfe53c67eeaa995d7e955eec63a065c5eec5 100644 --- a/paddle/fluid/operators/metrics/auc_op.h +++ b/paddle/fluid/operators/metrics/auc_op.h @@ -75,8 +75,13 @@ class AucKernel : public framework::OpKernel { const auto *label_data = label->data(); for (size_t i = 0; i < batch_size; i++) { - uint32_t binIdx = static_cast( - inference_data[i * inference_width + 1] * num_thresholds); + auto predict_data = inference_data[i * inference_width + 1]; + PADDLE_ENFORCE_LE(predict_data, 1, + "The predict data must less or equal 1."); + PADDLE_ENFORCE_GE(predict_data, 0, + "The predict data must gather or equal 0."); + + uint32_t binIdx = static_cast(predict_data * num_thresholds); if (label_data[i]) { (*stat_pos)[binIdx] += 1.0; } else { diff --git a/paddle/fluid/operators/pad2d_op.cc b/paddle/fluid/operators/pad2d_op.cc index a706d05fd7c35ef993f5199f0f893622cb863c5d..a9da21f47902e20cc7460461caca79c3f3292c5a 100644 --- a/paddle/fluid/operators/pad2d_op.cc +++ b/paddle/fluid/operators/pad2d_op.cc @@ -319,20 +319,46 @@ void Pad2DGradEdgeNHWC(T* d_in_data, const int num, const int channels, } } +static inline void GetPaddings(int* paddings, + const framework::ExecutionContext& context) { + auto* paddings_t = context.Input("Paddings"); + if (paddings_t) { + auto paddings_data = paddings_t->data(); + paddings[0] = paddings_data[0]; + paddings[1] = paddings_data[1]; + paddings[2] = paddings_data[2]; + paddings[3] = paddings_data[3]; + } else { + auto pads = context.Attr>("paddings"); + std::copy(pads.begin(), pads.end(), paddings); + } +} + template class Pad2dCPUKernel : public framework::OpKernel { public: void Compute(const framework::ExecutionContext& context) const override { - auto pads = context.Attr>("paddings"); + int pads[4]; + GetPaddings(pads, context); auto mode = context.Attr("mode"); auto data_format = context.Attr("data_format"); T value = context.Attr("pad_value"); + auto* x = context.Input("X"); - auto* out = context.Output("Out"); auto in_dims = x->dims(); - auto out_dims = out->dims(); const T* in_data = x->data(); + + auto* out = context.Output("Out"); + if (data_format == "NCHW") { + out->Resize({in_dims[0], in_dims[1], in_dims[2] + pads[0] + pads[1], + in_dims[3] + pads[2] + pads[3]}); + } else { + out->Resize({in_dims[0], in_dims[1] + pads[0] + pads[1], + in_dims[2] + pads[2] + pads[3], in_dims[3]}); + } + auto out_dims = out->dims(); T* out_data = out->mutable_data(context.GetPlace()); + const int pad_top = pads[0]; const int pad_left = pads[2]; const int num = in_dims[0]; @@ -376,7 +402,8 @@ template class Pad2dGradCPUKernel : public framework::OpKernel { public: void Compute(const framework::ExecutionContext& context) const override { - auto pads = context.Attr>("paddings"); + int pads[4]; + GetPaddings(pads, context); auto mode = context.Attr("mode"); auto data_format = context.Attr("data_format"); auto* d_out = context.Input(framework::GradVarName("Out")); @@ -442,21 +469,35 @@ class Pad2dOp : public framework::OperatorWithKernel { "Output(Out) of Pad2dOp should not be null."); auto x_dim = ctx->GetInputDim("X"); - auto paddings = ctx->Attrs().Get>("paddings"); PADDLE_ENFORCE_EQ(x_dim.size(), 4, - "Size of paddings should be equal to 4."); - std::vector out_dims(x_dim.size()); + "The size of input(X)'s dimension should be equal to 4."); + std::vector out_dims(x_dim.size()); auto data_format = ctx->Attrs().Get("data_format"); out_dims[0] = x_dim[0]; - if (data_format == "NCHW") { + if (ctx->HasInput("Paddings")) { + auto paddings_dim = ctx->GetInputDim("Paddings"); + PADDLE_ENFORCE_EQ( + paddings_dim.size(), 1, + "Size of Input(Paddings)'s dimension should be equal to 1."); + PADDLE_ENFORCE_EQ(paddings_dim[0], 4, + "Shape of Input(Paddings) should be equal to [4]."); out_dims[1] = x_dim[1]; - out_dims[2] = x_dim[2] + paddings[0] + paddings[1]; // height - out_dims[3] = x_dim[3] + paddings[2] + paddings[3]; // width - } else { // NHWC + out_dims[2] = x_dim[2]; out_dims[3] = x_dim[3]; - out_dims[1] = x_dim[1] + paddings[0] + paddings[1]; - out_dims[2] = x_dim[2] + paddings[2] + paddings[3]; + } else { + auto paddings = ctx->Attrs().Get>("paddings"); + PADDLE_ENFORCE_EQ(paddings.size(), 4, + "Size of paddings should be equal to 4."); + if (data_format == "NCHW") { + out_dims[1] = x_dim[1]; + out_dims[2] = x_dim[2] + paddings[0] + paddings[1]; // height + out_dims[3] = x_dim[3] + paddings[2] + paddings[3]; // width + } else { // NHWC + out_dims[3] = x_dim[3]; + out_dims[1] = x_dim[1] + paddings[0] + paddings[1]; + out_dims[2] = x_dim[2] + paddings[2] + paddings[3]; + } } ctx->SetOutputDim("Out", framework::make_ddim(out_dims)); @@ -466,6 +507,13 @@ class Pad2dOp : public framework::OperatorWithKernel { ctx->ShareLoD("X", /*->*/ "Out"); } } + + protected: + framework::OpKernelType GetExpectedKernelType( + const framework::ExecutionContext& ctx) const override { + return framework::OpKernelType( + framework::ToDataType(ctx.Input("X")->type()), ctx.GetPlace()); + } }; class Pad2dOpMaker : public framework::OpProtoAndCheckerMaker { @@ -477,6 +525,12 @@ class Pad2dOpMaker : public framework::OpProtoAndCheckerMaker { AddOutput("Out", "The output of pad2d op. " "A tensor with the same shape as X."); + AddInput("Paddings", + "A 1-D tensor to describe the padding rules." + "paddings=[0, 1, 2, 3] means " + "padding 0 row to top, 1 row to bottom, 2 columns to left " + "and 3 columns to right. Size of paddings must be 4.") + .AsDispensable(); AddAttr>( "paddings", "(vector) " @@ -554,6 +608,13 @@ class Pad2dOpGrad : public framework::OperatorWithKernel { ctx->SetOutputDim(x_grad_name, x_dims); } } + + protected: + framework::OpKernelType GetExpectedKernelType( + const framework::ExecutionContext& ctx) const override { + return framework::OpKernelType( + framework::ToDataType(ctx.Input("X")->type()), ctx.GetPlace()); + } }; class Pad2dOpGradMaker : public framework::SingleGradOpDescMaker { @@ -564,6 +625,7 @@ class Pad2dOpGradMaker : public framework::SingleGradOpDescMaker { std::unique_ptr Apply() const override { auto* bind = new framework::OpDesc(); bind->SetInput("X", Input("X")); + bind->SetInput("Paddings", Input("Paddings")); bind->SetInput(framework::GradVarName("Out"), OutputGrad("Out")); bind->SetOutput(framework::GradVarName("X"), InputGrad("X")); bind->SetAttrMap(Attrs()); diff --git a/paddle/fluid/operators/pad2d_op.cu b/paddle/fluid/operators/pad2d_op.cu index 9ba0ddbd84a43cfd5f028ce072b5c7606fae343d..72eca08b06b144335424a669241b5754beda758d 100644 --- a/paddle/fluid/operators/pad2d_op.cu +++ b/paddle/fluid/operators/pad2d_op.cu @@ -287,20 +287,50 @@ __global__ void Pad2DGradEdgeNHWC(const int out_size, T* d_in_data, } } +static inline void GetPaddings(int* paddings, + const framework::ExecutionContext& context) { + auto* paddings_t = context.Input("Paddings"); + if (paddings_t) { + Tensor pads; + framework::TensorCopySync(*paddings_t, platform::CPUPlace(), &pads); + auto pads_data = pads.data(); + paddings[0] = pads_data[0]; + paddings[1] = pads_data[1]; + paddings[2] = pads_data[2]; + paddings[3] = pads_data[3]; + } else { + auto pads = context.Attr>("paddings"); + std::copy(pads.begin(), pads.end(), paddings); + } +} + template class Pad2dCUDAKernel : public framework::OpKernel { public: void Compute(const framework::ExecutionContext& context) const override { - auto pads = context.Attr>("paddings"); + int pads[4]; + GetPaddings(pads, context); auto mode = context.Attr("mode"); auto data_format = context.Attr("data_format"); T value = context.Attr("pad_value"); + auto* x = context.Input("X"); - auto* out = context.Output("Out"); auto in_dims = x->dims(); - auto out_dims = out->dims(); const T* in_data = x->data(); - T* out_data = out->mutable_data(context.GetPlace()); + auto* out = context.Output("Out"); + auto out_dims = out->dims(); + if (data_format == "NCHW") { + out_dims[0] = in_dims[0]; + out_dims[1] = in_dims[1]; + out_dims[2] = in_dims[2] + pads[0] + pads[1]; + out_dims[3] = in_dims[3] + pads[2] + pads[3]; + } else { + out_dims[0] = in_dims[0]; + out_dims[1] = in_dims[1] + pads[0] + pads[1]; + out_dims[2] = in_dims[2] + pads[2] + pads[3]; + out_dims[3] = in_dims[3]; + } + T* out_data = out->mutable_data(out_dims, context.GetPlace()); const int pad_top = pads[0]; const int pad_left = pads[2]; const int num = in_dims[0]; @@ -356,7 +386,8 @@ template class Pad2dGradCUDAKernel : public framework::OpKernel { public: void Compute(const framework::ExecutionContext& context) const override { - auto pads = context.Attr>("paddings"); + int pads[4]; + GetPaddings(pads, context); auto mode = context.Attr("mode"); auto data_format = context.Attr("data_format"); auto* d_out = context.Input(framework::GradVarName("Out")); diff --git a/paddle/fluid/platform/assert.h b/paddle/fluid/platform/assert.h index 2ce9b31bb81de867ff4ed6ee14afddecd95317b9..2e8fa7c1b8f7f7b8f3154aae691bb100375981dd 100644 --- a/paddle/fluid/platform/assert.h +++ b/paddle/fluid/platform/assert.h @@ -36,6 +36,15 @@ limitations under the License. */ asm("trap;"); \ } \ } while (0) + +#define PADDLE_ASSERT_MSG_CODE(e, m, c) \ + do { \ + if (!(e)) { \ + printf("%s:%d Assertion `%s` failed (%s %d).\n", __FILE__, __LINE__, \ + TOSTRING(e), m, c); \ + asm("trap;"); \ + } \ + } while (0) #else #include // For cuda, the assertions can affect performance and it is therefore @@ -43,4 +52,5 @@ limitations under the License. */ // https://docs.nvidia.com/cuda/cuda-c-programming-guide/index.html#assertion #define PADDLE_ASSERT(e) assert((e)) #define PADDLE_ASSERT_MSG(e, m) assert((e) && (m)) +#define PADDLE_ASSERT_MSG_CODE(e, m, c) assert((e) && (m) && (c || 1)) #endif diff --git a/paddle/fluid/platform/dynload/cudnn.h b/paddle/fluid/platform/dynload/cudnn.h index db62377898339def415a13d185f85f34de326d7f..213cd8a9ce094512cea6f6405492ec8feff11516 100644 --- a/paddle/fluid/platform/dynload/cudnn.h +++ b/paddle/fluid/platform/dynload/cudnn.h @@ -111,7 +111,23 @@ extern void EnforceCUDNNLoaded(const char* fn_name); __macro(cudnnFindConvolutionForwardAlgorithmEx); \ __macro(cudnnFindConvolutionBackwardFilterAlgorithmEx); \ __macro(cudnnFindConvolutionBackwardDataAlgorithmEx); \ - __macro(cudnnGetErrorString); + __macro(cudnnGetErrorString); \ + __macro(cudnnCreateDropoutDescriptor); \ + __macro(cudnnDropoutGetStatesSize); \ + __macro(cudnnSetDropoutDescriptor); \ + __macro(cudnnCreateRNNDescriptor); \ + __macro(cudnnSetRNNDescriptor); \ + __macro(cudnnGetRNNParamsSize); \ + __macro(cudnnGetRNNWorkspaceSize); \ + __macro(cudnnGetRNNTrainingReserveSize); \ + __macro(cudnnRNNForwardTraining); \ + __macro(cudnnRNNBackwardData); \ + __macro(cudnnRNNBackwardWeights); \ + __macro(cudnnRNNForwardInference); \ + __macro(cudnnDestroyDropoutDescriptor); \ + __macro(cudnnDestroyRNNDescriptor); \ + __macro(cudnnSetRNNDescriptor_v6); + CUDNN_DNN_ROUTINE_EACH(DECLARE_DYNAMIC_LOAD_CUDNN_WRAP) #define CUDNN_DNN_ROUTINE_EACH_R2(__macro) \ diff --git a/paddle/fluid/pybind/CMakeLists.txt b/paddle/fluid/pybind/CMakeLists.txt index 25d241d9768c16e1da304a78f259d5a626f702fc..d602613fc82223e14f48830a87533880696eb550 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 prune feed_fetch_method pass_builder parallel_executor profiler) -set(PYBIND_SRCS pybind.cc exception.cc protobuf.cc const_value.cc recordio.cc) +set(PYBIND_DEPS pybind python proto_desc memory executor async_executor prune feed_fetch_method pass_builder parallel_executor profiler) +set(PYBIND_SRCS pybind.cc exception.cc protobuf.cc const_value.cc recordio.cc async_executor_py.cc) if(WITH_PYTHON) if(WITH_AMD_GPU) hip_library(paddle_pybind SHARED diff --git a/paddle/fluid/pybind/async_executor_py.cc b/paddle/fluid/pybind/async_executor_py.cc new file mode 100644 index 0000000000000000000000000000000000000000..470e8b050808295d49728bbdb757b6a612df9a01 --- /dev/null +++ b/paddle/fluid/pybind/async_executor_py.cc @@ -0,0 +1,53 @@ +/* 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 + +// To avoid conflicting definition in gcc-4.8.2 headers and pyconfig.h (2.7.3) +#ifdef _POSIX_C_SOURCE +#undef _POSIX_C_SOURCE +#endif + +#ifdef _XOPEN_SOURCE +#undef _XOPEN_SOURCE +#endif +#include +#include + +#include "google/protobuf/io/zero_copy_stream_impl.h" +#include "google/protobuf/text_format.h" +#include "paddle/fluid/framework/async_executor.h" +#include "paddle/fluid/framework/data_feed.h" +#include "paddle/fluid/framework/data_feed.pb.h" +#include "paddle/fluid/framework/scope.h" +#include "paddle/fluid/inference/io.h" +#include "paddle/fluid/platform/place.h" +#include "paddle/fluid/platform/variant.h" +#include "paddle/fluid/pybind/async_executor_py.h" + +namespace py = pybind11; +namespace pd = paddle::framework; + +namespace paddle { +namespace pybind { +using set_name_func = void (pd::DataFeedDesc::*)(const std::string&); +void BindAsyncExecutor(py::module* m) { + py::class_(*m, "AsyncExecutor") + .def(py::init([](framework::Scope* scope, const platform::Place& place) { + return std::unique_ptr( + new framework::AsyncExecutor(scope, place)); + })) + .def("run_from_files", &framework::AsyncExecutor::RunFromFile); +} // end BindAsyncExecutor +} // end namespace pybind +} // end namespace paddle diff --git a/paddle/fluid/pybind/async_executor_py.h b/paddle/fluid/pybind/async_executor_py.h new file mode 100644 index 0000000000000000000000000000000000000000..a99d6e04218c9310ede00de7d9bdfc015889bd22 --- /dev/null +++ b/paddle/fluid/pybind/async_executor_py.h @@ -0,0 +1,28 @@ +// 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 "pybind11/pybind11.h" +#include "pybind11/stl.h" + +namespace py = pybind11; + +namespace paddle { +namespace pybind { + +void BindAsyncExecutor(py::module* m); + +} // namespace pybind +} // namespace paddle diff --git a/paddle/fluid/pybind/pybind.cc b/paddle/fluid/pybind/pybind.cc index 1835c064055635a4284fc64f4ca4dd8728f933ca..fc7991d2974c9262e6225de1537025944c1068c1 100644 --- a/paddle/fluid/pybind/pybind.cc +++ b/paddle/fluid/pybind/pybind.cc @@ -42,6 +42,7 @@ limitations under the License. */ #include "paddle/fluid/platform/init.h" #include "paddle/fluid/platform/place.h" #include "paddle/fluid/platform/profiler.h" +#include "paddle/fluid/pybind/async_executor_py.h" #include "paddle/fluid/pybind/const_value.h" #include "paddle/fluid/pybind/exception.h" #include "paddle/fluid/pybind/protobuf.h" @@ -932,6 +933,7 @@ All parameter, weight, gradient are variables in Paddle. }); BindRecordIOWriter(&m); + BindAsyncExecutor(&m); } } // namespace pybind } // namespace paddle diff --git a/python/paddle/fluid/__init__.py b/python/paddle/fluid/__init__.py index f7fefb3e5b767e25373665058d4fd6a298fb3d60..a1ffbf42622bcda44ec038edf20811fb0032891f 100644 --- a/python/paddle/fluid/__init__.py +++ b/python/paddle/fluid/__init__.py @@ -20,6 +20,13 @@ from .framework import * # import all class inside executor into fluid module from . import executor from .executor import * + +from . import data_feed_desc +from .data_feed_desc import * + +from . import async_executor +from .async_executor import * + from . import trainer from . import inferencer @@ -54,7 +61,8 @@ Tensor = LoDTensor __all__ = framework.__all__ + executor.__all__ + \ trainer.__all__ + inferencer.__all__ + transpiler.__all__ + \ - parallel_executor.__all__ + lod_tensor.__all__ + [ + parallel_executor.__all__ + lod_tensor.__all__ + \ + data_feed_desc.__all__ + async_executor.__all__ + [ 'io', 'initializer', 'layers', diff --git a/python/paddle/fluid/async_executor.py b/python/paddle/fluid/async_executor.py new file mode 100644 index 0000000000000000000000000000000000000000..2664a7301db3bf471126ff26504e7042f02b7d84 --- /dev/null +++ b/python/paddle/fluid/async_executor.py @@ -0,0 +1,151 @@ +# 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 numpy as np +import contextlib +import six +from .framework import Program, default_main_program, Variable +from . import core +from .executor import global_scope, Executor +from paddle.fluid.proto import data_feed_pb2 +from google.protobuf import text_format +from . import io +from .data_feed_desc import DataFeedDesc + +__all__ = ['AsyncExecutor'] + + +class AsyncExecutor(object): + """ + An asynchronous Executor in Python. Through exploiting the power of + multi-core processor and data queueing, AsyncExecutor makes data reading + and cosuming decoupled, each run in multiple threads in parallel. + + Instead of reading data in python side, AsyncExecutor accepts a training + file list, which will be retrieved in C++, then training inputs will be + read, parsed and fed to training network within C++ code. + + AsyncExecutor is in active development and the API might change in the near + future. + + Example: + >>> data_feed = fluid.DataFeedDesc('data.proto') + >>> startup_program = fluid.default_startup_program() + >>> main_program = fluid.default_main_program() + >>> filelist = ["train_data/part-%d" % i for i in range(100)] + >>> thread_num = len(filelist) / 4 + >>> + >>> place = fluid.CPUPlace() + >>> async_executor = fluid.AsyncExecutor(place) + >>> + >>> async_executor.run_startup_program(startup_program) + >>> + >>> epoch = 10 + >>> for i in range(epoch): + >>> async_executor.run(main_program, + >>> data_feed, + >>> filelist, + >>> thread_num, + >>> [acc], + >>> debug=False) + + Args: + place(fluid.CPUPlace|None): indicate the executor run on which device. + Only CPUPlace supported + + Note: + For debugging complicated network in parallel-GPUs, you can test it + on the executor. They has the exactly same arguments, and expected + the same results. + + Note: Only running on CPUPlace supported. + """ + + def __init__(self, place=None): + if place is None: + place = core.CPUPlace() + if not isinstance(place, core.CPUPlace): + raise ValueError("AsyncExecutor only supports CPU device") + + p = core.Place() + p.set_place(place) + + scope = global_scope() + self.executor = core.AsyncExecutor(scope, p) + + def run(self, program, data_feed, filelist, thread_num, fetch, debug=False): + """ + Run program by this AsyncExecutor. Training dataset will be in filelist. + Users can also inspect certain variables by naming them in parameter + :code:`fetch`, like in fluid.Executor. Unlike fluid.Executor, however, + AsyncExecutor doesn't return fetched variables, instead, it will dump + the values of each fetched variable to stdandard output. + + Running the dataset will be on multiple threads, within each a thread + local scope will be created, then all OPs also created in that scope. + Parameters are updated by all the OPs simultaneously. + + Args: + program(Program): the program that need to run, if not provied, + then default_main_program will be used. + data_feed(DataFeedDesc): A DataFeedDesc object + filelist(str): a file containing the training dataset file list + thread_num(int): number of concurrent training threads. See + :code:`Note` for how to set this properly + fetch(str|list): the var name or a list of var names to inspect + debug(bool): When set to True, fetch vars will be printed to + standard output after each minibatch + + Note: + the executor will run all operators in the program but not only + the operators dependent by the fetch_list. + + Note: + Running AsyncExecutor will be on multiple threads, each bound to a + CPU core. To achieve best performance, it's suggested to set thread + num to be equal or slightly less than that of CPU cores. + """ + if program is None: + program = default_main_program() + program_desc = program.desc + + if data_feed is None: + raise ValueError('ValueError: data_feed should be provided') + + if filelist is None: + raise ValueError('ValueError: filelist should be provided') + + if isinstance(filelist, str): + filelist = [filelist] + + if not isinstance(thread_num, int): + raise TypeError('TypeError: thread_num should be a positive number') + + if fetch is not None: + if isinstance(fetch, Variable): + fetch = [fetch] + fetch_var_names = [var.name for var in fetch] + for fetch_var in fetch: + shape = fetch_var.shape + if shape[len(shape) - 1] != 1: + raise AssertionError( + "%s: Fetch variable has wrong shape. Only varibles " + "with the last dimension size 1 supported." % + (fetch_var.name)) + + self.executor.run_from_files(program_desc, + data_feed.desc(), filelist, thread_num, + fetch_var_names, debug) diff --git a/python/paddle/fluid/data_feed_desc.py b/python/paddle/fluid/data_feed_desc.py new file mode 100644 index 0000000000000000000000000000000000000000..d2ec74d6cfdeb34c1f48c086a3aa30d5100c3efb --- /dev/null +++ b/python/paddle/fluid/data_feed_desc.py @@ -0,0 +1,152 @@ +# 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 paddle.fluid.proto import data_feed_pb2 +from google.protobuf import text_format + +__all__ = ['DataFeedDesc'] + + +class DataFeedDesc(object): + """ + Datafeed descriptor, describing input training data format. This class is + currently only used for AsyncExecutor (See comments for class AsyncExecutor + for a brief introduction) + + DataFeedDesc shall be initialized from a valid protobuf message from disk: + >>> data_feed = fluid.DataFeedDesc('data.proto') + + See :code:`paddle/fluid/framework/data_feed.proto` for message definition. + A typical message might look like: + + >>> name: "MultiSlotDataFeed" + >>> batch_size: 2 + >>> multi_slot_desc { + >>> slots { + >>> name: "words" + >>> type: "uint64" + >>> is_dense: false + >>> is_used: true + >>> } + >>> slots { + >>> name: "label" + >>> type: "uint64" + >>> is_dense: false + >>> is_used: true + >>> } + >>> } + + However, users usually shouldn't care about the message format; instead, + they are encouragd to use :code:`Data Generator` as a tool to generate a + valid data description, in the process of converting their raw log files to + training files acceptable to AsyncExecutor. + + DataFeedDesc can also be changed during runtime. Once you got familiar with + what each field mean, you can modify it to better suit your need. E.g.: + >>> data_feed.set_batch_size(128) + >>> data_feed.set_dense_slots('wd') # The slot named 'wd' will be dense + >>> data_feed.set_use_slots('wd') # The slot named 'wd' will be used + + Finally, the content can be dumped out for debugging purpose: + >>> print(data_feed.desc()) + + Args: + proto_file(string): Disk file containing a data feed description. + + """ + + def __init__(self, proto_file): + self.proto_desc = data_feed_pb2.DataFeedDesc() + with open(proto_file, 'r') as f: + text_format.Parse(f.read(), self.proto_desc) + if self.proto_desc.name == "MultiSlotDataFeed": + self.__name_to_index = { + slot.name: i + for i, slot in enumerate(self.proto_desc.multi_slot_desc.slots) + } + + def set_batch_size(self, batch_size): + """ + Set batch size. Will be effective during training + + Example: + >>> data_feed = fluid.DataFeedDesc('data.proto') + >>> data_feed.set_batch_size(128) + + Args: + batch_size: batch size + + """ + self.proto_desc.batch_size = batch_size + + def set_dense_slots(self, dense_slots_name): + """ + Set if a specific slot will be dense. Will be effective during training. + features for a dense slot will be fed into a Tensor, while those for a + sparse slot will be fed into a LoDTensor + + Example: + >>> data_feed = fluid.DataFeedDesc('data.proto') + >>> data_feed.set_dense_slots(['words']) + + Args: + dense_slots_name: a list of slot names which will be set dense + + Note: + Default is sparse for all slots + """ + if self.proto_desc.name != "MultiSlotDataFeed": + raise ValueError( + "Only MultiSlotDataFeed need set_dense_slots, pls check your datafeed.proto" + ) + for name in dense_slots_name: + self.proto_desc.multi_slot_desc.slots[self.__name_to_index[ + name]].is_dense = True + + def set_use_slots(self, use_slots_name): + """ + Set if a specific slot will be used for training. A dataset shall + contain a lot of features, through this function one can select which + ones will be used for a specific model. + + Example: + >>> data_feed = fluid.DataFeedDesc('data.proto') + >>> data_feed.set_use_slots(['words']) + + Args: + use_slots_name: a list of slot names which will be used in training + + Note: + Default is not used for all slots + """ + if self.proto_desc.name != "MultiSlotDataFeed": + raise ValueError( + "Only MultiSlotDataFeed need set_use_slots, pls check your datafeed.proto" + ) + for name in use_slots_name: + self.proto_desc.multi_slot_desc.slots[self.__name_to_index[ + name]].is_used = True + + def desc(self): + """ + Returns a protobuf message for this DataFeedDesc + + Example: + >>> data_feed = fluid.DataFeedDesc('data.proto') + >>> print(data_feed.desc()) + + Returns: + A string message + """ + return text_format.MessageToString(self.proto_desc) diff --git a/python/paddle/fluid/executor.py b/python/paddle/fluid/executor.py index 288951cd7cd32155f136125fb817c35dd2ec6444..42c2484b284844a1f1acf53f79296e13da72676a 100644 --- a/python/paddle/fluid/executor.py +++ b/python/paddle/fluid/executor.py @@ -278,6 +278,7 @@ class Executor(object): p = core.Place() p.set_place(place) self.executor = core.Executor(p) + self.program_caches = dict() self._closed = False diff --git a/python/paddle/fluid/layers/nn.py b/python/paddle/fluid/layers/nn.py index 56366142eb77d20fdb5c1dee454ee5135bc446bc..dbc39afccbbbefa88873329e7c6790fe26dcc11e 100644 --- a/python/paddle/fluid/layers/nn.py +++ b/python/paddle/fluid/layers/nn.py @@ -169,6 +169,7 @@ __all__ = [ 'log_loss', 'add_position_encoding', 'bilinear_tensor_product', + 'lstm', ] @@ -472,6 +473,168 @@ def dynamic_lstm(input, return hidden, cell +def lstm(input, + init_h, + init_c, + max_len, + hidden_size, + num_layers, + dropout_prob=0.0, + is_bidirec=False, + is_test=False, + name=None, + default_initializer=None, + seed=-1): + """ + If Device is GPU, This op will use cudnn LSTM implementation + + A four-gate Long Short-Term Memory network with no peephole connections. + In the forward pass the output ht and cell output ct for a given iteration can be computed from the recurrent input ht-1, + the cell input ct-1 and the previous layer input xt given matrices W, R and biases bW, bR from the following equations: + + $$ i_t = \\sigma(W_{ix}x_{t} + W_{ih}h_{t-1} + bx_i + bh_i) $$ + + $$ f_t = \\sigma(W_{fx}x_{t} + W_{fh}h_{t-1} + bx_f + bh_f) $$ + + $$ o_t = \\sigma(W_{ox}x_{t} + W_{oh}h_{t-1} + bx_o + bh_o) $$ + + $$ \\tilde{c_t} = tanh(W_{cx}x_t + W_{ch}h_{t-1} + bx_c + bh_c) $$ + + $$ c_t = f_t \\odot c_{t-1} + i_t \\odot \\tilde{c_t} $$ + + $$ h_t = o_t \\odot tanh(c_t) $$ + + - W terms denote weight matrices (e.g. $W_{ix}$ is the matrix + of weights from the input gate to the input) + - The b terms denote bias vectors ($bx_i$ and $bh_i$ are the input gate bias vector). + - sigmoid is the logistic sigmoid function. + - $i, f, o$ and $c$ are the input gate, forget gate, output gate, + and cell activation vectors, respectively, all of which have the same size as + the cell output activation vector $h$. + - The $\odot$ is the element-wise product of the vectors. + - `tanh` is the activation functions. + - $\tilde{c_t}$ is also called candidate hidden state, + which is computed based on the current input and the previous hidden state. + + Where sigmoid is the sigmoid operator: sigmoid(x) = 1 / (1 + e^-x), * represents a point-wise multiplication, + X represensts a matrix multiplication + + + Args: + input (Variable): LSTM input tensor, shape MUST be ( seq_len x batch_size x input_size ) + init_h(Variable): The initial hidden state of the LSTM + This is a tensor with shape ( num_layers x batch_size x hidden_size) + if is_bidirec = True, shape should be ( num_layers*2 x batch_size x hidden_size) + init_c(Variable): The initial cell state of the LSTM. + This is a tensor with shape ( num_layers x batch_size x hidden_size ) + if is_bidirec = True, shape should be ( num_layers*2 x batch_size x hidden_size) + max_len (int): max length of LSTM. the first dim of input tensor CAN NOT greater than max_len + hidden_size (int): hidden size of the LSTM + num_layers (int): total layers number of the LSTM + dropout_prob(float|0.0): dropout prob, dropout ONLY work between rnn layers, NOT between time steps + There is NO dropout work on rnn output of the last RNN layers + is_bidirec (bool): If it is bidirectional + is_test (bool): If it is in test phrase + name (str|None): A name for this layer(optional). If set None, the layer + will be named automatically. + default_initializer(Initialize|None): Where use initializer to initialize the Weight + If set None, defaule initializer will be used + seed(int): Seed for dropout in LSTM, If it's -1, dropout will use random seed + + + Returns: + rnn_out(Tensor): result of LSTM hidden, shape is (seq_len x batch_size x hidden_size) + if is_bidirec set to True, shape will be ( seq_len x batch_sze x hidden_size*2) + last_h(Tensor): the hidden state of the last step of LSTM + shape is ( num_layers x batch_size x hidden_size ) + if is_bidirec set to True, shape will be ( num_layers*2 x batch_size x hidden_size) + last_c(Tensor): the cell state of the last step of LSTM + shape is ( num_layers x batch_size x hidden_size ) + if is_bidirec set to True, shape will be ( num_layers*2 x batch_size x hidden_size) + + + Examples: + .. code-block:: python + + input = embedding + batch_size = 20 + max_len = 100 + dropout_prob = 0.2 + input_size = 100 + hidden_size = 150 + num_layers = 1 + init_hidden1 = layers.fill_constant( [num_layers, batch_size, hidden_size], 'float32', 0.0, stop_grad=False) + init_cell1 = layers.fill_constant( [num_layers, batch_size, hidden_size], 'float32', 0.0, stop_grad=False) + + rnn_out, last_h, last_c = layers.lstm( input, init_h, init_c, \ + max_len, dropout_prob, input_size, hidden_size, \ + num_layers) + """ + + helper = LayerHelper('cudnn_lstm', **locals()) + + dtype = input.dtype + input_shape = list(input.shape) + input_size = input_shape[-1] + weight_size = 0 + for i in range(num_layers): + if i == 0: + input_weight_size = (input_size * hidden_size) * 4 + else: + if is_bidirec: + input_weight_size = (hidden_size * 2 * hidden_size) * 4 + else: + input_weight_size = (hidden_size * hidden_size) * 4 + + hidden_weight_size = (hidden_size * hidden_size) * 4 + + if is_bidirec: + weight_size += (input_weight_size + hidden_weight_size) * 2 + weight_size += hidden_size * 8 * 2 + else: + weight_size += input_weight_size + hidden_weight_size + weight_size += hidden_size * 8 + + weight = helper.create_parameter( + attr=helper.param_attr, + shape=[weight_size], + dtype=dtype, + default_initializer=default_initializer) + + out = helper.create_variable_for_type_inference(dtype) + last_h = helper.create_variable_for_type_inference(dtype) + last_c = helper.create_variable_for_type_inference(dtype) + + cache = helper.create_variable( + persistable=True, type=core.VarDesc.VarType.RAW, stop_gradient=True) + + helper.append_op( + type='cudnn_lstm', + inputs={ + 'Input': input, + 'InitH': init_h, + 'InitC': init_c, + 'W': weight, + 'Cache': cache, + }, + outputs={ + 'Out': out, + 'last_h': last_h, + 'last_c': last_c, + }, + attrs={ + 'max_len': max_len, + 'is_bidirec': is_bidirec, + 'input_size': input_size, + 'hidden_size': hidden_size, + 'num_layers': num_layers, + 'is_test': is_test, + 'dropout_prob': dropout_prob, + 'seed': seed, + }) + return out, last_h, last_c + + def dynamic_lstmp(input, size, proj_size, @@ -4250,8 +4413,15 @@ def ctc_greedy_decoder(input, blank, name=None): [0.5, 0.1, 0.3, 0.1]] input.lod = [[4, 4]] + + Computation: - Then: + step1: Apply argmax to first input sequence which is input.data[0:4]. Then we get: + [[0], [2], [1], [0]] + step2: merge repeated tokens and remove blank which is 0. Then we get first output sequence: + [[2], [1]] + + Finally: output.data = [[2], [1], @@ -4259,6 +4429,7 @@ def ctc_greedy_decoder(input, blank, name=None): output.lod = [[2, 1]] + Args: input(Variable): (LoDTensor), the probabilities of @@ -4273,8 +4444,10 @@ def ctc_greedy_decoder(input, blank, name=None): name (str): The name of this layer. It is optional. Returns: - Variable: CTC greedy decode result. If all the sequences in result were - empty, the result LoDTensor will be [-1] with LoD [[]] and dims [1, 1]. + Variable: CTC greedy decode result which is a 2-D tensor with shape [Lp, 1]. + 'Lp' is the sum if all output sequences' length. If all the sequences + in result were empty, the result LoDTensor will be [-1] with + LoD [[]] and dims [1, 1]. Examples: .. code-block:: python @@ -6924,7 +7097,7 @@ def pad2d(input, Args: input (Variable): The input image with [N, C, H, W] format or [N, H, W, C] format. - paddings (tuple|list): The padding size. If padding is a tuple, it must + paddings (tuple|list|Variable): The padding size. If padding is a tuple, it must contain four integers, (padding_top, padding_bottom, padding_left, padding_right). Default: padding = [0, 0, 0, 0]. mode (str): Three modes: constant(default), reflect, edge. Default: constant @@ -6949,16 +7122,17 @@ def pad2d(input, helper = LayerHelper('pad2d', **locals()) dtype = helper.input_dtype(input_param_name='input') out = helper.create_variable_for_type_inference(dtype) + inputs = {'X': input} + attrs = {'mode': mode, 'pad_value': pad_value, 'data_format': data_format} + + if isinstance(paddings, Variable): + inputs['Paddings'] = paddings + attrs['paddings'] = [] + else: + attrs['paddings'] = paddings + helper.append_op( - type='pad2d', - inputs={'X': input}, - outputs={"Out": out}, - attrs={ - 'paddings': paddings, - 'mode': mode, - 'pad_value': pad_value, - 'data_frmat': data_format - }) + type='pad2d', inputs=inputs, outputs={"Out": out}, attrs=attrs) return out @@ -7606,6 +7780,11 @@ def uniform_random_batch_size_like(input, Returns: out (Variable): ${out_comment} + Examples: + .. code-block:: python + + input = layers.data(name="input", shape=[13, 11], dtype='float32') + out = layers.uniform_random_batch_size_like(input, [-1, 11]) """ helper = LayerHelper('uniform_random_batch_size_like', **locals()) @@ -7643,6 +7822,10 @@ def gaussian_random(shape, mean=0.0, std=1.0, seed=0, dtype='float32'): Returns: out (Variable): ${out_comment} + Examples: + .. code-block:: python + + out = layers.gaussian_random(shape=[20, 30]) """ helper = LayerHelper('gaussian_random', **locals()) @@ -7678,6 +7861,16 @@ def sampling_id(x, min=0.0, max=1.0, seed=0, dtype='float32'): Returns: out (Variable): ${out_comment} + Examples: + .. code-block:: python + + x = layers.data( + name="X", + shape=[13, 11], + dtype='float32', + append_batch_size=False) + + out = layers.sampling_id(x) """ helper = LayerHelper('sampling_id', **locals()) @@ -7717,6 +7910,14 @@ def gaussian_random_batch_size_like(input, Returns: out (Variable): ${out_comment} + + Examples: + .. code-block:: python + + input = layers.data(name="input", shape=[13, 11], dtype='float32') + + out = layers.gaussian_random_batch_size_like( + input, shape=[-1, 11], mean=1.0, std=2.0) """ helper = LayerHelper('gaussian_random_batch_size_like', **locals()) @@ -7749,6 +7950,12 @@ def sum(x): Returns: out (Variable): ${out_comment} + + Examples: + .. code-block:: python + + input = layers.data(name="input", shape=[13, 11], dtype='float32') + out = layers.sum(input) """ helper = LayerHelper('sum', **locals()) @@ -7777,6 +7984,17 @@ def slice(input, axes, starts, ends): Returns: out (Variable): ${out_comment} + Examples: + .. code-block:: python + + starts = [1, 0, 2] + ends = [3, 3, 4] + axes = [0, 1, 2] + + input = layers.data( + name="input", shape=[3, 4, 5, 6], dtype='float32') + + out = layers.slice(input, axes=axes, starts=starts, ends=ends) """ helper = LayerHelper('slice', **locals()) @@ -7804,6 +8022,12 @@ def shape(input): Returns: out (Variable): ${out_comment} + Examples: + .. code-block:: python + + input = layers.data( + name="input", shape=[3, 100, 100], dtype="float32") + out = layers.shape(input) """ helper = LayerHelper('shape', **locals()) diff --git a/python/paddle/fluid/tests/demo/async_executor.py b/python/paddle/fluid/tests/demo/async_executor.py new file mode 100644 index 0000000000000000000000000000000000000000..fe8da0aab74bd5fc6219666236a04423a6d60489 --- /dev/null +++ b/python/paddle/fluid/tests/demo/async_executor.py @@ -0,0 +1,100 @@ +# 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 tarfile +import paddle.fluid as fluid +import paddle +from paddle.fluid import core + +URL = 'http://paddle-unittest-data.gz.bcebos.com/python_paddle_fluid_tests_demo_async-executor/train_data.tar.gz' +MD5 = '2a405a31508969b3ab823f42c0f522ca' + + +def bow_net(data, + label, + dict_dim=89528, + emb_dim=128, + hid_dim=128, + hid_dim2=96, + class_dim=2): + """ + BOW net + This model is from https://github.com/PaddlePaddle/models: + models/fluid/PaddleNLP/text_classification/nets.py + """ + # embedding + emb = fluid.layers.embedding( + input=data, size=[dict_dim, emb_dim], is_sparse=True) + bow = fluid.layers.sequence_pool(input=emb, pool_type='sum') + bowh = fluid.layers.tanh(bow) + # fc layer after conv + fc_1 = fluid.layers.fc(input=bowh, size=hid_dim, act="tanh") + fc_2 = fluid.layers.fc(input=fc_1, size=hid_dim2, act="tanh") + # probability of each class + prediction = fluid.layers.fc(input=[fc_2], size=class_dim, act="softmax") + # cross entropy loss + cost = fluid.layers.cross_entropy(input=prediction, label=label) + # mean loss + avg_cost = fluid.layers.mean(x=cost) + acc = fluid.layers.accuracy(input=prediction, label=label) + return avg_cost, acc, prediction + + +def train(): + # Download data + with tarfile.open(paddle.dataset.common.download(URL, "imdb", MD5)) as tarf: + tarf.extractall(path='./') + tarf.close() + + # Initialize dataset description + dataset = fluid.DataFeedDesc('train_data/data.prototxt') + dataset.set_batch_size(128) # See API doc for how to change other fields + print dataset.desc() # Debug purpose: see what we get + + # define network + # input text data + data = fluid.layers.data( + name="words", shape=[1], dtype="int64", lod_level=1) + # label data + label = fluid.layers.data(name="label", shape=[1], dtype="int64") + + avg_cost, acc, prediction = bow_net(data, label) + sgd_optimizer = fluid.optimizer.Adagrad(learning_rate=0.002) + opt_ops, weight_and_grad = sgd_optimizer.minimize(avg_cost) + + # Run startup program + startup_program = fluid.default_startup_program() + place = fluid.CPUPlace() + executor = fluid.Executor(place) + executor.run(startup_program) + + async_executor = fluid.AsyncExecutor(place) + main_program = fluid.default_main_program() + epochs = 10 + filelist = ["train_data/part-%d" % i for i in range(12)] + for i in range(epochs): + thread_num = 4 + async_executor.run( + main_program, # This can be changed during iteration + dataset, # This can be changed during iteration + filelist, # This can be changed during iteration + thread_num, # This can be changed during iteration + [data, acc], # Multiple fetch targets can be specified + debug=False) + fluid.io.save_inference_model('imdb/epoch%d.model' % i, + [data.name, label.name], [acc], executor) + + +if __name__ == "__main__": + train() diff --git a/python/paddle/fluid/tests/unittests/op_test.py b/python/paddle/fluid/tests/unittests/op_test.py index 271b9c740fd99554e9a7aa8d476a52cf6385b1d9..76a707efdc0804be0316ab12c347ffed6199529a 100644 --- a/python/paddle/fluid/tests/unittests/op_test.py +++ b/python/paddle/fluid/tests/unittests/op_test.py @@ -216,6 +216,15 @@ class OpTest(unittest.TestCase): self.dtype) outputs = append_input_output(block, op_proto, self.outputs, False, self.dtype) + + if hasattr(self, "cache_name_list"): + for name in self.cache_name_list: + inputs[name] = block.create_var( + name=name, + persistable=True, + type=core.VarDesc.VarType.RAW, + stop_gradient=True) + op = block.append_op( type=self.op_type, inputs=inputs, @@ -428,8 +437,17 @@ class OpTest(unittest.TestCase): op_inputs = self.inputs if hasattr(self, "inputs") else dict() op_outputs = self.outputs if hasattr(self, "outputs") else dict() op_attrs = self.attrs if hasattr(self, "attrs") else dict() - self.op = create_op(self.scope, self.op_type, op_inputs, op_outputs, - op_attrs) + + cache_list = None + if hasattr(self, "cache_name_list"): + cache_list = self.cache_name_list + self.op = create_op( + self.scope, + self.op_type, + op_inputs, + op_outputs, + op_attrs, + cache_list=cache_list) if no_grad_set is None: no_grad_set = set() diff --git a/python/paddle/fluid/tests/unittests/test_async_executor.py b/python/paddle/fluid/tests/unittests/test_async_executor.py new file mode 100644 index 0000000000000000000000000000000000000000..43855b95f9e3096d58ca3e8acfdb25f034bab175 --- /dev/null +++ b/python/paddle/fluid/tests/unittests/test_async_executor.py @@ -0,0 +1,142 @@ +# 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 unittest +import tarfile +import os +import shutil + +proto_str = ('name: "MultiSlotDataFeed"\n' + 'batch_size: 2\n' + 'multi_slot_desc {\n' + ' slots {\n' + ' name: "words"\n' + ' type: "uint64"\n' + ' is_dense: false\n' + ' is_used: true\n' + ' }\n' + ' slots {\n' + ' name: "label"\n' + ' type: "uint64"\n' + ' is_dense: false\n' + ' is_used: true\n' + ' }\n' + '}') + +URL = 'http://paddle-unittest-data.gz.bcebos.com/python_paddle_fluid_tests_demo_async-executor/train_data.tar.gz' +MD5 = '2a405a31508969b3ab823f42c0f522ca' + + +def bow_net(data, + label, + dict_dim=89528, + emb_dim=128, + hid_dim=128, + hid_dim2=96, + class_dim=2): + """ + BOW net + This model is from https://github.com/PaddlePaddle/models: + models/fluid/PaddleNLP/text_classification/nets.py + """ + # embedding + emb = fluid.layers.embedding( + input=data, size=[dict_dim, emb_dim], is_sparse=True) + bow = fluid.layers.sequence_pool(input=emb, pool_type='sum') + bowh = fluid.layers.tanh(bow) + # fc layer after conv + fc_1 = fluid.layers.fc(input=bowh, size=hid_dim, act="tanh") + fc_2 = fluid.layers.fc(input=fc_1, size=hid_dim2, act="tanh") + # probability of each class + prediction = fluid.layers.fc(input=[fc_2], size=class_dim, act="softmax") + # cross entropy loss + cost = fluid.layers.cross_entropy(input=prediction, label=label) + # mean loss + avg_cost = fluid.layers.mean(x=cost) + acc = fluid.layers.accuracy(input=prediction, label=label) + return avg_cost, acc, prediction + + +class TestAsyncExecutor(unittest.TestCase): + def setUp(self): + with open('./data.prototxt', 'w+') as f: + f.write(proto_str) + f.close() + + with tarfile.open(paddle.dataset.common.download(URL, "imdb", + MD5)) as tarf: + tarf.extractall(path='./') + tarf.close() + + def test_data_feed_desc(self): + data_feed = fluid.DataFeedDesc('./data.prototxt') + # assertEqueal(data_feed.proto_desc.batch, 2) + # assertEqual(len(data_feed.proto_desc.multi_slot_desc), 2) + self.assertEqual(" ".join(data_feed.desc().split()), + " ".join(proto_str.split())) + + def test_run(self): + # Initialize dataset description + data_feed = fluid.DataFeedDesc('train_data/data.prototxt') + data_feed.set_batch_size( + 128) # See API doc for how to change other fields + + # define network + # input text data + data = fluid.layers.data( + name="words", shape=[1], dtype="int64", lod_level=1) + # label data + label = fluid.layers.data(name="label", shape=[1], dtype="int64") + + avg_cost, acc, prediction = bow_net(data, label) + sgd_optimizer = fluid.optimizer.Adagrad(learning_rate=0.002) + opt_ops, weight_and_grad = sgd_optimizer.minimize(avg_cost) + + # Run startup program + startup_program = fluid.default_startup_program() + place = fluid.CPUPlace() + executor = fluid.Executor(place) + executor.run(startup_program) + + main_program = fluid.default_main_program() + async_executor = fluid.AsyncExecutor(place) + + self.assertRaises(TypeError, async_executor.run) + self.assertRaises(TypeError, async_executor.run, main_program) + self.assertRaises(TypeError, async_executor.run, main_program, + data_feed) + + filelist = ['train_data/part-%d' % i for i in range(10)] + self.assertRaises(TypeError, async_executor.run, main_program, + data_feed, filelist) + + thread_num = 4 + self.assertRaises(TypeError, async_executor.run, main_program, + data_feed, filelist, thread_num) + + async_executor.run(main_program, data_feed, filelist, thread_num, [acc]) + fluid.io.save_inference_model("imdb.model", [data.name, label.name], + [acc], executor) + statinfo = os.stat('imdb.model/__model__') + self.assertGreater(statinfo.st_size, 0) + + os.remove('./data.prototxt') + shutil.rmtree('./train_data') + shutil.rmtree('./imdb.model') + + +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 2004c917931a1a4ed06d35abcced34218dbfbbb8..7f477031ddef6b459dcda74a18d0831763f4aeca 100644 --- a/python/paddle/fluid/tests/unittests/test_layers.py +++ b/python/paddle/fluid/tests/unittests/test_layers.py @@ -636,13 +636,21 @@ class TestBook(unittest.TestCase): with program_guard(program): input = layers.data( name="input", shape=[3, 100, 100], dtype="float32") + paddings = layers.fill_constant(shape=[4], dtype='int32', value=1) out = layers.pad2d( input, paddings=[1, 2, 3, 4], mode='reflect', data_format='NCHW', name="shape") + out_1 = layers.pad2d( + input, + paddings=paddings, + mode='reflect', + data_format='NCHW', + name="shape") self.assertIsNotNone(out) + self.assertIsNotNone(out_1) print(str(program)) def test_prelu(self): diff --git a/python/paddle/fluid/tests/unittests/test_lstm_cudnn_op.py b/python/paddle/fluid/tests/unittests/test_lstm_cudnn_op.py new file mode 100644 index 0000000000000000000000000000000000000000..0e9e2e8429e51a328e397f9e2a05ab7209c9c1a2 --- /dev/null +++ b/python/paddle/fluid/tests/unittests/test_lstm_cudnn_op.py @@ -0,0 +1,192 @@ +# 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 paddle.fluid as fluid + +SIGMOID_THRESHOLD_MIN = -40.0 +SIGMOID_THRESHOLD_MAX = 13.0 +EXP_MAX_INPUT = 40.0 + + +def lstm_naive( + input, + w, ): + seq_len, batch_size, hidden_size = input.shape + + offset = 0 + wi = w[offset:offset + hidden_size * hidden_size].reshape( + (hidden_size, hidden_size)).transpose() + offset += hidden_size * hidden_size + wf = w[offset:offset + hidden_size * hidden_size].reshape( + (hidden_size, hidden_size)).transpose() + offset += hidden_size * hidden_size + wc = w[offset:offset + hidden_size * hidden_size].reshape( + (hidden_size, hidden_size)).transpose() + offset += hidden_size * hidden_size + wo = w[offset:offset + hidden_size * hidden_size].reshape( + (hidden_size, hidden_size)).transpose() + offset += hidden_size * hidden_size + ri = w[offset:offset + hidden_size * hidden_size].reshape( + (hidden_size, hidden_size)).transpose() + offset += hidden_size * hidden_size + rf = w[offset:offset + hidden_size * hidden_size].reshape( + (hidden_size, hidden_size)).transpose() + offset += hidden_size * hidden_size + rc = w[offset:offset + hidden_size * hidden_size].reshape( + (hidden_size, hidden_size)).transpose() + offset += hidden_size * hidden_size + ro = w[offset:offset + hidden_size * hidden_size].reshape( + (hidden_size, hidden_size)).transpose() + offset += hidden_size * hidden_size + + bi_1 = w[offset:offset + hidden_size] + offset += hidden_size + bf_1 = w[offset:offset + hidden_size] + offset += hidden_size + bc_1 = w[offset:offset + hidden_size] + offset += hidden_size + bo_1 = w[offset:offset + hidden_size] + offset += hidden_size + + bi_2 = w[offset:offset + hidden_size] + offset += hidden_size + bf_2 = w[offset:offset + hidden_size] + offset += hidden_size + bc_2 = w[offset:offset + hidden_size] + offset += hidden_size + bo_2 = w[offset:offset + hidden_size] + + def sigmoid(x): + y = np.copy(x) + y[x < SIGMOID_THRESHOLD_MIN] = SIGMOID_THRESHOLD_MIN + y[x > SIGMOID_THRESHOLD_MAX] = SIGMOID_THRESHOLD_MAX + return 1. / (1. + np.exp(-y)) + + def tanh(x): + y = -2. * x + y[y > EXP_MAX_INPUT] = EXP_MAX_INPUT + return (2. / (1. + np.exp(y))) - 1. + + output = [] + pre_h = np.zeros((batch_size, hidden_size), dtype=input.dtype) + pre_c = np.zeros((batch_size, hidden_size), dtype=input.dtype) + + for i in range(seq_len): + emb_1 = input[i] + + input_gate = sigmoid( + np.matmul(emb_1, wi) + np.matmul(pre_h, ri) + bi_1 + bi_2) + forget_gate = sigmoid( + np.matmul(emb_1, wf) + np.matmul(pre_h, rf) + bf_1 + bf_2) + output_gate = sigmoid( + np.matmul(emb_1, wo) + np.matmul(pre_h, ro) + bo_1 + bo_2) + c_t_temp = tanh( + np.matmul(emb_1, wc) + np.matmul(pre_h, rc) + bc_1 + bc_2) + new_c = input_gate * c_t_temp + forget_gate * pre_c + new_h = output_gate * tanh(new_c) + + pre_h = new_h + pre_c = new_c + + output.append(new_h) + + output = np.concatenate(output, -1) + output = output.reshape((batch_size, -1, hidden_size)) + + output = output.transpose((1, 0, 2)) + + return output, pre_h, pre_c + + +class TestCUDNNLstmOp(OpTest): + def setUp(self): + self.op_type = "cudnn_lstm" + self.dtype = np.float32 + + num_steps = 20 + batch_size = 5 + hidden_size = 20 + + input_weight_size = (hidden_size * hidden_size) * 4 + hidden_weight_size = (hidden_size * hidden_size) * 4 + weight_size = input_weight_size + hidden_weight_size + weight_size += hidden_size * 8 + + input = np.random.uniform( + low=-0.1, high=0.1, size=(num_steps, batch_size, + hidden_size)).astype(self.dtype) + flat_w = np.random.uniform( + low=-0.1, high=0.1, size=(weight_size)).astype(self.dtype) + + output, last_hidden, last_cell = lstm_naive(input, flat_w) + + init_h = np.zeros((batch_size, hidden_size), dtype=np.float32) + init_c = np.zeros((batch_size, hidden_size), dtype=np.float32) + scope = core.Scope() + program = fluid.Program() + block = program.global_block() + + cache_temp = block.create_var( + name="Cache", + persistable=True, + type=core.VarDesc.VarType.RAW, + stop_gradient=True) + self.inputs = { + 'Input': OpTest.np_dtype_to_fluid_dtype(input), + 'W': OpTest.np_dtype_to_fluid_dtype(flat_w), + 'InitH': OpTest.np_dtype_to_fluid_dtype(init_h), + 'InitC': OpTest.np_dtype_to_fluid_dtype(init_c), + } + self.cache_name_list = ['Cache'] + self.attrs = { + 'max_len': num_steps, + 'dropout_prob': 0.0, + 'is_bidirec': False, + 'input_size': hidden_size, + 'hidden_size': hidden_size, + 'num_layers': 1, + } + self.outputs = { + 'Out': output, + "last_h": last_hidden, + 'last_c': last_cell + } + + def test_output_with_place(self): + if self.testcuda(): + place = core.CUDAPlace(0) + self.check_output_with_place(place, atol=1e-5) + + def test_grad_with_place(self): + if core.is_compiled_with_cuda(): + place = core.CUDAPlace(0) + self.check_grad_with_place( + place, + set(['Input', 'W', 'InitH', 'InitC']), + ['Out', 'last_h', 'last_c'], + max_relative_error=0.02) + + def testcuda(self): + return core.is_compiled_with_cuda() + + +if __name__ == '__main__': + unittest.main() diff --git a/python/paddle/fluid/tests/unittests/test_pad2d_op.py b/python/paddle/fluid/tests/unittests/test_pad2d_op.py index 728b8c181a4410d7df7f304bcc8d2816e91ea6d8..5c4a6ca59e53d0edafda87eae19516a80ec32c40 100644 --- a/python/paddle/fluid/tests/unittests/test_pad2d_op.py +++ b/python/paddle/fluid/tests/unittests/test_pad2d_op.py @@ -20,11 +20,17 @@ from op_test import OpTest class TestPad2dOp(OpTest): def setUp(self): self.pad_value = 0.0 + self.variable_paddings = False self.initTestCase() self.op_type = "pad2d" self.inputs = {'X': np.random.random(self.shape).astype("float32"), } self.attrs = {} - self.attrs['paddings'] = np.array(self.paddings).flatten() + if self.variable_paddings: + self.attrs['paddings'] = [] + self.inputs['Paddings'] = np.array(self.paddings).flatten().astype( + "int32") + else: + self.attrs['paddings'] = np.array(self.paddings).flatten() self.attrs['pad_value'] = self.pad_value self.attrs['mode'] = self.mode self.attrs['data_format'] = self.data_format @@ -98,5 +104,24 @@ class TestCase5(TestPad2dOp): self.data_format = "NHWC" +class TestCase6(TestPad2dOp): + def initTestCase(self): + self.shape = (2, 4, 4, 2) + self.paddings = [0, 1, 2, 3] + self.mode = "constant" + self.pad_value = 1.2 + self.data_format = "NHWC" + self.variable_paddings = True + + +class TestCase7(TestPad2dOp): + def initTestCase(self): + self.shape = (2, 3, 4, 4) + self.paddings = [0, 1, 2, 3] + self.mode = "reflect" + self.data_format = "NCHW" + self.variable_paddings = True + + if __name__ == '__main__': unittest.main() diff --git a/python/paddle/fluid/tests/unittests/testsuite.py b/python/paddle/fluid/tests/unittests/testsuite.py index 34fbb1b549cf5fc5f75bcc0715e5c83665f1d200..dc3b2cb8bc15836a4bf067caa05c3a37a917ecad 100644 --- a/python/paddle/fluid/tests/unittests/testsuite.py +++ b/python/paddle/fluid/tests/unittests/testsuite.py @@ -20,7 +20,7 @@ import paddle.fluid.core as core from paddle.fluid.op import Operator -def create_op(scope, op_type, inputs, outputs, attrs): +def create_op(scope, op_type, inputs, outputs, attrs, cache_list=None): kwargs = dict() op_maker = core.op_proto_and_checker_maker @@ -43,6 +43,11 @@ def create_op(scope, op_type, inputs, outputs, attrs): __create_var__(in_name, sub_in_name) else: __create_var__(in_name, in_name) + if cache_list != None and isinstance(cache_list, list): + for name in cache_list: + kwargs[name] = [] + scope.var(name) + kwargs[name].append(name) for out_name, out_dup in Operator.get_op_outputs(op_type): if out_name in outputs: diff --git a/python/paddle/reader/tests/decorator_test.py b/python/paddle/reader/tests/decorator_test.py index b9af8348e16c051db64d57a9594aee303d83aef2..a9dddbbcc82e649b6c98db0fd58c62b58435b8db 100644 --- a/python/paddle/reader/tests/decorator_test.py +++ b/python/paddle/reader/tests/decorator_test.py @@ -62,10 +62,10 @@ class TestBuffered(unittest.TestCase): for idx, i in enumerate(b()): elapsed_time = time.time() - last_time if i == 0: - time.sleep(0.3) + time.sleep(1) else: # read time should be short, meaning already buffered. - self.assertLess(elapsed_time, 0.05) + self.assertLess(elapsed_time, 0.08) last_time = time.time()