Reader sequential and inference partial feed (#22699)

* sequential reader stage 1, test=develop

* fix ut, test=develop

* fix iterable=False reset bug, add some logs and polish code, test=develop

* inference feed partial data, test=develop

* Turn on keep_order=True for test, test=develop

* enhance ut to test more cases, test=develop

* test commit for reverting

* Revert "test commit for reverting", test=develop

This reverts commit 80aef42ef52ba1ee79627d6f663a624ec4f12f58.

* add ut of merged and unmerged results, test=develop

* add more uts for coverages and add en doc of api, test=develop

* follow comments, test=develop

* change note style, test=develop
revert-23830-2.0-beta
Zeng Jinle 5 years ago committed by GitHub
parent 95b356a069
commit acfc9b8a70
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23

@ -9,7 +9,7 @@ cc_library(computation_op_handle SRCS computation_op_handle.cc DEPS framework_pr
cc_library(share_tensor_buffer_op_handle SRCS share_tensor_buffer_op_handle.cc DEPS op_handle_base scope computation_op_handle share_tensor_buffer_functor)
cc_library(rpc_op_handle SRCS rpc_op_handle.cc DEPS framework_proto scope place operator op_registry)
cc_library(fetch_barrier_op_handle SRCS fetch_barrier_op_handle.cc DEPS framework_proto scope place operator op_registry)
cc_library(multi_devices_helper INTERFACE SRCS multi_devices_helper.cc DEPS graph graph_helper)
cc_library(multi_devices_helper SRCS multi_devices_helper.cc DEPS graph graph_helper)
cc_library(variable_visitor SRCS variable_visitor.cc DEPS lod_tensor selected_rows)
@ -65,13 +65,15 @@ cc_library(gather_op_handle SRCS gather_op_handle.cc DEPS op_handle_base scope d
cc_library(eager_deletion_op_handle SRCS eager_deletion_op_handle.cc DEPS lod_tensor selected_rows reference_count_pass_helper)
set(SSA_GRAPH_EXECUTOR_DEPS graph framework_proto
multi_devices_helper
sequential_execution_pass
modify_op_lock_and_record_event_pass
all_reduce_deps_pass
reference_count_pass
eager_deletion_pass
buffer_shared_inplace_op_pass
buffer_shared_cross_op_memory_reuse_pass)
buffer_shared_cross_op_memory_reuse_pass
set_reader_device_info_pass)
cc_library(ssa_graph_executor SRCS ssa_graph_executor.cc DEPS ${SSA_GRAPH_EXECUTOR_DEPS})
cc_library(threaded_ssa_graph_executor SRCS threaded_ssa_graph_executor.cc DEPS fetch_op_handle ssa_graph_executor scope

@ -66,6 +66,7 @@ class ParallelExecutorPassBuilder : public ir::PassBuilder {
AppendPrintGraphPass("graph_viz_pass", "_fused_graph");
AppendMultiDevPass();
AppendSetReaderDeviceIndexPass();
AppendMultiGraphOptPasses();
AppendPassToSetMkldnnAttr("mkldnn_placement_pass");
@ -227,6 +228,10 @@ class ParallelExecutorPassBuilder : public ir::PassBuilder {
&strategy_);
}
void AppendSetReaderDeviceIndexPass() {
AppendPass("set_reader_device_index_pass");
}
void AppendPrintGraphPass(const std::string &pass_name,
const std::string &debug_file_suffix) {
if (!strategy_.debug_graphviz_path_.empty()) {
@ -397,6 +402,9 @@ ir::Graph *BuildStrategy::Apply(ir::Graph *graph,
"GPU, skipped.";
continue;
}
} else if (pass->Type() == "set_reader_device_index_pass") {
pass->Erase(kPlaces);
pass->SetNotOwned<const std::vector<platform::Place>>(kPlaces, &places);
}
VLOG(1) << "Start Apply Pass " << pass->Type();
graph = pass->Apply(graph);
@ -433,6 +441,7 @@ USE_PASS(fuse_sgd_op_pass);
USE_PASS(fuse_momentum_op_pass);
USE_PASS(fuse_all_reduce_op_pass);
USE_PASS(runtime_context_cache_pass);
USE_PASS(set_reader_device_index_pass);
#ifdef PADDLE_WITH_MKLDNN
USE_PASS(mkldnn_placement_pass);
#endif

@ -34,6 +34,8 @@ class ComputationOpHandle : public OpHandleBase {
OperatorBase *GetOp() { return op_.get(); }
const OperatorBase *GetOp() const { return op_.get(); }
std::string Name() const override;
const Scope *GetScope() const { return scope_; }

@ -31,10 +31,12 @@ namespace framework {
namespace details {
EagerDeletionOpHandle::EagerDeletionOpHandle(
ir::Node *node, Scope *scope, const platform::Place &place,
ir::Node *node, Scope *scope, size_t scope_idx,
const platform::Place &place,
const std::unordered_set<ir::MemOptVarInfo *> &vars, GarbageCollector *gc)
: OpHandleBase(node),
scope_(scope),
scope_idx_(scope_idx),
place_(place),
var_infos_(vars.begin(), vars.end()),
gc_(gc) {

@ -34,7 +34,7 @@ namespace details {
class EagerDeletionOpHandle : public OpHandleBase {
public:
EagerDeletionOpHandle(ir::Node *node, Scope *scope,
EagerDeletionOpHandle(ir::Node *node, Scope *scope, size_t scope_idx,
const platform::Place &place,
const std::unordered_set<ir::MemOptVarInfo *> &vars,
GarbageCollector *gc);
@ -50,6 +50,8 @@ class EagerDeletionOpHandle : public OpHandleBase {
*/
Priority GetPriority() const override { return kHighest; }
size_t GetScopeIdx() const { return scope_idx_; }
protected:
void RunImpl() override;
@ -63,6 +65,7 @@ class EagerDeletionOpHandle : public OpHandleBase {
void CallOnce();
Scope *scope_;
size_t scope_idx_;
platform::Place place_;
std::vector<ir::MemOptVarInfo *> var_infos_; // not own
GarbageCollector *gc_; // not own

@ -12,9 +12,227 @@
// See the License for the specific language governing permissions and
// limitations under the License.
#include "paddle/fluid/framework/details/multi_devices_helper.h"
#include <algorithm>
#include <unordered_set>
#include "paddle/fluid/framework/details/computation_op_handle.h"
#include "paddle/fluid/framework/details/eager_deletion_op_handle.h"
#include "paddle/fluid/framework/details/share_tensor_buffer_op_handle.h"
#include "paddle/fluid/framework/ir/graph_helper.h"
namespace paddle {
namespace framework {
namespace details {} // namespace details
namespace details {
static constexpr size_t kUndefinedDevIdx = -1UL;
// NOTE(paddle-dev): the following ops are related to multi-device
// communication. If the graph contains any of the following ops,
// it cannot separate into multiple graphs on each device.
static std::unordered_set<std::string> kMultiDeviceOps{
"sync_batch_norm",
"sync_batch_norm_grad",
"allreduce",
"c_allreduce_sum",
"c_allreduce_prod",
"c_allreduce_min",
"c_allreduce_max",
"c_allgather",
"c_reducescatter",
"c_broadcast",
"c_comm_init",
"c_comm_init_all",
"c_gen_nccl_id",
"c_sync_comm_stream",
"send",
"recv",
"send_barrier",
"fetch_barrier",
};
static size_t GetScopeIdxFromOp(const details::OpHandleBase &op) {
if (auto *compute_op =
dynamic_cast<const details::ComputationOpHandle *>(&op)) {
return kMultiDeviceOps.count(compute_op->GetOp()->Type()) == 0
? compute_op->GetScopeIdx()
: kUndefinedDevIdx;
} else if (auto *gc_op =
dynamic_cast<const details::EagerDeletionOpHandle *>(&op)) {
return gc_op->GetScopeIdx();
} else if (auto *share_op =
dynamic_cast<const details::ShareTensorBufferOpHandle *>(
&op)) {
return share_op->GetScopeIdx();
} else {
return kUndefinedDevIdx;
}
}
static bool ContainMultiDeviceOp(const ProgramDesc &program,
size_t begin_block_idx) {
for (size_t block_idx = begin_block_idx; block_idx < program.Size();
++block_idx) {
for (auto *op_desc : program.Block(block_idx).AllOps()) {
if (kMultiDeviceOps.count(op_desc->Type()) > 0) {
return true;
}
}
}
return false;
}
static size_t GetUniqueDeviceIdOfOp(const details::OpHandleBase &op) {
size_t dev_idx = GetScopeIdxFromOp(op);
if (dev_idx == kUndefinedDevIdx) {
return kUndefinedDevIdx;
}
const auto &ins = op.Inputs();
const auto &outs = op.Outputs();
auto in_outs = ins;
in_outs.insert(in_outs.end(), outs.begin(), outs.end());
for (auto *var : in_outs) {
auto *var_handle = dynamic_cast<details::VarHandle *>(var);
if (var_handle == nullptr) {
continue;
}
if (dev_idx != var_handle->scope_idx()) {
return kUndefinedDevIdx;
}
}
return dev_idx;
}
/**
* This function tries to separate the original graph into multiple graphs, in
* which each graph would only run on single device. This is usually used to
* separate a data-parallel inference graph to multiple graphs on each device.
*
* The graph can be separated into multiple single device graphs if and only if:
*
* - the graph does not contain any ops related to multi-devices communication,
* such as allreduce, send, recv, sync_batch_norm, etc.
*
* - ops on different devices do not depend on each other. That is to say, the
* graph has several disconnected sub-graphs.
*/
std::vector<std::unique_ptr<ir::Graph>> TrySeparateToMultipleSingleDeviceGraphs(
ir::Graph *graph) {
// If sub-block contains multi-devices ops, we cannot separate
if (ContainMultiDeviceOp(graph->OriginProgram(), 1)) {
return {};
}
size_t place_num = 0;
auto op_handles = ir::FilterByNodeWrapper<OpHandleBase>(*graph);
if (op_handles.empty()) {
return {};
}
std::unordered_map<details::OpHandleBase *, size_t> op_to_dev_idx;
for (auto &op : op_handles) {
auto dev_idx = GetUniqueDeviceIdOfOp(*op);
if (dev_idx == kUndefinedDevIdx) {
VLOG(10) << "Op " << op->Name() << " is not determined";
return {};
}
place_num = std::max(place_num, dev_idx + 1);
op_to_dev_idx[op] = dev_idx;
}
for (auto &op : op_handles) {
auto dev_idx = op_to_dev_idx.at(op);
for (auto &in_var : op->Inputs()) {
if (in_var->GeneratedOp()) {
auto iter = op_to_dev_idx.find(in_var->GeneratedOp());
if (iter == op_to_dev_idx.end() || iter->second != dev_idx) {
return {};
}
}
}
for (auto &out_var : op->Outputs()) {
for (auto &pending_op : out_var->PendingOps()) {
auto iter = op_to_dev_idx.find(pending_op);
if (iter == op_to_dev_idx.end() || iter->second != dev_idx) {
return {};
}
}
}
}
PADDLE_ENFORCE_GE(
place_num, 1,
platform::errors::NotFound(
"No place found, this may be a bug.\nIt would be helpful if you "
"could inform us of how this conversion went by opening a github "
"issue at https://github.com/PaddlePaddle/Paddle/issues/new. And "
"we will resolve it with high priority."));
std::vector<std::unique_ptr<ir::Graph>> graphs(place_num);
for (auto &g : graphs) {
g.reset(new ir::Graph(ProgramDesc()));
g->Set(kGraphVars, new GraphVars(1UL));
g->Set(kGraphDepVars, new GraphDepVars());
}
for (auto &op : op_handles) {
auto dev_idx = op_to_dev_idx.at(op);
auto *ret_graph = graphs[dev_idx].get();
auto &ret_vars = ret_graph->Get<GraphVars>(kGraphVars)[0];
auto &ret_dummy_vars = ret_graph->Get<GraphDepVars>(kGraphDepVars);
auto &origin_vars = graph->Get<GraphVars>(kGraphVars)[dev_idx];
ret_graph->AddNode(graph->RemoveNode(op->Node()).release());
auto handler = [&](const std::vector<VarHandleBase *> &vars) {
for (auto *var : vars) {
if (graph->Nodes().count(var->Node()) > 0) {
ret_graph->AddNode(graph->RemoveNode(var->Node()).release());
auto *dummy_var = dynamic_cast<DummyVarHandle *>(var);
if (dummy_var == nullptr) {
ret_vars.emplace(var->Name(), origin_vars.at(var->Name()));
} else {
ret_dummy_vars.emplace(dummy_var);
}
}
}
};
handler(op->Inputs());
handler(op->Outputs());
}
graph->Erase(kGraphVars);
graph->Erase(kGraphDepVars);
return graphs;
}
static bool HasDropLastReadOpImpl(const ir::Graph &graph, bool drop_last) {
auto ops = ir::FilterByNodeWrapper<OpHandleBase>(graph);
for (auto *op : ops) {
auto *compute_op = dynamic_cast<ComputationOpHandle *>(op);
if (compute_op && compute_op->GetOp()->Type() == "read" &&
compute_op->GetOp()->Attr<bool>("drop_last") == drop_last) {
VLOG(10) << "The graph has drop_last=" << drop_last << " read op";
return true;
}
}
VLOG(10) << "The graph does not have drop_last=" << drop_last << " read op";
return false;
}
bool HasDropLastReadOp(const ir::Graph &graph) {
return HasDropLastReadOpImpl(graph, true);
}
bool HasKeepLastReadOp(const ir::Graph &graph) {
return HasDropLastReadOpImpl(graph, false);
}
} // namespace details
} // namespace framework
} // namespace paddle

@ -47,6 +47,7 @@ constexpr char kGraphVars[] = "vars";
constexpr char kNRanks[] = "nranks";
constexpr char kPlaces[] = "places";
constexpr char kGlobalScope[] = "global_scope";
constexpr char kLocalScopes[] = "local_scopes";
constexpr char kNCCLCtxs[] = "nccl_ctxs";
constexpr char kUseHierarchicalAllReduce[] = "use_hierarchical_allreduce";
@ -100,6 +101,13 @@ inline std::vector<std::string> GetOpRoleVarsOrEmpty(const OpDesc &op) {
return boost::get<std::vector<std::string>>(iter->second);
}
std::vector<std::unique_ptr<ir::Graph>> TrySeparateToMultipleSingleDeviceGraphs(
ir::Graph *graph);
bool HasDropLastReadOp(const ir::Graph &graph);
bool HasKeepLastReadOp(const ir::Graph &graph);
} // namespace details
} // namespace framework
} // namespace paddle

@ -13,6 +13,7 @@
// limitations under the License.
#include "paddle/fluid/framework/details/parallel_ssa_graph_executor.h"
#include <algorithm>
#include <memory>
#include <utility>
#include "paddle/fluid/framework/ir/graph_helper.h"
@ -21,11 +22,11 @@ namespace paddle {
namespace framework {
namespace details {
std::vector<std::unique_ptr<ir::Graph>>
ParallelSSAGraphExecutor::SeparateMultiDevicesGraph(ir::Graph *graph) {
static std::vector<std::unique_ptr<ir::Graph>> SeparateMultiDevicesGraph(
ir::Graph *graph, size_t place_num) {
std::vector<std::unique_ptr<ir::Graph>> graphs;
graphs.reserve(places_.size());
for (size_t i = 0; i < places_.size(); ++i) {
graphs.reserve(place_num);
for (size_t i = 0; i < place_num; ++i) {
ProgramDesc empty;
graphs.emplace_back(std::unique_ptr<ir::Graph>(new ir::Graph(empty)));
auto &g = graphs.back();
@ -64,7 +65,7 @@ ParallelSSAGraphExecutor::SeparateMultiDevicesGraph(ir::Graph *graph) {
}
}
for (size_t dev_id = 0; dev_id < places_.size(); ++dev_id) {
for (size_t dev_id = 0; dev_id < place_num; ++dev_id) {
auto &dev_vars = graphs[dev_id]->Get<GraphVars>(kGraphVars)[0];
auto &origin_vars = graph->Get<GraphVars>(kGraphVars)[dev_id];
for (auto &name_pair : origin_vars) {
@ -85,15 +86,34 @@ ParallelSSAGraphExecutor::ParallelSSAGraphExecutor(
const ExecutionStrategy &strategy, const std::vector<Scope *> &local_scopes,
const std::vector<Scope *> &local_exec_scopes,
const std::vector<platform::Place> &places, ir::Graph *graph)
// TODO(Yancey1989): Copying graphs is not safely since it deleted the
// attrs.
: ParallelSSAGraphExecutor(strategy, local_scopes, local_exec_scopes,
places,
SeparateMultiDevicesGraph(graph,
places.size())) {}
ParallelSSAGraphExecutor::ParallelSSAGraphExecutor(
const ExecutionStrategy &strategy, const std::vector<Scope *> &local_scopes,
const std::vector<Scope *> &local_exec_scopes,
const std::vector<platform::Place> &places,
std::vector<std::unique_ptr<ir::Graph>> graphs)
: strategy_(std::move(strategy)),
local_scopes_(std::move(local_scopes)),
pool_(places.size() >= 2 ? new ::ThreadPool(places.size()) : nullptr),
places_(std::move(places)),
// TODO(Yancey1989): Copying graphs is not safely since it deleted the
// attrs.
graphs_(SeparateMultiDevicesGraph(graph)) {
places_(places),
graphs_(std::move(graphs)),
feed_status_(places.size(), FeedStatus::kNone) {
PADDLE_ENFORCE_EQ(places_.size(), local_scopes_.size());
PADDLE_ENFORCE_EQ(places_.size(), graphs_.size(),
platform::errors::InvalidArgument(
"Graph number does not match place number"));
PADDLE_ENFORCE_GT(
places_.size(), 0,
platform::errors::InvalidArgument("place number must be larger than 0"));
auto seq_allreduce_pass =
ir::PassRegistry::Instance().Get("all_reduce_deps_pass");
seq_allreduce_pass->Set<bool>(kUseHierarchicalAllReduce, new bool(false));
@ -123,28 +143,41 @@ std::vector<ir::Graph *> ParallelSSAGraphExecutor::Graphs() {
return result;
}
enum ExceptionStatus { kSuccess = 0, kEOF, kOther };
FetchResultType ParallelSSAGraphExecutor::Run(
const std::vector<std::string> &fetch_tensors, bool return_merged) {
size_t feed_num = std::count(feed_status_.begin(), feed_status_.end(),
FeedStatus::kHasFeed);
bool has_feed = (feed_num > 0);
VLOG(10) << "Feed num " << feed_num;
size_t place_num = places_.size();
std::vector<std::future<FetchResultType>> run_futures;
std::vector<ExceptionStatus> exception_status(place_num,
ExceptionStatus::kSuccess);
std::vector<FetchResultType> fetch_data;
FetchResultType ret;
fetch_data.reserve(places_.size());
if (return_merged) {
ret = FeedFetchList();
} else {
ret = FetchUnmergedList();
}
fetch_data.reserve(place_num);
exception_holder_.Clear();
for (size_t i = 0; i < places_.size(); ++i) {
auto call = [this, i, return_merged, &fetch_tensors]() -> FetchResultType {
for (size_t i = 0; i < place_num; ++i) {
auto call = [&, i]() -> FetchResultType {
try {
return executors_[i]->Run(fetch_tensors, return_merged);
if (!support_partial_feed_ || !has_feed ||
feed_status_[i] == FeedStatus::kHasFeed) {
return executors_[i]->Run(fetch_tensors, return_merged);
}
} catch (platform::EOFException &) {
exception_status[i] = ExceptionStatus::kEOF;
exception_holder_.Catch(std::current_exception());
} catch (...) {
exception_status[i] = ExceptionStatus::kOther;
exception_holder_.Catch(std::current_exception());
}
if (return_merged) {
return FeedFetchList();
} else {
@ -161,46 +194,96 @@ FetchResultType ParallelSSAGraphExecutor::Run(
if (pool_) {
for (auto &f : run_futures) {
if (exception_holder_.IsCaught()) {
f.wait();
} else {
fetch_data.emplace_back(f.get());
fetch_data.emplace_back(f.get());
}
}
bool has_exception = exception_holder_.IsCaught();
if (!support_partial_feed_ && has_exception) {
VLOG(10) << "Exception rethrow because partial feed is not supported";
exception_holder_.ReThrow();
}
std::vector<bool> is_valid(place_num, true);
if (support_partial_feed_) {
if (has_feed) {
for (size_t i = 0; i < place_num; ++i) {
if (feed_status_[i] == FeedStatus::kNone) {
is_valid[i] = false;
} else if (exception_status[i] != ExceptionStatus::kSuccess) {
PADDLE_ENFORCE_EQ(has_exception, true,
platform::errors::InvalidArgument(
"Thread pool raises exception but not caught"));
VLOG(10) << "Exception rethrow because non-EOF exception raises when "
"feed is given";
exception_holder_.ReThrow();
}
}
} else {
for (size_t i = 0; i < place_num; ++i) {
if (exception_status[i] == ExceptionStatus::kOther) {
PADDLE_ENFORCE_EQ(has_exception, true,
platform::errors::InvalidArgument(
"Thread pool raises exception but not caught"));
VLOG(10) << "Exception rethrow because non-EOF exception raises when "
"feed is not given";
exception_holder_.ReThrow();
} else if (exception_status[i] != ExceptionStatus::kSuccess) {
is_valid[i] = false;
}
}
}
}
if (exception_holder_.IsCaught()) {
if (std::count(is_valid.begin(), is_valid.end(), true) == 0) {
PADDLE_ENFORCE_EQ(has_exception, true,
platform::errors::InvalidArgument(
"Thread pool raises exception but not caught"));
VLOG(10) << "Raise exception because there is no success worker";
exception_holder_.ReThrow();
}
if (return_merged) {
auto &ret_val = boost::get<FeedFetchList>(ret);
FeedFetchList ret;
ret.reserve(fetch_tensors.size());
for (size_t fetch_idx = 0; fetch_idx < fetch_tensors.size(); ++fetch_idx) {
std::vector<const LoDTensor *> lodtensor_ptrs;
lodtensor_ptrs.reserve(local_scopes_.size());
for (size_t scope_idx = 0; scope_idx < local_scopes_.size();
++scope_idx) {
auto &val = boost::get<FeedFetchList>(fetch_data.at(scope_idx));
lodtensor_ptrs.push_back(&val.at(fetch_idx));
lodtensor_ptrs.reserve(place_num);
for (size_t scope_idx = 0; scope_idx < place_num; ++scope_idx) {
if (!is_valid[scope_idx]) {
continue;
}
const auto &fetch_list =
boost::get<FeedFetchList>(fetch_data[scope_idx]);
lodtensor_ptrs.push_back(&fetch_list[fetch_idx]);
}
ret_val.emplace_back();
ret_val.back().MergeLoDTensor(lodtensor_ptrs, platform::CPUPlace());
ret.emplace_back();
ret.back().MergeLoDTensor(lodtensor_ptrs, platform::CPUPlace());
}
return ret;
} else {
auto &ret_val = boost::get<FetchUnmergedList>(ret);
FetchUnmergedList ret;
ret.reserve(fetch_tensors.size());
for (size_t fetch_idx = 0; fetch_idx < fetch_tensors.size(); ++fetch_idx) {
ret_val.emplace_back();
ret.emplace_back();
for (size_t scope_idx = 0; scope_idx < local_scopes_.size();
++scope_idx) {
auto &val = boost::get<FetchUnmergedList>(fetch_data.at(scope_idx));
if (!is_valid[scope_idx]) {
continue;
}
const auto &fetch_list =
boost::get<FetchUnmergedList>(fetch_data[scope_idx]);
PADDLE_ENFORCE_EQ(
val.at(fetch_idx).size(), 1,
fetch_list[fetch_idx].size(), 1,
platform::errors::Fatal(
"Each place must have only one fetched LoDTensor!"));
ret_val.back().emplace_back(val.at(fetch_idx)[0]);
ret.back().emplace_back(fetch_list[fetch_idx][0]);
}
}
return ret;
}
return ret;
}
} // namespace details

@ -27,12 +27,25 @@ namespace framework {
namespace details {
class ParallelSSAGraphExecutor : public SSAGraphExecutor {
public:
enum FeedStatus {
kNone = 0, // No feed
kHasFeed = 1 // Has feed
};
public:
ParallelSSAGraphExecutor(const ExecutionStrategy &strategy,
const std::vector<Scope *> &local_scopes,
const std::vector<Scope *> &local_exec_scopes,
const std::vector<platform::Place> &places,
ir::Graph *graph);
ParallelSSAGraphExecutor(const ExecutionStrategy &strategy,
const std::vector<Scope *> &local_scopes,
const std::vector<Scope *> &local_exec_scopes,
const std::vector<platform::Place> &places,
std::vector<std::unique_ptr<ir::Graph>> graphs);
~ParallelSSAGraphExecutor() final = default;
const ir::Graph &Graph() const override { return *graphs_[0]; }
@ -42,10 +55,15 @@ class ParallelSSAGraphExecutor : public SSAGraphExecutor {
FetchResultType Run(const std::vector<std::string> &fetch_tensors,
bool return_merged) override;
private:
std::vector<std::unique_ptr<ir::Graph>> SeparateMultiDevicesGraph(
ir::Graph *graph);
void SetHasFeed(size_t dev_idx, bool has_feed) {
feed_status_[dev_idx] = has_feed ? FeedStatus::kHasFeed : FeedStatus::kNone;
}
void EnablePartialFeedSupport() { support_partial_feed_ = true; }
bool SupportPartialFeed() const { return support_partial_feed_; }
private:
ExecutionStrategy strategy_;
std::vector<Scope *> local_scopes_;
std::unique_ptr<::ThreadPool> pool_{nullptr};
@ -55,6 +73,9 @@ class ParallelSSAGraphExecutor : public SSAGraphExecutor {
std::vector<std::unique_ptr<details::FastThreadedSSAGraphExecutor>>
executors_;
ExceptionHolder exception_holder_;
bool support_partial_feed_{false};
std::vector<FeedStatus> feed_status_;
};
} // namespace details

@ -228,7 +228,7 @@ void EagerDeletionPass::ApplyImpl(ir::Graph *graph) const {
}
auto *eager_deletion_op = new details::EagerDeletionOpHandle(
eager_deletion_node, op->GetScope(), op->GetPlace(),
eager_deletion_node, op->GetScope(), op->GetScopeIdx(), op->GetPlace(),
std::move(var_info), gcs.at(places[op->GetScopeIdx()]).get());
auto it = std::find_if(

@ -98,7 +98,7 @@ class ReferenceCountPassTestHelper {
ir::PassRegistry::Instance().Get("reference_count_pass");
ref_cnt_pass->SetNotOwned(ir::kMemOptVarInfoMapList, &mem_opt_var_infos_);
ref_cnt_pass->SetNotOwned(ir::kLastLiveOpsOfVars, &last_live_ops_of_vars_);
ref_cnt_pass->Apply(&graph_);
ref_cnt_pass->Apply(&const_cast<ir::Graph &>(executor_->Graph()));
}
bool IsLastLivedOps(const std::string &name,

@ -11,6 +11,7 @@ endif()
cc_library(multi_devices_graph_pass SRCS multi_devices_graph_pass.cc DEPS multi_devices_helper computation_op_handle
scale_loss_grad_op_handle rpc_op_handle fetch_barrier_op_handle ${ALL_REDUCE_OP_HANDLES} reduce_op_handle broadcast_op_handle fused_broadcast_op_handle)
cc_library(sequential_execution_pass SRCS sequential_execution_pass.cc DEPS graph graph_helper pass)
cc_library(set_reader_device_info_pass SRCS set_reader_device_info_pass.cc DEPS graph graph_helper pass multi_devices_graph_pass)
cc_library(fuse_all_reduce_op_pass SRCS fuse_all_reduce_op_pass.cc DEPS graph graph_helper fused_all_reduce_op_handle)
cc_library(all_reduce_deps_pass SRCS all_reduce_deps_pass.cc DEPS all_reduce_op_handle graph graph_helper pass)

@ -0,0 +1,101 @@
// Copyright (c) 2020 PaddlePaddle Authors. All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include "paddle/fluid/framework/details/computation_op_handle.h"
#include "paddle/fluid/framework/details/multi_devices_helper.h"
#include "paddle/fluid/framework/ir/graph.h"
#include "paddle/fluid/framework/ir/pass.h"
#include "paddle/fluid/operators/reader/lod_tensor_blocking_queue.h"
namespace paddle {
namespace framework {
namespace ir {
static int GetDeviceCountFromPassAttr(const Pass &pass) {
return static_cast<int>(
pass.Get<const std::vector<platform::Place>>(details::kPlaces).size());
}
static std::unordered_set<std::string> ReaderOpSet() {
return {"create_py_reader"};
}
class InitReaderDeviceCountPass : public Pass {
protected:
void ApplyImpl(Graph *graph) const override {
using QueueHolder =
operators::reader::OrderedMultiDeviceLoDTensorBlockingQueueHolder;
auto reader_ops = ReaderOpSet();
auto dev_cnt = GetDeviceCountFromPassAttr(*this);
const auto &scope = Get<const Scope>(details::kGlobalScope);
for (auto &node : graph->Nodes()) {
if (node->IsOp() && node->Op() &&
reader_ops.count(node->Op()->Type()) != 0) {
auto queue_name = node->Op()->Input("blocking_queue")[0];
auto var = scope.FindVar(queue_name);
if (var && var->IsType<QueueHolder>()) {
VLOG(10) << "Set device count of " << queue_name << " to be "
<< dev_cnt;
var->GetMutable<QueueHolder>()->GetQueue()->SetDeviceCount(dev_cnt);
}
}
}
}
};
class SetReaderDeviceIndexPass : public Pass {
protected:
void ApplyImpl(Graph *graph) const override {
auto dev_cnt = GetDeviceCountFromPassAttr(*this);
auto reader_ops = ReaderOpSet();
size_t found_op_num = 0;
for (auto &node : graph->Nodes()) {
if (node->IsOp() && node->Op() &&
reader_ops.count(node->Op()->Type()) != 0) {
auto &op_handle = dynamic_cast<details::ComputationOpHandle &>(
node->Wrapper<details::OpHandleBase>());
auto *op_desc = node->Op();
auto &op_base_attrs =
const_cast<framework::AttributeMap &>(op_handle.GetOp()->Attrs());
int dev_idx = static_cast<int>(op_handle.GetScopeIdx());
op_desc->SetAttr("device_index", dev_idx);
op_desc->SetAttr("device_count", dev_cnt);
op_base_attrs["device_index"] = dev_idx;
op_base_attrs["device_count"] = dev_cnt;
++found_op_num;
VLOG(10) << "Found op " << op_desc->Type() << " on device " << dev_idx;
}
}
VLOG(10) << "Found op number " << found_op_num;
}
};
} // namespace ir
} // namespace framework
} // namespace paddle
REGISTER_PASS(init_reader_device_count_pass,
paddle::framework::ir::InitReaderDeviceCountPass)
.RequirePassAttr(paddle::framework::details::kGlobalScope)
.RequirePassAttr(paddle::framework::details::kPlaces);
REGISTER_PASS(set_reader_device_index_pass,
paddle::framework::ir::SetReaderDeviceIndexPass)
.RequirePassAttr(paddle::framework::details::kPlaces);

@ -307,18 +307,18 @@ void DeserializeFromStream(std::istream &is, LoDTensor *tensor,
std::vector<LoDTensor> LoDTensor::SplitLoDTensor(
const std::vector<platform::Place> places) const {
PADDLE_ENFORCE_GT(places.size(), 0,
platform::errors::InvalidArgument(
"place number cannot be empty when splitting"));
check_memory_size();
int batch_size =
lod().empty() ? dims()[0] : static_cast<int>(lod()[0].size()) - 1;
size_t result_size = std::min(static_cast<size_t>(batch_size), places.size());
size_t remainder = batch_size % places.size();
size_t batch_size =
lod().empty() ? static_cast<size_t>(dims()[0]) : lod()[0].size() - 1;
std::vector<LoDTensor> results;
results.reserve(result_size);
// if result_size(batch_size) is 0, just return #places.size() copys of empty
// if batch_size is 0, just return #places.size() copys of empty
// tensors.
if (result_size == 0) {
if (batch_size == 0) {
std::vector<LoDTensor> empty_results;
empty_results.reserve(places.size());
for (size_t i = 0; i < places.size(); ++i) {
LoDTensor dst;
dst.Resize(dims());
@ -326,18 +326,22 @@ std::vector<LoDTensor> LoDTensor::SplitLoDTensor(
if (!lod().empty()) {
dst.set_lod(lod());
}
results.emplace_back(dst);
empty_results.emplace_back(std::move(dst));
}
return results;
return empty_results;
}
int step_width = static_cast<int>(batch_size / result_size);
auto step_width = (batch_size + places.size() - 1) / places.size();
auto result_size = (batch_size + step_width - 1) / step_width;
std::vector<LoDTensor> results;
results.reserve(result_size);
for (size_t i = 0; i < result_size; ++i) {
int begin = static_cast<int>(i * step_width);
int end = static_cast<int>((i + 1) * step_width);
if (i + 1 == places.size()) { // last
end += remainder;
}
auto begin = i * step_width;
auto end = std::min<size_t>((i + 1) * step_width, batch_size);
PADDLE_ENFORCE_LT(begin, end,
platform::errors::InvalidArgument(
"begin must be less than end, this may be a bug"));
LoDTensor dst;
if (lod().empty()) {
@ -362,7 +366,7 @@ std::vector<LoDTensor> LoDTensor::SplitLoDTensor(
}
dst.set_lod(my_lod);
}
results.emplace_back(dst);
results.emplace_back(std::move(dst));
}
return results;

File diff suppressed because it is too large Load Diff

@ -80,6 +80,8 @@ class ParallelExecutor {
FetchResultType Run(const std::vector<std::string> &fetch_tensors,
bool return_merged = true);
const ir::Graph &Graph() const;
private:
// broadcast the parameters from the 0th device.
// trainer_id the trainer index in nccl distributed training.

@ -117,6 +117,10 @@ class DecoratedReader : public ReaderBase,
~DecoratedReader();
const std::shared_ptr<ReaderBase>& UnderlyingReader() const {
return reader_;
}
protected:
void ShutdownImpl() override {
VLOG(1) << "ShutdownImpl";
@ -190,6 +194,8 @@ class ReaderHolder {
return reader_->NeedCheckFeed();
}
void Clear() { reader_.reset(); }
operator const std::shared_ptr<ReaderBase>&() const { return this->reader_; }
private:

@ -56,6 +56,7 @@ class CudnnRNNCache;
namespace reader {
class LoDTensorBlockingQueueHolder;
class OrderedMultiDeviceLoDTensorBlockingQueueHolder;
} // namespace reader
} // namespace operators
@ -139,6 +140,7 @@ using VarTypeRegistry = detail::VarTypeRegistryImpl<
Tensor, LoDTensor, SelectedRows, std::vector<Scope *>, LoDRankTable,
LoDTensorArray, platform::PlaceList, ReaderHolder, std::string, Scope *,
operators::reader::LoDTensorBlockingQueueHolder,
operators::reader::OrderedMultiDeviceLoDTensorBlockingQueueHolder,
#ifdef PADDLE_WITH_CUDA
#if defined(PADDLE_WITH_NCCL)
ncclUniqueId, platform::Communicator, platform::NCCLCommunicator,

@ -27,12 +27,20 @@ class CreateDoubleBufferReaderOp : public framework::OperatorBase {
const platform::Place& dev_place) const override {
auto* out = scope.FindVar(Output("Out"))
->template GetMutable<framework::ReaderHolder>();
if (out->Get() != nullptr) {
return;
}
const auto& underlying_reader = scope.FindVar(Input("UnderlyingReader"))
->Get<framework::ReaderHolder>();
if (out->Get() != nullptr) {
auto* decorated_reader =
dynamic_cast<framework::DecoratedReader*>(out->Get().get());
PADDLE_ENFORCE_NOT_NULL(
decorated_reader,
platform::errors::NotFound("Not inited with DecoratedReader"));
if (decorated_reader->UnderlyingReader() == underlying_reader.Get()) {
return;
}
}
auto place_str = Attr<std::string>("place");
platform::Place place;
if (place_str == "AUTO") {
@ -47,6 +55,8 @@ class CreateDoubleBufferReaderOp : public framework::OperatorBase {
place = platform::CUDAPlace(static_cast<int>(num));
}
VLOG(10) << "Create new double buffer reader on " << place;
out->Reset(framework::MakeDecoratedReader<BufferedReader>(underlying_reader,
place, 2));
}

@ -38,8 +38,21 @@ class CreatePyReaderOp : public framework::OperatorBase {
queue_holder_var,
"No LoDTensorBlockingQueueHolder variable with name %s found",
queue_name);
auto* queue_holder =
queue_holder_var->template GetMutable<LoDTensorBlockingQueueHolder>();
std::shared_ptr<LoDTensorBlockingQueue> queue;
std::shared_ptr<OrderedMultiDeviceLoDTensorBlockingQueue> ordered_queue;
int dev_idx = -1;
if (queue_holder_var->IsType<LoDTensorBlockingQueueHolder>()) {
queue = queue_holder_var->Get<LoDTensorBlockingQueueHolder>().GetQueue();
} else if (queue_holder_var
->IsType<OrderedMultiDeviceLoDTensorBlockingQueueHolder>()) {
auto* queue_holder =
queue_holder_var
->GetMutable<OrderedMultiDeviceLoDTensorBlockingQueueHolder>();
dev_idx = Attr<int>("device_index");
ordered_queue = queue_holder->GetQueue();
ordered_queue->SetDeviceCount(Attr<int>("device_count"));
queue = ordered_queue->GetQueue(dev_idx);
}
/* Coverting shape_concat and ranks into DDim of each data.
shape_concat and ranks are shapes and shape ranks of each data.E.g.
@ -71,8 +84,12 @@ class CreatePyReaderOp : public framework::OperatorBase {
for (size_t i = 0; i < need_check_feed_int.size(); ++i) {
need_check_feed.push_back(static_cast<bool>(need_check_feed_int[i]));
}
out->Reset(std::make_shared<PyReader>(queue_holder->GetQueue(), dims,
var_types, need_check_feed));
auto py_reader =
std::make_shared<PyReader>(queue, dims, var_types, need_check_feed);
if (ordered_queue) {
ordered_queue->SetResetMethod(dev_idx, [out] { out->Clear(); });
}
out->Reset(py_reader);
}
};
@ -82,6 +99,13 @@ class CreatePyReaderOpMaker : public FileReaderMakerBase {
AddInput("blocking_queue",
"Name of the `LoDTensorBlockingQueueHolder` variable");
AddAttr<int>("device_index", "The device index this reader offers data")
.SetDefault(0);
AddAttr<int>("device_count",
"The total device number this reader offers data")
.SetDefault(1);
AddComment(R"DOC(
Create PyReader to support LoDTensor data feeding in Python side.
)DOC");

@ -27,16 +27,13 @@ namespace paddle {
namespace operators {
namespace reader {
class LoDTensorBlockingQueueHolder;
class LoDTensorBlockingQueue {
friend class LoDTensorBlockingQueueHolder;
private:
public:
explicit LoDTensorBlockingQueue(size_t capacity, bool speed_test_mode = false)
: queue_(capacity, speed_test_mode) {}
public:
~LoDTensorBlockingQueue() { VLOG(10) << "Destruct LoDTensorBlockingQueue"; }
bool Push(const std::vector<framework::LoDTensor>& lod_tensor_vec) {
return queue_.Send(lod_tensor_vec);
}
@ -67,10 +64,140 @@ class LoDTensorBlockingQueue {
inline void Kill() { queue_.Kill(); }
inline bool WaitForInited(size_t) { return true; }
private:
BlockingQueue<std::vector<framework::LoDTensor>> queue_;
};
class OrderedMultiDeviceLoDTensorBlockingQueue {
public:
OrderedMultiDeviceLoDTensorBlockingQueue(size_t capacity,
bool speed_test_mode = false)
: capacity_(capacity), speed_test_mode_(speed_test_mode) {}
~OrderedMultiDeviceLoDTensorBlockingQueue() {
VLOG(10) << "Destruct OrderedMultiDeviceLoDTensorBlockingQueue";
}
bool WaitForInited(size_t milliseconds) {
std::unique_lock<std::mutex> lock(init_mutex_);
return cv_.wait_for(lock, std::chrono::milliseconds(milliseconds),
[this] { return !queues_.empty(); });
}
void SetDeviceCount(size_t dev_cnt) {
{
std::lock_guard<std::mutex> lock(init_mutex_);
PADDLE_ENFORCE_GE(dev_cnt, 1,
platform::errors::InvalidArgument(
"Device count to init "
"OrderedMultiDeviceLoDTensorBlockingQueue"
" must be larger than 1"));
if (!queues_.empty()) {
PADDLE_ENFORCE_EQ(queues_.size(), dev_cnt,
platform::errors::InvalidArgument(
"queues should be only inited once"));
return;
}
VLOG(1) << "Init queue with size " << dev_cnt;
queues_.resize(dev_cnt);
for (auto& item : queues_) {
auto cap = (capacity_ + dev_cnt - 1) / dev_cnt;
item.reset(new LoDTensorBlockingQueue(cap, speed_test_mode_));
}
}
cv_.notify_all();
}
const std::shared_ptr<LoDTensorBlockingQueue>& GetQueue(size_t idx) const {
EnforceIsInited();
PADDLE_ENFORCE_LT(
idx, queues_.size(),
platform::errors::OutOfRange("The queue index is out of range"));
return queues_[idx];
}
bool Push(const std::vector<framework::LoDTensor>& lod_tensor_vec) {
return CurQueue()->Push(lod_tensor_vec);
}
inline size_t Size() const {
size_t size = 0;
for (auto& item : queues_) {
size += item->Size();
}
return size;
}
inline void Close() {
for (auto& item : queues_) {
item->Close();
}
}
inline void Kill() {
for (auto& item : queues_) {
item->Kill();
}
}
inline void Reset() {
{
std::lock_guard<std::mutex> reset_lock(reset_mutex_);
for (auto& method : reset_methods_) {
if (method) method();
}
}
auto dev_cnt = queues_.size();
for (auto& item : queues_) {
auto cap = (capacity_ + dev_cnt - 1) / dev_cnt;
item.reset(new LoDTensorBlockingQueue(cap, speed_test_mode_));
}
data_index_ = 0;
}
inline void SetResetMethod(size_t idx,
const std::function<void()>& reset_method) {
std::lock_guard<std::mutex> reset_lock(reset_mutex_);
EnforceIsInited();
if (reset_methods_.size() <= idx) {
reset_methods_.resize(idx + 1);
}
reset_methods_[idx] = reset_method;
}
inline size_t Cap() const { return capacity_; }
private:
const std::shared_ptr<LoDTensorBlockingQueue>& CurQueue() {
return queues_[(data_index_++) % queues_.size()];
}
private:
void EnforceIsInited() const {
PADDLE_ENFORCE_EQ(queues_.empty(), false,
platform::errors::NotFound("queue has not been inited"));
}
private:
std::vector<std::shared_ptr<LoDTensorBlockingQueue>> queues_;
mutable uint64_t data_index_{0};
size_t dev_cnt_{0};
const size_t capacity_;
const bool speed_test_mode_;
bool is_closed_{false};
std::vector<std::function<void()>> reset_methods_;
mutable std::mutex reset_mutex_;
mutable std::mutex init_mutex_;
mutable std::condition_variable cv_;
};
class LoDTensorBlockingQueueHolder {
public:
void InitOnce(size_t capacity, bool speed_test_mode = false) {
@ -88,6 +215,26 @@ class LoDTensorBlockingQueueHolder {
std::shared_ptr<LoDTensorBlockingQueue> queue_;
};
class OrderedMultiDeviceLoDTensorBlockingQueueHolder {
public:
void InitOnce(size_t capacity, bool speed_test_mode = false) {
PADDLE_ENFORCE_EQ(queue_, nullptr,
platform::errors::AlreadyExists(
"OrderedMultiDeviceLoDTensorBlockingQueueHolder::"
"InitOnce() can only be called once"));
queue_.reset(new OrderedMultiDeviceLoDTensorBlockingQueue(capacity,
speed_test_mode));
}
inline const std::shared_ptr<OrderedMultiDeviceLoDTensorBlockingQueue>&
GetQueue() const {
return queue_;
}
private:
std::shared_ptr<OrderedMultiDeviceLoDTensorBlockingQueue> queue_;
};
} // namespace reader
} // namespace operators
} // namespace paddle

@ -156,6 +156,10 @@ class ReadOpMaker : public framework::OpProtoAndCheckerMaker {
" and it is set by ParallelExecutor instance, not users.")
.SetDefault(true);
AddAttr<bool>("infer_out", "").SetDefault(true);
AddAttr<bool>("drop_last",
"Whether to drop last batches whose number is less than CPU "
"cores/GPU cards number")
.SetDefault(true);
AddComment(R"DOC(
Read Operator

@ -51,7 +51,6 @@ limitations under the License. */
#include "paddle/fluid/memory/allocation/mmap_allocator.h"
#include "paddle/fluid/operators/activation_op.h"
#include "paddle/fluid/operators/py_func_op.h"
#include "paddle/fluid/operators/reader/lod_tensor_blocking_queue.h"
#include "paddle/fluid/platform/cpu_helper.h"
#include "paddle/fluid/platform/cpu_info.h"
#include "paddle/fluid/platform/dynload/dynamic_loader.h"
@ -94,9 +93,6 @@ limitations under the License. */
#include "pybind11/stl.h"
DEFINE_bool(reader_queue_speed_test_mode, false,
"If set true, the queue.pop will only get data from queue but not "
"remove the data from queue for speed testing");
DECLARE_bool(use_mkldnn);
#ifdef PADDLE_WITH_NGRAPH
DECLARE_bool(use_ngraph);
@ -997,35 +993,6 @@ All parameter, weight, gradient are variables in Paddle.
BindReader(&m);
using LoDTensorBlockingQueue =
::paddle::operators::reader::LoDTensorBlockingQueue;
using LoDTensorBlockingQueueHolder =
::paddle::operators::reader::LoDTensorBlockingQueueHolder;
py::class_<LoDTensorBlockingQueue, std::shared_ptr<LoDTensorBlockingQueue>>(
m, "LoDTensorBlockingQueue", "")
.def("push",
[](LoDTensorBlockingQueue &self,
const std::vector<framework::LoDTensor> &lod_tensor_vec) {
pybind11::gil_scoped_release release;
return self.Push(lod_tensor_vec);
})
.def("size", &LoDTensorBlockingQueue::Size)
.def("capacity", &LoDTensorBlockingQueue::Cap)
.def("close", &LoDTensorBlockingQueue::Close)
.def("kill", &LoDTensorBlockingQueue::Kill)
.def("is_closed", &LoDTensorBlockingQueue::IsClosed);
m.def("init_lod_tensor_blocking_queue",
[](Variable &var,
size_t capacity) -> std::shared_ptr<LoDTensorBlockingQueue> {
VLOG(1) << "init_lod_tensor_blocking_queue";
auto *holder = var.GetMutable<LoDTensorBlockingQueueHolder>();
holder->InitOnce(capacity, FLAGS_reader_queue_speed_test_mode);
return holder->GetQueue();
},
py::return_value_policy::copy);
py::class_<Scope>(m, "_Scope", R"DOC(
Scope is an association of a name to Variable. All variables belong to Scope.

File diff suppressed because it is too large Load Diff

@ -216,18 +216,12 @@ def check_feed_shape_type(var, feed, num_places=1):
the feed value
"""
if var.desc.need_check_feed():
feed_shape = feed.shape()
if six.PY2:
feed_shape[0] = long(feed_shape[0] /
num_places) if len(feed.lod()) == 0 else -1
else:
feed_shape[0] = int(feed_shape[0] /
num_places) if len(feed.lod()) == 0 else -1
if not dimension_is_compatible_with(feed_shape, var.shape):
diff_shape = core.diff_tensor_shape(feed, var.desc, num_places)
if diff_shape is not None:
raise ValueError(
'The fed Variable %r should have dimensions = %d, shape = '
'%r, but received fed shape %r on each device' %
(var.name, len(var.shape), var.shape, feed_shape))
(var.name, len(var.shape), var.shape, diff_shape))
if not dtype_is_compatible_with(feed._dtype(), var.dtype):
var_dtype_format = convert_dtype(var.dtype) if isinstance(
var.dtype, core.VarDesc.VarType) else var.dtype
@ -646,11 +640,6 @@ class Executor(object):
exe.feed_and_split_tensor_into_local_scopes(feed_tensor_dict)
elif isinstance(feed, list) or isinstance(feed, tuple):
if len(feed) != len(program._places):
raise ValueError(
"Feed a list of tensor, the list should be the same size as places"
)
res = list()
for i, each in enumerate(feed):
if not isinstance(each, dict):

Some files were not shown because too many files have changed in this diff Show More

Loading…
Cancel
Save