未验证 提交 93c58390 编写于 作者: S ShenLiang 提交者: GitHub

[Distributed] Opt nccl connection by lazy initialization (#55005)

上级 51c414b6
...@@ -21,6 +21,7 @@ ...@@ -21,6 +21,7 @@
#include <vector> #include <vector>
#include "paddle/fluid/distributed/collective/types.h" #include "paddle/fluid/distributed/collective/types.h"
#include "paddle/fluid/distributed/collective/utils.h"
#include "paddle/fluid/eager/api/utils/tensor_utils.h" // NOTE: this header is required somewhere #include "paddle/fluid/eager/api/utils/tensor_utils.h" // NOTE: this header is required somewhere
#include "paddle/phi/core/dense_tensor.h" #include "paddle/phi/core/dense_tensor.h"
#include "paddle/phi/core/device_context.h" #include "paddle/phi/core/device_context.h"
...@@ -34,22 +35,6 @@ namespace distributed { ...@@ -34,22 +35,6 @@ namespace distributed {
constexpr int kIgnoreId = -1; constexpr int kIgnoreId = -1;
enum class CommType : std::uint8_t {
BROADCAST = 0,
ALLREDUCE = 1,
ALLREDUCE_SPARSE = 2, // TODO(shenliang03): to support sparse in allreduce
REDUCE = 3,
ALLGATHER = 4,
GATHER = 5,
SCATTER = 6,
REDUCE_SCATTER = 7,
ALLTOALL = 8,
SEND = 9,
RECV = 10,
BARRIER = 11,
UNKNOWN = 100,
};
class ProcessGroup { class ProcessGroup {
public: public:
class Task { class Task {
...@@ -405,68 +390,57 @@ class ProcessGroup { ...@@ -405,68 +390,57 @@ class ProcessGroup {
// legacy APIs // legacy APIs
// TODO(liyurui): This API will be moved later // TODO(liyurui): This API will be moved later
virtual std::shared_ptr<ProcessGroup::Task> AllReduce( virtual std::shared_ptr<ProcessGroup::Task> AllReduce(
std::vector<phi::DenseTensor>& /* input tensors */, // NOLINT std::vector<phi::DenseTensor>& inputs, // NOLINT
std::vector<phi::DenseTensor>& /* output tensors */, // NOLINT std::vector<phi::DenseTensor>& outputs, // NOLINT
const AllreduceOptions& = AllreduceOptions()) { const AllreduceOptions& options = AllreduceOptions()) {
PADDLE_THROW(phi::errors::InvalidArgument( return AllReduce(outputs.data(), inputs.front(), options, false);
"ProcessGroup%s does not support allreduce", GetBackendName()));
} }
virtual std::shared_ptr<ProcessGroup::Task> AllReduce( virtual std::shared_ptr<ProcessGroup::Task> AllReduce(
std::vector<phi::DenseTensor>& /* input tensors */, // NOLINT std::vector<phi::DenseTensor>& inputs, // NOLINT
std::vector<phi::DenseTensor>& /* output tensors */, // NOLINT std::vector<phi::DenseTensor>& outputs, // NOLINT
const AllreduceOptions&, const AllreduceOptions& options,
bool) { bool sync_op) {
PADDLE_THROW(phi::errors::InvalidArgument( return AllReduce(outputs.data(), inputs.front(), options, sync_op);
"ProcessGroup%s does not support allreduce with sync_op flag",
GetBackendName()));
} }
// TODO(sunyilun): methods below will be removed later // TODO(sunyilun): methods below will be removed later
virtual std::shared_ptr<ProcessGroup::Task> Broadcast( virtual std::shared_ptr<ProcessGroup::Task> Broadcast(
std::vector<phi::DenseTensor>& /* input tensors */, // NOLINT std::vector<phi::DenseTensor>& inputs, // NOLINT
std::vector<phi::DenseTensor>& /* output tensors */, // NOLINT std::vector<phi::DenseTensor>& outputs, // NOLINT
const BroadcastOptions& = BroadcastOptions()) { const BroadcastOptions& options = BroadcastOptions()) {
PADDLE_THROW(phi::errors::InvalidArgument( return Broadcast(outputs.data(), inputs.front(), options, false);
"ProcessGroup%s does not support broadcast", GetBackendName()));
} }
virtual std::shared_ptr<ProcessGroup::Task> Broadcast( virtual std::shared_ptr<ProcessGroup::Task> Broadcast(
std::vector<phi::DenseTensor>& /* input tensors */, // NOLINT std::vector<phi::DenseTensor>& inputs, // NOLINT
std::vector<phi::DenseTensor>& /* output tensors */, // NOLINT std::vector<phi::DenseTensor>& outputs, // NOLINT
const BroadcastOptions&, const BroadcastOptions& options,
bool) { bool sync_op) {
PADDLE_THROW(phi::errors::InvalidArgument( return Broadcast(outputs.data(), inputs.front(), options, sync_op);
"ProcessGroup%s does not support broadcast with sync_op flag",
GetBackendName()));
} }
virtual std::shared_ptr<ProcessGroup::Task> Send( virtual std::shared_ptr<ProcessGroup::Task> Send(
std::vector<phi::DenseTensor>&, int) { // NOLINT std::vector<phi::DenseTensor>& tensors, int dst_rank) { // NOLINT
PADDLE_THROW(phi::errors::InvalidArgument( return Send(tensors.front(), dst_rank, false);
"ProcessGroup%s does not support send", GetBackendName()));
} }
virtual std::shared_ptr<ProcessGroup::Task> Recv( virtual std::shared_ptr<ProcessGroup::Task> Recv(
std::vector<phi::DenseTensor>&, int) { // NOLINT std::vector<phi::DenseTensor>& tensors, int src_rank) { // NOLINT
PADDLE_THROW(phi::errors::InvalidArgument( return Recv(&tensors.front(), src_rank, false);
"ProcessGroup%s does not support recv", GetBackendName()));
} }
virtual std::shared_ptr<ProcessGroup::Task> AllGather( virtual std::shared_ptr<ProcessGroup::Task> AllGather(
std::vector<phi::DenseTensor>&, // NOLINT std::vector<phi::DenseTensor>& in_tensors, // NOLINT
std::vector<phi::DenseTensor>&) { // NOLINT std::vector<phi::DenseTensor>& out_tensors) { // NOLINT
PADDLE_THROW(phi::errors::InvalidArgument( return AllGather(out_tensors.data(), in_tensors.front(), false);
"ProcessGroup%s does not support all_gather", GetBackendName()));
} }
virtual std::shared_ptr<ProcessGroup::Task> AllGather( virtual std::shared_ptr<ProcessGroup::Task> AllGather(
std::vector<phi::DenseTensor>&, // NOLINT std::vector<phi::DenseTensor>& in_tensors, // NOLINT
std::vector<phi::DenseTensor>&, // NOLINT std::vector<phi::DenseTensor>& out_tensors, // NOLINT
bool) { bool sync_op) {
PADDLE_THROW(phi::errors::InvalidArgument( return AllGather(out_tensors.data(), in_tensors.front(), sync_op);
"ProcessGroup%s does not support all_gather with sync_op flag",
GetBackendName()));
} }
virtual std::shared_ptr<ProcessGroup::Task> AllToAll( virtual std::shared_ptr<ProcessGroup::Task> AllToAll(
...@@ -477,19 +451,17 @@ class ProcessGroup { ...@@ -477,19 +451,17 @@ class ProcessGroup {
} }
virtual std::shared_ptr<ProcessGroup::Task> Reduce( virtual std::shared_ptr<ProcessGroup::Task> Reduce(
std::vector<phi::DenseTensor>&, // NOLINT std::vector<phi::DenseTensor>& ins, // NOLINT
std::vector<phi::DenseTensor>&, // NOLINT std::vector<phi::DenseTensor>& outs, // NOLINT
const ReduceOptions& opts) { const ReduceOptions& opts) {
PADDLE_THROW(phi::errors::InvalidArgument( return Reduce(outs.data(), ins.front(), opts, false);
"ProcessGroup%s does not support reduce", GetBackendName()));
} }
virtual std::shared_ptr<ProcessGroup::Task> Scatter( virtual std::shared_ptr<ProcessGroup::Task> Scatter(
std::vector<phi::DenseTensor>&, // NOLINT std::vector<phi::DenseTensor>& ins, // NOLINT
std::vector<phi::DenseTensor>&, // NOLINT std::vector<phi::DenseTensor>& outs, // NOLINT
const ScatterOptions&) { const ScatterOptions& opts) {
PADDLE_THROW(phi::errors::InvalidArgument( return Scatter(outs.data(), ins.front(), opts, false);
"ProcessGroup%s does not support scatter", GetBackendName()));
} }
protected: protected:
......
...@@ -16,7 +16,6 @@ ...@@ -16,7 +16,6 @@
#include "paddle/fluid/distributed/collective/bkcl_tools.h" #include "paddle/fluid/distributed/collective/bkcl_tools.h"
#include "paddle/fluid/distributed/collective/common.h" #include "paddle/fluid/distributed/collective/common.h"
#include "paddle/fluid/distributed/collective/utils.h"
#include "paddle/fluid/platform/device/xpu/bkcl_helper.h" #include "paddle/fluid/platform/device/xpu/bkcl_helper.h"
#include "paddle/fluid/platform/device/xpu/xpu_info.h" #include "paddle/fluid/platform/device/xpu/xpu_info.h"
#include "paddle/phi/api/lib/utils/allocator.h" #include "paddle/phi/api/lib/utils/allocator.h"
......
...@@ -16,7 +16,6 @@ ...@@ -16,7 +16,6 @@
#include "paddle/fluid/distributed/collective/common.h" #include "paddle/fluid/distributed/collective/common.h"
#include "paddle/fluid/distributed/collective/custom_ccl_tools.h" #include "paddle/fluid/distributed/collective/custom_ccl_tools.h"
#include "paddle/fluid/distributed/collective/utils.h"
#include "paddle/fluid/memory/malloc.h" #include "paddle/fluid/memory/malloc.h"
#include "paddle/fluid/platform/device_context.h" #include "paddle/fluid/platform/device_context.h"
#include "paddle/fluid/platform/place.h" #include "paddle/fluid/platform/place.h"
......
...@@ -16,7 +16,6 @@ ...@@ -16,7 +16,6 @@
#include "paddle/fluid/distributed/collective/common.h" #include "paddle/fluid/distributed/collective/common.h"
#include "paddle/fluid/distributed/collective/nccl_tools.h" #include "paddle/fluid/distributed/collective/nccl_tools.h"
#include "paddle/fluid/distributed/collective/utils.h"
#include "paddle/fluid/platform/cuda_device_guard.h" #include "paddle/fluid/platform/cuda_device_guard.h"
#include "paddle/fluid/platform/device/gpu/nccl_helper.h" #include "paddle/fluid/platform/device/gpu/nccl_helper.h"
#include "paddle/phi/api/lib/utils/allocator.h" #include "paddle/phi/api/lib/utils/allocator.h"
...@@ -35,6 +34,8 @@ constexpr int64_t kWaitBlockTImeout = 10; ...@@ -35,6 +34,8 @@ constexpr int64_t kWaitBlockTImeout = 10;
namespace paddle { namespace paddle {
namespace distributed { namespace distributed {
uint64_t ProcessGroupNCCL::s_group_call_counter = 0;
ProcessGroupNCCL::NCCLTask::NCCLTask(const Place& place, ProcessGroupNCCL::NCCLTask::NCCLTask(const Place& place,
int rank, int rank,
CommType comm_type, CommType comm_type,
...@@ -96,15 +97,21 @@ ProcessGroupNCCL::ProcessGroupNCCL( ...@@ -96,15 +97,21 @@ ProcessGroupNCCL::ProcessGroupNCCL(
void ProcessGroupNCCL::GroupStart() { void ProcessGroupNCCL::GroupStart() {
NCCL_CHECK(phi::dynload::ncclGroupStart()); NCCL_CHECK(phi::dynload::ncclGroupStart());
++s_group_call_counter;
} }
void ProcessGroupNCCL::GroupEnd() { NCCL_CHECK(phi::dynload::ncclGroupEnd()); } void ProcessGroupNCCL::GroupEnd() {
NCCL_CHECK(phi::dynload::ncclGroupEnd());
--s_group_call_counter;
}
phi::DeviceContext* ProcessGroupNCCL::GetDeviceContext( phi::DeviceContext* ProcessGroupNCCL::GetDeviceContext(
const Place& place) const { const Place& place) const {
return GetDeviceContext(place, /*use_calc_stream*/ false); return GetDeviceContext(place, /*use_calc_stream*/ false);
} }
// NOTE(shenliang03): GetDeviceContext is only used for collective, it can't
// be used for p2p op.
phi::DeviceContext* ProcessGroupNCCL::GetDeviceContext( phi::DeviceContext* ProcessGroupNCCL::GetDeviceContext(
const Place& place, bool use_calc_stream) const { const Place& place, bool use_calc_stream) const {
const std::string& key = GetKeyFromPlace(place); const std::string& key = GetKeyFromPlace(place);
...@@ -148,7 +155,7 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::AllGather( ...@@ -148,7 +155,7 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::AllGather(
/*dst_rank*/ rank_, /*dst_rank*/ rank_,
/*cur_rank*/ rank_, /*cur_rank*/ rank_,
size_); size_);
return RunFnInNCCLEnv( return Collective(
[&](ncclComm_t comm, gpuStream_t stream) { [&](ncclComm_t comm, gpuStream_t stream) {
if (FLAGS_enable_nccl_dynamic_check) { if (FLAGS_enable_nccl_dynamic_check) {
phi::distributed::NCCLDynamicCheck::CheckShape(*out_tensor, phi::distributed::NCCLDynamicCheck::CheckShape(*out_tensor,
...@@ -181,7 +188,7 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::AllReduce( ...@@ -181,7 +188,7 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::AllReduce(
/*dst_rank*/ rank_, /*dst_rank*/ rank_,
/*cur_rank*/ rank_, /*cur_rank*/ rank_,
size_); size_);
return RunFnInNCCLEnv( return Collective(
[&](ncclComm_t comm, gpuStream_t stream) { [&](ncclComm_t comm, gpuStream_t stream) {
if (FLAGS_enable_nccl_dynamic_check) { if (FLAGS_enable_nccl_dynamic_check) {
phi::distributed::NCCLDynamicCheck::CheckShape(*out_tensor, phi::distributed::NCCLDynamicCheck::CheckShape(*out_tensor,
...@@ -246,7 +253,7 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::AllToAll( ...@@ -246,7 +253,7 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::AllToAll(
size_, size_,
/*out_size_factor*/ 0, /*out_size_factor*/ 0,
/*in_size_factor*/ 0); /*in_size_factor*/ 0);
return RunFnInNCCLEnv( return Collective(
[&](ncclComm_t comm, gpuStream_t stream) { [&](ncclComm_t comm, gpuStream_t stream) {
if (FLAGS_enable_nccl_dynamic_check) { if (FLAGS_enable_nccl_dynamic_check) {
phi::distributed::NCCLDynamicCheck::CheckShape( phi::distributed::NCCLDynamicCheck::CheckShape(
...@@ -322,7 +329,7 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Broadcast( ...@@ -322,7 +329,7 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Broadcast(
/*dst_rank*/ rank_, /*dst_rank*/ rank_,
/*cur_rank*/ rank_, /*cur_rank*/ rank_,
size_); size_);
return RunFnInNCCLEnv( return Collective(
[&](ncclComm_t comm, gpuStream_t stream) { [&](ncclComm_t comm, gpuStream_t stream) {
int root = opts.source_rank + opts.source_root; int root = opts.source_rank + opts.source_root;
if (FLAGS_enable_nccl_dynamic_check) { if (FLAGS_enable_nccl_dynamic_check) {
...@@ -355,7 +362,7 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Reduce( ...@@ -355,7 +362,7 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Reduce(
/*dst_rank*/ opts.root_rank, /*dst_rank*/ opts.root_rank,
/*cur_rank*/ rank_, /*cur_rank*/ rank_,
size_); size_);
return RunFnInNCCLEnv( return Collective(
[&](ncclComm_t comm, gpuStream_t stream) { [&](ncclComm_t comm, gpuStream_t stream) {
if (FLAGS_enable_nccl_dynamic_check) { if (FLAGS_enable_nccl_dynamic_check) {
phi::distributed::NCCLDynamicCheck::CheckShape( phi::distributed::NCCLDynamicCheck::CheckShape(
...@@ -391,7 +398,7 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::ReduceScatter( ...@@ -391,7 +398,7 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::ReduceScatter(
/*dst_rank*/ rank_, /*dst_rank*/ rank_,
/*cur_rank*/ rank_, /*cur_rank*/ rank_,
size_); size_);
return RunFnInNCCLEnv( return Collective(
[&](ncclComm_t comm, gpuStream_t stream) { [&](ncclComm_t comm, gpuStream_t stream) {
if (FLAGS_enable_nccl_dynamic_check) { if (FLAGS_enable_nccl_dynamic_check) {
phi::distributed::NCCLDynamicCheck::CheckShape(*out_tensor, phi::distributed::NCCLDynamicCheck::CheckShape(*out_tensor,
...@@ -426,7 +433,7 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Scatter( ...@@ -426,7 +433,7 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Scatter(
/*dst_rank*/ opts.root_rank, /*dst_rank*/ opts.root_rank,
/*cur_rank*/ rank_, /*cur_rank*/ rank_,
size_); size_);
return RunFnInNCCLEnv( return Collective(
[&](ncclComm_t comm, gpuStream_t stream) { [&](ncclComm_t comm, gpuStream_t stream) {
if (FLAGS_enable_nccl_dynamic_check) { if (FLAGS_enable_nccl_dynamic_check) {
phi::distributed::NCCLDynamicCheck::CheckShape( phi::distributed::NCCLDynamicCheck::CheckShape(
...@@ -536,7 +543,7 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Gather( ...@@ -536,7 +543,7 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Gather(
stream)); stream));
GroupEnd(); GroupEnd();
}; };
return RunFnInNCCLEnv( return Collective(
gather_func, in_tensor, CommType::GATHER, sync_op, use_calc_stream); gather_func, in_tensor, CommType::GATHER, sync_op, use_calc_stream);
} }
...@@ -555,8 +562,8 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Recv( ...@@ -555,8 +562,8 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Recv(
} }
phi::distributed::CommStaticCheck::CheckShape(*tensor, rank_, size_); phi::distributed::CommStaticCheck::CheckShape(*tensor, rank_, size_);
return RunFnInNCCLEnv( return Point2Point(
[&](ncclComm_t comm, gpuStream_t stream) { [&](ncclComm_t comm, gpuStream_t stream, int rank_in_group) {
if (FLAGS_enable_nccl_dynamic_check) { if (FLAGS_enable_nccl_dynamic_check) {
phi::distributed::NCCLDynamicCheck::CheckShape(*tensor, phi::distributed::NCCLDynamicCheck::CheckShape(*tensor,
/*root_rank*/ src_rank, /*root_rank*/ src_rank,
...@@ -566,10 +573,11 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Recv( ...@@ -566,10 +573,11 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Recv(
NCCL_CHECK(phi::dynload::ncclRecv(tensor->data(), NCCL_CHECK(phi::dynload::ncclRecv(tensor->data(),
tensor->numel(), tensor->numel(),
phi::ToNCCLDataType(tensor->dtype()), phi::ToNCCLDataType(tensor->dtype()),
src_rank, rank_in_group,
comm, comm,
stream)); stream));
}, },
src_rank,
*tensor, *tensor,
CommType::RECV, CommType::RECV,
sync_op, sync_op,
...@@ -589,8 +597,8 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Send( ...@@ -589,8 +597,8 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Send(
phi::distributed::CommStaticCheck::CheckShape( phi::distributed::CommStaticCheck::CheckShape(
tensor_maybe_partial, rank_, size_); tensor_maybe_partial, rank_, size_);
return RunFnInNCCLEnv( return Point2Point(
[&](ncclComm_t comm, gpuStream_t stream) { [&](ncclComm_t comm, gpuStream_t stream, int rank_in_group) {
if (FLAGS_enable_nccl_dynamic_check) { if (FLAGS_enable_nccl_dynamic_check) {
phi::distributed::NCCLDynamicCheck::CheckShape(tensor_maybe_partial, phi::distributed::NCCLDynamicCheck::CheckShape(tensor_maybe_partial,
/*root_rank*/ rank_, /*root_rank*/ rank_,
...@@ -601,10 +609,11 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Send( ...@@ -601,10 +609,11 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Send(
tensor_maybe_partial.data(), tensor_maybe_partial.data(),
tensor_maybe_partial.numel(), tensor_maybe_partial.numel(),
phi::ToNCCLDataType(tensor_maybe_partial.dtype()), phi::ToNCCLDataType(tensor_maybe_partial.dtype()),
dst_rank, rank_in_group,
comm, comm,
stream)); stream));
}, },
dst_rank,
tensor_maybe_partial, tensor_maybe_partial,
CommType::SEND, CommType::SEND,
sync_op, sync_op,
...@@ -621,64 +630,89 @@ std::shared_ptr<ProcessGroupNCCL::NCCLTask> ProcessGroupNCCL::CreateTask( ...@@ -621,64 +630,89 @@ std::shared_ptr<ProcessGroupNCCL::NCCLTask> ProcessGroupNCCL::CreateTask(
place, rank, comm_type, is_sync, use_calc_stream); place, rank, comm_type, is_sync, use_calc_stream);
} }
void ProcessGroupNCCL::BroadcastUniqueNCCLID(ncclUniqueId* nccl_id) { void ProcessGroupNCCL::BroadcastUniqueNCCLID(ncclUniqueId* nccl_id,
const std::string key = bool is_p2p_op,
"ProcessGroupNCCL/nccl_ids/" + std::to_string(gid_) + "/0"; const std::string& p2p_key,
if (rank_ == 0) { int p2p_rank) {
std::string store_key;
if (!is_p2p_op) {
store_key = "ProcessGroupNCCL/nccl_ids/" + std::to_string(gid_) + "/0";
} else {
store_key =
"ProcessGroupNCCL/nccl_ids/" + std::to_string(gid_) + "/" + p2p_key;
}
if (rank_ == 0 || (is_p2p_op && p2p_rank == 0)) {
std::vector<uint8_t> nccl_id_wrapper( std::vector<uint8_t> nccl_id_wrapper(
reinterpret_cast<uint8_t*>(nccl_id), reinterpret_cast<uint8_t*>(nccl_id),
reinterpret_cast<uint8_t*>(nccl_id) + NCCL_UNIQUE_ID_BYTES); reinterpret_cast<uint8_t*>(nccl_id) + NCCL_UNIQUE_ID_BYTES);
store_->set(key, nccl_id_wrapper); store_->set(store_key, nccl_id_wrapper);
} else { } else {
const auto& nccl_id_wrapper = store_->get(key); const auto& nccl_id_wrapper = store_->get(store_key);
std::memcpy(nccl_id, nccl_id_wrapper.data(), nccl_id_wrapper.size()); std::memcpy(nccl_id, nccl_id_wrapper.data(), nccl_id_wrapper.size());
} }
} }
void ProcessGroupNCCL::CreateNCCLEnvCache(const Place& place, void ProcessGroupNCCL::CreateNCCLEnvCache(const Place& place,
const std::string& place_key) { const std::string& place_key,
if (place_to_comm_ctx_.size() > 0) { CommType comm_type,
VLOG(3) << "Warning: Tensors from multiple devices are not supported yet."; int p2p_rank) {
}
ncclUniqueId nccl_id; ncclUniqueId nccl_id;
if (rank_ == 0) {
bool is_batch_p2p = s_group_call_counter > 0;
bool is_p2p_op = IsP2POP(comm_type, is_batch_p2p);
if (rank_ == 0 || (is_p2p_op && p2p_rank == 0)) {
NCCL_CHECK(phi::dynload::ncclGetUniqueId(&nccl_id)); NCCL_CHECK(phi::dynload::ncclGetUniqueId(&nccl_id));
} }
BroadcastUniqueNCCLID(&nccl_id);
BroadcastUniqueNCCLID(&nccl_id, is_p2p_op, place_key, p2p_rank);
VLOG(3) << "init nccl rank: " << rank_ << ", nranks: " << size_ VLOG(3) << "init nccl rank: " << rank_ << ", nranks: " << size_
<< ", place: " << place_key << ", place key: " << place_key
<< ", nccl uniqueid: " << SerializeNCCLUniqueId(nccl_id); << ", nccl uniqueid: " << SerializeNCCLUniqueId(nccl_id);
auto* calc_ctx = static_cast<phi::GPUContext*>( for (size_t i = 0; i < s_group_call_counter; ++i) {
platform::DeviceContextPool::Instance().Get(place)); NCCL_CHECK(phi::dynload::ncclGroupEnd());
auto comm_ctx = std::make_unique<phi::GPUContext>(place); }
int num_ranks = is_p2p_op ? 2 : GetSize();
int rank = is_p2p_op ? p2p_rank : GetRank();
NCCL_CHECK(phi::dynload::ncclGroupStart());
ncclComm_t nccl_comm; ncclComm_t nccl_comm;
NCCL_CHECK(phi::dynload::ncclCommInitRank( NCCL_CHECK(
&nccl_comm, GetSize(), nccl_id, GetRank())); phi::dynload::ncclCommInitRank(&nccl_comm, num_ranks, nccl_id, rank));
NCCL_CHECK(phi::dynload::ncclGroupEnd());
VLOG(3) << "Get nccl comm: " << nccl_comm << " for place_key: " << place_key
<< " on rank: " << rank << " nranks: " << num_ranks;
auto comm_ctx = std::make_unique<phi::GPUContext>(place);
comm_ctx->set_nccl_comm(nccl_comm); comm_ctx->set_nccl_comm(nccl_comm);
auto* calc_ctx = static_cast<phi::GPUContext*>(
platform::DeviceContextPool::Instance().Get(place));
place_to_calc_event_.emplace(place_key, place); place_to_calc_event_.emplace(place_key, place);
place_to_calc_ctx_.emplace(place_key, calc_ctx); place_to_calc_ctx_.emplace(place_key, calc_ctx);
place_to_comm_ctx_.emplace(place_key, std::move(comm_ctx)); place_to_comm_ctx_.emplace(place_key, std::move(comm_ctx));
// TODO(sunyilun): for compatibility, will be removed later for (size_t i = 0; i < s_group_call_counter; ++i) {
std::vector<phi::GPUContext*> comm_ctx_wrapper{ NCCL_CHECK(phi::dynload::ncclGroupStart());
place_to_comm_ctx_[place_key].get()}; }
places_to_ctx_.emplace(place_key, comm_ctx_wrapper);
} }
void ProcessGroupNCCL::SyncCalcStream(const Place& place) { void ProcessGroupNCCL::SyncCalcStream(const Place& place,
const std::string& key = GetKeyFromPlace(place); const std::string& place_key) {
auto& calc_event = place_to_calc_event_.at(key); auto& calc_event = place_to_calc_event_.at(place_key);
const auto* calc_ctx = place_to_calc_ctx_.at(key); const auto* calc_ctx = place_to_calc_ctx_.at(place_key);
const auto* comm_ctx = place_to_comm_ctx_.at(key).get(); const auto* comm_ctx = place_to_comm_ctx_.at(place_key).get();
calc_event.Record(calc_ctx); calc_event.Record(calc_ctx);
calc_event.Wait(platform::Place2DeviceType(place), comm_ctx); calc_event.Wait(platform::Place2DeviceType(place), comm_ctx);
} }
std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::RunFnInNCCLEnv( std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Collective(
std::function<void(ncclComm_t, gpuStream_t)> fn, std::function<void(ncclComm_t, gpuStream_t)> fn,
const phi::DenseTensor& tensor, const phi::DenseTensor& tensor,
CommType comm_type, CommType comm_type,
...@@ -690,11 +724,11 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::RunFnInNCCLEnv( ...@@ -690,11 +724,11 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::RunFnInNCCLEnv(
platform::CUDADeviceGuard cuda_guard(place); platform::CUDADeviceGuard cuda_guard(place);
if (place_to_comm_ctx_.find(key) == place_to_comm_ctx_.end()) { if (place_to_comm_ctx_.find(key) == place_to_comm_ctx_.end()) {
CreateNCCLEnvCache(place, key); CreateNCCLEnvCache(place, key, comm_type);
} }
if (!use_calc_stream) { if (!use_calc_stream) {
SyncCalcStream(place); SyncCalcStream(place, key);
} }
auto task = CreateTask(place, rank_, comm_type, sync_op, use_calc_stream); auto task = CreateTask(place, rank_, comm_type, sync_op, use_calc_stream);
...@@ -716,506 +750,75 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::RunFnInNCCLEnv( ...@@ -716,506 +750,75 @@ std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::RunFnInNCCLEnv(
task->SetBlockCPUInWait(); task->SetBlockCPUInWait();
task->Wait(); task->Wait();
} }
return task;
}
// TODO(sunyilun): methods below will be removed later if (sync_op) {
void SyncDefaultStream(const std::vector<Place>& places, task->Wait();
platform::DeviceEvent& nccl_event, // NOLINT
std::vector<phi::GPUContext*>& dev_ctx) { // NOLINT
for (size_t i = 0; i < places.size(); ++i) {
auto* default_ctx = static_cast<phi::GPUContext*>(
platform::DeviceContextPool::Instance().Get(places[i]));
nccl_event.Record(default_ctx);
nccl_event.Wait(platform::Place2DeviceType(places[i]), dev_ctx[i]);
} }
return task;
} }
std::shared_ptr<ProcessGroupNCCL::NCCLTask> ProcessGroupNCCL::CreateTask( std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Point2Point(
std::vector<Place> places, std::function<void(ncclComm_t, gpuStream_t, int)> fn,
int rank, int peer,
const phi::DenseTensor& tensor,
CommType comm_type, CommType comm_type,
const std::vector<phi::DenseTensor>& inputs) { bool sync_op,
return std::make_shared<ProcessGroupNCCL::NCCLTask>( bool use_calc_stream) {
places, rank, comm_type, inputs); const auto& place = tensor.place();
}
ProcessGroupNCCL::NCCLTask::NCCLTask(
const std::vector<Place>& places,
int rank,
CommType CommType,
const std::vector<phi::DenseTensor>& inputs)
: TaskStream(rank, inputs, CommType),
comm_event_(places[0]),
task_place_(places[0]) {}
// create NCCLManager cache for places_key
void ProcessGroupNCCL::CreateNCCLManagerCache(
const std::string& places_key, const std::vector<Place>& places) {
PADDLE_ENFORCE_EQ(places_key.empty(),
false,
phi::errors::PreconditionNotMet(
"Not able to create/get the NCCL Communicator since "
"the GPU place are not known"));
ncclUniqueId nccl_id;
if (rank_ == 0) {
NCCL_CHECK(phi::dynload::ncclGetUniqueId(&nccl_id));
}
BroadcastUniqueNCCLID(&nccl_id);
VLOG(3) << "init nccl rank: " << rank_ << ", nranks: " << size_
<< ", place: " << places_key
<< ", nccl uniqueid: " << SerializeNCCLUniqueId(nccl_id);
std::vector<std::unique_ptr<phi::GPUContext>> dev_ctx;
dev_ctx.resize(places.size());
std::vector<phi::GPUContext*> dev_ctx_raw;
dev_ctx_raw.resize(places.size());
GroupStart();
for (size_t i = 0; i < places.size(); ++i) { int p2p_rank = 0;
platform::CUDADeviceGuard guard(places[i]); int p2p_target_rank = 0;
bool is_batch_p2p = s_group_call_counter > 0;
std::string key = "";
dev_ctx[i].reset(new phi::GPUContext(places[i])); if (is_batch_p2p) {
ncclComm_t nccl_comm; key = GetKeyFromPlace(place);
NCCL_CHECK(phi::dynload::ncclCommInitRank( p2p_rank = rank_;
&nccl_comm, GetSize(), nccl_id, GetRank())); p2p_target_rank = peer;
dev_ctx[i]->set_nccl_comm(nccl_comm); } else {
dev_ctx_raw[i] = dev_ctx[i].get(); int low_rank = rank_ < peer ? rank_ : peer;
int high_rank = rank_ < peer ? peer : rank_;
key = std::to_string(low_rank) + "->" + std::to_string(high_rank);
p2p_rank = rank_ < peer ? 0 : 1;
p2p_target_rank = 1 - p2p_rank;
} }
GroupEnd(); platform::CUDADeviceGuard cuda_guard(place);
// TODO(sunyilun): for compatibility, will be removed later
place_to_calc_event_.emplace(places_key, places[0]);
place_to_calc_ctx_.emplace(
places_key,
static_cast<phi::GPUContext*>(
platform::DeviceContextPool::Instance().Get(places[0])));
place_to_comm_ctx_.emplace(places_key, std::move(dev_ctx[0]));
// These caches will be useful to process sync/wait/communicate
places_to_ctx_.emplace(places_key, std::move(dev_ctx_raw));
}
template <typename Fn>
std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Collective(
std::vector<phi::DenseTensor>& inputs,
std::vector<phi::DenseTensor>& outputs,
Fn fn,
CommType op_type) {
const auto places = GetPlaceList(inputs);
const auto key = GetKeyFromPlaces(places);
{
std::lock_guard<std::mutex> lock(mutex_);
if (place_to_comm_ctx_.find(key) == place_to_comm_ctx_.end()) { if (place_to_comm_ctx_.find(key) == place_to_comm_ctx_.end()) {
CreateNCCLManagerCache(key, places); CreateNCCLEnvCache(place, key, comm_type, p2p_rank);
} }
}
SyncDefaultStream(
places, place_to_calc_event_.at(key), places_to_ctx_.at(key));
auto task = CreateTask(places, rank_, op_type, inputs);
// construct uninitialize guard for device if (!use_calc_stream) {
platform::CUDADeviceGuard cuda_guard; SyncCalcStream(place, key);
{
platform::NCCLGroupGuard nccl_guard;
for (size_t i = 0; i < inputs.size(); ++i) {
cuda_guard.SetDevice(places[i]);
const auto& nccl_stream = places_to_ctx_.at(key)[i]->stream();
fn(inputs[i],
outputs[i],
places_to_ctx_.at(key)[i]->nccl_comm(),
nccl_stream);
}
}
if (FLAGS_use_stream_safe_cuda_allocator) {
for (size_t i = 0; i < inputs.size(); ++i) {
cuda_guard.SetDevice(places[i]);
memory::RecordStream(inputs[i].Holder(),
places_to_ctx_.at(key)[i]->stream());
}
}
for (size_t i = 0; i < inputs.size(); ++i) {
cuda_guard.SetDevice(places[i]);
task->UpdateWaitChain(*places_to_ctx_.at(key)[i]);
}
return task;
}
template <typename Fn>
std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::PointToPoint(
std::vector<phi::DenseTensor>& tensors,
Fn fn,
int dst_rank,
CommType op_type) {
const auto places = GetPlaceList(tensors);
const auto key = GetKeyFromPlaces(places);
{
std::lock_guard<std::mutex> lock(mutex_);
if (place_to_comm_ctx_.find(key) == place_to_comm_ctx_.end()) {
CreateNCCLManagerCache(key, places);
}
} }
SyncDefaultStream( auto task = CreateTask(place, rank_, comm_type, sync_op, use_calc_stream);
places, place_to_calc_event_.at(key), places_to_ctx_.at(key)); const auto* calc_ctx = place_to_calc_ctx_.at(key);
const auto& comm_ctx = place_to_comm_ctx_.at(key);
auto task = CreateTask(places, rank_, op_type, tensors);
// construct uninitialize guard for device
platform::CUDADeviceGuard cuda_guard;
{ auto nccl_comm = comm_ctx->nccl_comm();
platform::NCCLGroupGuard nccl_guard; auto nccl_stream = use_calc_stream ? calc_ctx->stream() : comm_ctx->stream();
for (size_t i = 0; i < tensors.size(); ++i) { fn(nccl_comm, nccl_stream, p2p_target_rank);
cuda_guard.SetDevice(places[i]);
const auto& nccl_stream = places_to_ctx_.at(key)[i]->stream();
fn(tensors[i],
places_to_ctx_.at(key)[i]->nccl_comm(),
nccl_stream,
dst_rank);
}
}
if (!use_calc_stream) {
if (FLAGS_use_stream_safe_cuda_allocator) { if (FLAGS_use_stream_safe_cuda_allocator) {
for (size_t i = 0; i < tensors.size(); ++i) { memory::RecordStream(tensor.Holder(), nccl_stream);
cuda_guard.SetDevice(places[i]);
memory::RecordStream(tensors[i].Holder(),
places_to_ctx_.at(key)[i]->stream());
} }
task->UpdateWaitChain(*comm_ctx);
} }
for (size_t i = 0; i < tensors.size(); ++i) { if (FLAGS_enable_nccl_dynamic_check) {
cuda_guard.SetDevice(places[i]); task->SetBlockCPUInWait();
task->UpdateWaitChain(*places_to_ctx_.at(key)[i]); task->Wait();
} }
return task;
}
std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::AllReduce(
std::vector<phi::DenseTensor>& in_tensors,
std::vector<phi::DenseTensor>& out_tensors,
const AllreduceOptions& opts) {
PADDLE_ENFORCE_EQ(
CheckTensorsInCudaPlace(in_tensors),
true,
phi::errors::InvalidArgument("All inputs should be in CudaPlace."));
return Collective(
in_tensors,
out_tensors,
[&](const phi::DenseTensor& input,
phi::DenseTensor& output,
ncclComm_t comm,
const gpuStream_t& stream) {
return phi::dynload::ncclAllReduce(input.data(),
output.data(),
input.numel(),
phi::ToNCCLDataType(input.type()),
ToNCCLRedType(opts.reduce_op),
comm,
stream);
},
CommType::ALLREDUCE);
}
std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Broadcast(
std::vector<phi::DenseTensor>& in_tensors,
std::vector<phi::DenseTensor>& out_tensors,
const BroadcastOptions& opts) {
PADDLE_ENFORCE_EQ(
CheckTensorsInCudaPlace(in_tensors),
true,
phi::errors::InvalidArgument("All inputs should be in CudaPlace."));
return Collective(
in_tensors,
out_tensors,
[&](phi::DenseTensor& input,
phi::DenseTensor& output,
ncclComm_t comm,
const gpuStream_t& stream) {
const auto root =
opts.source_rank * in_tensors.size() + opts.source_root;
return phi::dynload::ncclBroadcast(input.data(),
output.data(),
input.numel(),
phi::ToNCCLDataType(input.type()),
root,
comm,
stream);
},
CommType::BROADCAST);
}
void CheckTensorsInDifferentDevices(
const std::vector<phi::DenseTensor>& tensors, const size_t num_devices) {
PADDLE_ENFORCE_EQ(
tensors.size() == 0,
false,
phi::errors::InvalidArgument("Tensor list must be nonempty."));
PADDLE_ENFORCE_LE(
tensors.size(),
num_devices,
phi::errors::InvalidArgument(
"Tensor list mustn't be larger than the number of available GPUs."));
std::set<Place> used_devices;
for (const auto& t : tensors) { if (sync_op) {
PADDLE_ENFORCE_EQ( task->Wait();
platform::is_gpu_place(t.place()),
true,
phi::errors::InvalidArgument("Tensors must be CUDA and dense tensor."));
const auto inserted = used_devices.insert(t.place()).second;
PADDLE_ENFORCE_EQ(inserted,
true,
phi::errors::InvalidArgument(
"Tensors must be on distinct GPU devices."));
} }
}
std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Send(
std::vector<phi::DenseTensor>& tensors, int dst_rank) {
CheckTensorsInDifferentDevices(tensors, static_cast<size_t>(GetSize()));
auto task = PointToPoint(
tensors,
[&](phi::DenseTensor& input,
ncclComm_t comm,
const gpuStream_t& stream,
int dst_rank) {
return phi::dynload::ncclSend(input.data(),
input.numel(),
phi::ToNCCLDataType(input.dtype()),
dst_rank,
comm,
stream);
},
dst_rank,
CommType::SEND);
return task; return task;
} }
std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Recv(
std::vector<phi::DenseTensor>& tensors, int src_rank) {
CheckTensorsInDifferentDevices(tensors, static_cast<size_t>(GetSize()));
auto task = PointToPoint(
tensors,
[&](phi::DenseTensor& output,
ncclComm_t comm,
const gpuStream_t& stream,
int src_rank) {
return phi::dynload::ncclRecv(output.data(),
output.numel(),
phi::ToNCCLDataType(output.dtype()),
src_rank,
comm,
stream);
},
src_rank,
CommType::RECV);
return task;
}
std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::AllGather(
std::vector<phi::DenseTensor>& in_tensors,
std::vector<phi::DenseTensor>& out_tensors) {
PADDLE_ENFORCE_EQ(
CheckTensorsInCudaPlace(in_tensors),
true,
phi::errors::InvalidArgument("All inputs should be in CudaPlace."));
PADDLE_ENFORCE_EQ(
CheckTensorsInCudaPlace(out_tensors),
true,
phi::errors::InvalidArgument("All outputs should be in CudaPlace."));
return Collective(
in_tensors,
out_tensors,
[&](const phi::DenseTensor& input,
phi::DenseTensor& output,
ncclComm_t comm,
const gpuStream_t& stream) {
return phi::dynload::ncclAllGather(input.data(),
output.data(),
input.numel(),
phi::ToNCCLDataType(input.dtype()),
comm,
stream);
},
CommType::ALLGATHER);
}
void* GetPointerByOffset(void* raw_pointer, size_t offset, phi::DataType type) {
if (type == phi::DataType::FLOAT32) {
return reinterpret_cast<void*>(reinterpret_cast<float*>(raw_pointer) +
offset);
} else if (type == phi::DataType::FLOAT64) {
return reinterpret_cast<void*>(reinterpret_cast<double*>(raw_pointer) +
offset);
} else if (type == phi::DataType::FLOAT16) {
return reinterpret_cast<void*>(reinterpret_cast<int16_t*>(raw_pointer) +
offset);
} else if (type == phi::DataType::INT32) {
return reinterpret_cast<void*>(reinterpret_cast<int32_t*>(raw_pointer) +
offset);
} else if (type == phi::DataType::INT64) {
return reinterpret_cast<void*>(reinterpret_cast<int64_t*>(raw_pointer) +
offset);
} else if (type == phi::DataType::INT8) {
return reinterpret_cast<void*>(reinterpret_cast<int8_t*>(raw_pointer) +
offset);
} else if (type == phi::DataType::UINT8) {
return reinterpret_cast<void*>(reinterpret_cast<uint8_t*>(raw_pointer) +
offset);
} else if (type == phi::DataType::BOOL) {
return reinterpret_cast<void*>(reinterpret_cast<bool*>(raw_pointer) +
offset);
} else if (type == phi::DataType::BFLOAT16) {
return reinterpret_cast<void*>(reinterpret_cast<uint16_t*>(raw_pointer) +
offset);
} else {
PADDLE_THROW(phi::errors::Unimplemented(
"Datatype %s in NCCL is not supported.", type));
}
return nullptr;
}
std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::AllToAll(
std::vector<phi::DenseTensor>& in_tensors,
std::vector<phi::DenseTensor>& out_tensors) {
PADDLE_ENFORCE_EQ(
CheckTensorsInCudaPlace(in_tensors),
true,
phi::errors::InvalidArgument("All inputs should be in CudaPlace."));
PADDLE_ENFORCE_EQ(
CheckTensorsInCudaPlace(out_tensors),
true,
phi::errors::InvalidArgument("All inputs should be in CudaPlace."));
return Collective(
in_tensors,
out_tensors,
[&](phi::DenseTensor& input,
phi::DenseTensor& output,
ncclComm_t comm,
const gpuStream_t& stream) {
size_t offset = 0;
GroupStart();
for (auto i = 0; i < size_; i++) {
PADDLE_ENFORCE_GPU_SUCCESS(phi::dynload::ncclSend(
GetPointerByOffset(input.data(), offset, input.dtype()),
input.numel() / size_,
phi::ToNCCLDataType(input.dtype()),
i,
comm,
stream));
PADDLE_ENFORCE_GPU_SUCCESS(phi::dynload::ncclRecv(
GetPointerByOffset(output.data(), offset, input.dtype()),
input.numel() / size_,
phi::ToNCCLDataType(input.dtype()),
i,
comm,
stream));
offset += input.numel() / size_;
}
GroupEnd();
},
CommType::ALLTOALL);
}
std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Reduce(
std::vector<phi::DenseTensor>& in_tensors,
std::vector<phi::DenseTensor>& out_tensors,
const ReduceOptions& opts) {
PADDLE_ENFORCE_EQ(
CheckTensorsInCudaPlace(in_tensors),
true,
phi::errors::InvalidArgument("All inputs should be in CudaPlace."));
return Collective(
in_tensors,
out_tensors,
[&](const phi::DenseTensor& input,
phi::DenseTensor& output,
ncclComm_t comm,
const gpuStream_t& stream) {
PADDLE_ENFORCE_GPU_SUCCESS(
phi::dynload::ncclReduce(input.data(),
output.data(),
input.numel(),
phi::ToNCCLDataType(input.dtype()),
ToNCCLRedType(opts.reduce_op),
opts.root_rank,
comm,
stream));
},
CommType::REDUCE);
}
std::shared_ptr<ProcessGroup::Task> ProcessGroupNCCL::Scatter(
std::vector<phi::DenseTensor>& in_tensors,
std::vector<phi::DenseTensor>& out_tensors,
const ScatterOptions& opts) {
PADDLE_ENFORCE_EQ(
CheckTensorsInCudaPlace(in_tensors),
true,
phi::errors::InvalidArgument("All inputs should be in CudaPlace."));
PADDLE_ENFORCE_EQ(
CheckTensorsInCudaPlace(out_tensors),
true,
phi::errors::InvalidArgument("All inputs should be in CudaPlace."));
return Collective(
in_tensors,
out_tensors,
[&](phi::DenseTensor& input,
phi::DenseTensor& output,
ncclComm_t comm,
const gpuStream_t& stream) {
size_t offset = 0;
if (rank_ == opts.root_rank) {
GroupStart();
for (auto i = 0; i < size_; i++) {
PADDLE_ENFORCE_GPU_SUCCESS(phi::dynload::ncclSend(
GetPointerByOffset(input.data(), offset, input.dtype()),
input.numel() / size_,
phi::ToNCCLDataType(input.dtype()),
i,
comm,
stream));
offset += input.numel() / size_;
}
PADDLE_ENFORCE_GPU_SUCCESS(
phi::dynload::ncclRecv(output.data(),
input.numel() / size_,
phi::ToNCCLDataType(input.dtype()),
opts.root_rank,
comm,
stream));
GroupEnd();
} else {
PADDLE_ENFORCE_GPU_SUCCESS(
phi::dynload::ncclRecv(output.data(),
input.numel() / size_,
phi::ToNCCLDataType(input.dtype()),
opts.root_rank,
comm,
stream));
}
},
CommType::SCATTER);
}
std::shared_ptr<ProcessGroupNCCL> ProcessGroupNCCL::CreateProcessGroupNCCL( std::shared_ptr<ProcessGroupNCCL> ProcessGroupNCCL::CreateProcessGroupNCCL(
const std::shared_ptr<phi::distributed::Store>& store, const std::shared_ptr<phi::distributed::Store>& store,
int rank, int rank,
......
...@@ -169,42 +169,6 @@ class ProcessGroupNCCL final : public ProcessGroupWithStream { ...@@ -169,42 +169,6 @@ class ProcessGroupNCCL final : public ProcessGroupWithStream {
ncclComm_t NCCLComm(const Place& place) const; ncclComm_t NCCLComm(const Place& place) const;
// TODO(liyurui): This API will be moved later
std::shared_ptr<ProcessGroup::Task> AllReduce(
std::vector<phi::DenseTensor>& in_tensors,
std::vector<phi::DenseTensor>& out_tensors,
const AllreduceOptions& = AllreduceOptions()) override;
// TODO(sunyilun): methods below will be removed later
std::shared_ptr<ProcessGroup::Task> Broadcast(
std::vector<phi::DenseTensor>& in_tensors,
std::vector<phi::DenseTensor>& out_tensors,
const BroadcastOptions& = BroadcastOptions()) override;
std::shared_ptr<ProcessGroup::Task> Send(
std::vector<phi::DenseTensor>& tensors, int dst_rank) override;
std::shared_ptr<ProcessGroup::Task> Recv(
std::vector<phi::DenseTensor>& tensors, int src_rank) override;
std::shared_ptr<ProcessGroup::Task> AllGather(
std::vector<phi::DenseTensor>& in_tensors,
std::vector<phi::DenseTensor>& out_tensors) override;
std::shared_ptr<ProcessGroup::Task> AllToAll(
std::vector<phi::DenseTensor>& in_tensors,
std::vector<phi::DenseTensor>& out_tensors) override;
std::shared_ptr<ProcessGroup::Task> Reduce(
std::vector<phi::DenseTensor>& tensors,
std::vector<phi::DenseTensor>& out_tensors,
const ReduceOptions& opts) override;
std::shared_ptr<ProcessGroup::Task> Scatter(
std::vector<phi::DenseTensor>& in_tensors,
std::vector<phi::DenseTensor>& out_tensors,
const ScatterOptions& opts) override;
private: private:
std::shared_ptr<ProcessGroupNCCL::NCCLTask> CreateTask(const Place& place, std::shared_ptr<ProcessGroupNCCL::NCCLTask> CreateTask(const Place& place,
int rank, int rank,
...@@ -212,42 +176,32 @@ class ProcessGroupNCCL final : public ProcessGroupWithStream { ...@@ -212,42 +176,32 @@ class ProcessGroupNCCL final : public ProcessGroupWithStream {
bool sync_op, bool sync_op,
bool use_calc_stream); bool use_calc_stream);
void BroadcastUniqueNCCLID(ncclUniqueId* nccl_id); void BroadcastUniqueNCCLID(ncclUniqueId* nccl_id,
bool is_p2p_op = false,
const std::string& p2p_key = "",
int p2p_rank = 0);
void CreateNCCLEnvCache(const Place& place, const std::string& place_key); void CreateNCCLEnvCache(const Place& place,
const std::string& place_key,
CommType comm_type,
int p2p_rank = 0);
void SyncCalcStream(const Place& place); void SyncCalcStream(const Place& place, const std::string& place_key);
std::shared_ptr<ProcessGroup::Task> RunFnInNCCLEnv( std::shared_ptr<ProcessGroup::Task> Collective(
std::function<void(ncclComm_t, gpuStream_t)> fn, std::function<void(ncclComm_t, gpuStream_t)> fn,
const phi::DenseTensor& tensor, const phi::DenseTensor& tensor,
CommType comm_type, CommType comm_type,
bool sync_op, bool sync_op,
bool use_calc_stream); bool use_calc_stream);
// TODO(sunyilun): methods below will be removed later std::shared_ptr<ProcessGroup::Task> Point2Point(
std::shared_ptr<ProcessGroupNCCL::NCCLTask> CreateTask( std::function<void(ncclComm_t, gpuStream_t, int)> fn,
std::vector<Place> places, int peer,
int rank, const phi::DenseTensor& tensor,
CommType op_type, CommType comm_type,
const std::vector<phi::DenseTensor>& inputs); bool sync_op,
bool use_calc_stream);
template <typename Fn>
std::shared_ptr<ProcessGroup::Task> Collective(
std::vector<phi::DenseTensor>& inputs, // NOLINT
std::vector<phi::DenseTensor>& outputs, // NOLINT
Fn fn,
CommType op_type);
template <typename Fn>
std::shared_ptr<ProcessGroup::Task> PointToPoint(
std::vector<phi::DenseTensor>& tensors, // NOLINT
Fn fn,
int dst_rank,
CommType op_type);
void CreateNCCLManagerCache(const std::string& places_key,
const std::vector<Place>& places);
private: private:
std::shared_ptr<phi::distributed::Store> store_; std::shared_ptr<phi::distributed::Store> store_;
...@@ -260,7 +214,7 @@ class ProcessGroupNCCL final : public ProcessGroupWithStream { ...@@ -260,7 +214,7 @@ class ProcessGroupNCCL final : public ProcessGroupWithStream {
// TODO(sunyilun): attrs below will be removed later // TODO(sunyilun): attrs below will be removed later
std::mutex mutex_; std::mutex mutex_;
std::unordered_map<std::string, std::vector<phi::GPUContext*>> places_to_ctx_; static uint64_t s_group_call_counter;
}; };
} // namespace distributed } // namespace distributed
......
...@@ -13,7 +13,6 @@ ...@@ -13,7 +13,6 @@
// limitations under the License. // limitations under the License.
#pragma once #pragma once
#include "paddle/phi/core/dense_tensor.h" #include "paddle/phi/core/dense_tensor.h"
namespace paddle { namespace paddle {
...@@ -28,5 +27,29 @@ inline phi::DenseTensor GetPartialTensor(const phi::DenseTensor& tensor, ...@@ -28,5 +27,29 @@ inline phi::DenseTensor GetPartialTensor(const phi::DenseTensor& tensor,
return tensor_flattened.Slice(offset, offset + numel); return tensor_flattened.Slice(offset, offset + numel);
} }
enum class CommType : std::uint8_t {
BROADCAST = 0,
ALLREDUCE = 1,
ALLREDUCE_SPARSE = 2, // TODO(shenliang03): to support sparse in allreduce
REDUCE = 3,
ALLGATHER = 4,
GATHER = 5,
SCATTER = 6,
REDUCE_SCATTER = 7,
ALLTOALL = 8,
SEND = 9,
RECV = 10,
BARRIER = 11,
UNKNOWN = 100,
};
inline bool IsP2POP(CommType comm_type, bool is_batch_p2p = false) {
if (is_batch_p2p) {
return false;
} else {
return comm_type == CommType::SEND || comm_type == CommType::RECV;
}
}
} // namespace distributed } // namespace distributed
} // namespace paddle } // namespace paddle
...@@ -15,6 +15,7 @@ limitations under the License. */ ...@@ -15,6 +15,7 @@ limitations under the License. */
#include "paddle/fluid/operators/collective/global_gather_op.h" #include "paddle/fluid/operators/collective/global_gather_op.h"
#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) #if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL)
#include "paddle/fluid/distributed/collective/process_group_nccl.h"
#include "paddle/fluid/platform/collective_helper.h" #include "paddle/fluid/platform/collective_helper.h"
#include "paddle/fluid/platform/device/gpu/nccl_helper.h" #include "paddle/fluid/platform/device/gpu/nccl_helper.h"
#endif #endif
...@@ -221,7 +222,7 @@ struct GlobalGatherProcessGroupFunctor<phi::GPUContext, T> { ...@@ -221,7 +222,7 @@ struct GlobalGatherProcessGroupFunctor<phi::GPUContext, T> {
out->mutable_data<T>(out_dims, place); out->mutable_data<T>(out_dims, place);
for (auto i = 0; i < n_expert; ++i) { for (auto i = 0; i < n_expert; ++i) {
PADDLE_ENFORCE_GPU_SUCCESS(platform::dynload::ncclGroupStart()); distributed::ProcessGroupNCCL::GroupStart();
for (auto j = 0; j < nranks; ++j) { for (auto j = 0; j < nranks; ++j) {
int idx = i + j * n_expert; int idx = i + j * n_expert;
if (cpu_global_count_data[idx]) { if (cpu_global_count_data[idx]) {
...@@ -241,7 +242,7 @@ struct GlobalGatherProcessGroupFunctor<phi::GPUContext, T> { ...@@ -241,7 +242,7 @@ struct GlobalGatherProcessGroupFunctor<phi::GPUContext, T> {
/*sync_op*/ true); /*sync_op*/ true);
} }
} }
PADDLE_ENFORCE_GPU_SUCCESS(platform::dynload::ncclGroupEnd()); distributed::ProcessGroupNCCL::GroupEnd();
} }
#ifdef PADDLE_WITH_CUDA #ifdef PADDLE_WITH_CUDA
......
...@@ -15,6 +15,7 @@ limitations under the License. */ ...@@ -15,6 +15,7 @@ limitations under the License. */
#include "paddle/fluid/operators/collective/global_scatter_op.h" #include "paddle/fluid/operators/collective/global_scatter_op.h"
#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) #if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL)
#include "paddle/fluid/distributed/collective/process_group_nccl.h"
#include "paddle/fluid/platform/collective_helper.h" #include "paddle/fluid/platform/collective_helper.h"
#include "paddle/fluid/platform/device/gpu/nccl_helper.h" #include "paddle/fluid/platform/device/gpu/nccl_helper.h"
#endif #endif
...@@ -219,7 +220,7 @@ struct GlobalScatterProcessGroupFunctor<phi::GPUContext, T> { ...@@ -219,7 +220,7 @@ struct GlobalScatterProcessGroupFunctor<phi::GPUContext, T> {
out->mutable_data<T>(out_dims, place); out->mutable_data<T>(out_dims, place);
for (auto i = 0; i < n_expert; ++i) { for (auto i = 0; i < n_expert; ++i) {
PADDLE_ENFORCE_GPU_SUCCESS(platform::dynload::ncclGroupStart()); distributed::ProcessGroupNCCL::GroupStart();
for (auto j = 0; j < nranks; ++j) { for (auto j = 0; j < nranks; ++j) {
int idx = i + j * n_expert; int idx = i + j * n_expert;
if (cpu_local_count_data[idx]) { if (cpu_local_count_data[idx]) {
...@@ -239,7 +240,7 @@ struct GlobalScatterProcessGroupFunctor<phi::GPUContext, T> { ...@@ -239,7 +240,7 @@ struct GlobalScatterProcessGroupFunctor<phi::GPUContext, T> {
recv_ptr += cpu_global_count_data[idx]; recv_ptr += cpu_global_count_data[idx];
} }
} }
PADDLE_ENFORCE_GPU_SUCCESS(platform::dynload::ncclGroupEnd()); distributed::ProcessGroupNCCL::GroupEnd();
} }
#ifdef PADDLE_WITH_CUDA #ifdef PADDLE_WITH_CUDA
......
...@@ -267,8 +267,8 @@ void BindDistributed(py::module *m) { ...@@ -267,8 +267,8 @@ void BindDistributed(py::module *m) {
in_tensor.impl()); in_tensor.impl());
auto in_dense = *p_in_tensor; auto in_dense = *p_in_tensor;
auto *dev_ctx = self.GetDeviceContext(in_tensor.place());
auto task = self.AllGather(out_dense, in_dense, sync_op); auto task = self.AllGather(out_dense, in_dense, sync_op);
auto *dev_ctx = self.GetDeviceContext(in_tensor.place());
SplitTensor(*dev_ctx, *out_dense, &out_tensor_list); SplitTensor(*dev_ctx, *out_dense, &out_tensor_list);
task->UpdateWaitChain(*dev_ctx); task->UpdateWaitChain(*dev_ctx);
return task; return task;
...@@ -322,8 +322,6 @@ void BindDistributed(py::module *m) { ...@@ -322,8 +322,6 @@ void BindDistributed(py::module *m) {
auto in_dense = *p_in_tensor; auto in_dense = *p_in_tensor;
// in_tensor_list should not be empty // in_tensor_list should not be empty
auto *dev_ctx =
self.GetDeviceContext(in_tensor_list.back().place());
int world_size = self.GetSize(); int world_size = self.GetSize();
auto task = auto task =
self.AllToAll(out_dense, self.AllToAll(out_dense,
...@@ -331,6 +329,8 @@ void BindDistributed(py::module *m) { ...@@ -331,6 +329,8 @@ void BindDistributed(py::module *m) {
GetDefaultSplitSizes(*out_dense, world_size), GetDefaultSplitSizes(*out_dense, world_size),
GetDefaultSplitSizes(in_dense, world_size), GetDefaultSplitSizes(in_dense, world_size),
sync_op); sync_op);
auto *dev_ctx =
self.GetDeviceContext(in_tensor_list.back().place());
SplitTensor(*dev_ctx, *out_dense, &out_tensor_list); SplitTensor(*dev_ctx, *out_dense, &out_tensor_list);
task->UpdateWaitChain(*dev_ctx); task->UpdateWaitChain(*dev_ctx);
return task; return task;
...@@ -544,11 +544,11 @@ void BindDistributed(py::module *m) { ...@@ -544,11 +544,11 @@ void BindDistributed(py::module *m) {
in_tensor.impl()); in_tensor.impl());
auto in_dense = *p_in_tensor; auto in_dense = *p_in_tensor;
auto *dev_ctx =
self.GetDeviceContext(in_tensor.place(), use_calc_stream);
distributed::GatherOptions gather_opts{dst}; distributed::GatherOptions gather_opts{dst};
auto task = self.Gather( auto task = self.Gather(
out_dense, in_dense, gather_opts, sync_op, use_calc_stream); out_dense, in_dense, gather_opts, sync_op, use_calc_stream);
auto *dev_ctx =
self.GetDeviceContext(in_tensor.place(), use_calc_stream);
SplitTensor(*dev_ctx, *out_dense, &out_tensor_list); SplitTensor(*dev_ctx, *out_dense, &out_tensor_list);
if (!use_calc_stream && if (!use_calc_stream &&
dev_ctx->GetPlace() != platform::CPUPlace()) { dev_ctx->GetPlace() != platform::CPUPlace()) {
...@@ -584,8 +584,7 @@ void BindDistributed(py::module *m) { ...@@ -584,8 +584,7 @@ void BindDistributed(py::module *m) {
opts.reduce_op = op; opts.reduce_op = op;
auto dense = auto dense =
std::dynamic_pointer_cast<phi::DenseTensor>(tensor.impl()); std::dynamic_pointer_cast<phi::DenseTensor>(tensor.impl());
std::vector<phi::DenseTensor> tensors = {*dense}; return self.AllReduce(dense.get(), *dense, opts, false);
return self.AllReduce(tensors, tensors, opts);
}, },
py::arg("tensor"), py::arg("tensor"),
py::arg("op") = distributed::ReduceOp::SUM, py::arg("op") = distributed::ReduceOp::SUM,
...@@ -601,8 +600,7 @@ void BindDistributed(py::module *m) { ...@@ -601,8 +600,7 @@ void BindDistributed(py::module *m) {
opts.source_rank = source_rank; opts.source_rank = source_rank;
auto dense = auto dense =
std::dynamic_pointer_cast<phi::DenseTensor>(tensor.impl()); std::dynamic_pointer_cast<phi::DenseTensor>(tensor.impl());
std::vector<phi::DenseTensor> tensors = {*dense}; return self.Broadcast(dense.get(), *dense, opts, false);
return self.Broadcast(tensors, tensors, opts);
}, },
py::arg("tensor"), py::arg("tensor"),
py::arg("source_rank"), py::arg("source_rank"),
...@@ -616,8 +614,7 @@ void BindDistributed(py::module *m) { ...@@ -616,8 +614,7 @@ void BindDistributed(py::module *m) {
auto tensor = CastPyArg2Tensor(py_tensor.ptr(), 0); auto tensor = CastPyArg2Tensor(py_tensor.ptr(), 0);
auto dense = auto dense =
std::dynamic_pointer_cast<phi::DenseTensor>(tensor.impl()); std::dynamic_pointer_cast<phi::DenseTensor>(tensor.impl());
std::vector<phi::DenseTensor> tensors = {*dense}; return self.Send(*dense, dst, false);
return self.Send(tensors, dst);
}, },
py::arg("tensor"), py::arg("tensor"),
py::arg("dst"), py::arg("dst"),
...@@ -631,8 +628,7 @@ void BindDistributed(py::module *m) { ...@@ -631,8 +628,7 @@ void BindDistributed(py::module *m) {
auto tensor = CastPyArg2Tensor(py_tensor.ptr(), 0); auto tensor = CastPyArg2Tensor(py_tensor.ptr(), 0);
auto dense = auto dense =
std::dynamic_pointer_cast<phi::DenseTensor>(tensor.impl()); std::dynamic_pointer_cast<phi::DenseTensor>(tensor.impl());
std::vector<phi::DenseTensor> tensors = {*dense}; return self.Recv(dense.get(), src, false);
return self.Recv(tensors, src);
}, },
py::arg("tensor"), py::arg("tensor"),
py::arg("src"), py::arg("src"),
...@@ -649,9 +645,7 @@ void BindDistributed(py::module *m) { ...@@ -649,9 +645,7 @@ void BindDistributed(py::module *m) {
in_tensor.impl()); in_tensor.impl());
auto out_dense = std::dynamic_pointer_cast<phi::DenseTensor>( auto out_dense = std::dynamic_pointer_cast<phi::DenseTensor>(
out_tensor.impl()); out_tensor.impl());
std::vector<phi::DenseTensor> in_tensors = {*in_dense}; return self.AllGather(out_dense.get(), *in_dense, false);
std::vector<phi::DenseTensor> out_tensors = {*out_dense};
return self.AllGather(in_tensors, out_tensors);
}, },
py::arg("in"), py::arg("in"),
py::arg("out"), py::arg("out"),
...@@ -697,9 +691,14 @@ void BindDistributed(py::module *m) { ...@@ -697,9 +691,14 @@ void BindDistributed(py::module *m) {
in_tensor.impl()); in_tensor.impl());
auto out_dense = std::dynamic_pointer_cast<phi::DenseTensor>( auto out_dense = std::dynamic_pointer_cast<phi::DenseTensor>(
out_tensor.impl()); out_tensor.impl());
std::vector<phi::DenseTensor> in_tensors = {*in_dense};
std::vector<phi::DenseTensor> out_tensors = {*out_dense}; int world_size = self.GetSize();
return self.AllToAll(in_tensors, out_tensors); return self.AllToAll(
out_dense.get(),
*in_dense,
GetDefaultSplitSizes(*out_dense, world_size),
GetDefaultSplitSizes(*in_dense, world_size),
false);
}, },
py::arg("in"), py::arg("in"),
py::arg("out"), py::arg("out"),
...@@ -743,8 +742,7 @@ void BindDistributed(py::module *m) { ...@@ -743,8 +742,7 @@ void BindDistributed(py::module *m) {
opts.root_rank = dst; opts.root_rank = dst;
auto dense = std::dynamic_pointer_cast<phi::DenseTensor>( auto dense = std::dynamic_pointer_cast<phi::DenseTensor>(
in_tensor.impl()); in_tensor.impl());
std::vector<phi::DenseTensor> tensors = {*dense}; return self.Reduce(dense.get(), *dense, opts, false);
return self.Reduce(tensors, tensors, opts);
}, },
py::arg("tensor"), py::arg("tensor"),
py::arg("dst"), py::arg("dst"),
...@@ -765,9 +763,7 @@ void BindDistributed(py::module *m) { ...@@ -765,9 +763,7 @@ void BindDistributed(py::module *m) {
in_tensor.impl()); in_tensor.impl());
auto out_dense = std::dynamic_pointer_cast<phi::DenseTensor>( auto out_dense = std::dynamic_pointer_cast<phi::DenseTensor>(
out_tensor.impl()); out_tensor.impl());
std::vector<phi::DenseTensor> in_tensors = {*in_dense}; return self.Scatter(out_dense.get(), *in_dense, opts, false);
std::vector<phi::DenseTensor> out_tensors = {*out_dense};
return self.Scatter(in_tensors, out_tensors, opts);
}, },
py::arg("in"), py::arg("in"),
py::arg("out"), py::arg("out"),
...@@ -790,12 +786,11 @@ void BindDistributed(py::module *m) { ...@@ -790,12 +786,11 @@ void BindDistributed(py::module *m) {
auto p_in_tensor = std::dynamic_pointer_cast<phi::DenseTensor>( auto p_in_tensor = std::dynamic_pointer_cast<phi::DenseTensor>(
in_tensor.impl()); in_tensor.impl());
auto in_dense = *p_in_tensor; auto in_dense = *p_in_tensor;
auto *dev_ctx = self.GetDeviceContext(in_tensor.place(), true);
auto task = self.AllGather(out_dense, auto task = self.AllGather(out_dense,
in_dense, in_dense,
/*sync_op*/ true, /*sync_op*/ true,
/*use_calc_stream*/ true); /*use_calc_stream*/ true);
auto *dev_ctx = self.GetDeviceContext(in_tensor.place(), true);
SplitTensor(*dev_ctx, *out_dense, &out_tensor_list); SplitTensor(*dev_ctx, *out_dense, &out_tensor_list);
return task; return task;
}, },
...@@ -902,8 +897,6 @@ void BindDistributed(py::module *m) { ...@@ -902,8 +897,6 @@ void BindDistributed(py::module *m) {
auto in_dense = *p_in_tensor; auto in_dense = *p_in_tensor;
// in_tensor_list should not be empty // in_tensor_list should not be empty
auto *dev_ctx = self.GetDeviceContext(
in_tensor_list.back().place(), /*use_calc_stream*/ true);
int world_size = self.GetSize(); int world_size = self.GetSize();
auto task = auto task =
self.AllToAll(out_dense, self.AllToAll(out_dense,
...@@ -912,6 +905,8 @@ void BindDistributed(py::module *m) { ...@@ -912,6 +905,8 @@ void BindDistributed(py::module *m) {
GetDefaultSplitSizes(in_dense, world_size), GetDefaultSplitSizes(in_dense, world_size),
/*sync_op*/ true, /*sync_op*/ true,
/*use_calc_stream*/ true); /*use_calc_stream*/ true);
auto *dev_ctx = self.GetDeviceContext(
in_tensor_list.back().place(), /*use_calc_stream*/ true);
SplitTensor(*dev_ctx, *out_dense, &out_tensor_list); SplitTensor(*dev_ctx, *out_dense, &out_tensor_list);
return task; return task;
}, },
......
...@@ -146,8 +146,25 @@ void InitTensorWithNumpyValue(TensorObject* self, ...@@ -146,8 +146,25 @@ void InitTensorWithNumpyValue(TensorObject* self,
if (platform::is_cpu_place(place)) { if (platform::is_cpu_place(place)) {
SetTensorFromPyArray<platform::CPUPlace>(impl_ptr, array, place, zero_copy); SetTensorFromPyArray<platform::CPUPlace>(impl_ptr, array, place, zero_copy);
} else if (platform::is_xpu_place(place)) { } else if (platform::is_xpu_place(place)) {
#if defined(PADDLE_WITH_XPU)
phi::backends::xpu::SetXPUDeviceId(place.device);
VLOG(4) << "CurrentDeviceId: "
<< phi::backends::xpu::GetXPUCurrentDeviceId() << " from "
<< static_cast<int>(place.device);
#else
PADDLE_THROW(paddle::platform::errors::PreconditionNotMet(
"PaddlePaddle should compile with XPU if use XPUPlace."));
#endif
SetTensorFromPyArray<platform::XPUPlace>(impl_ptr, array, place, zero_copy); SetTensorFromPyArray<platform::XPUPlace>(impl_ptr, array, place, zero_copy);
} else if (platform::is_gpu_place(place)) { } else if (platform::is_gpu_place(place)) {
#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP)
phi::backends::gpu::SetDeviceId(place.device);
VLOG(4) << "CurrentDeviceId: " << phi::backends::gpu::GetCurrentDeviceId()
<< " from " << static_cast<int>(place.device);
#else
PADDLE_THROW(paddle::platform::errors::PreconditionNotMet(
"PaddlePaddle should compile with GPU if use CUDAPlace."));
#endif
SetTensorFromPyArray<platform::CUDAPlace>( SetTensorFromPyArray<platform::CUDAPlace>(
impl_ptr, array, place, zero_copy); impl_ptr, array, place, zero_copy);
} else if (platform::is_cuda_pinned_place(place)) { } else if (platform::is_cuda_pinned_place(place)) {
...@@ -156,6 +173,15 @@ void InitTensorWithNumpyValue(TensorObject* self, ...@@ -156,6 +173,15 @@ void InitTensorWithNumpyValue(TensorObject* self,
} else if (platform::is_npu_place(place)) { } else if (platform::is_npu_place(place)) {
SetTensorFromPyArray<platform::NPUPlace>(impl_ptr, array, place, zero_copy); SetTensorFromPyArray<platform::NPUPlace>(impl_ptr, array, place, zero_copy);
} else if (platform::is_custom_place(place)) { } else if (platform::is_custom_place(place)) {
#if defined(PADDLE_WITH_CUSTOM_DEVICE)
phi::DeviceManager::SetDevice(place);
VLOG(4) << "CurrentDeviceId: "
<< phi::DeviceManager::GetDevice(place.GetDeviceType()) << " from "
<< static_cast<int>(place.device);
#else
PADDLE_THROW(paddle::platform::errors::PreconditionNotMet(
"PaddlePaddle should compile with CUSTOM_DEVICE if use CustomPlace."));
#endif
SetTensorFromPyArray<platform::CustomPlace>( SetTensorFromPyArray<platform::CustomPlace>(
impl_ptr, array, place, zero_copy); impl_ptr, array, place, zero_copy);
} else { } else {
......
...@@ -236,12 +236,6 @@ def new_group(ranks=None, backend=None, timeout=_default_timeout): ...@@ -236,12 +236,6 @@ def new_group(ranks=None, backend=None, timeout=_default_timeout):
# TODO: The method below is a new method for group management, will replace the previous # TODO: The method below is a new method for group management, will replace the previous
# three in the future. # three in the future.
_add_new_group(group) _add_new_group(group)
# TODO(shenliang03): This is a temporary solution to solve the problem of
# hang caused by tcp
paddle.distributed.barrier(group=group)
if paddle.distributed.get_world_size() > 1:
paddle.distributed.barrier()
return group return group
if not backend: if not backend:
......
...@@ -164,15 +164,25 @@ class HybridCommunicateGroup: ...@@ -164,15 +164,25 @@ class HybridCommunicateGroup:
) )
) )
# create comm group for pipe parallel
self._pp_group, self._pp_comm_group = self._set_comm_group("pipe")
# NOTE(shenliang03): In pipeline parallel, we use batch_isend_irecv.
# if batch_isend_irecv is the first collective operation, all ranks of
# the pipeline group must participate in this call. In order to avoid
# this situation, we perform a collective communication in advance and
# create a communicator.
paddle.distributed.all_reduce(
paddle.zeros([1], dtype="int32"),
op=paddle.distributed.ReduceOp.SUM,
group=self._pp_comm_group,
)
# create comm group for data parallel # create comm group for data parallel
self._dp_group, self._dp_comm_group = self._set_comm_group("data") self._dp_group, self._dp_comm_group = self._set_comm_group("data")
# create comm group for model parallel # create comm group for model parallel
self._mp_group, self._mp_comm_group = self._set_comm_group("model") self._mp_group, self._mp_comm_group = self._set_comm_group("model")
# create comm group for pipe parallel
self._pp_group, self._pp_comm_group = self._set_comm_group("pipe")
# create comm group for sharding parallel # create comm group for sharding parallel
self._sharding_group, self._sharding_comm_group = self._set_comm_group( self._sharding_group, self._sharding_comm_group = self._set_comm_group(
"sharding" "sharding"
......
...@@ -1115,8 +1115,6 @@ def init_parallel_env(): ...@@ -1115,8 +1115,6 @@ def init_parallel_env():
_set_group_map_backend(group, backend) _set_group_map_backend(group, backend)
_add_new_group(group) _add_new_group(group)
parallel_helper._set_parallel_ctx(True) parallel_helper._set_parallel_ctx(True)
paddle.distributed.barrier(group=group)
return group return group
node_num = {i.split(":")[0] for i in parallel_env.trainer_endpoints} node_num = {i.split(":")[0] for i in parallel_env.trainer_endpoints}
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册