(1) use channel instead of vector/BlockingQueue in Dataset,to keep same with existing implementation, and make code more readable and flexible (dataset single output channel or multi output channel). one previous memory out of limit problem is cause by not release memory after training.
(2) add Record because MultiSlotType costs too much memory (80B),fix memory out of limit problem.
(3) add Channel, Archive in paddle/fluid/framework
(4) change dataset from shared_ptr to unique_ptr in pybind
(5) move create/destroy readers from trainer to dataset
(6) move shuffle from datafeed to dataset. dataset holds memory, datafeed is only for load data and feed data to network.
(7) fix thread num bug of Dataset when filelist size < thread num
(8) support set_queue_num in InMemoryDataset
revert-18229-add_multi_gpu_install_check
jiaqi 6 years ago committed by GitHub
parent 5d54ed4a84
commit 3f8031e256
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

@ -55,6 +55,8 @@ class Dataset {
// set data fedd desc, which contains:
// data feed name, batch size, slots
virtual void SetDataFeedDesc(const std::string& data_feed_desc_str) = 0;
// set channel num
virtual void SetChannelNum(int channel_num) = 0;
// get file list
virtual const std::vector<std::string>& GetFileList() = 0;
// get thread num
@ -67,14 +69,21 @@ class Dataset {
virtual std::pair<std::string, std::string> GetHdfsConfig() = 0;
// get data fedd desc
virtual const paddle::framework::DataFeedDesc& GetDataFeedDesc() = 0;
// get channel num
virtual int GetChannelNum() = 0;
// get readers, the reader num depend both on thread num
// and filelist size
virtual std::vector<std::shared_ptr<paddle::framework::DataFeed>>&
GetReaders() = 0;
virtual std::vector<paddle::framework::DataFeed*> GetReaders() = 0;
// create input channel and output channel
virtual void CreateChannel() = 0;
// register message handler between workers
virtual void RegisterClientToClientMsgHandler() = 0;
// load all data into memory
virtual void LoadIntoMemory() = 0;
// load all data into memory in async mode
virtual void PreLoadIntoMemory() = 0;
// wait async load done
virtual void WaitPreLoadDone() = 0;
// release all memory data
virtual void ReleaseMemory() = 0;
// local shuffle data
@ -110,6 +119,7 @@ class DatasetImpl : public Dataset {
virtual void SetHdfsConfig(const std::string& fs_name,
const std::string& fs_ugi);
virtual void SetDataFeedDesc(const std::string& data_feed_desc_str);
virtual void SetChannelNum(int channel_num);
virtual const std::vector<std::string>& GetFileList() { return filelist_; }
virtual int GetThreadNum() { return thread_num_; }
@ -121,11 +131,13 @@ class DatasetImpl : public Dataset {
virtual const paddle::framework::DataFeedDesc& GetDataFeedDesc() {
return data_feed_desc_;
}
virtual std::vector<std::shared_ptr<paddle::framework::DataFeed>>&
GetReaders();
virtual int GetChannelNum() { return channel_num_; }
virtual std::vector<paddle::framework::DataFeed*> GetReaders();
virtual void CreateChannel();
virtual void RegisterClientToClientMsgHandler();
virtual void LoadIntoMemory();
virtual void PreLoadIntoMemory();
virtual void WaitPreLoadDone();
virtual void ReleaseMemory();
virtual void LocalShuffle();
virtual void GlobalShuffle();
@ -138,8 +150,14 @@ class DatasetImpl : public Dataset {
virtual int ReceiveFromClient(int msg_type, int client_id,
const std::string& msg);
std::vector<std::shared_ptr<paddle::framework::DataFeed>> readers_;
std::vector<T> memory_data_;
std::mutex mutex_for_update_memory_data_;
paddle::framework::Channel<T> input_channel_;
int channel_num_;
std::vector<paddle::framework::Channel<T>> multi_output_channel_;
std::vector<paddle::framework::Channel<T>> multi_consume_channel_;
// when read ins, we put ins from one channel to the other,
// and when finish reading, we set cur_channel = 1 - cur_channel,
// so if cur_channel=0, all data are in output_channel, else consume_channel
int cur_channel_;
int thread_num_;
paddle::framework::DataFeedDesc data_feed_desc_;
int trainer_num_;
@ -148,12 +166,13 @@ class DatasetImpl : public Dataset {
std::mutex mutex_for_pick_file_;
std::string fs_name_;
std::string fs_ugi_;
unsigned int rand_seed;
int64_t fleet_send_batch_size_;
int64_t fleet_send_sleep_seconds_;
std::vector<std::thread> preload_threads_;
};
// use std::vector<MultiSlotType> as data type
class MultiSlotDataset : public DatasetImpl<std::vector<MultiSlotType>> {
class MultiSlotDataset : public DatasetImpl<Record> {
public:
MultiSlotDataset() {}
virtual ~MultiSlotDataset() {}

@ -21,14 +21,14 @@ limitations under the License. */
namespace paddle {
namespace framework {
typedef std::shared_ptr<Dataset> (*CreateDatasetFunction)();
typedef std::unique_ptr<Dataset> (*CreateDatasetFunction)();
typedef std::unordered_map<std::string, CreateDatasetFunction> datasetMap;
datasetMap g_dataset_map;
#define REGISTER_DATASET_CLASS(dataset_class) \
namespace { \
std::shared_ptr<Dataset> Creator_##dataset_class() { \
return std::shared_ptr<Dataset>(new dataset_class); \
std::unique_ptr<Dataset> Creator_##dataset_class() { \
return std::unique_ptr<Dataset>(new dataset_class); \
} \
class __Registerer_##dataset_class { \
public: \
@ -50,7 +50,7 @@ std::string DatasetFactory::DatasetTypeList() {
return dataset_types;
}
std::shared_ptr<Dataset> DatasetFactory::CreateDataset(
std::unique_ptr<Dataset> DatasetFactory::CreateDataset(
std::string dataset_class) {
if (g_dataset_map.count(dataset_class) < 1) {
LOG(WARNING) << "Your Dataset " << dataset_class

@ -23,7 +23,7 @@ namespace framework {
class DatasetFactory {
public:
static std::string DatasetTypeList();
static std::shared_ptr<Dataset> CreateDataset(std::string dataset_class);
static std::unique_ptr<Dataset> CreateDataset(std::string dataset_class);
};
} // namespace framework
} // namespace paddle

@ -19,7 +19,7 @@ namespace framework {
void DeviceWorker::SetRootScope(Scope* root_scope) { root_scope_ = root_scope; }
void DeviceWorker::SetDataFeed(const std::shared_ptr<DataFeed>& data_feed) {
void DeviceWorker::SetDataFeed(DataFeed* data_feed) {
device_reader_ = data_feed;
}

@ -113,7 +113,7 @@ class DeviceWorker {
// will make this zero copy in the future
virtual void BindingDataFeedMemory() = 0;
virtual void SetRootScope(Scope* root_scope);
virtual void SetDataFeed(const std::shared_ptr<DataFeed>& data_feed);
virtual void SetDataFeed(DataFeed* data_feed);
virtual void SetPlace(const paddle::platform::Place& place) {
place_ = place;
}
@ -121,7 +121,7 @@ class DeviceWorker {
protected:
Scope* root_scope_;
paddle::platform::Place place_;
std::shared_ptr<DataFeed> device_reader_;
DataFeed* device_reader_;
int64_t batch_num_;
FetchConfig fetch_config_;
bool use_cvm_;

@ -27,8 +27,7 @@ void DistMultiTrainer::Initialize(const TrainerDesc& trainer_desc,
thread_num_ = trainer_desc.thread_num();
SetDataset(dataset);
dataset->CreateReaders();
const std::vector<std::shared_ptr<paddle::framework::DataFeed>> readers =
const std::vector<paddle::framework::DataFeed*> readers =
dataset->GetReaders();
thread_num_ = readers.size();
@ -72,7 +71,6 @@ void DistMultiTrainer::Finalize() {
th.join();
}
pull_dense_worker_->Stop();
dataset_ptr_->DestroyReaders();
root_scope_->DropKids();
}

@ -0,0 +1,32 @@
// Copyright (c) 2019 PaddlePaddle Authors. All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#pragma once
#if defined _WIN32 || defined __APPLE__
#else
#define _LINUX
#endif
#ifdef _LINUX
#ifndef likely
#define likely(x) __builtin_expect((x), 1)
#endif
#endif
#ifdef _LINUX
#ifndef unlikely
#define unlikely(x) __builtin_expect((x), 0)
#endif
#endif

@ -26,9 +26,7 @@ void MultiTrainer::Initialize(const TrainerDesc& trainer_desc,
thread_num_ = trainer_desc.thread_num();
SetDataset(dataset);
// get filelist from trainer_desc here
dataset->CreateReaders();
VLOG(3) << "readers created";
const std::vector<std::shared_ptr<paddle::framework::DataFeed>> readers =
const std::vector<paddle::framework::DataFeed*> readers =
dataset->GetReaders();
VLOG(3) << "readers num: " << readers.size();
// change thread num to readers num
@ -75,7 +73,6 @@ void MultiTrainer::Finalize() {
for (auto& th : threads_) {
th.join();
}
dataset_ptr_->DestroyReaders();
root_scope_->DropKids();
}

@ -28,9 +28,7 @@ void PipelineTrainer::Initialize(const TrainerDesc& trainer_desc,
SetDataset(dataset);
// get filelist from trainer_desc here
dataset->CreateReaders();
VLOG(3) << "readers created";
const std::vector<std::shared_ptr<paddle::framework::DataFeed>> readers =
const std::vector<paddle::framework::DataFeed*> readers =
dataset->GetReaders();
VLOG(3) << "readers num: " << readers.size();
@ -259,7 +257,6 @@ void PipelineTrainer::Finalize() {
pipeline_scopes_[0]->FindVar(var)->Get<LoDTensor>();
TensorCopySync(thread_tensor, platform::CPUPlace(), root_tensor);
}
dataset_ptr_->DestroyReaders();
root_scope_->DropKids();
}

@ -74,7 +74,7 @@ class MultiTrainer : public TrainerBase {
protected:
int thread_num_;
std::vector<std::thread> threads_;
std::vector<std::shared_ptr<DataFeed>> readers_;
std::vector<DataFeed*> readers_;
std::vector<std::shared_ptr<DeviceWorker>> workers_;
};
@ -136,7 +136,7 @@ class PipelineTrainer : public TrainerBase {
std::vector<std::unique_ptr<SyncFunctor>> sync_functors_;
std::shared_ptr<platform::NCCLContextMap> nccl_ctx_map_;
std::vector<std::shared_ptr<DataFeed>> readers_;
std::vector<DataFeed*> readers_;
void InitFirstScopeQueue(ScopeQueue* scope_queue, int pipeline_id,
const ProgramDesc& main_program);

@ -42,33 +42,64 @@ namespace paddle {
namespace pybind {
void BindDataset(py::module* m) {
py::class_<framework::Dataset, std::shared_ptr<framework::Dataset>>(*m,
py::class_<framework::Dataset, std::unique_ptr<framework::Dataset>>(*m,
"Dataset")
.def(py::init([](const std::string& name = "MultiSlotDataset") {
return framework::DatasetFactory::CreateDataset(name);
}))
.def("set_filelist", &framework::Dataset::SetFileList)
.def("set_thread_num", &framework::Dataset::SetThreadNum)
.def("set_trainer_num", &framework::Dataset::SetTrainerNum)
.def("set_filelist", &framework::Dataset::SetFileList,
py::call_guard<py::gil_scoped_release>())
.def("set_thread_num", &framework::Dataset::SetThreadNum,
py::call_guard<py::gil_scoped_release>())
.def("set_trainer_num", &framework::Dataset::SetTrainerNum,
py::call_guard<py::gil_scoped_release>())
.def("set_fleet_send_batch_size",
&framework::Dataset::SetFleetSendBatchSize)
.def("set_hdfs_config", &framework::Dataset::SetHdfsConfig)
.def("set_data_feed_desc", &framework::Dataset::SetDataFeedDesc)
.def("get_filelist", &framework::Dataset::GetFileList)
.def("get_thread_num", &framework::Dataset::GetThreadNum)
.def("get_trainer_num", &framework::Dataset::GetTrainerNum)
&framework::Dataset::SetFleetSendBatchSize,
py::call_guard<py::gil_scoped_release>())
.def("set_hdfs_config", &framework::Dataset::SetHdfsConfig,
py::call_guard<py::gil_scoped_release>())
.def("set_data_feed_desc", &framework::Dataset::SetDataFeedDesc,
py::call_guard<py::gil_scoped_release>())
.def("get_filelist", &framework::Dataset::GetFileList,
py::call_guard<py::gil_scoped_release>())
.def("get_thread_num", &framework::Dataset::GetThreadNum,
py::call_guard<py::gil_scoped_release>())
.def("get_trainer_num", &framework::Dataset::GetTrainerNum,
py::call_guard<py::gil_scoped_release>())
.def("get_fleet_send_batch_size",
&framework::Dataset::GetFleetSendBatchSize)
.def("get_hdfs_config", &framework::Dataset::GetHdfsConfig)
.def("get_data_feed_desc", &framework::Dataset::GetDataFeedDesc)
&framework::Dataset::GetFleetSendBatchSize,
py::call_guard<py::gil_scoped_release>())
.def("get_hdfs_config", &framework::Dataset::GetHdfsConfig,
py::call_guard<py::gil_scoped_release>())
.def("get_data_feed_desc", &framework::Dataset::GetDataFeedDesc,
py::call_guard<py::gil_scoped_release>())
.def("register_client2client_msg_handler",
&framework::Dataset::RegisterClientToClientMsgHandler)
.def("load_into_memory", &framework::Dataset::LoadIntoMemory)
.def("release_memory", &framework::Dataset::ReleaseMemory)
.def("local_shuffle", &framework::Dataset::LocalShuffle)
.def("global_shuffle", &framework::Dataset::GlobalShuffle)
.def("get_memory_data_size", &framework::Dataset::GetMemoryDataSize)
.def("get_shuffle_data_size", &framework::Dataset::GetShuffleDataSize);
&framework::Dataset::RegisterClientToClientMsgHandler,
py::call_guard<py::gil_scoped_release>())
.def("create_channel", &framework::Dataset::CreateChannel,
py::call_guard<py::gil_scoped_release>())
.def("create_readers", &framework::Dataset::CreateReaders,
py::call_guard<py::gil_scoped_release>())
.def("destroy_readers", &framework::Dataset::DestroyReaders,
py::call_guard<py::gil_scoped_release>())
.def("load_into_memory", &framework::Dataset::LoadIntoMemory,
py::call_guard<py::gil_scoped_release>())
.def("preload_into_memory", &framework::Dataset::PreLoadIntoMemory,
py::call_guard<py::gil_scoped_release>())
.def("wait_preload_done", &framework::Dataset::WaitPreLoadDone,
py::call_guard<py::gil_scoped_release>())
.def("release_memory", &framework::Dataset::ReleaseMemory,
py::call_guard<py::gil_scoped_release>())
.def("local_shuffle", &framework::Dataset::LocalShuffle,
py::call_guard<py::gil_scoped_release>())
.def("global_shuffle", &framework::Dataset::GlobalShuffle,
py::call_guard<py::gil_scoped_release>())
.def("get_memory_data_size", &framework::Dataset::GetMemoryDataSize,
py::call_guard<py::gil_scoped_release>())
.def("get_shuffle_data_size", &framework::Dataset::GetShuffleDataSize,
py::call_guard<py::gil_scoped_release>())
.def("set_queue_num", &framework::Dataset::SetChannelNum,
py::call_guard<py::gil_scoped_release>());
}
} // end namespace pybind

@ -71,6 +71,7 @@ class DatasetBase(object):
self.proto_desc.pipe_command = "cat"
self.dataset = core.Dataset("MultiSlotDataset")
self.thread_num = 0
self.filelist = []
def set_pipe_command(self, pipe_command):
"""
@ -139,6 +140,7 @@ class DatasetBase(object):
filelist(list): file list
"""
self.dataset.set_filelist(filelist)
self.filelist = filelist
def set_use_var(self, var_list):
"""
@ -193,7 +195,14 @@ class DatasetBase(object):
Set data_feed_desc before load or shuffle,
user no need to call this function.
"""
if self.thread_num > len(self.filelist):
self.thread_num = len(self.filelist)
self.dataset.set_thread_num(self.thread_num)
self.dataset.set_data_feed_desc(self.desc())
self.dataset.create_readers()
def _finish_to_run(self):
self.dataset.destroy_readers()
def desc(self):
"""
@ -226,6 +235,57 @@ class InMemoryDataset(DatasetBase):
""" Init. """
super(InMemoryDataset, self).__init__()
self.proto_desc.name = "MultiSlotInMemoryDataFeed"
self.fleet_send_batch_size = 80000
self.queue_num = None
def _prepare_to_run(self):
"""
Set data_feed_desc before load or shuffle,
user no need to call this function.
"""
if self.thread_num > len(self.filelist):
self.thread_num = len(self.filelist)
self.dataset.set_thread_num(self.thread_num)
if self.queue_num is None:
self.queue_num = self.thread_num
self.dataset.set_queue_num(self.queue_num)
self.dataset.set_data_feed_desc(self.desc())
self.dataset.create_channel()
self.dataset.create_readers()
def set_queue_num(self, queue_num):
"""
Set Dataset output queue num, training threads get data from queues
Args:
set_queue_num(int): dataset output queue num
Examples:
.. code-block:: python
import paddle.fluid as fluid
dataset = fluid.DatasetFactory().create_dataset("InMemoryDataset")
dataset.set_queue_num(12)
"""
self.queue_num = queue_num
def set_fleet_send_batch_size(self, fleet_send_batch_size):
"""
Set fleet send batch size, default is 80000
Args:
fleet_send_batch_size(int): fleet send batch size
Examples:
.. code-block:: python
import paddle.fluid as fluid
dataset = fluid.DatasetFactory().create_dataset("InMemoryDataset")
dataset.set_fleet_send_batch_size(800)
"""
self.fleet_send_batch_size = fleet_send_batch_size
def load_into_memory(self):
"""
@ -243,6 +303,39 @@ class InMemoryDataset(DatasetBase):
self._prepare_to_run()
self.dataset.load_into_memory()
def preload_into_memory(self):
"""
Load data into memory in async mode
Examples:
.. code-block:: python
import paddle.fluid as fluid
dataset = fluid.DatasetFactory().create_dataset("InMemoryDataset")
filelist = ["a.txt", "b.txt"]
dataset.set_filelist(filelist)
dataset.preload_into_memory()
dataset.wait_preload_done()
"""
self._prepare_to_run()
self.dataset.preload_into_memory()
def wait_preload_done(self):
"""
Wait preload_into_memory done
Examples:
.. code-block:: python
import paddle.fluid as fluid
dataset = fluid.DatasetFactory().create_dataset("InMemoryDataset")
filelist = ["a.txt", "b.txt"]
dataset.set_filelist(filelist)
dataset.preload_into_memory()
dataset.wait_preload_done()
"""
self.dataset.wait_preload_done()
def local_shuffle(self):
"""
Local shuffle
@ -282,13 +375,12 @@ class InMemoryDataset(DatasetBase):
"""
trainer_num = 1
fleet_send_batch_size = 80000
if fleet is not None:
fleet._role_maker._barrier_worker()
trainer_num = fleet.worker_num()
self.dataset.register_client2client_msg_handler()
self.dataset.set_trainer_num(trainer_num)
self.dataset.set_fleet_send_batch_size(fleet_send_batch_size)
self.dataset.set_fleet_send_batch_size(self.fleet_send_batch_size)
if fleet is not None:
fleet._role_maker._barrier_worker()
self.dataset.global_shuffle()

@ -889,6 +889,7 @@ class Executor(object):
if dataset == None:
raise RuntimeError("dataset is needed and should be initialized")
dataset._prepare_to_run()
scope, trainer = self._prepare_trainer(
program=program,
dataset=dataset,
@ -900,11 +901,11 @@ class Executor(object):
print_period=print_period)
trainer._set_infer(True)
trainer._gen_trainer_desc()
dataset._prepare_to_run()
self._dump_debug_info(program=program, trainer=trainer)
self._default_executor.run_from_dataset(program.desc, scope,
dataset.dataset,
trainer._desc())
dataset._finish_to_run()
return None
def train_from_dataset(self,
@ -969,6 +970,7 @@ class Executor(object):
if dataset == None:
raise RuntimeError("dataset is need and should be initialized")
dataset._prepare_to_run()
scope, trainer = self._prepare_trainer(
program=program,
dataset=dataset,
@ -979,9 +981,9 @@ class Executor(object):
fetch_info=fetch_info,
print_period=print_period)
trainer._gen_trainer_desc()
dataset._prepare_to_run()
self._dump_debug_info(program=program, trainer=trainer)
self._default_executor.run_from_dataset(program.desc, scope,
dataset.dataset,
trainer._desc())
dataset._finish_to_run()
return None

Loading…
Cancel
Save